[ 
https://issues.apache.org/jira/browse/NIFI-4325?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16195254#comment-16195254
 ] 

ASF GitHub Bot commented on NIFI-4325:
--------------------------------------

Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2113#discussion_r143295816
  
    --- Diff: 
nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-5-processors/src/main/java/org/apache/nifi/processors/elasticsearch/JsonQueryElasticsearch5.java
 ---
    @@ -0,0 +1,459 @@
    +/*
    + * 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.processors.elasticsearch;
    +
    +import org.apache.commons.io.IOUtils;
    +import org.apache.http.HttpEntity;
    +import org.apache.http.HttpHost;
    +import org.apache.http.auth.AuthScope;
    +import org.apache.http.auth.UsernamePasswordCredentials;
    +import org.apache.http.client.CredentialsProvider;
    +import org.apache.http.client.config.RequestConfig;
    +import org.apache.http.entity.ContentType;
    +import org.apache.http.impl.client.BasicCredentialsProvider;
    +import org.apache.http.impl.nio.client.HttpAsyncClientBuilder;
    +import org.apache.http.nio.entity.NStringEntity;
    +import org.apache.nifi.annotation.behavior.EventDriven;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.components.AllowableValue;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.codehaus.jackson.map.ObjectMapper;
    +import org.elasticsearch.client.Response;
    +import org.elasticsearch.client.RestClient;
    +import org.elasticsearch.client.RestClientBuilder;
    +
    +import javax.net.ssl.KeyManagerFactory;
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.TrustManagerFactory;
    +import java.io.FileInputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.net.URL;
    +import java.security.KeyStore;
    +import java.security.SecureRandom;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +
    +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
    +@EventDriven
    +@SupportsBatching
    +@Tags({"elasticsearch", "elasticsearch 5", "fetch", "read", "get"})
    +@CapabilityDescription("Retrieves a document from Elasticsearch using the 
specified connection properties and the "
    +        + "identifier of the document to retrieve. If the cluster has been 
configured for authorization and/or secure "
    +        + "transport (SSL/TLS), and the X-Pack plugin is available, secure 
connections can be made. This processor "
    +        + "supports Elasticsearch 5.x clusters.")
    +public class JsonQueryElasticsearch5 extends AbstractProcessor {
    +
    +    private RestClient client;
    +
    +    public static final Relationship REL_SUCCESS = new 
Relationship.Builder().name("success")
    +            .description("All original flowfiles that don't cause an error 
to occur go to this relationship").build();
    +
    +    public static final Relationship REL_FAILURE = new 
Relationship.Builder().name("failure")
    +            .description("All FlowFiles that cannot be read from 
Elasticsearch are routed to this relationship").build();
    +
    +    public static final Relationship REL_HITS = new 
Relationship.Builder().name("hits")
    +            .description("Search hits are routed to this relationship.")
    +            .build();
    +
    +    public static final Relationship REL_AGGREGATIONS = new 
Relationship.Builder().name("aggregations")
    +            .description("Aggregations are routed to this relationship.")
    +            .build();
    +
    +    public static final PropertyDescriptor INDEX = new 
PropertyDescriptor.Builder()
    +            .name("el5-fetch-index")
    +            .displayName("Index")
    +            .description("The name of the index to read from")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TYPE = new 
PropertyDescriptor.Builder()
    +            .name("el5-type")
    +            .displayName("Type")
    +            .description("The type of this document (used by Elasticsearch 
for indexing and searching)")
    +            .defaultValue("")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(Validator.VALID)
    +            .build();
    +
    +    public static final PropertyDescriptor QUERY = new 
PropertyDescriptor.Builder()
    +            .name("el5-query")
    +            .displayName("Query")
    +            .description("A query in JSON syntax, not Lucene syntax.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor HTTP_HOSTS = new 
PropertyDescriptor.Builder()
    +            .name("el5-http-hosts")
    +            .displayName("HTTP Hosts")
    +            .description("A comma-separated list of HTTP hosts that host 
ElasticSearch query nodes.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +    public static final PropertyDescriptor PROP_SSL_CONTEXT_SERVICE = new 
PropertyDescriptor.Builder()
    +            .name("el5-ssl-context-service")
    +            .displayName("SSL Context Service")
    +            .description("The SSL Context Service used to provide client 
certificate information for TLS/SSL "
    +                    + "connections. This service only applies if the 
Elasticsearch endpoint(s) have been secured with TLS/SSL.")
    +            .required(false)
    +            .identifiesControllerService(SSLContextService.class)
    +            .build();
    +    public static final PropertyDescriptor USERNAME = new 
PropertyDescriptor.Builder()
    +            .name("el5-username")
    +            .displayName("Username")
    +            .description("The username to use with XPack security.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +    public static final PropertyDescriptor PASSWORD = new 
PropertyDescriptor.Builder()
    +            .name("el5-password")
    +            .displayName("Password")
    +            .description("The password to use with XPack security.")
    +            .required(false)
    +            .sensitive(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +    public static final PropertyDescriptor CONNECT_TIMEOUT = new 
PropertyDescriptor.Builder()
    +            .name("el5-connect-timeout")
    +            .displayName("Connect timeout")
    +            .description("Controls the amount of time, in milliseconds, 
before a timeout occurs when trying to connect.")
    +            .required(true)
    +            .defaultValue("5000")
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +    public static final PropertyDescriptor SOCKET_TIMEOUT = new 
PropertyDescriptor.Builder()
    +            .name("el5-socket-timeout")
    +            .displayName("Read timeout")
    +            .description("Controls the amount of time, in milliseconds, 
before a timeout occurs when waiting for a response.")
    +            .required(true)
    +            .defaultValue("60000")
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +    public static final PropertyDescriptor RETRY_TIMEOUT = new 
PropertyDescriptor.Builder()
    +            .name("el5-retry-timeout")
    +            .displayName("Retry timeout")
    +            .description("Controls the amount of time, in milliseconds, 
before a timeout occurs when retrying the operation.")
    +            .required(true)
    +            .defaultValue("60000")
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final AllowableValue BREAK_UP_YES = new AllowableValue(
    +        "breakup-yes",
    +        "Yes",
    +        "Break up results."
    +    );
    +    public static final AllowableValue BREAK_UP_HITS_NO = new 
AllowableValue(
    +        "breakup-no",
    +        "No",
    +        "Don't break up results."
    +    );
    +
    +    public static final PropertyDescriptor BREAK_UP_HITS = new 
PropertyDescriptor.Builder()
    +            .name("el5-break-up-hits")
    +            .displayName("Break up search results")
    +            .description("Break up search results into one flowfile per 
result.")
    +            .allowableValues(BREAK_UP_HITS_NO, BREAK_UP_YES)
    +            .defaultValue(BREAK_UP_HITS_NO.getValue())
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .build();
    +    public static final PropertyDescriptor BREAK_UP_AGGREGATIONS = new 
PropertyDescriptor.Builder()
    +            .name("el5-break-up-aggregations")
    +            .displayName("Break up aggregation results")
    +            .description("Break up aggregation results into one flowfile 
per result.")
    +            .allowableValues(BREAK_UP_HITS_NO, BREAK_UP_YES)
    +            .defaultValue(BREAK_UP_HITS_NO.getValue())
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .build();
    +
    +    private static final Set<Relationship> relationships;
    +    private static final List<PropertyDescriptor> propertyDescriptors;
    +
    +    static {
    +        final Set<Relationship> _rels = new HashSet<>();
    +        _rels.add(REL_SUCCESS);
    +        _rels.add(REL_FAILURE);
    +        _rels.add(REL_HITS);
    +        _rels.add(REL_AGGREGATIONS);
    +        relationships = Collections.unmodifiableSet(_rels);
    +
    +        final List<PropertyDescriptor> descriptors = new ArrayList<>();
    +        descriptors.add(PROP_SSL_CONTEXT_SERVICE);
    +        descriptors.add(CONNECT_TIMEOUT);
    +        descriptors.add(SOCKET_TIMEOUT);
    +        descriptors.add(RETRY_TIMEOUT);
    +        descriptors.add(USERNAME);
    +        descriptors.add(PASSWORD);
    +        descriptors.add(HTTP_HOSTS);
    +        descriptors.add(QUERY);
    +        descriptors.add(INDEX);
    +        descriptors.add(TYPE);
    +        descriptors.add(BREAK_UP_HITS);
    +        descriptors.add(BREAK_UP_AGGREGATIONS);
    +
    +        propertyDescriptors = Collections.unmodifiableList(descriptors);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> 
getSupportedPropertyDescriptors() {
    +        return propertyDescriptors;
    +    }
    +
    +
    +    @OnUnscheduled
    +    public void onStartup() {
    --- End diff --
    
    Just a style note, but this might be a misnamed/confusing method for an 
@OnUnscheduled annotation


> Create a new ElasticSearch processor that supports the JSON DSL
> ---------------------------------------------------------------
>
>                 Key: NIFI-4325
>                 URL: https://issues.apache.org/jira/browse/NIFI-4325
>             Project: Apache NiFi
>          Issue Type: Improvement
>            Reporter: Mike Thomsen
>            Priority: Minor
>
> The existing ElasticSearch processors use the Lucene-style syntax for 
> querying, not the JSON DSL. A new processor is needed that can take a full 
> JSON query and execute it. It should also support aggregation queries in this 
> syntax. A user needs to be able to take a query as-is from Kibana and drop it 
> into NiFi and have it just run.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to