NIFI-11985: Add ConsumeElasticsearch processor

Signed-off-by: Joe Gresock <jgresock@gmail.com>
This closes #7671.
This commit is contained in:
Chris Sampson 2023-08-28 14:25:40 +01:00 committed by Joe Gresock
parent 2cad5c5ad6
commit c091347795
No known key found for this signature in database
GPG Key ID: 37F5B9B6E258C8B7
58 changed files with 2663 additions and 670 deletions

View File

@ -48,7 +48,8 @@ Execute the following script from the `nifi-elasticsearch-bundle` directory:
```bash ```bash
mvn --fail-at-end -Pcontrib-check clean install mvn --fail-at-end -Pcontrib-check clean install
es_versions=(elasticsearch6 elasticsearch7 elasticsearch8) # blank entry to run the default integration-tests profile, i.e. Elasticsearch 8
es_versions=(elasticsearch6 elasticsearch7 " ")
it_modules=(nifi-elasticsearch-client-service nifi-elasticsearch-restapi-processors) it_modules=(nifi-elasticsearch-client-service nifi-elasticsearch-restapi-processors)
for v in "${es_versions[@]}"; do for v in "${es_versions[@]}"; do
for m in "${it_modules[@]}"; do for m in "${it_modules[@]}"; do

View File

@ -145,7 +145,7 @@ class ElasticSearchClientService_IT extends AbstractElasticsearch_IT {
@Test @Test
void testVerifyFailedURL() { void testVerifyFailedURL() {
runner.disableControllerService(service); runner.disableControllerService(service);
runner.setProperty(service, ElasticSearchClientService.HTTP_HOSTS, "invalid"); runner.setProperty(service, ElasticSearchClientService.HTTP_HOSTS, "blah://invalid");
final List<ConfigVerificationResult> results = service.verify( final List<ConfigVerificationResult> results = service.verify(
new MockConfigurationContext(service, getClientServiceProperties(), runner.getProcessContext().getControllerServiceLookup(), null), new MockConfigurationContext(service, getClientServiceProperties(), runner.getProcessContext().getControllerServiceLookup(), null),

View File

@ -51,7 +51,7 @@ public abstract class AbstractByQueryElasticsearch extends AbstractProcessor imp
.build(); .build();
private static final Set<Relationship> relationships; private static final Set<Relationship> relationships;
private static final List<PropertyDescriptor> propertyDescriptors; static final List<PropertyDescriptor> byQueryPropertyDescriptors;
private final AtomicReference<ElasticSearchClientService> clientService = new AtomicReference<>(null); private final AtomicReference<ElasticSearchClientService> clientService = new AtomicReference<>(null);
@ -63,13 +63,16 @@ public abstract class AbstractByQueryElasticsearch extends AbstractProcessor imp
relationships = Collections.unmodifiableSet(rels); relationships = Collections.unmodifiableSet(rels);
final List<PropertyDescriptor> descriptors = new ArrayList<>(); final List<PropertyDescriptor> descriptors = new ArrayList<>();
descriptors.add(QUERY_DEFINITION_STYLE);
descriptors.add(QUERY); descriptors.add(QUERY);
descriptors.add(QUERY_CLAUSE);
descriptors.add(SCRIPT);
descriptors.add(QUERY_ATTRIBUTE); descriptors.add(QUERY_ATTRIBUTE);
descriptors.add(INDEX); descriptors.add(INDEX);
descriptors.add(TYPE); descriptors.add(TYPE);
descriptors.add(CLIENT_SERVICE); descriptors.add(CLIENT_SERVICE);
propertyDescriptors = Collections.unmodifiableList(descriptors); byQueryPropertyDescriptors = Collections.unmodifiableList(descriptors);
} }
abstract String getTookAttribute(); abstract String getTookAttribute();
@ -86,8 +89,8 @@ public abstract class AbstractByQueryElasticsearch extends AbstractProcessor imp
} }
@Override @Override
public final List<PropertyDescriptor> getSupportedPropertyDescriptors() { public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return propertyDescriptors; return byQueryPropertyDescriptors;
} }
@Override @Override

View File

@ -16,7 +16,6 @@
*/ */
package org.apache.nifi.processors.elasticsearch; package org.apache.nifi.processors.elasticsearch;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyDescriptor;
@ -112,7 +111,7 @@ public abstract class AbstractJsonQueryElasticsearch<Q extends JsonQueryParamete
.build(); .build();
private static final Set<Relationship> relationships; private static final Set<Relationship> relationships;
private static final List<PropertyDescriptor> propertyDescriptors; static final List<PropertyDescriptor> queryPropertyDescriptors;
ResultOutputStrategy hitStrategy; ResultOutputStrategy hitStrategy;
private SearchResultsFormat hitFormat; private SearchResultsFormat hitFormat;
@ -120,8 +119,6 @@ public abstract class AbstractJsonQueryElasticsearch<Q extends JsonQueryParamete
private AggregationResultsFormat aggregationFormat; private AggregationResultsFormat aggregationFormat;
private boolean outputNoHits; private boolean outputNoHits;
final ObjectMapper mapper = new ObjectMapper();
final AtomicReference<ElasticSearchClientService> clientService = new AtomicReference<>(null); final AtomicReference<ElasticSearchClientService> clientService = new AtomicReference<>(null);
static { static {
@ -133,7 +130,14 @@ public abstract class AbstractJsonQueryElasticsearch<Q extends JsonQueryParamete
relationships = Collections.unmodifiableSet(rels); relationships = Collections.unmodifiableSet(rels);
final List<PropertyDescriptor> descriptors = new ArrayList<>(); final List<PropertyDescriptor> descriptors = new ArrayList<>();
descriptors.add(QUERY_DEFINITION_STYLE);
descriptors.add(QUERY); descriptors.add(QUERY);
descriptors.add(QUERY_CLAUSE);
descriptors.add(SIZE);
descriptors.add(SORT);
descriptors.add(AGGREGATIONS);
descriptors.add(FIELDS);
descriptors.add(SCRIPT_FIELDS);
descriptors.add(QUERY_ATTRIBUTE); descriptors.add(QUERY_ATTRIBUTE);
descriptors.add(INDEX); descriptors.add(INDEX);
descriptors.add(TYPE); descriptors.add(TYPE);
@ -144,7 +148,7 @@ public abstract class AbstractJsonQueryElasticsearch<Q extends JsonQueryParamete
descriptors.add(AGGREGATION_RESULTS_FORMAT); descriptors.add(AGGREGATION_RESULTS_FORMAT);
descriptors.add(OUTPUT_NO_HITS); descriptors.add(OUTPUT_NO_HITS);
propertyDescriptors = Collections.unmodifiableList(descriptors); queryPropertyDescriptors = Collections.unmodifiableList(descriptors);
} }
@Override @Override
@ -154,7 +158,7 @@ public abstract class AbstractJsonQueryElasticsearch<Q extends JsonQueryParamete
@Override @Override
public List<PropertyDescriptor> getSupportedPropertyDescriptors() { public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return propertyDescriptors; return queryPropertyDescriptors;
} }
@Override @Override
@ -183,8 +187,8 @@ public abstract class AbstractJsonQueryElasticsearch<Q extends JsonQueryParamete
hitStrategy = ResultOutputStrategy.fromValue(context.getProperty(SEARCH_RESULTS_SPLIT).getValue()); hitStrategy = ResultOutputStrategy.fromValue(context.getProperty(SEARCH_RESULTS_SPLIT).getValue());
hitFormat = SearchResultsFormat.valueOf(context.getProperty(SEARCH_RESULTS_FORMAT).getValue()); hitFormat = SearchResultsFormat.valueOf(context.getProperty(SEARCH_RESULTS_FORMAT).getValue());
aggregationStrategy = ResultOutputStrategy.fromValue(context.getProperty(AGGREGATION_RESULTS_SPLIT).getValue()); aggregationStrategy = context.getProperty(AGGREGATION_RESULTS_SPLIT).isSet() ? ResultOutputStrategy.fromValue(context.getProperty(AGGREGATION_RESULTS_SPLIT).getValue()) : null;
aggregationFormat = AggregationResultsFormat.valueOf(context.getProperty(AGGREGATION_RESULTS_FORMAT).getValue()); aggregationFormat = context.getProperty(AGGREGATION_RESULTS_FORMAT).isSet() ? AggregationResultsFormat.valueOf(context.getProperty(AGGREGATION_RESULTS_FORMAT).getValue()) : null;
outputNoHits = context.getProperty(OUTPUT_NO_HITS).asBoolean(); outputNoHits = context.getProperty(OUTPUT_NO_HITS).asBoolean();
} }

View File

@ -42,6 +42,7 @@ import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
public abstract class AbstractPaginatedJsonQueryElasticsearch extends AbstractJsonQueryElasticsearch<PaginatedJsonQueryParameters> { public abstract class AbstractPaginatedJsonQueryElasticsearch extends AbstractJsonQueryElasticsearch<PaginatedJsonQueryParameters> {
public static final PropertyDescriptor SEARCH_RESULTS_SPLIT = new PropertyDescriptor.Builder() public static final PropertyDescriptor SEARCH_RESULTS_SPLIT = new PropertyDescriptor.Builder()
@ -77,18 +78,12 @@ public abstract class AbstractPaginatedJsonQueryElasticsearch extends AbstractJs
static final List<PropertyDescriptor> paginatedPropertyDescriptors; static final List<PropertyDescriptor> paginatedPropertyDescriptors;
static { static {
final List<PropertyDescriptor> descriptors = new ArrayList<>(); final List<PropertyDescriptor> descriptors = new ArrayList<>(
descriptors.add(QUERY_ATTRIBUTE); // replace SEARCH_RESULTS_SPLIT property to allow additional output strategies
descriptors.add(INDEX); queryPropertyDescriptors.stream().map(pd -> AbstractJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT.equals(pd) ? SEARCH_RESULTS_SPLIT : pd).collect(Collectors.toList())
descriptors.add(TYPE); );
descriptors.add(CLIENT_SERVICE);
descriptors.add(SEARCH_RESULTS_SPLIT);
descriptors.add(SEARCH_RESULTS_FORMAT);
descriptors.add(AGGREGATION_RESULTS_SPLIT);
descriptors.add(AGGREGATION_RESULTS_FORMAT);
descriptors.add(PAGINATION_TYPE); descriptors.add(PAGINATION_TYPE);
descriptors.add(PAGINATION_KEEP_ALIVE); descriptors.add(PAGINATION_KEEP_ALIVE);
descriptors.add(OUTPUT_NO_HITS);
paginatedPropertyDescriptors = Collections.unmodifiableList(descriptors); paginatedPropertyDescriptors = Collections.unmodifiableList(descriptors);
} }
@ -118,10 +113,13 @@ public abstract class AbstractPaginatedJsonQueryElasticsearch extends AbstractJs
// execute query/scroll // execute query/scroll
final String queryJson = updateQueryJson(newQuery, paginatedJsonQueryParameters); final String queryJson = updateQueryJson(newQuery, paginatedJsonQueryParameters);
final Map<String, String> requestParameters = getDynamicProperties(context, input);
if (!newQuery && paginationType == PaginationType.SCROLL) { if (!newQuery && paginationType == PaginationType.SCROLL) {
if (!requestParameters.isEmpty()) {
getLogger().warn("Elasticsearch _scroll API does not accept query parameters, ignoring dynamic properties {}", requestParameters.keySet());
}
response = clientService.get().scroll(queryJson); response = clientService.get().scroll(queryJson);
} else { } else {
final Map<String, String> requestParameters = getDynamicProperties(context, input);
if (paginationType == PaginationType.SCROLL) { if (paginationType == PaginationType.SCROLL) {
requestParameters.put("scroll", paginatedJsonQueryParameters.getKeepAlive()); requestParameters.put("scroll", paginatedJsonQueryParameters.getKeepAlive());
} }
@ -145,8 +143,7 @@ public abstract class AbstractPaginatedJsonQueryElasticsearch extends AbstractJs
); );
} }
// mark the paginated query for expiry if there are no hits (no more pages to obtain so stop looping on this query) updateQueryParameters(paginatedJsonQueryParameters, response);
updatePageExpirationTimestamp(paginatedJsonQueryParameters, !response.getHits().isEmpty());
hitsFlowFiles = handleResponse(response, newQuery, paginatedJsonQueryParameters, hitsFlowFiles, session, input, stopWatch); hitsFlowFiles = handleResponse(response, newQuery, paginatedJsonQueryParameters, hitsFlowFiles, session, input, stopWatch);
} while (!response.getHits().isEmpty() && (input != null || hitStrategy == ResultOutputStrategy.PER_QUERY)); } while (!response.getHits().isEmpty() && (input != null || hitStrategy == ResultOutputStrategy.PER_QUERY));
@ -268,11 +265,9 @@ public abstract class AbstractPaginatedJsonQueryElasticsearch extends AbstractJs
List<FlowFile> handleHits(final List<Map<String, Object>> hits, final boolean newQuery, final PaginatedJsonQueryParameters paginatedJsonQueryParameters, List<FlowFile> handleHits(final List<Map<String, Object>> hits, final boolean newQuery, final PaginatedJsonQueryParameters paginatedJsonQueryParameters,
final ProcessSession session, final FlowFile parent, final Map<String, String> attributes, final ProcessSession session, final FlowFile parent, final Map<String, String> attributes,
final List<FlowFile> hitsFlowFiles, final String transitUri, final StopWatch stopWatch) throws IOException { final List<FlowFile> hitsFlowFiles, final String transitUri, final StopWatch stopWatch) throws IOException {
paginatedJsonQueryParameters.incrementPageCount();
attributes.put("page.number", Integer.toString(paginatedJsonQueryParameters.getPageCount())); attributes.put("page.number", Integer.toString(paginatedJsonQueryParameters.getPageCount()));
if (hitStrategy == ResultOutputStrategy.PER_QUERY) { if (hitStrategy == ResultOutputStrategy.PER_QUERY) {
final List<Map<String, Object>> formattedHits = formatHits(hits); final List<Map<String, Object>> formattedHits = formatHits(hits);
combineHits(formattedHits, paginatedJsonQueryParameters, session, parent, attributes, hitsFlowFiles, newQuery); combineHits(formattedHits, paginatedJsonQueryParameters, session, parent, attributes, hitsFlowFiles, newQuery);
@ -289,8 +284,11 @@ public abstract class AbstractPaginatedJsonQueryElasticsearch extends AbstractJs
return hitsFlowFiles; return hitsFlowFiles;
} }
private void updatePageExpirationTimestamp(final PaginatedJsonQueryParameters paginatedJsonQueryParameters, final boolean hasHits) { void updateQueryParameters(final PaginatedJsonQueryParameters paginatedJsonQueryParameters, final SearchResponse response) {
final String keepAliveDuration = "PT" + (hasHits ? paginatedJsonQueryParameters.getKeepAlive() : "0s"); paginatedJsonQueryParameters.incrementPageCount();
// mark the paginated query for expiry if there are no hits (no more pages to obtain so stop looping on this query)
final String keepAliveDuration = "PT" + (!response.getHits().isEmpty() ? paginatedJsonQueryParameters.getKeepAlive() : "0s");
paginatedJsonQueryParameters.setPageExpirationTimestamp( paginatedJsonQueryParameters.setPageExpirationTimestamp(
String.valueOf(Instant.now().plus(Duration.parse(keepAliveDuration)).toEpochMilli()) String.valueOf(Instant.now().plus(Duration.parse(keepAliveDuration)).toEpochMilli())
); );

View File

@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.SerializationFeature; import com.fasterxml.jackson.databind.SerializationFeature;
import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue; import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationContext;
@ -30,6 +31,7 @@ import org.apache.nifi.elasticsearch.IndexOperationRequest;
import org.apache.nifi.elasticsearch.IndexOperationResponse; import org.apache.nifi.elasticsearch.IndexOperationResponse;
import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSession;
@ -108,8 +110,6 @@ public abstract class AbstractPutElasticsearch extends AbstractProcessor impleme
static final String BULK_HEADER_PREFIX = "BULK:"; static final String BULK_HEADER_PREFIX = "BULK:";
static final ObjectMapper MAPPER = new ObjectMapper();
boolean logErrors; boolean logErrors;
boolean outputErrorResponses; boolean outputErrorResponses;
boolean notFoundIsSuccessful; boolean notFoundIsSuccessful;
@ -201,6 +201,12 @@ public abstract class AbstractPutElasticsearch extends AbstractProcessor impleme
return validationResults; return validationResults;
} }
@Override
public List<ConfigVerificationResult> verifyAfterIndex(final ProcessContext context, final ComponentLog verificationLogger, final Map<String, String> attributes,
final ElasticSearchClientService verifyClientService, final String index, final boolean indexExists) {
return Collections.emptyList();
}
Map<String, String> getRequestURLParameters(final Map<String, String> dynamicProperties) { Map<String, String> getRequestURLParameters(final Map<String, String> dynamicProperties) {
return dynamicProperties.entrySet().stream().filter(e -> !e.getKey().startsWith(BULK_HEADER_PREFIX)) return dynamicProperties.entrySet().stream().filter(e -> !e.getKey().startsWith(BULK_HEADER_PREFIX))
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));

View File

@ -0,0 +1,321 @@
/*
* 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 com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.JsonNode;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
import org.apache.nifi.annotation.behavior.Stateful;
import org.apache.nifi.annotation.behavior.SystemResource;
import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
import org.apache.nifi.annotation.behavior.TriggerSerially;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.configuration.DefaultSchedule;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.components.state.StateMap;
import org.apache.nifi.elasticsearch.SearchResponse;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.util.JsonValidator;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.elasticsearch.api.PaginatedJsonQueryParameters;
import org.apache.nifi.util.StringUtils;
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.stream.Collectors;
@WritesAttributes({
@WritesAttribute(attribute = "mime.type", description = "application/json"),
@WritesAttribute(attribute = "page.number", description = "The number of the page (request), starting from 1, in which the results were returned that are in the output flowfile"),
@WritesAttribute(attribute = "hit.count", description = "The number of hits that are in the output flowfile"),
@WritesAttribute(attribute = "elasticsearch.query.error", description = "The error message provided by Elasticsearch if there is an error querying the index.")
})
@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
@TriggerSerially
@PrimaryNodeOnly
@DefaultSchedule(period="1 min")
@Tags({"elasticsearch", "elasticsearch5", "elasticsearch6", "elasticsearch7", "elasticsearch8", "query", "scroll", "page", "search", "json"})
@CapabilityDescription("A processor that repeatedly runs a paginated query against a field using a Range query to consume new Documents from an Elasticsearch index/query. " +
"The processor will retrieve multiple pages of results until either no more results are available or the Pagination Keep Alive expiration is reached, " +
"after which the Range query will automatically update the field constraint based on the last retrieved Document value.")
@SeeAlso({SearchElasticsearch.class, PaginatedJsonQueryElasticsearch.class})
@DynamicProperty(
name = "The name of a URL query parameter to add",
value = "The value of the URL query parameter",
expressionLanguageScope = ExpressionLanguageScope.ENVIRONMENT,
description = "Adds the specified property name/value as a query parameter in the Elasticsearch URL used for processing. " +
"These parameters will override any matching parameters in the query request body. " +
"For SCROLL type queries, these parameters are only used in the initial (first page) query as the " +
"Elasticsearch Scroll API does not support the same query parameters for subsequent pages of data.")
@Stateful(scopes = Scope.CLUSTER, description = "The pagination state (scrollId, searchAfter, pitId, hitCount, pageCount, pageExpirationTimestamp, trackingRangeValue) " +
"is retained in between invocations of this processor until the Scroll/PiT has expired " +
"(when the current time is later than the last query execution plus the Pagination Keep Alive interval).")
@SystemResourceConsideration(resource = SystemResource.MEMORY, description = "Care should be taken on the size of each page because each response " +
"from Elasticsearch will be loaded into memory all at once and converted into the resulting flowfiles.")
public class ConsumeElasticsearch extends SearchElasticsearch {
static final String STATE_RANGE_VALUE = "trackingRangeValue";
public static final PropertyDescriptor RANGE_FIELD = new PropertyDescriptor.Builder()
.name("es-rest-range-field")
.displayName("Range Query Field")
.description("Field to be tracked as part of an Elasticsearch Range query using a \"gt\" bound match. " +
"This field must exist within the Elasticsearch document for it to be retrieved.")
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.required(true)
.build();
public static final PropertyDescriptor RANGE_FIELD_SORT_ORDER = new PropertyDescriptor.Builder()
.name("es-rest-sort-order")
.displayName("Sort Order")
.description("The order in which to sort the \"" + RANGE_FIELD.getDisplayName() + "\". " +
"A \"sort\" clause for the \"" + RANGE_FIELD.getDisplayName() +
"\" field will be prepended to any provided \"" + SORT.getDisplayName() + "\" clauses. " +
"If a \"sort\" clause already exists for the \"" + RANGE_FIELD.getDisplayName() +
"\" field, it will not be updated.")
.allowableValues("asc", "desc")
.defaultValue("asc")
.required(true)
.build();
public static final PropertyDescriptor RANGE_INITIAL_VALUE = new PropertyDescriptor.Builder()
.name("es-rest-range-initial-value")
.displayName("Initial Value")
.description("The initial value to use for the query if the processor has not run previously. " +
"If the processor has run previously and stored a value in its state, this property will be ignored. " +
"If no value is provided, and the processor has not previously run, no Range query bounds will be used, " +
"i.e. all documents will be retrieved in the specified \"" + RANGE_FIELD_SORT_ORDER.getDisplayName() + "\".")
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.required(false)
.build();
public static final PropertyDescriptor RANGE_DATE_FORMAT = new PropertyDescriptor.Builder()
.name("es-rest-range-format")
.displayName(RANGE_INITIAL_VALUE.getDisplayName() + " Date Format")
.description("If the \"" + RANGE_FIELD.getDisplayName() + "\" is a Date field, convert the \"" + RANGE_INITIAL_VALUE.getDisplayName() + "\" to a date with this format. " +
"If not specified, Elasticsearch will use the date format provided by the \"" + RANGE_FIELD.getDisplayName() + "\"'s mapping. " +
"For valid syntax, see https://www.elastic.co/guide/en/elasticsearch/reference/current/mapping-date-format.html")
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.dependsOn(RANGE_INITIAL_VALUE)
.required(false)
.build();
public static final PropertyDescriptor RANGE_TIME_ZONE = new PropertyDescriptor.Builder()
.name("es-rest-range-time-zone")
.displayName(RANGE_INITIAL_VALUE.getDisplayName() + " Date Time Zone")
.description("If the \"" + RANGE_FIELD.getDisplayName() + "\" is a Date field, convert the \"" + RANGE_INITIAL_VALUE.getDisplayName() + "\" to UTC with this time zone. " +
"Valid values are ISO 8601 UTC offsets, such as \"+01:00\" or \"-08:00\", and IANA time zone IDs, such as \"Europe/London\".")
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.dependsOn(RANGE_INITIAL_VALUE)
.required(false)
.build();
public static final PropertyDescriptor ADDITIONAL_FILTERS = new PropertyDescriptor.Builder()
.name("es-rest-additional-filters")
.displayName("Additional Filters")
.description("One or more query filters in JSON syntax, not Lucene syntax. " +
"Ex: [{\"match\":{\"somefield\":\"somevalue\"}}, {\"match\":{\"anotherfield\":\"anothervalue\"}}]. " +
"These filters wil be used as part of a Bool query's filter.")
.addValidator(JsonValidator.INSTANCE)
.required(false)
.build();
private static final List<PropertyDescriptor> propertyDescriptors;
static {
final List<PropertyDescriptor> descriptors = new ArrayList<>();
descriptors.add(RANGE_FIELD);
descriptors.add(RANGE_FIELD_SORT_ORDER);
descriptors.add(RANGE_INITIAL_VALUE);
descriptors.add(RANGE_DATE_FORMAT);
descriptors.add(RANGE_TIME_ZONE);
descriptors.add(ADDITIONAL_FILTERS);
descriptors.addAll(scrollPropertyDescriptors.stream()
.filter(pd -> !QUERY.equals(pd) && !QUERY_CLAUSE.equals(pd) && !QUERY_DEFINITION_STYLE.equals(pd))
.collect(Collectors.toList()));
propertyDescriptors = Collections.unmodifiableList(descriptors);
}
protected String trackingRangeField;
protected String trackingSortOrder;
@Override
public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return propertyDescriptors;
}
@Override
Scope getStateScope() {
return Scope.CLUSTER;
}
@Override
@OnScheduled
public void onScheduled(final ProcessContext context) {
super.onScheduled(context);
// set tracking field information, so it can be used to update the StateMap after query execution (where ProcessContext is not available)
trackingRangeField = context.getProperty(RANGE_FIELD).getValue();
trackingSortOrder = context.getProperty(RANGE_FIELD_SORT_ORDER).getValue();
}
@Override
@OnStopped
public void onStopped() {
super.onStopped();
// reset tracking fields, so that we don't retain incorrect values between processor restarts
trackingRangeField = null;
trackingSortOrder = null;
}
private String getTrackingRangeField(final ProcessContext context) {
final String field;
if (trackingRangeField != null) {
field = trackingRangeField;
} else if (context != null) {
field = context.getProperty(RANGE_FIELD).getValue();
} else {
field = null;
}
return field;
}
private String getTrackingSortOrder(final ProcessContext context) {
final String sortOrder;
if (trackingSortOrder != null) {
sortOrder = trackingSortOrder;
} else if (context != null) {
sortOrder = context.getProperty(RANGE_FIELD_SORT_ORDER).getValue();
} else {
sortOrder = null;
}
return sortOrder;
}
@Override
PaginatedJsonQueryParameters buildJsonQueryParameters(final FlowFile input, final ProcessContext context, final ProcessSession session) throws IOException {
final PaginatedJsonQueryParameters paginatedQueryJsonParameters = super.buildJsonQueryParameters(input, context, session);
paginatedQueryJsonParameters.setTrackingRangeValue(getTrackingRangeValueOrDefault(context));
return paginatedQueryJsonParameters;
}
@Override
public void addQueryClause(final Map<String, Object> query, final Map<String, String> attributes, final ProcessContext context) throws IOException {
final List<Map<String, Object>> filters = new ArrayList<>(10);
// only retrieve documents with values greater than the last queried value (if present)
final String trackingRangeValue = getTrackingRangeValueOrDefault(context);
if (StringUtils.isNotBlank(trackingRangeValue)) {
filters.add(Collections.singletonMap("range", Collections.singletonMap(getTrackingRangeField(context),
new HashMap<String, String>(3, 1) {{
put("gt", trackingRangeValue);
if (context.getProperty(RANGE_DATE_FORMAT).isSet()) {
put("format", context.getProperty(RANGE_DATE_FORMAT).getValue());
}
if (context.getProperty(RANGE_TIME_ZONE).isSet()) {
put("time_zone", context.getProperty(RANGE_TIME_ZONE).getValue());
}
}})));
}
// add any additional filters specified as a property, allowing for one (Object) or multiple (Array of Objects) filters
if (context.getProperty(ADDITIONAL_FILTERS).isSet()) {
final JsonNode additionalFilters = mapper.readTree(context.getProperty(ADDITIONAL_FILTERS).getValue());
if (additionalFilters.isArray()) {
filters.addAll(mapper.convertValue(additionalFilters, new TypeReference<List<Map<String, Object>>>() {}));
} else {
filters.add(mapper.convertValue(additionalFilters, new TypeReference<Map<String, Object>>() {}));
}
}
if (!filters.isEmpty()) {
final Map<String, Object> bool = Collections.singletonMap("bool", Collections.singletonMap("filter", filters));
query.put("query", bool);
}
}
@Override
@SuppressWarnings("unchecked")
public void addSortClause(final Map<String, Object> query, final Map<String, String> attributes, final ProcessContext context) throws IOException {
super.addSortClause(query, attributes, context);
final List<Map<String, Object>> sort;
if (query.containsKey("sort")) {
sort = (List<Map<String, Object>>) query.get("sort");
} else {
sort = new ArrayList<>(1);
query.put("sort", sort);
}
if (sort.stream().noneMatch(s -> s.containsKey(getTrackingRangeField(context)))) {
sort.add(0, Collections.singletonMap(getTrackingRangeField(context), getTrackingSortOrder(context)));
}
}
@Override
void additionalState(final Map<String, String> newStateMap, final PaginatedJsonQueryParameters paginatedJsonQueryParameters) {
newStateMap.put(STATE_RANGE_VALUE, paginatedJsonQueryParameters.getTrackingRangeValue());
}
@Override
void updateQueryParameters(final PaginatedJsonQueryParameters paginatedJsonQueryParameters, final SearchResponse response) {
super.updateQueryParameters(paginatedJsonQueryParameters, response);
// update the tracking range value with first/last hit (depending upon sort order)
if (!response.getHits().isEmpty()) {
final int trackingHitIndex;
if ("desc".equals(getTrackingSortOrder(null)) && paginatedJsonQueryParameters.getPageCount() == 1) {
trackingHitIndex = 0;
} else if ("asc".equals(getTrackingSortOrder(null))) {
trackingHitIndex = response.getHits().size() - 1;
} else {
return;
}
@SuppressWarnings("unchecked")
final String nextValue = String.valueOf(((Map<String, Object>) response.getHits().get(trackingHitIndex).get("_source"))
.get(getTrackingRangeField(null)));
if (StringUtils.isNotBlank(nextValue)) {
paginatedJsonQueryParameters.setTrackingRangeValue(nextValue);
}
}
}
private String getTrackingRangeValueOrDefault(final ProcessContext context) throws IOException {
final StateMap stateMap = context.getStateManager().getState(getStateScope());
return stateMap == null || stateMap.get(STATE_RANGE_VALUE) == null
? context.getProperty(RANGE_INITIAL_VALUE).getValue()
: stateMap.get(STATE_RANGE_VALUE);
}
}

View File

@ -23,11 +23,16 @@ import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.elasticsearch.ElasticSearchClientService; import org.apache.nifi.elasticsearch.ElasticSearchClientService;
import org.apache.nifi.elasticsearch.OperationResponse; import org.apache.nifi.elasticsearch.OperationResponse;
import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.expression.ExpressionLanguageScope;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.stream.Collectors;
@WritesAttributes({ @WritesAttributes({
@WritesAttribute(attribute = "elasticsearch.delete.took", description = "The amount of time that it took to complete the delete operation in ms."), @WritesAttribute(attribute = "elasticsearch.delete.took", description = "The amount of time that it took to complete the delete operation in ms."),
@ -47,6 +52,21 @@ public class DeleteByQueryElasticsearch extends AbstractByQueryElasticsearch {
static final String TOOK_ATTRIBUTE = "elasticsearch.delete.took"; static final String TOOK_ATTRIBUTE = "elasticsearch.delete.took";
static final String ERROR_ATTRIBUTE = "elasticsearch.delete.error"; static final String ERROR_ATTRIBUTE = "elasticsearch.delete.error";
private static final List<PropertyDescriptor> propertyDescriptors;
static {
final List<PropertyDescriptor> descriptors = new ArrayList<>(
byQueryPropertyDescriptors.stream().filter(pd -> !SCRIPT.equals(pd)).collect(Collectors.toList())
);
propertyDescriptors = Collections.unmodifiableList(descriptors);
}
@Override
public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return propertyDescriptors;
}
@Override @Override
String getTookAttribute() { String getTookAttribute() {
return TOOK_ATTRIBUTE; return TOOK_ATTRIBUTE;

View File

@ -17,32 +17,43 @@
package org.apache.nifi.processors.elasticsearch; package org.apache.nifi.processors.elasticsearch;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ObjectNode;
import org.apache.nifi.components.ConfigVerificationResult; import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.Validator; import org.apache.nifi.components.Validator;
import org.apache.nifi.elasticsearch.ElasticSearchClientService; import org.apache.nifi.elasticsearch.ElasticSearchClientService;
import org.apache.nifi.elasticsearch.ElasticsearchException;
import org.apache.nifi.elasticsearch.SearchResponse;
import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.VerifiableProcessor; import org.apache.nifi.processor.VerifiableProcessor;
import org.apache.nifi.processor.util.JsonValidator; import org.apache.nifi.processor.util.JsonValidator;
import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.elasticsearch.api.QueryDefinitionType;
import org.apache.nifi.util.StringUtils; import org.apache.nifi.util.StringUtils;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
public interface ElasticsearchRestProcessor extends VerifiableProcessor { public interface ElasticsearchRestProcessor extends Processor, VerifiableProcessor {
String ATTR_RECORD_COUNT = "record.count"; String ATTR_RECORD_COUNT = "record.count";
String VERIFICATION_STEP_INDEX_EXISTS = "Elasticsearch Index Exists"; String VERIFICATION_STEP_INDEX_EXISTS = "Elasticsearch Index Exists";
String VERIFICATION_STEP_QUERY_JSON_VALID = "Elasticsearch Query JSON Valid";
String VERIFICATION_STEP_QUERY_VALID = "Elasticsearch Query Valid";
PropertyDescriptor INDEX = new PropertyDescriptor.Builder() PropertyDescriptor INDEX = new PropertyDescriptor.Builder()
.name("el-rest-fetch-index") .name("el-rest-fetch-index")
@ -62,11 +73,101 @@ public interface ElasticsearchRestProcessor extends VerifiableProcessor {
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build(); .build();
PropertyDescriptor QUERY_DEFINITION_STYLE = new PropertyDescriptor.Builder()
.name("el-rest-query-definition-style")
.displayName("Query Definition Style")
.description("How the JSON Query will be defined for use by the processor.")
.required(true)
.allowableValues(QueryDefinitionType.class)
.defaultValue(QueryDefinitionType.FULL_QUERY.getValue())
.build();
PropertyDescriptor QUERY = new PropertyDescriptor.Builder() PropertyDescriptor QUERY = new PropertyDescriptor.Builder()
.name("el-rest-query") .name("el-rest-query")
.displayName("Query") .displayName("Query")
.description("A query in JSON syntax, not Lucene syntax. Ex: {\"query\":{\"match\":{\"somefield\":\"somevalue\"}}}. " + .description("A query in JSON syntax, not Lucene syntax. Ex: {\"query\":{\"match\":{\"somefield\":\"somevalue\"}}}. " +
"If this parameter is not set, the query will be read from the flowfile content.") "If this parameter is not set, the query will be read from the flowfile content. " +
"If the query (property and flowfile content) is empty, a default empty JSON Object will be used, " +
"which will result in a \"match_all\" query in Elasticsearch.")
.dependsOn(QUERY_DEFINITION_STYLE, QueryDefinitionType.FULL_QUERY)
.required(false)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(JsonValidator.INSTANCE)
.build();
PropertyDescriptor QUERY_CLAUSE = new PropertyDescriptor.Builder()
.name("el-rest-query-clause")
.displayName("Query Clause")
.description("A \"query\" clause in JSON syntax, not Lucene syntax. Ex: {\"match\":{\"somefield\":\"somevalue\"}}. " +
"If the query is empty, a default JSON Object will be used, which will result in a \"match_all\" query in Elasticsearch.")
.dependsOn(QUERY_DEFINITION_STYLE, QueryDefinitionType.BUILD_QUERY)
.required(false)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(JsonValidator.INSTANCE)
.build();
PropertyDescriptor SCRIPT = new PropertyDescriptor.Builder()
.name("el-rest-script")
.displayName("Script")
.description("A \"script\" to execute during the operation, in JSON syntax. " +
"Ex: {\"source\": \"ctx._source.count++\", \"lang\": \"painless\"}")
.dependsOn(QUERY_DEFINITION_STYLE, QueryDefinitionType.BUILD_QUERY)
.required(false)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(JsonValidator.INSTANCE)
.build();
PropertyDescriptor SIZE = new PropertyDescriptor.Builder()
.name("es-rest-size")
.displayName("Size")
.description("The maximum number of documents to retrieve in the query. If the query is paginated, " +
"this \"size\" applies to each page of the query, not the \"size\" of the entire result set.")
.dependsOn(QUERY_DEFINITION_STYLE, QueryDefinitionType.BUILD_QUERY)
.required(false)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.build();
PropertyDescriptor AGGREGATIONS = new PropertyDescriptor.Builder()
.name("es-rest-query-aggs")
.displayName("Aggregations")
.description("One or more query aggregations (or \"aggs\"), in JSON syntax. " +
"Ex: {\"items\": {\"terms\": {\"field\": \"product\", \"size\": 10}}}")
.dependsOn(QUERY_DEFINITION_STYLE, QueryDefinitionType.BUILD_QUERY)
.required(false)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(JsonValidator.INSTANCE)
.build();
PropertyDescriptor SORT = new PropertyDescriptor.Builder()
.name("es-rest-query-sort")
.displayName("Sort")
.description("Sort results by one or more fields, in JSON syntax. " +
"Ex: [{\"price\" : {\"order\" : \"asc\", \"mode\" : \"avg\"}}, {\"post_date\" : {\"format\": \"strict_date_optional_time_nanos\"}}]")
.dependsOn(QUERY_DEFINITION_STYLE, QueryDefinitionType.BUILD_QUERY)
.required(false)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(JsonValidator.INSTANCE)
.build();
PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
.name("es-rest-query-fields")
.displayName("Fields")
.description("Fields of indexed documents to be retrieved, in JSON syntax. " +
"Ex: [\"user.id\", \"http.response.*\", {\"field\": \"@timestamp\", \"format\": \"epoch_millis\"}]")
.dependsOn(QUERY_DEFINITION_STYLE, QueryDefinitionType.BUILD_QUERY)
.required(false)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(JsonValidator.INSTANCE)
.build();
PropertyDescriptor SCRIPT_FIELDS = new PropertyDescriptor.Builder()
.name("es-rest-query-script-fields")
.displayName("Script Fields")
.description("Fields to created using script evaluation at query runtime, in JSON syntax. " +
"Ex: {\"test1\": {\"script\": {\"lang\": \"painless\", \"source\": \"doc['price'].value * 2\"}}, " +
"\"test2\": {\"script\": {\"lang\": \"painless\", \"source\": \"doc['price'].value * params.factor\", \"params\": {\"factor\": 2.0}}}}")
.dependsOn(QUERY_DEFINITION_STYLE, QueryDefinitionType.BUILD_QUERY)
.required(false) .required(false)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(JsonValidator.INSTANCE) .addValidator(JsonValidator.INSTANCE)
@ -109,19 +210,92 @@ public interface ElasticsearchRestProcessor extends VerifiableProcessor {
.description("All flowfiles that fail due to server/cluster availability go to this relationship.") .description("All flowfiles that fail due to server/cluster availability go to this relationship.")
.build(); .build();
default String getQuery(final FlowFile input, final ProcessContext context, final ProcessSession session) throws IOException { String DEFAULT_QUERY_JSON = "{}";
String retVal = null;
if (context.getProperty(QUERY).isSet()) {
retVal = context.getProperty(QUERY).evaluateAttributeExpressions(input).getValue();
} else if (input != null) {
final ByteArrayOutputStream out = new ByteArrayOutputStream();
session.exportTo(input, out);
out.close();
retVal = out.toString(); ObjectMapper mapper = new ObjectMapper();
default String getQuery(final FlowFile input, final ProcessContext context, final ProcessSession session) throws IOException {
String retVal = getQuery(input != null ? input.getAttributes() : Collections.emptyMap(), context);
if (DEFAULT_QUERY_JSON.equals(retVal) && input != null
&& QueryDefinitionType.FULL_QUERY.getValue().equals(context.getProperty(QUERY_DEFINITION_STYLE).getValue())
&& !context.getProperty(QUERY).isSet()) {
try (final ByteArrayOutputStream out = new ByteArrayOutputStream()) {
session.exportTo(input, out);
retVal = out.toString();
}
} }
return retVal; return StringUtils.isNotBlank(retVal) ? retVal : DEFAULT_QUERY_JSON;
}
default String getQuery(final Map<String, String> attributes, final ProcessContext context) throws IOException {
final String retVal;
if (QueryDefinitionType.FULL_QUERY.getValue().equals(context.getProperty(QUERY_DEFINITION_STYLE).getValue())) {
if (context.getProperty(QUERY).isSet()) {
retVal = context.getProperty(QUERY).evaluateAttributeExpressions(attributes).getValue();
} else {
retVal = null;
}
} else {
final Map<String, Object> query = new HashMap<>(7, 1);
addQueryClause(query, attributes, context);
if (context.getProperty(SIZE).isSet()) {
query.put("size", context.getProperty(SIZE).evaluateAttributeExpressions(attributes).asInteger());
}
addSortClause(query, attributes, context);
if (context.getProperty(AGGREGATIONS).isSet()) {
query.put("aggs", mapper.readTree(context.getProperty(AGGREGATIONS).evaluateAttributeExpressions(attributes).getValue()));
}
if (context.getProperty(SCRIPT).isSet()) {
query.put("script", mapper.readTree(context.getProperty(SCRIPT).evaluateAttributeExpressions(attributes).getValue()));
}
if (context.getProperty(FIELDS).isSet()) {
query.put("fields", mapper.readTree(context.getProperty(FIELDS).evaluateAttributeExpressions(attributes).getValue()));
}
if (context.getProperty(SCRIPT_FIELDS).isSet()) {
query.put("script_fields", mapper.readTree(context.getProperty(SCRIPT_FIELDS).evaluateAttributeExpressions(attributes).getValue()));
}
retVal = mapper.writeValueAsString(query);
}
// allow for no query to be specified, which will run a "match_all" query in Elasticsearch by default
return StringUtils.isNotBlank(retVal) ? retVal : DEFAULT_QUERY_JSON;
}
/**
* Add "query" clause to the Elasticsearch query object.
* Overridable method for processors that build a query clause from separate components, e.g. ConsumeElasticsearch
*
* @param query the Query object being constructed
* @param attributes (optional) input FlowFile attributes
* @param context ProcessContext of the running processor
*/
default void addQueryClause(final Map<String, Object> query, final Map<String, String> attributes, final ProcessContext context) throws IOException {
if (context.getProperty(QUERY_CLAUSE).isSet()) {
query.put("query", mapper.readTree(context.getProperty(QUERY_CLAUSE).evaluateAttributeExpressions(attributes).getValue()));
}
}
/**
* Add "sort" clause to the Elasticsearch query object.
* Overridable method for processors that build a sort clause from separate components, e.g. ConsumeElasticsearch
*
* @param query the Query object being constructed
* @param attributes (optional) input FlowFile attributes
* @param context ProcessContext of the running processor
*/
default void addSortClause(final Map<String, Object> query, final Map<String, String> attributes, final ProcessContext context) throws IOException {
if (context.getProperty(SORT).isSet()) {
// ensure sort is specified as a List for easier manipulation if needed later
final List<Map<String, Object>> sortList;
final JsonNode sort = mapper.readTree(context.getProperty(SORT).evaluateAttributeExpressions(attributes).getValue());
if (sort.isArray()) {
sortList = mapper.convertValue(sort, new TypeReference<List<Map<String, Object>>>() {});
} else {
sortList = Collections.singletonList(mapper.convertValue(sort, new TypeReference<Map<String, Object>>() {}));
}
query.put("sort", new ArrayList<>(sortList));
}
} }
default Map<String, String> getDynamicProperties(final ProcessContext context, final FlowFile flowFile) { default Map<String, String> getDynamicProperties(final ProcessContext context, final FlowFile flowFile) {
@ -189,9 +363,51 @@ public interface ElasticsearchRestProcessor extends VerifiableProcessor {
boolean isIndexNotExistSuccessful(); boolean isIndexNotExistSuccessful();
@SuppressWarnings("unused")
default List<ConfigVerificationResult> verifyAfterIndex(final ProcessContext context, final ComponentLog verificationLogger, final Map<String, String> attributes, default List<ConfigVerificationResult> verifyAfterIndex(final ProcessContext context, final ComponentLog verificationLogger, final Map<String, String> attributes,
final ElasticSearchClientService verifyClientService, final String index, final boolean indexExists) { final ElasticSearchClientService verifyClientService, final String index, final boolean indexExists) {
return Collections.emptyList(); final List<ConfigVerificationResult> results = new ArrayList<>();
final ConfigVerificationResult.Builder queryJsonValidResult = new ConfigVerificationResult.Builder()
.verificationStepName(VERIFICATION_STEP_QUERY_JSON_VALID);
final ConfigVerificationResult.Builder queryValidResult = new ConfigVerificationResult.Builder()
.verificationStepName(VERIFICATION_STEP_QUERY_VALID);
if (indexExists) {
try {
final String query = getQuery(attributes, context);
verificationLogger.debug("Query JSON: {}", query);
final ObjectNode queryJson = mapper.readValue(query, ObjectNode.class);
queryJsonValidResult.outcome(ConfigVerificationResult.Outcome.SUCCESSFUL).explanation("Query JSON successfully parsed");
if (queryJson.has("script")) {
verificationLogger.debug("Removing \"script\" field from verification Query, not valid for _search");
queryJson.remove("script");
}
final String type = context.getProperty(TYPE).evaluateAttributeExpressions(attributes).getValue();
final Map<String, String> requestParameters = new HashMap<>(getDynamicProperties(context, attributes));
requestParameters.putIfAbsent("_source", "false");
final SearchResponse response = verifyClientService.search(mapper.writeValueAsString(queryJson), index, type, requestParameters);
queryValidResult.outcome(ConfigVerificationResult.Outcome.SUCCESSFUL)
.explanation(String.format("Query found %d hits and %d aggregations in %d milliseconds, timed out: %s",
response.getNumberOfHits(), response.getAggregations() == null ? 0 : response.getAggregations().size(), response.getTook(), response.isTimedOut()));
} catch (final IOException ioe) {
verificationLogger.warn("Unable to parse Query as JSON", ioe);
queryJsonValidResult.outcome(ConfigVerificationResult.Outcome.FAILED)
.explanation(String.format("Query cannot be parsed as valid JSON: %s", ioe.getMessage()));
queryValidResult.outcome(ConfigVerificationResult.Outcome.SKIPPED).explanation("Query JSON could not be parsed");
} catch (final ElasticsearchException ee) {
verificationLogger.warn("Query failed in Elasticsearch", ee);
queryValidResult.outcome(ConfigVerificationResult.Outcome.FAILED)
.explanation(String.format("Query failed in Elasticsearch: %s", ee.getMessage()));
}
} else {
final String skippedReason = String.format("Index %s does not exist", index);
queryJsonValidResult.outcome(ConfigVerificationResult.Outcome.SKIPPED).explanation(skippedReason);
queryValidResult.outcome(ConfigVerificationResult.Outcome.SKIPPED).explanation(skippedReason);
}
results.add(queryJsonValidResult.build());
results.add(queryValidResult.build());
return results;
} }
} }

View File

@ -17,12 +17,12 @@
package org.apache.nifi.processors.elasticsearch; package org.apache.nifi.processors.elasticsearch;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.InputRequirement; import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.annotation.lifecycle.OnStopped;
@ -65,6 +65,7 @@ import java.util.concurrent.atomic.AtomicReference;
@WritesAttribute(attribute = "elasticsearch.type", description = "The Elasticsearch document type"), @WritesAttribute(attribute = "elasticsearch.type", description = "The Elasticsearch document type"),
@WritesAttribute(attribute = "elasticsearch.get.error", description = "The error message provided by Elasticsearch if there is an error fetching the document.") @WritesAttribute(attribute = "elasticsearch.get.error", description = "The error message provided by Elasticsearch if there is an error fetching the document.")
}) })
@SeeAlso(JsonQueryElasticsearch.class)
@DynamicProperty( @DynamicProperty(
name = "The name of a URL query parameter to add", name = "The name of a URL query parameter to add",
value = "The value of the URL query parameter", value = "The value of the URL query parameter",
@ -129,8 +130,6 @@ public class GetElasticsearch extends AbstractProcessor implements Elasticsearch
REL_DOC, REL_FAILURE, REL_RETRY, REL_NOT_FOUND REL_DOC, REL_FAILURE, REL_RETRY, REL_NOT_FOUND
))); )));
private final ObjectMapper mapper = new ObjectMapper();
private final AtomicReference<ElasticSearchClientService> clientService = new AtomicReference<>(null); private final AtomicReference<ElasticSearchClientService> clientService = new AtomicReference<>(null);
@Override @Override

