Skip to content

Commit

Permalink
Allow aggs to disable offloading sequential collection (elastic#98276)
Browse files Browse the repository at this point in the history
In elastic#98204 we are introducing unconditional offloading of collection to a
separate thread pool, even for requests of phases that don't enable
search concurrency. It turns out that some aggs don't support offloading
their collection to a separate thread, as their postCollect method is
executed on the search thread which trips a lucene assertion around
reusing data structures pulled from the search worker thread.

With this commit we allow aggs to specify when they don't support
offloading their sequential collection. Such aggs are a subset of the
ones that already declare that they don't support concurrency entirely.

Relates to elastic#96023
  • Loading branch information
javanna authored Aug 8, 2023
1 parent 93dd279 commit 36d60e2
Show file tree
Hide file tree
Showing 6 changed files with 45 additions and 8 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -220,15 +220,39 @@ public boolean isInSortOrderExecutionRequired() {
}

/**
* Return false if this aggregation or any of the child aggregations does not support concurrent search
* Return false if this aggregation or any of the child aggregations does not support concurrent search.
* As a result, such aggregation will always be executed sequentially despite concurrency is enabled for the query phase.
* Note: aggregations that don't support concurrency, may or may not support offloading their collection to the search worker threads,
* depending on what {@link #supportsOffloadingSequentialCollection()} returns.
*/
public boolean supportsConcurrentExecution() {
if (isInSortOrderExecutionRequired()) {
return false;
}
for (AggregationBuilder builder : factoriesBuilder.getAggregatorFactories()) {
if (builder.supportsConcurrentExecution() == false) {
return false;
}
}
return isInSortOrderExecutionRequired() == false;
return supportsOffloadingSequentialCollection();
}

/**
* Returns false if this aggregation or any of its child aggregations does not support offloading its sequential collection
* to a separate thread. As a result, such aggregation will always be executed sequentially, and fully in the search thread,
* without offloading its collection to the search worker threads.
* Note: aggregations that don't support offloading sequential collection, don't support concurrency by definition.
*/
public boolean supportsOffloadingSequentialCollection() {
if (isInSortOrderExecutionRequired()) {
return false;
}
for (AggregationBuilder builder : factoriesBuilder.getAggregatorFactories()) {
if (builder.supportsOffloadingSequentialCollection() == false) {
return false;
}
}
return true;
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ public boolean supportsSampling() {
}

@Override
public boolean supportsConcurrentExecution() {
public boolean supportsOffloadingSequentialCollection() {
return false;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ public BucketCardinality bucketCardinality() {
}

@Override
public boolean supportsConcurrentExecution() {
public boolean supportsOffloadingSequentialCollection() {
return false;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,7 +113,7 @@ public boolean supportsSampling() {
}

@Override
public boolean supportsConcurrentExecution() {
public boolean supportsOffloadingSequentialCollection() {
return false;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,8 +89,6 @@ public class CardinalityAggregatorTests extends AggregatorTestCase {
/** Script to extract a collection of numeric values from the 'numbers' field **/
public static final String NUMERIC_VALUES_SCRIPT = "doc['numbers']";

public static final int HASHER_DEFAULT_SEED = 17;

@Override
protected ScriptService getMockScriptService() {
final Map<String, Function<Map<String, Object>, Object>> scripts = new HashMap<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,10 +61,25 @@ public void testFromXContent() throws IOException {
public void testSupportsConcurrentExecution() {
AB builder = createTestAggregatorBuilder();
boolean supportsConcurrency = builder.supportsConcurrentExecution();
if (supportsConcurrency) {
assertTrue(builder.supportsOffloadingSequentialCollection());
}
AggregationBuilder bucketBuilder = new HistogramAggregationBuilder("test");
assertThat(bucketBuilder.supportsConcurrentExecution(), equalTo(true));
assertTrue(bucketBuilder.supportsConcurrentExecution());
bucketBuilder.subAggregation(builder);
assertThat(bucketBuilder.supportsConcurrentExecution(), equalTo(supportsConcurrency));
if (bucketBuilder.supportsConcurrentExecution()) {
assertTrue(bucketBuilder.supportsOffloadingSequentialCollection());
}
}

public void testSupportsOffloadingSequentialCollection() {
AB builder = createTestAggregatorBuilder();
boolean supportsOffloadingSequentialCollection = builder.supportsOffloadingSequentialCollection();
AggregationBuilder bucketBuilder = new HistogramAggregationBuilder("test");
assertTrue(bucketBuilder.supportsOffloadingSequentialCollection());
bucketBuilder.subAggregation(builder);
assertThat(bucketBuilder.supportsOffloadingSequentialCollection(), equalTo(supportsOffloadingSequentialCollection));
}

/**
Expand Down

0 comments on commit 36d60e2

Please sign in to comment.