diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/220_filters_bucket.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/220_filters_bucket.yml index e5011f1bfd65c..45b67a397efe5 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/220_filters_bucket.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/220_filters_bucket.yml @@ -341,3 +341,50 @@ setup: - length: { aggregations.f.buckets: 2 } - match: { aggregations.f.buckets.foo.doc_count: 4 } - match: { aggregations.f.buckets.foo_bar.doc_count: 4 } + +--- +nested: + # Tests that we don't accidentally match nested documents when the filter + # matches it. + - do: + indices.create: + index: test_nested + body: + settings: + number_of_replicas: 0 + mappings: + properties: + i: + type: integer + nested: + type: nested + properties: + j: + type: integer + + - do: + bulk: + refresh: true + index: test_nested + body: + - index: {} + - i: 1 + nested: + - j: 2 + - j: 3 + - j: 4 + + - do: + search: + index: test_nested + body: + size: 0 + aggs: + f: + filters: + filters: + foo: + match_all: {} + - match: { hits.total.value: 1 } + - length: { aggregations.f.buckets: 1 } + - match: { aggregations.f.buckets.foo.doc_count: 1 } diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 901cc96d06a89..9d284b1a014ba 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -941,7 +941,12 @@ private void parseSource(DefaultSearchContext context, SearchSourceBuilder sourc AggregationContext aggContext = new ProductionAggregationContext( context.getQueryShardContext(), source.aggregations().bytesToPreallocate(), - context.query() == null ? new MatchAllDocsQuery() : context.query(), + /* + * The query on the search context right now doesn't include + * the filter for nested documents or slicing so we have to + * delay reading it until the aggs ask for it. + */ + () -> context.query() == null ? new MatchAllDocsQuery() : context.query(), context.getProfilers() == null ? null : context.getProfilers().getAggregationProfiler(), multiBucketConsumerService.create(), () -> new SubSearchContext(context).parsedQuery(context.parsedQuery()).fetchFieldsContext(context.fetchFieldsContext()), diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java index b83b0244ec4e8..3b1e4e41fcb9d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java @@ -253,7 +253,7 @@ public static class ProductionAggregationContext extends AggregationContext { private final QueryShardContext context; private final PreallocatedCircuitBreakerService breakerService; private final BigArrays bigArrays; - private final Query topLevelQuery; + private final Supplier topLevelQuery; private final AggregationProfiler profiler; private final MultiBucketConsumer multiBucketConsumer; private final Supplier subSearchContextBuilder; @@ -267,7 +267,7 @@ public static class ProductionAggregationContext extends AggregationContext { public ProductionAggregationContext( QueryShardContext context, long bytesToPreallocate, - Query topLevelQuery, + Supplier topLevelQuery, @Nullable AggregationProfiler profiler, MultiBucketConsumer multiBucketConsumer, Supplier subSearchContextBuilder, @@ -308,7 +308,7 @@ public ProductionAggregationContext( @Override public Query query() { - return topLevelQuery; + return topLevelQuery.get(); } @Override diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java index 44526aa755437..83d3c46697f8b 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java @@ -23,6 +23,8 @@ import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.FilterDirectoryReader; import org.apache.lucene.index.LeafReader; +import org.apache.lucene.search.ConstantScoreQuery; +import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.store.AlreadyClosedException; import org.elasticsearch.ElasticsearchException; @@ -42,9 +44,11 @@ import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.IndexNotFoundException; @@ -74,8 +78,10 @@ import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.MultiBucketConsumerService; +import org.elasticsearch.search.aggregations.bucket.filter.FiltersAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; +import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.fetch.FetchSearchResult; @@ -104,6 +110,7 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; import java.util.function.Function; import static java.util.Collections.singletonList; @@ -824,6 +831,52 @@ public void testSetSearchThrottled() { service.canMatch(req, ActionListener.wrap(r -> assertSame(Thread.currentThread(), currentThread), e -> fail("unexpected"))); } + public void testAggContextGetsMatchAll() throws IOException { + createIndex("test"); + withAggregationContext("test", context -> assertThat(context.query(), equalTo(new MatchAllDocsQuery()))); + } + + public void testAggContextGetsNestedFilter() throws IOException { + XContentBuilder mapping = JsonXContent.contentBuilder().startObject().startObject("properties"); + mapping.startObject("nested").field("type", "nested").endObject(); + mapping.endObject().endObject(); + + createIndex("test", Settings.EMPTY, mapping); + withAggregationContext( + "test", + context -> assertThat(context.query(), equalTo(new ConstantScoreQuery(Queries.newNonNestedFilter()))) + ); + } + + /** + * Build an {@link AggregationContext} with the named index. + */ + private void withAggregationContext(String index, Consumer check) throws IOException { + IndexService indexService = getInstanceFromNode(IndicesService.class).indexServiceSafe(resolveIndex(index)); + ShardId shardId = new ShardId(indexService.index(), 0); + + SearchRequest request = new SearchRequest().indices(index) + .source(new SearchSourceBuilder().aggregation(new FiltersAggregationBuilder("test", new MatchAllQueryBuilder()))) + .allowPartialSearchResults(false); + ShardSearchRequest shardRequest = new ShardSearchRequest( + OriginalIndices.NONE, + request, + shardId, + 0, + 1, + AliasFilter.EMPTY, + 1, + 0, + null + ); + + try (ReaderContext readerContext = createReaderContext(indexService, indexService.getShard(0))) { + try (SearchContext context = getInstanceFromNode(SearchService.class).createContext(readerContext, shardRequest, null, true)) { + check.accept(context.aggregations().factories().context()); + } + } + } + public void testExpandSearchThrottled() { createIndex("throttled_threadpool_index"); client().execute( diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregatorTests.java index 608d7f99e76bd..5c322288a6ed1 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregatorTests.java @@ -29,16 +29,23 @@ import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.store.Directory; +import org.elasticsearch.common.CheckedConsumer; +import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.index.mapper.DateFieldMapper; import org.elasticsearch.index.mapper.DateFieldMapper.Resolution; import org.elasticsearch.index.mapper.KeywordFieldMapper; +import org.elasticsearch.index.mapper.KeywordFieldMapper.KeywordFieldType; import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.ObjectMapper; +import org.elasticsearch.index.query.MatchAllQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.RangeQueryBuilder; +import org.elasticsearch.index.query.TermQueryBuilder; import org.elasticsearch.search.aggregations.AggregationBuilder; import org.elasticsearch.search.aggregations.AggregatorTestCase; import org.elasticsearch.search.aggregations.bucket.filter.FiltersAggregator.KeyedFilter; +import org.elasticsearch.search.aggregations.bucket.nested.NestedAggregatorTests; import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper; import org.junit.Before; @@ -289,4 +296,44 @@ public void testFilterByFilterCost() throws IOException { ft ); } + + /** + * Check that we don't accidentally find nested documents when the filter + * matches it. + */ + public void testNested() throws IOException { + KeywordFieldType ft = new KeywordFieldType("author"); + CheckedConsumer buildIndex = iw -> iw.addDocuments( + NestedAggregatorTests.generateBook("test", new String[] { "foo", "bar" }, new int[] { 5, 10, 15, 20 }) + ); + testCase( + new FiltersAggregationBuilder("test", new KeyedFilter("q1", new TermQueryBuilder("author", "foo"))), + Queries.newNonNestedFilter(), + buildIndex, + result -> { + InternalFilters filters = (InternalFilters) result; + assertThat(filters.getBuckets(), hasSize(1)); + assertThat(filters.getBucketByKey("q1").getDocCount(), equalTo(1L)); + }, + ft + ); + testCase( + new FiltersAggregationBuilder("test", new KeyedFilter("q1", new MatchAllQueryBuilder())), + Queries.newNonNestedFilter(), + buildIndex, + result -> { + InternalFilters filters = (InternalFilters) result; + assertThat(filters.getBuckets(), hasSize(1)); + assertThat(filters.getBucketByKey("q1").getDocCount(), equalTo(1L)); + }, + ft + ); + } + + @Override + protected List objectMappers() { + return MOCK_OBJECT_MAPPERS; + } + + static final List MOCK_OBJECT_MAPPERS = List.of(NestedAggregatorTests.nestedObject("nested_chapters")); } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java index 864e056d8b784..c043a77e55d54 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java @@ -858,7 +858,7 @@ private static double[] generateDocuments(List documents, int numNeste return values; } - private List generateBook(String id, String[] authors, int[] numPages) { + public static List generateBook(String id, String[] authors, int[] numPages) { List documents = new ArrayList<>(); for (int numPage : numPages) { @@ -874,6 +874,7 @@ private List generateBook(String id, String[] authors, int[] numPages) document.add(new Field(NestedPathFieldMapper.NAME, "book", NestedPathFieldMapper.Defaults.FIELD_TYPE)); document.add(sequenceIDFields.primaryTerm); for (String author : authors) { + document.add(new Field("author", author, KeywordFieldMapper.Defaults.FIELD_TYPE)); document.add(new SortedSetDocValuesField("author", new BytesRef(author))); } documents.add(document); diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index 74fe47e6f0c04..c75b8c36cc66c 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -296,7 +296,7 @@ public void onCache(ShardId shardId, Accountable accountable) {} AggregationContext context = new ProductionAggregationContext( queryShardContext, bytesToPreallocate, - query, + () -> query, null, consumer, () -> buildSubSearchContext(indexSettings, queryShardContext, bitsetFilterCache),