First step towards incremental reduction of query responses (#23253)
Today all query results are buffered up until we received responses of all shards. This can hold on to a significant amount of memory if the number of shards is large. This commit adds a first step towards incrementally reducing aggregations results if a, per search request, configurable amount of responses are received. If enough query results have been received and buffered all so-far received aggregation responses will be reduced and released to be GCed.
This commit is contained in:
parent
39ed76c58b
commit
f933f80902
|
@ -42,7 +42,6 @@ import org.elasticsearch.transport.Transport;
|
|||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.StringJoiner;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.function.Function;
|
||||
|
@ -61,7 +60,7 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
**/
|
||||
private final Function<String, Transport.Connection> nodeIdToConnection;
|
||||
private final SearchTask task;
|
||||
private final AtomicArray<Result> results;
|
||||
private final SearchPhaseResults<Result> results;
|
||||
private final long clusterStateVersion;
|
||||
private final Map<String, AliasFilter> aliasFilter;
|
||||
private final Map<String, Float> concreteIndexBoosts;
|
||||
|
@ -76,7 +75,7 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
Map<String, AliasFilter> aliasFilter, Map<String, Float> concreteIndexBoosts,
|
||||
Executor executor, SearchRequest request,
|
||||
ActionListener<SearchResponse> listener, GroupShardsIterator shardsIts, long startTime,
|
||||
long clusterStateVersion, SearchTask task) {
|
||||
long clusterStateVersion, SearchTask task, SearchPhaseResults<Result> resultConsumer) {
|
||||
super(name, request, shardsIts, logger);
|
||||
this.startTime = startTime;
|
||||
this.logger = logger;
|
||||
|
@ -87,9 +86,9 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
this.listener = listener;
|
||||
this.nodeIdToConnection = nodeIdToConnection;
|
||||
this.clusterStateVersion = clusterStateVersion;
|
||||
results = new AtomicArray<>(shardsIts.size());
|
||||
this.concreteIndexBoosts = concreteIndexBoosts;
|
||||
this.aliasFilter = aliasFilter;
|
||||
this.results = resultConsumer;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -105,7 +104,7 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
* This is the main entry point for a search. This method starts the search execution of the initial phase.
|
||||
*/
|
||||
public final void start() {
|
||||
if (results.length() == 0) {
|
||||
if (getNumShards() == 0) {
|
||||
//no search shards to search on, bail with empty response
|
||||
//(it happens with search across _all with no indices around and consistent with broadcast operations)
|
||||
listener.onResponse(new SearchResponse(InternalSearchResponse.empty(), null, 0, 0, buildTookInMillis(),
|
||||
|
@ -130,8 +129,8 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
onPhaseFailure(currentPhase, "all shards failed", null);
|
||||
} else {
|
||||
if (logger.isTraceEnabled()) {
|
||||
final String resultsFrom = results.asList().stream()
|
||||
.map(r -> r.value.shardTarget().toString()).collect(Collectors.joining(","));
|
||||
final String resultsFrom = results.getSuccessfulResults()
|
||||
.map(r -> r.shardTarget().toString()).collect(Collectors.joining(","));
|
||||
logger.trace("[{}] Moving to next phase: [{}], based on results from: {} (cluster state version: {})",
|
||||
currentPhase.getName(), nextPhase.getName(), resultsFrom, clusterStateVersion);
|
||||
}
|
||||
|
@ -178,7 +177,7 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
synchronized (shardFailuresMutex) {
|
||||
shardFailures = this.shardFailures.get(); // read again otherwise somebody else has created it?
|
||||
if (shardFailures == null) { // still null so we are the first and create a new instance
|
||||
shardFailures = new AtomicArray<>(results.length());
|
||||
shardFailures = new AtomicArray<>(getNumShards());
|
||||
this.shardFailures.set(shardFailures);
|
||||
}
|
||||
}
|
||||
|
@ -194,7 +193,7 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
}
|
||||
}
|
||||
|
||||
if (results.get(shardIndex) != null) {
|
||||
if (results.hasResult(shardIndex)) {
|
||||
assert failure == null : "shard failed before but shouldn't: " + failure;
|
||||
successfulOps.decrementAndGet(); // if this shard was successful before (initial phase) we have to adjust the counter
|
||||
}
|
||||
|
@ -207,22 +206,22 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
* @param exception the exception explaining or causing the phase failure
|
||||
*/
|
||||
private void raisePhaseFailure(SearchPhaseExecutionException exception) {
|
||||
for (AtomicArray.Entry<Result> entry : results.asList()) {
|
||||
results.getSuccessfulResults().forEach((entry) -> {
|
||||
try {
|
||||
Transport.Connection connection = nodeIdToConnection.apply(entry.value.shardTarget().getNodeId());
|
||||
sendReleaseSearchContext(entry.value.id(), connection);
|
||||
Transport.Connection connection = nodeIdToConnection.apply(entry.shardTarget().getNodeId());
|
||||
sendReleaseSearchContext(entry.id(), connection);
|
||||
} catch (Exception inner) {
|
||||
inner.addSuppressed(exception);
|
||||
logger.trace("failed to release context", inner);
|
||||
}
|
||||
}
|
||||
});
|
||||
listener.onFailure(exception);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void onShardSuccess(int shardIndex, Result result) {
|
||||
successfulOps.incrementAndGet();
|
||||
results.set(shardIndex, result);
|
||||
results.consumeResult(shardIndex, result);
|
||||
if (logger.isTraceEnabled()) {
|
||||
logger.trace("got first-phase result from {}", result != null ? result.shardTarget() : null);
|
||||
}
|
||||
|
@ -242,7 +241,7 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
|
||||
@Override
|
||||
public final int getNumShards() {
|
||||
return results.length();
|
||||
return results.getNumShards();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -262,7 +261,7 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
|
||||
@Override
|
||||
public final SearchResponse buildSearchResponse(InternalSearchResponse internalSearchResponse, String scrollId) {
|
||||
return new SearchResponse(internalSearchResponse, scrollId, results.length(), successfulOps.get(),
|
||||
return new SearchResponse(internalSearchResponse, scrollId, getNumShards(), successfulOps.get(),
|
||||
buildTookInMillis(), buildShardFailures());
|
||||
}
|
||||
|
||||
|
@ -310,6 +309,5 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
* executed shard request
|
||||
* @param context the search context for the next phase
|
||||
*/
|
||||
protected abstract SearchPhase getNextPhase(AtomicArray<Result> results, SearchPhaseContext context);
|
||||
|
||||
protected abstract SearchPhase getNextPhase(SearchPhaseResults<Result> results, SearchPhaseContext context);
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
package org.elasticsearch.action.search;
|
||||
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||
import org.elasticsearch.common.util.concurrent.CountDown;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
|
@ -30,17 +29,13 @@ import org.elasticsearch.search.SearchShardTarget;
|
|||
* where the given index is used to set the result on the array.
|
||||
*/
|
||||
final class CountedCollector<R extends SearchPhaseResult> {
|
||||
private final AtomicArray<R> resultArray;
|
||||
private final ResultConsumer<R> resultConsumer;
|
||||
private final CountDown counter;
|
||||
private final Runnable onFinish;
|
||||
private final SearchPhaseContext context;
|
||||
|
||||
CountedCollector(AtomicArray<R> resultArray, int expectedOps, Runnable onFinish, SearchPhaseContext context) {
|
||||
if (expectedOps > resultArray.length()) {
|
||||
throw new IllegalStateException("unexpected number of operations. got: " + expectedOps + " but array size is: "
|
||||
+ resultArray.length());
|
||||
}
|
||||
this.resultArray = resultArray;
|
||||
CountedCollector(ResultConsumer<R> resultConsumer, int expectedOps, Runnable onFinish, SearchPhaseContext context) {
|
||||
this.resultConsumer = resultConsumer;
|
||||
this.counter = new CountDown(expectedOps);
|
||||
this.onFinish = onFinish;
|
||||
this.context = context;
|
||||
|
@ -63,7 +58,7 @@ final class CountedCollector<R extends SearchPhaseResult> {
|
|||
void onResult(int index, R result, SearchShardTarget target) {
|
||||
try {
|
||||
result.shardTarget(target);
|
||||
resultArray.set(index, result);
|
||||
resultConsumer.consume(index, result);
|
||||
} finally {
|
||||
countDown();
|
||||
}
|
||||
|
@ -80,4 +75,12 @@ final class CountedCollector<R extends SearchPhaseResult> {
|
|||
countDown();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A functional interface to plug in shard result consumers to this collector
|
||||
*/
|
||||
@FunctionalInterface
|
||||
public interface ResultConsumer<R extends SearchPhaseResult> {
|
||||
void consume(int shardIndex, R result);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,18 +40,19 @@ import java.util.function.Function;
|
|||
* @see CountedCollector#onFailure(int, SearchShardTarget, Exception)
|
||||
*/
|
||||
final class DfsQueryPhase extends SearchPhase {
|
||||
private final AtomicArray<QuerySearchResultProvider> queryResult;
|
||||
private final InitialSearchPhase.SearchPhaseResults<QuerySearchResultProvider> queryResult;
|
||||
private final SearchPhaseController searchPhaseController;
|
||||
private final AtomicArray<DfsSearchResult> dfsSearchResults;
|
||||
private final Function<AtomicArray<QuerySearchResultProvider>, SearchPhase> nextPhaseFactory;
|
||||
private final Function<InitialSearchPhase.SearchPhaseResults<QuerySearchResultProvider>, SearchPhase> nextPhaseFactory;
|
||||
private final SearchPhaseContext context;
|
||||
private final SearchTransportService searchTransportService;
|
||||
|
||||
DfsQueryPhase(AtomicArray<DfsSearchResult> dfsSearchResults,
|
||||
SearchPhaseController searchPhaseController,
|
||||
Function<AtomicArray<QuerySearchResultProvider>, SearchPhase> nextPhaseFactory, SearchPhaseContext context) {
|
||||
Function<InitialSearchPhase.SearchPhaseResults<QuerySearchResultProvider>, SearchPhase> nextPhaseFactory,
|
||||
SearchPhaseContext context) {
|
||||
super("dfs_query");
|
||||
this.queryResult = new AtomicArray<>(dfsSearchResults.length());
|
||||
this.queryResult = searchPhaseController.newSearchPhaseResults(context.getRequest(), context.getNumShards());
|
||||
this.searchPhaseController = searchPhaseController;
|
||||
this.dfsSearchResults = dfsSearchResults;
|
||||
this.nextPhaseFactory = nextPhaseFactory;
|
||||
|
@ -64,7 +65,8 @@ final class DfsQueryPhase extends SearchPhase {
|
|||
// TODO we can potentially also consume the actual per shard results from the initial phase here in the aggregateDfs
|
||||
// to free up memory early
|
||||
final AggregatedDfs dfs = searchPhaseController.aggregateDfs(dfsSearchResults);
|
||||
final CountedCollector<QuerySearchResultProvider> counter = new CountedCollector<>(queryResult, dfsSearchResults.asList().size(),
|
||||
final CountedCollector<QuerySearchResultProvider> counter = new CountedCollector<>(queryResult::consumeResult,
|
||||
dfsSearchResults.asList().size(),
|
||||
() -> {
|
||||
context.executeNextPhase(this, nextPhaseFactory.apply(queryResult));
|
||||
}, context);
|
||||
|
|
|
@ -49,29 +49,31 @@ final class FetchSearchPhase extends SearchPhase {
|
|||
private final Function<SearchResponse, SearchPhase> nextPhaseFactory;
|
||||
private final SearchPhaseContext context;
|
||||
private final Logger logger;
|
||||
private final InitialSearchPhase.SearchPhaseResults<QuerySearchResultProvider> resultConsumer;
|
||||
|
||||
FetchSearchPhase(AtomicArray<QuerySearchResultProvider> queryResults,
|
||||
FetchSearchPhase(InitialSearchPhase.SearchPhaseResults<QuerySearchResultProvider> resultConsumer,
|
||||
SearchPhaseController searchPhaseController,
|
||||
SearchPhaseContext context) {
|
||||
this(queryResults, searchPhaseController, context,
|
||||
this(resultConsumer, searchPhaseController, context,
|
||||
(response) -> new ExpandSearchPhase(context, response, // collapse only happens if the request has inner hits
|
||||
(finalResponse) -> sendResponsePhase(finalResponse, context)));
|
||||
}
|
||||
|
||||
FetchSearchPhase(AtomicArray<QuerySearchResultProvider> queryResults,
|
||||
FetchSearchPhase(InitialSearchPhase.SearchPhaseResults<QuerySearchResultProvider> resultConsumer,
|
||||
SearchPhaseController searchPhaseController,
|
||||
SearchPhaseContext context, Function<SearchResponse, SearchPhase> nextPhaseFactory) {
|
||||
super("fetch");
|
||||
if (context.getNumShards() != queryResults.length()) {
|
||||
if (context.getNumShards() != resultConsumer.getNumShards()) {
|
||||
throw new IllegalStateException("number of shards must match the length of the query results but doesn't:"
|
||||
+ context.getNumShards() + "!=" + queryResults.length());
|
||||
+ context.getNumShards() + "!=" + resultConsumer.getNumShards());
|
||||
}
|
||||
this.fetchResults = new AtomicArray<>(queryResults.length());
|
||||
this.fetchResults = new AtomicArray<>(resultConsumer.getNumShards());
|
||||
this.searchPhaseController = searchPhaseController;
|
||||
this.queryResults = queryResults;
|
||||
this.queryResults = resultConsumer.results;
|
||||
this.nextPhaseFactory = nextPhaseFactory;
|
||||
this.context = context;
|
||||
this.logger = context.getLogger();
|
||||
this.resultConsumer = resultConsumer;
|
||||
|
||||
}
|
||||
|
||||
|
@ -99,7 +101,7 @@ final class FetchSearchPhase extends SearchPhase {
|
|||
ScoreDoc[] sortedShardDocs = searchPhaseController.sortDocs(isScrollSearch, queryResults);
|
||||
String scrollId = isScrollSearch ? TransportSearchHelper.buildScrollId(queryResults) : null;
|
||||
List<AtomicArray.Entry<QuerySearchResultProvider>> queryResultsAsList = queryResults.asList();
|
||||
final SearchPhaseController.ReducedQueryPhase reducedQueryPhase = searchPhaseController.reducedQueryPhase(queryResultsAsList);
|
||||
final SearchPhaseController.ReducedQueryPhase reducedQueryPhase = resultConsumer.reduce();
|
||||
final boolean queryAndFetchOptimization = queryResults.length() == 1;
|
||||
final Runnable finishPhase = ()
|
||||
-> moveToNextPhase(searchPhaseController, sortedShardDocs, scrollId, reducedQueryPhase, queryAndFetchOptimization ?
|
||||
|
@ -119,7 +121,7 @@ final class FetchSearchPhase extends SearchPhase {
|
|||
final ScoreDoc[] lastEmittedDocPerShard = isScrollSearch ?
|
||||
searchPhaseController.getLastEmittedDocPerShard(reducedQueryPhase, sortedShardDocs, numShards)
|
||||
: null;
|
||||
final CountedCollector<FetchSearchResult> counter = new CountedCollector<>(fetchResults,
|
||||
final CountedCollector<FetchSearchResult> counter = new CountedCollector<>(fetchResults::set,
|
||||
docIdsToLoad.length, // we count down every shard in the result no matter if we got any results or not
|
||||
finishPhase, context);
|
||||
for (int i = 0; i < docIdsToLoad.length; i++) {
|
||||
|
|
|
@ -28,12 +28,14 @@ import org.elasticsearch.cluster.routing.GroupShardsIterator;
|
|||
import org.elasticsearch.cluster.routing.ShardIterator;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.transport.ConnectTransportException;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* This is an abstract base class that encapsulates the logic to fan out to all shards in provided {@link GroupShardsIterator}
|
||||
|
@ -213,4 +215,53 @@ abstract class InitialSearchPhase<FirstResult extends SearchPhaseResult> extends
|
|||
* @param listener the listener to notify on response
|
||||
*/
|
||||
protected abstract void executePhaseOnShard(ShardIterator shardIt, ShardRouting shard, ActionListener<FirstResult> listener);
|
||||
|
||||
/**
|
||||
* This class acts as a basic result collection that can be extended to do on-the-fly reduction or result processing
|
||||
*/
|
||||
static class SearchPhaseResults<Result extends SearchPhaseResult> {
|
||||
final AtomicArray<Result> results;
|
||||
|
||||
SearchPhaseResults(int size) {
|
||||
results = new AtomicArray<>(size);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of expected results this class should collect
|
||||
*/
|
||||
final int getNumShards() {
|
||||
return results.length();
|
||||
}
|
||||
|
||||
/**
|
||||
* A stream of all non-null (successful) shard results
|
||||
*/
|
||||
final Stream<Result> getSuccessfulResults() {
|
||||
return results.asList().stream().map(e -> e.value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Consumes a single shard result
|
||||
* @param shardIndex the shards index, this is a 0-based id that is used to establish a 1 to 1 mapping to the searched shards
|
||||
* @param result the shards result
|
||||
*/
|
||||
void consumeResult(int shardIndex, Result result) {
|
||||
assert results.get(shardIndex) == null : "shardIndex: " + shardIndex + " is already set";
|
||||
results.set(shardIndex, result);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> iff a result if present for the given shard ID.
|
||||
*/
|
||||
final boolean hasResult(int shardIndex) {
|
||||
return results.get(shardIndex) != null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reduces the collected results
|
||||
*/
|
||||
SearchPhaseController.ReducedQueryPhase reduce() {
|
||||
throw new UnsupportedOperationException("reduce is not supported");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.elasticsearch.action.ActionListener;
|
|||
import org.elasticsearch.cluster.routing.GroupShardsIterator;
|
||||
import org.elasticsearch.cluster.routing.ShardIterator;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||
import org.elasticsearch.search.dfs.DfsSearchResult;
|
||||
import org.elasticsearch.search.internal.AliasFilter;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
|
@ -43,7 +42,7 @@ final class SearchDfsQueryThenFetchAsyncAction extends AbstractSearchAsyncAction
|
|||
ActionListener<SearchResponse> listener, GroupShardsIterator shardsIts, long startTime,
|
||||
long clusterStateVersion, SearchTask task) {
|
||||
super("dfs", logger, searchTransportService, nodeIdToConnection, aliasFilter, concreteIndexBoosts, executor,
|
||||
request, listener, shardsIts, startTime, clusterStateVersion, task);
|
||||
request, listener, shardsIts, startTime, clusterStateVersion, task, new SearchPhaseResults<>(shardsIts.size()));
|
||||
this.searchPhaseController = searchPhaseController;
|
||||
}
|
||||
|
||||
|
@ -54,8 +53,8 @@ final class SearchDfsQueryThenFetchAsyncAction extends AbstractSearchAsyncAction
|
|||
}
|
||||
|
||||
@Override
|
||||
protected SearchPhase getNextPhase(AtomicArray<DfsSearchResult> results, SearchPhaseContext context) {
|
||||
return new DfsQueryPhase(results, searchPhaseController,
|
||||
protected SearchPhase getNextPhase(SearchPhaseResults<DfsSearchResult> results, SearchPhaseContext context) {
|
||||
return new DfsQueryPhase(results.results, searchPhaseController,
|
||||
(queryResults) -> new FetchSearchPhase(queryResults, searchPhaseController, context), context);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -114,4 +114,5 @@ interface SearchPhaseContext extends ActionListener<SearchResponse>, Executor {
|
|||
* a response is returned to the user indicating that all shards have failed.
|
||||
*/
|
||||
void executeNextPhase(SearchPhase currentPhase, SearchPhase nextPhase);
|
||||
|
||||
}
|
||||
|
|
|
@ -44,6 +44,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
|
|||
import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.aggregations.pipeline.SiblingPipelineAggregator;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.dfs.AggregatedDfs;
|
||||
import org.elasticsearch.search.dfs.DfsSearchResult;
|
||||
import org.elasticsearch.search.fetch.FetchSearchResult;
|
||||
|
@ -461,23 +462,50 @@ public class SearchPhaseController extends AbstractComponent {
|
|||
|
||||
/**
|
||||
* Reduces the given query results and consumes all aggregations and profile results.
|
||||
* @param queryResults a list of non-null query shard results
|
||||
*/
|
||||
public final ReducedQueryPhase reducedQueryPhase(List<? extends AtomicArray.Entry<? extends QuerySearchResultProvider>> queryResults) {
|
||||
return reducedQueryPhase(queryResults, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Reduces the given query results and consumes all aggregations and profile results.
|
||||
* @param queryResults a list of non-null query shard results
|
||||
* @param bufferdAggs a list of pre-collected / buffered aggregations. if this list is non-null all aggregations have been consumed
|
||||
* from all non-null query results.
|
||||
* @see QuerySearchResult#consumeAggs()
|
||||
* @see QuerySearchResult#consumeProfileResult()
|
||||
*/
|
||||
public final ReducedQueryPhase reducedQueryPhase(List<? extends AtomicArray.Entry<? extends QuerySearchResultProvider>> queryResults) {
|
||||
private ReducedQueryPhase reducedQueryPhase(List<? extends AtomicArray.Entry<? extends QuerySearchResultProvider>> queryResults,
|
||||
List<InternalAggregations> bufferdAggs) {
|
||||
long totalHits = 0;
|
||||
long fetchHits = 0;
|
||||
float maxScore = Float.NEGATIVE_INFINITY;
|
||||
boolean timedOut = false;
|
||||
Boolean terminatedEarly = null;
|
||||
if (queryResults.isEmpty()) {
|
||||
if (queryResults.isEmpty()) { // early terminate we have nothing to reduce
|
||||
return new ReducedQueryPhase(totalHits, fetchHits, maxScore, timedOut, terminatedEarly, null, null, null, null);
|
||||
}
|
||||
QuerySearchResult firstResult = queryResults.get(0).value.queryResult();
|
||||
final QuerySearchResult firstResult = queryResults.get(0).value.queryResult();
|
||||
final boolean hasSuggest = firstResult.suggest() != null;
|
||||
final boolean hasAggs = firstResult.hasAggs();
|
||||
final boolean hasProfileResults = firstResult.hasProfileResults();
|
||||
final List<InternalAggregations> aggregationsList = hasAggs ? new ArrayList<>(queryResults.size()) : Collections.emptyList();
|
||||
final boolean consumeAggs;
|
||||
final List<InternalAggregations> aggregationsList;
|
||||
if (bufferdAggs != null) {
|
||||
consumeAggs = false;
|
||||
// we already have results from intermediate reduces and just need to perform the final reduce
|
||||
assert firstResult.hasAggs() : "firstResult has no aggs but we got non null buffered aggs?";
|
||||
aggregationsList = bufferdAggs;
|
||||
} else if (firstResult.hasAggs()) {
|
||||
// the number of shards was less than the buffer size so we reduce agg results directly
|
||||
aggregationsList = new ArrayList<>(queryResults.size());
|
||||
consumeAggs = true;
|
||||
} else {
|
||||
// no aggregations
|
||||
aggregationsList = Collections.emptyList();
|
||||
consumeAggs = false;
|
||||
}
|
||||
|
||||
// count the total (we use the query result provider here, since we might not get any hits (we scrolled past them))
|
||||
final Map<String, List<Suggestion>> groupedSuggestions = hasSuggest ? new HashMap<>() : Collections.emptyMap();
|
||||
final Map<String, ProfileShardResult> profileResults = hasProfileResults ? new HashMap<>(queryResults.size())
|
||||
|
@ -506,7 +534,7 @@ public class SearchPhaseController extends AbstractComponent {
|
|||
suggestionList.add(suggestion);
|
||||
}
|
||||
}
|
||||
if (hasAggs) {
|
||||
if (consumeAggs) {
|
||||
aggregationsList.add((InternalAggregations) result.consumeAggs());
|
||||
}
|
||||
if (hasProfileResults) {
|
||||
|
@ -515,16 +543,27 @@ public class SearchPhaseController extends AbstractComponent {
|
|||
}
|
||||
}
|
||||
final Suggest suggest = groupedSuggestions.isEmpty() ? null : new Suggest(Suggest.reduce(groupedSuggestions));
|
||||
ReduceContext reduceContext = new ReduceContext(bigArrays, scriptService, true);
|
||||
final InternalAggregations aggregations = aggregationsList.isEmpty() ? null : reduceAggs(aggregationsList,
|
||||
firstResult.pipelineAggregators());
|
||||
firstResult.pipelineAggregators(), reduceContext);
|
||||
final SearchProfileShardResults shardResults = profileResults.isEmpty() ? null : new SearchProfileShardResults(profileResults);
|
||||
return new ReducedQueryPhase(totalHits, fetchHits, maxScore, timedOut, terminatedEarly, firstResult, suggest, aggregations,
|
||||
shardResults);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Performs an intermediate reduce phase on the aggregations. For instance with this reduce phase never prune information
|
||||
* that relevant for the final reduce step. For final reduce see {@link #reduceAggs(List, List, ReduceContext)}
|
||||
*/
|
||||
private InternalAggregations reduceAggsIncrementally(List<InternalAggregations> aggregationsList) {
|
||||
ReduceContext reduceContext = new ReduceContext(bigArrays, scriptService, false);
|
||||
return aggregationsList.isEmpty() ? null : reduceAggs(aggregationsList,
|
||||
null, reduceContext);
|
||||
}
|
||||
|
||||
private InternalAggregations reduceAggs(List<InternalAggregations> aggregationsList,
|
||||
List<SiblingPipelineAggregator> pipelineAggregators) {
|
||||
ReduceContext reduceContext = new ReduceContext(bigArrays, scriptService);
|
||||
List<SiblingPipelineAggregator> pipelineAggregators, ReduceContext reduceContext) {
|
||||
InternalAggregations aggregations = InternalAggregations.reduce(aggregationsList, reduceContext);
|
||||
if (pipelineAggregators != null) {
|
||||
List<InternalAggregation> newAggs = StreamSupport.stream(aggregations.spliterator(), false)
|
||||
|
@ -593,4 +632,91 @@ public class SearchPhaseController extends AbstractComponent {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A {@link org.elasticsearch.action.search.InitialSearchPhase.SearchPhaseResults} implementation
|
||||
* that incrementally reduces aggregation results as shard results are consumed.
|
||||
* This implementation can be configured to batch up a certain amount of results and only reduce them
|
||||
* iff the buffer is exhausted.
|
||||
*/
|
||||
static final class QueryPhaseResultConsumer
|
||||
extends InitialSearchPhase.SearchPhaseResults<QuerySearchResultProvider> {
|
||||
private final InternalAggregations[] buffer;
|
||||
private int index;
|
||||
private final SearchPhaseController controller;
|
||||
|
||||
/**
|
||||
* Creates a new {@link QueryPhaseResultConsumer}
|
||||
* @param controller a controller instance to reduce the query response objects
|
||||
* @param expectedResultSize the expected number of query results. Corresponds to the number of shards queried
|
||||
* @param bufferSize the size of the reduce buffer. if the buffer size is smaller than the number of expected results
|
||||
* the buffer is used to incrementally reduce aggregation results before all shards responded.
|
||||
*/
|
||||
private QueryPhaseResultConsumer(SearchPhaseController controller, int expectedResultSize, int bufferSize) {
|
||||
super(expectedResultSize);
|
||||
if (expectedResultSize != 1 && bufferSize < 2) {
|
||||
throw new IllegalArgumentException("buffer size must be >= 2 if there is more than one expected result");
|
||||
}
|
||||
if (expectedResultSize <= bufferSize) {
|
||||
throw new IllegalArgumentException("buffer size must be less than the expected result size");
|
||||
}
|
||||
this.controller = controller;
|
||||
// no need to buffer anything if we have less expected results. in this case we don't consume any results ahead of time.
|
||||
this.buffer = new InternalAggregations[bufferSize];
|
||||
}
|
||||
|
||||
@Override
|
||||
public void consumeResult(int shardIndex, QuerySearchResultProvider result) {
|
||||
super.consumeResult(shardIndex, result);
|
||||
QuerySearchResult queryResult = result.queryResult();
|
||||
assert queryResult.hasAggs() : "this collector should only be used if aggs are requested";
|
||||
consumeInternal(queryResult);
|
||||
}
|
||||
|
||||
private synchronized void consumeInternal(QuerySearchResult querySearchResult) {
|
||||
InternalAggregations aggregations = (InternalAggregations) querySearchResult.consumeAggs();
|
||||
if (index == buffer.length) {
|
||||
InternalAggregations reducedAggs = controller.reduceAggsIncrementally(Arrays.asList(buffer));
|
||||
Arrays.fill(buffer, null);
|
||||
buffer[0] = reducedAggs;
|
||||
index = 1;
|
||||
}
|
||||
final int i = index++;
|
||||
buffer[i] = aggregations;
|
||||
}
|
||||
|
||||
private synchronized List<InternalAggregations> getRemaining() {
|
||||
return Arrays.asList(buffer).subList(0, index);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReducedQueryPhase reduce() {
|
||||
return controller.reducedQueryPhase(results.asList(), getRemaining());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of buffered results
|
||||
*/
|
||||
int getNumBuffered() {
|
||||
return index;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a new SearchPhaseResults instance. This might return an instance that reduces search responses incrementally.
|
||||
*/
|
||||
InitialSearchPhase.SearchPhaseResults<QuerySearchResultProvider> newSearchPhaseResults(SearchRequest request, int numShards) {
|
||||
SearchSourceBuilder source = request.source();
|
||||
if (source != null && source.aggregations() != null) {
|
||||
if (request.getReduceUpTo() < numShards) {
|
||||
// only use this if there are aggs and if there are more shards than we should reduce at once
|
||||
return new QueryPhaseResultConsumer(this, numShards, request.getReduceUpTo());
|
||||
}
|
||||
}
|
||||
return new InitialSearchPhase.SearchPhaseResults(numShards) {
|
||||
@Override
|
||||
public ReducedQueryPhase reduce() {
|
||||
return reducedQueryPhase(results.asList());
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.elasticsearch.action.ActionListener;
|
|||
import org.elasticsearch.cluster.routing.GroupShardsIterator;
|
||||
import org.elasticsearch.cluster.routing.ShardIterator;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||
import org.elasticsearch.search.internal.AliasFilter;
|
||||
import org.elasticsearch.search.query.QuerySearchResultProvider;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
|
@ -44,17 +43,19 @@ final class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction<Qu
|
|||
GroupShardsIterator shardsIts, long startTime, long clusterStateVersion,
|
||||
SearchTask task) {
|
||||
super("query", logger, searchTransportService, nodeIdToConnection, aliasFilter, concreteIndexBoosts, executor,
|
||||
request, listener, shardsIts, startTime, clusterStateVersion, task);
|
||||
request, listener, shardsIts, startTime, clusterStateVersion, task,
|
||||
searchPhaseController.newSearchPhaseResults(request, shardsIts.size()));
|
||||
this.searchPhaseController = searchPhaseController;
|
||||
}
|
||||
|
||||
|
||||
protected void executePhaseOnShard(ShardIterator shardIt, ShardRouting shard, ActionListener listener) {
|
||||
getSearchTransport().sendExecuteQuery(getConnection(shard.currentNodeId()),
|
||||
buildShardSearchRequest(shardIt, shard), getTask(), listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SearchPhase getNextPhase(AtomicArray<QuerySearchResultProvider> results, SearchPhaseContext context) {
|
||||
protected SearchPhase getNextPhase(SearchPhaseResults<QuerySearchResultProvider> results, SearchPhaseContext context) {
|
||||
return new FetchSearchPhase(results, searchPhaseController, context);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.elasticsearch.action.search;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.action.IndicesRequest;
|
||||
|
@ -70,6 +71,8 @@ public final class SearchRequest extends ActionRequest implements IndicesRequest
|
|||
|
||||
private Scroll scroll;
|
||||
|
||||
private int reduceUpTo = 512;
|
||||
|
||||
private String[] types = Strings.EMPTY_ARRAY;
|
||||
|
||||
public static final IndicesOptions DEFAULT_INDICES_OPTIONS = IndicesOptions.strictExpandOpenAndForbidClosed();
|
||||
|
@ -274,6 +277,25 @@ public final class SearchRequest extends ActionRequest implements IndicesRequest
|
|||
return this.requestCache;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the number of shard results that should be reduced at once on the coordinating node. This value should be used as a protection
|
||||
* mechanism to reduce the memory overhead per search request if the potential number of shards in the request can be large.
|
||||
*/
|
||||
public void setReduceUpTo(int reduceUpTo) {
|
||||
if (reduceUpTo <= 1) {
|
||||
throw new IllegalArgumentException("reduceUpTo must be >= 2");
|
||||
}
|
||||
this.reduceUpTo = reduceUpTo;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of shard results that should be reduced at once on the coordinating node. This value should be used as a
|
||||
* protection mechanism to reduce the memory overhead per search request if the potential number of shards in the request can be large.
|
||||
*/
|
||||
public int getReduceUpTo() {
|
||||
return reduceUpTo;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true if the request only has suggest
|
||||
*/
|
||||
|
@ -320,6 +342,9 @@ public final class SearchRequest extends ActionRequest implements IndicesRequest
|
|||
types = in.readStringArray();
|
||||
indicesOptions = IndicesOptions.readIndicesOptions(in);
|
||||
requestCache = in.readOptionalBoolean();
|
||||
if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
|
||||
reduceUpTo = in.readVInt();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -337,6 +362,9 @@ public final class SearchRequest extends ActionRequest implements IndicesRequest
|
|||
out.writeStringArray(types);
|
||||
indicesOptions.writeIndicesOptions(out);
|
||||
out.writeOptionalBoolean(requestCache);
|
||||
if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
|
||||
out.writeVInt(reduceUpTo);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -523,4 +523,9 @@ public class SearchRequestBuilder extends ActionRequestBuilder<SearchRequest, Se
|
|||
}
|
||||
return request.source();
|
||||
}
|
||||
|
||||
public SearchRequestBuilder setReduceUpTo(int reduceUpTo) {
|
||||
this.request.setReduceUpTo(reduceUpTo);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -47,10 +47,21 @@ public abstract class InternalAggregation implements Aggregation, ToXContent, Na
|
|||
|
||||
private final BigArrays bigArrays;
|
||||
private final ScriptService scriptService;
|
||||
private final boolean isFinalReduce;
|
||||
|
||||
public ReduceContext(BigArrays bigArrays, ScriptService scriptService) {
|
||||
public ReduceContext(BigArrays bigArrays, ScriptService scriptService, boolean isFinalReduce) {
|
||||
this.bigArrays = bigArrays;
|
||||
this.scriptService = scriptService;
|
||||
this.isFinalReduce = isFinalReduce;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> iff the current reduce phase is the final reduce phase. This indicates if operations like
|
||||
* pipeline aggregations should be applied or if specific features like <tt>minDocCount</tt> should be taken into account.
|
||||
* Operations that are potentially loosing information can only be applied during the final reduce phase.
|
||||
*/
|
||||
public boolean isFinalReduce() {
|
||||
return isFinalReduce;
|
||||
}
|
||||
|
||||
public BigArrays bigArrays() {
|
||||
|
@ -111,9 +122,11 @@ public abstract class InternalAggregation implements Aggregation, ToXContent, Na
|
|||
*/
|
||||
public final InternalAggregation reduce(List<InternalAggregation> aggregations, ReduceContext reduceContext) {
|
||||
InternalAggregation aggResult = doReduce(aggregations, reduceContext);
|
||||
if (reduceContext.isFinalReduce()) {
|
||||
for (PipelineAggregator pipelineAggregator : pipelineAggregators) {
|
||||
aggResult = pipelineAggregator.reduce(aggResult, reduceContext);
|
||||
}
|
||||
}
|
||||
return aggResult;
|
||||
}
|
||||
|
||||
|
|
|
@ -192,7 +192,7 @@ public class InternalGeoHashGrid extends InternalMultiBucketAggregation<Internal
|
|||
}
|
||||
}
|
||||
|
||||
final int size = (int) Math.min(requiredSize, buckets.size());
|
||||
final int size = Math.toIntExact(reduceContext.isFinalReduce() == false ? buckets.size() : Math.min(requiredSize, buckets.size()));
|
||||
BucketPriorityQueue ordered = new BucketPriorityQueue(size);
|
||||
for (LongObjectPagedHashMap.Cursor<List<Bucket>> cursor : buckets) {
|
||||
List<Bucket> sameCellBuckets = cursor.value;
|
||||
|
|
|
@ -285,7 +285,7 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
|
|||
if (top.current.key != key) {
|
||||
// the key changes, reduce what we already buffered and reset the buffer for current buckets
|
||||
final Bucket reduced = currentBuckets.get(0).reduce(currentBuckets, reduceContext);
|
||||
if (reduced.getDocCount() >= minDocCount) {
|
||||
if (reduced.getDocCount() >= minDocCount || reduceContext.isFinalReduce() == false) {
|
||||
reducedBuckets.add(reduced);
|
||||
}
|
||||
currentBuckets.clear();
|
||||
|
@ -306,7 +306,7 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
|
|||
|
||||
if (currentBuckets.isEmpty() == false) {
|
||||
final Bucket reduced = currentBuckets.get(0).reduce(currentBuckets, reduceContext);
|
||||
if (reduced.getDocCount() >= minDocCount) {
|
||||
if (reduced.getDocCount() >= minDocCount || reduceContext.isFinalReduce() == false) {
|
||||
reducedBuckets.add(reduced);
|
||||
}
|
||||
}
|
||||
|
@ -382,7 +382,7 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
|
|||
addEmptyBuckets(reducedBuckets, reduceContext);
|
||||
}
|
||||
|
||||
if (order == InternalOrder.KEY_ASC) {
|
||||
if (order == InternalOrder.KEY_ASC || reduceContext.isFinalReduce() == false) {
|
||||
// nothing to do, data are already sorted since shards return
|
||||
// sorted buckets and the merge-sort performed by reduceBuckets
|
||||
// maintains order
|
||||
|
|
|
@ -308,7 +308,7 @@ public final class InternalHistogram extends InternalMultiBucketAggregation<Inte
|
|||
if (top.current.key != key) {
|
||||
// the key changes, reduce what we already buffered and reset the buffer for current buckets
|
||||
final Bucket reduced = currentBuckets.get(0).reduce(currentBuckets, reduceContext);
|
||||
if (reduced.getDocCount() >= minDocCount) {
|
||||
if (reduced.getDocCount() >= minDocCount || reduceContext.isFinalReduce() == false) {
|
||||
reducedBuckets.add(reduced);
|
||||
}
|
||||
currentBuckets.clear();
|
||||
|
@ -329,7 +329,7 @@ public final class InternalHistogram extends InternalMultiBucketAggregation<Inte
|
|||
|
||||
if (currentBuckets.isEmpty() == false) {
|
||||
final Bucket reduced = currentBuckets.get(0).reduce(currentBuckets, reduceContext);
|
||||
if (reduced.getDocCount() >= minDocCount) {
|
||||
if (reduced.getDocCount() >= minDocCount || reduceContext.isFinalReduce() == false) {
|
||||
reducedBuckets.add(reduced);
|
||||
}
|
||||
}
|
||||
|
@ -400,7 +400,7 @@ public final class InternalHistogram extends InternalMultiBucketAggregation<Inte
|
|||
addEmptyBuckets(reducedBuckets, reduceContext);
|
||||
}
|
||||
|
||||
if (order == InternalOrder.KEY_ASC) {
|
||||
if (order == InternalOrder.KEY_ASC || reduceContext.isFinalReduce() == false) {
|
||||
// nothing to do, data are already sorted since shards return
|
||||
// sorted buckets and the merge-sort performed by reduceBuckets
|
||||
// maintains order
|
||||
|
|
|
@ -196,15 +196,14 @@ public abstract class InternalSignificantTerms<A extends InternalSignificantTerm
|
|||
bucket.aggregations));
|
||||
}
|
||||
}
|
||||
|
||||
SignificanceHeuristic heuristic = getSignificanceHeuristic().rewrite(reduceContext);
|
||||
final int size = Math.min(requiredSize, buckets.size());
|
||||
final int size = reduceContext.isFinalReduce() == false ? buckets.size() : Math.min(requiredSize, buckets.size());
|
||||
BucketSignificancePriorityQueue<B> ordered = new BucketSignificancePriorityQueue<>(size);
|
||||
for (Map.Entry<String, List<B>> entry : buckets.entrySet()) {
|
||||
List<B> sameTermBuckets = entry.getValue();
|
||||
final B b = sameTermBuckets.get(0).reduce(sameTermBuckets, reduceContext);
|
||||
b.updateScore(heuristic);
|
||||
if ((b.score > 0) && (b.subsetDf >= minDocCount)) {
|
||||
if (((b.score > 0) && (b.subsetDf >= minDocCount)) || reduceContext.isFinalReduce() == false) {
|
||||
ordered.insertWithOverflow(b);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -248,8 +249,8 @@ public abstract class InternalTerms<A extends InternalTerms<A, B>, B extends Int
|
|||
}
|
||||
}
|
||||
|
||||
final int size = Math.min(requiredSize, buckets.size());
|
||||
BucketPriorityQueue<B> ordered = new BucketPriorityQueue<>(size, order.comparator(null));
|
||||
final int size = reduceContext.isFinalReduce() == false ? buckets.size() : Math.min(requiredSize, buckets.size());
|
||||
final BucketPriorityQueue<B> ordered = new BucketPriorityQueue<>(size, order.comparator(null));
|
||||
for (List<B> sameTermBuckets : buckets.values()) {
|
||||
final B b = sameTermBuckets.get(0).reduce(sameTermBuckets, reduceContext);
|
||||
if (b.docCountError != -1) {
|
||||
|
@ -259,7 +260,7 @@ public abstract class InternalTerms<A extends InternalTerms<A, B>, B extends Int
|
|||
b.docCountError = sumDocCountError - b.docCountError;
|
||||
}
|
||||
}
|
||||
if (b.docCount >= minDocCount) {
|
||||
if (b.docCount >= minDocCount || reduceContext.isFinalReduce() == false) {
|
||||
B removed = ordered.insertWithOverflow(b);
|
||||
if (removed != null) {
|
||||
otherDocCount += removed.getDocCount();
|
||||
|
|
|
@ -39,10 +39,15 @@ import java.util.Map;
|
|||
|
||||
public class InternalScriptedMetric extends InternalMetricsAggregation implements ScriptedMetric {
|
||||
private final Script reduceScript;
|
||||
private final Object aggregation;
|
||||
private final List<Object> aggregation;
|
||||
|
||||
public InternalScriptedMetric(String name, Object aggregation, Script reduceScript, List<PipelineAggregator> pipelineAggregators,
|
||||
Map<String, Object> metaData) {
|
||||
this(name, Collections.singletonList(aggregation), reduceScript, pipelineAggregators, metaData);
|
||||
}
|
||||
|
||||
private InternalScriptedMetric(String name, List<Object> aggregation, Script reduceScript, List<PipelineAggregator> pipelineAggregators,
|
||||
Map<String, Object> metaData) {
|
||||
super(name, pipelineAggregators, metaData);
|
||||
this.aggregation = aggregation;
|
||||
this.reduceScript = reduceScript;
|
||||
|
@ -54,13 +59,13 @@ public class InternalScriptedMetric extends InternalMetricsAggregation implement
|
|||
public InternalScriptedMetric(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
reduceScript = in.readOptionalWriteable(Script::new);
|
||||
aggregation = in.readGenericValue();
|
||||
aggregation = Collections.singletonList(in.readGenericValue());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeOptionalWriteable(reduceScript);
|
||||
out.writeGenericValue(aggregation);
|
||||
out.writeGenericValue(aggregation());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -70,7 +75,10 @@ public class InternalScriptedMetric extends InternalMetricsAggregation implement
|
|||
|
||||
@Override
|
||||
public Object aggregation() {
|
||||
return aggregation;
|
||||
if (aggregation.size() != 1) {
|
||||
throw new IllegalStateException("aggregation was not reduced");
|
||||
}
|
||||
return aggregation.get(0);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -78,11 +86,11 @@ public class InternalScriptedMetric extends InternalMetricsAggregation implement
|
|||
List<Object> aggregationObjects = new ArrayList<>();
|
||||
for (InternalAggregation aggregation : aggregations) {
|
||||
InternalScriptedMetric mapReduceAggregation = (InternalScriptedMetric) aggregation;
|
||||
aggregationObjects.add(mapReduceAggregation.aggregation());
|
||||
aggregationObjects.addAll(mapReduceAggregation.aggregation);
|
||||
}
|
||||
InternalScriptedMetric firstAggregation = ((InternalScriptedMetric) aggregations.get(0));
|
||||
Object aggregation;
|
||||
if (firstAggregation.reduceScript != null) {
|
||||
List<Object> aggregation;
|
||||
if (firstAggregation.reduceScript != null && reduceContext.isFinalReduce()) {
|
||||
Map<String, Object> vars = new HashMap<>();
|
||||
vars.put("_aggs", aggregationObjects);
|
||||
if (firstAggregation.reduceScript.getParams() != null) {
|
||||
|
@ -91,13 +99,16 @@ public class InternalScriptedMetric extends InternalMetricsAggregation implement
|
|||
CompiledScript compiledScript = reduceContext.scriptService().compile(
|
||||
firstAggregation.reduceScript, ScriptContext.Standard.AGGS);
|
||||
ExecutableScript script = reduceContext.scriptService().executable(compiledScript, vars);
|
||||
aggregation = script.run();
|
||||
aggregation = Collections.singletonList(script.run());
|
||||
} else if (reduceContext.isFinalReduce()) {
|
||||
aggregation = Collections.singletonList(aggregationObjects);
|
||||
} else {
|
||||
// if we are not an final reduce we have to maintain all the aggs from all the incoming one
|
||||
// until we hit the final reduce phase.
|
||||
aggregation = aggregationObjects;
|
||||
}
|
||||
return new InternalScriptedMetric(firstAggregation.getName(), aggregation, firstAggregation.reduceScript, pipelineAggregators(),
|
||||
getMetaData());
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -105,7 +116,7 @@ public class InternalScriptedMetric extends InternalMetricsAggregation implement
|
|||
if (path.isEmpty()) {
|
||||
return this;
|
||||
} else if (path.size() == 1 && "value".equals(path.get(0))) {
|
||||
return aggregation;
|
||||
return aggregation();
|
||||
} else {
|
||||
throw new IllegalArgumentException("path not supported for [" + getName() + "]: " + path);
|
||||
}
|
||||
|
@ -113,7 +124,7 @@ public class InternalScriptedMetric extends InternalMetricsAggregation implement
|
|||
|
||||
@Override
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder.field("value", aggregation);
|
||||
return builder.field("value", aggregation());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -96,7 +96,18 @@ public class InternalTopHits extends InternalMetricsAggregation implements TopHi
|
|||
|
||||
@Override
|
||||
public InternalAggregation doReduce(List<InternalAggregation> aggregations, ReduceContext reduceContext) {
|
||||
SearchHits[] shardHits = new SearchHits[aggregations.size()];
|
||||
final SearchHits[] shardHits = new SearchHits[aggregations.size()];
|
||||
final int from;
|
||||
final int size;
|
||||
if (reduceContext.isFinalReduce()) {
|
||||
from = this.from;
|
||||
size = this.size;
|
||||
} else {
|
||||
// if we are not in the final reduce we need to ensure we maintain all possible elements during reduce
|
||||
// hence for pagination we need to maintain all hits until we are in the final phase.
|
||||
from = 0;
|
||||
size = this.from + this.size;
|
||||
}
|
||||
|
||||
final TopDocs reducedTopDocs;
|
||||
final TopDocs[] shardDocs;
|
||||
|
@ -130,7 +141,7 @@ public class InternalTopHits extends InternalMetricsAggregation implements TopHi
|
|||
} while (shardDocs[scoreDoc.shardIndex].scoreDocs[position] != scoreDoc);
|
||||
hits[i] = shardHits[scoreDoc.shardIndex].getAt(position);
|
||||
}
|
||||
return new InternalTopHits(name, from, size, reducedTopDocs, new SearchHits(hits, reducedTopDocs.totalHits,
|
||||
return new InternalTopHits(name, this.from, this.size, reducedTopDocs, new SearchHits(hits, reducedTopDocs.totalHits,
|
||||
reducedTopDocs.getMaxScore()),
|
||||
pipelineAggregators(), getMetaData());
|
||||
}
|
||||
|
|
|
@ -46,7 +46,7 @@ public class CountedCollectorTests extends ESTestCase {
|
|||
runnable.run();
|
||||
}
|
||||
};
|
||||
CountedCollector<SearchPhaseResult> collector = new CountedCollector<>(results, numResultsExpected,
|
||||
CountedCollector<SearchPhaseResult> collector = new CountedCollector<>(results::set, numResultsExpected,
|
||||
latch::countDown, context);
|
||||
for (int i = 0; i < numResultsExpected; i++) {
|
||||
int shardID = i;
|
||||
|
|
|
@ -95,7 +95,7 @@ public class DfsQueryPhaseTests extends ESTestCase {
|
|||
(response) -> new SearchPhase("test") {
|
||||
@Override
|
||||
public void run() throws IOException {
|
||||
responseRef.set(response);
|
||||
responseRef.set(response.results);
|
||||
}
|
||||
}, mockSearchPhaseContext);
|
||||
assertEquals("dfs_query", phase.getName());
|
||||
|
@ -147,7 +147,7 @@ public class DfsQueryPhaseTests extends ESTestCase {
|
|||
(response) -> new SearchPhase("test") {
|
||||
@Override
|
||||
public void run() throws IOException {
|
||||
responseRef.set(response);
|
||||
responseRef.set(response.results);
|
||||
}
|
||||
}, mockSearchPhaseContext);
|
||||
assertEquals("dfs_query", phase.getName());
|
||||
|
@ -202,7 +202,7 @@ public class DfsQueryPhaseTests extends ESTestCase {
|
|||
(response) -> new SearchPhase("test") {
|
||||
@Override
|
||||
public void run() throws IOException {
|
||||
responseRef.set(response);
|
||||
responseRef.set(response.results);
|
||||
}
|
||||
}, mockSearchPhaseContext);
|
||||
assertEquals("dfs_query", phase.getName());
|
||||
|
|
|
@ -46,7 +46,10 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
public class FetchSearchPhaseTests extends ESTestCase {
|
||||
|
||||
public void testShortcutQueryAndFetchOptimization() throws IOException {
|
||||
AtomicArray<QuerySearchResultProvider> results = new AtomicArray<>(1);
|
||||
SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null);
|
||||
MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(1);
|
||||
InitialSearchPhase.SearchPhaseResults<QuerySearchResultProvider> results =
|
||||
controller.newSearchPhaseResults(mockSearchPhaseContext.getRequest(), 1);
|
||||
AtomicReference<SearchResponse> responseRef = new AtomicReference<>();
|
||||
boolean hasHits = randomBoolean();
|
||||
final int numHits;
|
||||
|
@ -56,14 +59,12 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
queryResult.size(1);
|
||||
FetchSearchResult fetchResult = new FetchSearchResult();
|
||||
fetchResult.hits(new SearchHits(new SearchHit[] {new SearchHit(42)}, 1, 1.0F));
|
||||
results.set(0, new QueryFetchSearchResult(queryResult, fetchResult));
|
||||
results.consumeResult(0, new QueryFetchSearchResult(queryResult, fetchResult));
|
||||
numHits = 1;
|
||||
} else {
|
||||
numHits = 0;
|
||||
}
|
||||
|
||||
SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null);
|
||||
MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(1);
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, mockSearchPhaseContext,
|
||||
(searchResponse) -> new SearchPhase("test") {
|
||||
@Override
|
||||
|
@ -83,20 +84,22 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testFetchTwoDocument() throws IOException {
|
||||
AtomicArray<QuerySearchResultProvider> results = new AtomicArray<>(2);
|
||||
MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(2);
|
||||
SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null);
|
||||
InitialSearchPhase.SearchPhaseResults<QuerySearchResultProvider> results =
|
||||
controller.newSearchPhaseResults(mockSearchPhaseContext.getRequest(), 2);
|
||||
AtomicReference<SearchResponse> responseRef = new AtomicReference<>();
|
||||
int resultSetSize = randomIntBetween(2, 10);
|
||||
QuerySearchResult queryResult = new QuerySearchResult(123, new SearchShardTarget("node1", new Index("test", "na"), 0));
|
||||
queryResult.topDocs(new TopDocs(1, new ScoreDoc[] {new ScoreDoc(42, 1.0F)}, 2.0F), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize); // the size of the result set
|
||||
results.set(0, queryResult);
|
||||
results.consumeResult(0, queryResult);
|
||||
|
||||
queryResult = new QuerySearchResult(321, new SearchShardTarget("node2", new Index("test", "na"), 1));
|
||||
queryResult.topDocs(new TopDocs(1, new ScoreDoc[] {new ScoreDoc(84, 2.0F)}, 2.0F), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize);
|
||||
results.set(1, queryResult);
|
||||
results.consumeResult(1, queryResult);
|
||||
|
||||
SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null);
|
||||
SearchTransportService searchTransportService = new SearchTransportService(
|
||||
Settings.builder().put("search.remote.connect", false).build(), null, null) {
|
||||
@Override
|
||||
|
@ -112,7 +115,6 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
listener.onResponse(fetchResult);
|
||||
}
|
||||
};
|
||||
MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(2);
|
||||
mockSearchPhaseContext.searchTransport = searchTransportService;
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, mockSearchPhaseContext,
|
||||
(searchResponse) -> new SearchPhase("test") {
|
||||
|
@ -134,20 +136,22 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testFailFetchOneDoc() throws IOException {
|
||||
AtomicArray<QuerySearchResultProvider> results = new AtomicArray<>(2);
|
||||
MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(2);
|
||||
SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null);
|
||||
InitialSearchPhase.SearchPhaseResults<QuerySearchResultProvider> results =
|
||||
controller.newSearchPhaseResults(mockSearchPhaseContext.getRequest(), 2);
|
||||
AtomicReference<SearchResponse> responseRef = new AtomicReference<>();
|
||||
int resultSetSize = randomIntBetween(2, 10);
|
||||
QuerySearchResult queryResult = new QuerySearchResult(123, new SearchShardTarget("node1", new Index("test", "na"), 0));
|
||||
queryResult.topDocs(new TopDocs(1, new ScoreDoc[] {new ScoreDoc(42, 1.0F)}, 2.0F), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize); // the size of the result set
|
||||
results.set(0, queryResult);
|
||||
results.consumeResult(0, queryResult);
|
||||
|
||||
queryResult = new QuerySearchResult(321, new SearchShardTarget("node2", new Index("test", "na"), 1));
|
||||
queryResult.topDocs(new TopDocs(1, new ScoreDoc[] {new ScoreDoc(84, 2.0F)}, 2.0F), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize);
|
||||
results.set(1, queryResult);
|
||||
results.consumeResult(1, queryResult);
|
||||
|
||||
SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null);
|
||||
SearchTransportService searchTransportService = new SearchTransportService(
|
||||
Settings.builder().put("search.remote.connect", false).build(), null, null) {
|
||||
@Override
|
||||
|
@ -163,7 +167,6 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
|
||||
}
|
||||
};
|
||||
MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(2);
|
||||
mockSearchPhaseContext.searchTransport = searchTransportService;
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, mockSearchPhaseContext,
|
||||
(searchResponse) -> new SearchPhase("test") {
|
||||
|
@ -190,15 +193,17 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
int resultSetSize = randomIntBetween(0, 100);
|
||||
// we use at least 2 hits otherwise this is subject to single shard optimization and we trip an assert...
|
||||
int numHits = randomIntBetween(2, 100); // also numshards --> 1 hit per shard
|
||||
AtomicArray<QuerySearchResultProvider> results = new AtomicArray<>(numHits);
|
||||
SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null);
|
||||
MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(numHits);
|
||||
InitialSearchPhase.SearchPhaseResults<QuerySearchResultProvider> results =
|
||||
controller.newSearchPhaseResults(mockSearchPhaseContext.getRequest(), numHits);
|
||||
AtomicReference<SearchResponse> responseRef = new AtomicReference<>();
|
||||
for (int i = 0; i < numHits; i++) {
|
||||
QuerySearchResult queryResult = new QuerySearchResult(i, new SearchShardTarget("node1", new Index("test", "na"), 0));
|
||||
queryResult.topDocs(new TopDocs(1, new ScoreDoc[] {new ScoreDoc(i+1, i)}, i), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize); // the size of the result set
|
||||
results.set(i, queryResult);
|
||||
results.consumeResult(i, queryResult);
|
||||
}
|
||||
SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null);
|
||||
SearchTransportService searchTransportService = new SearchTransportService(
|
||||
Settings.builder().put("search.remote.connect", false).build(), null, null) {
|
||||
@Override
|
||||
|
@ -211,7 +216,6 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
}).start();
|
||||
}
|
||||
};
|
||||
MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(numHits);
|
||||
mockSearchPhaseContext.searchTransport = searchTransportService;
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, mockSearchPhaseContext,
|
||||
|
@ -243,20 +247,22 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testExceptionFailsPhase() throws IOException {
|
||||
AtomicArray<QuerySearchResultProvider> results = new AtomicArray<>(2);
|
||||
MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(2);
|
||||
SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null);
|
||||
InitialSearchPhase.SearchPhaseResults<QuerySearchResultProvider> results =
|
||||
controller.newSearchPhaseResults(mockSearchPhaseContext.getRequest(), 2);
|
||||
AtomicReference<SearchResponse> responseRef = new AtomicReference<>();
|
||||
int resultSetSize = randomIntBetween(2, 10);
|
||||
QuerySearchResult queryResult = new QuerySearchResult(123, new SearchShardTarget("node1", new Index("test", "na"), 0));
|
||||
queryResult.topDocs(new TopDocs(1, new ScoreDoc[] {new ScoreDoc(42, 1.0F)}, 2.0F), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize); // the size of the result set
|
||||
results.set(0, queryResult);
|
||||
results.consumeResult(0, queryResult);
|
||||
|
||||
queryResult = new QuerySearchResult(321, new SearchShardTarget("node2", new Index("test", "na"), 1));
|
||||
queryResult.topDocs(new TopDocs(1, new ScoreDoc[] {new ScoreDoc(84, 2.0F)}, 2.0F), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize);
|
||||
results.set(1, queryResult);
|
||||
results.consumeResult(1, queryResult);
|
||||
AtomicInteger numFetches = new AtomicInteger(0);
|
||||
SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null);
|
||||
SearchTransportService searchTransportService = new SearchTransportService(
|
||||
Settings.builder().put("search.remote.connect", false).build(), null, null) {
|
||||
@Override
|
||||
|
@ -275,7 +281,6 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
listener.onResponse(fetchResult);
|
||||
}
|
||||
};
|
||||
MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(2);
|
||||
mockSearchPhaseContext.searchTransport = searchTransportService;
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, mockSearchPhaseContext,
|
||||
(searchResponse) -> new SearchPhase("test") {
|
||||
|
@ -293,20 +298,22 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testCleanupIrrelevantContexts() throws IOException { // contexts that are not fetched should be cleaned up
|
||||
AtomicArray<QuerySearchResultProvider> results = new AtomicArray<>(2);
|
||||
MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(2);
|
||||
SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null);
|
||||
InitialSearchPhase.SearchPhaseResults<QuerySearchResultProvider> results =
|
||||
controller.newSearchPhaseResults(mockSearchPhaseContext.getRequest(), 2);
|
||||
AtomicReference<SearchResponse> responseRef = new AtomicReference<>();
|
||||
int resultSetSize = 1;
|
||||
QuerySearchResult queryResult = new QuerySearchResult(123, new SearchShardTarget("node1", new Index("test", "na"), 0));
|
||||
queryResult.topDocs(new TopDocs(1, new ScoreDoc[] {new ScoreDoc(42, 1.0F)}, 2.0F), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize); // the size of the result set
|
||||
results.set(0, queryResult);
|
||||
results.consumeResult(0, queryResult);
|
||||
|
||||
queryResult = new QuerySearchResult(321, new SearchShardTarget("node2", new Index("test", "na"), 1));
|
||||
queryResult.topDocs(new TopDocs(1, new ScoreDoc[] {new ScoreDoc(84, 2.0F)}, 2.0F), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize);
|
||||
results.set(1, queryResult);
|
||||
results.consumeResult(1, queryResult);
|
||||
|
||||
SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null);
|
||||
SearchTransportService searchTransportService = new SearchTransportService(
|
||||
Settings.builder().put("search.remote.connect", false).build(), null, null) {
|
||||
@Override
|
||||
|
@ -321,7 +328,6 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
listener.onResponse(fetchResult);
|
||||
}
|
||||
};
|
||||
MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(2);
|
||||
mockSearchPhaseContext.searchTransport = searchTransportService;
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, mockSearchPhaseContext,
|
||||
(searchResponse) -> new SearchPhase("test") {
|
||||
|
|
|
@ -32,13 +32,11 @@ import org.elasticsearch.common.io.stream.StreamInput;
|
|||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.internal.AliasFilter;
|
||||
import org.elasticsearch.search.internal.ShardSearchTransportRequest;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportException;
|
||||
|
@ -53,7 +51,6 @@ import java.util.HashSet;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
|
@ -96,7 +93,8 @@ public class SearchAsyncActionTests extends ESTestCase {
|
|||
lookup.put(replicaNode.getId(), new MockConnection(replicaNode));
|
||||
Map<String, AliasFilter> aliasFilters = Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY));
|
||||
AbstractSearchAsyncAction asyncAction = new AbstractSearchAsyncAction<TestSearchPhaseResult>("test", logger, transportService,
|
||||
lookup::get, aliasFilters, Collections.emptyMap(), null, request, responseListener, shardsIter, 0, 0, null) {
|
||||
lookup::get, aliasFilters, Collections.emptyMap(), null, request, responseListener, shardsIter, 0, 0, null,
|
||||
new InitialSearchPhase.SearchPhaseResults<>(shardsIter.size())) {
|
||||
TestSearchResponse response = new TestSearchResponse();
|
||||
|
||||
@Override
|
||||
|
@ -115,12 +113,12 @@ public class SearchAsyncActionTests extends ESTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected SearchPhase getNextPhase(AtomicArray<TestSearchPhaseResult> results, SearchPhaseContext context) {
|
||||
protected SearchPhase getNextPhase(SearchPhaseResults<TestSearchPhaseResult> results, SearchPhaseContext context) {
|
||||
return new SearchPhase("test") {
|
||||
@Override
|
||||
public void run() throws IOException {
|
||||
for (int i = 0; i < results.length(); i++) {
|
||||
TestSearchPhaseResult result = results.get(i);
|
||||
for (int i = 0; i < results.getNumShards(); i++) {
|
||||
TestSearchPhaseResult result = results.results.get(i);
|
||||
assertEquals(result.node.getId(), result.shardTarget().getNodeId());
|
||||
sendReleaseSearchContext(result.id(), new MockConnection(result.node));
|
||||
}
|
||||
|
|
|
@ -26,7 +26,14 @@ import org.elasticsearch.common.text.Text;
|
|||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.search.DocValueFormat;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.aggregations.AggregationBuilders;
|
||||
import org.elasticsearch.search.aggregations.Aggregations;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.aggregations.metrics.max.InternalMax;
|
||||
import org.elasticsearch.search.aggregations.metrics.sum.InternalSum;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.fetch.FetchSearchResult;
|
||||
import org.elasticsearch.search.SearchHit;
|
||||
import org.elasticsearch.search.SearchHits;
|
||||
|
@ -45,10 +52,13 @@ import java.util.Collections;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.instanceOf;
|
||||
import static org.hamcrest.Matchers.not;
|
||||
|
||||
public class SearchPhaseControllerTests extends ESTestCase {
|
||||
private SearchPhaseController searchPhaseController;
|
||||
|
@ -230,4 +240,98 @@ public class SearchPhaseControllerTests extends ESTestCase {
|
|||
}
|
||||
return fetchResults;
|
||||
}
|
||||
|
||||
public void testConsumer() {
|
||||
int bufferSize = randomIntBetween(2, 3);
|
||||
SearchRequest request = new SearchRequest();
|
||||
request.source(new SearchSourceBuilder().aggregation(AggregationBuilders.avg("foo")));
|
||||
request.setReduceUpTo(bufferSize);
|
||||
InitialSearchPhase.SearchPhaseResults<QuerySearchResultProvider> consumer = searchPhaseController.newSearchPhaseResults(request, 3);
|
||||
QuerySearchResult result = new QuerySearchResult(0, new SearchShardTarget("node", new Index("a", "b"), 0));
|
||||
result.topDocs(new TopDocs(0, new ScoreDoc[0], 0.0F), new DocValueFormat[0]);
|
||||
InternalAggregations aggs = new InternalAggregations(Arrays.asList(new InternalMax("test", 1.0D, DocValueFormat.RAW,
|
||||
Collections.emptyList(), Collections.emptyMap())));
|
||||
result.aggregations(aggs);
|
||||
consumer.consumeResult(0, result);
|
||||
|
||||
result = new QuerySearchResult(1, new SearchShardTarget("node", new Index("a", "b"), 0));
|
||||
result.topDocs(new TopDocs(0, new ScoreDoc[0], 0.0F), new DocValueFormat[0]);
|
||||
aggs = new InternalAggregations(Arrays.asList(new InternalMax("test", 3.0D, DocValueFormat.RAW,
|
||||
Collections.emptyList(), Collections.emptyMap())));
|
||||
result.aggregations(aggs);
|
||||
consumer.consumeResult(2, result);
|
||||
|
||||
result = new QuerySearchResult(1, new SearchShardTarget("node", new Index("a", "b"), 0));
|
||||
result.topDocs(new TopDocs(0, new ScoreDoc[0], 0.0F), new DocValueFormat[0]);
|
||||
aggs = new InternalAggregations(Arrays.asList(new InternalMax("test", 2.0D, DocValueFormat.RAW,
|
||||
Collections.emptyList(), Collections.emptyMap())));
|
||||
result.aggregations(aggs);
|
||||
consumer.consumeResult(1, result);
|
||||
|
||||
if (bufferSize == 2) {
|
||||
assertThat(consumer, instanceOf(SearchPhaseController.QueryPhaseResultConsumer.class));
|
||||
assertEquals(2, ((SearchPhaseController.QueryPhaseResultConsumer)consumer).getNumBuffered());
|
||||
} else {
|
||||
assertThat(consumer, not(instanceOf(SearchPhaseController.QueryPhaseResultConsumer.class)));
|
||||
}
|
||||
|
||||
SearchPhaseController.ReducedQueryPhase reduce = consumer.reduce();
|
||||
InternalMax max = (InternalMax) reduce.aggregations.asList().get(0);
|
||||
assertEquals(3.0D, max.getValue(), 0.0D);
|
||||
}
|
||||
|
||||
public void testConsumerConcurrently() throws InterruptedException {
|
||||
int expectedNumResults = randomIntBetween(1, 100);
|
||||
int bufferSize = randomIntBetween(2, 200);
|
||||
|
||||
SearchRequest request = new SearchRequest();
|
||||
request.source(new SearchSourceBuilder().aggregation(AggregationBuilders.avg("foo")));
|
||||
request.setReduceUpTo(bufferSize);
|
||||
InitialSearchPhase.SearchPhaseResults<QuerySearchResultProvider> consumer =
|
||||
searchPhaseController.newSearchPhaseResults(request, expectedNumResults);
|
||||
AtomicInteger max = new AtomicInteger();
|
||||
CountDownLatch latch = new CountDownLatch(expectedNumResults);
|
||||
for (int i = 0; i < expectedNumResults; i++) {
|
||||
int id = i;
|
||||
Thread t = new Thread(() -> {
|
||||
int number = randomIntBetween(1, 1000);
|
||||
max.updateAndGet(prev -> Math.max(prev, number));
|
||||
QuerySearchResult result = new QuerySearchResult(id, new SearchShardTarget("node", new Index("a", "b"), id));
|
||||
result.topDocs(new TopDocs(id, new ScoreDoc[0], 0.0F), new DocValueFormat[0]);
|
||||
InternalAggregations aggs = new InternalAggregations(Arrays.asList(new InternalMax("test", (double) number,
|
||||
DocValueFormat.RAW, Collections.emptyList(), Collections.emptyMap())));
|
||||
result.aggregations(aggs);
|
||||
consumer.consumeResult(id, result);
|
||||
latch.countDown();
|
||||
|
||||
});
|
||||
t.start();
|
||||
}
|
||||
latch.await();
|
||||
SearchPhaseController.ReducedQueryPhase reduce = consumer.reduce();
|
||||
InternalMax internalMax = (InternalMax) reduce.aggregations.asList().get(0);
|
||||
assertEquals(max.get(), internalMax.getValue(), 0.0D);
|
||||
}
|
||||
|
||||
public void testNewSearchPhaseResults() {
|
||||
for (int i = 0; i < 10; i++) {
|
||||
int expectedNumResults = randomIntBetween(1, 10);
|
||||
int bufferSize = randomIntBetween(2, 10);
|
||||
SearchRequest request = new SearchRequest();
|
||||
final boolean hasAggs;
|
||||
if ((hasAggs = randomBoolean())) {
|
||||
request.source(new SearchSourceBuilder().aggregation(AggregationBuilders.avg("foo")));
|
||||
}
|
||||
request.setReduceUpTo(bufferSize);
|
||||
InitialSearchPhase.SearchPhaseResults<QuerySearchResultProvider> consumer
|
||||
= searchPhaseController.newSearchPhaseResults(request, expectedNumResults);
|
||||
if (hasAggs && expectedNumResults > bufferSize) {
|
||||
assertThat("expectedNumResults: " + expectedNumResults + " bufferSize: " + bufferSize,
|
||||
consumer, instanceOf(SearchPhaseController.QueryPhaseResultConsumer.class));
|
||||
} else {
|
||||
assertThat("expectedNumResults: " + expectedNumResults + " bufferSize: " + bufferSize,
|
||||
consumer, not(instanceOf(SearchPhaseController.QueryPhaseResultConsumer.class)));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -180,8 +180,18 @@ public abstract class AggregatorTestCase extends ESTestCase {
|
|||
if (aggs.isEmpty()) {
|
||||
return null;
|
||||
} else {
|
||||
if (aggs.size() > 2 && randomBoolean()) {
|
||||
// sometimes do an incremental reduce
|
||||
List<InternalAggregation> internalAggregations = randomSubsetOf(randomIntBetween(2, aggs.size()-1), aggs);
|
||||
A internalAgg = (A) aggs.get(0).doReduce(internalAggregations,
|
||||
new InternalAggregation.ReduceContext(root.context().bigArrays(), null, false));
|
||||
aggs.removeAll(internalAggregations);
|
||||
aggs.add(internalAgg);
|
||||
}
|
||||
// now do the final reduce
|
||||
@SuppressWarnings("unchecked")
|
||||
A internalAgg = (A) aggs.get(0).doReduce(aggs, new InternalAggregation.ReduceContext(root.context().bigArrays(), null));
|
||||
A internalAgg = (A) aggs.get(0).doReduce(aggs, new InternalAggregation.ReduceContext(root.context().bigArrays(), null,
|
||||
true));
|
||||
return internalAgg;
|
||||
}
|
||||
} finally {
|
||||
|
|
|
@ -57,8 +57,17 @@ public abstract class InternalAggregationTestCase<T extends InternalAggregation>
|
|||
inputs.add(t);
|
||||
toReduce.add(t);
|
||||
}
|
||||
if (randomBoolean() && toReduceSize >= 2) {
|
||||
List<InternalAggregation> internalAggregations = randomSubsetOf(randomIntBetween(2, toReduceSize - 2), toReduce);
|
||||
InternalAggregation.ReduceContext context = new InternalAggregation.ReduceContext(null, null, true);
|
||||
@SuppressWarnings("unchecked")
|
||||
T reduced = (T) inputs.get(0).reduce(toReduce, null);
|
||||
T reduced = (T) inputs.get(0).reduce(internalAggregations, context);
|
||||
toReduce.removeAll(internalAggregations);
|
||||
toReduce.add(reduced);
|
||||
}
|
||||
InternalAggregation.ReduceContext context = new InternalAggregation.ReduceContext(null, null, true);
|
||||
@SuppressWarnings("unchecked")
|
||||
T reduced = (T) inputs.get(0).reduce(toReduce, context);
|
||||
assertReduced(reduced, inputs);
|
||||
}
|
||||
|
||||
|
|
|
@ -170,9 +170,9 @@ public class MinDocCountIT extends AbstractTermsTestCase {
|
|||
if (size2++ == size) {
|
||||
break;
|
||||
}
|
||||
assertTrue(it2.hasNext());
|
||||
assertTrue("minDocCount: " + minDocCount, it2.hasNext());
|
||||
final Terms.Bucket bucket2 = it2.next();
|
||||
assertEquals(bucket1.getDocCount(), bucket2.getDocCount());
|
||||
assertEquals("minDocCount: " + minDocCount, bucket1.getDocCount(), bucket2.getDocCount());
|
||||
}
|
||||
}
|
||||
assertFalse(it2.hasNext());
|
||||
|
@ -336,24 +336,8 @@ public class MinDocCountIT extends AbstractTermsTestCase {
|
|||
.shardSize(cardinality + randomInt(10))
|
||||
.minDocCount(minDocCount)).request();
|
||||
final SearchResponse response = client().search(request).get();
|
||||
try {
|
||||
assertAllSuccessful(response);
|
||||
assertSubset(allTerms, (Terms) response.getAggregations().get("terms"), minDocCount, size, include);
|
||||
} catch (AssertionError ae) {
|
||||
if (!retry) {
|
||||
throw ae;
|
||||
}
|
||||
logger.info("test failed. trying to see if it recovers after 1m.", ae);
|
||||
try {
|
||||
Thread.sleep(60000);
|
||||
logger.debug("1m passed. retrying.");
|
||||
testMinDocCountOnTerms(field, script, order, include, false);
|
||||
} catch (Exception secondFailure) {
|
||||
secondFailure.addSuppressed(ae);
|
||||
logger.error("exception on retry (will re-throw the original in a sec)", secondFailure);
|
||||
}
|
||||
throw ae;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -20,7 +20,10 @@
|
|||
package org.elasticsearch.search.aggregations.bucket;
|
||||
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.client.FilterClient;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
|
@ -30,10 +33,12 @@ import org.elasticsearch.search.aggregations.bucket.terms.Terms.Bucket;
|
|||
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorFactory.ExecutionMode;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.elasticsearch.test.client.RandomizingClient;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.function.Function;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.sum;
|
||||
|
@ -53,12 +58,43 @@ public class TermsDocCountErrorIT extends ESIntegTestCase {
|
|||
private static final String LONG_FIELD_NAME = "l_value";
|
||||
private static final String DOUBLE_FIELD_NAME = "d_value";
|
||||
|
||||
|
||||
public static String randomExecutionHint() {
|
||||
return randomBoolean() ? null : randomFrom(ExecutionMode.values()).toString();
|
||||
}
|
||||
|
||||
private static int numRoutingValues;
|
||||
|
||||
public static Client client() {
|
||||
Client client = ESIntegTestCase.client();
|
||||
if (client instanceof RandomizingClient) {
|
||||
return new FilterClient(client) {
|
||||
/* this test doesn't work with multiple reduce phases since:
|
||||
* the error for a term is the sum of the errors across all aggs that need to be reduced.
|
||||
* if the term is in the aggregation, then we just use the associated error, but if it is not we need to use the
|
||||
* aggregation-level error, ie. the maximum count that a doc that is not in the top list could have.
|
||||
*
|
||||
* the problem is that the logic we have today assumes there is a single reduce. So for instance for the agg-level error
|
||||
* it takes the count of the last term. This is correct if the agg was produced on a shard: if it had a greater count
|
||||
* then it would be in the top list. However if we are on an intermediate reduce, this does not work anymore.
|
||||
*
|
||||
* Another assumption that does not hold is that right now if a term is present in an agg, we assume its count is accurate.
|
||||
* Again this is true if the agg was produced on a shard, but not if this is the result of an intermediate reduce.
|
||||
*
|
||||
* try with this seed and remove the setReduceUpTo below
|
||||
* -Dtests.seed=B32081B1E8589AE5 -Dtests.class=org.elasticsearch.search.aggregations.bucket.TermsDocCountErrorIT
|
||||
* -Dtests.method="testDoubleValueField" -Dtests.locale=lv -Dtests.timezone=WET
|
||||
* This must will be addressed in a followup to #23253
|
||||
*/
|
||||
@Override
|
||||
public SearchRequestBuilder prepareSearch(String... indices) {
|
||||
return this.in.prepareSearch(indices).setReduceUpTo(512);
|
||||
}
|
||||
};
|
||||
}
|
||||
return client;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setupSuiteScopeCluster() throws Exception {
|
||||
assertAcked(client().admin().indices().prepareCreate("idx")
|
||||
|
|
|
@ -146,7 +146,8 @@ public class SignificanceHeuristicTests extends ESTestCase {
|
|||
|
||||
public void testReduce() {
|
||||
List<InternalAggregation> aggs = createInternalAggregations();
|
||||
SignificantTerms reducedAgg = (SignificantTerms) aggs.get(0).doReduce(aggs, null);
|
||||
InternalAggregation.ReduceContext context = new InternalAggregation.ReduceContext(null, null, true);
|
||||
SignificantTerms reducedAgg = (SignificantTerms) aggs.get(0).doReduce(aggs, context);
|
||||
assertThat(reducedAgg.getBuckets().size(), equalTo(2));
|
||||
assertThat(reducedAgg.getBuckets().get(0).getSubsetDf(), equalTo(8L));
|
||||
assertThat(reducedAgg.getBuckets().get(0).getSubsetSize(), equalTo(16L));
|
||||
|
|
|
@ -131,7 +131,7 @@ public class TermsAggregatorTests extends AggregatorTestCase {
|
|||
}
|
||||
InternalAggregation.ReduceContext ctx =
|
||||
new InternalAggregation.ReduceContext(new MockBigArrays(Settings.EMPTY,
|
||||
new NoneCircuitBreakerService()), null);
|
||||
new NoneCircuitBreakerService()), null, true);
|
||||
for (InternalAggregation internalAgg : aggs) {
|
||||
InternalAggregation mergedAggs = internalAgg.doReduce(aggs, ctx);
|
||||
assertTrue(mergedAggs instanceof DoubleTerms);
|
||||
|
|
|
@ -421,7 +421,7 @@ public class TopHitsIT extends ESIntegTestCase {
|
|||
assertThat(hits.getHits().length, equalTo(3));
|
||||
|
||||
assertThat(hits.getAt(0).getSourceAsMap().size(), equalTo(4));
|
||||
id --;
|
||||
id--;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -452,7 +452,8 @@ public class TopHitsIT extends ESIntegTestCase {
|
|||
.executionHint(randomExecutionHint())
|
||||
.field(TERMS_AGGS_FIELD)
|
||||
.subAggregation(
|
||||
topHits("hits").sort(SortBuilders.fieldSort(SORT_FIELD).order(SortOrder.DESC))
|
||||
topHits("hits")
|
||||
.sort(SortBuilders.fieldSort(SORT_FIELD).order(SortOrder.DESC))
|
||||
.from(from)
|
||||
.size(size)
|
||||
)
|
||||
|
@ -483,7 +484,8 @@ public class TopHitsIT extends ESIntegTestCase {
|
|||
assertThat(hits.getTotalHits(), equalTo(controlHits.getTotalHits()));
|
||||
assertThat(hits.getHits().length, equalTo(controlHits.getHits().length));
|
||||
for (int i = 0; i < hits.getHits().length; i++) {
|
||||
logger.info("{}: top_hits: [{}][{}] control: [{}][{}]", i, hits.getAt(i).getId(), hits.getAt(i).getSortValues()[0], controlHits.getAt(i).getId(), controlHits.getAt(i).getSortValues()[0]);
|
||||
logger.info("{}: top_hits: [{}][{}] control: [{}][{}]", i, hits.getAt(i).getId(), hits.getAt(i).getSortValues()[0],
|
||||
controlHits.getAt(i).getId(), controlHits.getAt(i).getSortValues()[0]);
|
||||
assertThat(hits.getAt(i).getId(), equalTo(controlHits.getAt(i).getId()));
|
||||
assertThat(hits.getAt(i).getSortValues()[0], equalTo(controlHits.getAt(i).getSortValues()[0]));
|
||||
}
|
||||
|
@ -1000,6 +1002,7 @@ public class TopHitsIT extends ESIntegTestCase {
|
|||
* not using a script does get cached.
|
||||
*/
|
||||
public void testDontCacheScripts() throws Exception {
|
||||
try {
|
||||
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=long")
|
||||
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
|
||||
.get());
|
||||
|
@ -1045,6 +1048,9 @@ public class TopHitsIT extends ESIntegTestCase {
|
|||
.getHitCount(), equalTo(0L));
|
||||
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
|
||||
.getMissCount(), equalTo(1L));
|
||||
} finally {
|
||||
assertAcked(client().admin().indices().prepareDelete("cache_test_idx")); // delete this - if we use tests.iters it would fail
|
||||
}
|
||||
}
|
||||
|
||||
public void testWithRescore() {
|
||||
|
|
|
@ -50,8 +50,7 @@ public class InternalAvgTests extends InternalAggregationTestCase<InternalAvg> {
|
|||
counts += in.getCount();
|
||||
}
|
||||
assertEquals(counts, reduced.getCount());
|
||||
assertEquals(sum, reduced.getSum(), Double.MIN_VALUE);
|
||||
assertEquals(sum / counts, reduced.value(), Double.MIN_VALUE);
|
||||
assertEquals(sum, reduced.getSum(), 0.00000001);
|
||||
assertEquals(sum / counts, reduced.value(), 0.00000001);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -36,6 +36,7 @@ public class RandomizingClient extends FilterClient {
|
|||
|
||||
private final SearchType defaultSearchType;
|
||||
private final String defaultPreference;
|
||||
private final int reduceUpTo;
|
||||
|
||||
|
||||
public RandomizingClient(Client client, Random random) {
|
||||
|
@ -53,12 +54,13 @@ public class RandomizingClient extends FilterClient {
|
|||
} else {
|
||||
defaultPreference = null;
|
||||
}
|
||||
this.reduceUpTo = 2 + random.nextInt(10);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public SearchRequestBuilder prepareSearch(String... indices) {
|
||||
return in.prepareSearch(indices).setSearchType(defaultSearchType).setPreference(defaultPreference);
|
||||
return in.prepareSearch(indices).setSearchType(defaultSearchType).setPreference(defaultPreference).setReduceUpTo(reduceUpTo);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
Loading…
Reference in New Issue