|
@@ -15,9 +15,12 @@ import org.apache.lucene.search.TopDocs;
|
|
|
import org.elasticsearch.action.search.SearchPhaseController.TopDocsStats;
|
|
|
import org.elasticsearch.common.breaker.CircuitBreaker;
|
|
|
import org.elasticsearch.common.breaker.CircuitBreakingException;
|
|
|
+import org.elasticsearch.common.collect.Iterators;
|
|
|
import org.elasticsearch.common.io.stream.DelayableWriteable;
|
|
|
import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore;
|
|
|
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
|
|
+import org.elasticsearch.core.Releasable;
|
|
|
+import org.elasticsearch.core.Releasables;
|
|
|
import org.elasticsearch.search.SearchPhaseResult;
|
|
|
import org.elasticsearch.search.SearchService;
|
|
|
import org.elasticsearch.search.SearchShardTarget;
|
|
@@ -31,6 +34,7 @@ import java.util.ArrayDeque;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Collections;
|
|
|
import java.util.Comparator;
|
|
|
+import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
import java.util.concurrent.Executor;
|
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
@@ -174,14 +178,10 @@ public class QueryPhaseResultConsumer extends ArraySearchPhaseResults<SearchPhas
|
|
|
this.mergeResult = null;
|
|
|
final int resultSize = buffer.size() + (mergeResult == null ? 0 : 1);
|
|
|
final List<TopDocs> topDocsList = hasTopDocs ? new ArrayList<>(resultSize) : null;
|
|
|
- final List<DelayableWriteable<InternalAggregations>> aggsList = hasAggs ? new ArrayList<>(resultSize) : null;
|
|
|
if (mergeResult != null) {
|
|
|
if (topDocsList != null) {
|
|
|
topDocsList.add(mergeResult.reducedTopDocs);
|
|
|
}
|
|
|
- if (aggsList != null) {
|
|
|
- aggsList.add(DelayableWriteable.referencing(mergeResult.reducedAggs));
|
|
|
- }
|
|
|
}
|
|
|
for (QuerySearchResult result : buffer) {
|
|
|
topDocsStats.add(result.topDocs(), result.searchTimedOut(), result.terminatedEarly());
|
|
@@ -190,34 +190,39 @@ public class QueryPhaseResultConsumer extends ArraySearchPhaseResults<SearchPhas
|
|
|
setShardIndex(topDocs.topDocs, result.getShardIndex());
|
|
|
topDocsList.add(topDocs.topDocs);
|
|
|
}
|
|
|
- if (aggsList != null) {
|
|
|
- aggsList.add(result.getAggs());
|
|
|
- }
|
|
|
}
|
|
|
SearchPhaseController.ReducedQueryPhase reducePhase;
|
|
|
long breakerSize = circuitBreakerBytes;
|
|
|
+ final InternalAggregations aggs;
|
|
|
try {
|
|
|
- if (aggsList != null) {
|
|
|
+ if (hasAggs) {
|
|
|
// Add an estimate of the final reduce size
|
|
|
breakerSize = addEstimateAndMaybeBreak(estimateRamBytesUsedForReduce(breakerSize));
|
|
|
+ aggs = aggregate(
|
|
|
+ buffer.iterator(),
|
|
|
+ mergeResult,
|
|
|
+ resultSize,
|
|
|
+ performFinalReduce ? aggReduceContextBuilder.forFinalReduction() : aggReduceContextBuilder.forPartialReduction()
|
|
|
+ );
|
|
|
+ } else {
|
|
|
+ aggs = null;
|
|
|
}
|
|
|
reducePhase = SearchPhaseController.reducedQueryPhase(
|
|
|
results.asList(),
|
|
|
- aggsList,
|
|
|
+ aggs,
|
|
|
topDocsList == null ? Collections.emptyList() : topDocsList,
|
|
|
topDocsStats,
|
|
|
numReducePhases,
|
|
|
false,
|
|
|
- aggReduceContextBuilder,
|
|
|
- queryPhaseRankCoordinatorContext,
|
|
|
- performFinalReduce
|
|
|
+ queryPhaseRankCoordinatorContext
|
|
|
);
|
|
|
+ buffer = null;
|
|
|
} finally {
|
|
|
releaseAggs(buffer);
|
|
|
}
|
|
|
if (hasAggs
|
|
|
// reduced aggregations can be null if all shards failed
|
|
|
- && reducePhase.aggregations() != null) {
|
|
|
+ && aggs != null) {
|
|
|
|
|
|
// Update the circuit breaker to replace the estimation with the serialized size of the newly reduced result
|
|
|
long finalSize = DelayableWriteable.getSerializedSize(reducePhase.aggregations()) - breakerSize;
|
|
@@ -249,17 +254,7 @@ public class QueryPhaseResultConsumer extends ArraySearchPhaseResults<SearchPhas
|
|
|
toConsume.sort(RESULT_COMPARATOR);
|
|
|
|
|
|
final TopDocs newTopDocs;
|
|
|
- final InternalAggregations newAggs;
|
|
|
- final List<DelayableWriteable<InternalAggregations>> aggsList;
|
|
|
final int resultSetSize = toConsume.size() + (lastMerge != null ? 1 : 0);
|
|
|
- if (hasAggs) {
|
|
|
- aggsList = new ArrayList<>(resultSetSize);
|
|
|
- if (lastMerge != null) {
|
|
|
- aggsList.add(DelayableWriteable.referencing(lastMerge.reducedAggs));
|
|
|
- }
|
|
|
- } else {
|
|
|
- aggsList = null;
|
|
|
- }
|
|
|
List<TopDocs> topDocsList;
|
|
|
if (hasTopDocs) {
|
|
|
topDocsList = new ArrayList<>(resultSetSize);
|
|
@@ -269,14 +264,12 @@ public class QueryPhaseResultConsumer extends ArraySearchPhaseResults<SearchPhas
|
|
|
} else {
|
|
|
topDocsList = null;
|
|
|
}
|
|
|
+ final InternalAggregations newAggs;
|
|
|
try {
|
|
|
for (QuerySearchResult result : toConsume) {
|
|
|
topDocsStats.add(result.topDocs(), result.searchTimedOut(), result.terminatedEarly());
|
|
|
SearchShardTarget target = result.getSearchShardTarget();
|
|
|
processedShards.add(new SearchShard(target.getClusterAlias(), target.getShardId()));
|
|
|
- if (aggsList != null) {
|
|
|
- aggsList.add(result.getAggs());
|
|
|
- }
|
|
|
if (topDocsList != null) {
|
|
|
TopDocsAndMaxScore topDocs = result.consumeTopDocs();
|
|
|
setShardIndex(topDocs.topDocs, result.getShardIndex());
|
|
@@ -285,9 +278,10 @@ public class QueryPhaseResultConsumer extends ArraySearchPhaseResults<SearchPhas
|
|
|
}
|
|
|
// we have to merge here in the same way we collect on a shard
|
|
|
newTopDocs = topDocsList == null ? null : mergeTopDocs(topDocsList, topNSize, 0);
|
|
|
- newAggs = aggsList == null
|
|
|
- ? null
|
|
|
- : InternalAggregations.topLevelReduceDelayable(aggsList, aggReduceContextBuilder.forPartialReduction());
|
|
|
+ newAggs = hasAggs
|
|
|
+ ? aggregate(toConsume.iterator(), lastMerge, resultSetSize, aggReduceContextBuilder.forPartialReduction())
|
|
|
+ : null;
|
|
|
+ toConsume = null;
|
|
|
} finally {
|
|
|
releaseAggs(toConsume);
|
|
|
}
|
|
@@ -302,6 +296,45 @@ public class QueryPhaseResultConsumer extends ArraySearchPhaseResults<SearchPhas
|
|
|
return new MergeResult(processedShards, newTopDocs, newAggs, newAggs != null ? DelayableWriteable.getSerializedSize(newAggs) : 0);
|
|
|
}
|
|
|
|
|
|
+ private static InternalAggregations aggregate(
|
|
|
+ Iterator<QuerySearchResult> toConsume,
|
|
|
+ MergeResult lastMerge,
|
|
|
+ int resultSetSize,
|
|
|
+ AggregationReduceContext reduceContext
|
|
|
+ ) {
|
|
|
+ interface ReleasableIterator extends Iterator<InternalAggregations>, Releasable {}
|
|
|
+ try (var aggsIter = new ReleasableIterator() {
|
|
|
+
|
|
|
+ private Releasable toRelease;
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void close() {
|
|
|
+ Releasables.close(toRelease);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public boolean hasNext() {
|
|
|
+ return toConsume.hasNext();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public InternalAggregations next() {
|
|
|
+ var res = toConsume.next().consumeAggs();
|
|
|
+ Releasables.close(toRelease);
|
|
|
+ toRelease = res;
|
|
|
+ return res.expand();
|
|
|
+ }
|
|
|
+ }) {
|
|
|
+ return InternalAggregations.topLevelReduce(
|
|
|
+ lastMerge == null ? aggsIter : Iterators.concat(Iterators.single(lastMerge.reducedAggs), aggsIter),
|
|
|
+ resultSetSize,
|
|
|
+ reduceContext
|
|
|
+ );
|
|
|
+ } finally {
|
|
|
+ toConsume.forEachRemaining(QuerySearchResult::releaseAggs);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
public int getNumReducePhases() {
|
|
|
return numReducePhases;
|
|
|
}
|
|
@@ -517,8 +550,10 @@ public class QueryPhaseResultConsumer extends ArraySearchPhaseResults<SearchPhas
|
|
|
}
|
|
|
|
|
|
private static void releaseAggs(List<QuerySearchResult> toConsume) {
|
|
|
- for (QuerySearchResult result : toConsume) {
|
|
|
- result.releaseAggs();
|
|
|
+ if (toConsume != null) {
|
|
|
+ for (QuerySearchResult result : toConsume) {
|
|
|
+ result.releaseAggs();
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|