From issues-return-64610-archive-asf-public=cust-asf.ponee.io@nifi.apache.org Sat Sep 15 02:47:04 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 4BD7518067A for ; Sat, 15 Sep 2018 02:47:03 +0200 (CEST) Received: (qmail 70020 invoked by uid 500); 15 Sep 2018 00:47:02 -0000 Mailing-List: contact issues-help@nifi.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@nifi.apache.org Delivered-To: mailing list issues@nifi.apache.org Received: (qmail 70011 invoked by uid 99); 15 Sep 2018 00:47:02 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 15 Sep 2018 00:47:02 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 0B36DC9142 for ; Sat, 15 Sep 2018 00:47:02 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -110.301 X-Spam-Level: X-Spam-Status: No, score=-110.301 tagged_above=-999 required=6.31 tests=[ENV_AND_HDR_SPF_MATCH=-0.5, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, USER_IN_DEF_SPF_WL=-7.5, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id iXp9MlyWzVJZ for ; Sat, 15 Sep 2018 00:47:01 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id D2BF05F3EE for ; Sat, 15 Sep 2018 00:47:00 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 67828E008D for ; Sat, 15 Sep 2018 00:47:00 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 2AB7123FA0 for ; Sat, 15 Sep 2018 00:47:00 +0000 (UTC) Date: Sat, 15 Sep 2018 00:47:00 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@nifi.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (NIFI-5051) Create a LookupService that uses ElasticSearch MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/NIFI-5051?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16615541#comment-16615541 ] ASF GitHub Bot commented on NIFI-5051: -------------------------------------- Github user mattyb149 commented on a diff in the pull request: https://github.com/apache/nifi/pull/2615#discussion_r217870054 --- Diff: nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchLookupService.java --- @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.elasticsearch; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.nifi.annotation.lifecycle.OnEnabled; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.expression.ExpressionLanguageScope; +import org.apache.nifi.lookup.LookupFailureException; +import org.apache.nifi.lookup.LookupService; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.serialization.JsonInferenceSchemaRegistryService; +import org.apache.nifi.serialization.record.MapRecord; +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordSchema; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +public class ElasticSearchLookupService extends JsonInferenceSchemaRegistryService implements LookupService { + public static final PropertyDescriptor CLIENT_SERVICE = new PropertyDescriptor.Builder() + .name("el-rest-client-service") + .displayName("Client Service") + .description("An ElasticSearch client service to use for running queries.") + .identifiesControllerService(ElasticSearchClientService.class) + .required(true) + .build(); + public static final PropertyDescriptor INDEX = new PropertyDescriptor.Builder() + .name("el-lookup-index") + .displayName("Index") + .description("The name of the index to read from") + .required(true) + .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder() + .name("el-lookup-type") + .displayName("Type") + .description("The type of this document (used by Elasticsearch for indexing and searching)") + .required(false) + .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + private ElasticSearchClientService clientService; + + private String index; + private String type; + private ObjectMapper mapper; + + private final List DESCRIPTORS; + + public ElasticSearchLookupService() { + List _desc = new ArrayList<>(); + _desc.addAll(super.getSupportedPropertyDescriptors()); + _desc.add(CLIENT_SERVICE); + _desc.add(INDEX); + _desc.add(TYPE); + DESCRIPTORS = Collections.unmodifiableList(_desc); + } + + @Override + @OnEnabled + public void onEnabled(final ConfigurationContext context) { + clientService = context.getProperty(CLIENT_SERVICE).asControllerService(ElasticSearchClientService.class); + index = context.getProperty(INDEX).evaluateAttributeExpressions().getValue(); + type = context.getProperty(TYPE).evaluateAttributeExpressions().getValue(); + mapper = new ObjectMapper(); + + super.onEnabled(context); + } + + @Override + protected List getSupportedPropertyDescriptors() { + return DESCRIPTORS; + } + + @Override + public Optional lookup(Map coordinates) throws LookupFailureException { + Map context = coordinates.entrySet().stream() + .collect(Collectors.toMap( + e -> e.getKey(), + e -> e.getValue().toString() + )); + return lookup(coordinates, context); + } + + @Override + public Optional lookup(Map coordinates, Map context) throws LookupFailureException { + validateCoordinates(coordinates); + + try { + Record record; + if (coordinates.containsKey("_id")) { + record = getById((String)coordinates.get("_id"), context); + } else { + record = getByQuery(coordinates, context); + } + + return record == null ? Optional.empty() : Optional.of(record); + } catch (Exception ex) { + getLogger().error("Error during lookup.", ex); + throw new LookupFailureException(ex); + } + } + + private RecordSchema getSchemaFromCoordinates(Map coordinates) { + Map variables = coordinates.entrySet().stream() + .collect(Collectors.toMap( + e -> e.getKey(), + e -> e.getValue().toString() + )); + try { + return getSchema(variables, null); + } catch (SchemaNotFoundException | IOException e) { + if (getLogger().isDebugEnabled()) { + getLogger().debug("Could not load schema, will create one from the results.", e); + } + return null; + } + } + + private void validateCoordinates(Map coordinates) throws LookupFailureException { + List reasons = new ArrayList<>(); + + if (coordinates.containsKey("_id") && !(coordinates.get("_id") instanceof String)) { + reasons.add("_id was supplied, but it was not a String."); + } + + if (coordinates.containsKey("_id") && coordinates.size() > 1) { + reasons.add("When _id is used, it can be the only key used in the lookup."); + } + + if (reasons.size() > 0) { + String error = String.join("\n", reasons); + throw new LookupFailureException(error); + } + } + + private Record getById(final String _id, Map context) throws IOException, LookupFailureException, SchemaNotFoundException { + Map query = new HashMap(){{ + put("query", new HashMap() {{ + put("match", new HashMap(){{ + put("_id", _id); + }}); + }}); + }}; + + String json = mapper.writeValueAsString(query); + + SearchResponse response = clientService.search(json, index, type); + + if (response.getNumberOfHits() > 1) { + throw new LookupFailureException(String.format("Expected 1 response, got %d for query %s", + response.getNumberOfHits(), json)); + } else if (response.getNumberOfHits() == 0) { + return null; + } + + final Map source = (Map)response.getHits().get(0).get("_source"); + + RecordSchema toUse = getSchema(context, source, null); + + return new MapRecord(toUse, source); + } + + Map getNested(String key, Object value) { + String path = key.substring(0, key.lastIndexOf(".")); + + return new HashMap(){{ + put("path", path); + put("query", new HashMap(){{ + put("match", new HashMap(){{ + put(key, value); + }}); + }}); + }}; + } + + private Map buildQuery(Map coordinates) { + Map query = new HashMap(){{ + put("bool", new HashMap(){{ + put("must", coordinates.entrySet().stream() + .map(e -> new HashMap(){{ + if (e.getKey().contains(".")) { --- End diff -- I've run the unit and integration tests and the code LGTM, but I'd feel better if I could get an example going where I do the lookup on a field that's not at the top level. I have a document containing a "user" field, which contains other fields such as "name", and "name" contains other fields like "first" and "last". I tried using this with a simple CSV input containing an id and a first name, and tried to use the lookup service to match "user.name.first" and return the value of "user.name.last", but got an error saying I was trying to do a nested query on a field that wasn't nested. I didn't add an explicit mapping for the index, just put the complex JSON docs into ES. Am I configuring it wrong, or is this not supported, or could there be a bug? > Create a LookupService that uses ElasticSearch > ---------------------------------------------- > > Key: NIFI-5051 > URL: https://issues.apache.org/jira/browse/NIFI-5051 > Project: Apache NiFi > Issue Type: New Feature > Reporter: Mike Thomsen > Assignee: Mike Thomsen > Priority: Major > -- This message was sent by Atlassian JIRA (v7.6.3#76005)