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

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

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

    https://github.com/apache/nifi/pull/733#discussion_r79914319
  
    --- Diff: 
nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/ScrollElasticsearchHttp.java
 ---
    @@ -0,0 +1,415 @@
    +/*
    + * 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 java.io.IOException;
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +import java.util.stream.Stream;
    +
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.annotation.behavior.EventDriven;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.Stateful;
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.state.Scope;
    +import org.apache.nifi.components.state.StateManager;
    +import org.apache.nifi.components.state.StateMap;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.logging.ComponentLog;
    +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.util.StandardValidators;
    +import org.apache.nifi.stream.io.ByteArrayInputStream;
    +import org.codehaus.jackson.JsonNode;
    +
    +import okhttp3.HttpUrl;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Response;
    +import okhttp3.ResponseBody;
    +
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@EventDriven
    +@SupportsBatching
    +@Tags({ "elasticsearch", "query", "scroll", "read", "get", "http" })
    +@CapabilityDescription("Scrolls through an Elasticsearch query using the 
specified connection properties. "
    +        + "This processor is intended to be run on the primary node, and 
is designed for scrolling through "
    +        + "huge result sets, as in the case of a reindex.  The state must 
be cleared before another query "
    +        + "can be run.  Each page of results is returned, wrapped in a 
JSON object like so: { \"hits\" : [ <doc1>, <doc2>, <docn> ] }.  "
    +        + "Note that the full body of each page of documents will be read 
into memory before being "
    +        + "written to a Flow File for transfer.")
    +@WritesAttributes({
    +        @WritesAttribute(attribute = "es.index", description = "The 
Elasticsearch index containing the document"),
    +        @WritesAttribute(attribute = "es.type", description = "The 
Elasticsearch document type") })
    +@Stateful(description = "After each successful scroll page, the latest 
scroll_id is persisted in scrollId as input for the next scroll call.  "
    +        + "Once the entire query is complete, finishedQuery state will be 
set to true, and the processor will not execute unless this is cleared.", 
scopes = { Scope.LOCAL })
    +public class ScrollElasticsearchHttp extends 
AbstractElasticsearchHttpProcessor {
    +
    +    private static final String FINISHED_QUERY_STATE = "finishedQuery";
    +    private static final String SCROLL_ID_STATE = "scrollId";
    +    private static final String FIELD_INCLUDE_QUERY_PARAM = 
"_source_include";
    +    private static final String QUERY_QUERY_PARAM = "q";
    +    private static final String SORT_QUERY_PARAM = "sort";
    +    private static final String SCROLL_QUERY_PARAM = "scroll";
    +    private static final String SCROLL_ID_QUERY_PARAM = "scroll_id";
    +    private static final String SIZE_QUERY_PARAM = "size";
    +
    +    public static final Relationship REL_SUCCESS = new 
Relationship.Builder()
    +            .name("success")
    +            .description(
    +                    "All FlowFiles that are read from Elasticsearch are 
routed 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. Note that only incoming "
    +                            + "flow files will be routed to 
failure.").build();
    +
    +    public static final PropertyDescriptor QUERY = new 
PropertyDescriptor.Builder()
    +            .name("scroll-es-query").displayName("Query")
    +            .description("The Lucene-style query to run against 
ElasticSearch").required(true)
    --- End diff --
    
    I'll add an example, but I feel like username:tiger should work.  Are you 
sure the username field is analyzed in your schema?  Does this query work in 
your browser against your ES instance?
    http://localhost:9200/<index>/_search?q=username:tiger


> Implement Query and Scroll processors for ElasticSearch
> -------------------------------------------------------
>
>                 Key: NIFI-2417
>                 URL: https://issues.apache.org/jira/browse/NIFI-2417
>             Project: Apache NiFi
>          Issue Type: New Feature
>          Components: Extensions
>    Affects Versions: 1.0.0
>            Reporter: Joseph Gresock
>            Assignee: Joseph Gresock
>            Priority: Minor
>             Fix For: 1.1.0
>
>
> FetchElasticsearchHttp allows users to select a single document from 
> Elasticsearch in NiFi, but there is no way to run a query to retrieve 
> multiple documents.
> We should add a QueryElasticsearchHttp processor for running a query and 
> returning a flow file per result, for small result sets.  This should allow 
> both input and non-input execution.  
> A separate ScrollElasticsearchHttp processor would also be useful for 
> scrolling through a huge result set.  This should use the state manager to 
> maintain the scroll_id value, and use this as input to the next scroll page.  
> As a result, this processor should not allow flow file input, but should 
> retrieve one page per run.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to