Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Remove needless locking and volatile access from QueryPhaseResultConsumer #119808

Closed
Closed
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -81,8 +81,8 @@ public class QueryPhaseResultConsumer extends ArraySearchPhaseResults<SearchPhas

private final TopDocsStats topDocsStats;
private volatile MergeResult mergeResult;
private volatile boolean hasPartialReduce;
private volatile int numReducePhases;
private boolean hasPartialReduce;
private int numReducePhases;

/**
* Creates a {@link QueryPhaseResultConsumer} that incrementally reduces aggregation results
Expand Down Expand Up @@ -152,39 +152,42 @@ public void consumeResult(SearchPhaseResult result, Runnable next) {

@Override
public SearchPhaseController.ReducedQueryPhase reduce() throws Exception {
var mergeResult = this.mergeResult;
this.mergeResult = null;
if (hasPendingMerges()) {
throw new AssertionError("partial reduce in-flight");
}
Exception f = failure.get();
if (f != null) {
throw f;
}

final List<QuerySearchResult> buffer;
synchronized (this) {
buffer = this.buffer;
}
// ensure consistent ordering
sortBuffer();
buffer.sort(RESULT_COMPARATOR);
final TopDocsStats topDocsStats = this.topDocsStats;
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;
synchronized (this) {
if (mergeResult != null) {
if (topDocsList != null) {
topDocsList.add(mergeResult.reducedTopDocs);
}
if (aggsList != null) {
aggsList.add(DelayableWriteable.referencing(mergeResult.reducedAggs));
}
if (mergeResult != null) {
if (topDocsList != null) {
topDocsList.add(mergeResult.reducedTopDocs);
}
for (QuerySearchResult result : buffer) {
topDocsStats.add(result.topDocs(), result.searchTimedOut(), result.terminatedEarly());
if (topDocsList != null) {
TopDocsAndMaxScore topDocs = result.consumeTopDocs();
setShardIndex(topDocs.topDocs, result.getShardIndex());
topDocsList.add(topDocs.topDocs);
}
if (aggsList != null) {
aggsList.add(result.getAggs());
}
if (aggsList != null) {
aggsList.add(DelayableWriteable.referencing(mergeResult.reducedAggs));
}
}
for (QuerySearchResult result : buffer) {
topDocsStats.add(result.topDocs(), result.searchTimedOut(), result.terminatedEarly());
if (topDocsList != null) {
TopDocsAndMaxScore topDocs = result.consumeTopDocs();
setShardIndex(topDocs.topDocs, result.getShardIndex());
topDocsList.add(topDocs.topDocs);
}
if (aggsList != null) {
aggsList.add(result.getAggs());
}
}
SearchPhaseController.ReducedQueryPhase reducePhase;
Expand Down Expand Up @@ -295,6 +298,7 @@ private MergeResult partialReduce(
return new MergeResult(processedShards, newTopDocs, newAggs, newAggs != null ? DelayableWriteable.getSerializedSize(newAggs) : 0);
}

// only used in tests
public int getNumReducePhases() {
return numReducePhases;
}
Expand All @@ -307,12 +311,6 @@ private boolean hasPendingMerges() {
return queue.isEmpty() == false || runningTask.get() != null;
}

void sortBuffer() {
if (buffer.size() > 0) {
buffer.sort(RESULT_COMPARATOR);
}
}

private synchronized void addWithoutBreaking(long size) {
circuitBreaker.addWithoutBreaking(size);
circuitBreakerBytes += size;
Expand Down Expand Up @@ -458,8 +456,7 @@ protected void doRun() {
long estimatedMergeSize = estimateRamBytesUsedForReduce(estimatedTotalSize);
addEstimateAndMaybeBreak(estimatedMergeSize);
estimatedTotalSize += estimatedMergeSize;
++numReducePhases;
newMerge = partialReduce(toConsume, mergeTask.emptyResults, topDocsStats, thisMergeResult, numReducePhases);
newMerge = partialReduce(toConsume, mergeTask.emptyResults, topDocsStats, thisMergeResult, ++numReducePhases);
} catch (Exception t) {
QueryPhaseResultConsumer.releaseAggs(toConsume);
onMergeFailure(t);
Expand All @@ -473,13 +470,13 @@ protected void doRun() {
if (hasAggs) {
// Update the circuit breaker to remove the size of the source aggregations
// and replace the estimation with the serialized size of the newly reduced result.
long newSize = mergeResult.estimatedSize - estimatedTotalSize;
long newSize = newMerge.estimatedSize - estimatedTotalSize;
addWithoutBreaking(newSize);
if (logger.isTraceEnabled()) {
logger.trace(
"aggs partial reduction [{}->{}] max [{}]",
estimatedTotalSize,
mergeResult.estimatedSize,
newMerge.estimatedSize,
maxAggsCurrentBufferSize
);
}
Expand Down