View File

@ -21,6 +21,7 @@ import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.elasticsearch.SearchResponse; import org.apache.nifi.elasticsearch.SearchResponse;
import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.expression.ExpressionLanguageScope;
@ -47,6 +48,7 @@ import java.util.concurrent.TimeUnit;
"Elasticsearch JSON DSL. It does not automatically paginate queries for the user. If an incoming relationship is added to this " + "Elasticsearch JSON DSL. It does not automatically paginate queries for the user. If an incoming relationship is added to this " +
"processor, it will use the flowfile's content for the query. Care should be taken on the size of the query because the entire response " + "processor, it will use the flowfile's content for the query. Care should be taken on the size of the query because the entire response " +
"from Elasticsearch will be loaded into memory all at once and converted into the resulting flowfiles.") "from Elasticsearch will be loaded into memory all at once and converted into the resulting flowfiles.")
@SeeAlso(PaginatedJsonQueryElasticsearch.class)
@DynamicProperty( @DynamicProperty(
name = "The name of a URL query parameter to add", name = "The name of a URL query parameter to add",
value = "The value of the URL query parameter", value = "The value of the URL query parameter",

View File

@ -23,6 +23,7 @@ import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.elasticsearch.SearchResponse; import org.apache.nifi.elasticsearch.SearchResponse;
@ -32,15 +33,13 @@ import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processors.elasticsearch.api.PaginatedJsonQueryParameters; import org.apache.nifi.processors.elasticsearch.api.PaginatedJsonQueryParameters;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List; import java.util.List;
@WritesAttributes({ @WritesAttributes({
@WritesAttribute(attribute = "mime.type", description = "application/json"), @WritesAttribute(attribute = "mime.type", description = "application/json"),
@WritesAttribute(attribute = "aggregation.name", description = "The name of the aggregation whose results are in the output flowfile"), @WritesAttribute(attribute = "aggregation.name", description = "The name of the aggregation whose results are in the output flowfile"),
@WritesAttribute(attribute = "aggregation.number", description = "The number of the aggregation whose results are in the output flowfile"), @WritesAttribute(attribute = "aggregation.number", description = "The number of the aggregation whose results are in the output flowfile"),
@WritesAttribute(attribute = "page.number", description = "The number of the page (request) in which the results were returned that are in the output flowfile"), @WritesAttribute(attribute = "page.number", description = "The number of the page (request), starting from 1, in which the results were returned that are in the output flowfile"),
@WritesAttribute(attribute = "hit.count", description = "The number of hits that are in the output flowfile"), @WritesAttribute(attribute = "hit.count", description = "The number of hits that are in the output flowfile"),
@WritesAttribute(attribute = "elasticsearch.query.error", description = "The error message provided by Elasticsearch if there is an error querying the index.") @WritesAttribute(attribute = "elasticsearch.query.error", description = "The error message provided by Elasticsearch if there is an error querying the index.")
}) })
@ -49,6 +48,7 @@ import java.util.List;
@CapabilityDescription("A processor that allows the user to run a paginated query (with aggregations) written with the Elasticsearch JSON DSL. " + @CapabilityDescription("A processor that allows the user to run a paginated query (with aggregations) written with the Elasticsearch JSON DSL. " +
"It will use the flowfile's content for the query unless the QUERY attribute is populated. " + "It will use the flowfile's content for the query unless the QUERY attribute is populated. " +
"Search After/Point in Time queries must include a valid \"sort\" field.") "Search After/Point in Time queries must include a valid \"sort\" field.")
@SeeAlso({JsonQueryElasticsearch.class, ConsumeElasticsearch.class, SearchElasticsearch.class})
@DynamicProperty( @DynamicProperty(
name = "The name of a URL query parameter to add", name = "The name of a URL query parameter to add",
value = "The value of the URL query parameter", value = "The value of the URL query parameter",
@ -60,19 +60,9 @@ import java.util.List;
@SystemResourceConsideration(resource = SystemResource.MEMORY, description = "Care should be taken on the size of each page because each response " + @SystemResourceConsideration(resource = SystemResource.MEMORY, description = "Care should be taken on the size of each page because each response " +
"from Elasticsearch will be loaded into memory all at once and converted into the resulting flowfiles.") "from Elasticsearch will be loaded into memory all at once and converted into the resulting flowfiles.")
public class PaginatedJsonQueryElasticsearch extends AbstractPaginatedJsonQueryElasticsearch { public class PaginatedJsonQueryElasticsearch extends AbstractPaginatedJsonQueryElasticsearch {
private static final List<PropertyDescriptor> propertyDescriptors;
static {
final List<PropertyDescriptor> descriptors = new ArrayList<>();
descriptors.add(QUERY);
descriptors.addAll(paginatedPropertyDescriptors);
propertyDescriptors = Collections.unmodifiableList(descriptors);
}
@Override @Override
public final List<PropertyDescriptor> getSupportedPropertyDescriptors() { public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return propertyDescriptors; return paginatedPropertyDescriptors;
} }
@Override @Override

View File

@ -18,7 +18,6 @@
package org.apache.nifi.processors.elasticsearch; package org.apache.nifi.processors.elasticsearch;
import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
import org.apache.nifi.annotation.behavior.DynamicProperties; import org.apache.nifi.annotation.behavior.DynamicProperties;
import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.DynamicProperty;
@ -28,6 +27,7 @@ import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyDescriptor;
@ -64,6 +64,7 @@ import java.util.stream.Collectors;
description = "The error message if there is an issue parsing the FlowFile, sending the parsed document to Elasticsearch or parsing the Elasticsearch response"), description = "The error message if there is an issue parsing the FlowFile, sending the parsed document to Elasticsearch or parsing the Elasticsearch response"),
@WritesAttribute(attribute = "elasticsearch.bulk.error", description = "The _bulk response if there was an error during processing the document within Elasticsearch.") @WritesAttribute(attribute = "elasticsearch.bulk.error", description = "The _bulk response if there was an error during processing the document within Elasticsearch.")
}) })
@SeeAlso(PutElasticsearchRecord.class)
@DynamicProperties({ @DynamicProperties({
@DynamicProperty( @DynamicProperty(
name = "The name of the Bulk request header", name = "The name of the Bulk request header",
@ -179,7 +180,6 @@ public class PutElasticsearchJson extends AbstractPutElasticsearch {
))); )));
private boolean outputErrors; private boolean outputErrors;
private final ObjectMapper objectMapper = new ObjectMapper();
@Override @Override
Set<Relationship> getBaseRelationships() { Set<Relationship> getBaseRelationships() {
@ -261,7 +261,7 @@ public class PutElasticsearchJson extends AbstractPutElasticsearch {
try (final InputStream inStream = session.read(input)) { try (final InputStream inStream = session.read(input)) {
final byte[] result = IOUtils.toByteArray(inStream); final byte[] result = IOUtils.toByteArray(inStream);
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
final Map<String, Object> contentMap = objectMapper.readValue(new String(result, charset), Map.class); final Map<String, Object> contentMap = mapper.readValue(new String(result, charset), Map.class);
final IndexOperationRequest.Operation o = IndexOperationRequest.Operation.forValue(indexOp); final IndexOperationRequest.Operation o = IndexOperationRequest.Operation.forValue(indexOp);
operations.add(new IndexOperationRequest(index, type, id, contentMap, o, scriptMap, scriptedUpsert, dynamicTemplatesMap, bulkHeaderFields)); operations.add(new IndexOperationRequest(index, type, id, contentMap, o, scriptMap, scriptedUpsert, dynamicTemplatesMap, bulkHeaderFields));
@ -284,7 +284,7 @@ public class PutElasticsearchJson extends AbstractPutElasticsearch {
private Map<String, Object> getMapFromAttribute(final PropertyDescriptor propertyDescriptor, final ProcessContext context, final FlowFile input) { private Map<String, Object> getMapFromAttribute(final PropertyDescriptor propertyDescriptor, final ProcessContext context, final FlowFile input) {
final String dynamicTemplates = context.getProperty(propertyDescriptor).evaluateAttributeExpressions(input).getValue(); final String dynamicTemplates = context.getProperty(propertyDescriptor).evaluateAttributeExpressions(input).getValue();
try { try {
return StringUtils.isNotBlank(dynamicTemplates) ? MAPPER.readValue(dynamicTemplates, Map.class) : Collections.emptyMap(); return StringUtils.isNotBlank(dynamicTemplates) ? mapper.readValue(dynamicTemplates, Map.class) : Collections.emptyMap();
} catch (final JsonProcessingException jpe) { } catch (final JsonProcessingException jpe) {
throw new ProcessException(propertyDescriptor.getDisplayName() + " must be a String parsable into a JSON Object", jpe); throw new ProcessException(propertyDescriptor.getDisplayName() + " must be a String parsable into a JSON Object", jpe);
} }
@ -300,7 +300,7 @@ public class PutElasticsearchJson extends AbstractPutElasticsearch {
errors.forEach((index, error) -> { errors.forEach((index, error) -> {
String errorMessage; String errorMessage;
try { try {
errorMessage = objectMapper.writeValueAsString(error); errorMessage = mapper.writeValueAsString(error);
} catch (JsonProcessingException e) { } catch (JsonProcessingException e) {
errorMessage = String.format( errorMessage = String.format(
"{\"error\": {\"type\": \"elasticsearch_response_parse_error\", \"reason\": \"%s\"}}", "{\"error\": {\"type\": \"elasticsearch_response_parse_error\", \"reason\": \"%s\"}}",

View File

@ -26,6 +26,7 @@ import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyDescriptor;
@ -93,6 +94,7 @@ import java.util.stream.Collectors;
@WritesAttribute(attribute = "elasticsearch.put.success.count", description = "The number of records that were successfully processed by the Elasticsearch _bulk API."), @WritesAttribute(attribute = "elasticsearch.put.success.count", description = "The number of records that were successfully processed by the Elasticsearch _bulk API."),
@WritesAttribute(attribute = "elasticsearch.bulk.error", description = "The _bulk response if there was an error during processing the record within Elasticsearch.") @WritesAttribute(attribute = "elasticsearch.bulk.error", description = "The _bulk response if there was an error during processing the record within Elasticsearch.")
}) })
@SeeAlso(PutElasticsearchJson.class)
@DynamicProperties({ @DynamicProperties({
@DynamicProperty( @DynamicProperty(
name = "The name of the Bulk request header", name = "The name of the Bulk request header",
@ -460,7 +462,7 @@ public class PutElasticsearchRecord extends AbstractPutElasticsearch {
stopWatch.getDuration(TimeUnit.MILLISECONDS) stopWatch.getDuration(TimeUnit.MILLISECONDS)
); );
input = session.putAllAttributes(input, new HashMap<>() {{ input = session.putAllAttributes(input, new HashMap<String, String>() {{
put("elasticsearch.put.error.count", String.valueOf(erroredRecords.get())); put("elasticsearch.put.error.count", String.valueOf(erroredRecords.get()));
put("elasticsearch.put.success.count", String.valueOf(successfulRecords.get())); put("elasticsearch.put.success.count", String.valueOf(successfulRecords.get()));
}}); }});
@ -666,7 +668,7 @@ public class PutElasticsearchRecord extends AbstractPutElasticsearch {
map = DataTypeUtils.toMap(fieldValue.getValue(), path.getPath()); map = DataTypeUtils.toMap(fieldValue.getValue(), path.getPath());
} else { } else {
try { try {
map = MAPPER.readValue(fieldValue.getValue().toString(), Map.class); map = mapper.readValue(fieldValue.getValue().toString(), Map.class);
} catch (final JsonProcessingException jpe) { } catch (final JsonProcessingException jpe) {
getLogger().error("Unable to parse field {} as Map", path.getPath(), jpe); getLogger().error("Unable to parse field {} as Map", path.getPath(), jpe);
throw new ProcessException( throw new ProcessException(
@ -825,7 +827,7 @@ public class PutElasticsearchRecord extends AbstractPutElasticsearch {
this.writer.write(record); this.writer.write(record);
if (errorType != null && exampleError == null && error != null) { if (errorType != null && exampleError == null && error != null) {
try { try {
exampleError = MAPPER.writeValueAsString(error); exampleError = mapper.writeValueAsString(error);
} catch (JsonProcessingException e) { } catch (JsonProcessingException e) {
exampleError = String.format( exampleError = String.format(
"{\"error\": {\"type\": \"elasticsearch_response_parse_error\", \"reason\": \"%s\"}}", "{\"error\": {\"type\": \"elasticsearch_response_parse_error\", \"reason\": \"%s\"}}",

View File

@ -27,6 +27,7 @@ import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.configuration.DefaultSchedule; import org.apache.nifi.annotation.configuration.DefaultSchedule;
import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.Scope; import org.apache.nifi.components.state.Scope;
@ -50,12 +51,13 @@ import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors;
@WritesAttributes({ @WritesAttributes({
@WritesAttribute(attribute = "mime.type", description = "application/json"), @WritesAttribute(attribute = "mime.type", description = "application/json"),
@WritesAttribute(attribute = "aggregation.name", description = "The name of the aggregation whose results are in the output flowfile"), @WritesAttribute(attribute = "aggregation.name", description = "The name of the aggregation whose results are in the output flowfile"),
@WritesAttribute(attribute = "aggregation.number", description = "The number of the aggregation whose results are in the output flowfile"), @WritesAttribute(attribute = "aggregation.number", description = "The number of the aggregation whose results are in the output flowfile"),
@WritesAttribute(attribute = "page.number", description = "The number of the page (request) in which the results were returned that are in the output flowfile"), @WritesAttribute(attribute = "page.number", description = "The number of the page (request), starting from 1, in which the results were returned that are in the output flowfile"),
@WritesAttribute(attribute = "hit.count", description = "The number of hits that are in the output flowfile"), @WritesAttribute(attribute = "hit.count", description = "The number of hits that are in the output flowfile"),
@WritesAttribute(attribute = "elasticsearch.query.error", description = "The error message provided by Elasticsearch if there is an error querying the index.") @WritesAttribute(attribute = "elasticsearch.query.error", description = "The error message provided by Elasticsearch if there is an error querying the index.")
}) })
@ -68,10 +70,11 @@ import java.util.Set;
"Search After/Point in Time queries must include a valid \"sort\" field. The processor will retrieve multiple pages of results " + "Search After/Point in Time queries must include a valid \"sort\" field. The processor will retrieve multiple pages of results " +
"until either no more results are available or the Pagination Keep Alive expiration is reached, after which the query will " + "until either no more results are available or the Pagination Keep Alive expiration is reached, after which the query will " +
"restart with the first page of results being retrieved.") "restart with the first page of results being retrieved.")
@SeeAlso({PaginatedJsonQueryElasticsearch.class, ConsumeElasticsearch.class})
@DynamicProperty( @DynamicProperty(
name = "The name of a URL query parameter to add", name = "The name of a URL query parameter to add",
value = "The value of the URL query parameter", value = "The value of the URL query parameter",
expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES, expressionLanguageScope = ExpressionLanguageScope.ENVIRONMENT,
description = "Adds the specified property name/value as a query parameter in the Elasticsearch URL used for processing. " + description = "Adds the specified property name/value as a query parameter in the Elasticsearch URL used for processing. " +
"These parameters will override any matching parameters in the query request body. " + "These parameters will override any matching parameters in the query request body. " +
"For SCROLL type queries, these parameters are only used in the initial (first page) query as the " + "For SCROLL type queries, these parameters are only used in the initial (first page) query as the " +
@ -90,14 +93,13 @@ public class SearchElasticsearch extends AbstractPaginatedJsonQueryElasticsearch
static final String STATE_HIT_COUNT = "hitCount"; static final String STATE_HIT_COUNT = "hitCount";
static final PropertyDescriptor QUERY = new PropertyDescriptor.Builder().fromPropertyDescriptor(ElasticsearchRestProcessor.QUERY) static final PropertyDescriptor QUERY = new PropertyDescriptor.Builder().fromPropertyDescriptor(ElasticsearchRestProcessor.QUERY)
.name("el-rest-query") .description("A query in JSON syntax, not Lucene syntax. Ex: {\"query\":{\"match\":{\"somefield\":\"somevalue\"}}}. " +
.description("A query in JSON syntax, not Lucene syntax. Ex: {\"query\":{\"match\":{\"somefield\":\"somevalue\"}}}.") "If the query is empty, a default JSON Object will be used, which will result in a \"match_all\" query in Elasticsearch.")
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.required(true)
.build(); .build();
private static final Set<Relationship> relationships; private static final Set<Relationship> relationships;
private static final List<PropertyDescriptor> propertyDescriptors;
static final List<PropertyDescriptor> scrollPropertyDescriptors;
static { static {
final Set<Relationship> rels = new HashSet<>(); final Set<Relationship> rels = new HashSet<>();
@ -106,10 +108,14 @@ public class SearchElasticsearch extends AbstractPaginatedJsonQueryElasticsearch
relationships = Collections.unmodifiableSet(rels); relationships = Collections.unmodifiableSet(rels);
final List<PropertyDescriptor> descriptors = new ArrayList<>(); final List<PropertyDescriptor> descriptors = new ArrayList<>();
descriptors.add(SearchElasticsearch.QUERY); // ensure QUERY_DEFINITION_STYLE first for consistency between Elasticsearch processors
descriptors.addAll(paginatedPropertyDescriptors); descriptors.add(QUERY_DEFINITION_STYLE);
descriptors.add(QUERY);
propertyDescriptors = Collections.unmodifiableList(descriptors); descriptors.addAll(paginatedPropertyDescriptors.stream().filter(
// override QUERY to change description (no FlowFile content used by SearchElasticsearch)
pd -> !ElasticsearchRestProcessor.QUERY.equals(pd) && !QUERY_DEFINITION_STYLE.equals(pd)
).collect(Collectors.toList()));
scrollPropertyDescriptors = descriptors;
} }
@Override @Override
@ -118,15 +124,19 @@ public class SearchElasticsearch extends AbstractPaginatedJsonQueryElasticsearch
} }
@Override @Override
public final List<PropertyDescriptor> getSupportedPropertyDescriptors() { public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return propertyDescriptors; return scrollPropertyDescriptors;
}
Scope getStateScope() {
return Scope.LOCAL;
} }
@Override @Override
PaginatedJsonQueryParameters buildJsonQueryParameters(final FlowFile input, final ProcessContext context, final ProcessSession session) throws IOException { PaginatedJsonQueryParameters buildJsonQueryParameters(final FlowFile input, final ProcessContext context, final ProcessSession session) throws IOException {
final PaginatedJsonQueryParameters paginatedQueryJsonParameters = super.buildJsonQueryParameters(input, context, session); final PaginatedJsonQueryParameters paginatedQueryJsonParameters = super.buildJsonQueryParameters(input, context, session);
final StateMap stateMap = context.getStateManager().getState(Scope.LOCAL); final StateMap stateMap = context.getStateManager().getState(getStateScope());
paginatedQueryJsonParameters.setHitCount(stateMap.get(STATE_HIT_COUNT) == null ? 0 : Integer.parseInt(stateMap.get(STATE_HIT_COUNT))); paginatedQueryJsonParameters.setHitCount(stateMap.get(STATE_HIT_COUNT) == null ? 0 : Integer.parseInt(stateMap.get(STATE_HIT_COUNT)));
paginatedQueryJsonParameters.setPageCount(stateMap.get(STATE_PAGE_COUNT) == null ? 0 : Integer.parseInt(stateMap.get(STATE_PAGE_COUNT))); paginatedQueryJsonParameters.setPageCount(stateMap.get(STATE_PAGE_COUNT) == null ? 0 : Integer.parseInt(stateMap.get(STATE_PAGE_COUNT)));
paginatedQueryJsonParameters.setScrollId(stateMap.get(STATE_SCROLL_ID)); paginatedQueryJsonParameters.setScrollId(stateMap.get(STATE_SCROLL_ID));
@ -138,15 +148,16 @@ public class SearchElasticsearch extends AbstractPaginatedJsonQueryElasticsearch
} }
@Override @Override
void finishQuery(final FlowFile input, final PaginatedJsonQueryParameters paginatedQueryJsonParameters, void finishQuery(final FlowFile input, final PaginatedJsonQueryParameters paginatedJsonQueryParameters,
final ProcessSession session, final ProcessContext context, final SearchResponse response) throws IOException { final ProcessSession session, final ProcessContext context, final SearchResponse response) throws IOException {
if (response.getHits().isEmpty()) { final Map<String, String> newStateMap = new HashMap<>(10, 1);
getLogger().debug("No more results for paginated query, resetting local state for future queries"); additionalState(newStateMap, paginatedJsonQueryParameters);
resetProcessorState(context);
} else { if (response.getHits().isEmpty()) {
getLogger().debug("Updating local state for next execution"); getLogger().debug("No more results for paginated query, resetting state for future queries");
} else {
getLogger().debug("Updating state for next execution");
final Map<String, String> newStateMap = new HashMap<>();
if (paginationType == PaginationType.SCROLL) { if (paginationType == PaginationType.SCROLL) {
newStateMap.put(STATE_SCROLL_ID, response.getScrollId()); newStateMap.put(STATE_SCROLL_ID, response.getScrollId());
} else { } else {
@ -156,11 +167,15 @@ public class SearchElasticsearch extends AbstractPaginatedJsonQueryElasticsearch
newStateMap.put(STATE_PIT_ID, response.getPitId()); newStateMap.put(STATE_PIT_ID, response.getPitId());
} }
} }
newStateMap.put(STATE_HIT_COUNT, Integer.toString(paginatedQueryJsonParameters.getHitCount())); newStateMap.put(STATE_HIT_COUNT, Integer.toString(paginatedJsonQueryParameters.getHitCount()));
newStateMap.put(STATE_PAGE_COUNT, Integer.toString(paginatedQueryJsonParameters.getPageCount())); newStateMap.put(STATE_PAGE_COUNT, Integer.toString(paginatedJsonQueryParameters.getPageCount()));
newStateMap.put(STATE_PAGE_EXPIRATION_TIMESTAMP, paginatedQueryJsonParameters.getPageExpirationTimestamp()); newStateMap.put(STATE_PAGE_EXPIRATION_TIMESTAMP, paginatedJsonQueryParameters.getPageExpirationTimestamp());
context.getStateManager().setState(newStateMap, Scope.LOCAL);
} }
updateProcessorState(context, newStateMap);
}
void additionalState(final Map<String, String> newStateMap, final PaginatedJsonQueryParameters paginatedJsonQueryParameters) {
// intentionally blank, allows ConsumeElasticsearch to track range value between sessions
} }
@Override @Override
@ -168,10 +183,13 @@ public class SearchElasticsearch extends AbstractPaginatedJsonQueryElasticsearch
final SearchResponse response) throws IOException { final SearchResponse response) throws IOException {
final boolean expiredQuery = StringUtils.isNotEmpty(paginatedJsonQueryParameters.getPageExpirationTimestamp()) final boolean expiredQuery = StringUtils.isNotEmpty(paginatedJsonQueryParameters.getPageExpirationTimestamp())
&& Instant.ofEpochMilli(Long.parseLong(paginatedJsonQueryParameters.getPageExpirationTimestamp())).isBefore(Instant.now()); && Instant.ofEpochMilli(Long.parseLong(paginatedJsonQueryParameters.getPageExpirationTimestamp())).isBefore(Instant.now());
if (expiredQuery) { if (expiredQuery) {
getLogger().debug("Existing paginated query has expired, resetting for new query"); getLogger().debug("Existing paginated query has expired, resetting for new query");
resetProcessorState(context); final Map<String, String> newStateMap = new HashMap<>(1, 1);
additionalState(newStateMap, paginatedJsonQueryParameters);
updateProcessorState(context, newStateMap);
paginatedJsonQueryParameters.setPageCount(0); paginatedJsonQueryParameters.setPageCount(0);
paginatedJsonQueryParameters.setHitCount(0); paginatedJsonQueryParameters.setHitCount(0);
@ -186,20 +204,24 @@ public class SearchElasticsearch extends AbstractPaginatedJsonQueryElasticsearch
@Override @Override
String getScrollId(final ProcessContext context, final SearchResponse response) throws IOException { String getScrollId(final ProcessContext context, final SearchResponse response) throws IOException {
return response == null || StringUtils.isBlank(response.getScrollId()) return response == null || StringUtils.isBlank(response.getScrollId())
? context.getStateManager().getState(Scope.LOCAL).get(STATE_SCROLL_ID) ? context.getStateManager().getState(getStateScope()).get(STATE_SCROLL_ID)
: response.getScrollId(); : response.getScrollId();
} }
@Override @Override
String getPitId(final ProcessContext context, final SearchResponse response) throws IOException { String getPitId(final ProcessContext context, final SearchResponse response) throws IOException {
return response == null || StringUtils.isBlank(response.getScrollId()) return response == null || StringUtils.isBlank(response.getScrollId())
? context.getStateManager().getState(Scope.LOCAL).get(STATE_PIT_ID) ? context.getStateManager().getState(getStateScope()).get(STATE_PIT_ID)
: response.getPitId(); : response.getPitId();
} }
private void resetProcessorState(final ProcessContext context) throws IOException { void updateProcessorState(final ProcessContext context, final Map<String, String> newStateMap) throws IOException {
// using ProcessContext#stateManager instead of ProcessSession#*State methods because the latter don't // using ProcessContext#stateManager instead of ProcessSession#*State methods because the latter don't
// seem to persist things properly between sessions if the processor is scheduled to run very quickly, e.g. every second (NIFI-9050) // seem to persist things properly between sessions if the processor is scheduled to run very quickly, e.g. every second (NIFI-9050)
context.getStateManager().clear(Scope.LOCAL); if (newStateMap == null || newStateMap.isEmpty()) {
context.getStateManager().clear(getStateScope());
} else {
context.getStateManager().setState(newStateMap, getStateScope());
}
} }
} }

View File

@ -24,6 +24,7 @@ public class PaginatedJsonQueryParameters extends JsonQueryParameters {
private String pitId = null; private String pitId = null;
private String pageExpirationTimestamp = null; private String pageExpirationTimestamp = null;
private String keepAlive; private String keepAlive;
private String trackingRangeValue;
public int getPageCount() { public int getPageCount() {
return pageCount; return pageCount;
@ -76,4 +77,12 @@ public class PaginatedJsonQueryParameters extends JsonQueryParameters {
public void setKeepAlive(final String keepAlive) { public void setKeepAlive(final String keepAlive) {
this.keepAlive = keepAlive; this.keepAlive = keepAlive;
} }
public String getTrackingRangeValue() {
return trackingRangeValue;
}
public void setTrackingRangeValue(String trackingRangeValue) {
this.trackingRangeValue = trackingRangeValue;
}
} }

View File

@ -0,0 +1,55 @@
/*
* 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.api;
import org.apache.nifi.components.DescribedValue;
import java.util.Arrays;
public enum QueryDefinitionType implements DescribedValue {
FULL_QUERY("full", "Provide the full Query."),
BUILD_QUERY("build", "Build the Query from separate JSON objects.");
private final String value;
private final String description;
QueryDefinitionType(final String value, final String description) {
this.value = value;
this.description = description;
}
@Override
public String getValue() {
return value;
}
@Override
public String getDisplayName() {
return name();
}
@Override
public String getDescription() {
return description;
}
public static QueryDefinitionType fromValue(final String value) {
return Arrays.stream(QueryDefinitionType.values()).filter(v -> v.getValue().equals(value)).findFirst()
.orElseThrow(() -> new IllegalArgumentException(String.format("Unknown value %s", value)));
}
}

View File

@ -17,8 +17,8 @@ org.apache.nifi.processors.elasticsearch.DeleteByQueryElasticsearch
org.apache.nifi.processors.elasticsearch.JsonQueryElasticsearch org.apache.nifi.processors.elasticsearch.JsonQueryElasticsearch
org.apache.nifi.processors.elasticsearch.PaginatedJsonQueryElasticsearch org.apache.nifi.processors.elasticsearch.PaginatedJsonQueryElasticsearch
org.apache.nifi.processors.elasticsearch.SearchElasticsearch org.apache.nifi.processors.elasticsearch.SearchElasticsearch
org.apache.nifi.processors.elasticsearch.ConsumeElasticsearch
org.apache.nifi.processors.elasticsearch.PutElasticsearchRecord org.apache.nifi.processors.elasticsearch.PutElasticsearchRecord
org.apache.nifi.processors.elasticsearch.PutElasticsearchJson org.apache.nifi.processors.elasticsearch.PutElasticsearchJson
org.apache.nifi.processors.elasticsearch.UpdateByQueryElasticsearch org.apache.nifi.processors.elasticsearch.UpdateByQueryElasticsearch
org.apache.nifi.processors.elasticsearch.GetElasticsearch org.apache.nifi.processors.elasticsearch.GetElasticsearch

View File

@ -0,0 +1,79 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>SearchElasticsearch</title>
<link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
</head>
<body>
<p>This processor is intended for use with the Elasticsearch JSON DSL and Elasticsearch 5.X and newer. It is designed
to be able to create a JSON query using input properties and execute it against an Elasticsearch cluster in a paginated manner.
Like all processors in the "restapi" bundle, it uses the official Elastic client APIs, so it supports leader detection.</p>
<p>The query is paginated in Elasticsearch using one of the available methods - "Scroll" or "Search After" (optionally
with a "Point in Time" for Elasticsearch 7.10+ with XPack enabled). The number of results per page can be controlled using
the <em>size</em> property.</p>
<p>Results will be sorted on the field that is to be tracked, with the <em>sort order</em> set as a property.</p>
<p>Search results and aggregation results can be split up into multiple flowfiles. Aggregation results
will only be split at the top level because nested aggregations lose their context (and thus lose their value) if
separated from their parent aggregation. Additionally, the results from all pages can be combined into a single
flowfile (but the processor will only load each page of data into memory at any one time).</p>
<p>The following is an example query that would be created for tracking an "@timestamp" field:</p>
<pre>
{
"query": {
"size": 10000,
"sort": {"@timestamp": "desc"},
"bool": {
"filter": [
{"range": {"@timestamp": {"gt": "2023-09-01"}}}
]
}
}
}
</pre>
<p>Additional "filter" entries can be added as a JSON string in the <em>query filter</em> property, for example:</p>
<pre>
[
{"term": {"department": "accounts"}},
{"term": {"title.keyword": "Annual Report"}}
]
</pre>
<h2>Query Pagination Across Processor Executions</h2>
<p>This processor runs on a schedule in order to execute the same query repeatedly. Once a paginated query has been
initiated within Elasticsearch, this processor will continue to retrieve results for that same query until no
further results are available. After that point, a new paginated query will be initiated using the same Query JSON,
but with the "range" filter query's "gt" field set to the last value obtained from previous results.</p>
<p>If the results are "Combined" from this processor, then the paginated query will run continually within a
single invocation until no more results are available (then the processor will start a new paginated query upon its
next invocation). If the results are "Split" or "Per Page", then each invocation of this processor will retrieve the
next page of results until either there are no more results or the paginated query expires within Elasticsearch.</p>
<h2>Resetting Queries / Clearing Processor State</h2>
<p>Cluster State is used to track the progress of a paginated query within this processor. If there is need to restart
the query completely or change the processor configuration after a paginated query has already been started,
be sure to "Clear State" of the processor once it has been stopped and before restarting.</p>
<p>Note that clearing the processor's state will lose details of where the processor was up to with tracking documents retrieved.
Update the "Initial Value" with the appropriate value before restarting the processor to continue with where it was up to.</p>
<h2>Duplicate Results</h2>
<p>This processor does not attempt to de-duplicate results between queries, for example if the same query runs twice
and (some or all of) the results are identical, the output will contain these same results for both invocations.
This might happen if the NiFi Primary Node changes while a page of data is being retrieved, or if the processor
state is cleared, then the processor is restarted.</p>
</body>
</html>

View File

@ -27,7 +27,7 @@
If the Query Attribute property is configured, the executed query JSON will be placed in the attribute provided by this property.</p> If the Query Attribute property is configured, the executed query JSON will be placed in the attribute provided by this property.</p>
<p>The query is paginated in Elasticsearch using one of the available methods - "Scroll" or "Search After" (optionally <p>The query is paginated in Elasticsearch using one of the available methods - "Scroll" or "Search After" (optionally
with a "Point in Time" for Elasticsearch 7.10+ with XPack enabled). The number of results per page can be controlled using with a "Point in Time" for Elasticsearch 7.10+ with XPack enabled). The number of results per page can be controlled using
the <em>size</em> parameter in the Query JSON. For Search After functionality, a <em>sort</em> parameter <strong>must</strong> the <em>size</em> parameter in the Query JSON. For Search After functionality, a <em>sort</em> parameter <strong>must</strong>
be present within the Query JSON.</p> be present within the Query JSON.</p>
<p>Search results and aggregation results can be split up into multiple flowfiles. Aggregation results <p>Search results and aggregation results can be split up into multiple flowfiles. Aggregation results
will only be split at the top level because nested aggregations lose their context (and thus lose their value) if will only be split at the top level because nested aggregations lose their context (and thus lose their value) if

View File

@ -60,17 +60,27 @@
} }
} }
</pre> </pre>
<h2>Query Pagination Across Processor Executions</h2>
<p>This processor runs on a schedule in order to execute the same query repeatedly. Once a paginated query has been <p>This processor runs on a schedule in order to execute the same query repeatedly. Once a paginated query has been
initiated within Elasticsearch, this processor will continue to retrieve results for that same query until no initiated within Elasticsearch, this processor will continue to retrieve results for that same query until no
further results are available. After that point, a new paginated query will be initiated using the same Query JSON. further results are available. After that point, a new paginated query will be initiated using the same Query JSON.</p>
This processor does not attempt to de-duplicate results between queries, for example if the same query runs twice
and (some or all of) the results are identical, the output will contain these same results for both invocations.</p>
<p>If the results are "Combined" from this processor, then the paginated query will run continually within a <p>If the results are "Combined" from this processor, then the paginated query will run continually within a
single invocation until no more results are available (then the processor will start a new paginated query upon its single invocation until no more results are available (then the processor will start a new paginated query upon its
next invocation). If the results are "Split" or "Per Page", then each invocation of this processor will retrieve the next invocation). If the results are "Split" or "Per Page", then each invocation of this processor will retrieve the
next page of results until either there are no more results or the paginated query expires within Elasticsearch.</p> next page of results until either there are no more results or the paginated query expires within Elasticsearch.</p>
<h2>Resetting Queries / Clearing Processor State</h2>
<p>Local State is used to track the progress of a paginated query within this processor. If there is need to restart <p>Local State is used to track the progress of a paginated query within this processor. If there is need to restart
the query completely or change the processor configuration after a paginated query has already been started, the query completely or change the processor configuration after a paginated query has already been started,
be sure to "Clear State" of the processor once it has been stopped and before restarting.</p> be sure to "Clear State" of the processor once it has been stopped and before restarting.</p>
<h2>Duplicate Results</h2>
<p>This processor does not attempt to de-duplicate results between queries, for example if the same query runs twice
and (some or all of) the results are identical, the output will contain these same results for both invocations.
This might happen if the NiFi Primary Node changes while a page of data is being retrieved, or if the processor
state is cleared, then the processor is restarted.</p>
<p>This processor will continually run the same query unless the processor properties are updated, so unless
the data in Elasticsearch has changed, the same data will be retrieved multiple times.s</p>
</body> </body>
</html> </html>

View File

@ -16,20 +16,28 @@
*/ */
package org.apache.nifi.processors.elasticsearch; package org.apache.nifi.processors.elasticsearch;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.nifi.processors.elasticsearch.api.QueryDefinitionType;
import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners; import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import java.nio.file.Files;
import java.nio.file.Paths; import java.nio.file.Paths;
import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
public abstract class AbstractByQueryElasticsearchTest { public abstract class AbstractByQueryElasticsearchTest {
@ -37,8 +45,11 @@ public abstract class AbstractByQueryElasticsearchTest {
private static final String INDEX = "test_idx"; private static final String INDEX = "test_idx";
private static final String TYPE = "test_type"; private static final String TYPE = "test_type";
private static final String CLIENT_NAME = "clientService"; private static final String CLIENT_NAME = "clientService";
private TestElasticsearchClientService client;
private static final ObjectMapper TEST_MAPPER = new ObjectMapper();
private static String matchAllQuery; private static String matchAllQuery;
private TestElasticsearchClientService client;
private TestRunner runner; private TestRunner runner;
public abstract String queryAttr(); public abstract String queryAttr();
@ -47,13 +58,13 @@ public abstract class AbstractByQueryElasticsearchTest {
public abstract String errorAttr(); public abstract String errorAttr();
public abstract Class<? extends AbstractByQueryElasticsearch> getTestProcessor(); public abstract AbstractByQueryElasticsearch getTestProcessor();
public abstract void expectError(final TestElasticsearchClientService client); public abstract void expectError(final TestElasticsearchClientService client);
@BeforeAll @BeforeAll
public static void setUpBeforeClass() throws Exception { public static void setUpBeforeClass() throws Exception {
matchAllQuery = Files.readString(Paths.get("src/test/resources/common/matchAllQuery.json")); matchAllQuery = JsonUtils.readString(Paths.get("src/test/resources/common/matchAllQuery.json"));
} }
@BeforeEach @BeforeEach
@ -65,7 +76,7 @@ public abstract class AbstractByQueryElasticsearchTest {
runner.setProperty(AbstractByQueryElasticsearch.CLIENT_SERVICE, CLIENT_NAME); runner.setProperty(AbstractByQueryElasticsearch.CLIENT_SERVICE, CLIENT_NAME);
} }
private void postTest(TestRunner runner, String queryParam) { private void postTest(final TestRunner runner, final String queryParam) {
runner.assertTransferCount(AbstractByQueryElasticsearch.REL_FAILURE, 0); runner.assertTransferCount(AbstractByQueryElasticsearch.REL_FAILURE, 0);
runner.assertTransferCount(AbstractByQueryElasticsearch.REL_SUCCESS, 1); runner.assertTransferCount(AbstractByQueryElasticsearch.REL_SUCCESS, 1);
@ -78,6 +89,88 @@ public abstract class AbstractByQueryElasticsearchTest {
assertEquals(queryParam, query); assertEquals(queryParam, query);
} }
@Test
void testMandatoryProperties() {
runner.removeProperty(ElasticsearchRestProcessor.CLIENT_SERVICE);
runner.removeProperty(ElasticsearchRestProcessor.INDEX);
runner.removeProperty(ElasticsearchRestProcessor.TYPE);
runner.removeProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE);
runner.removeProperty(ElasticsearchRestProcessor.QUERY);
runner.removeProperty(ElasticsearchRestProcessor.QUERY_ATTRIBUTE);
final AssertionError assertionError = assertThrows(AssertionError.class, runner::run);
final String expected = String.format("Processor has 2 validation failures:\n" + "'%s' is invalid because %s is required\n" + "'%s' is invalid because %s is required\n",
ElasticsearchRestProcessor.INDEX.getDisplayName(), ElasticsearchRestProcessor.INDEX.getDisplayName(),
ElasticsearchRestProcessor.CLIENT_SERVICE.getDisplayName(), ElasticsearchRestProcessor.CLIENT_SERVICE.getDisplayName());
assertEquals(expected, assertionError.getMessage());
}
@Test
void testInvalidProperties() {
runner.setProperty(ElasticsearchRestProcessor.CLIENT_SERVICE, "not-a-service");
runner.setProperty(ElasticsearchRestProcessor.INDEX, "");
runner.setProperty(ElasticsearchRestProcessor.TYPE, "");
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, "not-valid");
final AssertionError assertionError = assertThrows(AssertionError.class, runner::run);
final String expected = String.format("Processor has 5 validation failures:\n" +
"'%s' validated against 'not-valid' is invalid because Given value not found in allowed set '%s'\n" +
"'%s' validated against '' is invalid because %s cannot be empty\n" +
"'%s' validated against '' is invalid because %s cannot be empty\n" +
"'%s' validated against 'not-a-service' is invalid because" +
" Property references a Controller Service that does not exist\n" +
"'%s' validated against 'not-a-service' is invalid because Invalid Controller Service: not-a-service is not a valid Controller Service Identifier\n",
ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE.getName(),
Arrays.stream(QueryDefinitionType.values()).map(QueryDefinitionType::getValue).collect(Collectors.joining(", ")),
ElasticsearchRestProcessor.INDEX.getName(), ElasticsearchRestProcessor.INDEX.getName(),
ElasticsearchRestProcessor.TYPE.getName(), ElasticsearchRestProcessor.TYPE.getName(),
ElasticsearchRestProcessor.CLIENT_SERVICE.getDisplayName(), ElasticsearchRestProcessor.CLIENT_SERVICE.getDisplayName());
assertEquals(expected, assertionError.getMessage());
}
@Test
void testInvalidQueryProperty() {
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, QueryDefinitionType.FULL_QUERY.getValue());
runner.setProperty(ElasticsearchRestProcessor.INDEX, "test-index");
runner.setProperty(ElasticsearchRestProcessor.QUERY, "not-json");
final AssertionError assertionError = assertThrows(AssertionError.class, runner::run);
final String expected = String.format("Processor has 1 validation failures:\n" +
"'%s' validated against 'not-json' is invalid because %s is not a valid JSON representation due to Unrecognized token 'not': was expecting" +
" (JSON String, Number, Array, Object or token 'null', 'true' or 'false')\n" +
" at [Source: (String)\"not-json\"; line: 1, column: 4]\n",
ElasticsearchRestProcessor.QUERY.getName(), ElasticsearchRestProcessor.QUERY.getName());
assertEquals(expected, assertionError.getMessage());
}
@Test
void testInvalidQueryBuilderProperties() {
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, QueryDefinitionType.BUILD_QUERY.getValue());
runner.setProperty(ElasticsearchRestProcessor.INDEX, "test-index");
runner.setProperty(ElasticsearchRestProcessor.QUERY_CLAUSE, "not-json");
runner.setProperty(ElasticsearchRestProcessor.SCRIPT, "not-json-script");
final AssertionError assertionError = assertThrows(AssertionError.class, runner::run);
String expected;
if (getTestProcessor() instanceof DeleteByQueryElasticsearch) {
// no SCRIPT in Query Builder
expected = "Processor has 1 validation failures:\n";
} else {
expected = "Processor has 2 validation failures:\n";
}
expected += String.format("'%s' validated against 'not-json' is invalid because %s is not a valid JSON representation due to Unrecognized token 'not': was expecting" +
" (JSON String, Number, Array, Object or token 'null', 'true' or 'false')\n" +
" at [Source: (String)\"not-json\"; line: 1, column: 4]\n",
ElasticsearchRestProcessor.QUERY_CLAUSE.getName(), ElasticsearchRestProcessor.QUERY_CLAUSE.getName());
if (getTestProcessor() instanceof UpdateByQueryElasticsearch) {
expected += String.format("'%s' validated against 'not-json-script' is invalid because %s is not a valid JSON representation due to Unrecognized token 'not': was expecting " +
"(JSON String, Number, Array, Object or token 'null', 'true' or 'false')\n" +
" at [Source: (String)\"not-json-script\"; line: 1, column: 4]\n",
ElasticsearchRestProcessor.SCRIPT.getName(), ElasticsearchRestProcessor.SCRIPT.getName());
}
assertEquals(expected, assertionError.getMessage());
}
@Test @Test
public void testWithFlowfileInput() { public void testWithFlowfileInput() {
final String query = matchAllQuery; final String query = matchAllQuery;
@ -114,7 +207,7 @@ public abstract class AbstractByQueryElasticsearchTest {
@Test @Test
public void testWithQuery() throws Exception { public void testWithQuery() throws Exception {
final String query = Files.readString(Paths.get(TEST_DIR,"matchUsingExpressionLanguageQuery.json")); final String query = JsonUtils.readString(Paths.get(TEST_DIR,"matchUsingExpressionLanguageQuery.json"));
runner.setProperty(AbstractByQueryElasticsearch.QUERY, query); runner.setProperty(AbstractByQueryElasticsearch.QUERY, query);
runner.setProperty(AbstractByQueryElasticsearch.INDEX, INDEX); runner.setProperty(AbstractByQueryElasticsearch.INDEX, INDEX);
runner.setProperty(AbstractByQueryElasticsearch.TYPE, TYPE); runner.setProperty(AbstractByQueryElasticsearch.TYPE, TYPE);
@ -127,7 +220,7 @@ public abstract class AbstractByQueryElasticsearchTest {
runner.clearTransferState(); runner.clearTransferState();
final String query2 = Files.readString(Paths.get(TEST_DIR, "matchQuery.json")); final String query2 = JsonUtils.readString(Paths.get(TEST_DIR, "matchQuery.json"));
runner.setProperty(AbstractByQueryElasticsearch.QUERY, query2); runner.setProperty(AbstractByQueryElasticsearch.QUERY, query2);
runner.setIncomingConnection(false); runner.setIncomingConnection(false);
runner.assertValid(); runner.assertValid();
@ -205,4 +298,34 @@ public abstract class AbstractByQueryElasticsearchTest {
assertEquals("true", client.getRequestParameters().get("refresh")); assertEquals("true", client.getRequestParameters().get("refresh"));
assertEquals("auto", client.getRequestParameters().get("slices")); assertEquals("auto", client.getRequestParameters().get("slices"));
} }
@ParameterizedTest
@MethodSource
void testQueryBuilder(final String queryClause, final String script, final String expectedQuery) throws Exception {
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, QueryDefinitionType.BUILD_QUERY.getValue());
if (queryClause != null) {
runner.setProperty(ElasticsearchRestProcessor.QUERY_CLAUSE, queryClause);
}
if (script != null) {
runner.setProperty(ElasticsearchRestProcessor.SCRIPT, script);
}
final String query = getTestProcessor().getQuery(null, runner.getProcessContext(), null);
assertNotNull(query);
assertEquals(TEST_MAPPER.readTree(expectedQuery), TEST_MAPPER.readTree(query));
}
private static Stream<Arguments> testQueryBuilder() {
// bool query with range filter as constructed by ConsumeElasticsearch
final String queryClause = "{\"bool\": {\"filter\": [{\"range\": {\"@timestamp\": {\"gt\": \"123456\"}}}]}}";
final String script = "{\"source\": \"ctx._source.count++\", \"lang\": \"painless\"}";
return Stream.of(
Arguments.of(null, null, "{}"),
Arguments.of(queryClause, null, String.format("{\"query\": %s}", queryClause)),
Arguments.of(null, script, String.format("{\"script\": %s}", script)),
Arguments.of(queryClause, script, String.format("{\"query\": %s, \"script\": %s}", queryClause, script))
);
}
} }

View File

@ -16,8 +16,16 @@
*/ */
package org.apache.nifi.processors.elasticsearch; package org.apache.nifi.processors.elasticsearch;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ArrayNode;
import com.fasterxml.jackson.databind.node.ObjectNode;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.Scope; import org.apache.nifi.components.state.Scope;
import org.apache.nifi.processors.elasticsearch.api.AggregationResultsFormat; import org.apache.nifi.processors.elasticsearch.api.AggregationResultsFormat;
import org.apache.nifi.processors.elasticsearch.api.JsonQueryParameters;
import org.apache.nifi.processors.elasticsearch.api.QueryDefinitionType;
import org.apache.nifi.processors.elasticsearch.api.ResultOutputStrategy; import org.apache.nifi.processors.elasticsearch.api.ResultOutputStrategy;
import org.apache.nifi.processors.elasticsearch.api.SearchResultsFormat; import org.apache.nifi.processors.elasticsearch.api.SearchResultsFormat;
import org.apache.nifi.provenance.ProvenanceEventType; import org.apache.nifi.provenance.ProvenanceEventType;
@ -25,16 +33,21 @@ import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners; import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.Assumptions;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.EnumSource;
import org.junit.jupiter.params.provider.MethodSource;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Paths; import java.nio.file.Paths;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertFalse;
@ -43,58 +56,62 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQueryElasticsearch> { public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQueryElasticsearch<? extends JsonQueryParameters>> {
private static final String TEST_DIR = "src/test/resources/AbstractJsonQueryElasticsearchTest"; private static final String TEST_DIR = "src/test/resources/AbstractJsonQueryElasticsearchTest";
private static final String TEST_COMMON_DIR = "src/test/resources/common"; private static final String TEST_COMMON_DIR = "src/test/resources/common";
private static final String INDEX_NAME = "messages"; private static final String INDEX_NAME = "messages";
static final String RANGE_FIELD_NAME = "msg";
static final String RANGE_SORT_ORDER = "asc";
static final String RANGE_FIELD_VALUE = "123456";
// bool query with range filter as constructed by ConsumeElasticsearch
private static final String QUERY_CLAUSE = String.format("{\"bool\": {\"filter\": [{\"range\": {\"%s\": {\"gt\": \"123456\"}}}]}}", RANGE_FIELD_NAME);
static final String CONSUME_ELASTICSEARCH_SORT_CLAUSE = String.format("{\"%s\":\"%s\"}", RANGE_FIELD_NAME, RANGE_SORT_ORDER);
protected static String matchAllQuery; protected static String matchAllQuery;
protected static String matchAllAggregationWithDefaultTermsQuery; protected static String matchAllAggregationWithDefaultTermsQuery;
public abstract P getProcessor(); static final ObjectMapper TEST_MAPPER = new ObjectMapper();
public abstract boolean isStateUsed(); abstract P getProcessor();
public abstract boolean isInput(); abstract Scope getStateScope();
abstract boolean isInput();
@BeforeAll @BeforeAll
public static void setUpBeforeClass() throws Exception { public static void setUpBeforeClass() throws Exception {
matchAllQuery = Files.readString(Paths.get(TEST_COMMON_DIR, "matchAllQuery.json")); matchAllQuery = JsonUtils.readString(Paths.get(TEST_COMMON_DIR, "matchAllQuery.json"));
matchAllAggregationWithDefaultTermsQuery = Files.readString(Paths.get(TEST_DIR,"matchAllAggregationWithDefaultTermsQuery.json")); matchAllAggregationWithDefaultTermsQuery = JsonUtils.readString(Paths.get(TEST_DIR,"matchAllAggregationWithDefaultTermsQuery.json"));
} }
@Test @Test
public void testMandatoryProperties() { void testMandatoryProperties() {
final TestRunner runner = createRunner(false); final TestRunner runner = createRunner(false);
runner.removeProperty(AbstractJsonQueryElasticsearch.CLIENT_SERVICE); runner.removeProperty(ElasticsearchRestProcessor.CLIENT_SERVICE);
runner.removeProperty(AbstractJsonQueryElasticsearch.INDEX); runner.removeProperty(ElasticsearchRestProcessor.INDEX);
runner.removeProperty(AbstractJsonQueryElasticsearch.TYPE); runner.removeProperty(ElasticsearchRestProcessor.TYPE);
runner.removeProperty(AbstractJsonQueryElasticsearch.QUERY); runner.removeProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE);
runner.removeProperty(AbstractJsonQueryElasticsearch.QUERY_ATTRIBUTE); runner.removeProperty(ElasticsearchRestProcessor.QUERY);
runner.removeProperty(ElasticsearchRestProcessor.QUERY_ATTRIBUTE);
runner.removeProperty(AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_SPLIT); runner.removeProperty(AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_SPLIT);
runner.removeProperty(AbstractJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT); runner.removeProperty(AbstractJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT);
runner.removeProperty(AbstractJsonQueryElasticsearch.OUTPUT_NO_HITS); runner.removeProperty(AbstractJsonQueryElasticsearch.OUTPUT_NO_HITS);
final AssertionError assertionError = assertThrows(AssertionError.class, runner::run); final AssertionError assertionError = assertThrows(AssertionError.class, runner::run);
if (getProcessor() instanceof SearchElasticsearch) { final String expected = String.format("Processor has 2 validation failures:\n" + "'%s' is invalid because %s is required\n" + "'%s' is invalid because %s is required\n",
assertEquals(String.format("Processor has 3 validation failures:\n" + "'%s' is invalid because %s is required\n" ElasticsearchRestProcessor.INDEX.getDisplayName(), ElasticsearchRestProcessor.INDEX.getDisplayName(),
+ "'%s' is invalid because %s is required\n" + "'%s' is invalid because %s is required\n", ElasticsearchRestProcessor.CLIENT_SERVICE.getDisplayName(), ElasticsearchRestProcessor.CLIENT_SERVICE.getDisplayName());
AbstractJsonQueryElasticsearch.QUERY.getDisplayName(), AbstractJsonQueryElasticsearch.QUERY.getDisplayName(), assertEquals(expected, assertionError.getMessage());
AbstractJsonQueryElasticsearch.INDEX.getDisplayName(), AbstractJsonQueryElasticsearch.INDEX.getDisplayName(),
AbstractJsonQueryElasticsearch.CLIENT_SERVICE.getDisplayName(), AbstractJsonQueryElasticsearch.CLIENT_SERVICE.getDisplayName()), assertionError.getMessage());
} else {
assertEquals(String.format("Processor has 2 validation failures:\n" + "'%s' is invalid because %s is required\n" + "'%s' is invalid because %s is required\n",
AbstractJsonQueryElasticsearch.INDEX.getDisplayName(), AbstractJsonQueryElasticsearch.INDEX.getDisplayName(),
AbstractJsonQueryElasticsearch.CLIENT_SERVICE.getDisplayName(), AbstractJsonQueryElasticsearch.CLIENT_SERVICE.getDisplayName()), assertionError.getMessage());
}
} }
@Test @Test
public void testInvalidProperties() { void testInvalidProperties() {
final TestRunner runner = createRunner(false); final TestRunner runner = createRunner(false);
runner.setProperty(AbstractJsonQueryElasticsearch.CLIENT_SERVICE, "not-a-service"); runner.setProperty(ElasticsearchRestProcessor.CLIENT_SERVICE, "not-a-service");
runner.setProperty(AbstractJsonQueryElasticsearch.INDEX, ""); runner.setProperty(ElasticsearchRestProcessor.INDEX, "");
runner.setProperty(AbstractJsonQueryElasticsearch.TYPE, ""); runner.setProperty(ElasticsearchRestProcessor.TYPE, "");
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, "not-json"); runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, "not-valid");
runner.setProperty(AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_SPLIT, "not-enum"); runner.setProperty(AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_SPLIT, "not-enum");
runner.setProperty(AbstractJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT, "not-enum2"); runner.setProperty(AbstractJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT, "not-enum2");
runner.setProperty(AbstractJsonQueryElasticsearch.OUTPUT_NO_HITS, "not-boolean"); runner.setProperty(AbstractJsonQueryElasticsearch.OUTPUT_NO_HITS, "not-boolean");
@ -108,25 +125,105 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
.collect(Collectors.joining(", ")) : nonPaginatedResultOutputStrategies; .collect(Collectors.joining(", ")) : nonPaginatedResultOutputStrategies;
final AssertionError assertionError = assertThrows(AssertionError.class, runner::run); final AssertionError assertionError = assertThrows(AssertionError.class, runner::run);
String expected = String.format("Processor has 8 validation failures:\n" + String expected;
"'%s' validated against 'not-json' is invalid because %s is not a valid JSON representation due to Unrecognized token 'not': was expecting" + if (runner.getProcessor() instanceof ConsumeElasticsearch) {
" (JSON String, Number, Array, Object or token 'null', 'true' or 'false')\n" + expected = "Processor has 7 validation failures:\n";
" at [Source: (String)\"not-json\"; line: 1, column: 4]\n" + "'%s' validated against '' is invalid because %s cannot be empty\n" + } else {
"'%s' validated against '' is invalid because %s cannot be empty\n" + "'%s' validated against 'not-a-service' is invalid because" + expected = String.format("Processor has 8 validation failures:\n" +
" Property references a Controller Service that does not exist\n" + "'%s' validated against 'not-enum2' is invalid because Given value not found in allowed set '%s'\n" + "'%s' validated against 'not-valid' is invalid because Given value not found in allowed set '%s'\n",
ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE.getName(),
Arrays.stream(QueryDefinitionType.values()).map(QueryDefinitionType::getValue).collect(Collectors.joining(", ")));
}
expected += String.format(
"'%s' validated against '' is invalid because %s cannot be empty\n" +
"'%s' validated against '' is invalid because %s cannot be empty\n" +
"'%s' validated against 'not-a-service' is invalid because" +
" Property references a Controller Service that does not exist\n" +
"'%s' validated against 'not-enum2' is invalid because Given value not found in allowed set '%s'\n" +
"'%s' validated against 'not-enum' is invalid because Given value not found in allowed set '%s'\n" + "'%s' validated against 'not-enum' is invalid because Given value not found in allowed set '%s'\n" +
"'%s' validated against 'not-boolean' is invalid because Given value not found in allowed set 'true, false'\n" + "'%s' validated against 'not-boolean' is invalid because Given value not found in allowed set 'true, false'\n" +
"'%s' validated against 'not-a-service' is invalid because Invalid Controller Service: not-a-service is not a valid Controller Service Identifier\n", "'%s' validated against 'not-a-service' is invalid because Invalid Controller Service: not-a-service is not a valid Controller Service Identifier\n",
AbstractJsonQueryElasticsearch.QUERY.getName(), AbstractJsonQueryElasticsearch.QUERY.getName(), AbstractJsonQueryElasticsearch.INDEX.getName(), ElasticsearchRestProcessor.INDEX.getName(), ElasticsearchRestProcessor.INDEX.getName(),
AbstractJsonQueryElasticsearch.INDEX.getName(), AbstractJsonQueryElasticsearch.TYPE.getName(), AbstractJsonQueryElasticsearch.TYPE.getName(), ElasticsearchRestProcessor.TYPE.getName(), ElasticsearchRestProcessor.TYPE.getName(),
AbstractJsonQueryElasticsearch.CLIENT_SERVICE.getDisplayName(), AbstractJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT.getName(), expectedAllowedSplitHits, ElasticsearchRestProcessor.CLIENT_SERVICE.getDisplayName(),
AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_SPLIT.getName(), nonPaginatedResultOutputStrategies, AbstractJsonQueryElasticsearch.OUTPUT_NO_HITS.getName(), AbstractJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT.getName(), expectedAllowedSplitHits,
AbstractJsonQueryElasticsearch.CLIENT_SERVICE.getDisplayName()); AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_SPLIT.getName(), nonPaginatedResultOutputStrategies,
AbstractJsonQueryElasticsearch.OUTPUT_NO_HITS.getName(), ElasticsearchRestProcessor.CLIENT_SERVICE.getDisplayName());
assertEquals(expected, assertionError.getMessage()); assertEquals(expected, assertionError.getMessage());
} }
@Test @Test
public void testBasicQuery() { void testInvalidQueryProperty() {
final TestRunner runner = createRunner(false);
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, QueryDefinitionType.FULL_QUERY.getValue());
final PropertyDescriptor queryPropertyDescriptor;
if (runner.getProcessor() instanceof SearchElasticsearch) {
queryPropertyDescriptor = SearchElasticsearch.QUERY;
} else {
queryPropertyDescriptor = ElasticsearchRestProcessor.QUERY;
}
runner.setProperty(queryPropertyDescriptor, "not-json");
if (runner.getProcessor() instanceof ConsumeElasticsearch) {
// ConsumeElasticsearch does not use the QUERY property
runner.assertValid();
} else {
final AssertionError assertionError = assertThrows(AssertionError.class, runner::run);
final String expected = String.format("Processor has 1 validation failures:\n" +
"'%s' validated against 'not-json' is invalid because %s is not a valid JSON representation due to Unrecognized token 'not': was expecting" +
" (JSON String, Number, Array, Object or token 'null', 'true' or 'false')\n" +
" at [Source: (String)\"not-json\"; line: 1, column: 4]\n",
queryPropertyDescriptor.getName(), queryPropertyDescriptor.getName());
assertEquals(expected, assertionError.getMessage());
}
}
@Test
void testInvalidQueryBuilderProperties() {
final TestRunner runner = createRunner(false);
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, QueryDefinitionType.BUILD_QUERY.getValue());
runner.setProperty(ElasticsearchRestProcessor.QUERY_CLAUSE, "not-json");
runner.setProperty(ElasticsearchRestProcessor.SIZE, "-1");
runner.setProperty(ElasticsearchRestProcessor.AGGREGATIONS, "not-json-aggs");
runner.setProperty(ElasticsearchRestProcessor.SORT, "not-json-sort");
runner.setProperty(ElasticsearchRestProcessor.FIELDS, "not-json-fields");
runner.setProperty(ElasticsearchRestProcessor.SCRIPT_FIELDS, "not-json-script_fields");
final AssertionError assertionError = assertThrows(AssertionError.class, runner::run);
String expected;
if (runner.getProcessor() instanceof ConsumeElasticsearch) {
// ConsumeElasticsearch doesn't use QUERY_CLAUSE
expected = "Processor has 5 validation failures:\n";
} else {
expected = String.format("Processor has 6 validation failures:\n" +
"'%s' validated against 'not-json' is invalid because %s is not a valid JSON representation due to Unrecognized token 'not': was expecting" +
" (JSON String, Number, Array, Object or token 'null', 'true' or 'false')\n" +
" at [Source: (String)\"not-json\"; line: 1, column: 4]\n",
ElasticsearchRestProcessor.QUERY_CLAUSE.getName(), ElasticsearchRestProcessor.QUERY_CLAUSE.getName());
}
expected += String.format("'%s' validated against '-1' is invalid because not a positive value\n" +
"'%s' validated against 'not-json-sort' is invalid because %s is not a valid JSON representation due to Unrecognized token 'not': was expecting" +
" (JSON String, Number, Array, Object or token 'null', 'true' or 'false')\n" +
" at [Source: (String)\"not-json-sort\"; line: 1, column: 4]\n" +
"'%s' validated against 'not-json-aggs' is invalid because %s is not a valid JSON representation due to Unrecognized token 'not': was expecting" +
" (JSON String, Number, Array, Object or token 'null', 'true' or 'false')\n" +
" at [Source: (String)\"not-json-aggs\"; line: 1, column: 4]\n" +
"'%s' validated against 'not-json-fields' is invalid because %s is not a valid JSON representation due to Unrecognized token 'not': was expecting" +
" (JSON String, Number, Array, Object or token 'null', 'true' or 'false')\n" +
" at [Source: (String)\"not-json-fields\"; line: 1, column: 4]\n" +
"'%s' validated against 'not-json-script_fields' is invalid because %s is not a valid JSON representation due to Unrecognized token 'not': was expecting" +
" (JSON String, Number, Array, Object or token 'null', 'true' or 'false')\n" +
" at [Source: (String)\"not-json-script_fields\"; line: 1, column: 4]\n",
ElasticsearchRestProcessor.SIZE.getName(),
ElasticsearchRestProcessor.SORT.getName(), ElasticsearchRestProcessor.SORT.getName(),
ElasticsearchRestProcessor.AGGREGATIONS.getName(), ElasticsearchRestProcessor.AGGREGATIONS.getName(),
ElasticsearchRestProcessor.FIELDS.getName(), ElasticsearchRestProcessor.FIELDS.getName(),
ElasticsearchRestProcessor.SCRIPT_FIELDS.getName(), ElasticsearchRestProcessor.SCRIPT_FIELDS.getName());
assertEquals(expected, assertionError.getMessage());
}
@Test
void testBasicQuery() {
// test hits (no splitting) - full hit format // test hits (no splitting) - full hit format
final TestRunner runner = createRunner(false); final TestRunner runner = createRunner(false);
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllQuery); runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllQuery);
@ -192,7 +289,7 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
} }
@Test @Test
public void testNoHits() { void testNoHits() {
// test no hits (no output) // test no hits (no output)
final TestRunner runner = createRunner(false); final TestRunner runner = createRunner(false);
final TestElasticsearchClientService service = getService(runner); final TestElasticsearchClientService service = getService(runner);
@ -224,14 +321,14 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Test @Test
public void testAggregationsFullFormat() { void testAggregationsFullFormat() {
final TestRunner runner = createRunner(true); final TestRunner runner = createRunner(true);
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllAggregationWithDefaultTermsQuery); runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllAggregationWithDefaultTermsQuery);
runner.setProperty(AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_FORMAT, AggregationResultsFormat.FULL.getValue()); runner.setProperty(AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_FORMAT, AggregationResultsFormat.FULL.getValue());
runOnce(runner); runOnce(runner);
testCounts(runner, isInput() ? 1 : 0, 1, 0, 1); testCounts(runner, isInput() ? 1 : 0, 1, 0, 1);
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "10"); runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "10");
MockFlowFile aggregations = runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_AGGREGATIONS).get(0); final MockFlowFile aggregations = runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_AGGREGATIONS).get(0);
aggregations.assertAttributeNotExists("aggregation.number"); aggregations.assertAttributeNotExists("aggregation.number");
aggregations.assertAttributeNotExists("aggregation.name"); aggregations.assertAttributeNotExists("aggregation.name");
// count == 1 because aggregations is a single Map rather than a List of Maps, even when there are multiple aggs // count == 1 because aggregations is a single Map rather than a List of Maps, even when there are multiple aggs
@ -248,7 +345,7 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Test @Test
public void testWithQueryParameterNoIncomingConnectionAndBucketsAggregationFormat() { void testWithQueryParameterNoIncomingConnectionAndBucketsAggregationFormat() {
final TestRunner runner = createRunner(true); final TestRunner runner = createRunner(true);
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllAggregationWithDefaultTermsQuery); runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllAggregationWithDefaultTermsQuery);
runner.setIncomingConnection(false); runner.setIncomingConnection(false);
@ -273,7 +370,7 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
} }
@Test @Test
public void testSplittingAggregationsMetadataOnlyFormat() { void testSplittingAggregationsMetadataOnlyFormat() {
final TestRunner runner = createRunner(true); final TestRunner runner = createRunner(true);
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllAggregationWithDefaultTermsQuery); runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllAggregationWithDefaultTermsQuery);
runner.setProperty(AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_SPLIT, ResultOutputStrategy.PER_HIT.getValue()); runner.setProperty(AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_SPLIT, ResultOutputStrategy.PER_HIT.getValue());
@ -282,21 +379,21 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
testCounts(runner, isInput() ? 1 : 0, 1, 0, 2); testCounts(runner, isInput() ? 1 : 0, 1, 0, 2);
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "10"); runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "10");
int a = 0; int a = 0;
for (MockFlowFile termAgg : runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_AGGREGATIONS)) { for (final MockFlowFile termAgg : runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_AGGREGATIONS)) {
termAgg.assertAttributeEquals("aggregation.name", a == 0 ? "term_agg" : "term_agg2"); termAgg.assertAttributeEquals("aggregation.name", a == 0 ? "term_agg" : "term_agg2");
termAgg.assertAttributeEquals("aggregation.number", Integer.toString(++a)); termAgg.assertAttributeEquals("aggregation.number", Integer.toString(++a));
assertOutputContent(termAgg.getContent(), 1, false); assertOutputContent(termAgg.getContent(), 1, false);
Map<String, Object> aggContent = JsonUtils.readMap(termAgg.getContent()); final Map<String, Object> aggContent = JsonUtils.readMap(termAgg.getContent());
// agg Map (metadata, no buckets) // agg Map (metadata, no buckets)
assertTrue(aggContent.containsKey("doc_count_error_upper_bound")); assertTrue(aggContent.containsKey("doc_count_error_upper_bound"));
assertFalse(aggContent.containsKey("buckets")); assertFalse(aggContent.containsKey("buckets"));
} }
} }
@Test @Test
public void testAggregationsUsingExpressionLanguage() throws Exception { void testAggregationsUsingExpressionLanguage() throws Exception {
final TestRunner runner = createRunner(true); final TestRunner runner = createRunner(true);
String query = Files.readString(Paths.get(TEST_DIR, "matchAllAggregationWithDefaultTermsInExpressionLanguageQuery.json")); final String query = JsonUtils.readString(Paths.get(TEST_DIR, "matchAllAggregationWithDefaultTermsInExpressionLanguageQuery.json"));
runner.setEnvironmentVariableValue("fieldValue", "msg"); runner.setEnvironmentVariableValue("fieldValue", "msg");
runner.setEnvironmentVariableValue("es.index", INDEX_NAME); runner.setEnvironmentVariableValue("es.index", INDEX_NAME);
runner.setEnvironmentVariableValue("es.type", "msg"); runner.setEnvironmentVariableValue("es.type", "msg");
@ -309,7 +406,7 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
testCounts(runner, isInput() ? 1 : 0, 1, 0, 2); testCounts(runner, isInput() ? 1 : 0, 1, 0, 2);
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "10"); runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "10");
int a = 0; int a = 0;
for (MockFlowFile termAgg : runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_AGGREGATIONS)) { for (final MockFlowFile termAgg : runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_AGGREGATIONS)) {
termAgg.assertAttributeEquals("aggregation.name", a == 0 ? "term_agg" : "term_agg2"); termAgg.assertAttributeEquals("aggregation.name", a == 0 ? "term_agg" : "term_agg2");
termAgg.assertAttributeEquals("aggregation.number", Integer.toString(++a)); termAgg.assertAttributeEquals("aggregation.number", Integer.toString(++a));
assertOutputContent(termAgg.getContent(), 1, false); assertOutputContent(termAgg.getContent(), 1, false);
@ -317,7 +414,7 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
} }
@Test @Test
public void testErrorDuringSearch() { void testErrorDuringSearch() {
final TestRunner runner = createRunner(true); final TestRunner runner = createRunner(true);
getService(runner).setThrowErrorInSearch(true); getService(runner).setThrowErrorInSearch(true);
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllAggregationWithDefaultTermsQuery); runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllAggregationWithDefaultTermsQuery);
@ -325,15 +422,22 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
testCounts(runner, 0, 0, isInput() ? 1 : 0, 0); testCounts(runner, 0, 0, isInput() ? 1 : 0, 0);
} }
@Test @ParameterizedTest
public void testQueryAttribute() { @EnumSource(QueryDefinitionType.class)
String query = matchAllAggregationWithDefaultTermsQuery; void testQueryAttribute(final QueryDefinitionType queryDefinitionType) throws JsonProcessingException {
Assumptions.assumeFalse(QueryDefinitionType.FULL_QUERY.equals(queryDefinitionType) && getProcessor() instanceof ConsumeElasticsearch,
"ConsumeElasticsearch doesn't use the FULL_QUERY definition type");
final String query = matchAllAggregationWithDefaultTermsQuery;
final String queryAttr = "es.query"; final String queryAttr = "es.query";
final TestRunner runner = createRunner(true); final TestRunner runner = createRunner(true);
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, query);
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY_ATTRIBUTE, queryAttr); runner.setProperty(AbstractJsonQueryElasticsearch.QUERY_ATTRIBUTE, queryAttr);
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, queryDefinitionType.getValue());
setQuery(runner, query);
runOnce(runner); runOnce(runner);
testCounts(runner, isInput() ? 1 : 0, 1, 0, 1); testCounts(runner, isInput() ? 1 : 0, 1, 0, 1);
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_AGGREGATIONS); final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_AGGREGATIONS);
flowFiles.addAll(runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS)); flowFiles.addAll(runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS));
@ -341,12 +445,19 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
for (final MockFlowFile mockFlowFile : flowFiles) { for (final MockFlowFile mockFlowFile : flowFiles) {
final String attr = mockFlowFile.getAttribute(queryAttr); final String attr = mockFlowFile.getAttribute(queryAttr);
assertNotNull(attr, "Missing query attribute"); assertNotNull(attr, "Missing query attribute");
assertEquals(query, attr, "Query had wrong value.");
final ObjectNode expected = TEST_MAPPER.readValue(query, ObjectNode.class);
if (runner.getProcessor() instanceof ConsumeElasticsearch) {
// no "query" will be defined with Range Field but no initial value provided
expected.remove("query");
addExpectedSort(expected);
}
assertEquals(expected, TEST_MAPPER.readTree(attr), "Query had wrong value.");
} }
} }
@Test @Test
public void testInputHandling() { void testInputHandling() {
final TestRunner runner = createRunner(false); final TestRunner runner = createRunner(false);
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllQuery); runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllQuery);
@ -361,9 +472,11 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
} }
@Test @Test
public void testRequestParameters() { void testRequestParameters() {
final TestRunner runner = createRunner(false); final TestRunner runner = createRunner(false);
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllQuery); if (!(runner.getProcessor() instanceof ConsumeElasticsearch)) {
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllQuery);
}
runner.setProperty("refresh", "true"); runner.setProperty("refresh", "true");
runner.setProperty("slices", "${slices}"); runner.setProperty("slices", "${slices}");
runner.setEnvironmentVariableValue("slices", "auto"); runner.setEnvironmentVariableValue("slices", "auto");
@ -371,7 +484,7 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
runOnce(runner); runOnce(runner);
final TestElasticsearchClientService service = getService(runner); final TestElasticsearchClientService service = getService(runner);
if (getProcessor() instanceof SearchElasticsearch || getProcessor() instanceof PaginatedJsonQueryElasticsearch) { if (runner.getProcessor() instanceof AbstractPaginatedJsonQueryElasticsearch) {
assertEquals(3, service.getRequestParameters().size()); assertEquals(3, service.getRequestParameters().size());
assertEquals("600s", service.getRequestParameters().get("scroll")); assertEquals("600s", service.getRequestParameters().get("scroll"));
} else { } else {
@ -382,7 +495,142 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
assertEquals("auto", service.getRequestParameters().get("slices")); assertEquals("auto", service.getRequestParameters().get("slices"));
} }
public static void testCounts(TestRunner runner, int original, int hits, int failure, int aggregations) { @ParameterizedTest
@MethodSource
void testQueryBuilder(final String queryClause, final Integer size, final String aggs, final String sort,
final String fields, final String scriptFields, final String expectedQuery) throws Exception {
final TestRunner runner = createRunner(false);
if (runner.getProcessor() instanceof ConsumeElasticsearch) {
// test Range Field and initial value defined
runner.setProperty(ConsumeElasticsearch.RANGE_FIELD, RANGE_FIELD_NAME);
runner.setProperty(ConsumeElasticsearch.RANGE_INITIAL_VALUE, RANGE_FIELD_VALUE);
// as onScheduled won't run (the processor will not actually br triggered), set these fields directly
((ConsumeElasticsearch) runner.getProcessor()).trackingRangeField = RANGE_FIELD_NAME;
((ConsumeElasticsearch) runner.getProcessor()).trackingSortOrder = RANGE_SORT_ORDER;
} else {
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, QueryDefinitionType.BUILD_QUERY.getValue());
}
if (queryClause != null) {
runner.setProperty(ElasticsearchRestProcessor.QUERY_CLAUSE, queryClause);
}
if (size != null) {
runner.setProperty(ElasticsearchRestProcessor.SIZE, String.valueOf(size));
}
if (aggs != null) {
runner.setProperty(ElasticsearchRestProcessor.AGGREGATIONS, aggs);
}
if (sort != null) {
runner.setProperty(ElasticsearchRestProcessor.SORT, sort);
}
if (fields != null) {
runner.setProperty(ElasticsearchRestProcessor.FIELDS, fields);
}
if (scriptFields != null) {
runner.setProperty(ElasticsearchRestProcessor.SCRIPT_FIELDS, scriptFields);
}
@SuppressWarnings("unchecked")
final String query = ((P) runner.getProcessor()).getQuery(null, runner.getProcessContext(), null);
assertNotNull(query);
final ObjectNode expected = TEST_MAPPER.readValue(expectedQuery, ObjectNode.class);
if (runner.getProcessor() instanceof ConsumeElasticsearch) {
// ConsumeElasticsearch should build the "query" based upon the Range Field and initial value
expected.set("query", TEST_MAPPER.readTree(QUERY_CLAUSE));
addExpectedSort(expected);
}
assertEquals(expected, TEST_MAPPER.readTree(query));
}
private void addExpectedSort(final ObjectNode expected) throws JsonProcessingException {
// ConsumeElasticsearch should add the "sort" for the Range field
final ArrayNode expectedSort;
if (expected.has("sort")) {
expectedSort = expected.withArray("sort");
} else {
expectedSort = TEST_MAPPER.getNodeFactory().arrayNode(1);
expected.set("sort", expectedSort);
}
expectedSort.insert(0, TEST_MAPPER.readTree(CONSUME_ELASTICSEARCH_SORT_CLAUSE));
}
private static Stream<Arguments> testQueryBuilder() {
final int size = 123;
final String aggs = "{\"foo_terms\": {\"terms\": {\"field\": \"foo.keyword\"}}}";
final String sort = "[{\"price\" : {\"order\" : \"asc\", \"mode\" : \"avg\"}}, {\"post_date\" : {\"format\": \"strict_date_optional_time_nanos\"}}]";
final String fields = "[\"user.id\", \"http.response.*\", {\"field\": \"@timestamp\", \"format\": \"epoch_millis\"}]";
final String scriptFields = "{\"test1\": {\"script\": {\"lang\": \"painless\", \"source\": \"doc['price'].value * 2\"}}}";
return Stream.of(
Arguments.of(null, null, null, null, null, null, "{}"),
Arguments.of(QUERY_CLAUSE, null, null, null, null, null, String.format("{\"query\": %s}", QUERY_CLAUSE)),
Arguments.of(null, size, null, null, null, null, String.format("{\"size\": %d}", size)),
Arguments.of(null, null, aggs, null, null, null, String.format("{\"aggs\": %s}", aggs)),
Arguments.of(null, null, null, sort, null, null, String.format("{\"sort\": %s}", sort)),
Arguments.of(null, null, null, null, fields, null, String.format("{\"fields\": %s}", fields)),
Arguments.of(null, null, null, null, null, scriptFields, String.format("{\"script_fields\": %s}", scriptFields)),
Arguments.of(QUERY_CLAUSE, size, null, null, null, null, String.format("{\"query\": %s, \"size\": %d}", QUERY_CLAUSE, size)),
Arguments.of(QUERY_CLAUSE, size, aggs, sort, fields, scriptFields,
String.format("{\"query\": %s, \"size\": %d, \"aggs\": %s, \"sort\": %s, \"fields\": %s, \"script_fields\": %s}",
QUERY_CLAUSE, size, aggs, sort, fields, scriptFields)
)
);
}
@Test
void testDefaultQuery() {
final TestRunner runner = createRunner(false);
runner.removeProperty(ElasticsearchRestProcessor.QUERY);
runner.removeProperty(ElasticsearchRestProcessor.QUERY_CLAUSE);
final String expected;
if (runner.getProcessor() instanceof ConsumeElasticsearch) {
// test Range Field defined but no initial value
runner.setProperty(ConsumeElasticsearch.QUERY_DEFINITION_STYLE, QueryDefinitionType.BUILD_QUERY.getValue());
runner.setProperty(ConsumeElasticsearch.RANGE_FIELD, RANGE_FIELD_NAME);
// should be no "query" (with no initial value) but "sort" added
expected = String.format("{\"sort\":[%s]}", CONSUME_ELASTICSEARCH_SORT_CLAUSE);
} else {
expected = "{}";
}
runOnce(runner, "");
assertEquals(expected, getService(runner).getQuery());
}
void setQuery(final TestRunner runner, final String query) throws JsonProcessingException {
if (runner.getProcessor() instanceof ConsumeElasticsearch) {
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, QueryDefinitionType.BUILD_QUERY.getValue());
}
if (QueryDefinitionType.BUILD_QUERY.getValue().equals(runner.getProcessContext().getProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE).getValue())) {
final Map<String, Object> queryMap = TEST_MAPPER.readValue(query, new TypeReference<Map<String, Object>>(){});
if (queryMap.containsKey("query")) {
if (runner.getProcessor() instanceof ConsumeElasticsearch) {
runner.setProperty(ConsumeElasticsearch.RANGE_FIELD, RANGE_FIELD_NAME);
} else {
runner.setProperty(ElasticsearchRestProcessor.QUERY_CLAUSE, TEST_MAPPER.writeValueAsString(queryMap.get("query")));
}
}
if (queryMap.containsKey("size")) {
runner.setProperty(ElasticsearchRestProcessor.SIZE, TEST_MAPPER.writeValueAsString(queryMap.get("size")));
}
if (queryMap.containsKey("aggs")) {
runner.setProperty(ElasticsearchRestProcessor.AGGREGATIONS, TEST_MAPPER.writeValueAsString(queryMap.get("aggs")));
}
if (queryMap.containsKey("sort")) {
runner.setProperty(ElasticsearchRestProcessor.SORT, TEST_MAPPER.writeValueAsString(queryMap.get("sort")));
}
} else {
runner.setProperty(ElasticsearchRestProcessor.QUERY, query);
}
}
static void testCounts(final TestRunner runner, final int original, final int hits, final int failure, final int aggregations) {
runner.assertTransferCount(AbstractJsonQueryElasticsearch.REL_ORIGINAL, original); runner.assertTransferCount(AbstractJsonQueryElasticsearch.REL_ORIGINAL, original);
runner.assertTransferCount(AbstractJsonQueryElasticsearch.REL_HITS, hits); runner.assertTransferCount(AbstractJsonQueryElasticsearch.REL_HITS, hits);
runner.assertTransferCount(AbstractJsonQueryElasticsearch.REL_FAILURE, failure); runner.assertTransferCount(AbstractJsonQueryElasticsearch.REL_FAILURE, failure);
@ -390,7 +638,7 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
runner.assertTransferCount(AbstractJsonQueryElasticsearch.REL_RETRY, 0); runner.assertTransferCount(AbstractJsonQueryElasticsearch.REL_RETRY, 0);
} }
public static void assertOutputContent(final String content, final int count, final boolean ndjson) { static void assertOutputContent(final String content, final int count, final boolean ndjson) {
if (ndjson) { if (ndjson) {
assertEquals(count, (content.split("\n").length)); assertEquals(count, (content.split("\n").length));
} else { } else {
@ -404,13 +652,13 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
} }
} }
public TestRunner createRunner(final boolean returnAggs) { TestRunner createRunner(final boolean returnAggs) {
final P processor = getProcessor(); final P processor = getProcessor();
final TestRunner runner = TestRunners.newTestRunner(processor); final TestRunner runner = TestRunners.newTestRunner(processor);
final TestElasticsearchClientService service = new TestElasticsearchClientService(returnAggs); final TestElasticsearchClientService service = new TestElasticsearchClientService(returnAggs);
try { try {
runner.addControllerService("esService", service); runner.addControllerService("esService", service);
} catch (InitializationException e) { } catch (final InitializationException e) {
throw new RuntimeException(e); throw new RuntimeException(e);
} }
runner.enableControllerService(service); runner.enableControllerService(service);
@ -422,11 +670,15 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
return runner; return runner;
} }
public MockFlowFile runOnce(final TestRunner runner) { MockFlowFile runOnce(final TestRunner runner) {
return runOnce(runner, "test");
}
MockFlowFile runOnce(final TestRunner runner, final String data) {
final MockFlowFile ff; final MockFlowFile ff;
if (isInput()) { if (isInput()) {
runner.setIncomingConnection(true); runner.setIncomingConnection(true);
ff = runner.enqueue("test"); ff = runner.enqueue(data);
} else { } else {
runner.setIncomingConnection(false); runner.setIncomingConnection(false);
ff = null; ff = null;
@ -436,17 +688,17 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
return ff; return ff;
} }
public static TestElasticsearchClientService getService(final TestRunner runner) { static TestElasticsearchClientService getService(final TestRunner runner) {
return runner.getControllerService("esService", TestElasticsearchClientService.class); return runner.getControllerService("esService", TestElasticsearchClientService.class);
} }
public void reset(final TestRunner runner) { void reset(final TestRunner runner) {
runner.clearProvenanceEvents(); runner.clearProvenanceEvents();
runner.clearTransferState(); runner.clearTransferState();
if (isStateUsed()) { if (getStateScope() != null) {
try { try {
runner.getStateManager().clear(Scope.LOCAL); runner.getStateManager().clear(getStateScope());
} catch (IOException e) { } catch (final IOException e) {
throw new RuntimeException(e); throw new RuntimeException(e);
} }
} }

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.nifi.processors.elasticsearch; package org.apache.nifi.processors.elasticsearch;
import com.fasterxml.jackson.core.JsonProcessingException;
import org.apache.nifi.processors.elasticsearch.api.AggregationResultsFormat; import org.apache.nifi.processors.elasticsearch.api.AggregationResultsFormat;
import org.apache.nifi.processors.elasticsearch.api.PaginationType; import org.apache.nifi.processors.elasticsearch.api.PaginationType;
import org.apache.nifi.processors.elasticsearch.api.ResultOutputStrategy; import org.apache.nifi.processors.elasticsearch.api.ResultOutputStrategy;
@ -25,8 +26,9 @@ import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunner;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.EnumSource;
import java.nio.file.Files;
import java.nio.file.Paths; import java.nio.file.Paths;
import java.util.Map; import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -38,30 +40,28 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
public abstract class AbstractPaginatedJsonQueryElasticsearchTest extends AbstractJsonQueryElasticsearchTest<AbstractPaginatedJsonQueryElasticsearch> { public abstract class AbstractPaginatedJsonQueryElasticsearchTest extends AbstractJsonQueryElasticsearchTest<AbstractPaginatedJsonQueryElasticsearch> {
protected static String matchAllWithSortByMsgWithSizeQuery; protected static String matchAllWithSortByMsgWithSizeQuery;
private static final String TEST_DIR = "src/test/resources/AbstractPaginatedJsonQueryElasticsearchTest"; private static final String TEST_DIR = "src/test/resources/AbstractPaginatedJsonQueryElasticsearchTest";
private static String matchAllWithSortByMessage; private static String matchAllWithSortByMessage;
private static String matchAllWithSortByMsgWithoutSize; private static String matchAllWithSortByMsgWithoutSize;
@BeforeAll @BeforeAll
public static void setUpBeforeClass() throws Exception { public static void setUpBeforeClass() throws Exception {
AbstractJsonQueryElasticsearchTest.setUpBeforeClass(); AbstractJsonQueryElasticsearchTest.setUpBeforeClass();
matchAllWithSortByMessage = Files.readString(Paths.get(TEST_DIR, "matchAllWithSortByMessageQuery.json")); matchAllWithSortByMessage = JsonUtils.readString(Paths.get(TEST_DIR, "matchAllWithSortByMessageQuery.json"));
matchAllWithSortByMsgWithoutSize = Files.readString(Paths.get(TEST_DIR,"matchAllWithSortByMsgQueryWithoutSize.json")); matchAllWithSortByMsgWithoutSize = JsonUtils.readString(Paths.get(TEST_DIR,"matchAllWithSortByMsgQueryWithoutSize.json"));
matchAllWithSortByMsgWithSizeQuery = Files.readString(Paths.get(TEST_DIR, "matchAllWithSortByMsgQueryWithSize.json")); matchAllWithSortByMsgWithSizeQuery = JsonUtils.readString(Paths.get(TEST_DIR, "matchAllWithSortByMsgQueryWithSize.json"));
} }
public abstract boolean isInput();
@Test @Test
public void testInvalidPaginationProperties() { void testInvalidPaginationProperties() {
final TestRunner runner = createRunner(false); final TestRunner runner = createRunner(false);
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllQuery); runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllQuery);
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_KEEP_ALIVE, "not-a-period"); runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_KEEP_ALIVE, "not-a-period");
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, "not-enum"); runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, "not-enum");
final AssertionError assertionError = assertThrows(AssertionError.class, runner::run); final AssertionError assertionError = assertThrows(AssertionError.class, runner::run);
String expected = String.format("Processor has 2 validation failures:\n" + final String expected = String.format("Processor has 2 validation failures:\n" +
"'%s' validated against 'not-enum' is invalid because Given value not found in allowed set '%s'\n" + "'%s' validated against 'not-enum' is invalid because Given value not found in allowed set '%s'\n" +
"'%s' validated against 'not-a-period' is invalid because Must be of format <duration> <TimeUnit> where <duration> " + "'%s' validated against 'not-a-period' is invalid because Must be of format <duration> <TimeUnit> where <duration> " +
"is a non-negative integer and TimeUnit is a supported Time Unit, such as: nanos, millis, secs, mins, hrs, days\n", "is a non-negative integer and TimeUnit is a supported Time Unit, such as: nanos, millis, secs, mins, hrs, days\n",
@ -72,7 +72,7 @@ public abstract class AbstractPaginatedJsonQueryElasticsearchTest extends Abstra
} }
@Test @Test
public void testSinglePage() { void testSinglePage() {
// paged query hits (no splitting) // paged query hits (no splitting)
final TestRunner runner = createRunner(false); final TestRunner runner = createRunner(false);
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllQuery); runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllQuery);
@ -115,8 +115,7 @@ public abstract class AbstractPaginatedJsonQueryElasticsearchTest extends Abstra
assertSendEvent(runner, input); assertSendEvent(runner, input);
} }
static void assertFormattedResult(final SearchResultsFormat searchResultsFormat, final Map<String, Object> hit) {
public static void assertFormattedResult(final SearchResultsFormat searchResultsFormat, final Map<String, Object> hit) {
assertFalse(hit.isEmpty()); assertFalse(hit.isEmpty());
switch (searchResultsFormat) { switch (searchResultsFormat) {
case SOURCE_ONLY: case SOURCE_ONLY:
@ -137,8 +136,8 @@ public abstract class AbstractPaginatedJsonQueryElasticsearchTest extends Abstra
} }
private void assertResultsFormat(final TestRunner runner, final ResultOutputStrategy resultOutputStrategy, final SearchResultsFormat searchResultsFormat) { private void assertResultsFormat(final TestRunner runner, final ResultOutputStrategy resultOutputStrategy, final SearchResultsFormat searchResultsFormat) {
int flowFileCount; final int flowFileCount;
String hitsCount; final String hitsCount;
boolean ndjson = false; boolean ndjson = false;
switch (resultOutputStrategy) { switch (resultOutputStrategy) {
@ -179,31 +178,30 @@ public abstract class AbstractPaginatedJsonQueryElasticsearchTest extends Abstra
}); });
} }
@Test @ParameterizedTest
public void testResultsFormat() { @EnumSource(ResultOutputStrategy.class)
for (final ResultOutputStrategy resultOutputStrategy : ResultOutputStrategy.values()) { void testResultsFormat(final ResultOutputStrategy resultOutputStrategy) throws JsonProcessingException {
final TestRunner runner = createRunner(false); final TestRunner runner = createRunner(false);
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllWithSortByMessage); setQuery(runner, matchAllWithSortByMessage);
runner.setProperty(AbstractJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT, resultOutputStrategy.getValue()); runner.setProperty(AbstractJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT, resultOutputStrategy.getValue());
// Test against each results format // Test against each result format
for (final SearchResultsFormat searchResultsFormat : SearchResultsFormat.values()) { for (final SearchResultsFormat searchResultsFormat : SearchResultsFormat.values()) {
runner.setProperty(AbstractJsonQueryElasticsearch.SEARCH_RESULTS_FORMAT, searchResultsFormat.getValue()); runner.setProperty(AbstractJsonQueryElasticsearch.SEARCH_RESULTS_FORMAT, searchResultsFormat.getValue());
// Test against each pagination type // Test against each pagination type
for (final PaginationType paginationType : PaginationType.values()) { for (final PaginationType paginationType : PaginationType.values()) {
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, paginationType.getValue()); runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, paginationType.getValue());
runOnce(runner); runOnce(runner);
assertResultsFormat(runner, resultOutputStrategy, searchResultsFormat); assertResultsFormat(runner, resultOutputStrategy, searchResultsFormat);
reset(runner); reset(runner);
}
} }
} }
} }
@Test @Test
public void testScrollError() { void testDeleteScrollError() {
final TestRunner runner = createRunner(false); final TestRunner runner = createRunner(false);
final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner); final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner);
service.setThrowErrorInDelete(true); service.setThrowErrorInDelete(true);
@ -219,14 +217,39 @@ public abstract class AbstractPaginatedJsonQueryElasticsearchTest extends Abstra
} }
@Test @Test
public void testDeletePitError() { void testScrollError() {
final TestRunner runner = createRunner(false);
final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner);
service.setMaxPages(2);
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, PaginationType.SCROLL.getValue());
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllWithSortByMsgWithSizeQuery);
if (getStateScope() != null) {
// initialize search for SearchElasticsearch, first page successful
service.setThrowErrorInSearch(false);
runOnce(runner);
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 1, 0, 0);
runner.clearTransferState();
}
// scroll (error)
service.setThrowErrorInSearch(true);
runOnce(runner);
// fir PaginatedJsonQueryElasticsearch, the input flowfile will be routed to failure, for SearchElasticsearch, there will be no output
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 0, getStateScope() == null ? 1 : 0, 0);
assertTrue(runner.getLogger().getErrorMessages().stream().anyMatch(logMessage ->
logMessage.getMsg().contains("Could not query documents") && logMessage.getThrowable().getMessage().contains("Simulated IOException")));
}
@Test
void testDeletePitError() throws JsonProcessingException {
final TestRunner runner = createRunner(false); final TestRunner runner = createRunner(false);
final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner); final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner);
service.setThrowErrorInDelete(true); service.setThrowErrorInDelete(true);
runner.setProperty(AbstractJsonQueryElasticsearch.SEARCH_RESULTS_FORMAT, SearchResultsFormat.FULL.getValue()); runner.setProperty(AbstractJsonQueryElasticsearch.SEARCH_RESULTS_FORMAT, SearchResultsFormat.FULL.getValue());
runner.setProperty(AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_FORMAT, AggregationResultsFormat.FULL.getValue()); runner.setProperty(AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_FORMAT, AggregationResultsFormat.FULL.getValue());
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, PaginationType.POINT_IN_TIME.getValue()); runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, PaginationType.POINT_IN_TIME.getValue());
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllWithSortByMsgWithoutSize); setQuery(runner, matchAllWithSortByMsgWithoutSize);
// still expect "success" output for exception during final clean-up // still expect "success" output for exception during final clean-up
runMultiple(runner); runMultiple(runner);
@ -237,12 +260,12 @@ public abstract class AbstractPaginatedJsonQueryElasticsearchTest extends Abstra
} }
@Test @Test
public void testInitialisePitError() { void testInitialisePitError() throws JsonProcessingException {
final TestRunner runner = createRunner(false); final TestRunner runner = createRunner(false);
final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner); final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner);
service.setThrowErrorInPit(true); service.setThrowErrorInPit(true);
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, PaginationType.POINT_IN_TIME.getValue()); runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, PaginationType.POINT_IN_TIME.getValue());
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllWithSortByMsgWithoutSize); setQuery(runner, matchAllWithSortByMsgWithoutSize);
// expect "failure" output for exception during query setup // expect "failure" output for exception during query setup
runOnce(runner); runOnce(runner);
@ -252,77 +275,62 @@ public abstract class AbstractPaginatedJsonQueryElasticsearchTest extends Abstra
logMessage.getMsg().contains("Could not query documents") && logMessage.getThrowable().getMessage().contains("Simulated IOException - initialisePointInTime"))); logMessage.getMsg().contains("Could not query documents") && logMessage.getThrowable().getMessage().contains("Simulated IOException - initialisePointInTime")));
} }
@Test @ParameterizedTest
public void testQuerySortError() { @EnumSource(PaginationType.class)
// test PiT without sort void testPaginatedQueryWithoutSort(final PaginationType paginationType) throws JsonProcessingException {
final TestRunner runner = createRunner(false); final TestRunner runner = createRunner(false);
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, PaginationType.POINT_IN_TIME.getValue()); runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, paginationType.getValue());
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllQuery); setQuery(runner, matchAllQuery);
// expect "failure" output for exception during query setup if (PaginationType.SCROLL == paginationType) {
runOnce(runner); // test scroll without sort (should succeed)
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 0, isInput() ? 1 : 0, 0); runMultiple(runner);
AbstractJsonQueryElasticsearchTest.testCounts(runner, isInput() ? 1 : 0, 1, 0, 0);
} else {
// test PiT/search_after without sort
runOnce(runner);
if (runner.getProcessor() instanceof ConsumeElasticsearch) {
// sort is added based upon the Range Field (which cannot be empty)
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 1, 0, 0);
} else {
// expect "failure" output for exception during query setup
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 0, isInput() ? 1 : 0, 0);
// check error was caught and logged // check error was caught and logged
assertTrue(runner.getLogger().getErrorMessages().stream().anyMatch(logMessage -> assertTrue(runner.getLogger().getErrorMessages().stream().anyMatch(logMessage ->
logMessage.getMsg().contains("Could not query documents") && logMessage.getThrowable().getMessage().equals("Query using pit/search_after must contain a \"sort\" field"))); logMessage.getMsg().contains("Could not query documents") && logMessage.getThrowable().getMessage().equals("Query using pit/search_after must contain a \"sort\" field")));
reset(runner);
// test search_after without sort
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, PaginationType.SEARCH_AFTER.getValue());
runOnce(runner);
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 0, isInput() ? 1 : 0, 0);
assertTrue(runner.getLogger().getErrorMessages().stream().anyMatch(logMessage ->
logMessage.getMsg().contains("Could not query documents") && logMessage.getThrowable().getMessage().equals("Query using pit/search_after must contain a \"sort\" field")));
reset(runner);
// test scroll without sort (should succeed)
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, PaginationType.SCROLL.getValue());
runMultiple(runner);
AbstractJsonQueryElasticsearchTest.testCounts(runner, isInput() ? 1 : 0, 1, 0, 0);
}
@Test
public void testScroll() throws Exception {
testPagination(PaginationType.SCROLL);
}
@Test
public void testPit() throws Exception {
testPagination(PaginationType.POINT_IN_TIME);
}
@Test
public void testSearchAfter() throws Exception {
testPagination(PaginationType.SEARCH_AFTER);
}
private void testPagination(final PaginationType paginationType) throws Exception {
final TestRunner runner = createRunner(false);
final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner);
service.setMaxPages(2);
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, paginationType.getValue());
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllWithSortByMsgWithSizeQuery);
// Tests flowfile per page, hits splitting and hits combined
for (final ResultOutputStrategy resultOutputStrategy : ResultOutputStrategy.values()) {
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT, resultOutputStrategy.getValue());
for (int iteration = 1; iteration < 4; iteration++) {
// Check that changing OUTPUT_NO_HITS doesn't have any adverse effects on pagination
runner.setProperty(AbstractJsonQueryElasticsearch.OUTPUT_NO_HITS, String.valueOf(iteration % 2 > 0).toLowerCase());
runOnce(runner);
validatePagination(runner, resultOutputStrategy, paginationType, iteration);
if (ResultOutputStrategy.PER_QUERY.equals(resultOutputStrategy)) {
break;
}
runner.clearTransferState();
if (!isStateUsed()) {
// reset PaginatedJsonQueryElasticsearch to re-run the query with different OUTPUT_NO_HITS setting
reset(runner);
}
}
reset(runner);
} }
}
}
@ParameterizedTest
@EnumSource(PaginationType.class)
void testPagination(final PaginationType paginationType) throws Exception {
final TestRunner runner = createRunner(false);
final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner);
service.setMaxPages(2);
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, paginationType.getValue());
setQuery(runner, matchAllWithSortByMsgWithSizeQuery);
// Tests flowfile per page, hits splitting and hits combined
for (final ResultOutputStrategy resultOutputStrategy : ResultOutputStrategy.values()) {
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT, resultOutputStrategy.getValue());
for (int iteration = 1; iteration < 4; iteration++) {
// Check that changing OUTPUT_NO_HITS doesn't have any adverse effects on pagination
runner.setProperty(AbstractJsonQueryElasticsearch.OUTPUT_NO_HITS, String.valueOf(iteration % 2 > 0).toLowerCase());
runOnce(runner);
validatePagination(runner, resultOutputStrategy, paginationType, iteration);
if (ResultOutputStrategy.PER_QUERY.equals(resultOutputStrategy)) {
break;
}
runner.clearTransferState();
if (getStateScope() == null) {
// reset PaginatedJsonQueryElasticsearch to re-run the query with different OUTPUT_NO_HITS setting
reset(runner);
}
}
reset(runner);
}
} }
abstract void validatePagination(final TestRunner runner, final ResultOutputStrategy resultOutputStrategy, final PaginationType paginationType, int iteration) throws Exception; abstract void validatePagination(final TestRunner runner, final ResultOutputStrategy resultOutputStrategy, final PaginationType paginationType, int iteration) throws Exception;
@ -348,28 +356,26 @@ public abstract class AbstractPaginatedJsonQueryElasticsearchTest extends Abstra
} }
} }
@Test @ParameterizedTest
public void testEmptyHitsFlowFileIsProducedForEachResultSplitSetup() { @EnumSource(PaginationType.class)
void testEmptyHitsFlowFileIsProducedForEachResultSplitSetup(final PaginationType paginationType) throws JsonProcessingException {
final TestRunner runner = createRunner(false); final TestRunner runner = createRunner(false);
final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner); final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner);
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllWithSortByMessage);
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.OUTPUT_NO_HITS, "true");
service.setMaxPages(0); service.setMaxPages(0);
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, paginationType.getValue());
setQuery(runner, matchAllWithSortByMessage);
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.OUTPUT_NO_HITS, "true");
for (final PaginationType paginationType : PaginationType.values()) { for (final ResultOutputStrategy resultOutputStrategy : ResultOutputStrategy.values()) {
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, paginationType.getValue()); // test that an empty flow file is produced for a per query setup
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT, resultOutputStrategy.getValue());
runOnce(runner);
AbstractJsonQueryElasticsearchTest.testCounts(runner, isInput() ? 1 : 0, 1, 0, 0);
for (final ResultOutputStrategy resultOutputStrategy : ResultOutputStrategy.values()) { runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "0");
// test that an empty flow file is produced for a per query setup runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("page.number", "1");
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT, resultOutputStrategy.getValue()); assertEquals(0, runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).getSize());
runOnce(runner); reset(runner);
AbstractJsonQueryElasticsearchTest.testCounts(runner, isInput() ? 1 : 0, 1, 0, 0);
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "0");
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("page.number", "1");
assertEquals(0, runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).getSize());
reset(runner);
}
} }
} }
} }

