mirror of https://github.com/apache/nifi.git
NIFI-11985: Add ConsumeElasticsearch processor
Signed-off-by: Joe Gresock <jgresock@gmail.com> This closes #7671.
This commit is contained in:
parent
2cad5c5ad6
commit
c091347795
|
@ -48,7 +48,8 @@ Execute the following script from the `nifi-elasticsearch-bundle` directory:
|
|||
```bash
|
||||
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)
|
||||
for v in "${es_versions[@]}"; do
|
||||
for m in "${it_modules[@]}"; do
|
||||
|
|
|
@ -145,7 +145,7 @@ class ElasticSearchClientService_IT extends AbstractElasticsearch_IT {
|
|||
@Test
|
||||
void testVerifyFailedURL() {
|
||||
runner.disableControllerService(service);
|
||||
runner.setProperty(service, ElasticSearchClientService.HTTP_HOSTS, "invalid");
|
||||
runner.setProperty(service, ElasticSearchClientService.HTTP_HOSTS, "blah://invalid");
|
||||
|
||||
final List<ConfigVerificationResult> results = service.verify(
|
||||
new MockConfigurationContext(service, getClientServiceProperties(), runner.getProcessContext().getControllerServiceLookup(), null),
|
||||
|
|
|
@ -51,7 +51,7 @@ public abstract class AbstractByQueryElasticsearch extends AbstractProcessor imp
|
|||
.build();
|
||||
|
||||
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);
|
||||
|
||||
|
@ -63,13 +63,16 @@ public abstract class AbstractByQueryElasticsearch extends AbstractProcessor imp
|
|||
relationships = Collections.unmodifiableSet(rels);
|
||||
|
||||
final List<PropertyDescriptor> descriptors = new ArrayList<>();
|
||||
descriptors.add(QUERY_DEFINITION_STYLE);
|
||||
descriptors.add(QUERY);
|
||||
descriptors.add(QUERY_CLAUSE);
|
||||
descriptors.add(SCRIPT);
|
||||
descriptors.add(QUERY_ATTRIBUTE);
|
||||
descriptors.add(INDEX);
|
||||
descriptors.add(TYPE);
|
||||
descriptors.add(CLIENT_SERVICE);
|
||||
|
||||
propertyDescriptors = Collections.unmodifiableList(descriptors);
|
||||
byQueryPropertyDescriptors = Collections.unmodifiableList(descriptors);
|
||||
}
|
||||
|
||||
abstract String getTookAttribute();
|
||||
|
@ -86,8 +89,8 @@ public abstract class AbstractByQueryElasticsearch extends AbstractProcessor imp
|
|||
}
|
||||
|
||||
@Override
|
||||
public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return propertyDescriptors;
|
||||
public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return byQueryPropertyDescriptors;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
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.OnStopped;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
|
@ -112,7 +111,7 @@ public abstract class AbstractJsonQueryElasticsearch<Q extends JsonQueryParamete
|
|||
.build();
|
||||
|
||||
private static final Set<Relationship> relationships;
|
||||
private static final List<PropertyDescriptor> propertyDescriptors;
|
||||
static final List<PropertyDescriptor> queryPropertyDescriptors;
|
||||
|
||||
ResultOutputStrategy hitStrategy;
|
||||
private SearchResultsFormat hitFormat;
|
||||
|
@ -120,8 +119,6 @@ public abstract class AbstractJsonQueryElasticsearch<Q extends JsonQueryParamete
|
|||
private AggregationResultsFormat aggregationFormat;
|
||||
private boolean outputNoHits;
|
||||
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
|
||||
final AtomicReference<ElasticSearchClientService> clientService = new AtomicReference<>(null);
|
||||
|
||||
static {
|
||||
|
@ -133,7 +130,14 @@ public abstract class AbstractJsonQueryElasticsearch<Q extends JsonQueryParamete
|
|||
relationships = Collections.unmodifiableSet(rels);
|
||||
|
||||
final List<PropertyDescriptor> descriptors = new ArrayList<>();
|
||||
descriptors.add(QUERY_DEFINITION_STYLE);
|
||||
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(INDEX);
|
||||
descriptors.add(TYPE);
|
||||
|
@ -144,7 +148,7 @@ public abstract class AbstractJsonQueryElasticsearch<Q extends JsonQueryParamete
|
|||
descriptors.add(AGGREGATION_RESULTS_FORMAT);
|
||||
descriptors.add(OUTPUT_NO_HITS);
|
||||
|
||||
propertyDescriptors = Collections.unmodifiableList(descriptors);
|
||||
queryPropertyDescriptors = Collections.unmodifiableList(descriptors);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -154,7 +158,7 @@ public abstract class AbstractJsonQueryElasticsearch<Q extends JsonQueryParamete
|
|||
|
||||
@Override
|
||||
public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return propertyDescriptors;
|
||||
return queryPropertyDescriptors;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -183,8 +187,8 @@ public abstract class AbstractJsonQueryElasticsearch<Q extends JsonQueryParamete
|
|||
|
||||
hitStrategy = ResultOutputStrategy.fromValue(context.getProperty(SEARCH_RESULTS_SPLIT).getValue());
|
||||
hitFormat = SearchResultsFormat.valueOf(context.getProperty(SEARCH_RESULTS_FORMAT).getValue());
|
||||
aggregationStrategy = ResultOutputStrategy.fromValue(context.getProperty(AGGREGATION_RESULTS_SPLIT).getValue());
|
||||
aggregationFormat = AggregationResultsFormat.valueOf(context.getProperty(AGGREGATION_RESULTS_FORMAT).getValue());
|
||||
aggregationStrategy = context.getProperty(AGGREGATION_RESULTS_SPLIT).isSet() ? ResultOutputStrategy.fromValue(context.getProperty(AGGREGATION_RESULTS_SPLIT).getValue()) : null;
|
||||
aggregationFormat = context.getProperty(AGGREGATION_RESULTS_FORMAT).isSet() ? AggregationResultsFormat.valueOf(context.getProperty(AGGREGATION_RESULTS_FORMAT).getValue()) : null;
|
||||
|
||||
outputNoHits = context.getProperty(OUTPUT_NO_HITS).asBoolean();
|
||||
}
|
||||
|
|
|
@ -42,6 +42,7 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public abstract class AbstractPaginatedJsonQueryElasticsearch extends AbstractJsonQueryElasticsearch<PaginatedJsonQueryParameters> {
|
||||
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> descriptors = new ArrayList<>();
|
||||
descriptors.add(QUERY_ATTRIBUTE);
|
||||
descriptors.add(INDEX);
|
||||
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);
|
||||
final List<PropertyDescriptor> descriptors = new ArrayList<>(
|
||||
// replace SEARCH_RESULTS_SPLIT property to allow additional output strategies
|
||||
queryPropertyDescriptors.stream().map(pd -> AbstractJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT.equals(pd) ? SEARCH_RESULTS_SPLIT : pd).collect(Collectors.toList())
|
||||
);
|
||||
descriptors.add(PAGINATION_TYPE);
|
||||
descriptors.add(PAGINATION_KEEP_ALIVE);
|
||||
descriptors.add(OUTPUT_NO_HITS);
|
||||
|
||||
paginatedPropertyDescriptors = Collections.unmodifiableList(descriptors);
|
||||
}
|
||||
|
@ -118,10 +113,13 @@ public abstract class AbstractPaginatedJsonQueryElasticsearch extends AbstractJs
|
|||
|
||||
// execute query/scroll
|
||||
final String queryJson = updateQueryJson(newQuery, paginatedJsonQueryParameters);
|
||||
final Map<String, String> requestParameters = getDynamicProperties(context, input);
|
||||
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);
|
||||
} else {
|
||||
final Map<String, String> requestParameters = getDynamicProperties(context, input);
|
||||
if (paginationType == PaginationType.SCROLL) {
|
||||
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)
|
||||
updatePageExpirationTimestamp(paginatedJsonQueryParameters, !response.getHits().isEmpty());
|
||||
updateQueryParameters(paginatedJsonQueryParameters, response);
|
||||
|
||||
hitsFlowFiles = handleResponse(response, newQuery, paginatedJsonQueryParameters, hitsFlowFiles, session, input, stopWatch);
|
||||
} 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,
|
||||
final ProcessSession session, final FlowFile parent, final Map<String, String> attributes,
|
||||
final List<FlowFile> hitsFlowFiles, final String transitUri, final StopWatch stopWatch) throws IOException {
|
||||
paginatedJsonQueryParameters.incrementPageCount();
|
||||
attributes.put("page.number", Integer.toString(paginatedJsonQueryParameters.getPageCount()));
|
||||
|
||||
if (hitStrategy == ResultOutputStrategy.PER_QUERY) {
|
||||
|
||||
final List<Map<String, Object>> formattedHits = formatHits(hits);
|
||||
combineHits(formattedHits, paginatedJsonQueryParameters, session, parent, attributes, hitsFlowFiles, newQuery);
|
||||
|
||||
|
@ -289,8 +284,11 @@ public abstract class AbstractPaginatedJsonQueryElasticsearch extends AbstractJs
|
|||
return hitsFlowFiles;
|
||||
}
|
||||
|
||||
private void updatePageExpirationTimestamp(final PaginatedJsonQueryParameters paginatedJsonQueryParameters, final boolean hasHits) {
|
||||
final String keepAliveDuration = "PT" + (hasHits ? paginatedJsonQueryParameters.getKeepAlive() : "0s");
|
||||
void updateQueryParameters(final PaginatedJsonQueryParameters paginatedJsonQueryParameters, final SearchResponse response) {
|
||||
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(
|
||||
String.valueOf(Instant.now().plus(Duration.parse(keepAliveDuration)).toEpochMilli())
|
||||
);
|
||||
|
|
|
@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.fasterxml.jackson.databind.SerializationFeature;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.annotation.lifecycle.OnStopped;
|
||||
import org.apache.nifi.components.ConfigVerificationResult;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.PropertyValue;
|
||||
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.expression.ExpressionLanguageScope;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
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 ObjectMapper MAPPER = new ObjectMapper();
|
||||
|
||||
boolean logErrors;
|
||||
boolean outputErrorResponses;
|
||||
boolean notFoundIsSuccessful;
|
||||
|
@ -201,6 +201,12 @@ public abstract class AbstractPutElasticsearch extends AbstractProcessor impleme
|
|||
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) {
|
||||
return dynamicProperties.entrySet().stream().filter(e -> !e.getKey().startsWith(BULK_HEADER_PREFIX))
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -23,11 +23,16 @@ import org.apache.nifi.annotation.behavior.WritesAttribute;
|
|||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.elasticsearch.ElasticSearchClientService;
|
||||
import org.apache.nifi.elasticsearch.OperationResponse;
|
||||
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.stream.Collectors;
|
||||
|
||||
@WritesAttributes({
|
||||
@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 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
|
||||
String getTookAttribute() {
|
||||
return TOOK_ATTRIBUTE;
|
||||
|
|
|
@ -17,32 +17,43 @@
|
|||
|
||||
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.PropertyDescriptor;
|
||||
import org.apache.nifi.components.Validator;
|
||||
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.flowfile.FlowFile;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.Processor;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.VerifiableProcessor;
|
||||
import org.apache.nifi.processor.util.JsonValidator;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.processors.elasticsearch.api.QueryDefinitionType;
|
||||
import org.apache.nifi.util.StringUtils;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
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;
|
||||
|
||||
public interface ElasticsearchRestProcessor extends VerifiableProcessor {
|
||||
public interface ElasticsearchRestProcessor extends Processor, VerifiableProcessor {
|
||||
String ATTR_RECORD_COUNT = "record.count";
|
||||
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()
|
||||
.name("el-rest-fetch-index")
|
||||
|
@ -62,11 +73,101 @@ public interface ElasticsearchRestProcessor extends VerifiableProcessor {
|
|||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.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()
|
||||
.name("el-rest-query")
|
||||
.displayName("Query")
|
||||
.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)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.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.")
|
||||
.build();
|
||||
|
||||
default String getQuery(final FlowFile input, final ProcessContext context, final ProcessSession session) throws IOException {
|
||||
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();
|
||||
String DEFAULT_QUERY_JSON = "{}";
|
||||
|
||||
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) {
|
||||
|
@ -189,9 +363,51 @@ public interface ElasticsearchRestProcessor extends VerifiableProcessor {
|
|||
|
||||
boolean isIndexNotExistSuccessful();
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
default 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();
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,12 +17,12 @@
|
|||
|
||||
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.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
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.get.error", description = "The error message provided by Elasticsearch if there is an error fetching the document.")
|
||||
})
|
||||
@SeeAlso(JsonQueryElasticsearch.class)
|
||||
@DynamicProperty(
|
||||
name = "The name of a URL query parameter to add",
|
||||
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
|
||||
)));
|
||||
|
||||
private final ObjectMapper mapper = new ObjectMapper();
|
||||
|
||||
private final AtomicReference<ElasticSearchClientService> clientService = new AtomicReference<>(null);
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,6 +21,7 @@ import org.apache.nifi.annotation.behavior.InputRequirement;
|
|||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.elasticsearch.SearchResponse;
|
||||
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 " +
|
||||
"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.")
|
||||
@SeeAlso(PaginatedJsonQueryElasticsearch.class)
|
||||
@DynamicProperty(
|
||||
name = "The name of a URL query parameter to add",
|
||||
value = "The value of the URL query parameter",
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
|
|||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
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.processors.elasticsearch.api.PaginatedJsonQueryParameters;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
@WritesAttributes({
|
||||
@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.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 = "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. " +
|
||||
"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.")
|
||||
@SeeAlso({JsonQueryElasticsearch.class, ConsumeElasticsearch.class, SearchElasticsearch.class})
|
||||
@DynamicProperty(
|
||||
name = "The name of a URL query parameter to add",
|
||||
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 " +
|
||||
"from Elasticsearch will be loaded into memory all at once and converted into the resulting flowfiles.")
|
||||
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
|
||||
public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return propertyDescriptors;
|
||||
return paginatedPropertyDescriptors;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
package org.apache.nifi.processors.elasticsearch;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperties;
|
||||
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.WritesAttributes;
|
||||
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.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"),
|
||||
@WritesAttribute(attribute = "elasticsearch.bulk.error", description = "The _bulk response if there was an error during processing the document within Elasticsearch.")
|
||||
})
|
||||
@SeeAlso(PutElasticsearchRecord.class)
|
||||
@DynamicProperties({
|
||||
@DynamicProperty(
|
||||
name = "The name of the Bulk request header",
|
||||
|
@ -179,7 +180,6 @@ public class PutElasticsearchJson extends AbstractPutElasticsearch {
|
|||
)));
|
||||
|
||||
private boolean outputErrors;
|
||||
private final ObjectMapper objectMapper = new ObjectMapper();
|
||||
|
||||
@Override
|
||||
Set<Relationship> getBaseRelationships() {
|
||||
|
@ -261,7 +261,7 @@ public class PutElasticsearchJson extends AbstractPutElasticsearch {
|
|||
try (final InputStream inStream = session.read(input)) {
|
||||
final byte[] result = IOUtils.toByteArray(inStream);
|
||||
@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);
|
||||
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) {
|
||||
final String dynamicTemplates = context.getProperty(propertyDescriptor).evaluateAttributeExpressions(input).getValue();
|
||||
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) {
|
||||
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) -> {
|
||||
String errorMessage;
|
||||
try {
|
||||
errorMessage = objectMapper.writeValueAsString(error);
|
||||
errorMessage = mapper.writeValueAsString(error);
|
||||
} catch (JsonProcessingException e) {
|
||||
errorMessage = String.format(
|
||||
"{\"error\": {\"type\": \"elasticsearch_response_parse_error\", \"reason\": \"%s\"}}",
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
|
|||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
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.bulk.error", description = "The _bulk response if there was an error during processing the record within Elasticsearch.")
|
||||
})
|
||||
@SeeAlso(PutElasticsearchJson.class)
|
||||
@DynamicProperties({
|
||||
@DynamicProperty(
|
||||
name = "The name of the Bulk request header",
|
||||
|
@ -460,7 +462,7 @@ public class PutElasticsearchRecord extends AbstractPutElasticsearch {
|
|||
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.success.count", String.valueOf(successfulRecords.get()));
|
||||
}});
|
||||
|
@ -666,7 +668,7 @@ public class PutElasticsearchRecord extends AbstractPutElasticsearch {
|
|||
map = DataTypeUtils.toMap(fieldValue.getValue(), path.getPath());
|
||||
} else {
|
||||
try {
|
||||
map = MAPPER.readValue(fieldValue.getValue().toString(), Map.class);
|
||||
map = mapper.readValue(fieldValue.getValue().toString(), Map.class);
|
||||
} catch (final JsonProcessingException jpe) {
|
||||
getLogger().error("Unable to parse field {} as Map", path.getPath(), jpe);
|
||||
throw new ProcessException(
|
||||
|
@ -825,7 +827,7 @@ public class PutElasticsearchRecord extends AbstractPutElasticsearch {
|
|||
this.writer.write(record);
|
||||
if (errorType != null && exampleError == null && error != null) {
|
||||
try {
|
||||
exampleError = MAPPER.writeValueAsString(error);
|
||||
exampleError = mapper.writeValueAsString(error);
|
||||
} catch (JsonProcessingException e) {
|
||||
exampleError = String.format(
|
||||
"{\"error\": {\"type\": \"elasticsearch_response_parse_error\", \"reason\": \"%s\"}}",
|
||||
|
|
|
@ -27,6 +27,7 @@ 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.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.state.Scope;
|
||||
|
@ -50,12 +51,13 @@ import java.util.HashSet;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@WritesAttributes({
|
||||
@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.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 = "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 " +
|
||||
"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.")
|
||||
@SeeAlso({PaginatedJsonQueryElasticsearch.class, ConsumeElasticsearch.class})
|
||||
@DynamicProperty(
|
||||
name = "The name of a URL query parameter to add",
|
||||
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. " +
|
||||
"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 " +
|
||||
|
@ -90,14 +93,13 @@ public class SearchElasticsearch extends AbstractPaginatedJsonQueryElasticsearch
|
|||
static final String STATE_HIT_COUNT = "hitCount";
|
||||
|
||||
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\"}}}.")
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
|
||||
.required(true)
|
||||
.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.")
|
||||
.build();
|
||||
|
||||
private static final Set<Relationship> relationships;
|
||||
private static final List<PropertyDescriptor> propertyDescriptors;
|
||||
|
||||
static final List<PropertyDescriptor> scrollPropertyDescriptors;
|
||||
|
||||
static {
|
||||
final Set<Relationship> rels = new HashSet<>();
|
||||
|
@ -106,10 +108,14 @@ public class SearchElasticsearch extends AbstractPaginatedJsonQueryElasticsearch
|
|||
relationships = Collections.unmodifiableSet(rels);
|
||||
|
||||
final List<PropertyDescriptor> descriptors = new ArrayList<>();
|
||||
descriptors.add(SearchElasticsearch.QUERY);
|
||||
descriptors.addAll(paginatedPropertyDescriptors);
|
||||
|
||||
propertyDescriptors = Collections.unmodifiableList(descriptors);
|
||||
// ensure QUERY_DEFINITION_STYLE first for consistency between Elasticsearch processors
|
||||
descriptors.add(QUERY_DEFINITION_STYLE);
|
||||
descriptors.add(QUERY);
|
||||
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
|
||||
|
@ -118,15 +124,19 @@ public class SearchElasticsearch extends AbstractPaginatedJsonQueryElasticsearch
|
|||
}
|
||||
|
||||
@Override
|
||||
public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return propertyDescriptors;
|
||||
public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return scrollPropertyDescriptors;
|
||||
}
|
||||
|
||||
Scope getStateScope() {
|
||||
return Scope.LOCAL;
|
||||
}
|
||||
|
||||
@Override
|
||||
PaginatedJsonQueryParameters buildJsonQueryParameters(final FlowFile input, final ProcessContext context, final ProcessSession session) throws IOException {
|
||||
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.setPageCount(stateMap.get(STATE_PAGE_COUNT) == null ? 0 : Integer.parseInt(stateMap.get(STATE_PAGE_COUNT)));
|
||||
paginatedQueryJsonParameters.setScrollId(stateMap.get(STATE_SCROLL_ID));
|
||||
|
@ -138,15 +148,16 @@ public class SearchElasticsearch extends AbstractPaginatedJsonQueryElasticsearch
|
|||
}
|
||||
|
||||
@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 {
|
||||
if (response.getHits().isEmpty()) {
|
||||
getLogger().debug("No more results for paginated query, resetting local state for future queries");
|
||||
resetProcessorState(context);
|
||||
} else {
|
||||
getLogger().debug("Updating local state for next execution");
|
||||
final Map<String, String> newStateMap = new HashMap<>(10, 1);
|
||||
additionalState(newStateMap, paginatedJsonQueryParameters);
|
||||
|
||||
if (response.getHits().isEmpty()) {
|
||||
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) {
|
||||
newStateMap.put(STATE_SCROLL_ID, response.getScrollId());
|
||||
} else {
|
||||
|
@ -156,11 +167,15 @@ public class SearchElasticsearch extends AbstractPaginatedJsonQueryElasticsearch
|
|||
newStateMap.put(STATE_PIT_ID, response.getPitId());
|
||||
}
|
||||
}
|
||||
newStateMap.put(STATE_HIT_COUNT, Integer.toString(paginatedQueryJsonParameters.getHitCount()));
|
||||
newStateMap.put(STATE_PAGE_COUNT, Integer.toString(paginatedQueryJsonParameters.getPageCount()));
|
||||
newStateMap.put(STATE_PAGE_EXPIRATION_TIMESTAMP, paginatedQueryJsonParameters.getPageExpirationTimestamp());
|
||||
context.getStateManager().setState(newStateMap, Scope.LOCAL);
|
||||
newStateMap.put(STATE_HIT_COUNT, Integer.toString(paginatedJsonQueryParameters.getHitCount()));
|
||||
newStateMap.put(STATE_PAGE_COUNT, Integer.toString(paginatedJsonQueryParameters.getPageCount()));
|
||||
newStateMap.put(STATE_PAGE_EXPIRATION_TIMESTAMP, paginatedJsonQueryParameters.getPageExpirationTimestamp());
|
||||
}
|
||||
updateProcessorState(context, newStateMap);
|
||||
}
|
||||
|
||||
void additionalState(final Map<String, String> newStateMap, final PaginatedJsonQueryParameters paginatedJsonQueryParameters) {
|
||||
// intentionally blank, allows ConsumeElasticsearch to track range value between sessions
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -168,10 +183,13 @@ public class SearchElasticsearch extends AbstractPaginatedJsonQueryElasticsearch
|
|||
final SearchResponse response) throws IOException {
|
||||
final boolean expiredQuery = StringUtils.isNotEmpty(paginatedJsonQueryParameters.getPageExpirationTimestamp())
|
||||
&& Instant.ofEpochMilli(Long.parseLong(paginatedJsonQueryParameters.getPageExpirationTimestamp())).isBefore(Instant.now());
|
||||
|
||||
if (expiredQuery) {
|
||||
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.setHitCount(0);
|
||||
|
@ -186,20 +204,24 @@ public class SearchElasticsearch extends AbstractPaginatedJsonQueryElasticsearch
|
|||
@Override
|
||||
String getScrollId(final ProcessContext context, final SearchResponse response) throws IOException {
|
||||
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();
|
||||
}
|
||||
|
||||
@Override
|
||||
String getPitId(final ProcessContext context, final SearchResponse response) throws IOException {
|
||||
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();
|
||||
}
|
||||
|
||||
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
|
||||
// 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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ public class PaginatedJsonQueryParameters extends JsonQueryParameters {
|
|||
private String pitId = null;
|
||||
private String pageExpirationTimestamp = null;
|
||||
private String keepAlive;
|
||||
private String trackingRangeValue;
|
||||
|
||||
public int getPageCount() {
|
||||
return pageCount;
|
||||
|
@ -76,4 +77,12 @@ public class PaginatedJsonQueryParameters extends JsonQueryParameters {
|
|||
public void setKeepAlive(final String keepAlive) {
|
||||
this.keepAlive = keepAlive;
|
||||
}
|
||||
|
||||
public String getTrackingRangeValue() {
|
||||
return trackingRangeValue;
|
||||
}
|
||||
|
||||
public void setTrackingRangeValue(String trackingRangeValue) {
|
||||
this.trackingRangeValue = trackingRangeValue;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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)));
|
||||
}
|
||||
}
|
|
@ -17,8 +17,8 @@ org.apache.nifi.processors.elasticsearch.DeleteByQueryElasticsearch
|
|||
org.apache.nifi.processors.elasticsearch.JsonQueryElasticsearch
|
||||
org.apache.nifi.processors.elasticsearch.PaginatedJsonQueryElasticsearch
|
||||
org.apache.nifi.processors.elasticsearch.SearchElasticsearch
|
||||
org.apache.nifi.processors.elasticsearch.ConsumeElasticsearch
|
||||
org.apache.nifi.processors.elasticsearch.PutElasticsearchRecord
|
||||
org.apache.nifi.processors.elasticsearch.PutElasticsearchJson
|
||||
org.apache.nifi.processors.elasticsearch.UpdateByQueryElasticsearch
|
||||
org.apache.nifi.processors.elasticsearch.GetElasticsearch
|
||||
|
||||
|
|
|
@ -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>
|
|
@ -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>
|
||||
<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> 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>
|
||||
<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
|
||||
|
|
|
@ -60,17 +60,27 @@
|
|||
}
|
||||
}
|
||||
</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.
|
||||
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>
|
||||
further results are available. After that point, a new paginated query will be initiated using the same Query JSON.</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>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,
|
||||
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>
|
||||
</html>
|
|
@ -16,20 +16,28 @@
|
|||
*/
|
||||
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.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
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.Arguments;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
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.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public abstract class AbstractByQueryElasticsearchTest {
|
||||
|
@ -37,8 +45,11 @@ public abstract class AbstractByQueryElasticsearchTest {
|
|||
private static final String INDEX = "test_idx";
|
||||
private static final String TYPE = "test_type";
|
||||
private static final String CLIENT_NAME = "clientService";
|
||||
private TestElasticsearchClientService client;
|
||||
|
||||
private static final ObjectMapper TEST_MAPPER = new ObjectMapper();
|
||||
|
||||
private static String matchAllQuery;
|
||||
private TestElasticsearchClientService client;
|
||||
private TestRunner runner;
|
||||
|
||||
public abstract String queryAttr();
|
||||
|
@ -47,13 +58,13 @@ public abstract class AbstractByQueryElasticsearchTest {
|
|||
|
||||
public abstract String errorAttr();
|
||||
|
||||
public abstract Class<? extends AbstractByQueryElasticsearch> getTestProcessor();
|
||||
public abstract AbstractByQueryElasticsearch getTestProcessor();
|
||||
|
||||
public abstract void expectError(final TestElasticsearchClientService client);
|
||||
|
||||
@BeforeAll
|
||||
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
|
||||
|
@ -65,7 +76,7 @@ public abstract class AbstractByQueryElasticsearchTest {
|
|||
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_SUCCESS, 1);
|
||||
|
||||
|
@ -78,6 +89,88 @@ public abstract class AbstractByQueryElasticsearchTest {
|
|||
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
|
||||
public void testWithFlowfileInput() {
|
||||
final String query = matchAllQuery;
|
||||
|
@ -114,7 +207,7 @@ public abstract class AbstractByQueryElasticsearchTest {
|
|||
|
||||
@Test
|
||||
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.INDEX, INDEX);
|
||||
runner.setProperty(AbstractByQueryElasticsearch.TYPE, TYPE);
|
||||
|
@ -127,7 +220,7 @@ public abstract class AbstractByQueryElasticsearchTest {
|
|||
|
||||
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.setIncomingConnection(false);
|
||||
runner.assertValid();
|
||||
|
@ -205,4 +298,34 @@ public abstract class AbstractByQueryElasticsearchTest {
|
|||
assertEquals("true", client.getRequestParameters().get("refresh"));
|
||||
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))
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,8 +16,16 @@
|
|||
*/
|
||||
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.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.SearchResultsFormat;
|
||||
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.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.jupiter.api.Assumptions;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
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.nio.file.Files;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
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.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.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_COMMON_DIR = "src/test/resources/common";
|
||||
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 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
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
matchAllQuery = Files.readString(Paths.get(TEST_COMMON_DIR, "matchAllQuery.json"));
|
||||
matchAllAggregationWithDefaultTermsQuery = Files.readString(Paths.get(TEST_DIR,"matchAllAggregationWithDefaultTermsQuery.json"));
|
||||
matchAllQuery = JsonUtils.readString(Paths.get(TEST_COMMON_DIR, "matchAllQuery.json"));
|
||||
matchAllAggregationWithDefaultTermsQuery = JsonUtils.readString(Paths.get(TEST_DIR,"matchAllAggregationWithDefaultTermsQuery.json"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMandatoryProperties() {
|
||||
void testMandatoryProperties() {
|
||||
final TestRunner runner = createRunner(false);
|
||||
runner.removeProperty(AbstractJsonQueryElasticsearch.CLIENT_SERVICE);
|
||||
runner.removeProperty(AbstractJsonQueryElasticsearch.INDEX);
|
||||
runner.removeProperty(AbstractJsonQueryElasticsearch.TYPE);
|
||||
runner.removeProperty(AbstractJsonQueryElasticsearch.QUERY);
|
||||
runner.removeProperty(AbstractJsonQueryElasticsearch.QUERY_ATTRIBUTE);
|
||||
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);
|
||||
runner.removeProperty(AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_SPLIT);
|
||||
runner.removeProperty(AbstractJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT);
|
||||
runner.removeProperty(AbstractJsonQueryElasticsearch.OUTPUT_NO_HITS);
|
||||
|
||||
final AssertionError assertionError = assertThrows(AssertionError.class, runner::run);
|
||||
if (getProcessor() instanceof SearchElasticsearch) {
|
||||
assertEquals(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",
|
||||
AbstractJsonQueryElasticsearch.QUERY.getDisplayName(), AbstractJsonQueryElasticsearch.QUERY.getDisplayName(),
|
||||
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());
|
||||
}
|
||||
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
|
||||
public void testInvalidProperties() {
|
||||
void testInvalidProperties() {
|
||||
final TestRunner runner = createRunner(false);
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.CLIENT_SERVICE, "not-a-service");
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.INDEX, "");
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.TYPE, "");
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, "not-json");
|
||||
runner.setProperty(ElasticsearchRestProcessor.CLIENT_SERVICE, "not-a-service");
|
||||
runner.setProperty(ElasticsearchRestProcessor.INDEX, "");
|
||||
runner.setProperty(ElasticsearchRestProcessor.TYPE, "");
|
||||
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, "not-valid");
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_SPLIT, "not-enum");
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT, "not-enum2");
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.OUTPUT_NO_HITS, "not-boolean");
|
||||
|
@ -108,25 +125,105 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
|
|||
.collect(Collectors.joining(", ")) : nonPaginatedResultOutputStrategies;
|
||||
|
||||
final AssertionError assertionError = assertThrows(AssertionError.class, runner::run);
|
||||
String expected = String.format("Processor has 8 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" + "'%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" +
|
||||
String expected;
|
||||
if (runner.getProcessor() instanceof ConsumeElasticsearch) {
|
||||
expected = "Processor has 7 validation failures:\n";
|
||||
} else {
|
||||
expected = String.format("Processor has 8 validation failures:\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-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",
|
||||
AbstractJsonQueryElasticsearch.QUERY.getName(), AbstractJsonQueryElasticsearch.QUERY.getName(), AbstractJsonQueryElasticsearch.INDEX.getName(),
|
||||
AbstractJsonQueryElasticsearch.INDEX.getName(), AbstractJsonQueryElasticsearch.TYPE.getName(), AbstractJsonQueryElasticsearch.TYPE.getName(),
|
||||
AbstractJsonQueryElasticsearch.CLIENT_SERVICE.getDisplayName(), AbstractJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT.getName(), expectedAllowedSplitHits,
|
||||
AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_SPLIT.getName(), nonPaginatedResultOutputStrategies, AbstractJsonQueryElasticsearch.OUTPUT_NO_HITS.getName(),
|
||||
AbstractJsonQueryElasticsearch.CLIENT_SERVICE.getDisplayName());
|
||||
ElasticsearchRestProcessor.INDEX.getName(), ElasticsearchRestProcessor.INDEX.getName(),
|
||||
ElasticsearchRestProcessor.TYPE.getName(), ElasticsearchRestProcessor.TYPE.getName(),
|
||||
ElasticsearchRestProcessor.CLIENT_SERVICE.getDisplayName(),
|
||||
AbstractJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT.getName(), expectedAllowedSplitHits,
|
||||
AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_SPLIT.getName(), nonPaginatedResultOutputStrategies,
|
||||
AbstractJsonQueryElasticsearch.OUTPUT_NO_HITS.getName(), ElasticsearchRestProcessor.CLIENT_SERVICE.getDisplayName());
|
||||
assertEquals(expected, assertionError.getMessage());
|
||||
}
|
||||
|
||||
@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
|
||||
final TestRunner runner = createRunner(false);
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllQuery);
|
||||
|
@ -192,7 +289,7 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testNoHits() {
|
||||
void testNoHits() {
|
||||
// test no hits (no output)
|
||||
final TestRunner runner = createRunner(false);
|
||||
final TestElasticsearchClientService service = getService(runner);
|
||||
|
@ -224,14 +321,14 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
public void testAggregationsFullFormat() {
|
||||
void testAggregationsFullFormat() {
|
||||
final TestRunner runner = createRunner(true);
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllAggregationWithDefaultTermsQuery);
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_FORMAT, AggregationResultsFormat.FULL.getValue());
|
||||
runOnce(runner);
|
||||
testCounts(runner, isInput() ? 1 : 0, 1, 0, 1);
|
||||
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.name");
|
||||
// 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")
|
||||
@Test
|
||||
public void testWithQueryParameterNoIncomingConnectionAndBucketsAggregationFormat() {
|
||||
void testWithQueryParameterNoIncomingConnectionAndBucketsAggregationFormat() {
|
||||
final TestRunner runner = createRunner(true);
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllAggregationWithDefaultTermsQuery);
|
||||
runner.setIncomingConnection(false);
|
||||
|
@ -273,7 +370,7 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testSplittingAggregationsMetadataOnlyFormat() {
|
||||
void testSplittingAggregationsMetadataOnlyFormat() {
|
||||
final TestRunner runner = createRunner(true);
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllAggregationWithDefaultTermsQuery);
|
||||
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);
|
||||
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "10");
|
||||
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.number", Integer.toString(++a));
|
||||
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)
|
||||
assertTrue(aggContent.containsKey("doc_count_error_upper_bound"));
|
||||
assertFalse(aggContent.containsKey("buckets"));
|
||||
}
|
||||
}
|
||||
@Test
|
||||
public void testAggregationsUsingExpressionLanguage() throws Exception {
|
||||
void testAggregationsUsingExpressionLanguage() throws Exception {
|
||||
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("es.index", INDEX_NAME);
|
||||
runner.setEnvironmentVariableValue("es.type", "msg");
|
||||
|
@ -309,7 +406,7 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
|
|||
testCounts(runner, isInput() ? 1 : 0, 1, 0, 2);
|
||||
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("hit.count", "10");
|
||||
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.number", Integer.toString(++a));
|
||||
assertOutputContent(termAgg.getContent(), 1, false);
|
||||
|
@ -317,7 +414,7 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testErrorDuringSearch() {
|
||||
void testErrorDuringSearch() {
|
||||
final TestRunner runner = createRunner(true);
|
||||
getService(runner).setThrowErrorInSearch(true);
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllAggregationWithDefaultTermsQuery);
|
||||
|
@ -325,15 +422,22 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
|
|||
testCounts(runner, 0, 0, isInput() ? 1 : 0, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQueryAttribute() {
|
||||
String query = matchAllAggregationWithDefaultTermsQuery;
|
||||
@ParameterizedTest
|
||||
@EnumSource(QueryDefinitionType.class)
|
||||
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 TestRunner runner = createRunner(true);
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, query);
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY_ATTRIBUTE, queryAttr);
|
||||
runner.setProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE, queryDefinitionType.getValue());
|
||||
setQuery(runner, query);
|
||||
|
||||
runOnce(runner);
|
||||
|
||||
testCounts(runner, isInput() ? 1 : 0, 1, 0, 1);
|
||||
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_AGGREGATIONS);
|
||||
flowFiles.addAll(runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS));
|
||||
|
@ -341,12 +445,19 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
|
|||
for (final MockFlowFile mockFlowFile : flowFiles) {
|
||||
final String attr = mockFlowFile.getAttribute(queryAttr);
|
||||
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
|
||||
public void testInputHandling() {
|
||||
void testInputHandling() {
|
||||
final TestRunner runner = createRunner(false);
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllQuery);
|
||||
|
||||
|
@ -361,9 +472,11 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testRequestParameters() {
|
||||
void testRequestParameters() {
|
||||
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("slices", "${slices}");
|
||||
runner.setEnvironmentVariableValue("slices", "auto");
|
||||
|
@ -371,7 +484,7 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
|
|||
runOnce(runner);
|
||||
|
||||
final TestElasticsearchClientService service = getService(runner);
|
||||
if (getProcessor() instanceof SearchElasticsearch || getProcessor() instanceof PaginatedJsonQueryElasticsearch) {
|
||||
if (runner.getProcessor() instanceof AbstractPaginatedJsonQueryElasticsearch) {
|
||||
assertEquals(3, service.getRequestParameters().size());
|
||||
assertEquals("600s", service.getRequestParameters().get("scroll"));
|
||||
} else {
|
||||
|
@ -382,7 +495,142 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
|
|||
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_HITS, hits);
|
||||
runner.assertTransferCount(AbstractJsonQueryElasticsearch.REL_FAILURE, failure);
|
||||
|
@ -390,7 +638,7 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
|
|||
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) {
|
||||
assertEquals(count, (content.split("\n").length));
|
||||
} 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 TestRunner runner = TestRunners.newTestRunner(processor);
|
||||
final TestElasticsearchClientService service = new TestElasticsearchClientService(returnAggs);
|
||||
try {
|
||||
runner.addControllerService("esService", service);
|
||||
} catch (InitializationException e) {
|
||||
} catch (final InitializationException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
runner.enableControllerService(service);
|
||||
|
@ -422,11 +670,15 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
|
|||
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;
|
||||
if (isInput()) {
|
||||
runner.setIncomingConnection(true);
|
||||
ff = runner.enqueue("test");
|
||||
ff = runner.enqueue(data);
|
||||
} else {
|
||||
runner.setIncomingConnection(false);
|
||||
ff = null;
|
||||
|
@ -436,17 +688,17 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
|
|||
return ff;
|
||||
}
|
||||
|
||||
public static TestElasticsearchClientService getService(final TestRunner runner) {
|
||||
static TestElasticsearchClientService getService(final TestRunner runner) {
|
||||
return runner.getControllerService("esService", TestElasticsearchClientService.class);
|
||||
}
|
||||
|
||||
public void reset(final TestRunner runner) {
|
||||
void reset(final TestRunner runner) {
|
||||
runner.clearProvenanceEvents();
|
||||
runner.clearTransferState();
|
||||
if (isStateUsed()) {
|
||||
if (getStateScope() != null) {
|
||||
try {
|
||||
runner.getStateManager().clear(Scope.LOCAL);
|
||||
} catch (IOException e) {
|
||||
runner.getStateManager().clear(getStateScope());
|
||||
} catch (final IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
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.PaginationType;
|
||||
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.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.nio.file.Files;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Map;
|
||||
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;
|
||||
|
||||
public abstract class AbstractPaginatedJsonQueryElasticsearchTest extends AbstractJsonQueryElasticsearchTest<AbstractPaginatedJsonQueryElasticsearch> {
|
||||
protected static String matchAllWithSortByMsgWithSizeQuery;
|
||||
private static final String TEST_DIR = "src/test/resources/AbstractPaginatedJsonQueryElasticsearchTest";
|
||||
private static String matchAllWithSortByMessage;
|
||||
private static String matchAllWithSortByMsgWithoutSize;
|
||||
protected static String matchAllWithSortByMsgWithSizeQuery;
|
||||
private static final String TEST_DIR = "src/test/resources/AbstractPaginatedJsonQueryElasticsearchTest";
|
||||
private static String matchAllWithSortByMessage;
|
||||
private static String matchAllWithSortByMsgWithoutSize;
|
||||
|
||||
@BeforeAll
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
AbstractJsonQueryElasticsearchTest.setUpBeforeClass();
|
||||
matchAllWithSortByMessage = Files.readString(Paths.get(TEST_DIR, "matchAllWithSortByMessageQuery.json"));
|
||||
matchAllWithSortByMsgWithoutSize = Files.readString(Paths.get(TEST_DIR,"matchAllWithSortByMsgQueryWithoutSize.json"));
|
||||
matchAllWithSortByMsgWithSizeQuery = Files.readString(Paths.get(TEST_DIR, "matchAllWithSortByMsgQueryWithSize.json"));
|
||||
matchAllWithSortByMessage = JsonUtils.readString(Paths.get(TEST_DIR, "matchAllWithSortByMessageQuery.json"));
|
||||
matchAllWithSortByMsgWithoutSize = JsonUtils.readString(Paths.get(TEST_DIR,"matchAllWithSortByMsgQueryWithoutSize.json"));
|
||||
matchAllWithSortByMsgWithSizeQuery = JsonUtils.readString(Paths.get(TEST_DIR, "matchAllWithSortByMsgQueryWithSize.json"));
|
||||
}
|
||||
|
||||
public abstract boolean isInput();
|
||||
|
||||
@Test
|
||||
public void testInvalidPaginationProperties() {
|
||||
void testInvalidPaginationProperties() {
|
||||
final TestRunner runner = createRunner(false);
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllQuery);
|
||||
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_KEEP_ALIVE, "not-a-period");
|
||||
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, "not-enum");
|
||||
|
||||
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-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",
|
||||
|
@ -72,7 +72,7 @@ public abstract class AbstractPaginatedJsonQueryElasticsearchTest extends Abstra
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testSinglePage() {
|
||||
void testSinglePage() {
|
||||
// paged query hits (no splitting)
|
||||
final TestRunner runner = createRunner(false);
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllQuery);
|
||||
|
@ -115,8 +115,7 @@ public abstract class AbstractPaginatedJsonQueryElasticsearchTest extends Abstra
|
|||
assertSendEvent(runner, input);
|
||||
}
|
||||
|
||||
|
||||
public static void assertFormattedResult(final SearchResultsFormat searchResultsFormat, final Map<String, Object> hit) {
|
||||
static void assertFormattedResult(final SearchResultsFormat searchResultsFormat, final Map<String, Object> hit) {
|
||||
assertFalse(hit.isEmpty());
|
||||
switch (searchResultsFormat) {
|
||||
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) {
|
||||
int flowFileCount;
|
||||
String hitsCount;
|
||||
final int flowFileCount;
|
||||
final String hitsCount;
|
||||
boolean ndjson = false;
|
||||
|
||||
switch (resultOutputStrategy) {
|
||||
|
@ -179,31 +178,30 @@ public abstract class AbstractPaginatedJsonQueryElasticsearchTest extends Abstra
|
|||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testResultsFormat() {
|
||||
for (final ResultOutputStrategy resultOutputStrategy : ResultOutputStrategy.values()) {
|
||||
final TestRunner runner = createRunner(false);
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllWithSortByMessage);
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT, resultOutputStrategy.getValue());
|
||||
@ParameterizedTest
|
||||
@EnumSource(ResultOutputStrategy.class)
|
||||
void testResultsFormat(final ResultOutputStrategy resultOutputStrategy) throws JsonProcessingException {
|
||||
final TestRunner runner = createRunner(false);
|
||||
setQuery(runner, matchAllWithSortByMessage);
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.SEARCH_RESULTS_SPLIT, resultOutputStrategy.getValue());
|
||||
|
||||
// Test against each results format
|
||||
for (final SearchResultsFormat searchResultsFormat : SearchResultsFormat.values()) {
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.SEARCH_RESULTS_FORMAT, searchResultsFormat.getValue());
|
||||
// Test against each result format
|
||||
for (final SearchResultsFormat searchResultsFormat : SearchResultsFormat.values()) {
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.SEARCH_RESULTS_FORMAT, searchResultsFormat.getValue());
|
||||
|
||||
// Test against each pagination type
|
||||
for (final PaginationType paginationType : PaginationType.values()) {
|
||||
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, paginationType.getValue());
|
||||
// Test against each pagination type
|
||||
for (final PaginationType paginationType : PaginationType.values()) {
|
||||
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, paginationType.getValue());
|
||||
|
||||
runOnce(runner);
|
||||
assertResultsFormat(runner, resultOutputStrategy, searchResultsFormat);
|
||||
reset(runner);
|
||||
}
|
||||
runOnce(runner);
|
||||
assertResultsFormat(runner, resultOutputStrategy, searchResultsFormat);
|
||||
reset(runner);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScrollError() {
|
||||
void testDeleteScrollError() {
|
||||
final TestRunner runner = createRunner(false);
|
||||
final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner);
|
||||
service.setThrowErrorInDelete(true);
|
||||
|
@ -219,14 +217,39 @@ public abstract class AbstractPaginatedJsonQueryElasticsearchTest extends Abstra
|
|||
}
|
||||
|
||||
@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 TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner);
|
||||
service.setThrowErrorInDelete(true);
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.SEARCH_RESULTS_FORMAT, SearchResultsFormat.FULL.getValue());
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.AGGREGATION_RESULTS_FORMAT, AggregationResultsFormat.FULL.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
|
||||
runMultiple(runner);
|
||||
|
@ -237,12 +260,12 @@ public abstract class AbstractPaginatedJsonQueryElasticsearchTest extends Abstra
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testInitialisePitError() {
|
||||
void testInitialisePitError() throws JsonProcessingException {
|
||||
final TestRunner runner = createRunner(false);
|
||||
final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner);
|
||||
service.setThrowErrorInPit(true);
|
||||
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
|
||||
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")));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQuerySortError() {
|
||||
// test PiT without sort
|
||||
@ParameterizedTest
|
||||
@EnumSource(PaginationType.class)
|
||||
void testPaginatedQueryWithoutSort(final PaginationType paginationType) throws JsonProcessingException {
|
||||
final TestRunner runner = createRunner(false);
|
||||
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, PaginationType.POINT_IN_TIME.getValue());
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllQuery);
|
||||
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, paginationType.getValue());
|
||||
setQuery(runner, matchAllQuery);
|
||||
|
||||
// expect "failure" output for exception during query setup
|
||||
runOnce(runner);
|
||||
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 0, isInput() ? 1 : 0, 0);
|
||||
if (PaginationType.SCROLL == paginationType) {
|
||||
// test scroll without sort (should succeed)
|
||||
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
|
||||
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 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);
|
||||
// check error was caught and logged
|
||||
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")));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@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;
|
||||
|
@ -348,28 +356,26 @@ public abstract class AbstractPaginatedJsonQueryElasticsearchTest extends Abstra
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEmptyHitsFlowFileIsProducedForEachResultSplitSetup() {
|
||||
@ParameterizedTest
|
||||
@EnumSource(PaginationType.class)
|
||||
void testEmptyHitsFlowFileIsProducedForEachResultSplitSetup(final PaginationType paginationType) throws JsonProcessingException {
|
||||
final TestRunner runner = createRunner(false);
|
||||
final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner);
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllWithSortByMessage);
|
||||
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.OUTPUT_NO_HITS, "true");
|
||||
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()) {
|
||||
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, paginationType.getValue());
|
||||
for (final ResultOutputStrategy resultOutputStrategy : ResultOutputStrategy.values()) {
|
||||
// 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()) {
|
||||
// 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);
|
||||
|
||||
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);
|
||||
}
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -39,7 +39,7 @@ public abstract class AbstractPutElasticsearchTest<P extends AbstractPutElastics
|
|||
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";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -33,8 +33,8 @@ public class DeleteByQueryElasticsearchTest extends AbstractByQueryElasticsearch
|
|||
}
|
||||
|
||||
@Override
|
||||
public Class<? extends AbstractByQueryElasticsearch> getTestProcessor() {
|
||||
return DeleteByQueryElasticsearch.class;
|
||||
public AbstractByQueryElasticsearch getTestProcessor() {
|
||||
return new DeleteByQueryElasticsearch();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -59,7 +59,7 @@ public class GetElasticsearchTest {
|
|||
runner.removeProperty(GetElasticsearch.ATTRIBUTE_NAME);
|
||||
|
||||
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",
|
||||
|
@ -79,7 +79,7 @@ public class GetElasticsearchTest {
|
|||
runner.setProperty(GetElasticsearch.ATTRIBUTE_NAME, "");
|
||||
|
||||
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" +
|
||||
|
@ -101,7 +101,7 @@ public class GetElasticsearchTest {
|
|||
runner.setProperty(GetElasticsearch.ATTRIBUTE_NAME, "");
|
||||
|
||||
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",
|
||||
GetElasticsearch.ATTRIBUTE_NAME.getName(), GetElasticsearch.ATTRIBUTE_NAME.getName());
|
||||
assertEquals(expected, assertionError.getMessage());
|
||||
|
|
|
@ -16,19 +16,22 @@
|
|||
*/
|
||||
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
|
||||
public AbstractJsonQueryElasticsearch getProcessor() {
|
||||
AbstractJsonQueryElasticsearch<JsonQueryParameters> getProcessor() {
|
||||
return new JsonQueryElasticsearch();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isStateUsed() {
|
||||
return false;
|
||||
Scope getStateScope() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isInput() {
|
||||
boolean isInput() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,9 @@ package org.apache.nifi.processors.elasticsearch;
|
|||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
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.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
@ -27,40 +30,44 @@ public class JsonUtils {
|
|||
private static final ObjectMapper MAPPER = new ObjectMapper();
|
||||
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 {
|
||||
return MAPPER.writeValueAsString(object);
|
||||
} catch (JsonProcessingException e) {
|
||||
} catch (final JsonProcessingException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
static String prettyPrint(Object object) {
|
||||
static String prettyPrint(final Object object) {
|
||||
try {
|
||||
return MAPPER.writerWithDefaultPrettyPrinter().writeValueAsString(object);
|
||||
} catch (JsonProcessingException e) {
|
||||
} catch (final JsonProcessingException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
static Map<String, Object> readMap(String json) {
|
||||
static Map<String, Object> readMap(final String json) {
|
||||
try {
|
||||
return MAPPER.readValue(json, Map.class);
|
||||
} catch (JsonProcessingException e) {
|
||||
} catch (final JsonProcessingException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
static List<String> readListOfMapsAsIndividualJson(String json) {
|
||||
static List<String> readListOfMapsAsIndividualJson(final String json) {
|
||||
return readListOfMaps(json).stream()
|
||||
.map(JsonUtils::prettyPrint)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
static List<Map<String, Object>> readListOfMaps(String json) {
|
||||
static List<Map<String, Object>> readListOfMaps(final String json) {
|
||||
try {
|
||||
return MAPPER.readValue(json, List.class);
|
||||
} catch (JsonProcessingException e) {
|
||||
} catch (final JsonProcessingException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
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.ResultOutputStrategy;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
|
@ -31,27 +32,26 @@ public class PaginatedJsonQueryElasticsearchTest extends AbstractPaginatedJsonQu
|
|||
public static void setUpBeforeClass() throws Exception {
|
||||
AbstractPaginatedJsonQueryElasticsearchTest.setUpBeforeClass();
|
||||
}
|
||||
|
||||
public AbstractPaginatedJsonQueryElasticsearch getProcessor() {
|
||||
AbstractPaginatedJsonQueryElasticsearch getProcessor() {
|
||||
return new PaginatedJsonQueryElasticsearch();
|
||||
}
|
||||
|
||||
public boolean isStateUsed() {
|
||||
return false;
|
||||
Scope getStateScope() {
|
||||
return null;
|
||||
}
|
||||
|
||||
public boolean isInput() {
|
||||
boolean isInput() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@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();
|
||||
switch (resultOutputStrategy) {
|
||||
case PER_RESPONSE:
|
||||
AbstractJsonQueryElasticsearchTest.testCounts(runner, 1, 2, 0, 0);
|
||||
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("page.number", Integer.toString(page));
|
||||
}
|
||||
|
@ -66,8 +66,8 @@ public class PaginatedJsonQueryElasticsearchTest extends AbstractPaginatedJsonQu
|
|||
case PER_HIT:
|
||||
AbstractJsonQueryElasticsearchTest.testCounts(runner, 1, 20, 0, 0);
|
||||
long count = 1;
|
||||
ValueRange firstPage = ValueRange.of(1, 10);
|
||||
for (MockFlowFile hit : runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS)) {
|
||||
final ValueRange firstPage = ValueRange.of(1, 10);
|
||||
for (final MockFlowFile hit : runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS)) {
|
||||
hit.assertAttributeEquals("hit.count", "1");
|
||||
// 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");
|
||||
|
|
|
@ -62,10 +62,10 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
|
|||
|
||||
@BeforeAll
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
sampleErrorResponse = Files.readString(Paths.get(TEST_COMMON_DIR,"sampleErrorResponse.json"));
|
||||
flowFileContents = Files.readString(Paths.get(TEST_DIR, "flowFileContents.json"));
|
||||
script = Files.readString(Paths.get(TEST_DIR,"script.json"));
|
||||
dynamicTemplates = Files.readString(Paths.get(TEST_COMMON_DIR,"dynamicTemplates.json"));
|
||||
sampleErrorResponse = JsonUtils.readString(Paths.get(TEST_COMMON_DIR,"sampleErrorResponse.json"));
|
||||
flowFileContents = JsonUtils.readString(Paths.get(TEST_DIR, "flowFileContents.json"));
|
||||
script = JsonUtils.readString(Paths.get(TEST_DIR,"script.json"));
|
||||
dynamicTemplates = JsonUtils.readString(Paths.get(TEST_COMMON_DIR,"dynamicTemplates.json"));
|
||||
expectedScript = new LinkedHashMap<>();
|
||||
expectedScript.put("_source", "some script");
|
||||
expectedScript.put("language", "painless");
|
||||
|
@ -99,16 +99,16 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
|
|||
runner.assertValid();
|
||||
}
|
||||
|
||||
public void basicTest(int failure, int retry, int success) {
|
||||
Consumer<List<IndexOperationRequest>> consumer = (List<IndexOperationRequest> items) -> {
|
||||
long nullIdCount = items.stream().filter(item -> item.getId() == null).count();
|
||||
long indexCount = items.stream().filter(item -> "test_index".equals(item.getIndex())).count();
|
||||
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();
|
||||
long emptyScriptCount = items.stream().filter(item -> item.getScript().isEmpty()).count();
|
||||
long falseScriptedUpsertCount = items.stream().filter(item -> !item.isScriptedUpsert()).count();
|
||||
long emptyDynamicTemplatesCount = items.stream().filter(item -> item.getDynamicTemplates().isEmpty()).count();
|
||||
long emptyHeaderFields = items.stream().filter(item -> item.getHeaderFields().isEmpty()).count();
|
||||
public void basicTest(final int failure, final int retry, final int success) {
|
||||
final Consumer<List<IndexOperationRequest>> consumer = (final List<IndexOperationRequest> items) -> {
|
||||
final long nullIdCount = items.stream().filter(item -> item.getId() == null).count();
|
||||
final long indexCount = items.stream().filter(item -> "test_index".equals(item.getIndex())).count();
|
||||
final long typeCount = items.stream().filter(item -> "test_type".equals(item.getType())).count();
|
||||
final long opCount = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count();
|
||||
final long emptyScriptCount = items.stream().filter(item -> item.getScript().isEmpty()).count();
|
||||
final long falseScriptedUpsertCount = items.stream().filter(item -> !item.isScriptedUpsert()).count();
|
||||
final long emptyDynamicTemplatesCount = items.stream().filter(item -> item.getDynamicTemplates().isEmpty()).count();
|
||||
final long emptyHeaderFields = items.stream().filter(item -> item.getHeaderFields().isEmpty()).count();
|
||||
|
||||
assertEquals(1L, nullIdCount);
|
||||
assertEquals(1L, indexCount);
|
||||
|
@ -123,12 +123,12 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
|
|||
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);
|
||||
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) {
|
||||
runner.enqueue(flowFileContents, attr);
|
||||
} else {
|
||||
|
@ -168,25 +168,25 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
|
|||
runner.setEnvironmentVariableValue("version", "external");
|
||||
runner.assertValid();
|
||||
|
||||
clientService.setEvalParametersConsumer((Map<String, String> params) -> {
|
||||
clientService.setEvalParametersConsumer((final Map<String, String> params) -> {
|
||||
assertEquals(2, params.size());
|
||||
assertEquals("true", params.get("refresh"));
|
||||
assertEquals("auto", params.get("slices"));
|
||||
});
|
||||
|
||||
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> {
|
||||
long idCount = items.stream().filter(item -> "123".equals(item.getId())).count();
|
||||
long indexCount = items.stream().filter(item -> "test_index".equals(item.getIndex())).count();
|
||||
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();
|
||||
long headerFieldsCount = items.stream().filter(item -> !item.getHeaderFields().isEmpty()).count();
|
||||
clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
|
||||
final long idCount = items.stream().filter(item -> "123".equals(item.getId())).count();
|
||||
final long indexCount = items.stream().filter(item -> "test_index".equals(item.getIndex())).count();
|
||||
final long typeCount = items.stream().filter(item -> "test_type".equals(item.getType())).count();
|
||||
final long opCount = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count();
|
||||
final long headerFieldsCount = items.stream().filter(item -> !item.getHeaderFields().isEmpty()).count();
|
||||
assertEquals(1L, idCount);
|
||||
assertEquals(1L, indexCount);
|
||||
assertEquals(1L, typeCount);
|
||||
assertEquals(1L, opCount);
|
||||
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("1", headerFields.get("routing"));
|
||||
assertEquals("external", headerFields.get("version"));
|
||||
|
@ -205,25 +205,25 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
|
|||
runner.setProperty(AbstractPutElasticsearch.BULK_HEADER_PREFIX + "empty", "${empty}");
|
||||
runner.assertValid();
|
||||
|
||||
clientService.setEvalParametersConsumer((Map<String, String> params) -> {
|
||||
clientService.setEvalParametersConsumer((final Map<String, String> params) -> {
|
||||
assertEquals(2, params.size());
|
||||
assertEquals("true", params.get("refresh"));
|
||||
assertEquals("auto", params.get("slices"));
|
||||
});
|
||||
|
||||
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> {
|
||||
long nullIdCount = items.stream().filter(item -> item.getId() == null).count();
|
||||
long headerFieldsCount = items.stream().filter(item -> !item.getHeaderFields().isEmpty()).count();
|
||||
clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
|
||||
final long nullIdCount = items.stream().filter(item -> item.getId() == null).count();
|
||||
final long headerFieldsCount = items.stream().filter(item -> !item.getHeaderFields().isEmpty()).count();
|
||||
assertEquals(1L, nullIdCount);
|
||||
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("1", headerFields.get("routing"));
|
||||
assertEquals("external", headerFields.get("version"));
|
||||
});
|
||||
|
||||
Map<String, String> attributes = new LinkedHashMap<>();
|
||||
final Map<String, String> attributes = new LinkedHashMap<>();
|
||||
attributes.put("slices", "auto");
|
||||
attributes.put("version", "external");
|
||||
attributes.put("blank", " ");
|
||||
|
@ -235,10 +235,10 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
|
|||
public void simpleTestWithScriptAndDynamicTemplates() {
|
||||
runner.setProperty(PutElasticsearchJson.SCRIPT, script);
|
||||
runner.setProperty(PutElasticsearchJson.DYNAMIC_TEMPLATES, dynamicTemplates);
|
||||
Consumer<List<IndexOperationRequest>> consumer = (List<IndexOperationRequest> items) -> {
|
||||
long scriptCount = items.stream().filter(item -> item.getScript().equals(expectedScript)).count();
|
||||
long falseScriptedUpsertCount = items.stream().filter(item -> !item.isScriptedUpsert()).count();
|
||||
long dynamicTemplatesCount = items.stream().filter(item -> item.getDynamicTemplates().equals(expectedDynamicTemplate)).count();
|
||||
final Consumer<List<IndexOperationRequest>> consumer = (final List<IndexOperationRequest> items) -> {
|
||||
final long scriptCount = items.stream().filter(item -> item.getScript().equals(expectedScript)).count();
|
||||
final long falseScriptedUpsertCount = items.stream().filter(item -> !item.isScriptedUpsert()).count();
|
||||
final long dynamicTemplatesCount = items.stream().filter(item -> item.getDynamicTemplates().equals(expectedDynamicTemplate)).count();
|
||||
assertEquals(1L, scriptCount);
|
||||
assertEquals(1L, falseScriptedUpsertCount);
|
||||
assertEquals(1L, dynamicTemplatesCount);
|
||||
|
@ -252,10 +252,10 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
|
|||
runner.setProperty(PutElasticsearchJson.DYNAMIC_TEMPLATES, dynamicTemplates);
|
||||
runner.setProperty(PutElasticsearchJson.INDEX_OP, IndexOperationRequest.Operation.Upsert.getValue().toLowerCase());
|
||||
runner.setProperty(PutElasticsearchJson.SCRIPTED_UPSERT, "true");
|
||||
Consumer<List<IndexOperationRequest>> consumer = (List<IndexOperationRequest> items) -> {
|
||||
long scriptCount = items.stream().filter(item -> item.getScript().equals(expectedScript)).count();
|
||||
long trueScriptedUpsertCount = items.stream().filter(IndexOperationRequest::isScriptedUpsert).count();
|
||||
long dynamicTemplatesCount = items.stream().filter(item -> item.getDynamicTemplates().equals(expectedDynamicTemplate)).count();
|
||||
final Consumer<List<IndexOperationRequest>> consumer = (final List<IndexOperationRequest> items) -> {
|
||||
final long scriptCount = items.stream().filter(item -> item.getScript().equals(expectedScript)).count();
|
||||
final long trueScriptedUpsertCount = items.stream().filter(IndexOperationRequest::isScriptedUpsert).count();
|
||||
final long dynamicTemplatesCount = items.stream().filter(item -> item.getDynamicTemplates().equals(expectedDynamicTemplate)).count();
|
||||
|
||||
assertEquals(1L, scriptCount);
|
||||
assertEquals(1L, trueScriptedUpsertCount);
|
||||
|
@ -324,7 +324,7 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
|
|||
runner.setProperty(PutElasticsearchJson.BATCH_SIZE, "100");
|
||||
runner.setProperty(PutElasticsearchJson.NOT_FOUND_IS_SUCCESSFUL, "true");
|
||||
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));
|
||||
runner.assertValid();
|
||||
runner.run();
|
||||
|
@ -365,7 +365,7 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
|
|||
runner.setProperty(PutElasticsearchJson.NOT_FOUND_IS_SUCCESSFUL, "false");
|
||||
runner.setProperty(PutElasticsearchJson.OUTPUT_ERROR_RESPONSES, "true");
|
||||
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));
|
||||
runner.assertValid();
|
||||
runner.run();
|
||||
|
@ -416,7 +416,7 @@ public class PutElasticsearchJsonTest extends AbstractPutElasticsearchTest<PutEl
|
|||
runner.setProperty(PutElasticsearchJson.LOG_ERROR_RESPONSES, "false");
|
||||
runner.setProperty(PutElasticsearchJson.BATCH_SIZE, "100");
|
||||
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);
|
||||
}
|
||||
|
||||
|
|
|
@ -92,7 +92,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
|
||||
@BeforeAll
|
||||
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"));
|
||||
recordPathTestSchema = getRecordSchema(Paths.get(TEST_DIR, "recordPathTestSchema.json"));
|
||||
dateTimeFormattingTestSchema = getRecordSchema(Paths.get(TEST_DIR, "dateTimeFormattingTestSchema.json"));
|
||||
|
@ -105,7 +105,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
clientService.setResponse(new IndexOperationResponse(1500));
|
||||
registry = new MockSchemaRegistry();
|
||||
registry.addSchema("simple", simpleSchema);
|
||||
RecordReaderFactory reader = new JsonTreeReader();
|
||||
final RecordReaderFactory reader = new JsonTreeReader();
|
||||
runner = TestRunners.newTestRunner(getTestProcessor());
|
||||
runner.addControllerService("registry", registry);
|
||||
runner.addControllerService("reader", reader);
|
||||
|
@ -126,16 +126,16 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
runner.assertValid();
|
||||
}
|
||||
|
||||
public void basicTest(int failure, int retry, int success) {
|
||||
Consumer<List<IndexOperationRequest>> consumer = (List<IndexOperationRequest> items) -> {
|
||||
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();
|
||||
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();
|
||||
long emptyScriptCount = items.stream().filter(item -> item.getScript().isEmpty()).count();
|
||||
long falseScriptedUpsertCount = items.stream().filter(item -> !item.isScriptedUpsert()).count();
|
||||
long emptyDynamicTemplatesCount = items.stream().filter(item -> item.getDynamicTemplates().isEmpty()).count();
|
||||
long emptyHeaderFields = items.stream().filter(item -> item.getHeaderFields().isEmpty()).count();
|
||||
public void basicTest(final int failure, final int retry, final int success) {
|
||||
final Consumer<List<IndexOperationRequest>> consumer = (final List<IndexOperationRequest> items) -> {
|
||||
final long timestampDefaultCount = items.stream().filter(item -> "test_timestamp".equals(item.getFields().get("@timestamp"))).count();
|
||||
final long indexCount = items.stream().filter(item -> "test_index".equals(item.getIndex())).count();
|
||||
final long typeCount = items.stream().filter(item -> "test_type".equals(item.getType())).count();
|
||||
final long opCount = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count();
|
||||
final long emptyScriptCount = items.stream().filter(item -> item.getScript().isEmpty()).count();
|
||||
final long falseScriptedUpsertCount = items.stream().filter(item -> !item.isScriptedUpsert()).count();
|
||||
final long emptyDynamicTemplatesCount = items.stream().filter(item -> item.getDynamicTemplates().isEmpty()).count();
|
||||
final long emptyHeaderFields = items.stream().filter(item -> item.getHeaderFields().isEmpty()).count();
|
||||
assertEquals(2, timestampDefaultCount);
|
||||
assertEquals(2, indexCount);
|
||||
assertEquals(2, typeCount);
|
||||
|
@ -149,11 +149,11 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
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);
|
||||
}
|
||||
|
||||
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);
|
||||
runner.enqueue(flowFileContentMaps, attributes != null && !attributes.isEmpty() ? attributes : Collections.singletonMap(SCHEMA_NAME_ATTRIBUTE, "simple"));
|
||||
runner.run();
|
||||
|
@ -185,7 +185,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
|
||||
@Test
|
||||
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());
|
||||
|
||||
runner.setProperty(PutElasticsearchRecord.AT_TIMESTAMP, "100");
|
||||
|
@ -202,7 +202,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
|
||||
@Test
|
||||
public void simpleTestWithRequestParametersAndBulkHeadersFlowFileEL() {
|
||||
Map<String, String> attributes = new LinkedHashMap<>();
|
||||
final Map<String, String> attributes = new LinkedHashMap<>();
|
||||
attributes.put(SCHEMA_NAME_ATTRIBUTE, "simple");
|
||||
attributes.put("version", "/version");
|
||||
attributes.put("slices", "auto");
|
||||
|
@ -212,7 +212,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
|
||||
@Test
|
||||
public void simpleTestWithMockReader() throws Exception{
|
||||
MockRecordParser mockReader = new MockRecordParser();
|
||||
final MockRecordParser mockReader = new MockRecordParser();
|
||||
mockReader.addSchemaField("msg", RecordFieldType.STRING);
|
||||
mockReader.addSchemaField("from", RecordFieldType.STRING);
|
||||
mockReader.addRecord("foo", "bar");
|
||||
|
@ -239,29 +239,29 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
|
||||
@Test
|
||||
public void testRecordPathFeatures() throws Exception {
|
||||
Map<String, Object> script =
|
||||
JsonUtils.readMap(Files.readString(Paths.get(TEST_DIR, "script.json")));
|
||||
Map<String, Object> dynamicTemplates =
|
||||
JsonUtils.readMap(Files.readString(Paths.get(TEST_COMMON_DIR, "dynamicTemplates.json")));
|
||||
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> {
|
||||
long a = items.stream().filter(item -> "bulk_a".equals(item.getIndex())).count();
|
||||
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();
|
||||
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();
|
||||
long empties = items.stream().filter(item -> ("".equals(item.getFields().get("msg")))).count();
|
||||
long nulls = items.stream().filter(item -> null == item.getFields().get("msg")).count();
|
||||
long timestamp = items.stream().filter(item ->
|
||||
final Map<String, Object> script =
|
||||
JsonUtils.readMap(JsonUtils.readString(Paths.get(TEST_DIR, "script.json")));
|
||||
final Map<String, Object> dynamicTemplates =
|
||||
JsonUtils.readMap(JsonUtils.readString(Paths.get(TEST_COMMON_DIR, "dynamicTemplates.json")));
|
||||
clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
|
||||
final long a = items.stream().filter(item -> "bulk_a".equals(item.getIndex())).count();
|
||||
final long b = items.stream().filter(item -> "bulk_b".equals(item.getIndex())).count();
|
||||
final long index = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count();
|
||||
final long create = items.stream().filter(item -> IndexOperationRequest.Operation.Create.equals(item.getOperation())).count();
|
||||
final long msg = items.stream().filter(item -> "Hello".equals(item.getFields().get("msg"))).count();
|
||||
final long empties = items.stream().filter(item -> ("".equals(item.getFields().get("msg")))).count();
|
||||
final long nulls = items.stream().filter(item -> null == item.getFields().get("msg")).count();
|
||||
final long timestamp = items.stream().filter(item ->
|
||||
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();
|
||||
long ts = items.stream().filter(item -> item.getFields().get("ts") != null).count();
|
||||
long id = items.stream().filter(item -> item.getFields().get("id") != null).count();
|
||||
long emptyScript = items.stream().filter(item -> item.getScript().isEmpty()).count();
|
||||
long falseScriptedUpsertCount = items.stream().filter(item -> !item.isScriptedUpsert()).count();
|
||||
long trueScriptedUpsertCount = items.stream().filter(IndexOperationRequest::isScriptedUpsert).count();
|
||||
long s = items.stream().filter(item -> script.equals(item.getScript())).count();
|
||||
long emptyDynamicTemplates = items.stream().filter(item -> item.getDynamicTemplates().isEmpty()).count();
|
||||
long dt = items.stream().filter(item -> dynamicTemplates.equals(item.getDynamicTemplates())).count();
|
||||
final long timestampDefault = items.stream().filter(item -> "test_timestamp".equals(item.getFields().get("@timestamp"))).count();
|
||||
final long ts = items.stream().filter(item -> item.getFields().get("ts") != null).count();
|
||||
final long id = items.stream().filter(item -> item.getFields().get("id") != null).count();
|
||||
final long emptyScript = items.stream().filter(item -> item.getScript().isEmpty()).count();
|
||||
final long falseScriptedUpsertCount = items.stream().filter(item -> !item.isScriptedUpsert()).count();
|
||||
final long trueScriptedUpsertCount = items.stream().filter(IndexOperationRequest::isScriptedUpsert).count();
|
||||
final long s = items.stream().filter(item -> script.equals(item.getScript())).count();
|
||||
final long emptyDynamicTemplates = items.stream().filter(item -> item.getDynamicTemplates().isEmpty()).count();
|
||||
final long dt = items.stream().filter(item -> dynamicTemplates.equals(item.getDynamicTemplates())).count();
|
||||
items.forEach(item -> {
|
||||
assertNotNull(item.getId());
|
||||
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.SCRIPTED_UPSERT_RECORD_PATH, "/scripted_upsert");
|
||||
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()));
|
||||
runner.enqueue(flowFileContents, Collections.singletonMap(SCHEMA_NAME_ATTRIBUTE, RECORD_PATH_TEST_SCHEMA));
|
||||
|
||||
|
@ -312,22 +312,22 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
|
||||
@Test
|
||||
public void testTimestampDateFormatAndScriptRecordPath() throws Exception {
|
||||
Map<String, Object> script =
|
||||
JsonUtils.readMap(Files.readString(Paths.get(TEST_DIR, "script.json")));
|
||||
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> {
|
||||
long testTypeCount = items.stream().filter(item -> "test_type".equals(item.getType())).count();
|
||||
long messageTypeCount = items.stream().filter(item -> "message".equals(item.getType())).count();
|
||||
long testIndexCount = items.stream().filter(item -> "test_index".equals(item.getIndex())).count();
|
||||
long bulkIndexCount = items.stream().filter(item -> item.getIndex().startsWith("bulk_")).count();
|
||||
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();
|
||||
long timestampCount = items.stream().filter(item ->
|
||||
final Map<String, Object> script =
|
||||
JsonUtils.readMap(JsonUtils.readString(Paths.get(TEST_DIR, "script.json")));
|
||||
clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
|
||||
final long testTypeCount = items.stream().filter(item -> "test_type".equals(item.getType())).count();
|
||||
final long messageTypeCount = items.stream().filter(item -> "message".equals(item.getType())).count();
|
||||
final long testIndexCount = items.stream().filter(item -> "test_index".equals(item.getIndex())).count();
|
||||
final long bulkIndexCount = items.stream().filter(item -> item.getIndex().startsWith("bulk_")).count();
|
||||
final long indexOperationCount = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count();
|
||||
final long updateOperationCount = items.stream().filter(item -> IndexOperationRequest.Operation.Update.equals(item.getOperation())).count();
|
||||
final long timestampCount = items.stream().filter(item ->
|
||||
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();
|
||||
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();
|
||||
long emptyScriptCount = items.stream().filter(item -> item.getScript().isEmpty()).count();
|
||||
long scriptCount = items.stream().filter(item -> script.equals(item.getScript())).count();
|
||||
final long dateCount = items.stream().filter(item -> item.getFields().get("date") != null).count();
|
||||
final long idCount = items.stream().filter(item -> item.getFields().get("id") != null).count();
|
||||
final long defaultCoercedTimestampCount = items.stream().filter(item -> Long.valueOf(100).equals(item.getFields().get("@timestamp"))).count();
|
||||
final long emptyScriptCount = items.stream().filter(item -> item.getScript().isEmpty()).count();
|
||||
final long scriptCount = items.stream().filter(item -> script.equals(item.getScript())).count();
|
||||
assertEquals(5, testTypeCount, getUnexpectedCountMsg("test type"));
|
||||
assertEquals(1, messageTypeCount, getUnexpectedCountMsg("message type"));
|
||||
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.INDEX_RECORD_PATH, "/index");
|
||||
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("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()));
|
||||
runner.enqueue(flowFileContents, attributes);
|
||||
runner.run();
|
||||
|
@ -370,12 +370,12 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
|
||||
@Test
|
||||
public void testNullRecordPaths() throws Exception {
|
||||
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> {
|
||||
long nullTypeCount = items.stream().filter(item -> item.getType() == null).count();
|
||||
long messageTypeCount = items.stream().filter(item -> "message".equals(item.getType())).count();
|
||||
long nullIdCount = items.stream().filter(item -> item.getId() == null).count();
|
||||
long recIdCount = items.stream().filter(item -> StringUtils.startsWith(item.getId(), "rec-")).count();
|
||||
long timestampCount = items.stream().filter(item ->
|
||||
clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
|
||||
final long nullTypeCount = items.stream().filter(item -> item.getType() == null).count();
|
||||
final long messageTypeCount = items.stream().filter(item -> "message".equals(item.getType())).count();
|
||||
final long nullIdCount = items.stream().filter(item -> item.getId() == null).count();
|
||||
final long recIdCount = items.stream().filter(item -> StringUtils.startsWith(item.getId(), "rec-")).count();
|
||||
final long timestampCount = items.stream().filter(item ->
|
||||
LOCAL_TIME.format(DateTimeFormatter.ofPattern(RecordFieldType.TIME.getDefaultFormat())).equals(item.getFields().get("@timestamp"))).count();
|
||||
assertEquals(5, nullTypeCount, getUnexpectedCountMsg("null type"));
|
||||
assertEquals(1, messageTypeCount, getUnexpectedCountMsg("message type"));
|
||||
|
@ -391,7 +391,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
runner.removeProperty(PutElasticsearchRecord.TYPE);
|
||||
runner.setProperty(PutElasticsearchRecord.ID_RECORD_PATH, "/id");
|
||||
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()));
|
||||
runner.enqueue(flowFileContents, Collections.singletonMap(SCHEMA_NAME_ATTRIBUTE, RECORD_PATH_TEST_SCHEMA));
|
||||
runner.run();
|
||||
|
@ -405,14 +405,14 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
|
||||
@Test
|
||||
public void testIndexOperationRecordPath() throws Exception {
|
||||
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> {
|
||||
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();
|
||||
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();
|
||||
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();
|
||||
long noTimestampCount = items.stream().filter(item -> !item.getFields().containsKey("@timestamp") ).count();
|
||||
clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
|
||||
final long index = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count();
|
||||
final long create = items.stream().filter(item -> IndexOperationRequest.Operation.Create.equals(item.getOperation())).count();
|
||||
final long update = items.stream().filter(item -> IndexOperationRequest.Operation.Update.equals(item.getOperation())).count();
|
||||
final long upsert = items.stream().filter(item -> IndexOperationRequest.Operation.Upsert.equals(item.getOperation())).count();
|
||||
final long delete = items.stream().filter(item -> IndexOperationRequest.Operation.Delete.equals(item.getOperation())).count();
|
||||
final long timestampCount = items.stream().filter(item -> Long.valueOf(101).equals(item.getFields().get("@timestamp"))).count();
|
||||
final long noTimestampCount = items.stream().filter(item -> !item.getFields().containsKey("@timestamp") ).count();
|
||||
assertEquals(1, index, getUnexpectedCountMsg("index"));
|
||||
assertEquals(2, create, getUnexpectedCountMsg("create"));
|
||||
assertEquals(1, update, getUnexpectedCountMsg("update"));
|
||||
|
@ -438,8 +438,8 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
|
||||
@Test
|
||||
public void testIncompatibleTimestampRecordPath() throws Exception {
|
||||
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> {
|
||||
long timestampCount = items.stream().filter(item -> "Hello".equals(item.getFields().get("@timestamp"))).count();
|
||||
clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
|
||||
final long timestampCount = items.stream().filter(item -> "Hello".equals(item.getFields().get("@timestamp"))).count();
|
||||
assertEquals(1, timestampCount);
|
||||
});
|
||||
|
||||
|
@ -463,7 +463,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
|
||||
@Test
|
||||
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("will_be_empty", "/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_SUCCESSFUL_RECORDS, 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"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRecordPathFieldDefaults() throws Exception {
|
||||
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> {
|
||||
long idNotNull = items.stream().filter(item -> item.getId() != null).count();
|
||||
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();
|
||||
long indexA = items.stream().filter(item -> "bulk_a".equals(item.getIndex())).count();
|
||||
long indexC = items.stream().filter(item -> "bulk_c".equals(item.getIndex())).count();
|
||||
long typeMessage = items.stream().filter(item -> "message".equals(item.getType())).count();
|
||||
long typeBlah = items.stream().filter(item -> "blah".equals(item.getType())).count();
|
||||
clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
|
||||
final long idNotNull = items.stream().filter(item -> item.getId() != null).count();
|
||||
final long opIndex = items.stream().filter(item -> IndexOperationRequest.Operation.Index.equals(item.getOperation())).count();
|
||||
final long opCreate = items.stream().filter(item -> IndexOperationRequest.Operation.Create.equals(item.getOperation())).count();
|
||||
final long indexA = items.stream().filter(item -> "bulk_a".equals(item.getIndex())).count();
|
||||
final long indexC = items.stream().filter(item -> "bulk_c".equals(item.getIndex())).count();
|
||||
final long typeMessage = items.stream().filter(item -> "message".equals(item.getType())).count();
|
||||
final long typeBlah = items.stream().filter(item -> "blah".equals(item.getType())).count();
|
||||
assertEquals(4, idNotNull, getUnexpectedCountMsg("id not null"));
|
||||
assertEquals(3, opIndex, getUnexpectedCountMsg("op index"));
|
||||
assertEquals(3, opCreate, getUnexpectedCountMsg("op create"));
|
||||
|
@ -529,22 +529,22 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
|
||||
@Test
|
||||
public void testDefaultDateTimeFormatting() throws Exception{
|
||||
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> {
|
||||
long msg = items.stream().filter(item -> (item.getFields().get("msg") != null)).count();
|
||||
long timestamp = items.stream().filter(item ->
|
||||
clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
|
||||
final long msg = items.stream().filter(item -> (item.getFields().get("msg") != null)).count();
|
||||
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"
|
||||
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"
|
||||
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"
|
||||
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();
|
||||
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();
|
||||
long tsNull = items.stream().filter(item -> item.getFields().get("ts") == null).count();
|
||||
long dateNull = items.stream().filter(item -> item.getFields().get("date") == null).count();
|
||||
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 choiceNotTs = items.stream().filter(item -> "not-timestamp".equals(item.getFields().get("choice_ts"))).count();
|
||||
final long atTimestampDefault = items.stream().filter(item -> "test_timestamp".equals(item.getFields().get("@timestamp"))).count();
|
||||
final long tsNull = items.stream().filter(item -> item.getFields().get("ts") == null).count();
|
||||
final long dateNull = items.stream().filter(item -> item.getFields().get("date") == null).count();
|
||||
final long timeNull = items.stream().filter(item -> item.getFields().get("time") == null).count();
|
||||
final long choiceTsNull = items.stream().filter(item -> item.getFields().get("choice_ts") == null).count();
|
||||
assertEquals(5, msg, getUnexpectedCountMsg("msg"));
|
||||
assertEquals(2, timestamp, getUnexpectedCountMsg("timestamp"));
|
||||
assertEquals(2, date, getUnexpectedCountMsg("date"));
|
||||
|
@ -575,23 +575,23 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
final String timestampFormat = "yy MMM d H";
|
||||
final String dateFormat = "dd/MM/yyyy";
|
||||
final String timeFormat = "HHmmss";
|
||||
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> {
|
||||
String timestampOutput = LOCAL_DATE_TIME.format(DateTimeFormatter.ofPattern(timestampFormat));
|
||||
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();
|
||||
long date = items.stream().filter(item ->
|
||||
clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
|
||||
final String timestampOutput = LOCAL_DATE_TIME.format(DateTimeFormatter.ofPattern(timestampFormat));
|
||||
final long msg = items.stream().filter(item -> (item.getFields().get("msg") != null)).count();
|
||||
final long timestamp = items.stream().filter(item -> timestampOutput.equals(item.getFields().get("ts"))).count();
|
||||
final long date = items.stream().filter(item ->
|
||||
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
|
||||
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();
|
||||
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();
|
||||
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();
|
||||
long dateNull = items.stream().filter(item -> item.getFields().get("date") == null).count();
|
||||
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 choiceTs = items.stream().filter(item -> timestampOutput.equals(item.getFields().get("choice_ts"))).count();
|
||||
final long choiceNotTs = items.stream().filter(item -> "not-timestamp".equals(item.getFields().get("choice_ts"))).count();
|
||||
final long atTimestampDefault = items.stream().filter(item -> "test_timestamp".equals(item.getFields().get("@timestamp"))).count();
|
||||
final long atTimestamp = items.stream().filter(item -> timestampOutput.equals(item.getFields().get("@timestamp"))).count();
|
||||
final long tsNull = items.stream().filter(item -> item.getFields().get("ts") == null).count();
|
||||
final long dateNull = items.stream().filter(item -> item.getFields().get("date") == null).count();
|
||||
final long timeNull = items.stream().filter(item -> item.getFields().get("time") == null).count();
|
||||
final long choiceTsNull = items.stream().filter(item -> item.getFields().get("choice_ts") == null).count();
|
||||
assertEquals(5, msg, getUnexpectedCountMsg("msg"));
|
||||
assertEquals(2, timestamp, getUnexpectedCountMsg("timestamp"));
|
||||
assertEquals(2, date, getUnexpectedCountMsg("date"));
|
||||
|
@ -659,8 +659,8 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
public void testFailedRecordsOutput() throws Exception {
|
||||
runner.setProperty(PutElasticsearchRecord.NOT_FOUND_IS_SUCCESSFUL, "true");
|
||||
runner.setProperty(PutElasticsearchRecord.LOG_ERROR_RESPONSES, "true");
|
||||
int errorCount = 3;
|
||||
int successCount = 4;
|
||||
final int errorCount = 3;
|
||||
final int successCount = 4;
|
||||
testErrorRelationship(errorCount, successCount, true);
|
||||
|
||||
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.LOG_ERROR_RESPONSES, "true");
|
||||
runner.setProperty(PutElasticsearchRecord.GROUP_BULK_ERRORS_BY_TYPE, "true");
|
||||
int successCount = 4;
|
||||
final int successCount = 4;
|
||||
testErrorRelationship(3, successCount, true);
|
||||
|
||||
runner.assertTransferCount(PutElasticsearchRecord.REL_FAILED_RECORDS, 2);
|
||||
|
@ -699,8 +699,8 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
public void testNotFoundResponsesTreatedAsFailedRecords() throws Exception {
|
||||
runner.setProperty(PutElasticsearchRecord.NOT_FOUND_IS_SUCCESSFUL, "false");
|
||||
runner.setProperty(PutElasticsearchRecord.GROUP_BULK_ERRORS_BY_TYPE, "false");
|
||||
int errorCount = 4;
|
||||
int successCount = 3;
|
||||
final int errorCount = 4;
|
||||
final int successCount = 3;
|
||||
testErrorRelationship(errorCount, successCount, true);
|
||||
|
||||
runner.assertTransferCount(PutElasticsearchRecord.REL_FAILED_RECORDS, 1);
|
||||
|
@ -717,8 +717,8 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
public void testNotFoundFailedRecordsGroupedAsErrorType() throws Exception {
|
||||
runner.setProperty(PutElasticsearchRecord.NOT_FOUND_IS_SUCCESSFUL, "false");
|
||||
runner.setProperty(PutElasticsearchRecord.GROUP_BULK_ERRORS_BY_TYPE, "true");
|
||||
int errorCount = 4;
|
||||
int successCount = 3;
|
||||
final int errorCount = 4;
|
||||
final int successCount = 3;
|
||||
testErrorRelationship(errorCount, successCount, true);
|
||||
|
||||
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.OUTPUT_ERROR_RESPONSES, "true");
|
||||
runner.setProperty(PutElasticsearchRecord.GROUP_BULK_ERRORS_BY_TYPE, "false");
|
||||
int errorCount = 4;
|
||||
int successCount = 3;
|
||||
final int errorCount = 4;
|
||||
final int successCount = 3;
|
||||
testErrorRelationship(errorCount, successCount, false);
|
||||
runner.assertTransferCount(PutElasticsearchRecord.REL_FAILED_RECORDS, 0);
|
||||
runner.assertTransferCount(PutElasticsearchRecord.REL_SUCCESSFUL_RECORDS, 0);
|
||||
|
@ -766,21 +766,21 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
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)));
|
||||
}
|
||||
|
||||
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 JsonRecordSetWriter writer = new JsonRecordSetWriter();
|
||||
runner.addControllerService("writer", writer);
|
||||
runner.setProperty(writer, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.SCHEMA_NAME_PROPERTY);
|
||||
runner.setProperty(writer, SchemaAccessUtils.SCHEMA_REGISTRY, "registry");
|
||||
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);
|
||||
|
||||
if(recordWriter) {
|
||||
if (recordWriter) {
|
||||
runner.setProperty(PutElasticsearchRecord.RESULT_RECORD_WRITER, "writer");
|
||||
}
|
||||
|
||||
|
@ -795,13 +795,13 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
|
||||
assertEquals(1,
|
||||
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 {
|
||||
clientService.setEvalConsumer((List<IndexOperationRequest> items) -> {
|
||||
long nullIdCount = items.stream().filter(item -> item.getId() == null).count();
|
||||
long noTimestampCount = items.stream().filter(item -> !item.getFields().containsKey("@timestamp")).count();
|
||||
private void testInvalidELRecordPaths(final String idRecordPath, final String atTimestampRecordPath, final Path path, final Map<String, String> attributes) throws IOException {
|
||||
clientService.setEvalConsumer((final List<IndexOperationRequest> items) -> {
|
||||
final long nullIdCount = items.stream().filter(item -> item.getId() == null).count();
|
||||
final long noTimestampCount = items.stream().filter(item -> !item.getFields().containsKey("@timestamp")).count();
|
||||
assertEquals(1, nullIdCount, getUnexpectedCountMsg("null id"));
|
||||
assertEquals(1, noTimestampCount, getUnexpectedCountMsg("noTimestamp"));
|
||||
});
|
||||
|
@ -820,7 +820,7 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
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("slices", "${slices}");
|
||||
runner.setProperty(AbstractPutElasticsearch.BULK_HEADER_PREFIX + "routing", "/routing");
|
||||
|
@ -829,16 +829,16 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
runner.setEnvironmentVariableValue("blank", " ");
|
||||
runner.assertValid();
|
||||
|
||||
clientService.setEvalParametersConsumer( (Map<String, String> params) -> {
|
||||
clientService.setEvalParametersConsumer( (final Map<String, String> params) -> {
|
||||
assertEquals(2, params.size());
|
||||
assertEquals("true", params.get("refresh"));
|
||||
assertEquals("auto", params.get("slices"));
|
||||
});
|
||||
|
||||
Consumer<List<IndexOperationRequest>> consumer = (List<IndexOperationRequest> items) -> {
|
||||
long headerFieldsCount = items.stream().filter(item -> !item.getHeaderFields().isEmpty()).count();
|
||||
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 Consumer<List<IndexOperationRequest>> consumer = (final List<IndexOperationRequest> items) -> {
|
||||
final long headerFieldsCount = items.stream().filter(item -> !item.getHeaderFields().isEmpty()).count();
|
||||
final long routingCount = items.stream().filter(item -> "1".equals(item.getHeaderFields().get("routing"))).count();
|
||||
final long versionCount = items.stream().filter(item -> "external".equals(item.getHeaderFields().get("version"))).count();
|
||||
assertEquals(2, headerFieldsCount);
|
||||
assertEquals(1, routingCount);
|
||||
assertEquals(1, versionCount);
|
||||
|
@ -848,8 +848,8 @@ public class PutElasticsearchRecordTest extends AbstractPutElasticsearchTest<Put
|
|||
}
|
||||
|
||||
private String getDateTimeFormattingJson() throws Exception {
|
||||
String json = Files.readString(Paths.get(TEST_DIR, "10_flowFileContents.json"));
|
||||
List<Map<String, Object>> parsedJson = JsonUtils.readListOfMaps(json);
|
||||
final String json = JsonUtils.readString(Paths.get(TEST_DIR, "10_flowFileContents.json"));
|
||||
final List<Map<String, Object>> parsedJson = JsonUtils.readListOfMaps(json);
|
||||
parsedJson.forEach(msg -> {
|
||||
msg.computeIfPresent("ts", (key, val) -> Timestamp.valueOf(LOCAL_DATE_TIME).toInstant().toEpochMilli());
|
||||
msg.computeIfPresent("date", (key, val) -> Date.valueOf(LOCAL_DATE).getTime());
|
||||
|
|
|
@ -22,12 +22,14 @@ import org.apache.nifi.processors.elasticsearch.api.ResultOutputStrategy;
|
|||
import org.apache.nifi.state.MockStateManager;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
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.time.Instant;
|
||||
|
||||
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.fail;
|
||||
|
||||
|
@ -37,73 +39,41 @@ public class SearchElasticsearchTest extends AbstractPaginatedJsonQueryElasticse
|
|||
AbstractPaginatedJsonQueryElasticsearchTest.setUpBeforeClass();
|
||||
}
|
||||
|
||||
public AbstractPaginatedJsonQueryElasticsearch getProcessor() {
|
||||
AbstractPaginatedJsonQueryElasticsearch getProcessor() {
|
||||
return new SearchElasticsearch();
|
||||
}
|
||||
|
||||
public boolean isStateUsed() {
|
||||
return true;
|
||||
Scope getStateScope() {
|
||||
return Scope.LOCAL;
|
||||
}
|
||||
|
||||
public boolean isInput() {
|
||||
boolean isInput() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScrollError() {
|
||||
final TestRunner runner = createRunner(false);
|
||||
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{
|
||||
@ParameterizedTest
|
||||
@EnumSource(PaginationType.class)
|
||||
void testPaginationExpiration(final PaginationType paginationType) throws Exception {
|
||||
// test flowfile per page
|
||||
final TestRunner runner = createRunner(false);
|
||||
final TestElasticsearchClientService service = AbstractJsonQueryElasticsearchTest.getService(runner);
|
||||
service.setMaxPages(2);
|
||||
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_TYPE, paginationType.getValue());
|
||||
runner.setProperty(AbstractPaginatedJsonQueryElasticsearch.PAGINATION_KEEP_ALIVE, "1 sec");
|
||||
runner.setProperty(AbstractJsonQueryElasticsearch.QUERY, matchAllWithSortByMsgWithSizeQuery);
|
||||
setQuery(runner, matchAllWithSortByMsgWithSizeQuery);
|
||||
|
||||
// first page
|
||||
runOnce(runner);
|
||||
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("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
|
||||
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())) {
|
||||
Thread.sleep(10);
|
||||
}
|
||||
|
@ -121,7 +91,11 @@ public class SearchElasticsearchTest extends AbstractPaginatedJsonQueryElasticse
|
|||
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("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();
|
||||
|
||||
// second page
|
||||
|
@ -129,63 +103,81 @@ public class SearchElasticsearchTest extends AbstractPaginatedJsonQueryElasticse
|
|||
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("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();
|
||||
}
|
||||
|
||||
@Override
|
||||
void validatePagination(final TestRunner runner, final ResultOutputStrategy resultOutputStrategy, final PaginationType paginationType, int iteration) throws IOException {
|
||||
boolean perResponseResultOutputStrategy = ResultOutputStrategy.PER_RESPONSE.equals(resultOutputStrategy);
|
||||
boolean perHitResultOutputStrategy = ResultOutputStrategy.PER_HIT.equals(resultOutputStrategy);
|
||||
void validatePagination(final TestRunner runner, final ResultOutputStrategy resultOutputStrategy, final PaginationType paginationType, final int iteration) throws IOException {
|
||||
final boolean perResponseResultOutputStrategy = ResultOutputStrategy.PER_RESPONSE.equals(resultOutputStrategy);
|
||||
final boolean perHitResultOutputStrategy = ResultOutputStrategy.PER_HIT.equals(resultOutputStrategy);
|
||||
final int expectedHitCount = 10 * iteration;
|
||||
|
||||
if (perResponseResultOutputStrategy && (iteration == 1 || iteration == 2)) {
|
||||
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("page.number", String.valueOf(iteration));
|
||||
assertState(runner.getStateManager(), paginationType, expectedHitCount, iteration);
|
||||
assertState(runner, paginationType, expectedHitCount, iteration);
|
||||
} else if (perHitResultOutputStrategy && (iteration == 1 || iteration == 2)) {
|
||||
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 10, 0, 0);
|
||||
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).forEach(hit -> {
|
||||
hit.assertAttributeEquals("hit.count", "1");
|
||||
hit.assertAttributeEquals("page.number", String.valueOf(iteration));
|
||||
});
|
||||
assertState(runner.getStateManager(), paginationType, expectedHitCount, iteration);
|
||||
assertState(runner, paginationType, expectedHitCount, iteration);
|
||||
} else if ((perResponseResultOutputStrategy || perHitResultOutputStrategy) && iteration == 3) {
|
||||
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)) {
|
||||
AbstractJsonQueryElasticsearchTest.testCounts(runner, 0, 1, 0, 0);
|
||||
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
|
||||
runner.getFlowFilesForRelationship(AbstractJsonQueryElasticsearch.REL_HITS).get(0).assertAttributeEquals("page.number", "2");
|
||||
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 {
|
||||
stateManager.assertStateEquals(SearchElasticsearch.STATE_HIT_COUNT, Integer.toString(hitCount), Scope.LOCAL);
|
||||
stateManager.assertStateEquals(SearchElasticsearch.STATE_PAGE_COUNT, Integer.toString(pageCount), Scope.LOCAL);
|
||||
private void assertState(final TestRunner runner, final PaginationType paginationType, final int hitCount, final int pageCount) throws IOException {
|
||||
final MockStateManager stateManager = runner.getStateManager();
|
||||
|
||||
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());
|
||||
|
||||
switch (paginationType) {
|
||||
case SCROLL:
|
||||
stateManager.assertStateEquals(SearchElasticsearch.STATE_SCROLL_ID, "scrollId-" + pageCount, Scope.LOCAL);
|
||||
stateManager.assertStateNotSet(SearchElasticsearch.STATE_PIT_ID, Scope.LOCAL);
|
||||
stateManager.assertStateNotSet(SearchElasticsearch.STATE_SEARCH_AFTER, Scope.LOCAL);
|
||||
stateManager.assertStateEquals(SearchElasticsearch.STATE_SCROLL_ID, "scrollId-" + pageCount, getStateScope());
|
||||
stateManager.assertStateNotSet(SearchElasticsearch.STATE_PIT_ID, getStateScope());
|
||||
stateManager.assertStateNotSet(SearchElasticsearch.STATE_SEARCH_AFTER, getStateScope());
|
||||
break;
|
||||
case POINT_IN_TIME:
|
||||
stateManager.assertStateNotSet(SearchElasticsearch.STATE_SCROLL_ID, Scope.LOCAL);
|
||||
stateManager.assertStateEquals(SearchElasticsearch.STATE_PIT_ID, "pitId-" + pageCount, Scope.LOCAL);
|
||||
stateManager.assertStateEquals(SearchElasticsearch.STATE_SEARCH_AFTER, "[\"searchAfter-" + pageCount + "\"]", Scope.LOCAL);
|
||||
stateManager.assertStateNotSet(SearchElasticsearch.STATE_SCROLL_ID, getStateScope());
|
||||
stateManager.assertStateEquals(SearchElasticsearch.STATE_PIT_ID, "pitId-" + pageCount, getStateScope());
|
||||
stateManager.assertStateEquals(SearchElasticsearch.STATE_SEARCH_AFTER, "[\"searchAfter-" + pageCount + "\"]", getStateScope());
|
||||
break;
|
||||
case SEARCH_AFTER:
|
||||
stateManager.assertStateNotSet(SearchElasticsearch.STATE_SCROLL_ID, Scope.LOCAL);
|
||||
stateManager.assertStateNotSet(SearchElasticsearch.STATE_PIT_ID, Scope.LOCAL);
|
||||
stateManager.assertStateEquals(SearchElasticsearch.STATE_SEARCH_AFTER, "[\"searchAfter-" + pageCount + "\"]", Scope.LOCAL);
|
||||
stateManager.assertStateNotSet(SearchElasticsearch.STATE_SCROLL_ID, getStateScope());
|
||||
stateManager.assertStateNotSet(SearchElasticsearch.STATE_PIT_ID, getStateScope());
|
||||
stateManager.assertStateEquals(SearchElasticsearch.STATE_SEARCH_AFTER, "[\"searchAfter-" + pageCount + "\"]", getStateScope());
|
||||
break;
|
||||
default:
|
||||
fail("Unknown paginationType: " + paginationType);
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.nifi.logging.ComponentLog;
|
|||
import org.apache.nifi.processors.elasticsearch.mock.MockElasticsearchException;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
|
@ -43,9 +42,9 @@ public class TestElasticsearchClientService extends AbstractControllerService im
|
|||
|
||||
static {
|
||||
try {
|
||||
AGGS_RESULT = Files.readString(Paths.get("src/test/resources/TestElasticsearchClientService/aggsResult.json"));
|
||||
HITS_RESULT = Files.readString(Paths.get("src/test/resources/TestElasticsearchClientService/hitsResult.json"));
|
||||
} catch (IOException e) {
|
||||
AGGS_RESULT = JsonUtils.readString(Paths.get("src/test/resources/TestElasticsearchClientService/aggsResult.json"));
|
||||
HITS_RESULT = JsonUtils.readString(Paths.get("src/test/resources/TestElasticsearchClientService/hitsResult.json"));
|
||||
} catch (final IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
@ -61,11 +60,14 @@ public class TestElasticsearchClientService extends AbstractControllerService im
|
|||
private int maxPages = 1;
|
||||
private Map<String, String> requestParameters;
|
||||
|
||||
public TestElasticsearchClientService(boolean returnAggs) {
|
||||
private boolean scrolling = false;
|
||||
private String query;
|
||||
|
||||
public TestElasticsearchClientService(final boolean 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 (throwNotFoundInGet) {
|
||||
throw new MockElasticsearchException(false, true);
|
||||
|
@ -78,62 +80,65 @@ public class TestElasticsearchClientService extends AbstractControllerService im
|
|||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
|
||||
@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);
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexOperationResponse bulk(List<IndexOperationRequest> operations, Map<String, String> requestParameters) {
|
||||
public IndexOperationResponse bulk(final List<IndexOperationRequest> operations, final Map<String, String> requestParameters) {
|
||||
try {
|
||||
common(false, requestParameters);
|
||||
} catch (IOException e) {
|
||||
} catch (final IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return new IndexOperationResponse(100L);
|
||||
}
|
||||
|
||||
@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 {
|
||||
common(false, requestParameters);
|
||||
} catch (IOException e) {
|
||||
} catch (final IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
this.query = query;
|
||||
return null;
|
||||
}
|
||||
|
||||
@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);
|
||||
}
|
||||
|
||||
@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 {
|
||||
common(throwErrorInDelete, requestParameters);
|
||||
} catch (IOException e) {
|
||||
} catch (final IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return new DeleteOperationResponse(100L);
|
||||
}
|
||||
|
||||
@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);
|
||||
}
|
||||
|
||||
@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 {
|
||||
common(throwErrorInUpdate, requestParameters);
|
||||
} catch (IOException e) {
|
||||
} catch (final IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
this.query = query;
|
||||
return new UpdateOperationResponse(100L);
|
||||
}
|
||||
|
||||
|
@ -147,34 +152,37 @@ public class TestElasticsearchClientService extends AbstractControllerService im
|
|||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
|
||||
@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 {
|
||||
common(throwErrorInGet || throwNotFoundInGet, requestParameters);
|
||||
} catch (IOException e) {
|
||||
} catch (final IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
Map<String, Object> map = new LinkedHashMap<>(1);
|
||||
final Map<String, Object> map = new LinkedHashMap<>(1);
|
||||
map.put("msg", "one");
|
||||
return map;
|
||||
}
|
||||
|
||||
@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 {
|
||||
common(throwErrorInSearch, requestParameters);
|
||||
} catch (IOException e) {
|
||||
} catch (final IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
if (!scrolling) {
|
||||
this.query = query;
|
||||
}
|
||||
final SearchResponse response;
|
||||
if (pageCount++ < maxPages) {
|
||||
List<Map<String, Object>> hits = JsonUtils.readListOfMaps(HITS_RESULT);
|
||||
Map<String, Object> aggs = returnAggs && pageCount == 1 ? JsonUtils.readMap(AGGS_RESULT) : null;
|
||||
final List<Map<String, Object>> hits = JsonUtils.readListOfMaps(HITS_RESULT);
|
||||
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);
|
||||
} else {
|
||||
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
|
||||
public SearchResponse scroll(String scroll) {
|
||||
public SearchResponse scroll(final String scroll) {
|
||||
if (throwErrorInSearch) {
|
||||
throw new RuntimeException(new IOException("Simulated IOException - scroll"));
|
||||
}
|
||||
|
||||
|
||||
return search(null, null, null, requestParameters);
|
||||
scrolling = true;
|
||||
final SearchResponse response = search(null, null, null, requestParameters);
|
||||
scrolling = false;
|
||||
return response;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String initialisePointInTime(String index, String keepAlive) {
|
||||
public String initialisePointInTime(final String index, final String keepAlive) {
|
||||
if (throwErrorInPit) {
|
||||
throw new RuntimeException(new IOException("Simulated IOException - initialisePointInTime"));
|
||||
}
|
||||
|
@ -205,7 +215,7 @@ public class TestElasticsearchClientService extends AbstractControllerService im
|
|||
}
|
||||
|
||||
@Override
|
||||
public DeleteOperationResponse deletePointInTime(String pitId) {
|
||||
public DeleteOperationResponse deletePointInTime(final String pitId) {
|
||||
if (throwErrorInDelete) {
|
||||
throw new RuntimeException(new IOException("Simulated IOException - deletePointInTime"));
|
||||
}
|
||||
|
@ -214,7 +224,7 @@ public class TestElasticsearchClientService extends AbstractControllerService im
|
|||
}
|
||||
|
||||
@Override
|
||||
public DeleteOperationResponse deleteScroll(String scrollId) {
|
||||
public DeleteOperationResponse deleteScroll(final String scrollId) {
|
||||
if (throwErrorInDelete) {
|
||||
throw new RuntimeException(new IOException("Simulated IOException - deleteScroll"));
|
||||
}
|
||||
|
@ -227,27 +237,27 @@ public class TestElasticsearchClientService extends AbstractControllerService im
|
|||
return "http://localhost:9400/" + index + "/" + type;
|
||||
}
|
||||
|
||||
public void setThrowNotFoundInGet(boolean throwNotFoundInGet) {
|
||||
public void setThrowNotFoundInGet(final boolean throwNotFoundInGet) {
|
||||
this.throwNotFoundInGet = throwNotFoundInGet;
|
||||
}
|
||||
|
||||
public void setThrowErrorInGet(boolean throwErrorInGet) {
|
||||
public void setThrowErrorInGet(final boolean throwErrorInGet) {
|
||||
this.throwErrorInGet = throwErrorInGet;
|
||||
}
|
||||
|
||||
public void setThrowErrorInSearch(boolean throwErrorInSearch) {
|
||||
public void setThrowErrorInSearch(final boolean throwErrorInSearch) {
|
||||
this.throwErrorInSearch = throwErrorInSearch;
|
||||
}
|
||||
|
||||
public void setThrowErrorInDelete(boolean throwErrorInDelete) {
|
||||
public void setThrowErrorInDelete(final boolean throwErrorInDelete) {
|
||||
this.throwErrorInDelete = throwErrorInDelete;
|
||||
}
|
||||
|
||||
public void setThrowErrorInPit(boolean throwErrorInPit) {
|
||||
public void setThrowErrorInPit(final boolean throwErrorInPit) {
|
||||
this.throwErrorInPit = throwErrorInPit;
|
||||
}
|
||||
|
||||
public void setThrowErrorInUpdate(boolean throwErrorInUpdate) {
|
||||
public void setThrowErrorInUpdate(final boolean throwErrorInUpdate) {
|
||||
this.throwErrorInUpdate = throwErrorInUpdate;
|
||||
}
|
||||
|
||||
|
@ -255,11 +265,15 @@ public class TestElasticsearchClientService extends AbstractControllerService im
|
|||
this.pageCount = 0;
|
||||
}
|
||||
|
||||
public void setMaxPages(int maxPages) {
|
||||
public void setMaxPages(final int maxPages) {
|
||||
this.maxPages = maxPages;
|
||||
}
|
||||
|
||||
public Map<String, String> getRequestParameters() {
|
||||
return this.requestParameters;
|
||||
}
|
||||
|
||||
public String getQuery() {
|
||||
return query;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,8 +33,8 @@ public class UpdateByQueryElasticsearchTest extends AbstractByQueryElasticsearch
|
|||
}
|
||||
|
||||
@Override
|
||||
public Class<? extends AbstractByQueryElasticsearch> getTestProcessor() {
|
||||
return UpdateByQueryElasticsearch.class;
|
||||
public AbstractByQueryElasticsearch getTestProcessor() {
|
||||
return new UpdateByQueryElasticsearch();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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()
|
||||
);
|
||||
}
|
||||
}
|
|
@ -20,7 +20,6 @@ import org.apache.nifi.components.ConfigVerificationResult;
|
|||
import org.apache.nifi.elasticsearch.ElasticSearchClientService;
|
||||
import org.apache.nifi.elasticsearch.ElasticSearchClientServiceImpl;
|
||||
import org.apache.nifi.elasticsearch.integration.AbstractElasticsearchITBase;
|
||||
import org.apache.nifi.processor.Processor;
|
||||
import org.apache.nifi.processor.VerifiableProcessor;
|
||||
import org.apache.nifi.processors.elasticsearch.ElasticsearchRestProcessor;
|
||||
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.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 String ID = "1";
|
||||
|
||||
ElasticSearchClientServiceImpl service;
|
||||
|
||||
abstract Class<? extends Processor> getTestProcessorClass();
|
||||
abstract P getProcessor();
|
||||
|
||||
@BeforeEach
|
||||
void before() throws Exception {
|
||||
runner = TestRunners.newTestRunner(getTestProcessorClass());
|
||||
runner = TestRunners.newTestRunner(getProcessor());
|
||||
|
||||
service = new ElasticSearchClientServiceImpl();
|
||||
runner.addControllerService(CLIENT_SERVICE_NAME, service);
|
||||
|
@ -110,9 +109,8 @@ abstract class AbstractElasticsearch_IT extends AbstractElasticsearchITBase {
|
|||
assertEquals(1, indexResults.size(), results.toString());
|
||||
final ConfigVerificationResult result = indexResults.get(0);
|
||||
|
||||
final ElasticsearchRestProcessor processor = (ElasticsearchRestProcessor) getTestProcessorClass().getConstructor().newInstance();
|
||||
final ConfigVerificationResult.Outcome expectedOutcome;
|
||||
if (processor.isIndexNotExistSuccessful()) {
|
||||
if (getProcessor().isIndexNotExistSuccessful()) {
|
||||
expectedOutcome = ConfigVerificationResult.Outcome.SUCCESSFUL;
|
||||
} else {
|
||||
if (expectedExists) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -17,7 +17,6 @@
|
|||
package org.apache.nifi.processors.elasticsearch.integration;
|
||||
|
||||
import org.apache.nifi.components.ConfigVerificationResult;
|
||||
import org.apache.nifi.processor.Processor;
|
||||
import org.apache.nifi.processor.VerifiableProcessor;
|
||||
import org.apache.nifi.processors.elasticsearch.ElasticsearchRestProcessor;
|
||||
import org.apache.nifi.processors.elasticsearch.GetElasticsearch;
|
||||
|
@ -31,8 +30,8 @@ import java.util.Objects;
|
|||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
class GetElasticsearch_IT extends AbstractElasticsearch_IT {
|
||||
Class<? extends Processor> getTestProcessorClass() {
|
||||
return GetElasticsearch.class;
|
||||
ElasticsearchRestProcessor getProcessor() {
|
||||
return new GetElasticsearch();
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -16,11 +16,11 @@
|
|||
*/
|
||||
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;
|
||||
|
||||
class PutElasticsearchJson_IT extends AbstractElasticsearch_IT {
|
||||
Class<? extends Processor> getTestProcessorClass() {
|
||||
return PutElasticsearchJson.class;
|
||||
class PutElasticsearchJson_IT extends AbstractElasticsearch_IT<AbstractPutElasticsearch> {
|
||||
AbstractPutElasticsearch getProcessor() {
|
||||
return new PutElasticsearchJson();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -33,42 +33,42 @@ import java.util.Map;
|
|||
|
||||
public class AbstractMockElasticsearchClient extends AbstractControllerService implements ElasticSearchClientService {
|
||||
@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;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexOperationResponse add(IndexOperationRequest operation, Map<String, String> requestParameters) {
|
||||
public IndexOperationResponse add(final IndexOperationRequest operation, final Map<String, String> requestParameters) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
|
||||
|
@ -82,42 +82,42 @@ public class AbstractMockElasticsearchClient extends AbstractControllerService i
|
|||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SearchResponse scroll(String scroll) {
|
||||
public SearchResponse scroll(final String scroll) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String initialisePointInTime(String index, String keepAlive) {
|
||||
public String initialisePointInTime(final String index, final String keepAlive) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DeleteOperationResponse deletePointInTime(String pitId) {
|
||||
public DeleteOperationResponse deletePointInTime(final String pitId) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DeleteOperationResponse deleteScroll(String scrollId) {
|
||||
public DeleteOperationResponse deleteScroll(final String scrollId) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@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 : "");
|
||||
}
|
||||
|
||||
|
@ -129,7 +129,7 @@ public class AbstractMockElasticsearchClient extends AbstractControllerService i
|
|||
return throwRetriableError;
|
||||
}
|
||||
|
||||
public void setThrowRetriableError(boolean throwRetriableError) {
|
||||
public void setThrowRetriableError(final boolean throwRetriableError) {
|
||||
this.throwRetriableError = throwRetriableError;
|
||||
}
|
||||
|
||||
|
@ -141,7 +141,7 @@ public class AbstractMockElasticsearchClient extends AbstractControllerService i
|
|||
return throwFatalError;
|
||||
}
|
||||
|
||||
public void setThrowFatalError(boolean throwFatalError) {
|
||||
public void setThrowFatalError(final boolean throwFatalError) {
|
||||
this.throwFatalError = throwFatalError;
|
||||
}
|
||||
|
||||
|
|
|
@ -29,7 +29,7 @@ public class MockBulkLoadClientService extends AbstractMockElasticsearchClient {
|
|||
private Consumer<Map<String, String>> evalParametersConsumer;
|
||||
|
||||
@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()) {
|
||||
throw new MockElasticsearchException(true, false);
|
||||
} else if (getThrowFatalError()) {
|
||||
|
@ -47,15 +47,15 @@ public class MockBulkLoadClientService extends AbstractMockElasticsearchClient {
|
|||
return response;
|
||||
}
|
||||
|
||||
public void setResponse(IndexOperationResponse response) {
|
||||
public void setResponse(final IndexOperationResponse response) {
|
||||
this.response = response;
|
||||
}
|
||||
|
||||
public void setEvalConsumer(Consumer<List<IndexOperationRequest>> evalConsumer) {
|
||||
public void setEvalConsumer(final Consumer<List<IndexOperationRequest>> evalConsumer) {
|
||||
this.evalConsumer = evalConsumer;
|
||||
}
|
||||
|
||||
public void setEvalParametersConsumer(Consumer<Map<String, String>> evalParametersConsumer) {
|
||||
public void setEvalParametersConsumer(final Consumer<Map<String, String>> evalParametersConsumer) {
|
||||
this.evalParametersConsumer = evalParametersConsumer;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,13 +19,13 @@ package org.apache.nifi.processors.elasticsearch.mock;
|
|||
import org.apache.nifi.elasticsearch.ElasticsearchException;
|
||||
|
||||
public class MockElasticsearchException extends ElasticsearchException {
|
||||
public MockElasticsearchException(boolean elastic, boolean notFound) {
|
||||
public MockElasticsearchException(final boolean elastic, final boolean notFound) {
|
||||
this(new Exception());
|
||||
this.elastic = elastic;
|
||||
this.notFound = notFound;
|
||||
}
|
||||
|
||||
public MockElasticsearchException(Exception ex) {
|
||||
public MockElasticsearchException(final Exception ex) {
|
||||
super(ex);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -14,8 +14,9 @@
|
|||
# limitations under the License.
|
||||
|
||||
#create mapping
|
||||
PUT:messages/:{ "mappings":{"_doc":{ "properties":{ "msg":{"type":"keyword"}}}}}
|
||||
PUT:messages/:{ "mappings":{"_doc":{ "properties":{ "msg":{"type":"keyword"}, "num":{"type":"integer"}}}}}
|
||||
|
||||
#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 }
|
||||
|
|
|
@ -14,7 +14,9 @@
|
|||
# limitations under the License.
|
||||
|
||||
#create mapping
|
||||
PUT:messages/:{ "mappings":{ "properties":{ "msg":{"type":"keyword"}}}}
|
||||
PUT:messages/:{ "mappings":{ "properties":{ "msg":{"type":"keyword"}, "num":{"type":"integer"}}}}
|
||||
|
||||
#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 }
|
||||
|
|
|
@ -14,7 +14,9 @@
|
|||
# limitations under the License.
|
||||
|
||||
#create mapping
|
||||
PUT:messages/:{ "mappings":{ "properties":{ "msg":{"type":"keyword"}}}}
|
||||
PUT:messages/:{ "mappings":{ "properties":{ "msg":{"type":"keyword"}, "num":{"type":"integer"}}}}
|
||||
|
||||
#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 }
|
||||
|
|
|
@ -52,7 +52,7 @@ import static org.apache.http.auth.AuthScope.ANY;
|
|||
public abstract class AbstractElasticsearchITBase {
|
||||
// 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
|
||||
.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));
|
||||
private static final int PORT = 9200;
|
||||
protected static final ElasticsearchContainer ELASTICSEARCH_CONTAINER = new ElasticsearchContainer(IMAGE)
|
||||
|
|
|
@ -94,7 +94,7 @@ language governing permissions and limitations under the License. -->
|
|||
</activation>
|
||||
<properties>
|
||||
<!-- 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>
|
||||
</properties>
|
||||
<build>
|
||||
|
@ -125,7 +125,7 @@ language governing permissions and limitations under the License. -->
|
|||
<profile>
|
||||
<id>elasticsearch7</id>
|
||||
<properties>
|
||||
<elasticsearch_docker_image>7.17.10</elasticsearch_docker_image>
|
||||
<elasticsearch_docker_image>7.17.13</elasticsearch_docker_image>
|
||||
</properties>
|
||||
</profile>
|
||||
</profiles>
|
||||
|
|
Loading…
Reference in New Issue