Skip to content

Commit

Permalink
Change InternalSignificantTerms to only sum shard level counts in fin…
Browse files Browse the repository at this point in the history
…al reduce

Signed-off-by: Jay Deng <jayd0104@gmail.com>
  • Loading branch information
jed326 authored and Jay Deng committed Jul 18, 2023
1 parent 57d5e90 commit a4ab229
Show file tree
Hide file tree
Showing 10 changed files with 100 additions and 9 deletions.
1 change: 1 addition & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/),
- Migrate client transports to Apache HttpClient / Core 5.x ([#4459](https://github.com/opensearch-project/OpenSearch/pull/4459))
- Change http code on create index API with bad input raising NotXContentException from 500 to 400 ([#4773](https://github.com/opensearch-project/OpenSearch/pull/4773))
- Improve summary error message for invalid setting updates ([#4792](https://github.com/opensearch-project/OpenSearch/pull/4792))
- Change InternalSignificantTerms to sum shard-level superset counts only in final reduce ([#8735](https://github.com/opensearch-project/OpenSearch/pull/8735))

### Deprecated

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@

package org.opensearch.search.aggregations.bucket;

import org.opensearch.action.admin.indices.delete.DeleteIndexRequest;
import org.opensearch.action.index.IndexRequestBuilder;
import org.opensearch.action.search.SearchRequestBuilder;
import org.opensearch.action.search.SearchResponse;
Expand All @@ -42,6 +43,7 @@
import org.opensearch.common.xcontent.XContentType;
import org.opensearch.index.query.QueryBuilder;
import org.opensearch.index.query.QueryBuilders;
import org.opensearch.index.query.TermQueryBuilder;
import org.opensearch.plugins.Plugin;
import org.opensearch.plugins.SearchPlugin;
import org.opensearch.script.MockScriptPlugin;
Expand Down Expand Up @@ -210,6 +212,34 @@ public void testXContentResponse() throws Exception {

}

public void testConsistencyWithDifferentShardCounts() throws Exception {
// The purpose of this test is to validate that the aggregation results do not change with shard count.
// bg_count for is summed up across shards, so in this test we compare a 1 shard and 2 shard search request
String type = randomBoolean() ? "text" : "long";
String settings = "{\"index.number_of_shards\": 1, \"index.number_of_replicas\": 0}";
SharedSignificantTermsTestMethods.index01Docs(type, settings, this);

SearchRequestBuilder request = client().prepareSearch(INDEX_NAME)
.setQuery(new TermQueryBuilder(CLASS_FIELD, "0"))
.addAggregation((significantTerms("sig_terms").field(TEXT_FIELD)));

SearchResponse response1 = request.get();

assertAcked(client().admin().indices().delete(new DeleteIndexRequest("*")).get());

settings = "{\"index.number_of_shards\": 2, \"index.number_of_replicas\": 0}";
// We use a custom routing strategy here to ensure that each shard will have at least 1 bucket.
// If there are no buckets collected for a shard, then that will affect the scoring and bg_count and our assertion will not be
// valid.
SharedSignificantTermsTestMethods.index01DocsWithRouting(type, settings, this);
SearchResponse response2 = request.get();

assertEquals(
response1.getAggregations().asMap().get("sig_terms").toString(),
response2.getAggregations().asMap().get("sig_terms").toString()
);
}

public void testPopularTermManyDeletedDocs() throws Exception {
String settings = "{\"index.number_of_shards\": 1, \"index.number_of_replicas\": 0}";
assertAcked(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -917,8 +917,10 @@ public ReaderContext readerContext() {
}

@Override
public InternalAggregation.ReduceContext partial() {
return requestToAggReduceContextBuilder.apply(request.source()).forPartialReduction();
public InternalAggregation.ReduceContext partialOnShard() {
InternalAggregation.ReduceContext rc = requestToAggReduceContextBuilder.apply(request.source()).forPartialReduction();
rc.setSliceLevel(isConcurrentSegmentSearchEnabled());
return rc;

Check warning on line 923 in server/src/main/java/org/opensearch/search/DefaultSearchContext.java

View check run for this annotation

Codecov / codecov/patch

server/src/main/java/org/opensearch/search/DefaultSearchContext.java#L921-L923

Added lines #L921 - L923 were not covered by tests
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ public ReduceableSearchResult reduce(Collection<Collector> collectors) throws IO
// using reduce is fine here instead of topLevelReduce as pipeline aggregation is evaluated on the coordinator after all
// documents are collected across shards for an aggregation
return new AggregationReduceableSearchResult(
InternalAggregations.reduce(Collections.singletonList(internalAggregations), context.partial())
InternalAggregations.reduce(Collections.singletonList(internalAggregations), context.partialOnShard())
);
} else {
return new AggregationReduceableSearchResult(internalAggregations);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,8 @@ public static class ReduceContext {
private final ScriptService scriptService;
private final IntConsumer multiBucketConsumer;
private final PipelineTree pipelineTreeRoot;

private boolean isSliceLevel;
/**
* Supplies the pipelines when the result of the reduce is serialized
* to node versions that need pipeline aggregators to be serialized
Expand Down Expand Up @@ -138,6 +140,7 @@ private ReduceContext(
this.multiBucketConsumer = multiBucketConsumer;
this.pipelineTreeRoot = pipelineTreeRoot;
this.pipelineTreeForBwcSerialization = pipelineTreeForBwcSerialization;
this.isSliceLevel = false;
}

/**
Expand All @@ -149,6 +152,14 @@ public boolean isFinalReduce() {
return pipelineTreeRoot != null;
}

public void setSliceLevel(boolean sliceLevel) {
this.isSliceLevel = sliceLevel;
}

Check warning on line 157 in server/src/main/java/org/opensearch/search/aggregations/InternalAggregation.java

View check run for this annotation

Codecov / codecov/patch

server/src/main/java/org/opensearch/search/aggregations/InternalAggregation.java#L156-L157

Added lines #L156 - L157 were not covered by tests

public boolean isSliceLevel() {
return this.isSliceLevel;
}

public BigArrays bigArrays() {
return bigArrays;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -232,7 +232,12 @@ public InternalAggregation reduce(List<InternalAggregation> aggregations, Reduce
@SuppressWarnings("unchecked")
InternalSignificantTerms<A, B> terms = (InternalSignificantTerms<A, B>) aggregation;
globalSubsetSize += terms.getSubsetSize();
globalSupersetSize += terms.getSupersetSize();
// supersetSize is a shard level count, if we sum it across slices we would produce num_buckets * supersetSize
if (reduceContext.isSliceLevel()) {
globalSupersetSize = terms.getSupersetSize();

Check warning on line 237 in server/src/main/java/org/opensearch/search/aggregations/bucket/terms/InternalSignificantTerms.java

View check run for this annotation

Codecov / codecov/patch

server/src/main/java/org/opensearch/search/aggregations/bucket/terms/InternalSignificantTerms.java#L237

Added line #L237 was not covered by tests
} else {
globalSupersetSize += terms.getSupersetSize();
}
}
Map<String, List<B>> buckets = new HashMap<>();
for (InternalAggregation aggregation : aggregations) {
Expand Down Expand Up @@ -291,7 +296,12 @@ protected B reduceBucket(List<B> buckets, ReduceContext context) {
List<InternalAggregations> aggregationsList = new ArrayList<>(buckets.size());
for (B bucket : buckets) {
subsetDf += bucket.subsetDf;
supersetDf += bucket.supersetDf;
// supersetDf is a shard level count, if we sum it across slices we would produce num_buckets * actual_supersetDf
if (context.isSliceLevel()) {
supersetDf = bucket.supersetDf;

Check warning on line 301 in server/src/main/java/org/opensearch/search/aggregations/bucket/terms/InternalSignificantTerms.java

View check run for this annotation

Codecov / codecov/patch

server/src/main/java/org/opensearch/search/aggregations/bucket/terms/InternalSignificantTerms.java#L301

Added line #L301 was not covered by tests
} else {
supersetDf += bucket.supersetDf;
}
aggregationsList.add(bucket.aggregations);
}
InternalAggregations aggs = InternalAggregations.reduce(aggregationsList, context);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -546,8 +546,8 @@ public ReaderContext readerContext() {
}

@Override
public InternalAggregation.ReduceContext partial() {
return in.partial();
public InternalAggregation.ReduceContext partialOnShard() {
return in.partialOnShard();

Check warning on line 550 in server/src/main/java/org/opensearch/search/internal/FilteredSearchContext.java

View check run for this annotation

Codecov / codecov/patch

server/src/main/java/org/opensearch/search/internal/FilteredSearchContext.java#L550

Added line #L550 was not covered by tests
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -465,7 +465,7 @@ public String toString() {

public abstract ReaderContext readerContext();

public abstract InternalAggregation.ReduceContext partial();
public abstract InternalAggregation.ReduceContext partialOnShard();

// processor used for bucket collectors
public abstract void setBucketCollectorProcessor(BucketCollectorProcessor bucketCollectorProcessor);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,4 +113,41 @@ public static void index01Docs(String type, String settings, OpenSearchIntegTest
indexRequestBuilderList.add(client().prepareIndex(INDEX_NAME).setId("7").setSource(TEXT_FIELD, "0", CLASS_FIELD, "0"));
testCase.indexRandom(true, false, indexRequestBuilderList);
}

public static void index01DocsWithRouting(String type, String settings, OpenSearchIntegTestCase testCase) throws ExecutionException,
InterruptedException {
String textMappings = "type=" + type;
if (type.equals("text")) {
textMappings += ",fielddata=true";
}
assertAcked(
testCase.prepareCreate(INDEX_NAME)
.setSettings(settings, XContentType.JSON)
.setMapping("text", textMappings, CLASS_FIELD, "type=keyword")
);
String[] gb = { "0", "1" };
List<IndexRequestBuilder> indexRequestBuilderList = new ArrayList<>();
indexRequestBuilderList.add(
client().prepareIndex(INDEX_NAME).setId("1").setSource(TEXT_FIELD, "1", CLASS_FIELD, "1").setRouting("0")
);
indexRequestBuilderList.add(
client().prepareIndex(INDEX_NAME).setId("2").setSource(TEXT_FIELD, "1", CLASS_FIELD, "1").setRouting("0")
);
indexRequestBuilderList.add(
client().prepareIndex(INDEX_NAME).setId("3").setSource(TEXT_FIELD, "0", CLASS_FIELD, "0").setRouting("0")
);
indexRequestBuilderList.add(
client().prepareIndex(INDEX_NAME).setId("4").setSource(TEXT_FIELD, "0", CLASS_FIELD, "0").setRouting("1")
);
indexRequestBuilderList.add(
client().prepareIndex(INDEX_NAME).setId("5").setSource(TEXT_FIELD, gb, CLASS_FIELD, "1").setRouting("1")
);
indexRequestBuilderList.add(
client().prepareIndex(INDEX_NAME).setId("6").setSource(TEXT_FIELD, gb, CLASS_FIELD, "0").setRouting("0")
);
indexRequestBuilderList.add(
client().prepareIndex(INDEX_NAME).setId("7").setSource(TEXT_FIELD, "0", CLASS_FIELD, "0").setRouting("0")
);
testCase.indexRandom(true, false, indexRequestBuilderList);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -659,7 +659,7 @@ public ReaderContext readerContext() {
}

@Override
public InternalAggregation.ReduceContext partial() {
public InternalAggregation.ReduceContext partialOnShard() {
return InternalAggregationTestCase.emptyReduceContextBuilder().forPartialReduction();
}

Expand Down

0 comments on commit a4ab229

Please sign in to comment.