View File

@ -39,7 +39,7 @@ public abstract class AbstractPutElasticsearchTest<P extends AbstractPutElastics
assertFalse(runner.getProcessor().getRelationships().contains(AbstractPutElasticsearch.REL_ERROR_RESPONSES)); assertFalse(runner.getProcessor().getRelationships().contains(AbstractPutElasticsearch.REL_ERROR_RESPONSES));
} }
protected String getUnexpectedCountMsg(String countName) { protected String getUnexpectedCountMsg(final String countName) {
return "Did not get expected " + countName + " count"; return "Did not get expected " + countName + " count";
} }
} }

View File

@ -0,0 +1,336 @@
/*
* 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 com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.core.type.TypeReference;
import com.jayway.jsonpath.JsonPath;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.elasticsearch.SearchResponse;
import org.apache.nifi.processors.elasticsearch.api.PaginatedJsonQueryParameters;
import org.apache.nifi.util.StringUtils;
import org.apache.nifi.util.TestRunner;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.CsvSource;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertEquals;
public class ConsumeElasticsearchTest extends SearchElasticsearchTest {
private static final String DEFAULT_QUERY_SORT_CLAUSE_ONLY = String.format("{\"sort\":[%s]}", CONSUME_ELASTICSEARCH_SORT_CLAUSE);
private static final String DEFAULT_QUERY_FILTERS_ONLY = "{\"query\":{\"bool\":{\"filter\":[]}}}";
private static final String DEFAULT_RANGE_FILTER = String.format("{\"range\":{\"%s\":{\"gt\":\"%s\"}}}", RANGE_FIELD_NAME, RANGE_FIELD_VALUE);
private static final List<Map<String, Object>> FIRST_PAGE_OF_HITS = Arrays.asList(
createHit(0),
createHit(1),
createHit(2),
createHit(3),
createHit(4)
);
private static final List<Map<String, Object>> SECOND_PAGE_OF_HITS = Arrays.asList(
createHit(5),
createHit(6),
createHit(7),
createHit(8),
createHit(9)
);
private static Map<String, Object> createHit(final int value) {
return Collections.singletonMap("_source", Collections.singletonMap(RANGE_FIELD_NAME, value));
}
@BeforeAll
public static void setUpBeforeClass() throws Exception {
AbstractPaginatedJsonQueryElasticsearchTest.setUpBeforeClass();
}
AbstractPaginatedJsonQueryElasticsearch getProcessor() {
return new ConsumeElasticsearch();
}
Scope getStateScope() {
return Scope.CLUSTER;
}
private TestRunner runner;
@BeforeEach
void setUp() {
runner = createRunner(false);
}
TestRunner createRunner(final boolean returnAggs) {
final TestRunner runner = super.createRunner(returnAggs);
// onScheduled method isn't always triggered (because the processor isn't always executed through the TestRunner)
// so set the trackingRange fields directly as well as in the ProcessContext
runner.setProperty(ConsumeElasticsearch.RANGE_FIELD, RANGE_FIELD_NAME);
((ConsumeElasticsearch) runner.getProcessor()).trackingRangeField = RANGE_FIELD_NAME;
setTrackingSortOrder(runner, RANGE_SORT_ORDER);
return runner;
}
void setTrackingSortOrder(final TestRunner runner, final String sortOrder) {
runner.setProperty(ConsumeElasticsearch.RANGE_FIELD_SORT_ORDER, sortOrder);
((ConsumeElasticsearch) runner.getProcessor()).trackingSortOrder = sortOrder;
}
@ParameterizedTest
@CsvSource({",", "123456,", ",654321", "123456,654321"})
void testRangeValue(final String initialValue, final String stateValue) throws IOException {
if (StringUtils.isNotBlank(initialValue)) {
runner.setProperty(ConsumeElasticsearch.RANGE_INITIAL_VALUE, initialValue);
}
if (StringUtils.isNotBlank(stateValue)) {
runner.getStateManager().setState(Collections.singletonMap(ConsumeElasticsearch.STATE_RANGE_VALUE, stateValue), getStateScope());
}
runOnce(runner);
final String query = getService(runner).getQuery();
if (StringUtils.isBlank(initialValue) && StringUtils.isBlank(stateValue)) {
// no "query" if no initial/existing state value, but "sort" is always present
assertEquals(DEFAULT_QUERY_SORT_CLAUSE_ONLY, query);
} else {
// existing state value should override any initial value from processor properties
final String rangeValue = JsonPath.read(query, String.format("$.query.bool.filter[0].range.%s.gt", RANGE_FIELD_NAME));
assertEquals(StringUtils.isNotBlank(stateValue) ? stateValue : initialValue, rangeValue);
}
}
@ParameterizedTest
@CsvSource({"asc,0,true", "asc,0,false", "asc,1,true", "asc,1,false",
"desc,0,true", "desc,0,false", "desc,1,true", "desc,1,false"})
void testTrackingValueFromHits(final String sortOrder, final int pageCount, final boolean hasHits) {
setTrackingSortOrder(runner, sortOrder);
final List<Map<String, Object>> hits;
if (!hasHits) {
hits = Collections.emptyList();
} else {
hits = pageCount == 0 ? FIRST_PAGE_OF_HITS : SECOND_PAGE_OF_HITS;
}
final SearchResponse response = new SearchResponse(hits, null, null, null, null, hits.size(), 1, false, null);
final String defaultUnset = "unset";
final PaginatedJsonQueryParameters paginatedJsonQueryParameters = new PaginatedJsonQueryParameters();
paginatedJsonQueryParameters.setKeepAlive("10S");
paginatedJsonQueryParameters.setPageCount(pageCount);
paginatedJsonQueryParameters.setTrackingRangeValue(defaultUnset);
((ConsumeElasticsearch) runner.getProcessor()).updateQueryParameters(paginatedJsonQueryParameters, response);
if ("asc".equals(sortOrder)) {
if (hasHits) {
final List<Map<String, Object>> expectedHits = pageCount == 0 ? FIRST_PAGE_OF_HITS : SECOND_PAGE_OF_HITS;
assertEquals(getHitValue(expectedHits.get(expectedHits.size() - 1)), paginatedJsonQueryParameters.getTrackingRangeValue());
} else {
assertEquals(defaultUnset, paginatedJsonQueryParameters.getTrackingRangeValue());
}
} else {
if (pageCount == 0) {
if (hasHits) {
assertEquals(getHitValue(FIRST_PAGE_OF_HITS.get(0)), paginatedJsonQueryParameters.getTrackingRangeValue());
} else {
assertEquals(defaultUnset, paginatedJsonQueryParameters.getTrackingRangeValue());
}
} else {
assertEquals(defaultUnset, paginatedJsonQueryParameters.getTrackingRangeValue());
}
}
}
@SuppressWarnings("unchecked")
private String getHitValue(final Map<String, Object> hit) {
return String.valueOf(((Map<String, Object>) hit.get("_source")).get(RANGE_FIELD_NAME));
}
@Test
void testNoSorts() throws IOException {
final TestRunner runner = createRunner(false);
runner.removeProperty(ElasticsearchRestProcessor.SORT);
final Map<String, Object> query = new HashMap<>();
((ConsumeElasticsearch) runner.getProcessor()).addSortClause(query, null, runner.getProcessContext());
assertEquals(getDefaultQuerySortOnly(), query);
}
@Test
void testSingleAdditionalSort() throws IOException {
final TestRunner runner = createRunner(false);
runner.setProperty(ElasticsearchRestProcessor.SORT, "{\"foo\":\"bar\"}");
final Map<String, Object> query = new HashMap<>();
((ConsumeElasticsearch) runner.getProcessor()).addSortClause(query, null, runner.getProcessContext());
final Map<String, List<Map<String, Object>>> expected = getDefaultQuerySortOnly();
addExpectedSortClause(expected, Collections.singletonMap("foo", "bar"));
assertEquals(expected, query);
}
@Test
void testMultipleAdditionalSorts() throws IOException {
final TestRunner runner = createRunner(false);
runner.setProperty(ElasticsearchRestProcessor.SORT, "[{\"foo\":\"bar\"},{\"baz\":\"biz\"}]");
final Map<String, Object> query = new HashMap<>();
((ConsumeElasticsearch) runner.getProcessor()).addSortClause(query, null, runner.getProcessContext());
final Map<String, List<Map<String, Object>>> expected = getDefaultQuerySortOnly();
addExpectedSortClause(expected, Collections.singletonMap("foo", "bar"));
addExpectedSortClause(expected, Collections.singletonMap("baz", "biz"));
assertEquals(expected, query);
}
@Test
void testTrackingFieldSortAlreadyPresent() throws IOException {
final TestRunner runner = createRunner(false);
final String existingRangeFieldSort = String.format("{\"%s\":\"bar\"}", RANGE_FIELD_NAME);
runner.setProperty(ElasticsearchRestProcessor.SORT, existingRangeFieldSort);
final Map<String, Object> query = new HashMap<>();
((ConsumeElasticsearch) runner.getProcessor()).addSortClause(query, null, runner.getProcessContext());
final Map<String, Object> expected = TEST_MAPPER.readValue(String.format("{\"sort\":[%s]}", existingRangeFieldSort), new TypeReference<Map<String, Object>>() {});
assertEquals(expected, query);
}
private Map<String, List<Map<String, Object>>> getDefaultQuerySortOnly() throws JsonProcessingException {
return TEST_MAPPER.readValue(DEFAULT_QUERY_SORT_CLAUSE_ONLY, new TypeReference<Map<String, List<Map<String, Object>>>>() {});
}
private void addExpectedSortClause(final Map<String, List<Map<String, Object>>> expectedQuery, final Map<String, Object> expectedSortClause) {
expectedQuery.get("sort").add(expectedSortClause);
}
@Test
void testSingleAdditionalFilterNoInitialRangeValue() throws IOException {
final TestRunner runner = createRunner(false);
runner.removeProperty(ConsumeElasticsearch.RANGE_INITIAL_VALUE);
runner.setProperty(ConsumeElasticsearch.ADDITIONAL_FILTERS, "{\"foo\":\"bar\"}");
final Map<String, Object> query = new HashMap<>();
((ConsumeElasticsearch) runner.getProcessor()).addQueryClause(query, null, runner.getProcessContext());
final Map<String, Map<String, Map<String, List<Map<String, Object>>>>> expected = getDefaultQueryFiltersOnly();
addExpectedFilterClause(expected, Collections.singletonMap("foo", "bar"));
assertEquals(expected, query);
}
@Test
void testSingleAdditionalFilterWithInitialRangeValue() throws IOException {
final TestRunner runner = createRunner(false);
runner.setProperty(ConsumeElasticsearch.RANGE_INITIAL_VALUE, RANGE_FIELD_VALUE);
runner.setProperty(ConsumeElasticsearch.ADDITIONAL_FILTERS, "{\"foo\":\"bar\"}");
final Map<String, Object> query = new HashMap<>();
((ConsumeElasticsearch) runner.getProcessor()).addQueryClause(query, null, runner.getProcessContext());
final Map<String, Map<String, Map<String, List<Map<String, Object>>>>> expected = getDefaultQueryFiltersOnly();
addExpectedFilterClause(expected, getDefaultRangeFilterClause());
addExpectedFilterClause(expected, Collections.singletonMap("foo", "bar"));
assertEquals(expected, query);
}
@Test
void testMultipleAdditionalFilters() throws IOException {
final TestRunner runner = createRunner(false);
runner.setProperty(ConsumeElasticsearch.RANGE_INITIAL_VALUE, RANGE_FIELD_VALUE);
runner.setProperty(ConsumeElasticsearch.ADDITIONAL_FILTERS, "[{\"foo\":\"bar\"},{\"biz\":\"baz\"}]");
final Map<String, Object> query = new HashMap<>();
((ConsumeElasticsearch) runner.getProcessor()).addQueryClause(query, null, runner.getProcessContext());
final Map<String, Map<String, Map<String, List<Map<String, Object>>>>> expected = getDefaultQueryFiltersOnly();
addExpectedFilterClause(expected, getDefaultRangeFilterClause());
addExpectedFilterClause(expected, Collections.singletonMap("foo", "bar"));
addExpectedFilterClause(expected, Collections.singletonMap("biz", "baz"));
assertEquals(expected, query);
}
@Test
void testRangeDateFormat() throws IOException {
final TestRunner runner = createRunner(false);
runner.setProperty(ConsumeElasticsearch.RANGE_INITIAL_VALUE, RANGE_FIELD_VALUE);
runner.setProperty(ConsumeElasticsearch.RANGE_DATE_FORMAT, "epoch-millis");
final Map<String, Object> query = new HashMap<>();
((ConsumeElasticsearch) runner.getProcessor()).addQueryClause(query, null, runner.getProcessContext());
final Map<String, Object> rangeFilterClause = getDefaultRangeFilterClause();
addExpectedRangeFilterClauseField(rangeFilterClause, "format", "epoch-millis");
final Map<String, Map<String, Map<String, List<Map<String, Object>>>>> expected = getDefaultQueryFiltersOnly();
addExpectedFilterClause(expected, rangeFilterClause);
assertEquals(expected, query);
}
@Test
void testRangeTimezone() throws IOException {
final TestRunner runner = createRunner(false);
runner.setProperty(ConsumeElasticsearch.RANGE_INITIAL_VALUE, RANGE_FIELD_VALUE);
runner.setProperty(ConsumeElasticsearch.RANGE_TIME_ZONE, "Europe/London");
final Map<String, Object> query = new HashMap<>();
((ConsumeElasticsearch) runner.getProcessor()).addQueryClause(query, null, runner.getProcessContext());
final Map<String, Object> rangeFilterClause = getDefaultRangeFilterClause();
addExpectedRangeFilterClauseField(rangeFilterClause, "time_zone", "Europe/London");
final Map<String, Map<String, Map<String, List<Map<String, Object>>>>> expected = getDefaultQueryFiltersOnly();
addExpectedFilterClause(expected, rangeFilterClause);
assertEquals(expected, query);
}
private Map<String, Map<String, Map<String, List<Map<String, Object>>>>> getDefaultQueryFiltersOnly() throws JsonProcessingException {
return TEST_MAPPER.readValue(DEFAULT_QUERY_FILTERS_ONLY, new TypeReference<Map<String, Map<String, Map<String, List<Map<String, Object>>>>>>() {});
}
private void addExpectedFilterClause(final Map<String, Map<String, Map<String, List<Map<String, Object>>>>> expectedQuery, final Map<String, Object> expectedFilterClause) {
expectedQuery.get("query").get("bool").get("filter").add(expectedFilterClause);
}
private Map<String, Object> getDefaultRangeFilterClause() throws JsonProcessingException {
return TEST_MAPPER.readValue(DEFAULT_RANGE_FILTER, new TypeReference<Map<String, Object>>() {});
}
@SuppressWarnings("unchecked")
private void addExpectedRangeFilterClauseField(final Map<String, Object> filterClause, final String fieldName, final Object fieldValue) {
((Map<String, Map<String, Object>>) filterClause.get("range")).get(RANGE_FIELD_NAME).put(fieldName, fieldValue);
}
}

View File

@ -33,8 +33,8 @@ public class DeleteByQueryElasticsearchTest extends AbstractByQueryElasticsearch
} }
@Override @Override
public Class<? extends AbstractByQueryElasticsearch> getTestProcessor() { public AbstractByQueryElasticsearch getTestProcessor() {
return DeleteByQueryElasticsearch.class; return new DeleteByQueryElasticsearch();
} }
@Override @Override

View File

@ -59,7 +59,7 @@ public class GetElasticsearchTest {
runner.removeProperty(GetElasticsearch.ATTRIBUTE_NAME); runner.removeProperty(GetElasticsearch.ATTRIBUTE_NAME);
final AssertionError assertionError = assertThrows(AssertionError.class, () -> runner.run()); final AssertionError assertionError = assertThrows(AssertionError.class, () -> runner.run());
String expected = String.format("Processor has 3 validation failures:\n" + final String expected = String.format("Processor has 3 validation failures:\n" +
"'%s' is invalid because %s is required\n" + "'%s' is invalid because %s is required\n" +
"'%s' is invalid because %s is required\n" + "'%s' is invalid because %s is required\n" +
"'%s' is invalid because %s is required\n", "'%s' is invalid because %s is required\n",
@ -79,7 +79,7 @@ public class GetElasticsearchTest {
runner.setProperty(GetElasticsearch.ATTRIBUTE_NAME, ""); runner.setProperty(GetElasticsearch.ATTRIBUTE_NAME, "");
final AssertionError assertionError = assertThrows(AssertionError.class, () -> runner.run()); final AssertionError assertionError = assertThrows(AssertionError.class, () -> runner.run());
String expected = String.format("Processor has 6 validation failures:\n" + final String expected = String.format("Processor has 6 validation failures:\n" +
"'%s' validated against '' is invalid because %s cannot be empty\n" + "'%s' validated against '' is invalid because %s cannot be empty\n" +
"'%s' validated against '' is invalid because %s cannot be empty\n" + "'%s' validated against '' is invalid because %s cannot be empty\n" +
"'%s' validated against '' is invalid because %s cannot be empty\n" + "'%s' validated against '' is invalid because %s cannot be empty\n" +
@ -101,7 +101,7 @@ public class GetElasticsearchTest {
runner.setProperty(GetElasticsearch.ATTRIBUTE_NAME, ""); runner.setProperty(GetElasticsearch.ATTRIBUTE_NAME, "");
final AssertionError assertionError = assertThrows(AssertionError.class, () -> runner.run()); final AssertionError assertionError = assertThrows(AssertionError.class, () -> runner.run());
String expected = String.format("Processor has 1 validation failures:\n" + final String expected = String.format("Processor has 1 validation failures:\n" +
"'%s' validated against '' is invalid because %s cannot be empty\n", "'%s' validated against '' is invalid because %s cannot be empty\n",
GetElasticsearch.ATTRIBUTE_NAME.getName(), GetElasticsearch.ATTRIBUTE_NAME.getName()); GetElasticsearch.ATTRIBUTE_NAME.getName(), GetElasticsearch.ATTRIBUTE_NAME.getName());
assertEquals(expected, assertionError.getMessage()); assertEquals(expected, assertionError.getMessage());

View File

@ -16,19 +16,22 @@
*/ */
package org.apache.nifi.processors.elasticsearch; package org.apache.nifi.processors.elasticsearch;
public class JsonQueryElasticsearchTest extends AbstractJsonQueryElasticsearchTest<AbstractJsonQueryElasticsearch> { import org.apache.nifi.components.state.Scope;
import org.apache.nifi.processors.elasticsearch.api.JsonQueryParameters;
public class JsonQueryElasticsearchTest extends AbstractJsonQueryElasticsearchTest<AbstractJsonQueryElasticsearch<JsonQueryParameters>> {
@Override @Override
public AbstractJsonQueryElasticsearch getProcessor() { AbstractJsonQueryElasticsearch<JsonQueryParameters> getProcessor() {
return new JsonQueryElasticsearch(); return new JsonQueryElasticsearch();
} }
@Override @Override
public boolean isStateUsed() { Scope getStateScope() {
return false; return null;
} }
@Override @Override
public boolean isInput() { boolean isInput() {
return true; return true;
} }
} }

