Skip to content

Commit b0747c5

Browse files
authored
Fix bug with nested and filters agg (#67043)
Fixes a bug where nested documents that match a filter in the `filters` agg will be counted as matching the filter. Usually nested documents only match if you explicitly ask to match them. Worse, we only mach them in the "filter by filter" mode that we wrote to speed up date_histogram. The `filters` agg is fairly rare, but with #63643 we run `date_histogram` and `range` aggregations using `filters.
1 parent 981775f commit b0747c5

File tree

7 files changed

+159
-6
lines changed

7 files changed

+159
-6
lines changed

rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/220_filters_bucket.yml

Lines changed: 47 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -341,3 +341,50 @@ setup:
341341
- length: { aggregations.f.buckets: 2 }
342342
- match: { aggregations.f.buckets.foo.doc_count: 4 }
343343
- match: { aggregations.f.buckets.foo_bar.doc_count: 4 }
344+
345+
---
346+
nested:
347+
# Tests that we don't accidentally match nested documents when the filter
348+
# matches it.
349+
- do:
350+
indices.create:
351+
index: test_nested
352+
body:
353+
settings:
354+
number_of_replicas: 0
355+
mappings:
356+
properties:
357+
i:
358+
type: integer
359+
nested:
360+
type: nested
361+
properties:
362+
j:
363+
type: integer
364+
365+
- do:
366+
bulk:
367+
refresh: true
368+
index: test_nested
369+
body:
370+
- index: {}
371+
- i: 1
372+
nested:
373+
- j: 2
374+
- j: 3
375+
- j: 4
376+
377+
- do:
378+
search:
379+
index: test_nested
380+
body:
381+
size: 0
382+
aggs:
383+
f:
384+
filters:
385+
filters:
386+
foo:
387+
match_all: {}
388+
- match: { hits.total.value: 1 }
389+
- length: { aggregations.f.buckets: 1 }
390+
- match: { aggregations.f.buckets.foo.doc_count: 1 }

server/src/main/java/org/elasticsearch/search/SearchService.java

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -941,7 +941,12 @@ private void parseSource(DefaultSearchContext context, SearchSourceBuilder sourc
941941
AggregationContext aggContext = new ProductionAggregationContext(
942942
context.getQueryShardContext(),
943943
source.aggregations().bytesToPreallocate(),
944-
context.query() == null ? new MatchAllDocsQuery() : context.query(),
944+
/*
945+
* The query on the search context right now doesn't include
946+
* the filter for nested documents or slicing so we have to
947+
* delay reading it until the aggs ask for it.
948+
*/
949+
() -> context.query() == null ? new MatchAllDocsQuery() : context.query(),
945950
context.getProfilers() == null ? null : context.getProfilers().getAggregationProfiler(),
946951
multiBucketConsumerService.create(),
947952
() -> new SubSearchContext(context).parsedQuery(context.parsedQuery()).fetchFieldsContext(context.fetchFieldsContext()),

server/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -253,7 +253,7 @@ public static class ProductionAggregationContext extends AggregationContext {
253253
private final QueryShardContext context;
254254
private final PreallocatedCircuitBreakerService breakerService;
255255
private final BigArrays bigArrays;
256-
private final Query topLevelQuery;
256+
private final Supplier<Query> topLevelQuery;
257257
private final AggregationProfiler profiler;
258258
private final MultiBucketConsumer multiBucketConsumer;
259259
private final Supplier<SubSearchContext> subSearchContextBuilder;
@@ -267,7 +267,7 @@ public static class ProductionAggregationContext extends AggregationContext {
267267
public ProductionAggregationContext(
268268
QueryShardContext context,
269269
long bytesToPreallocate,
270-
Query topLevelQuery,
270+
Supplier<Query> topLevelQuery,
271271
@Nullable AggregationProfiler profiler,
272272
MultiBucketConsumer multiBucketConsumer,
273273
Supplier<SubSearchContext> subSearchContextBuilder,
@@ -308,7 +308,7 @@ public ProductionAggregationContext(
308308

309309
@Override
310310
public Query query() {
311-
return topLevelQuery;
311+
return topLevelQuery.get();
312312
}
313313

314314
@Override

server/src/test/java/org/elasticsearch/search/SearchServiceTests.java

Lines changed: 53 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,8 @@
2323
import org.apache.lucene.index.DirectoryReader;
2424
import org.apache.lucene.index.FilterDirectoryReader;
2525
import org.apache.lucene.index.LeafReader;
26+
import org.apache.lucene.search.ConstantScoreQuery;
27+
import org.apache.lucene.search.MatchAllDocsQuery;
2628
import org.apache.lucene.search.Query;
2729
import org.apache.lucene.store.AlreadyClosedException;
2830
import org.elasticsearch.ElasticsearchException;
@@ -42,9 +44,11 @@
4244
import org.elasticsearch.common.UUIDs;
4345
import org.elasticsearch.common.io.stream.StreamInput;
4446
import org.elasticsearch.common.io.stream.StreamOutput;
47+
import org.elasticsearch.common.lucene.search.Queries;
4548
import org.elasticsearch.common.settings.Settings;
4649
import org.elasticsearch.common.unit.TimeValue;
4750
import org.elasticsearch.common.xcontent.XContentBuilder;
51+
import org.elasticsearch.common.xcontent.json.JsonXContent;
4852
import org.elasticsearch.index.Index;
4953
import org.elasticsearch.index.IndexModule;
5054
import org.elasticsearch.index.IndexNotFoundException;
@@ -74,8 +78,10 @@
7478
import org.elasticsearch.search.aggregations.AggregationBuilders;
7579
import org.elasticsearch.search.aggregations.InternalAggregation;
7680
import org.elasticsearch.search.aggregations.MultiBucketConsumerService;
81+
import org.elasticsearch.search.aggregations.bucket.filter.FiltersAggregationBuilder;
7782
import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder;
7883
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
84+
import org.elasticsearch.search.aggregations.support.AggregationContext;
7985
import org.elasticsearch.search.aggregations.support.ValueType;
8086
import org.elasticsearch.search.builder.SearchSourceBuilder;
8187
import org.elasticsearch.search.fetch.FetchSearchResult;
@@ -104,6 +110,7 @@
104110
import java.util.concurrent.Semaphore;
105111
import java.util.concurrent.atomic.AtomicBoolean;
106112
import java.util.concurrent.atomic.AtomicInteger;
113+
import java.util.function.Consumer;
107114
import java.util.function.Function;
108115

109116
import static java.util.Collections.singletonList;
@@ -824,6 +831,52 @@ public void testSetSearchThrottled() {
824831
service.canMatch(req, ActionListener.wrap(r -> assertSame(Thread.currentThread(), currentThread), e -> fail("unexpected")));
825832
}
826833

834+
public void testAggContextGetsMatchAll() throws IOException {
835+
createIndex("test");
836+
withAggregationContext("test", context -> assertThat(context.query(), equalTo(new MatchAllDocsQuery())));
837+
}
838+
839+
public void testAggContextGetsNestedFilter() throws IOException {
840+
XContentBuilder mapping = JsonXContent.contentBuilder().startObject().startObject("properties");
841+
mapping.startObject("nested").field("type", "nested").endObject();
842+
mapping.endObject().endObject();
843+
844+
createIndex("test", Settings.EMPTY, mapping);
845+
withAggregationContext(
846+
"test",
847+
context -> assertThat(context.query(), equalTo(new ConstantScoreQuery(Queries.newNonNestedFilter())))
848+
);
849+
}
850+
851+
/**
852+
* Build an {@link AggregationContext} with the named index.
853+
*/
854+
private void withAggregationContext(String index, Consumer<AggregationContext> check) throws IOException {
855+
IndexService indexService = getInstanceFromNode(IndicesService.class).indexServiceSafe(resolveIndex(index));
856+
ShardId shardId = new ShardId(indexService.index(), 0);
857+
858+
SearchRequest request = new SearchRequest().indices(index)
859+
.source(new SearchSourceBuilder().aggregation(new FiltersAggregationBuilder("test", new MatchAllQueryBuilder())))
860+
.allowPartialSearchResults(false);
861+
ShardSearchRequest shardRequest = new ShardSearchRequest(
862+
OriginalIndices.NONE,
863+
request,
864+
shardId,
865+
0,
866+
1,
867+
AliasFilter.EMPTY,
868+
1,
869+
0,
870+
null
871+
);
872+
873+
try (ReaderContext readerContext = createReaderContext(indexService, indexService.getShard(0))) {
874+
try (SearchContext context = getInstanceFromNode(SearchService.class).createContext(readerContext, shardRequest, null, true)) {
875+
check.accept(context.aggregations().factories().context());
876+
}
877+
}
878+
}
879+
827880
public void testExpandSearchThrottled() {
828881
createIndex("throttled_threadpool_index");
829882
client().execute(

server/src/test/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregatorTests.java

Lines changed: 47 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -29,16 +29,23 @@
2929
import org.apache.lucene.search.MatchAllDocsQuery;
3030
import org.apache.lucene.search.Query;
3131
import org.apache.lucene.store.Directory;
32+
import org.elasticsearch.common.CheckedConsumer;
33+
import org.elasticsearch.common.lucene.search.Queries;
3234
import org.elasticsearch.index.mapper.DateFieldMapper;
3335
import org.elasticsearch.index.mapper.DateFieldMapper.Resolution;
3436
import org.elasticsearch.index.mapper.KeywordFieldMapper;
37+
import org.elasticsearch.index.mapper.KeywordFieldMapper.KeywordFieldType;
3538
import org.elasticsearch.index.mapper.MappedFieldType;
39+
import org.elasticsearch.index.mapper.ObjectMapper;
40+
import org.elasticsearch.index.query.MatchAllQueryBuilder;
3641
import org.elasticsearch.index.query.QueryBuilder;
3742
import org.elasticsearch.index.query.QueryBuilders;
3843
import org.elasticsearch.index.query.RangeQueryBuilder;
44+
import org.elasticsearch.index.query.TermQueryBuilder;
3945
import org.elasticsearch.search.aggregations.AggregationBuilder;
4046
import org.elasticsearch.search.aggregations.AggregatorTestCase;
4147
import org.elasticsearch.search.aggregations.bucket.filter.FiltersAggregator.KeyedFilter;
48+
import org.elasticsearch.search.aggregations.bucket.nested.NestedAggregatorTests;
4249
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
4350
import org.junit.Before;
4451

@@ -289,4 +296,44 @@ public void testFilterByFilterCost() throws IOException {
289296
ft
290297
);
291298
}
299+
300+
/**
301+
* Check that we don't accidentally find nested documents when the filter
302+
* matches it.
303+
*/
304+
public void testNested() throws IOException {
305+
KeywordFieldType ft = new KeywordFieldType("author");
306+
CheckedConsumer<RandomIndexWriter, IOException> buildIndex = iw -> iw.addDocuments(
307+
NestedAggregatorTests.generateBook("test", new String[] { "foo", "bar" }, new int[] { 5, 10, 15, 20 })
308+
);
309+
testCase(
310+
new FiltersAggregationBuilder("test", new KeyedFilter("q1", new TermQueryBuilder("author", "foo"))),
311+
Queries.newNonNestedFilter(),
312+
buildIndex,
313+
result -> {
314+
InternalFilters filters = (InternalFilters) result;
315+
assertThat(filters.getBuckets(), hasSize(1));
316+
assertThat(filters.getBucketByKey("q1").getDocCount(), equalTo(1L));
317+
},
318+
ft
319+
);
320+
testCase(
321+
new FiltersAggregationBuilder("test", new KeyedFilter("q1", new MatchAllQueryBuilder())),
322+
Queries.newNonNestedFilter(),
323+
buildIndex,
324+
result -> {
325+
InternalFilters filters = (InternalFilters) result;
326+
assertThat(filters.getBuckets(), hasSize(1));
327+
assertThat(filters.getBucketByKey("q1").getDocCount(), equalTo(1L));
328+
},
329+
ft
330+
);
331+
}
332+
333+
@Override
334+
protected List<ObjectMapper> objectMappers() {
335+
return MOCK_OBJECT_MAPPERS;
336+
}
337+
338+
static final List<ObjectMapper> MOCK_OBJECT_MAPPERS = List.of(NestedAggregatorTests.nestedObject("nested_chapters"));
292339
}

server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -858,7 +858,7 @@ private static double[] generateDocuments(List<Document> documents, int numNeste
858858
return values;
859859
}
860860

861-
private List<Document> generateBook(String id, String[] authors, int[] numPages) {
861+
public static List<Document> generateBook(String id, String[] authors, int[] numPages) {
862862
List<Document> documents = new ArrayList<>();
863863

864864
for (int numPage : numPages) {
@@ -874,6 +874,7 @@ private List<Document> generateBook(String id, String[] authors, int[] numPages)
874874
document.add(new Field(NestedPathFieldMapper.NAME, "book", NestedPathFieldMapper.Defaults.FIELD_TYPE));
875875
document.add(sequenceIDFields.primaryTerm);
876876
for (String author : authors) {
877+
document.add(new Field("author", author, KeywordFieldMapper.Defaults.FIELD_TYPE));
877878
document.add(new SortedSetDocValuesField("author", new BytesRef(author)));
878879
}
879880
documents.add(document);

test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -296,7 +296,7 @@ public void onCache(ShardId shardId, Accountable accountable) {}
296296
AggregationContext context = new ProductionAggregationContext(
297297
queryShardContext,
298298
bytesToPreallocate,
299-
query,
299+
() -> query,
300300
null,
301301
consumer,
302302
() -> buildSubSearchContext(indexSettings, queryShardContext, bitsetFilterCache),

0 commit comments

Comments
 (0)