View File

@ -19,6 +19,9 @@ package org.apache.nifi.processors.elasticsearch;
import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -27,40 +30,44 @@ public class JsonUtils {
private static final ObjectMapper MAPPER = new ObjectMapper(); private static final ObjectMapper MAPPER = new ObjectMapper();
private JsonUtils() {} private JsonUtils() {}
static String toJson(Object object) { static String readString(final Path path) throws IOException {
return Files.readString(path);
}
static String toJson(final Object object) {
try { try {
return MAPPER.writeValueAsString(object); return MAPPER.writeValueAsString(object);
} catch (JsonProcessingException e) { } catch (final JsonProcessingException e) {
throw new RuntimeException(e); throw new RuntimeException(e);
} }
} }
static String prettyPrint(Object object) { static String prettyPrint(final Object object) {
try { try {
return MAPPER.writerWithDefaultPrettyPrinter().writeValueAsString(object); return MAPPER.writerWithDefaultPrettyPrinter().writeValueAsString(object);
} catch (JsonProcessingException e) { } catch (final JsonProcessingException e) {
throw new RuntimeException(e); throw new RuntimeException(e);
} }
} }
static Map<String, Object> readMap(String json) { static Map<String, Object> readMap(final String json) {
try { try {
return MAPPER.readValue(json, Map.class); return MAPPER.readValue(json, Map.class);
} catch (JsonProcessingException e) { } catch (final JsonProcessingException e) {
throw new RuntimeException(e); throw new RuntimeException(e);
} }
} }
static List<String> readListOfMapsAsIndividualJson(String json) { static List<String> readListOfMapsAsIndividualJson(final String json) {
return readListOfMaps(json).stream() return readListOfMaps(json).stream()
.map(JsonUtils::prettyPrint) .map(JsonUtils::prettyPrint)
.collect(Collectors.toList()); .collect(Collectors.toList());
} }
static List<Map<String, Object>> readListOfMaps(String json) { static List<Map<String, Object>> readListOfMaps(final String json) {
try { try {
return MAPPER.readValue(json, List.class); return MAPPER.readValue(json, List.class);
} catch (JsonProcessingException e) { } catch (final JsonProcessingException e) {
throw new RuntimeException(e); throw new RuntimeException(e);
} }
} }

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.nifi.processors.elasticsearch; package org.apache.nifi.processors.elasticsearch;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.processors.elasticsearch.api.PaginationType; import org.apache.nifi.processors.elasticsearch.api.PaginationType;
import org.apache.nifi.processors.elasticsearch.api.ResultOutputStrategy; import org.apache.nifi.processors.elasticsearch.api.ResultOutputStrategy;
import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.MockFlowFile;
@ -31,27 +32,26 @@ public class PaginatedJsonQueryElasticsearchTest extends AbstractPaginatedJsonQu
public static void setUpBeforeClass() throws Exception { public static void setUpBeforeClass() throws Exception {
AbstractPaginatedJsonQueryElasticsearchTest.setUpBeforeClass(); AbstractPaginatedJsonQueryElasticsearchTest.setUpBeforeClass();
} }
AbstractPaginatedJsonQueryElasticsearch getProcessor() {
public AbstractPaginatedJsonQueryElasticsearch getProcessor() {
return new PaginatedJsonQueryElasticsearch(); return new PaginatedJsonQueryElasticsearch();
} }
public boolean isStateUsed() { Scope getStateScope() {
return false; return null;
} }
public boolean isInput() { boolean isInput() {
return true; return true;
} }
@Override @Override
void validatePagination(final TestRunner runner, final ResultOutputStrategy resultOutputStrategy, final PaginationType paginationType, int iteration) { void validatePagination(final TestRunner runner, final ResultOutputStrategy resultOutputStrategy, final PaginationType paginationType, final int iteration) {
runner.getStateManager().assertStateNotSet(); runner.getStateManager().assertStateNotSet();
switch (resultOutputStrategy) { switch (resultOutputStrategy) {
case PER_RESPONSE: case PER_RESPONSE:
AbstractJsonQueryElasticsearchTest.testCounts(runner, 1, 2, 0, 0); AbstractJsonQueryElasticsearchTest.testCounts(runner, 1, 2, 0, 0);
for(int page = 1; page <= 2; page++) { for(int page = 1; page <= 2; page++) {
MockFlowFile hit = runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(page - 1); final MockFlowFile hit = runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(page - 1);
hit.assertAttributeEquals("hit.count", "10"); hit.assertAttributeEquals("hit.count", "10");
hit.assertAttributeEquals("page.number", Integer.toString(page)); hit.assertAttributeEquals("page.number", Integer.toString(page));
} }
@ -66,8 +66,8 @@ public class PaginatedJsonQueryElasticsearchTest extends AbstractPaginatedJsonQu
case PER_HIT: case PER_HIT:
AbstractJsonQueryElasticsearchTest.testCounts(runner, 1, 20, 0, 0); AbstractJsonQueryElasticsearchTest.testCounts(runner, 1, 20, 0, 0);
long count = 1; long count = 1;
ValueRange firstPage = ValueRange.of(1, 10); final ValueRange firstPage = ValueRange.of(1, 10);
for (MockFlowFile hit : runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS)) { for (final MockFlowFile hit : runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS)) {
hit.assertAttributeEquals("hit.count", "1"); hit.assertAttributeEquals("hit.count", "1");
// 10 hits per page, so first 10 flow files should be page.number 1, the rest page.number 2 // 10 hits per page, so first 10 flow files should be page.number 1, the rest page.number 2
hit.assertAttributeEquals("page.number", firstPage.isValidValue(count) ? "1" : "2"); hit.assertAttributeEquals("page.number", firstPage.isValidValue(count) ? "1" : "2");

View File

@ -62,10 +62,10 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
@BeforeAll @BeforeAll
public static void setUpBeforeClass() throws Exception { public static void setUpBeforeClass() throws Exception {
sampleErrorResponse = Files.readString(Paths.get(TEST_COMMON_DIR,"sampleErrorResponse.json")); sampleErrorResponse = JsonUtils.readString(Paths.get(TEST_COMMON_DIR,"sampleErrorResponse.json"));
flowFileContents = Files.readString(Paths.get(TEST_DIR, "flowFileContents.json")); flowFileContents = JsonUtils.readString(Paths.get(TEST_DIR, "flowFileContents.json"));
script = Files.readString(Paths.get(TEST_DIR,"script.json")); script = JsonUtils.readString(Paths.get(TEST_DIR,"script.json"));
dynamicTemplates = Files.readString(Paths.get(TEST_COMMON_DIR,"dynamicTemplates.json")); dynamicTemplates = JsonUtils.readString(Paths.get(TEST_COMMON_DIR,"dynamicTemplates.json"));
expectedScript = new LinkedHashMap<>(); expectedScript = new LinkedHashMap<>();
expectedScript.put("_source", "some script"); expectedScript.put("_source", "some script");
expectedScript.put("language", "painless"); expectedScript.put("language", "painless");
@ -99,16 +99,16 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
runner.assertValid(); runner.assertValid();
} }
public void basicTest(int failure, int retry, int success) { public void basicTest(final int failure, final int retry, final int success) {
Consumer<List<IndexOperationRequest>> consumer = (List<IndexOperationRequest> items) -> { final Consumer<List<IndexOperationRequest>> consumer = (final List<IndexOperationRequest> items) -> {
long nullIdCount = items.stream().filter(item -> item.getId() == null).count(); final long nullIdCount = items.stream().filter(item -> item.getId() == null).count();
long indexCount = items.stream().filter(item -> "test_index".equals(item.getIndex())).count(); final long indexCount = items.stream().filter(item -> "test_index".equals(item.getIndex())).count();
long typeCount = items.stream().filter(item -> "test_type".equals(item.getType())).count(); final long typeCount = items.stream().filter(item -> "test_type".equals(item.getType())).count();
long opCount = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count(); final long opCount = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count();
long emptyScriptCount = items.stream().filter(item -> item.getScript().isEmpty()).count(); final long emptyScriptCount = items.stream().filter(item -> item.getScript().isEmpty()).count();
long falseScriptedUpsertCount = items.stream().filter(item -> !item.isScriptedUpsert()).count(); final long falseScriptedUpsertCount = items.stream().filter(item -> !item.isScriptedUpsert()).count();
long emptyDynamicTemplatesCount = items.stream().filter(item -> item.getDynamicTemplates().isEmpty()).count(); final long emptyDynamicTemplatesCount = items.stream().filter(item -> item.getDynamicTemplates().isEmpty()).count();
long emptyHeaderFields = items.stream().filter(item -> item.getHeaderFields().isEmpty()).count(); final long emptyHeaderFields = items.stream().filter(item -> item.getHeaderFields().isEmpty()).count();
assertEquals(1L, nullIdCount); assertEquals(1L, nullIdCount);
assertEquals(1L, indexCount); assertEquals(1L, indexCount);
@ -123,12 +123,12 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
basicTest(failure, retry, success, consumer, null); basicTest(failure, retry, success, consumer, null);
} }
public void basicTest(int failure, int retry, int success, Consumer<List<IndexOperationRequest>> consumer, Map<String, String> attr) { public void basicTest(final int failure, final int retry, final int success, final Consumer<List<IndexOperationRequest>> consumer, final Map<String, String> attr) {
clientService.setEvalConsumer(consumer); clientService.setEvalConsumer(consumer);
basicTest(failure, retry, success, attr); basicTest(failure, retry, success, attr);
} }
public void basicTest(int failure, int retry, int success, Map<String, String> attr) { public void basicTest(final int failure, final int retry, final int success, final Map<String, String> attr) {
if (attr != null) { if (attr != null) {
runner.enqueue(flowFileContents, attr); runner.enqueue(flowFileContents, attr);
} else { } else {
@ -168,25 +168,25 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
runner.setEnvironmentVariableValue("version", "external"); runner.setEnvironmentVariableValue("version", "external");
runner.assertValid(); runner.assertValid();
clientService.setEvalParametersConsumer((Map<String, String> params) -> { clientService.setEvalParametersConsumer((final Map<String, String> params) -> {
assertEquals(2, params.size()); assertEquals(2, params.size());
assertEquals("true", params.get("refresh")); assertEquals("true", params.get("refresh"));
assertEquals("auto", params.get("slices")); assertEquals("auto", params.get("slices"));
}); });
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> { clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
long idCount = items.stream().filter(item -> "123".equals(item.getId())).count(); final long idCount = items.stream().filter(item -> "123".equals(item.getId())).count();
long indexCount = items.stream().filter(item -> "test_index".equals(item.getIndex())).count(); final long indexCount = items.stream().filter(item -> "test_index".equals(item.getIndex())).count();
long typeCount = items.stream().filter(item -> "test_type".equals(item.getType())).count(); final long typeCount = items.stream().filter(item -> "test_type".equals(item.getType())).count();
long opCount = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count(); final long opCount = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count();
long headerFieldsCount = items.stream().filter(item -> !item.getHeaderFields().isEmpty()).count(); final long headerFieldsCount = items.stream().filter(item -> !item.getHeaderFields().isEmpty()).count();
assertEquals(1L, idCount); assertEquals(1L, idCount);
assertEquals(1L, indexCount); assertEquals(1L, indexCount);
assertEquals(1L, typeCount); assertEquals(1L, typeCount);
assertEquals(1L, opCount); assertEquals(1L, opCount);
assertEquals(1L, headerFieldsCount); assertEquals(1L, headerFieldsCount);
Map<String, String> headerFields = items.get(0).getHeaderFields(); final Map<String, String> headerFields = items.get(0).getHeaderFields();
assertEquals(2, headerFields.size()); assertEquals(2, headerFields.size());
assertEquals("1", headerFields.get("routing")); assertEquals("1", headerFields.get("routing"));
assertEquals("external", headerFields.get("version")); assertEquals("external", headerFields.get("version"));
@ -205,25 +205,25 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
runner.setProperty(AbstractPutElasticsearch.BULK_HEADER_PREFIX + "empty", "${empty}"); runner.setProperty(AbstractPutElasticsearch.BULK_HEADER_PREFIX + "empty", "${empty}");
runner.assertValid(); runner.assertValid();
clientService.setEvalParametersConsumer((Map<String, String> params) -> { clientService.setEvalParametersConsumer((final Map<String, String> params) -> {
assertEquals(2, params.size()); assertEquals(2, params.size());
assertEquals("true", params.get("refresh")); assertEquals("true", params.get("refresh"));
assertEquals("auto", params.get("slices")); assertEquals("auto", params.get("slices"));
}); });
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> { clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
long nullIdCount = items.stream().filter(item -> item.getId() == null).count(); final long nullIdCount = items.stream().filter(item -> item.getId() == null).count();
long headerFieldsCount = items.stream().filter(item -> !item.getHeaderFields().isEmpty()).count(); final long headerFieldsCount = items.stream().filter(item -> !item.getHeaderFields().isEmpty()).count();
assertEquals(1L, nullIdCount); assertEquals(1L, nullIdCount);
assertEquals(1L, headerFieldsCount); assertEquals(1L, headerFieldsCount);
Map<String, String> headerFields = items.get(0).getHeaderFields(); final Map<String, String> headerFields = items.get(0).getHeaderFields();
assertEquals(2, headerFields.size()); assertEquals(2, headerFields.size());
assertEquals("1", headerFields.get("routing")); assertEquals("1", headerFields.get("routing"));
assertEquals("external", headerFields.get("version")); assertEquals("external", headerFields.get("version"));
}); });
Map<String, String> attributes = new LinkedHashMap<>(); final Map<String, String> attributes = new LinkedHashMap<>();
attributes.put("slices", "auto"); attributes.put("slices", "auto");
attributes.put("version", "external"); attributes.put("version", "external");
attributes.put("blank", " "); attributes.put("blank", " ");
@ -235,10 +235,10 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
public void simpleTestWithScriptAndDynamicTemplates() { public void simpleTestWithScriptAndDynamicTemplates() {
runner.setProperty(PutElasticsearchJson.SCRIPT, script); runner.setProperty(PutElasticsearchJson.SCRIPT, script);
runner.setProperty(PutElasticsearchJson.DYNAMIC_TEMPLATES, dynamicTemplates); runner.setProperty(PutElasticsearchJson.DYNAMIC_TEMPLATES, dynamicTemplates);
Consumer<List<IndexOperationRequest>> consumer = (List<IndexOperationRequest> items) -> { final Consumer<List<IndexOperationRequest>> consumer = (final List<IndexOperationRequest> items) -> {
long scriptCount = items.stream().filter(item -> item.getScript().equals(expectedScript)).count(); final long scriptCount = items.stream().filter(item -> item.getScript().equals(expectedScript)).count();
long falseScriptedUpsertCount = items.stream().filter(item -> !item.isScriptedUpsert()).count(); final long falseScriptedUpsertCount = items.stream().filter(item -> !item.isScriptedUpsert()).count();
long dynamicTemplatesCount = items.stream().filter(item -> item.getDynamicTemplates().equals(expectedDynamicTemplate)).count(); final long dynamicTemplatesCount = items.stream().filter(item -> item.getDynamicTemplates().equals(expectedDynamicTemplate)).count();
assertEquals(1L, scriptCount); assertEquals(1L, scriptCount);
assertEquals(1L, falseScriptedUpsertCount); assertEquals(1L, falseScriptedUpsertCount);
assertEquals(1L, dynamicTemplatesCount); assertEquals(1L, dynamicTemplatesCount);
@ -252,10 +252,10 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
runner.setProperty(PutElasticsearchJson.DYNAMIC_TEMPLATES, dynamicTemplates); runner.setProperty(PutElasticsearchJson.DYNAMIC_TEMPLATES, dynamicTemplates);
runner.setProperty(PutElasticsearchJson.INDEX_OP, IndexOperationRequest.Operation.Upsert.getValue().toLowerCase()); runner.setProperty(PutElasticsearchJson.INDEX_OP, IndexOperationRequest.Operation.Upsert.getValue().toLowerCase());
runner.setProperty(PutElasticsearchJson.SCRIPTED_UPSERT, "true"); runner.setProperty(PutElasticsearchJson.SCRIPTED_UPSERT, "true");
Consumer<List<IndexOperationRequest>> consumer = (List<IndexOperationRequest> items) -> { final Consumer<List<IndexOperationRequest>> consumer = (final List<IndexOperationRequest> items) -> {
long scriptCount = items.stream().filter(item -> item.getScript().equals(expectedScript)).count(); final long scriptCount = items.stream().filter(item -> item.getScript().equals(expectedScript)).count();
long trueScriptedUpsertCount = items.stream().filter(IndexOperationRequest::isScriptedUpsert).count(); final long trueScriptedUpsertCount = items.stream().filter(IndexOperationRequest::isScriptedUpsert).count();
long dynamicTemplatesCount = items.stream().filter(item -> item.getDynamicTemplates().equals(expectedDynamicTemplate)).count(); final long dynamicTemplatesCount = items.stream().filter(item -> item.getDynamicTemplates().equals(expectedDynamicTemplate)).count();
assertEquals(1L, scriptCount); assertEquals(1L, scriptCount);
assertEquals(1L, trueScriptedUpsertCount); assertEquals(1L, trueScriptedUpsertCount);
@ -324,7 +324,7 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
runner.setProperty(PutElasticsearchJson.BATCH_SIZE, "100"); runner.setProperty(PutElasticsearchJson.BATCH_SIZE, "100");
runner.setProperty(PutElasticsearchJson.NOT_FOUND_IS_SUCCESSFUL, "true"); runner.setProperty(PutElasticsearchJson.NOT_FOUND_IS_SUCCESSFUL, "true");
clientService.setResponse(IndexOperationResponse.fromJsonResponse(sampleErrorResponse)); clientService.setResponse(IndexOperationResponse.fromJsonResponse(sampleErrorResponse));
List<String> values = JsonUtils.readListOfMapsAsIndividualJson(Files.readString(BATCH_WITH_ERROR)); final List<String> values = JsonUtils.readListOfMapsAsIndividualJson(Files.readString(BATCH_WITH_ERROR));
values.forEach(val -> runner.enqueue(val)); values.forEach(val -> runner.enqueue(val));
runner.assertValid(); runner.assertValid();
runner.run(); runner.run();
@ -365,7 +365,7 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
runner.setProperty(PutElasticsearchJson.NOT_FOUND_IS_SUCCESSFUL, "false"); runner.setProperty(PutElasticsearchJson.NOT_FOUND_IS_SUCCESSFUL, "false");
runner.setProperty(PutElasticsearchJson.OUTPUT_ERROR_RESPONSES, "true"); runner.setProperty(PutElasticsearchJson.OUTPUT_ERROR_RESPONSES, "true");
clientService.setResponse(IndexOperationResponse.fromJsonResponse(sampleErrorResponse)); clientService.setResponse(IndexOperationResponse.fromJsonResponse(sampleErrorResponse));
List<String> values = JsonUtils.readListOfMapsAsIndividualJson(Files.readString(BATCH_WITH_ERROR)); final List<String> values = JsonUtils.readListOfMapsAsIndividualJson(Files.readString(BATCH_WITH_ERROR));
values.forEach(val -> runner.enqueue(val)); values.forEach(val -> runner.enqueue(val));
runner.assertValid(); runner.assertValid();
runner.run(); runner.run();
@ -416,7 +416,7 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
runner.setProperty(PutElasticsearchJson.LOG_ERROR_RESPONSES, "false"); runner.setProperty(PutElasticsearchJson.LOG_ERROR_RESPONSES, "false");
runner.setProperty(PutElasticsearchJson.BATCH_SIZE, "100"); runner.setProperty(PutElasticsearchJson.BATCH_SIZE, "100");
clientService.setResponse(IndexOperationResponse.fromJsonResponse(sampleErrorResponse)); clientService.setResponse(IndexOperationResponse.fromJsonResponse(sampleErrorResponse));
for (final String val : JsonUtils.readListOfMapsAsIndividualJson(Files.readString(Paths.get(TEST_DIR, "batchWithoutError.json")))) { for (final String val : JsonUtils.readListOfMapsAsIndividualJson(JsonUtils.readString(Paths.get(TEST_DIR, "batchWithoutError.json")))) {
runner.enqueue(val); runner.enqueue(val);
} }

View File

@ -92,7 +92,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
@BeforeAll @BeforeAll
public static void setUpBeforeClass() throws Exception { public static void setUpBeforeClass() throws Exception {
flowFileContentMaps = Files.readString(Paths.get(TEST_DIR, "flowFileContentMaps.json")); flowFileContentMaps = JsonUtils.readString(Paths.get(TEST_DIR, "flowFileContentMaps.json"));
simpleSchema = getRecordSchema(Paths.get(TEST_DIR, "simpleSchema.json")); simpleSchema = getRecordSchema(Paths.get(TEST_DIR, "simpleSchema.json"));
recordPathTestSchema = getRecordSchema(Paths.get(TEST_DIR, "recordPathTestSchema.json")); recordPathTestSchema = getRecordSchema(Paths.get(TEST_DIR, "recordPathTestSchema.json"));
dateTimeFormattingTestSchema = getRecordSchema(Paths.get(TEST_DIR, "dateTimeFormattingTestSchema.json")); dateTimeFormattingTestSchema = getRecordSchema(Paths.get(TEST_DIR, "dateTimeFormattingTestSchema.json"));
@ -105,7 +105,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
clientService.setResponse(new IndexOperationResponse(1500)); clientService.setResponse(new IndexOperationResponse(1500));
registry = new MockSchemaRegistry(); registry = new MockSchemaRegistry();
registry.addSchema("simple", simpleSchema); registry.addSchema("simple", simpleSchema);
RecordReaderFactory reader = new JsonTreeReader(); final RecordReaderFactory reader = new JsonTreeReader();
runner = TestRunners.newTestRunner(getTestProcessor()); runner = TestRunners.newTestRunner(getTestProcessor());
runner.addControllerService("registry", registry); runner.addControllerService("registry", registry);
runner.addControllerService("reader", reader); runner.addControllerService("reader", reader);
@ -126,16 +126,16 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
runner.assertValid(); runner.assertValid();
} }
public void basicTest(int failure, int retry, int success) { public void basicTest(final int failure, final int retry, final int success) {
Consumer<List<IndexOperationRequest>> consumer = (List<IndexOperationRequest> items) -> { final Consumer<List<IndexOperationRequest>> consumer = (final List<IndexOperationRequest> items) -> {
long timestampDefaultCount = items.stream().filter(item -> "test_timestamp".equals(item.getFields().get("@timestamp"))).count(); final long timestampDefaultCount = items.stream().filter(item -> "test_timestamp".equals(item.getFields().get("@timestamp"))).count();
long indexCount = items.stream().filter(item -> "test_index".equals(item.getIndex())).count(); final long indexCount = items.stream().filter(item -> "test_index".equals(item.getIndex())).count();
long typeCount = items.stream().filter(item -> "test_type".equals(item.getType())).count(); final long typeCount = items.stream().filter(item -> "test_type".equals(item.getType())).count();
long opCount = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count(); final long opCount = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count();
long emptyScriptCount = items.stream().filter(item -> item.getScript().isEmpty()).count(); final long emptyScriptCount = items.stream().filter(item -> item.getScript().isEmpty()).count();
long falseScriptedUpsertCount = items.stream().filter(item -> !item.isScriptedUpsert()).count(); final long falseScriptedUpsertCount = items.stream().filter(item -> !item.isScriptedUpsert()).count();
long emptyDynamicTemplatesCount = items.stream().filter(item -> item.getDynamicTemplates().isEmpty()).count(); final long emptyDynamicTemplatesCount = items.stream().filter(item -> item.getDynamicTemplates().isEmpty()).count();
long emptyHeaderFields = items.stream().filter(item -> item.getHeaderFields().isEmpty()).count(); final long emptyHeaderFields = items.stream().filter(item -> item.getHeaderFields().isEmpty()).count();
assertEquals(2, timestampDefaultCount); assertEquals(2, timestampDefaultCount);
assertEquals(2, indexCount); assertEquals(2, indexCount);
assertEquals(2, typeCount); assertEquals(2, typeCount);
@ -149,11 +149,11 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
basicTest(failure, retry, success, consumer); basicTest(failure, retry, success, consumer);
} }
public void basicTest(int failure, int retry, int success, Consumer<List<IndexOperationRequest>> consumer) { public void basicTest(final int failure, final int retry, final int success, final Consumer<List<IndexOperationRequest>> consumer) {
basicTest(failure, retry, success, consumer, null); basicTest(failure, retry, success, consumer, null);
} }
public void basicTest(int failure, int retry, int success, Consumer<List<IndexOperationRequest>> consumer, Map<String, String> attributes) { public void basicTest(final int failure, final int retry, final int success, final Consumer<List<IndexOperationRequest>> consumer, final Map<String, String> attributes) {
clientService.setEvalConsumer(consumer); clientService.setEvalConsumer(consumer);
runner.enqueue(flowFileContentMaps, attributes != null && !attributes.isEmpty() ? attributes : Collections.singletonMap(SCHEMA_NAME_ATTRIBUTE, "simple")); runner.enqueue(flowFileContentMaps, attributes != null && !attributes.isEmpty() ? attributes : Collections.singletonMap(SCHEMA_NAME_ATTRIBUTE, "simple"));
runner.run(); runner.run();
@ -185,7 +185,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
@Test @Test
public void simpleTestCoercedDefaultTimestamp() { public void simpleTestCoercedDefaultTimestamp() {
Consumer<List<IndexOperationRequest>> consumer = (List<IndexOperationRequest> items) -> final Consumer<List<IndexOperationRequest>> consumer = (List<IndexOperationRequest> items) ->
assertEquals(2L, items.stream().filter(item -> Long.valueOf(100).equals(item.getFields().get("@timestamp"))).count()); assertEquals(2L, items.stream().filter(item -> Long.valueOf(100).equals(item.getFields().get("@timestamp"))).count());
runner.setProperty(PutElasticsearchRecord.AT_TIMESTAMP, "100"); runner.setProperty(PutElasticsearchRecord.AT_TIMESTAMP, "100");
@ -202,7 +202,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
@Test @Test
public void simpleTestWithRequestParametersAndBulkHeadersFlowFileEL() { public void simpleTestWithRequestParametersAndBulkHeadersFlowFileEL() {
Map<String, String> attributes = new LinkedHashMap<>(); final Map<String, String> attributes = new LinkedHashMap<>();
attributes.put(SCHEMA_NAME_ATTRIBUTE, "simple"); attributes.put(SCHEMA_NAME_ATTRIBUTE, "simple");
attributes.put("version", "/version"); attributes.put("version", "/version");
attributes.put("slices", "auto"); attributes.put("slices", "auto");
@ -212,7 +212,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
@Test @Test
public void simpleTestWithMockReader() throws Exception{ public void simpleTestWithMockReader() throws Exception{
MockRecordParser mockReader = new MockRecordParser(); final MockRecordParser mockReader = new MockRecordParser();
mockReader.addSchemaField("msg", RecordFieldType.STRING); mockReader.addSchemaField("msg", RecordFieldType.STRING);
mockReader.addSchemaField("from", RecordFieldType.STRING); mockReader.addSchemaField("from", RecordFieldType.STRING);
mockReader.addRecord("foo", "bar"); mockReader.addRecord("foo", "bar");
@ -239,29 +239,29 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
@Test @Test
public void testRecordPathFeatures() throws Exception { public void testRecordPathFeatures() throws Exception {
Map<String, Object> script = final Map<String, Object> script =
JsonUtils.readMap(Files.readString(Paths.get(TEST_DIR, "script.json"))); JsonUtils.readMap(JsonUtils.readString(Paths.get(TEST_DIR, "script.json")));
Map<String, Object> dynamicTemplates = final Map<String, Object> dynamicTemplates =
JsonUtils.readMap(Files.readString(Paths.get(TEST_COMMON_DIR, "dynamicTemplates.json"))); JsonUtils.readMap(JsonUtils.readString(Paths.get(TEST_COMMON_DIR, "dynamicTemplates.json")));
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> { clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
long a = items.stream().filter(item -> "bulk_a".equals(item.getIndex())).count(); final long a = items.stream().filter(item -> "bulk_a".equals(item.getIndex())).count();
long b = items.stream().filter(item -> "bulk_b".equals(item.getIndex())).count(); final long b = items.stream().filter(item -> "bulk_b".equals(item.getIndex())).count();
long index = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count(); final long index = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count();
long create = items.stream().filter(item -> IndexOperationRequest.Operation.Create.equals(item.getOperation())).count(); final long create = items.stream().filter(item -> IndexOperationRequest.Operation.Create.equals(item.getOperation())).count();
long msg = items.stream().filter(item -> "Hello".equals(item.getFields().get("msg"))).count(); final long msg = items.stream().filter(item -> "Hello".equals(item.getFields().get("msg"))).count();
long empties = items.stream().filter(item -> ("".equals(item.getFields().get("msg")))).count(); final long empties = items.stream().filter(item -> ("".equals(item.getFields().get("msg")))).count();
long nulls = items.stream().filter(item -> null == item.getFields().get("msg")).count(); final long nulls = items.stream().filter(item -> null == item.getFields().get("msg")).count();
long timestamp = items.stream().filter(item -> final long timestamp = items.stream().filter(item ->
LOCAL_DATE_TIME.format(DateTimeFormatter.ofPattern(RecordFieldType.TIMESTAMP.getDefaultFormat())).equals(item.getFields().get("@timestamp"))).count(); LOCAL_DATE_TIME.format(DateTimeFormatter.ofPattern(RecordFieldType.TIMESTAMP.getDefaultFormat())).equals(item.getFields().get("@timestamp"))).count();
long timestampDefault = items.stream().filter(item -> "test_timestamp".equals(item.getFields().get("@timestamp"))).count(); final long timestampDefault = items.stream().filter(item -> "test_timestamp".equals(item.getFields().get("@timestamp"))).count();
long ts = items.stream().filter(item -> item.getFields().get("ts") != null).count(); final long ts = items.stream().filter(item -> item.getFields().get("ts") != null).count();
long id = items.stream().filter(item -> item.getFields().get("id") != null).count(); final long id = items.stream().filter(item -> item.getFields().get("id") != null).count();
long emptyScript = items.stream().filter(item -> item.getScript().isEmpty()).count(); final long emptyScript = items.stream().filter(item -> item.getScript().isEmpty()).count();
long falseScriptedUpsertCount = items.stream().filter(item -> !item.isScriptedUpsert()).count(); final long falseScriptedUpsertCount = items.stream().filter(item -> !item.isScriptedUpsert()).count();
long trueScriptedUpsertCount = items.stream().filter(IndexOperationRequest::isScriptedUpsert).count(); final long trueScriptedUpsertCount = items.stream().filter(IndexOperationRequest::isScriptedUpsert).count();
long s = items.stream().filter(item -> script.equals(item.getScript())).count(); final long s = items.stream().filter(item -> script.equals(item.getScript())).count();
long emptyDynamicTemplates = items.stream().filter(item -> item.getDynamicTemplates().isEmpty()).count(); final long emptyDynamicTemplates = items.stream().filter(item -> item.getDynamicTemplates().isEmpty()).count();
long dt = items.stream().filter(item -> dynamicTemplates.equals(item.getDynamicTemplates())).count(); final long dt = items.stream().filter(item -> dynamicTemplates.equals(item.getDynamicTemplates())).count();
items.forEach(item -> { items.forEach(item -> {
assertNotNull(item.getId()); assertNotNull(item.getId());
assertTrue(item.getId().startsWith("rec-")); assertTrue(item.getId().startsWith("rec-"));
@ -297,7 +297,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
runner.setProperty(PutElasticsearchRecord.SCRIPT_RECORD_PATH, "/script"); runner.setProperty(PutElasticsearchRecord.SCRIPT_RECORD_PATH, "/script");
runner.setProperty(PutElasticsearchRecord.SCRIPTED_UPSERT_RECORD_PATH, "/scripted_upsert"); runner.setProperty(PutElasticsearchRecord.SCRIPTED_UPSERT_RECORD_PATH, "/scripted_upsert");
runner.setProperty(PutElasticsearchRecord.DYNAMIC_TEMPLATES_RECORD_PATH, "/dynamic_templates"); runner.setProperty(PutElasticsearchRecord.DYNAMIC_TEMPLATES_RECORD_PATH, "/dynamic_templates");
String flowFileContents = Files.readString(Paths.get(TEST_DIR, "1_flowFileContents.json")); String flowFileContents = JsonUtils.readString(Paths.get(TEST_DIR, "1_flowFileContents.json"));
flowFileContents = flowFileContents.replaceFirst("\\d{13}", String.valueOf(Timestamp.valueOf(LOCAL_DATE_TIME).toInstant().toEpochMilli())); flowFileContents = flowFileContents.replaceFirst("\\d{13}", String.valueOf(Timestamp.valueOf(LOCAL_DATE_TIME).toInstant().toEpochMilli()));
runner.enqueue(flowFileContents, Collections.singletonMap(SCHEMA_NAME_ATTRIBUTE, RECORD_PATH_TEST_SCHEMA)); runner.enqueue(flowFileContents, Collections.singletonMap(SCHEMA_NAME_ATTRIBUTE, RECORD_PATH_TEST_SCHEMA));
@ -312,22 +312,22 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
@Test @Test
public void testTimestampDateFormatAndScriptRecordPath() throws Exception { public void testTimestampDateFormatAndScriptRecordPath() throws Exception {
Map<String, Object> script = final Map<String, Object> script =
JsonUtils.readMap(Files.readString(Paths.get(TEST_DIR, "script.json"))); JsonUtils.readMap(JsonUtils.readString(Paths.get(TEST_DIR, "script.json")));
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> { clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
long testTypeCount = items.stream().filter(item -> "test_type".equals(item.getType())).count(); final long testTypeCount = items.stream().filter(item -> "test_type".equals(item.getType())).count();
long messageTypeCount = items.stream().filter(item -> "message".equals(item.getType())).count(); final long messageTypeCount = items.stream().filter(item -> "message".equals(item.getType())).count();
long testIndexCount = items.stream().filter(item -> "test_index".equals(item.getIndex())).count(); final long testIndexCount = items.stream().filter(item -> "test_index".equals(item.getIndex())).count();
long bulkIndexCount = items.stream().filter(item -> item.getIndex().startsWith("bulk_")).count(); final long bulkIndexCount = items.stream().filter(item -> item.getIndex().startsWith("bulk_")).count();
long indexOperationCount = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count(); final long indexOperationCount = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count();
long updateOperationCount = items.stream().filter(item -> IndexOperationRequest.Operation.Update.equals(item.getOperation())).count(); final long updateOperationCount = items.stream().filter(item -> IndexOperationRequest.Operation.Update.equals(item.getOperation())).count();
long timestampCount = items.stream().filter(item -> final long timestampCount = items.stream().filter(item ->
LOCAL_DATE.format(DateTimeFormatter.ofPattern("dd/MM/yyyy")).equals(item.getFields().get("@timestamp"))).count(); LOCAL_DATE.format(DateTimeFormatter.ofPattern("dd/MM/yyyy")).equals(item.getFields().get("@timestamp"))).count();
long dateCount = items.stream().filter(item -> item.getFields().get("date") != null).count(); final long dateCount = items.stream().filter(item -> item.getFields().get("date") != null).count();
long idCount = items.stream().filter(item -> item.getFields().get("id") != null).count(); final long idCount = items.stream().filter(item -> item.getFields().get("id") != null).count();
long defaultCoercedTimestampCount = items.stream().filter(item -> Long.valueOf(100).equals(item.getFields().get("@timestamp"))).count(); final long defaultCoercedTimestampCount = items.stream().filter(item -> Long.valueOf(100).equals(item.getFields().get("@timestamp"))).count();
long emptyScriptCount = items.stream().filter(item -> item.getScript().isEmpty()).count(); final long emptyScriptCount = items.stream().filter(item -> item.getScript().isEmpty()).count();
long scriptCount = items.stream().filter(item -> script.equals(item.getScript())).count(); final long scriptCount = items.stream().filter(item -> script.equals(item.getScript())).count();
assertEquals(5, testTypeCount, getUnexpectedCountMsg("test type")); assertEquals(5, testTypeCount, getUnexpectedCountMsg("test type"));
assertEquals(1, messageTypeCount, getUnexpectedCountMsg("message type")); assertEquals(1, messageTypeCount, getUnexpectedCountMsg("message type"));
assertEquals(5, testIndexCount, getUnexpectedCountMsg("test index")); assertEquals(5, testIndexCount, getUnexpectedCountMsg("test index"));
@ -353,10 +353,10 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
runner.setProperty(PutElasticsearchRecord.TYPE_RECORD_PATH, "/type"); runner.setProperty(PutElasticsearchRecord.TYPE_RECORD_PATH, "/type");
runner.setProperty(PutElasticsearchRecord.INDEX_RECORD_PATH, "/index"); runner.setProperty(PutElasticsearchRecord.INDEX_RECORD_PATH, "/index");
runner.setProperty(PutElasticsearchRecord.SCRIPT_RECORD_PATH, "/script_record"); runner.setProperty(PutElasticsearchRecord.SCRIPT_RECORD_PATH, "/script_record");
Map<String, String> attributes = new LinkedHashMap<>(); final Map<String, String> attributes = new LinkedHashMap<>();
attributes.put(SCHEMA_NAME_ATTRIBUTE, RECORD_PATH_TEST_SCHEMA); attributes.put(SCHEMA_NAME_ATTRIBUTE, RECORD_PATH_TEST_SCHEMA);
attributes.put("operation", "index"); attributes.put("operation", "index");
String flowFileContents = Files.readString(Paths.get(TEST_DIR, "2_flowFileContents.json")); String flowFileContents = JsonUtils.readString(Paths.get(TEST_DIR, "2_flowFileContents.json"));
flowFileContents = flowFileContents.replaceFirst("\\d{13}", String.valueOf(Date.valueOf(LOCAL_DATE).getTime())); flowFileContents = flowFileContents.replaceFirst("\\d{13}", String.valueOf(Date.valueOf(LOCAL_DATE).getTime()));
runner.enqueue(flowFileContents, attributes); runner.enqueue(flowFileContents, attributes);
runner.run(); runner.run();
@ -370,12 +370,12 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
@Test @Test
public void testNullRecordPaths() throws Exception { public void testNullRecordPaths() throws Exception {
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> { clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
long nullTypeCount = items.stream().filter(item -> item.getType() == null).count(); final long nullTypeCount = items.stream().filter(item -> item.getType() == null).count();
long messageTypeCount = items.stream().filter(item -> "message".equals(item.getType())).count(); final long messageTypeCount = items.stream().filter(item -> "message".equals(item.getType())).count();
long nullIdCount = items.stream().filter(item -> item.getId() == null).count(); final long nullIdCount = items.stream().filter(item -> item.getId() == null).count();
long recIdCount = items.stream().filter(item -> StringUtils.startsWith(item.getId(), "rec-")).count(); final long recIdCount = items.stream().filter(item -> StringUtils.startsWith(item.getId(), "rec-")).count();
long timestampCount = items.stream().filter(item -> final long timestampCount = items.stream().filter(item ->
LOCAL_TIME.format(DateTimeFormatter.ofPattern(RecordFieldType.TIME.getDefaultFormat())).equals(item.getFields().get("@timestamp"))).count(); LOCAL_TIME.format(DateTimeFormatter.ofPattern(RecordFieldType.TIME.getDefaultFormat())).equals(item.getFields().get("@timestamp"))).count();
assertEquals(5, nullTypeCount, getUnexpectedCountMsg("null type")); assertEquals(5, nullTypeCount, getUnexpectedCountMsg("null type"));
assertEquals(1, messageTypeCount, getUnexpectedCountMsg("message type")); assertEquals(1, messageTypeCount, getUnexpectedCountMsg("message type"));
@ -391,7 +391,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
runner.removeProperty(PutElasticsearchRecord.TYPE); runner.removeProperty(PutElasticsearchRecord.TYPE);
runner.setProperty(PutElasticsearchRecord.ID_RECORD_PATH, "/id"); runner.setProperty(PutElasticsearchRecord.ID_RECORD_PATH, "/id");
runner.setProperty(PutElasticsearchRecord.TYPE_RECORD_PATH, "/type"); runner.setProperty(PutElasticsearchRecord.TYPE_RECORD_PATH, "/type");
String flowFileContents = Files.readString(Paths.get(TEST_DIR, "3_flowFileContents.json")); String flowFileContents = JsonUtils.readString(Paths.get(TEST_DIR, "3_flowFileContents.json"));
flowFileContents = flowFileContents.replaceFirst("\\d{8}", String.valueOf(Time.valueOf(LOCAL_TIME).getTime())); flowFileContents = flowFileContents.replaceFirst("\\d{8}", String.valueOf(Time.valueOf(LOCAL_TIME).getTime()));
runner.enqueue(flowFileContents, Collections.singletonMap(SCHEMA_NAME_ATTRIBUTE, RECORD_PATH_TEST_SCHEMA)); runner.enqueue(flowFileContents, Collections.singletonMap(SCHEMA_NAME_ATTRIBUTE, RECORD_PATH_TEST_SCHEMA));
runner.run(); runner.run();
@ -405,14 +405,14 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
@Test @Test
public void testIndexOperationRecordPath() throws Exception { public void testIndexOperationRecordPath() throws Exception {
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> { clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
long index = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count(); final long index = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count();
long create = items.stream().filter(item -> IndexOperationRequest.Operation.Create.equals(item.getOperation())).count(); final long create = items.stream().filter(item -> IndexOperationRequest.Operation.Create.equals(item.getOperation())).count();
long update = items.stream().filter(item -> IndexOperationRequest.Operation.Update.equals(item.getOperation())).count(); final long update = items.stream().filter(item -> IndexOperationRequest.Operation.Update.equals(item.getOperation())).count();
long upsert = items.stream().filter(item -> IndexOperationRequest.Operation.Upsert.equals(item.getOperation())).count(); final long upsert = items.stream().filter(item -> IndexOperationRequest.Operation.Upsert.equals(item.getOperation())).count();
long delete = items.stream().filter(item -> IndexOperationRequest.Operation.Delete.equals(item.getOperation())).count(); final long delete = items.stream().filter(item -> IndexOperationRequest.Operation.Delete.equals(item.getOperation())).count();
long timestampCount = items.stream().filter(item -> Long.valueOf(101).equals(item.getFields().get("@timestamp"))).count(); final long timestampCount = items.stream().filter(item -> Long.valueOf(101).equals(item.getFields().get("@timestamp"))).count();
long noTimestampCount = items.stream().filter(item -> !item.getFields().containsKey("@timestamp") ).count(); final long noTimestampCount = items.stream().filter(item -> !item.getFields().containsKey("@timestamp") ).count();
assertEquals(1, index, getUnexpectedCountMsg("index")); assertEquals(1, index, getUnexpectedCountMsg("index"));
assertEquals(2, create, getUnexpectedCountMsg("create")); assertEquals(2, create, getUnexpectedCountMsg("create"));
assertEquals(1, update, getUnexpectedCountMsg("update")); assertEquals(1, update, getUnexpectedCountMsg("update"));
@ -438,8 +438,8 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
@Test @Test
public void testIncompatibleTimestampRecordPath() throws Exception { public void testIncompatibleTimestampRecordPath() throws Exception {
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> { clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
long timestampCount = items.stream().filter(item -> "Hello".equals(item.getFields().get("@timestamp"))).count(); final long timestampCount = items.stream().filter(item -> "Hello".equals(item.getFields().get("@timestamp"))).count();
assertEquals(1, timestampCount); assertEquals(1, timestampCount);
}); });
@ -463,7 +463,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
@Test @Test
public void testEmptyELRecordPaths() throws Exception { public void testEmptyELRecordPaths() throws Exception {
Map<String, String> attributes = new LinkedHashMap<>(); final Map<String, String> attributes = new LinkedHashMap<>();
attributes.put(SCHEMA_NAME_ATTRIBUTE, RECORD_PATH_TEST_SCHEMA); attributes.put(SCHEMA_NAME_ATTRIBUTE, RECORD_PATH_TEST_SCHEMA);
attributes.put("will_be_empty", "/empty"); attributes.put("will_be_empty", "/empty");
testInvalidELRecordPaths("${will_be_empty}", "${will_be_empty}", testInvalidELRecordPaths("${will_be_empty}", "${will_be_empty}",
@ -483,20 +483,20 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
runner.assertTransferCount(PutElasticsearchRecord.REL_FAILED_RECORDS, 0); runner.assertTransferCount(PutElasticsearchRecord.REL_FAILED_RECORDS, 0);
runner.assertTransferCount(PutElasticsearchRecord.REL_SUCCESSFUL_RECORDS, 0); runner.assertTransferCount(PutElasticsearchRecord.REL_SUCCESSFUL_RECORDS, 0);
runner.assertTransferCount(PutElasticsearchRecord.REL_ERROR_RESPONSES, 0); runner.assertTransferCount(PutElasticsearchRecord.REL_ERROR_RESPONSES, 0);
MockFlowFile failure = runner.getFlowFilesForRelationship(PutElasticsearchRecord.REL_FAILURE).get(0); final MockFlowFile failure = runner.getFlowFilesForRelationship(PutElasticsearchRecord.REL_FAILURE).get(0);
failure.assertAttributeEquals("elasticsearch.put.error", String.format("Field referenced by %s must be Map-type compatible or a String parsable into a JSON Object", "/dynamic_templates")); failure.assertAttributeEquals("elasticsearch.put.error", String.format("Field referenced by %s must be Map-type compatible or a String parsable into a JSON Object", "/dynamic_templates"));
} }
@Test @Test
public void testRecordPathFieldDefaults() throws Exception { public void testRecordPathFieldDefaults() throws Exception {
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> { clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
long idNotNull = items.stream().filter(item -> item.getId() != null).count(); final long idNotNull = items.stream().filter(item -> item.getId() != null).count();
long opIndex = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count(); final long opIndex = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count();
long opCreate = items.stream().filter(item -> IndexOperationRequest.Operation.Create.equals(item.getOperation())).count(); final long opCreate = items.stream().filter(item -> IndexOperationRequest.Operation.Create.equals(item.getOperation())).count();
long indexA = items.stream().filter(item -> "bulk_a".equals(item.getIndex())).count(); final long indexA = items.stream().filter(item -> "bulk_a".equals(item.getIndex())).count();
long indexC = items.stream().filter(item -> "bulk_c".equals(item.getIndex())).count(); final long indexC = items.stream().filter(item -> "bulk_c".equals(item.getIndex())).count();
long typeMessage = items.stream().filter(item -> "message".equals(item.getType())).count(); final long typeMessage = items.stream().filter(item -> "message".equals(item.getType())).count();
long typeBlah = items.stream().filter(item -> "blah".equals(item.getType())).count(); final long typeBlah = items.stream().filter(item -> "blah".equals(item.getType())).count();
assertEquals(4, idNotNull, getUnexpectedCountMsg("id not null")); assertEquals(4, idNotNull, getUnexpectedCountMsg("id not null"));
assertEquals(3, opIndex, getUnexpectedCountMsg("op index")); assertEquals(3, opIndex, getUnexpectedCountMsg("op index"));
assertEquals(3, opCreate, getUnexpectedCountMsg("op create")); assertEquals(3, opCreate, getUnexpectedCountMsg("op create"));
@ -529,22 +529,22 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
@Test @Test
public void testDefaultDateTimeFormatting() throws Exception{ public void testDefaultDateTimeFormatting() throws Exception{
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> { clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
long msg = items.stream().filter(item -> (item.getFields().get("msg") != null)).count(); final long msg = items.stream().filter(item -> (item.getFields().get("msg") != null)).count();
long timestamp = items.stream().filter(item -> final long timestamp = items.stream().filter(item ->
LOCAL_DATE_TIME.format(DateTimeFormatter.ofPattern(RecordFieldType.TIMESTAMP.getDefaultFormat())).equals(item.getFields().get("ts"))).count(); // "yyyy-MM-dd HH:mm:ss" LOCAL_DATE_TIME.format(DateTimeFormatter.ofPattern(RecordFieldType.TIMESTAMP.getDefaultFormat())).equals(item.getFields().get("ts"))).count(); // "yyyy-MM-dd HH:mm:ss"
long date = items.stream().filter(item -> final long date = items.stream().filter(item ->
LOCAL_DATE.format(DateTimeFormatter.ofPattern(RecordFieldType.DATE.getDefaultFormat())).equals(item.getFields().get("date"))).count(); // "yyyy-MM-dd" LOCAL_DATE.format(DateTimeFormatter.ofPattern(RecordFieldType.DATE.getDefaultFormat())).equals(item.getFields().get("date"))).count(); // "yyyy-MM-dd"
long time = items.stream().filter(item -> final long time = items.stream().filter(item ->
LOCAL_TIME.format(DateTimeFormatter.ofPattern(RecordFieldType.TIME.getDefaultFormat())).equals(item.getFields().get("time"))).count(); // "HH:mm:ss" LOCAL_TIME.format(DateTimeFormatter.ofPattern(RecordFieldType.TIME.getDefaultFormat())).equals(item.getFields().get("time"))).count(); // "HH:mm:ss"
long choiceTs = items.stream().filter(item -> final long choiceTs = items.stream().filter(item ->
LOCAL_DATE_TIME.format(DateTimeFormatter.ofPattern(RecordFieldType.TIMESTAMP.getDefaultFormat())).equals(item.getFields().get("choice_ts"))).count(); LOCAL_DATE_TIME.format(DateTimeFormatter.ofPattern(RecordFieldType.TIMESTAMP.getDefaultFormat())).equals(item.getFields().get("choice_ts"))).count();
long choiceNotTs = items.stream().filter(item -> "not-timestamp".equals(item.getFields().get("choice_ts"))).count(); final long choiceNotTs = items.stream().filter(item -> "not-timestamp".equals(item.getFields().get("choice_ts"))).count();
long atTimestampDefault = items.stream().filter(item -> "test_timestamp".equals(item.getFields().get("@timestamp"))).count(); final long atTimestampDefault = items.stream().filter(item -> "test_timestamp".equals(item.getFields().get("@timestamp"))).count();
long tsNull = items.stream().filter(item -> item.getFields().get("ts") == null).count(); final long tsNull = items.stream().filter(item -> item.getFields().get("ts") == null).count();
long dateNull = items.stream().filter(item -> item.getFields().get("date") == null).count(); final long dateNull = items.stream().filter(item -> item.getFields().get("date") == null).count();
long timeNull = items.stream().filter(item -> item.getFields().get("time") == null).count(); final long timeNull = items.stream().filter(item -> item.getFields().get("time") == null).count();
long choiceTsNull = items.stream().filter(item -> item.getFields().get("choice_ts") == null).count(); final long choiceTsNull = items.stream().filter(item -> item.getFields().get("choice_ts") == null).count();
assertEquals(5, msg, getUnexpectedCountMsg("msg")); assertEquals(5, msg, getUnexpectedCountMsg("msg"));
assertEquals(2, timestamp, getUnexpectedCountMsg("timestamp")); assertEquals(2, timestamp, getUnexpectedCountMsg("timestamp"));
assertEquals(2, date, getUnexpectedCountMsg("date")); assertEquals(2, date, getUnexpectedCountMsg("date"));
@ -575,23 +575,23 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
final String timestampFormat = "yy MMM d H"; final String timestampFormat = "yy MMM d H";
final String dateFormat = "dd/MM/yyyy"; final String dateFormat = "dd/MM/yyyy";
final String timeFormat = "HHmmss"; final String timeFormat = "HHmmss";
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> { clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
String timestampOutput = LOCAL_DATE_TIME.format(DateTimeFormatter.ofPattern(timestampFormat)); final String timestampOutput = LOCAL_DATE_TIME.format(DateTimeFormatter.ofPattern(timestampFormat));
long msg = items.stream().filter(item -> (item.getFields().get("msg") != null)).count(); final long msg = items.stream().filter(item -> (item.getFields().get("msg") != null)).count();
long timestamp = items.stream().filter(item -> timestampOutput.equals(item.getFields().get("ts"))).count(); final long timestamp = items.stream().filter(item -> timestampOutput.equals(item.getFields().get("ts"))).count();
long date = items.stream().filter(item -> final long date = items.stream().filter(item ->
LOCAL_DATE.format(DateTimeFormatter.ofPattern(dateFormat)).equals(item.getFields().get("date"))).count(); LOCAL_DATE.format(DateTimeFormatter.ofPattern(dateFormat)).equals(item.getFields().get("date"))).count();
long time = items.stream().filter(item -> final long time = items.stream().filter(item ->
// converted to a Long because the output is completely numerical // converted to a Long because the output is completely numerical
Long.valueOf(LOCAL_TIME.format(DateTimeFormatter.ofPattern(timeFormat))).equals(item.getFields().get("time"))).count(); Long.valueOf(LOCAL_TIME.format(DateTimeFormatter.ofPattern(timeFormat))).equals(item.getFields().get("time"))).count();
long choiceTs = items.stream().filter(item -> timestampOutput.equals(item.getFields().get("choice_ts"))).count(); final long choiceTs = items.stream().filter(item -> timestampOutput.equals(item.getFields().get("choice_ts"))).count();
long choiceNotTs = items.stream().filter(item -> "not-timestamp".equals(item.getFields().get("choice_ts"))).count(); final long choiceNotTs = items.stream().filter(item -> "not-timestamp".equals(item.getFields().get("choice_ts"))).count();
long atTimestampDefault = items.stream().filter(item -> "test_timestamp".equals(item.getFields().get("@timestamp"))).count(); final long atTimestampDefault = items.stream().filter(item -> "test_timestamp".equals(item.getFields().get("@timestamp"))).count();
long atTimestamp = items.stream().filter(item -> timestampOutput.equals(item.getFields().get("@timestamp"))).count(); final long atTimestamp = items.stream().filter(item -> timestampOutput.equals(item.getFields().get("@timestamp"))).count();
long tsNull = items.stream().filter(item -> item.getFields().get("ts") == null).count(); final long tsNull = items.stream().filter(item -> item.getFields().get("ts") == null).count();
long dateNull = items.stream().filter(item -> item.getFields().get("date") == null).count(); final long dateNull = items.stream().filter(item -> item.getFields().get("date") == null).count();
long timeNull = items.stream().filter(item -> item.getFields().get("time") == null).count(); final long timeNull = items.stream().filter(item -> item.getFields().get("time") == null).count();
long choiceTsNull = items.stream().filter(item -> item.getFields().get("choice_ts") == null).count(); final long choiceTsNull = items.stream().filter(item -> item.getFields().get("choice_ts") == null).count();
assertEquals(5, msg, getUnexpectedCountMsg("msg")); assertEquals(5, msg, getUnexpectedCountMsg("msg"));
assertEquals(2, timestamp, getUnexpectedCountMsg("timestamp")); assertEquals(2, timestamp, getUnexpectedCountMsg("timestamp"));
assertEquals(2, date, getUnexpectedCountMsg("date")); assertEquals(2, date, getUnexpectedCountMsg("date"));
@ -659,8 +659,8 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
public void testFailedRecordsOutput() throws Exception { public void testFailedRecordsOutput() throws Exception {
runner.setProperty(PutElasticsearchRecord.NOT_FOUND_IS_SUCCESSFUL, "true"); runner.setProperty(PutElasticsearchRecord.NOT_FOUND_IS_SUCCESSFUL, "true");
runner.setProperty(PutElasticsearchRecord.LOG_ERROR_RESPONSES, "true"); runner.setProperty(PutElasticsearchRecord.LOG_ERROR_RESPONSES, "true");
int errorCount = 3; final int errorCount = 3;
int successCount = 4; final int successCount = 4;
testErrorRelationship(errorCount, successCount, true); testErrorRelationship(errorCount, successCount, true);
runner.assertTransferCount(PutElasticsearchRecord.REL_FAILED_RECORDS, 1); runner.assertTransferCount(PutElasticsearchRecord.REL_FAILED_RECORDS, 1);
@ -676,7 +676,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
runner.setProperty(PutElasticsearchRecord.NOT_FOUND_IS_SUCCESSFUL, "true"); runner.setProperty(PutElasticsearchRecord.NOT_FOUND_IS_SUCCESSFUL, "true");
runner.setProperty(PutElasticsearchRecord.LOG_ERROR_RESPONSES, "true"); runner.setProperty(PutElasticsearchRecord.LOG_ERROR_RESPONSES, "true");
runner.setProperty(PutElasticsearchRecord.GROUP_BULK_ERRORS_BY_TYPE, "true"); runner.setProperty(PutElasticsearchRecord.GROUP_BULK_ERRORS_BY_TYPE, "true");
int successCount = 4; final int successCount = 4;
testErrorRelationship(3, successCount, true); testErrorRelationship(3, successCount, true);
runner.assertTransferCount(PutElasticsearchRecord.REL_FAILED_RECORDS, 2); runner.assertTransferCount(PutElasticsearchRecord.REL_FAILED_RECORDS, 2);
@ -699,8 +699,8 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
public void testNotFoundResponsesTreatedAsFailedRecords() throws Exception { public void testNotFoundResponsesTreatedAsFailedRecords() throws Exception {
runner.setProperty(PutElasticsearchRecord.NOT_FOUND_IS_SUCCESSFUL, "false"); runner.setProperty(PutElasticsearchRecord.NOT_FOUND_IS_SUCCESSFUL, "false");
runner.setProperty(PutElasticsearchRecord.GROUP_BULK_ERRORS_BY_TYPE, "false"); runner.setProperty(PutElasticsearchRecord.GROUP_BULK_ERRORS_BY_TYPE, "false");
int errorCount = 4; final int errorCount = 4;
int successCount = 3; final int successCount = 3;
testErrorRelationship(errorCount, successCount, true); testErrorRelationship(errorCount, successCount, true);
runner.assertTransferCount(PutElasticsearchRecord.REL_FAILED_RECORDS, 1); runner.assertTransferCount(PutElasticsearchRecord.REL_FAILED_RECORDS, 1);
@ -717,8 +717,8 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
public void testNotFoundFailedRecordsGroupedAsErrorType() throws Exception { public void testNotFoundFailedRecordsGroupedAsErrorType() throws Exception {
runner.setProperty(PutElasticsearchRecord.NOT_FOUND_IS_SUCCESSFUL, "false"); runner.setProperty(PutElasticsearchRecord.NOT_FOUND_IS_SUCCESSFUL, "false");
runner.setProperty(PutElasticsearchRecord.GROUP_BULK_ERRORS_BY_TYPE, "true"); runner.setProperty(PutElasticsearchRecord.GROUP_BULK_ERRORS_BY_TYPE, "true");
int errorCount = 4; final int errorCount = 4;
int successCount = 3; final int successCount = 3;
testErrorRelationship(errorCount, successCount, true); testErrorRelationship(errorCount, successCount, true);
runner.assertTransferCount(PutElasticsearchRecord.REL_FAILED_RECORDS, 3); runner.assertTransferCount(PutElasticsearchRecord.REL_FAILED_RECORDS, 3);
@ -746,8 +746,8 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
runner.setProperty(PutElasticsearchRecord.NOT_FOUND_IS_SUCCESSFUL, "false"); runner.setProperty(PutElasticsearchRecord.NOT_FOUND_IS_SUCCESSFUL, "false");
runner.setProperty(PutElasticsearchRecord.OUTPUT_ERROR_RESPONSES, "true"); runner.setProperty(PutElasticsearchRecord.OUTPUT_ERROR_RESPONSES, "true");
runner.setProperty(PutElasticsearchRecord.GROUP_BULK_ERRORS_BY_TYPE, "false"); runner.setProperty(PutElasticsearchRecord.GROUP_BULK_ERRORS_BY_TYPE, "false");
int errorCount = 4; final int errorCount = 4;
int successCount = 3; final int successCount = 3;
testErrorRelationship(errorCount, successCount, false); testErrorRelationship(errorCount, successCount, false);
runner.assertTransferCount(PutElasticsearchRecord.REL_FAILED_RECORDS, 0); runner.assertTransferCount(PutElasticsearchRecord.REL_FAILED_RECORDS, 0);
runner.assertTransferCount(PutElasticsearchRecord.REL_SUCCESSFUL_RECORDS, 0); runner.assertTransferCount(PutElasticsearchRecord.REL_SUCCESSFUL_RECORDS, 0);
@ -766,21 +766,21 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
runner.assertNotValid(); runner.assertNotValid();
} }
private static RecordSchema getRecordSchema(Path schema) throws IOException { private static RecordSchema getRecordSchema(final Path schema) throws IOException {
return AvroTypeUtil.createSchema(new Schema.Parser().parse(Files.readString(schema))); return AvroTypeUtil.createSchema(new Schema.Parser().parse(Files.readString(schema)));
} }
private void testErrorRelationship(final int errorCount, final int successCount, boolean recordWriter) throws Exception { private void testErrorRelationship(final int errorCount, final int successCount, final boolean recordWriter) throws Exception {
final String schemaName = "errorTest"; final String schemaName = "errorTest";
final JsonRecordSetWriter writer = new JsonRecordSetWriter(); final JsonRecordSetWriter writer = new JsonRecordSetWriter();
runner.addControllerService("writer", writer); runner.addControllerService("writer", writer);
runner.setProperty(writer, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.SCHEMA_NAME_PROPERTY); runner.setProperty(writer, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.SCHEMA_NAME_PROPERTY);
runner.setProperty(writer, SchemaAccessUtils.SCHEMA_REGISTRY, "registry"); runner.setProperty(writer, SchemaAccessUtils.SCHEMA_REGISTRY, "registry");
runner.enableControllerService(writer); runner.enableControllerService(writer);
clientService.setResponse(IndexOperationResponse.fromJsonResponse(Files.readString(Paths.get(TEST_COMMON_DIR, "sampleErrorResponse.json")))); clientService.setResponse(IndexOperationResponse.fromJsonResponse(JsonUtils.readString(Paths.get(TEST_COMMON_DIR, "sampleErrorResponse.json"))));
registry.addSchema(schemaName, errorTestSchema); registry.addSchema(schemaName, errorTestSchema);
if(recordWriter) { if (recordWriter) {
runner.setProperty(PutElasticsearchRecord.RESULT_RECORD_WRITER, "writer"); runner.setProperty(PutElasticsearchRecord.RESULT_RECORD_WRITER, "writer");
} }
@ -795,13 +795,13 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
assertEquals(1, assertEquals(1,
runner.getProvenanceEvents().stream().filter(e -> ProvenanceEventType.SEND.equals(e.getEventType()) runner.getProvenanceEvents().stream().filter(e -> ProvenanceEventType.SEND.equals(e.getEventType())
&& "1 Elasticsearch _bulk operation batch(es) [%s error(s), %s success(es)]".formatted(errorCount, successCount).equals(e.getDetails())).count()); && String.format("1 Elasticsearch _bulk operation batch(es) [%d error(s), %d success(es)]", errorCount, successCount).equals(e.getDetails())).count());
} }
private void testInvalidELRecordPaths(String idRecordPath, String atTimestampRecordPath, Path path, Map<String, String> attributes) throws IOException { private void testInvalidELRecordPaths(final String idRecordPath, final String atTimestampRecordPath, final Path path, final Map<String, String> attributes) throws IOException {
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> { clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
long nullIdCount = items.stream().filter(item -> item.getId() == null).count(); final long nullIdCount = items.stream().filter(item -> item.getId() == null).count();
long noTimestampCount = items.stream().filter(item -> !item.getFields().containsKey("@timestamp")).count(); final long noTimestampCount = items.stream().filter(item -> !item.getFields().containsKey("@timestamp")).count();
assertEquals(1, nullIdCount, getUnexpectedCountMsg("null id")); assertEquals(1, nullIdCount, getUnexpectedCountMsg("null id"));
assertEquals(1, noTimestampCount, getUnexpectedCountMsg("noTimestamp")); assertEquals(1, noTimestampCount, getUnexpectedCountMsg("noTimestamp"));
}); });
@ -820,7 +820,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
runner.assertTransferCount(PutElasticsearchRecord.REL_ERROR_RESPONSES, 0); runner.assertTransferCount(PutElasticsearchRecord.REL_ERROR_RESPONSES, 0);
} }
private void testWithRequestParametersAndBulkHeaders(Map<String, String> attributes) { private void testWithRequestParametersAndBulkHeaders(final Map<String, String> attributes) {
runner.setProperty("refresh", "true"); runner.setProperty("refresh", "true");
runner.setProperty("slices", "${slices}"); runner.setProperty("slices", "${slices}");
runner.setProperty(AbstractPutElasticsearch.BULK_HEADER_PREFIX + "routing", "/routing"); runner.setProperty(AbstractPutElasticsearch.BULK_HEADER_PREFIX + "routing", "/routing");
@ -829,16 +829,16 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
runner.setEnvironmentVariableValue("blank", " "); runner.setEnvironmentVariableValue("blank", " ");
runner.assertValid(); runner.assertValid();
clientService.setEvalParametersConsumer( (Map<String, String> params) -> { clientService.setEvalParametersConsumer( (final Map<String, String> params) -> {
assertEquals(2, params.size()); assertEquals(2, params.size());
assertEquals("true", params.get("refresh")); assertEquals("true", params.get("refresh"));
assertEquals("auto", params.get("slices")); assertEquals("auto", params.get("slices"));
}); });
Consumer<List<IndexOperationRequest>> consumer = (List<IndexOperationRequest> items) -> { final Consumer<List<IndexOperationRequest>> consumer = (final List<IndexOperationRequest> items) -> {
long headerFieldsCount = items.stream().filter(item -> !item.getHeaderFields().isEmpty()).count(); final long headerFieldsCount = items.stream().filter(item -> !item.getHeaderFields().isEmpty()).count();
long routingCount = items.stream().filter(item -> "1".equals(item.getHeaderFields().get("routing"))).count(); final long routingCount = items.stream().filter(item -> "1".equals(item.getHeaderFields().get("routing"))).count();
long versionCount = items.stream().filter(item -> "external".equals(item.getHeaderFields().get("version"))).count(); final long versionCount = items.stream().filter(item -> "external".equals(item.getHeaderFields().get("version"))).count();
assertEquals(2, headerFieldsCount); assertEquals(2, headerFieldsCount);
assertEquals(1, routingCount); assertEquals(1, routingCount);
assertEquals(1, versionCount); assertEquals(1, versionCount);
@ -848,8 +848,8 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
} }
private String getDateTimeFormattingJson() throws Exception { private String getDateTimeFormattingJson() throws Exception {
String json = Files.readString(Paths.get(TEST_DIR, "10_flowFileContents.json")); final String json = JsonUtils.readString(Paths.get(TEST_DIR, "10_flowFileContents.json"));
List<Map<String, Object>> parsedJson = JsonUtils.readListOfMaps(json); final List<Map<String, Object>> parsedJson = JsonUtils.readListOfMaps(json);
parsedJson.forEach(msg -> { parsedJson.forEach(msg -> {
msg.computeIfPresent("ts", (key, val) -> Timestamp.valueOf(LOCAL_DATE_TIME).toInstant().toEpochMilli()); msg.computeIfPresent("ts", (key, val) -> Timestamp.valueOf(LOCAL_DATE_TIME).toInstant().toEpochMilli());
msg.computeIfPresent("date", (key, val) -> Date.valueOf(LOCAL_DATE).getTime()); msg.computeIfPresent("date", (key, val) -> Date.valueOf(LOCAL_DATE).getTime());

View File

@ -22,12 +22,14 @@ import org.apache.nifi.processors.elasticsearch.api.ResultOutputStrategy;
import org.apache.nifi.state.MockStateManager; import org.apache.nifi.state.MockStateManager;
import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunner;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.EnumSource;
import java.io.IOException; import java.io.IOException;
import java.time.Instant; import java.time.Instant;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail; import static org.junit.jupiter.api.Assertions.fail;
@ -37,73 +39,41 @@ public class SearchElasticsearchTest extends AbstractPaginatedJsonQueryElasticse
AbstractPaginatedJsonQueryElasticsearchTest.setUpBeforeClass(); AbstractPaginatedJsonQueryElasticsearchTest.setUpBeforeClass();
} }
public AbstractPaginatedJsonQueryElasticsearch getProcessor() { AbstractPaginatedJsonQueryElasticsearch getProcessor() {
return new SearchElasticsearch(); return new SearchElasticsearch();
} }
public boolean isStateUsed() { Scope getStateScope() {
return true; return Scope.LOCAL;
} }
public boolean isInput() { boolean isInput() {
return false; return false;
} }
@Test @ParameterizedTest
public void testScrollError() { @EnumSource(PaginationType.class)
final TestRunner runner = createRunner(false); void testPaginationExpiration(final PaginationType paginationType) throws Exception {
final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner);
service.setMaxPages(2);
service.setThrowErrorInSearch(false);
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, PaginationType.SCROLL.getValue());
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllWithSortByMsgWithSizeQuery);
// initialize search
runOnce(runner);
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 1, 0, 0);
runner.clearTransferState();
// scroll (error)
service.setThrowErrorInSearch(true);
runOnce(runner);
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 0, 0, 0);
assertTrue(runner.getLogger().getErrorMessages().stream().anyMatch(logMessage ->
logMessage.getMsg().contains("Could not query documents") && logMessage.getThrowable().getMessage().contains("Simulated IOException - scroll")));
}
@Test
public void testScrollExpiration() throws Exception {
testPaginationExpiration(PaginationType.SCROLL);
}
@Test
public void testPitExpiration() throws Exception {
testPaginationExpiration(PaginationType.POINT_IN_TIME);
}
@Test
public void testSearchAfterExpiration() throws Exception {
testPaginationExpiration(PaginationType.SEARCH_AFTER);
}
private void testPaginationExpiration(final PaginationType paginationType) throws Exception{
// test flowfile per page // test flowfile per page
final TestRunner runner = createRunner(false); final TestRunner runner = createRunner(false);
final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner); final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner);
service.setMaxPages(2); service.setMaxPages(2);
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, paginationType.getValue()); runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, paginationType.getValue());
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_KEEP_ALIVE, "1 sec"); runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_KEEP_ALIVE, "1 sec");
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllWithSortByMsgWithSizeQuery); setQuery(runner, matchAllWithSortByMsgWithSizeQuery);
// first page // first page
runOnce(runner); runOnce(runner);
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 1, 0, 0); AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 1, 0, 0);
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "10"); runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "10");
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("page.number", "1"); runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("page.number", "1");
assertState(runner.getStateManager(), paginationType, 10, 1); assertState(runner, paginationType, 10, 1);
if (runner.getProcessor() instanceof ConsumeElasticsearch) {
assertFalse(getService(runner).getQuery().contains("\"five\""));
}
// wait for expiration // wait for expiration
final Instant expiration = Instant.ofEpochMilli(Long.parseLong(runner.getStateManager().getState(Scope.LOCAL).get(SearchElasticsearch.STATE_PAGE_EXPIRATION_TIMESTAMP))); final Instant expiration = Instant.ofEpochMilli(Long.parseLong(runner.getStateManager().getState(getStateScope()).get(SearchElasticsearch.STATE_PAGE_EXPIRATION_TIMESTAMP)));
while (expiration.isAfter(Instant.now())) { while (expiration.isAfter(Instant.now())) {
Thread.sleep(10); Thread.sleep(10);
} }
@ -121,7 +91,11 @@ public class SearchElasticsearchTest extends AbstractPaginatedJsonQueryElasticse
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 1, 0, 0); AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 1, 0, 0);
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "10"); runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "10");
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("page.number", "1"); runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("page.number", "1");
assertState(runner.getStateManager(), paginationType, 10, 1); assertState(runner, paginationType, 10, 1);
if (runner.getProcessor() instanceof ConsumeElasticsearch) {
// trackingRangeValue should be retained after previous query expiry
assertTrue(getService(runner).getQuery().contains("\"five\""));
}
runner.clearTransferState(); runner.clearTransferState();
// second page // second page
@ -129,63 +103,81 @@ public class SearchElasticsearchTest extends AbstractPaginatedJsonQueryElasticse
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 1, 0, 0); AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 1, 0, 0);
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "10"); runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "10");
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("page.number", "2"); runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("page.number", "2");
assertState(runner.getStateManager(), paginationType, 20, 2); assertState(runner, paginationType, 20, 2);
if (runner.getProcessor() instanceof ConsumeElasticsearch) {
assertTrue(getService(runner).getQuery().contains("\"five\""));
}
runner.clearTransferState(); runner.clearTransferState();
} }
@Override @Override
void validatePagination(final TestRunner runner, final ResultOutputStrategy resultOutputStrategy, final PaginationType paginationType, int iteration) throws IOException { void validatePagination(final TestRunner runner, final ResultOutputStrategy resultOutputStrategy, final PaginationType paginationType, final int iteration) throws IOException {
boolean perResponseResultOutputStrategy = ResultOutputStrategy.PER_RESPONSE.equals(resultOutputStrategy); final boolean perResponseResultOutputStrategy = ResultOutputStrategy.PER_RESPONSE.equals(resultOutputStrategy);
boolean perHitResultOutputStrategy = ResultOutputStrategy.PER_HIT.equals(resultOutputStrategy); final boolean perHitResultOutputStrategy = ResultOutputStrategy.PER_HIT.equals(resultOutputStrategy);
final int expectedHitCount = 10 * iteration; final int expectedHitCount = 10 * iteration;
if (perResponseResultOutputStrategy && (iteration == 1 || iteration == 2)) { if (perResponseResultOutputStrategy && (iteration == 1 || iteration == 2)) {
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 1, 0, 0); AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 1, 0, 0);
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "10"); runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "10");
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("page.number", String.valueOf(iteration)); runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("page.number", String.valueOf(iteration));
assertState(runner.getStateManager(), paginationType, expectedHitCount, iteration); assertState(runner, paginationType, expectedHitCount, iteration);
} else if (perHitResultOutputStrategy && (iteration == 1 || iteration == 2)) { } else if (perHitResultOutputStrategy && (iteration == 1 || iteration == 2)) {
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 10, 0, 0); AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 10, 0, 0);
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).forEach(hit -> { runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).forEach(hit -> {
hit.assertAttributeEquals("hit.count", "1"); hit.assertAttributeEquals("hit.count", "1");
hit.assertAttributeEquals("page.number", String.valueOf(iteration)); hit.assertAttributeEquals("page.number", String.valueOf(iteration));
}); });
assertState(runner.getStateManager(), paginationType, expectedHitCount, iteration); assertState(runner, paginationType, expectedHitCount, iteration);
} else if ((perResponseResultOutputStrategy || perHitResultOutputStrategy) && iteration == 3) { } else if ((perResponseResultOutputStrategy || perHitResultOutputStrategy) && iteration == 3) {
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 0, 0, 0); AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 0, 0, 0);
assertTrue(runner.getStateManager().getState(Scope.LOCAL).toMap().isEmpty()); if (runner.getProcessor() instanceof ConsumeElasticsearch) {
assertEquals("five", runner.getStateManager().getState(getStateScope()).get(ConsumeElasticsearch.STATE_RANGE_VALUE));
} else {
assertTrue(runner.getStateManager().getState(getStateScope()).toMap().isEmpty());
}
} else if (ResultOutputStrategy.PER_QUERY.equals(resultOutputStrategy)) { } else if (ResultOutputStrategy.PER_QUERY.equals(resultOutputStrategy)) {
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 1, 0, 0); AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 1, 0, 0);
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "20"); runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "20");
// the "last" page.number is used, so 2 here because there were 2 pages of hits // the "last" page.number is used, so 2 here because there were 2 pages of hits
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("page.number", "2"); runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("page.number", "2");
assertEquals(20, runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).getContent().split("\n").length); assertEquals(20, runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).getContent().split("\n").length);
assertTrue(runner.getStateManager().getState(Scope.LOCAL).toMap().isEmpty()); if (runner.getProcessor() instanceof ConsumeElasticsearch) {
assertEquals("five", runner.getStateManager().getState(getStateScope()).get(ConsumeElasticsearch.STATE_RANGE_VALUE));
} else {
assertTrue(runner.getStateManager().getState(getStateScope()).toMap().isEmpty());
}
} }
} }
private void assertState(final MockStateManager stateManager, final PaginationType paginationType, final int hitCount, final int pageCount) throws IOException { private void assertState(final TestRunner runner, final PaginationType paginationType, final int hitCount, final int pageCount) throws IOException {
stateManager.assertStateEquals(SearchElasticsearch.STATE_HIT_COUNT, Integer.toString(hitCount), Scope.LOCAL); final MockStateManager stateManager = runner.getStateManager();
stateManager.assertStateEquals(SearchElasticsearch.STATE_PAGE_COUNT, Integer.toString(pageCount), Scope.LOCAL);
final String pageExpirationTimestamp = stateManager.getState(Scope.LOCAL).get(SearchElasticsearch.STATE_PAGE_EXPIRATION_TIMESTAMP); stateManager.assertStateEquals(SearchElasticsearch.STATE_HIT_COUNT, Integer.toString(hitCount), getStateScope());
stateManager.assertStateEquals(SearchElasticsearch.STATE_PAGE_COUNT, Integer.toString(pageCount), getStateScope());
if (runner.getProcessor() instanceof ConsumeElasticsearch) {
stateManager.assertStateEquals(ConsumeElasticsearch.STATE_RANGE_VALUE, "five", getStateScope());
} else {
stateManager.assertStateNotSet(ConsumeElasticsearch.STATE_RANGE_VALUE, getStateScope());
}
final String pageExpirationTimestamp = stateManager.getState(getStateScope()).get(SearchElasticsearch.STATE_PAGE_EXPIRATION_TIMESTAMP);
assertTrue(Long.parseLong(pageExpirationTimestamp) > Instant.now().toEpochMilli()); assertTrue(Long.parseLong(pageExpirationTimestamp) > Instant.now().toEpochMilli());
switch (paginationType) { switch (paginationType) {
case SCROLL: case SCROLL:
stateManager.assertStateEquals(SearchElasticsearch.STATE_SCROLL_ID, "scrollId-" + pageCount, Scope.LOCAL); stateManager.assertStateEquals(SearchElasticsearch.STATE_SCROLL_ID, "scrollId-" + pageCount, getStateScope());
stateManager.assertStateNotSet(SearchElasticsearch.STATE_PIT_ID, Scope.LOCAL); stateManager.assertStateNotSet(SearchElasticsearch.STATE_PIT_ID, getStateScope());
stateManager.assertStateNotSet(SearchElasticsearch.STATE_SEARCH_AFTER, Scope.LOCAL); stateManager.assertStateNotSet(SearchElasticsearch.STATE_SEARCH_AFTER, getStateScope());
break; break;
case POINT_IN_TIME: case POINT_IN_TIME:
stateManager.assertStateNotSet(SearchElasticsearch.STATE_SCROLL_ID, Scope.LOCAL); stateManager.assertStateNotSet(SearchElasticsearch.STATE_SCROLL_ID, getStateScope());
stateManager.assertStateEquals(SearchElasticsearch.STATE_PIT_ID, "pitId-" + pageCount, Scope.LOCAL); stateManager.assertStateEquals(SearchElasticsearch.STATE_PIT_ID, "pitId-" + pageCount, getStateScope());
stateManager.assertStateEquals(SearchElasticsearch.STATE_SEARCH_AFTER, "[\"searchAfter-" + pageCount + "\"]", Scope.LOCAL); stateManager.assertStateEquals(SearchElasticsearch.STATE_SEARCH_AFTER, "[\"searchAfter-" + pageCount + "\"]", getStateScope());
break; break;
case SEARCH_AFTER: case SEARCH_AFTER:
stateManager.assertStateNotSet(SearchElasticsearch.STATE_SCROLL_ID, Scope.LOCAL); stateManager.assertStateNotSet(SearchElasticsearch.STATE_SCROLL_ID, getStateScope());
stateManager.assertStateNotSet(SearchElasticsearch.STATE_PIT_ID, Scope.LOCAL); stateManager.assertStateNotSet(SearchElasticsearch.STATE_PIT_ID, getStateScope());
stateManager.assertStateEquals(SearchElasticsearch.STATE_SEARCH_AFTER, "[\"searchAfter-" + pageCount + "\"]", Scope.LOCAL); stateManager.assertStateEquals(SearchElasticsearch.STATE_SEARCH_AFTER, "[\"searchAfter-" + pageCount + "\"]", getStateScope());
break; break;
default: default:
fail("Unknown paginationType: " + paginationType); fail("Unknown paginationType: " + paginationType);

View File

@ -29,7 +29,6 @@ import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processors.elasticsearch.mock.MockElasticsearchException; import org.apache.nifi.processors.elasticsearch.mock.MockElasticsearchException;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Paths; import java.nio.file.Paths;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
@ -43,9 +42,9 @@ public class TestElasticsearchClientService extends AbstractControllerService im
static { static {
try { try {
AGGS_RESULT = Files.readString(Paths.get("src/test/resources/TestElasticsearchClientService/aggsResult.json")); AGGS_RESULT = JsonUtils.readString(Paths.get("src/test/resources/TestElasticsearchClientService/aggsResult.json"));
HITS_RESULT = Files.readString(Paths.get("src/test/resources/TestElasticsearchClientService/hitsResult.json")); HITS_RESULT = JsonUtils.readString(Paths.get("src/test/resources/TestElasticsearchClientService/hitsResult.json"));
} catch (IOException e) { } catch (final IOException e) {
throw new RuntimeException(e); throw new RuntimeException(e);
} }
} }
@ -61,11 +60,14 @@ public class TestElasticsearchClientService extends AbstractControllerService im
private int maxPages = 1; private int maxPages = 1;
private Map<String, String> requestParameters; private Map<String, String> requestParameters;
public TestElasticsearchClientService(boolean returnAggs) { private boolean scrolling = false;
private String query;
public TestElasticsearchClientService(final boolean returnAggs) {
this.returnAggs = returnAggs; this.returnAggs = returnAggs;
} }
private void common(boolean throwError, Map<String, String> requestParameters) throws IOException { private void common(final boolean throwError, final Map<String, String> requestParameters) throws IOException {
if (throwError) { if (throwError) {
if (throwNotFoundInGet) { if (throwNotFoundInGet) {
throw new MockElasticsearchException(false, true); throw new MockElasticsearchException(false, true);
@ -78,62 +80,65 @@ public class TestElasticsearchClientService extends AbstractControllerService im
} }
@Override @Override
public List<ConfigVerificationResult> verify(ConfigurationContext context, ComponentLog verificationLogger, Map<String, String> variables) { public List<ConfigVerificationResult> verify(final ConfigurationContext context, final ComponentLog verificationLogger, final Map<String, String> variables) {
return null; return null;
} }
@Override @Override
public IndexOperationResponse add(IndexOperationRequest operation, Map<String, String> requestParameters) { public IndexOperationResponse add(final IndexOperationRequest operation, final Map<String, String> requestParameters) {
return bulk(Collections.singletonList(operation), requestParameters); return bulk(Collections.singletonList(operation), requestParameters);
} }
@Override @Override
public IndexOperationResponse bulk(List<IndexOperationRequest> operations, Map<String, String> requestParameters) { public IndexOperationResponse bulk(final List<IndexOperationRequest> operations, final Map<String, String> requestParameters) {
try { try {
common(false, requestParameters); common(false, requestParameters);
} catch (IOException e) { } catch (final IOException e) {
throw new RuntimeException(e); throw new RuntimeException(e);
} }
return new IndexOperationResponse(100L); return new IndexOperationResponse(100L);
} }
@Override @Override
public Long count(String query, String index, String type, Map<String, String> requestParameters) { public Long count(final String query, final String index, final String type, final Map<String, String> requestParameters) {
try { try {
common(false, requestParameters); common(false, requestParameters);
} catch (IOException e) { } catch (final IOException e) {
throw new RuntimeException(e); throw new RuntimeException(e);
} }
this.query = query;
return null; return null;
} }
@Override @Override
public DeleteOperationResponse deleteById(String index, String type, String id, Map<String, String> requestParameters) { public DeleteOperationResponse deleteById(final String index, final String type, final String id, final Map<String, String> requestParameters) {
return deleteById(index, type, Collections.singletonList(id), requestParameters); return deleteById(index, type, Collections.singletonList(id), requestParameters);
} }
@Override @Override
public DeleteOperationResponse deleteById(String index, String type, List<String> ids, Map<String, String> requestParameters) { public DeleteOperationResponse deleteById(final String index, final String type, final List<String> ids, final Map<String, String> requestParameters) {
try { try {
common(throwErrorInDelete, requestParameters); common(throwErrorInDelete, requestParameters);
} catch (IOException e) { } catch (final IOException e) {
throw new RuntimeException(e); throw new RuntimeException(e);
} }
return new DeleteOperationResponse(100L); return new DeleteOperationResponse(100L);
} }
@Override @Override
public DeleteOperationResponse deleteByQuery(String query, String index, String type, Map<String, String> requestParameters) { public DeleteOperationResponse deleteByQuery(final String query, final String index, final String type, final Map<String, String> requestParameters) {
this.query = query;
return deleteById(index, type, Collections.singletonList("1"), requestParameters); return deleteById(index, type, Collections.singletonList("1"), requestParameters);
} }
@Override @Override
public UpdateOperationResponse updateByQuery(String query, String index, String type, Map<String, String> requestParameters) { public UpdateOperationResponse updateByQuery(final String query, final String index, final String type, final Map<String, String> requestParameters) {
try { try {
common(throwErrorInUpdate, requestParameters); common(throwErrorInUpdate, requestParameters);
} catch (IOException e) { } catch (final IOException e) {
throw new RuntimeException(e); throw new RuntimeException(e);
} }
this.query = query;
return new UpdateOperationResponse(100L); return new UpdateOperationResponse(100L);
} }
@ -147,34 +152,37 @@ public class TestElasticsearchClientService extends AbstractControllerService im
} }
@Override @Override
public boolean documentExists(String index, String type, String id, Map<String, String> requestParameters) { public boolean documentExists(final String index, final String type, final String id, final Map<String, String> requestParameters) {
return true; return true;
} }
@Override @Override
public Map<String, Object> get(String index, String type, String id, Map<String, String> requestParameters) { public Map<String, Object> get(final String index, final String type, final String id, final Map<String, String> requestParameters) {
try { try {
common(throwErrorInGet || throwNotFoundInGet, requestParameters); common(throwErrorInGet || throwNotFoundInGet, requestParameters);
} catch (IOException e) { } catch (final IOException e) {
throw new RuntimeException(e); throw new RuntimeException(e);
} }
Map<String, Object> map = new LinkedHashMap<>(1); final Map<String, Object> map = new LinkedHashMap<>(1);
map.put("msg", "one"); map.put("msg", "one");
return map; return map;
} }
@Override @Override
public SearchResponse search(String query, String index, String type, Map<String, String> requestParameters) { public SearchResponse search(final String query, final String index, final String type, final Map<String, String> requestParameters) {
try { try {
common(throwErrorInSearch, requestParameters); common(throwErrorInSearch, requestParameters);
} catch (IOException e) { } catch (final IOException e) {
throw new RuntimeException(e); throw new RuntimeException(e);
} }
if (!scrolling) {
this.query = query;
}
final SearchResponse response; final SearchResponse response;
if (pageCount++ < maxPages) { if (pageCount++ < maxPages) {
List<Map<String, Object>> hits = JsonUtils.readListOfMaps(HITS_RESULT); final List<Map<String, Object>> hits = JsonUtils.readListOfMaps(HITS_RESULT);
Map<String, Object> aggs = returnAggs && pageCount == 1 ? JsonUtils.readMap(AGGS_RESULT) : null; final Map<String, Object> aggs = returnAggs && pageCount == 1 ? JsonUtils.readMap(AGGS_RESULT) : null;
response = new SearchResponse(hits, aggs, "pitId-" + pageCount, "scrollId-" + pageCount, "[\"searchAfter-" + pageCount + "\"]", 15, 5, false, null); response = new SearchResponse(hits, aggs, "pitId-" + pageCount, "scrollId-" + pageCount, "[\"searchAfter-" + pageCount + "\"]", 15, 5, false, null);
} else { } else {
response = new SearchResponse(new ArrayList<>(), new LinkedHashMap<>(), "pitId-" + pageCount, "scrollId-" + pageCount, "[\"searchAfter-" + pageCount + "\"]", 0, 1, false, null); response = new SearchResponse(new ArrayList<>(), new LinkedHashMap<>(), "pitId-" + pageCount, "scrollId-" + pageCount, "[\"searchAfter-" + pageCount + "\"]", 0, 1, false, null);
@ -184,17 +192,19 @@ public class TestElasticsearchClientService extends AbstractControllerService im
} }
@Override @Override
public SearchResponse scroll(String scroll) { public SearchResponse scroll(final String scroll) {
if (throwErrorInSearch) { if (throwErrorInSearch) {
throw new RuntimeException(new IOException("Simulated IOException - scroll")); throw new RuntimeException(new IOException("Simulated IOException - scroll"));
} }
scrolling = true;
return search(null, null, null, requestParameters); final SearchResponse response = search(null, null, null, requestParameters);
scrolling = false;
return response;
} }
@Override @Override
public String initialisePointInTime(String index, String keepAlive) { public String initialisePointInTime(final String index, final String keepAlive) {
if (throwErrorInPit) { if (throwErrorInPit) {
throw new RuntimeException(new IOException("Simulated IOException - initialisePointInTime")); throw new RuntimeException(new IOException("Simulated IOException - initialisePointInTime"));
} }
@ -205,7 +215,7 @@ public class TestElasticsearchClientService extends AbstractControllerService im
} }
@Override @Override
public DeleteOperationResponse deletePointInTime(String pitId) { public DeleteOperationResponse deletePointInTime(final String pitId) {
if (throwErrorInDelete) { if (throwErrorInDelete) {
throw new RuntimeException(new IOException("Simulated IOException - deletePointInTime")); throw new RuntimeException(new IOException("Simulated IOException - deletePointInTime"));
} }
@ -214,7 +224,7 @@ public class TestElasticsearchClientService extends AbstractControllerService im
} }
@Override @Override
public DeleteOperationResponse deleteScroll(String scrollId) { public DeleteOperationResponse deleteScroll(final String scrollId) {
if (throwErrorInDelete) { if (throwErrorInDelete) {
throw new RuntimeException(new IOException("Simulated IOException - deleteScroll")); throw new RuntimeException(new IOException("Simulated IOException - deleteScroll"));
} }
@ -227,27 +237,27 @@ public class TestElasticsearchClientService extends AbstractControllerService im
return "http://localhost:9400/" + index + "/" + type; return "http://localhost:9400/" + index + "/" + type;
} }
public void setThrowNotFoundInGet(boolean throwNotFoundInGet) { public void setThrowNotFoundInGet(final boolean throwNotFoundInGet) {
this.throwNotFoundInGet = throwNotFoundInGet; this.throwNotFoundInGet = throwNotFoundInGet;
} }
public void setThrowErrorInGet(boolean throwErrorInGet) { public void setThrowErrorInGet(final boolean throwErrorInGet) {
this.throwErrorInGet = throwErrorInGet; this.throwErrorInGet = throwErrorInGet;
} }
public void setThrowErrorInSearch(boolean throwErrorInSearch) { public void setThrowErrorInSearch(final boolean throwErrorInSearch) {
this.throwErrorInSearch = throwErrorInSearch; this.throwErrorInSearch = throwErrorInSearch;
} }
public void setThrowErrorInDelete(boolean throwErrorInDelete) { public void setThrowErrorInDelete(final boolean throwErrorInDelete) {
this.throwErrorInDelete = throwErrorInDelete; this.throwErrorInDelete = throwErrorInDelete;
} }
public void setThrowErrorInPit(boolean throwErrorInPit) { public void setThrowErrorInPit(final boolean throwErrorInPit) {
this.throwErrorInPit = throwErrorInPit; this.throwErrorInPit = throwErrorInPit;
} }
public void setThrowErrorInUpdate(boolean throwErrorInUpdate) { public void setThrowErrorInUpdate(final boolean throwErrorInUpdate) {
this.throwErrorInUpdate = throwErrorInUpdate; this.throwErrorInUpdate = throwErrorInUpdate;
} }
@ -255,11 +265,15 @@ public class TestElasticsearchClientService extends AbstractControllerService im
this.pageCount = 0; this.pageCount = 0;
} }
public void setMaxPages(int maxPages) { public void setMaxPages(final int maxPages) {
this.maxPages = maxPages; this.maxPages = maxPages;
} }
public Map<String, String> getRequestParameters() { public Map<String, String> getRequestParameters() {
return this.requestParameters; return this.requestParameters;
} }
public String getQuery() {
return query;
}
} }

View File

@ -33,8 +33,8 @@ public class UpdateByQueryElasticsearchTest extends AbstractByQueryElasticsearch
} }
@Override @Override
public Class<? extends AbstractByQueryElasticsearch> getTestProcessor() { public AbstractByQueryElasticsearch getTestProcessor() {
return UpdateByQueryElasticsearch.class; return new UpdateByQueryElasticsearch();
} }
@Override @Override

View File

@ -0,0 +1,36 @@
/*
* 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.integration;
import org.apache.nifi.processors.elasticsearch.DeleteByQueryElasticsearch;
import org.apache.nifi.processors.elasticsearch.ElasticsearchRestProcessor;
import org.apache.nifi.processors.elasticsearch.api.QueryDefinitionType;
import org.junit.jupiter.api.Assumptions;
import org.junit.jupiter.api.Test;
public abstract class AbstractByQueryElasticsearch_IT extends AbstractElasticsearchRestProcessor_IT {
@Test
void testVerifyBuildQueryValidScript() {
Assumptions.assumeFalse(runner.getProcessor() instanceof DeleteByQueryElasticsearch,
"DeleteByQueryElasticsearch does not use the SCRIPT property");
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, QueryDefinitionType.BUILD_QUERY.getValue());
runner.setProperty(ElasticsearchRestProcessor.SCRIPT, "{\"source\": \"ctx._source.num++\", \"lang\": \"painless\"}");
assertQueryVerify(3, 0);
}
}

View File

@ -0,0 +1,143 @@
/*
* 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.integration;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.processor.VerifiableProcessor;
import org.apache.nifi.processors.elasticsearch.ConsumeElasticsearch;
import org.apache.nifi.processors.elasticsearch.ElasticsearchRestProcessor;
import org.apache.nifi.processors.elasticsearch.api.QueryDefinitionType;
import org.junit.jupiter.api.Assumptions;
import org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import static org.junit.jupiter.api.Assertions.assertEquals;
public abstract class AbstractElasticsearchRestProcessor_IT extends AbstractElasticsearch_IT<ElasticsearchRestProcessor> {
private boolean isConsumeElasticsearch() {
return runner.getProcessor() instanceof ConsumeElasticsearch;
}
@Test
void testVerifyFullQueryInvalidJson() {
Assumptions.assumeFalse(isConsumeElasticsearch(), "ConsumeElasticsearch does not use the FULL_QUERY Definition Type");
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, QueryDefinitionType.FULL_QUERY.getValue());
runner.setProperty(ElasticsearchRestProcessor.QUERY, "{\"query\":");
final List<ConfigVerificationResult> results = ((VerifiableProcessor) runner.getProcessor()).verify(
runner.getProcessContext(), runner.getLogger(), Collections.emptyMap()
);
assertEquals(3, results.size());
assertEquals(1, results.stream().filter(result -> result.getOutcome() == ConfigVerificationResult.Outcome.SUCCESSFUL).count(), results.toString());
// check Query JSON result, index covered in AbstractElasticsearch_IT#testVerifyIndexExists
assertEquals(1, results.stream().filter(
result -> Objects.equals(result.getVerificationStepName(), ElasticsearchRestProcessor.VERIFICATION_STEP_QUERY_JSON_VALID)
&& result.getExplanation().startsWith("Query cannot be parsed as valid JSON: Unexpected end-of-input within/between Object entries")
&& result.getOutcome() == ConfigVerificationResult.Outcome.FAILED).count(),
results.toString()
);
// check Query result
assertEquals(1, results.stream().filter(
result -> Objects.equals(result.getVerificationStepName(), ElasticsearchRestProcessor.VERIFICATION_STEP_QUERY_VALID)
&& Objects.equals(result.getExplanation(), "Query JSON could not be parsed")
&& result.getOutcome() == ConfigVerificationResult.Outcome.SKIPPED).count(),
results.toString()
);
}
@Test
void testVerifyFullQueryValid() {
Assumptions.assumeFalse(isConsumeElasticsearch(), "ConsumeElasticsearch does not use the FULL_QUERY Definition Type");
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, QueryDefinitionType.FULL_QUERY.getValue());
runner.setProperty(ElasticsearchRestProcessor.QUERY, "{\"query\":{\"term\":{\"msg\":\"one\"}}, \"aggs\":{\"messages\":{\"terms\":{\"field\":\"msg\"}}}}");
assertQueryVerify(1, 1);
}
@Test
void testVerifyFullQueryValidEmptyQuery() {
Assumptions.assumeFalse(isConsumeElasticsearch(), "ConsumeElasticsearch does not use the FULL_QUERY Definition Type");
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, QueryDefinitionType.FULL_QUERY.getValue());
runner.removeProperty(ElasticsearchRestProcessor.QUERY); // should run a default "match_all" query
assertQueryVerify(3, 0);
}
@Test
void testVerifyFullQueryInvalid() {
Assumptions.assumeFalse(isConsumeElasticsearch(), "ConsumeElasticsearch does not use the FULL_QUERY Definition Type");
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, QueryDefinitionType.FULL_QUERY.getValue());
runner.setProperty(ElasticsearchRestProcessor.QUERY, "{\"query\":{\"unknown\":{}}}");
final List<ConfigVerificationResult> results = assertVerify(2);
// check Query result
assertEquals(1, results.stream().filter(
result -> Objects.equals(result.getVerificationStepName(), ElasticsearchRestProcessor.VERIFICATION_STEP_QUERY_VALID)
&& result.getExplanation().startsWith("Query failed in Elasticsearch: ")
&& result.getExplanation().contains("[unknown]")
&& result.getOutcome() == ConfigVerificationResult.Outcome.FAILED).count(),
results.toString()
);
}
@Test
void testVerifyBuildQueryValidQueryClause() {
Assumptions.assumeFalse(isConsumeElasticsearch(), "ConsumeElasticsearch does not use the QUERY_CLAUSE");
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, QueryDefinitionType.BUILD_QUERY.getValue());
runner.setProperty(ElasticsearchRestProcessor.QUERY_CLAUSE, "{\"term\":{\"msg\":\"one\"}}");
assertQueryVerify(1, 0);
}
List<ConfigVerificationResult> assertVerify(final int numSuccess) {
final List<ConfigVerificationResult> results = ((VerifiableProcessor) runner.getProcessor()).verify(
runner.getProcessContext(), runner.getLogger(), Collections.emptyMap()
);
assertEquals(3, results.size());
assertEquals(numSuccess, results.stream().filter(result -> result.getOutcome() == ConfigVerificationResult.Outcome.SUCCESSFUL).count(), results.toString());
// check Query JSON result, index covered in AbstractElasticsearch_IT#testVerifyIndexExists
assertEquals(1, results.stream().filter(
result -> Objects.equals(result.getVerificationStepName(), ElasticsearchRestProcessor.VERIFICATION_STEP_QUERY_JSON_VALID)
&& Objects.equals(result.getExplanation(), "Query JSON successfully parsed")
&& result.getOutcome() == ConfigVerificationResult.Outcome.SUCCESSFUL).count(),
results.toString()
);
return results;
}
void assertQueryVerify(final int numHits, final int numAggs) {
final List<ConfigVerificationResult> results = assertVerify(3);
// check Query result
assertEquals(1, results.stream().filter(
result -> Objects.equals(result.getVerificationStepName(), ElasticsearchRestProcessor.VERIFICATION_STEP_QUERY_VALID)
&& result.getExplanation().matches(String.format("Query found %d hits and %d aggregations in \\d+ milliseconds, timed out: false", numHits, numAggs))
&& result.getOutcome() == ConfigVerificationResult.Outcome.SUCCESSFUL).count(),
results.toString()
);
}
}

View File

@ -20,7 +20,6 @@ import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.elasticsearch.ElasticSearchClientService; import org.apache.nifi.elasticsearch.ElasticSearchClientService;
import org.apache.nifi.elasticsearch.ElasticSearchClientServiceImpl; import org.apache.nifi.elasticsearch.ElasticSearchClientServiceImpl;
import org.apache.nifi.elasticsearch.integration.AbstractElasticsearchITBase; import org.apache.nifi.elasticsearch.integration.AbstractElasticsearchITBase;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.processor.VerifiableProcessor; import org.apache.nifi.processor.VerifiableProcessor;
import org.apache.nifi.processors.elasticsearch.ElasticsearchRestProcessor; import org.apache.nifi.processors.elasticsearch.ElasticsearchRestProcessor;
import org.apache.nifi.util.TestRunners; import org.apache.nifi.util.TestRunners;
@ -37,18 +36,18 @@ import java.util.stream.Collectors;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
abstract class AbstractElasticsearch_IT extends AbstractElasticsearchITBase { abstract class AbstractElasticsearch_IT<P extends ElasticsearchRestProcessor> extends AbstractElasticsearchITBase {
static final List<String> TEST_INDICES = Collections.singletonList("messages"); static final List<String> TEST_INDICES = Collections.singletonList("messages");
static final String ID = "1"; static final String ID = "1";
ElasticSearchClientServiceImpl service; ElasticSearchClientServiceImpl service;
abstract Class<? extends Processor> getTestProcessorClass(); abstract P getProcessor();
@BeforeEach @BeforeEach
void before() throws Exception { void before() throws Exception {
runner = TestRunners.newTestRunner(getTestProcessorClass()); runner = TestRunners.newTestRunner(getProcessor());
service = new ElasticSearchClientServiceImpl(); service = new ElasticSearchClientServiceImpl();
runner.addControllerService(CLIENT_SERVICE_NAME, service); runner.addControllerService(CLIENT_SERVICE_NAME, service);
@ -110,9 +109,8 @@ abstract class AbstractElasticsearch_IT extends AbstractElasticsearchITBase {
assertEquals(1, indexResults.size(), results.toString()); assertEquals(1, indexResults.size(), results.toString());
final ConfigVerificationResult result = indexResults.get(0); final ConfigVerificationResult result = indexResults.get(0);
final ElasticsearchRestProcessor processor = (ElasticsearchRestProcessor) getTestProcessorClass().getConstructor().newInstance();
final ConfigVerificationResult.Outcome expectedOutcome; final ConfigVerificationResult.Outcome expectedOutcome;
if (processor.isIndexNotExistSuccessful()) { if (getProcessor().isIndexNotExistSuccessful()) {
expectedOutcome = ConfigVerificationResult.Outcome.SUCCESSFUL; expectedOutcome = ConfigVerificationResult.Outcome.SUCCESSFUL;
} else { } else {
if (expectedExists) { if (expectedExists) {

View File

@ -0,0 +1,80 @@
/*
* 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.integration;
import org.apache.nifi.processors.elasticsearch.ElasticsearchRestProcessor;
import org.apache.nifi.processors.elasticsearch.api.QueryDefinitionType;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
public abstract class AbstractJsonQueryElasticsearch_IT extends AbstractElasticsearchRestProcessor_IT {
@BeforeEach
public void setUp() {
// set Query Definition Style and default Query Clause for all tests, allowing for ConsumeElasticsearch test override
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, QueryDefinitionType.BUILD_QUERY.getValue());
runner.setProperty(ElasticsearchRestProcessor.QUERY_CLAUSE, "{\"match_all\":{}}");
}
@Test
void testVerifyBuildQueryValidAggregations() {
runner.removeProperty(ElasticsearchRestProcessor.QUERY_CLAUSE);
runner.setProperty(ElasticsearchRestProcessor.AGGREGATIONS, "{\"messages\":{\"terms\":{\"field\":\"msg\"}}}");
assertQueryVerify(3, 1);
}
@Test
void testVerifyBuildQueryValidQueryAndAggregations() {
runner.setProperty(ElasticsearchRestProcessor.AGGREGATIONS, "{\"messages\":{\"terms\":{\"field\":\"msg\"}}}");
assertQueryVerify(3, 1);
}
@Test
void testVerifyBuildQueryValidAll() {
runner.setProperty(ElasticsearchRestProcessor.AGGREGATIONS, "{\"messages\":{\"terms\":{\"field\":\"msg\"}}}");
if (getElasticMajorVersion() > 6) {
// "fields" didn't exist before Elasticsearch 7
runner.setProperty(ElasticsearchRestProcessor.FIELDS, "[\"msg\", \"test*\"]");
}
runner.setProperty(ElasticsearchRestProcessor.SCRIPT_FIELDS, "{\"test1\": {\"script\": {\"lang\": \"painless\", \"source\": \"doc['num'].value * 2\"}}}");
assertQueryVerify(3, 1);
}
@Test
void testVerifyBuildQueryValidSize() {
runner.setProperty(ElasticsearchRestProcessor.SIZE, "1");
// looks a bit odd, but 3 documents match (hits.total); although only 1 is returned (size)
assertQueryVerify(3, 0);
}
@Test
void testVerifyBuildQueryValidSingleSort() {
runner.setProperty(ElasticsearchRestProcessor.SORT, "{\"msg\":\"asc\"}");
assertQueryVerify(3, 0);
}
@Test
void testVerifyBuildQueryValidMultipleSorts() {
runner.setProperty(ElasticsearchRestProcessor.SORT, "[{\"msg\":\"desc\"},{\"num\":\"asc\"}]");
assertQueryVerify(3, 0);
}
}

View File

@ -0,0 +1,77 @@
/*
* 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.integration;
import org.apache.nifi.processors.elasticsearch.ConsumeElasticsearch;
import org.apache.nifi.processors.elasticsearch.ElasticsearchRestProcessor;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
class ConsumeElasticsearch_IT extends AbstractJsonQueryElasticsearch_IT {
private static final String RANGE_FIELD = "num";
private static final String RANGE_SORT_ORDER = "asc";
ElasticsearchRestProcessor getProcessor() {
return new TestConsumeElasticsearch();
}
@BeforeEach
public void setUp() {
// Range Field is required; no Initial Value should result in a default "match_all" query being constructed
runner.setProperty(ConsumeElasticsearch.RANGE_FIELD, RANGE_FIELD);
runner.setProperty(ConsumeElasticsearch.RANGE_FIELD_SORT_ORDER, RANGE_SORT_ORDER);
runner.removeProperty(ConsumeElasticsearch.RANGE_INITIAL_VALUE);
}
@Test
void testVerifyBuildQueryValidInitialValue() {
runner.setProperty(ConsumeElasticsearch.RANGE_INITIAL_VALUE, "2");
assertQueryVerify(1, 0);
}
@Test
void testVerifyBuildQueryValidSortOrder() {
runner.setProperty(ConsumeElasticsearch.RANGE_FIELD_SORT_ORDER, "desc");
assertQueryVerify(3, 0);
}
@Test
void testVerifyBuildQueryValidFormatAndTimeZone() {
runner.setProperty(ConsumeElasticsearch.RANGE_INITIAL_VALUE, "2");
runner.setProperty(ConsumeElasticsearch.RANGE_DATE_FORMAT, "dd");
runner.setProperty(ConsumeElasticsearch.RANGE_TIME_ZONE, "Europe/London");
assertQueryVerify(1, 0);
}
@Test
void testVerifyBuildQueryValidSingleAdditionalFilter() {
runner.setProperty(ConsumeElasticsearch.RANGE_INITIAL_VALUE, "1");
runner.setProperty(ConsumeElasticsearch.ADDITIONAL_FILTERS, "{\"term\":{\"msg\":\"two\"}}");
assertQueryVerify(1, 0);
}
@Test
void testVerifyBuildQueryValidMultipleAdditionalFilters() {
runner.setProperty(ConsumeElasticsearch.ADDITIONAL_FILTERS, "[{\"term\":{\"msg\":\"two\"}},{\"term\":{\"num\":2}}]");
assertQueryVerify(1, 0);
}
}

View File

@ -0,0 +1,26 @@
/*
* 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.integration;
import org.apache.nifi.processors.elasticsearch.DeleteByQueryElasticsearch;
import org.apache.nifi.processors.elasticsearch.ElasticsearchRestProcessor;
public class DeleteByQueryElasticsearch_IT extends AbstractByQueryElasticsearch_IT {
ElasticsearchRestProcessor getProcessor() {
return new DeleteByQueryElasticsearch();
}
}

View File

@ -17,7 +17,6 @@
package org.apache.nifi.processors.elasticsearch.integration; package org.apache.nifi.processors.elasticsearch.integration;
import org.apache.nifi.components.ConfigVerificationResult; import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.processor.VerifiableProcessor; import org.apache.nifi.processor.VerifiableProcessor;
import org.apache.nifi.processors.elasticsearch.ElasticsearchRestProcessor; import org.apache.nifi.processors.elasticsearch.ElasticsearchRestProcessor;
import org.apache.nifi.processors.elasticsearch.GetElasticsearch; import org.apache.nifi.processors.elasticsearch.GetElasticsearch;
@ -31,8 +30,8 @@ import java.util.Objects;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
class GetElasticsearch_IT extends AbstractElasticsearch_IT { class GetElasticsearch_IT extends AbstractElasticsearch_IT {
Class<? extends Processor> getTestProcessorClass() { ElasticsearchRestProcessor getProcessor() {
return GetElasticsearch.class; return new GetElasticsearch();
} }
@BeforeEach @BeforeEach

View File

@ -0,0 +1,26 @@
/*
* 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.integration;
import org.apache.nifi.processors.elasticsearch.ElasticsearchRestProcessor;
import org.apache.nifi.processors.elasticsearch.JsonQueryElasticsearch;
public class JsonQueryElasticsearch_IT extends AbstractJsonQueryElasticsearch_IT {
ElasticsearchRestProcessor getProcessor() {
return new JsonQueryElasticsearch();
}
}

View File

@ -0,0 +1,26 @@
/*
* 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.integration;
import org.apache.nifi.processors.elasticsearch.ElasticsearchRestProcessor;
import org.apache.nifi.processors.elasticsearch.PaginatedJsonQueryElasticsearch;
public class PaginatedJsonQueryElasticsearch_IT extends AbstractJsonQueryElasticsearch_IT {
ElasticsearchRestProcessor getProcessor() {
return new PaginatedJsonQueryElasticsearch();
}
}

View File

@ -16,11 +16,11 @@
*/ */
package org.apache.nifi.processors.elasticsearch.integration; package org.apache.nifi.processors.elasticsearch.integration;
import org.apache.nifi.processor.Processor; import org.apache.nifi.processors.elasticsearch.AbstractPutElasticsearch;
import org.apache.nifi.processors.elasticsearch.PutElasticsearchJson; import org.apache.nifi.processors.elasticsearch.PutElasticsearchJson;
class PutElasticsearchJson_IT extends AbstractElasticsearch_IT { class PutElasticsearchJson_IT extends AbstractElasticsearch_IT<AbstractPutElasticsearch> {
Class<? extends Processor> getTestProcessorClass() { AbstractPutElasticsearch getProcessor() {
return PutElasticsearchJson.class; return new PutElasticsearchJson();
} }
} }

View File

@ -0,0 +1,26 @@
/*
* 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.integration;
import org.apache.nifi.processors.elasticsearch.AbstractPutElasticsearch;
import org.apache.nifi.processors.elasticsearch.PutElasticsearchRecord;
class PutElasticsearchRecord_IT extends AbstractElasticsearch_IT<AbstractPutElasticsearch> {
AbstractPutElasticsearch getProcessor() {
return new PutElasticsearchRecord();
}
}

View File

@ -0,0 +1,26 @@
/*
* 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.integration;
import org.apache.nifi.processors.elasticsearch.ElasticsearchRestProcessor;
import org.apache.nifi.processors.elasticsearch.SearchElasticsearch;
public class SearchElasticsearch_IT extends AbstractJsonQueryElasticsearch_IT {
ElasticsearchRestProcessor getProcessor() {
return new SearchElasticsearch();
}
}

View File

@ -0,0 +1,29 @@
/*
* 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.integration;
import org.apache.nifi.processors.elasticsearch.ConsumeElasticsearch;
public class TestConsumeElasticsearch extends ConsumeElasticsearch {
void setTrackingRangeField(final String trackingRangeField) {
super.trackingRangeField = trackingRangeField;
}
void setTrackingSortOrder(final String trackingSortOrder) {
super.trackingSortOrder = trackingSortOrder;
}
}

View File

@ -0,0 +1,26 @@
/*
* 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.integration;
import org.apache.nifi.processors.elasticsearch.ElasticsearchRestProcessor;
import org.apache.nifi.processors.elasticsearch.UpdateByQueryElasticsearch;
public class UpdateByQueryElasticsearch_IT extends AbstractByQueryElasticsearch_IT {
ElasticsearchRestProcessor getProcessor() {
return new UpdateByQueryElasticsearch();
}
}

View File

@ -33,42 +33,42 @@ import java.util.Map;
public class AbstractMockElasticsearchClient extends AbstractControllerService implements ElasticSearchClientService { public class AbstractMockElasticsearchClient extends AbstractControllerService implements ElasticSearchClientService {
@Override @Override
public List<ConfigVerificationResult> verify(ConfigurationContext context, ComponentLog verificationLogger, Map<String, String> variables) { public List<ConfigVerificationResult> verify(final ConfigurationContext context, final ComponentLog verificationLogger, final Map<String, String> variables) {
return null; return null;
} }
@Override @Override
public IndexOperationResponse add(IndexOperationRequest operation, Map<String, String> requestParameters) { public IndexOperationResponse add(final IndexOperationRequest operation, final Map<String, String> requestParameters) {
return null; return null;
} }
@Override @Override
public IndexOperationResponse bulk(List<IndexOperationRequest> operations, Map<String, String> requestParameters) { public IndexOperationResponse bulk(final List<IndexOperationRequest> operations, final Map<String, String> requestParameters) {
return null; return null;
} }
@Override @Override
public Long count(String query, String index, String type, Map<String, String> requestParameters) { public Long count(final String query, final String index, final String type, final Map<String, String> requestParameters) {
return null; return null;
} }
@Override @Override
public DeleteOperationResponse deleteById(String index, String type, String id, Map<String, String> requestParameters) { public DeleteOperationResponse deleteById(final String index, final String type, final String id, final Map<String, String> requestParameters) {
return null; return null;
} }
@Override @Override
public DeleteOperationResponse deleteById(String index, String type, List<String> ids, Map<String, String> requestParameters) { public DeleteOperationResponse deleteById(final String index, final String type, final List<String> ids, final Map<String, String> requestParameters) {
return null; return null;
} }
@Override @Override
public DeleteOperationResponse deleteByQuery(String query, String index, String type, Map<String, String> requestParameters) { public DeleteOperationResponse deleteByQuery(final String query, final String index, final String type, final Map<String, String> requestParameters) {
return null; return null;
} }
@Override @Override
public UpdateOperationResponse updateByQuery(String query, String index, String type, Map<String, String> requestParameters) { public UpdateOperationResponse updateByQuery(final String query, final String index, final String type, final Map<String, String> requestParameters) {
return null; return null;
} }
@ -82,42 +82,42 @@ public class AbstractMockElasticsearchClient extends AbstractControllerService i
} }
@Override @Override
public boolean documentExists(String index, String type, String id, Map<String, String> requestParameters) { public boolean documentExists(final String index, final String type, final String id, final Map<String, String> requestParameters) {
return true; return true;
} }
@Override @Override
public Map<String, Object> get(String index, String type, String id, Map<String, String> requestParameters) { public Map<String, Object> get(final String index, final String type, final String id, final Map<String, String> requestParameters) {
return null; return null;
} }
@Override @Override
public SearchResponse search(String query, String index, String type, Map<String, String> requestParameters) { public SearchResponse search(final String query, final String index, final String type, final Map<String, String> requestParameters) {
return null; return null;
} }
@Override @Override
public SearchResponse scroll(String scroll) { public SearchResponse scroll(final String scroll) {
return null; return null;
} }
@Override @Override
public String initialisePointInTime(String index, String keepAlive) { public String initialisePointInTime(final String index, final String keepAlive) {
return null; return null;
} }
@Override @Override
public DeleteOperationResponse deletePointInTime(String pitId) { public DeleteOperationResponse deletePointInTime(final String pitId) {
return null; return null;
} }
@Override @Override
public DeleteOperationResponse deleteScroll(String scrollId) { public DeleteOperationResponse deleteScroll(final String scrollId) {
return null; return null;
} }
@Override @Override
public String getTransitUrl(String index, String type) { public String getTransitUrl(final String index, final String type) {
return String.format("http://localhost:9200/%s/%s", index, StringUtils.isNotBlank(type) ? type : ""); return String.format("http://localhost:9200/%s/%s", index, StringUtils.isNotBlank(type) ? type : "");
} }
@ -129,7 +129,7 @@ public class AbstractMockElasticsearchClient extends AbstractControllerService i
return throwRetriableError; return throwRetriableError;
} }
public void setThrowRetriableError(boolean throwRetriableError) { public void setThrowRetriableError(final boolean throwRetriableError) {
this.throwRetriableError = throwRetriableError; this.throwRetriableError = throwRetriableError;
} }
@ -141,7 +141,7 @@ public class AbstractMockElasticsearchClient extends AbstractControllerService i
return throwFatalError; return throwFatalError;
} }
public void setThrowFatalError(boolean throwFatalError) { public void setThrowFatalError(final boolean throwFatalError) {
this.throwFatalError = throwFatalError; this.throwFatalError = throwFatalError;
} }

View File

@ -29,7 +29,7 @@ public class MockBulkLoadClientService extends AbstractMockElasticsearchClient {
private Consumer<Map<String, String>> evalParametersConsumer; private Consumer<Map<String, String>> evalParametersConsumer;
@Override @Override
public IndexOperationResponse bulk(List<IndexOperationRequest> items, Map<String, String> requestParameters) { public IndexOperationResponse bulk(final List<IndexOperationRequest> items, final Map<String, String> requestParameters) {
if (getThrowRetriableError()) { if (getThrowRetriableError()) {
throw new MockElasticsearchException(true, false); throw new MockElasticsearchException(true, false);
} else if (getThrowFatalError()) { } else if (getThrowFatalError()) {
@ -47,15 +47,15 @@ public class MockBulkLoadClientService extends AbstractMockElasticsearchClient {
return response; return response;
} }
public void setResponse(IndexOperationResponse response) { public void setResponse(final IndexOperationResponse response) {
this.response = response; this.response = response;
} }
public void setEvalConsumer(Consumer<List<IndexOperationRequest>> evalConsumer) { public void setEvalConsumer(final Consumer<List<IndexOperationRequest>> evalConsumer) {
this.evalConsumer = evalConsumer; this.evalConsumer = evalConsumer;
} }
public void setEvalParametersConsumer(Consumer<Map<String, String>> evalParametersConsumer) { public void setEvalParametersConsumer(final Consumer<Map<String, String>> evalParametersConsumer) {
this.evalParametersConsumer = evalParametersConsumer; this.evalParametersConsumer = evalParametersConsumer;
} }
} }

View File

@ -19,13 +19,13 @@ package org.apache.nifi.processors.elasticsearch.mock;
import org.apache.nifi.elasticsearch.ElasticsearchException; import org.apache.nifi.elasticsearch.ElasticsearchException;
public class MockElasticsearchException extends ElasticsearchException { public class MockElasticsearchException extends ElasticsearchException {
public MockElasticsearchException(boolean elastic, boolean notFound) { public MockElasticsearchException(final boolean elastic, final boolean notFound) {
this(new Exception()); this(new Exception());
this.elastic = elastic; this.elastic = elastic;
this.notFound = notFound; this.notFound = notFound;
} }
public MockElasticsearchException(Exception ex) { public MockElasticsearchException(final Exception ex) {
super(ex); super(ex);
} }
} }

View File

@ -14,8 +14,9 @@
# limitations under the License. # limitations under the License.
#create mapping #create mapping
PUT:messages/:{ "mappings":{"_doc":{ "properties":{ "msg":{"type":"keyword"}}}}} PUT:messages/:{ "mappings":{"_doc":{ "properties":{ "msg":{"type":"keyword"}, "num":{"type":"integer"}}}}}
#add document #add document
PUT:messages/_doc/1:{ "msg":"one" } PUT:messages/_doc/1:{ "msg":"one", "num": 1 }
PUT:messages/_doc/2:{ "msg":"two", "num": 2 }
PUT:messages/_doc/3:{ "msg":"three", "num": 3 }

View File

@ -14,7 +14,9 @@
# limitations under the License. # limitations under the License.
#create mapping #create mapping
PUT:messages/:{ "mappings":{ "properties":{ "msg":{"type":"keyword"}}}} PUT:messages/:{ "mappings":{ "properties":{ "msg":{"type":"keyword"}, "num":{"type":"integer"}}}}
#add document #add document
POST:messages/_doc/1:{ "msg":"one" } POST:messages/_doc/1:{ "msg":"one", "num": 1 }
POST:messages/_doc/2:{ "msg":"two", "num": 2 }
POST:messages/_doc/3:{ "msg":"three", "num": 3 }

View File

@ -14,7 +14,9 @@
# limitations under the License. # limitations under the License.
#create mapping #create mapping
PUT:messages/:{ "mappings":{ "properties":{ "msg":{"type":"keyword"}}}} PUT:messages/:{ "mappings":{ "properties":{ "msg":{"type":"keyword"}, "num":{"type":"integer"}}}}
#add document #add document
POST:messages/_doc/1:{ "msg":"one" } POST:messages/_doc/1:{ "msg":"one", "num": 1 }
POST:messages/_doc/2:{ "msg":"two", "num": 2 }
POST:messages/_doc/3:{ "msg":"three", "num": 3 }

View File

@ -52,7 +52,7 @@ import static org.apache.http.auth.AuthScope.ANY;
public abstract class AbstractElasticsearchITBase { public abstract class AbstractElasticsearchITBase {
// default Elasticsearch version should (ideally) match that in the nifi-elasticsearch-bundle#pom.xml for the integration-tests profile // default Elasticsearch version should (ideally) match that in the nifi-elasticsearch-bundle#pom.xml for the integration-tests profile
protected static final DockerImageName IMAGE = DockerImageName protected static final DockerImageName IMAGE = DockerImageName
.parse(System.getProperty("elasticsearch.docker.image", "docker.elastic.co/elasticsearch/elasticsearch:8.8.1")); .parse(System.getProperty("elasticsearch.docker.image", "docker.elastic.co/elasticsearch/elasticsearch:8.10.2"));
protected static final String ELASTIC_USER_PASSWORD = System.getProperty("elasticsearch.elastic_user.password", RandomStringUtils.randomAlphanumeric(10, 20)); protected static final String ELASTIC_USER_PASSWORD = System.getProperty("elasticsearch.elastic_user.password", RandomStringUtils.randomAlphanumeric(10, 20));
private static final int PORT = 9200; private static final int PORT = 9200;
protected static final ElasticsearchContainer ELASTICSEARCH_CONTAINER = new ElasticsearchContainer(IMAGE) protected static final ElasticsearchContainer ELASTICSEARCH_CONTAINER = new ElasticsearchContainer(IMAGE)

View File

@ -94,7 +94,7 @@ language governing permissions and limitations under the License. -->
</activation> </activation>
<properties> <properties>
<!-- also update the default Elasticsearch version in nifi-elasticsearch-test-utils#src/main/java/org/apache/nifi/elasticsearch/integration/AbstractElasticsearchITBase.java--> <!-- also update the default Elasticsearch version in nifi-elasticsearch-test-utils#src/main/java/org/apache/nifi/elasticsearch/integration/AbstractElasticsearchITBase.java-->
<elasticsearch_docker_image>8.8.1</elasticsearch_docker_image> <elasticsearch_docker_image>8.10.2</elasticsearch_docker_image>
<elasticsearch.elastic.password>s3cret</elasticsearch.elastic.password> <elasticsearch.elastic.password>s3cret</elasticsearch.elastic.password>
</properties> </properties>
<build> <build>
@ -125,7 +125,7 @@ language governing permissions and limitations under the License. -->
<profile> <profile>
<id>elasticsearch7</id> <id>elasticsearch7</id>
<properties> <properties>
<elasticsearch_docker_image>7.17.10</elasticsearch_docker_image> <elasticsearch_docker_image>7.17.13</elasticsearch_docker_image>
</properties> </properties>
</profile> </profile>
</profiles> </profiles>