diff --git a/server/src/main/java/org/elasticsearch/search/SearchModule.java b/server/src/main/java/org/elasticsearch/search/SearchModule.java index e170b0b8ad3d1..237a9e67162fa 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchModule.java +++ b/server/src/main/java/org/elasticsearch/search/SearchModule.java @@ -381,7 +381,8 @@ private void registerAggregations(List plugins) { .addResultReader(StringTerms.NAME, StringTerms::new) .addResultReader(UnmappedTerms.NAME, UnmappedTerms::new) .addResultReader(LongTerms.NAME, LongTerms::new) - .addResultReader(DoubleTerms.NAME, DoubleTerms::new)); + .addResultReader(DoubleTerms.NAME, DoubleTerms::new) + .setAggregatorRegistrar(TermsAggregationBuilder::registerAggregators)); registerAggregation(new AggregationSpec(RareTermsAggregationBuilder.NAME, RareTermsAggregationBuilder::new, RareTermsAggregationBuilder::parse) .addResultReader(StringRareTerms.NAME, StringRareTerms::new) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregationBuilder.java index 110929ffccb67..6a3b1da5f5020 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregationBuilder.java @@ -42,12 +42,14 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper; +import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceType; import java.io.IOException; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.concurrent.atomic.AtomicBoolean; public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder implements MultiBucketAggregationBuilder { @@ -100,6 +102,13 @@ public static AggregationBuilder parse(String aggregationName, XContentParser pa return PARSER.parse(parser, new TermsAggregationBuilder(aggregationName), null); } + private static AtomicBoolean wasRegistered = new AtomicBoolean(false); + public static void registerAggregators(ValuesSourceRegistry valuesSourceRegistry) { + if (wasRegistered.compareAndSet(false, true) == true) { + TermsAggregatorFactory.registerAggregators(valuesSourceRegistry); + } + } + private BucketOrder order = BucketOrder.compound(BucketOrder.count(false)); // automatically adds tie-breaker key asc order private IncludeExclude includeExclude = null; private String executionHint = null; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java index b39587aff480c..9880822b63c62 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java @@ -38,9 +38,12 @@ import org.elasticsearch.search.aggregations.bucket.BucketUtils; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.aggregations.support.AggregatorSupplier; +import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; @@ -59,18 +62,141 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory { private final TermsAggregator.BucketCountThresholds bucketCountThresholds; private final boolean showTermDocCountError; + // TODO: Registration should happen on the actual aggregator classes + static void registerAggregators(ValuesSourceRegistry valuesSourceRegistry) { + valuesSourceRegistry.register(TermsAggregationBuilder.NAME, + List.of(CoreValuesSourceType.BYTES, CoreValuesSourceType.IP), + TermsAggregatorFactory.bytesSupplier()); + + valuesSourceRegistry.register(TermsAggregationBuilder.NAME, + List.of(CoreValuesSourceType.DATE, CoreValuesSourceType.BOOLEAN, CoreValuesSourceType.NUMERIC), + TermsAggregatorFactory.numericSupplier()); + } + + /** + * This supplier is used for all the field types that should be aggregated as bytes/strings, + * including those that need global ordinals + */ + private static TermsAggregatorSupplier bytesSupplier() { + return new TermsAggregatorSupplier() { + @Override + public Aggregator build(String name, + AggregatorFactories factories, + ValuesSource valuesSource, + BucketOrder order, + DocValueFormat format, + TermsAggregator.BucketCountThresholds bucketCountThresholds, + IncludeExclude includeExclude, + String executionHint, + SearchContext context, + Aggregator parent, + SubAggCollectionMode subAggCollectMode, + boolean showTermDocCountError, + List pipelineAggregators, + Map metaData) throws IOException { + + ExecutionMode execution = null; + if (executionHint != null) { + execution = ExecutionMode.fromString(executionHint, deprecationLogger); + } + // In some cases, using ordinals is just not supported: override it + if (valuesSource instanceof ValuesSource.Bytes.WithOrdinals == false) { + execution = ExecutionMode.MAP; + } + final long maxOrd = execution == ExecutionMode.GLOBAL_ORDINALS ? getMaxOrd(valuesSource, context.searcher()) : -1; + if (execution == null) { + execution = ExecutionMode.GLOBAL_ORDINALS; + } + if (subAggCollectMode == null) { + subAggCollectMode = SubAggCollectionMode.DEPTH_FIRST; + // TODO can we remove concept of AggregatorFactories.EMPTY? + if (factories != AggregatorFactories.EMPTY) { + subAggCollectMode = subAggCollectionMode(bucketCountThresholds.getShardSize(), maxOrd); + } + } + + if ((includeExclude != null) && (includeExclude.isRegexBased()) && format != DocValueFormat.RAW) { + // TODO this exception message is not really accurate for the string case. It's really disallowing regex + formatter + throw new AggregationExecutionException("Aggregation [" + name + "] cannot support regular expression style " + + "include/exclude settings as they can only be applied to string fields. Use an array of values for " + + "include/exclude clauses"); + } + + // TODO: [Zach] we might want refactor and remove ExecutionMode#create(), moving that logic outside the enum + return execution.create(name, factories, valuesSource, order, format, bucketCountThresholds, includeExclude, + context, parent, subAggCollectMode, showTermDocCountError, pipelineAggregators, metaData); + + } + }; + } + + /** + * This supplier is used for all fields that expect to be aggregated as a numeric value. + * This includes floating points, and formatted types that use numerics internally for storage (date, boolean, etc) + */ + private static TermsAggregatorSupplier numericSupplier() { + return new TermsAggregatorSupplier() { + @Override + public Aggregator build(String name, + AggregatorFactories factories, + ValuesSource valuesSource, + BucketOrder order, + DocValueFormat format, + TermsAggregator.BucketCountThresholds bucketCountThresholds, + IncludeExclude includeExclude, + String executionHint, + SearchContext context, + Aggregator parent, + SubAggCollectionMode subAggCollectMode, + boolean showTermDocCountError, + List pipelineAggregators, + Map metaData) throws IOException { + + if ((includeExclude != null) && (includeExclude.isRegexBased())) { + throw new AggregationExecutionException("Aggregation [" + name + "] cannot support regular expression style " + + "include/exclude settings as they can only be applied to string fields. Use an array of numeric values for " + + "include/exclude clauses used to filter numeric fields"); + } + + IncludeExclude.LongFilter longFilter = null; + if (subAggCollectMode == null) { + // TODO can we remove concept of AggregatorFactories.EMPTY? + if (factories != AggregatorFactories.EMPTY) { + subAggCollectMode = subAggCollectionMode(bucketCountThresholds.getShardSize(), -1); + } else { + subAggCollectMode = SubAggCollectionMode.DEPTH_FIRST; + } + } + if (((ValuesSource.Numeric) valuesSource).isFloatingPoint()) { + if (includeExclude != null) { + longFilter = includeExclude.convertToDoubleFilter(); + } + return new DoubleTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, format, order, + bucketCountThresholds, context, parent, subAggCollectMode, showTermDocCountError, longFilter, + pipelineAggregators, metaData); + } + if (includeExclude != null) { + longFilter = includeExclude.convertToLongFilter(format); + } + return new LongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, format, order, + bucketCountThresholds, context, parent, subAggCollectMode, showTermDocCountError, longFilter, + pipelineAggregators, metaData); + } + }; + } + TermsAggregatorFactory(String name, - ValuesSourceConfig config, - BucketOrder order, - IncludeExclude includeExclude, - String executionHint, - SubAggCollectionMode collectMode, - TermsAggregator.BucketCountThresholds bucketCountThresholds, - boolean showTermDocCountError, - QueryShardContext queryShardContext, - AggregatorFactory parent, - AggregatorFactories.Builder subFactoriesBuilder, - Map metaData) throws IOException { + ValuesSourceConfig config, + BucketOrder order, + IncludeExclude includeExclude, + String executionHint, + SubAggCollectionMode collectMode, + TermsAggregator.BucketCountThresholds bucketCountThresholds, + boolean showTermDocCountError, + QueryShardContext queryShardContext, + AggregatorFactory parent, + AggregatorFactories.Builder subFactoriesBuilder, + Map metaData) throws IOException { super(name, config, queryShardContext, parent, subFactoriesBuilder, metaData); this.order = order; this.includeExclude = includeExclude; @@ -114,89 +240,36 @@ private static boolean isAggregationSort(BucketOrder order) { @Override protected Aggregator doCreateInternal(ValuesSource valuesSource, - SearchContext searchContext, - Aggregator parent, - boolean collectsFromSingleBucket, - List pipelineAggregators, - Map metaData) throws IOException { + SearchContext searchContext, + Aggregator parent, + boolean collectsFromSingleBucket, + List pipelineAggregators, + Map metaData) throws IOException { if (collectsFromSingleBucket == false) { return asMultiBucketAggregator(this, searchContext, parent); } + + AggregatorSupplier aggregatorSupplier = ValuesSourceRegistry.getInstance().getAggregator(config.valueSourceType(), + TermsAggregationBuilder.NAME); + if (aggregatorSupplier instanceof TermsAggregatorSupplier == false) { + throw new AggregationExecutionException("Registry miss-match - expected TermsAggregatorSupplier, found [" + + aggregatorSupplier.getClass().toString() + "]"); + } + + TermsAggregatorSupplier termsAggregatorSupplier = (TermsAggregatorSupplier) aggregatorSupplier; BucketCountThresholds bucketCountThresholds = new BucketCountThresholds(this.bucketCountThresholds); if (InternalOrder.isKeyOrder(order) == false - && bucketCountThresholds.getShardSize() == TermsAggregationBuilder.DEFAULT_BUCKET_COUNT_THRESHOLDS.getShardSize()) { + && bucketCountThresholds.getShardSize() == TermsAggregationBuilder.DEFAULT_BUCKET_COUNT_THRESHOLDS.getShardSize()) { // The user has not made a shardSize selection. Use default // heuristic to avoid any wrong-ranking caused by distributed // counting bucketCountThresholds.setShardSize(BucketUtils.suggestShardSideQueueSize(bucketCountThresholds.getRequiredSize())); } bucketCountThresholds.ensureValidity(); - if (valuesSource instanceof ValuesSource.Bytes) { - ExecutionMode execution = null; - if (executionHint != null) { - execution = ExecutionMode.fromString(executionHint, deprecationLogger); - } - // In some cases, using ordinals is just not supported: override it - if (valuesSource instanceof ValuesSource.Bytes.WithOrdinals == false) { - execution = ExecutionMode.MAP; - } - final long maxOrd = execution == ExecutionMode.GLOBAL_ORDINALS ? getMaxOrd(valuesSource, searchContext.searcher()) : -1; - if (execution == null) { - execution = ExecutionMode.GLOBAL_ORDINALS; - } - SubAggCollectionMode cm = collectMode; - if (cm == null) { - cm = SubAggCollectionMode.DEPTH_FIRST; - if (factories != AggregatorFactories.EMPTY) { - cm = subAggCollectionMode(bucketCountThresholds.getShardSize(), maxOrd); - } - } - - DocValueFormat format = config.format(); - if ((includeExclude != null) && (includeExclude.isRegexBased()) && format != DocValueFormat.RAW) { - throw new AggregationExecutionException("Aggregation [" + name + "] cannot support regular expression style " - + "include/exclude settings as they can only be applied to string fields. Use an array of values for " - + "include/exclude clauses"); - } - - return execution.create(name, factories, valuesSource, order, format, - bucketCountThresholds, includeExclude, searchContext, parent, cm, showTermDocCountError, pipelineAggregators, metaData); - } - - if ((includeExclude != null) && (includeExclude.isRegexBased())) { - throw new AggregationExecutionException("Aggregation [" + name + "] cannot support regular expression style " - + "include/exclude settings as they can only be applied to string fields. Use an array of numeric values for " - + "include/exclude clauses used to filter numeric fields"); - } - - if (valuesSource instanceof ValuesSource.Numeric) { - IncludeExclude.LongFilter longFilter = null; - SubAggCollectionMode cm = collectMode; - if (cm == null) { - if (factories != AggregatorFactories.EMPTY) { - cm = subAggCollectionMode(bucketCountThresholds.getShardSize(), -1); - } else { - cm = SubAggCollectionMode.DEPTH_FIRST; - } - } - if (((ValuesSource.Numeric) valuesSource).isFloatingPoint()) { - if (includeExclude != null) { - longFilter = includeExclude.convertToDoubleFilter(); - } - return new DoubleTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), order, - bucketCountThresholds, searchContext, parent, cm, showTermDocCountError, longFilter, - pipelineAggregators, metaData); - } - if (includeExclude != null) { - longFilter = includeExclude.convertToLongFilter(config.format()); - } - return new LongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), order, - bucketCountThresholds, searchContext, parent, cm, showTermDocCountError, longFilter, pipelineAggregators, - metaData); - } - throw new AggregationExecutionException("terms aggregation cannot be applied to field [" + config.fieldContext().field() - + "]. It can only be applied to numeric or string fields."); + return termsAggregatorSupplier.build(name, factories, valuesSource, order, config.format(), + bucketCountThresholds, includeExclude, executionHint, searchContext, parent, collectMode, + showTermDocCountError, pipelineAggregators, metaData); } // return the SubAggCollectionMode that this aggregation should use based on the expected size @@ -217,7 +290,7 @@ static SubAggCollectionMode subAggCollectionMode(int expectedSize, long maxOrd) * Get the maximum global ordinal value for the provided {@link ValuesSource} or -1 * if the values source is not an instance of {@link ValuesSource.Bytes.WithOrdinals}. */ - static long getMaxOrd(ValuesSource source, IndexSearcher searcher) throws IOException { + private static long getMaxOrd(ValuesSource source, IndexSearcher searcher) throws IOException { if (source instanceof ValuesSource.Bytes.WithOrdinals) { ValuesSource.Bytes.WithOrdinals valueSourceWithOrdinals = (ValuesSource.Bytes.WithOrdinals) source; return valueSourceWithOrdinals.globalMaxOrd(searcher); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorSupplier.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorSupplier.java new file mode 100644 index 0000000000000..ccfb082ac98ce --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorSupplier.java @@ -0,0 +1,51 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.search.aggregations.bucket.terms; + +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.BucketOrder; +import org.elasticsearch.search.aggregations.bucket.terms.IncludeExclude; +import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.aggregations.support.AggregatorSupplier; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +interface TermsAggregatorSupplier extends AggregatorSupplier { + Aggregator build(String name, + AggregatorFactories factories, + ValuesSource valuesSource, + BucketOrder order, + DocValueFormat format, + TermsAggregator.BucketCountThresholds bucketCountThresholds, + IncludeExclude includeExclude, + String executionHint, + SearchContext context, + Aggregator parent, + Aggregator.SubAggCollectionMode subAggCollectMode, + boolean showTermDocCountError, + List pipelineAggregators, + Map metaData) throws IOException; +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/BinaryTermsAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/BinaryTermsAggregatorTests.java new file mode 100644 index 0000000000000..31859a7a4fa04 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/BinaryTermsAggregatorTests.java @@ -0,0 +1,176 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.search.aggregations.bucket.terms; + +import org.apache.lucene.document.Document; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.MatchNoDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.automaton.RegExp; +import org.elasticsearch.common.Numbers; +import org.elasticsearch.index.mapper.BinaryFieldMapper; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregatorTestCase; +import org.elasticsearch.search.aggregations.support.ValueType; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Consumer; + +import static org.hamcrest.Matchers.equalTo; + +public class BinaryTermsAggregatorTests extends AggregatorTestCase { + private static final String BINARY_FIELD = "binary"; + + private static final List dataset; + static { + List d = new ArrayList<>(45); + for (int i = 0; i < 10; i++) { + for (int j = 0; j < i; j++) { + d.add((long) i); + } + } + dataset = d; + } + + public void testMatchNoDocs() throws IOException { + testBothCases(new MatchNoDocsQuery(), dataset, + aggregation -> aggregation.field(BINARY_FIELD), + agg -> assertEquals(0, agg.getBuckets().size()), ValueType.STRING + ); + } + + public void testMatchAllDocs() throws IOException { + Query query = new MatchAllDocsQuery(); + + testBothCases(query, dataset, + aggregation -> aggregation.field(BINARY_FIELD), + agg -> { + assertEquals(9, agg.getBuckets().size()); + for (int i = 0; i < 9; i++) { + StringTerms.Bucket bucket = (StringTerms.Bucket) agg.getBuckets().get(i); + byte[] bytes = Numbers.longToBytes(9L - i); + String bytesAsString = (String) DocValueFormat.BINARY.format(new BytesRef(bytes)); + assertThat(bucket.getKey(), equalTo(bytesAsString)); + assertThat(bucket.getDocCount(), equalTo(9L - i)); + } + }, null); + } + + public void testBadIncludeExclude() throws IOException { + IncludeExclude includeExclude = new IncludeExclude(new RegExp("foo"), null); + + // Make sure the include/exclude fails regardless of how the user tries to type hint the agg + AggregationExecutionException e = expectThrows(AggregationExecutionException.class, + () -> testBothCases(new MatchNoDocsQuery(), dataset, + aggregation -> aggregation.field(BINARY_FIELD).includeExclude(includeExclude).format("yyyy-MM-dd"), + agg -> fail("test should have failed with exception"), null // default, no hint + )); + assertThat(e.getMessage(), equalTo("Aggregation [_name] cannot support regular expression style include/exclude settings as " + + "they can only be applied to string fields. Use an array of values for include/exclude clauses")); + + e = expectThrows(AggregationExecutionException.class, + () -> testBothCases(new MatchNoDocsQuery(), dataset, + aggregation -> aggregation.field(BINARY_FIELD).includeExclude(includeExclude).format("yyyy-MM-dd"), + agg -> fail("test should have failed with exception"), ValueType.STRING // string type hint + )); + assertThat(e.getMessage(), equalTo("Aggregation [_name] cannot support regular expression style include/exclude settings as " + + "they can only be applied to string fields. Use an array of values for include/exclude clauses")); + + e = expectThrows(AggregationExecutionException.class, () -> testBothCases(new MatchNoDocsQuery(), dataset, + aggregation -> aggregation.field(BINARY_FIELD).includeExclude(includeExclude), + agg -> fail("test should have failed with exception"), ValueType.NUMERIC // numeric type hint + )); + assertThat(e.getMessage(), equalTo("Aggregation [_name] cannot support regular expression style include/exclude settings as " + + "they can only be applied to string fields. Use an array of values for include/exclude clauses")); + } + + private void testSearchCase(Query query, List dataset, + Consumer configure, + Consumer verify, ValueType valueType) throws IOException { + executeTestCase(false, query, dataset, configure, verify, valueType); + } + + private void testSearchAndReduceCase(Query query, List dataset, + Consumer configure, + Consumer verify, ValueType valueType) throws IOException { + executeTestCase(true, query, dataset, configure, verify, valueType); + } + + private void testBothCases(Query query, List dataset, + Consumer configure, + Consumer verify, ValueType valueType) throws IOException { + testSearchCase(query, dataset, configure, verify, valueType); + testSearchAndReduceCase(query, dataset, configure, verify, valueType); + } + + private void executeTestCase(boolean reduced, Query query, List dataset, + Consumer configure, + Consumer verify, ValueType valueType) throws IOException { + + try (Directory directory = newDirectory()) { + try (RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory)) { + Document document = new Document(); + for (Long value : dataset) { + if (frequently()) { + indexWriter.commit(); + } + + document.add(new BinaryFieldMapper.CustomBinaryDocValuesField(BINARY_FIELD, Numbers.longToBytes(value))); + indexWriter.addDocument(document); + document.clear(); + } + } + + try (IndexReader indexReader = DirectoryReader.open(directory)) { + IndexSearcher indexSearcher = newIndexSearcher(indexReader); + + TermsAggregationBuilder aggregationBuilder = new TermsAggregationBuilder("_name"); + if (valueType != null) { + aggregationBuilder.userValueTypeHint(valueType); + } + if (configure != null) { + configure.accept(aggregationBuilder); + } + + MappedFieldType binaryFieldType = new BinaryFieldMapper.Builder(BINARY_FIELD).fieldType(); + binaryFieldType.setName(BINARY_FIELD); + binaryFieldType.setHasDocValues(true); + + InternalMappedTerms rareTerms; + if (reduced) { + rareTerms = searchAndReduce(indexSearcher, query, aggregationBuilder, binaryFieldType); + } else { + rareTerms = search(indexSearcher, query, aggregationBuilder, binaryFieldType); + } + verify.accept(rareTerms); + } + } + } + +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/KeywordTermsAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/KeywordTermsAggregatorTests.java new file mode 100644 index 0000000000000..098e94d3d4450 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/KeywordTermsAggregatorTests.java @@ -0,0 +1,161 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.search.aggregations.bucket.terms; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.SortedSetDocValuesField; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.MatchNoDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.index.mapper.KeywordFieldMapper; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.search.aggregations.AggregatorTestCase; +import org.elasticsearch.search.aggregations.support.ValueType; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Consumer; + +import static org.hamcrest.Matchers.equalTo; + +public class KeywordTermsAggregatorTests extends AggregatorTestCase { + private static final String KEYWORD_FIELD = "keyword"; + + private static final List dataset; + static { + List d = new ArrayList<>(45); + for (int i = 0; i < 10; i++) { + for (int j = 0; j < i; j++) { + d.add(String.valueOf(i)); + } + } + dataset = d; + } + + public void testMatchNoDocs() throws IOException { + testBothCases(new MatchNoDocsQuery(), dataset, + aggregation -> aggregation.field(KEYWORD_FIELD), + agg -> assertEquals(0, agg.getBuckets().size()), null // without type hint + ); + + testBothCases(new MatchNoDocsQuery(), dataset, + aggregation -> aggregation.field(KEYWORD_FIELD), + agg -> assertEquals(0, agg.getBuckets().size()), ValueType.STRING // with type hint + ); + } + + public void testMatchAllDocs() throws IOException { + Query query = new MatchAllDocsQuery(); + + testBothCases(query, dataset, + aggregation -> aggregation.field(KEYWORD_FIELD), + agg -> { + assertEquals(9, agg.getBuckets().size()); + for (int i = 0; i < 9; i++) { + StringTerms.Bucket bucket = (StringTerms.Bucket) agg.getBuckets().get(i); + assertThat(bucket.getKey(), equalTo(String.valueOf(9L - i))); + assertThat(bucket.getDocCount(), equalTo(9L - i)); + } + }, null // without type hint + ); + + testBothCases(query, dataset, + aggregation -> aggregation.field(KEYWORD_FIELD), + agg -> { + assertEquals(9, agg.getBuckets().size()); + for (int i = 0; i < 9; i++) { + StringTerms.Bucket bucket = (StringTerms.Bucket) agg.getBuckets().get(i); + assertThat(bucket.getKey(), equalTo(String.valueOf(9L - i))); + assertThat(bucket.getDocCount(), equalTo(9L - i)); + } + }, ValueType.STRING // with type hint + ); + } + + private void testSearchCase(Query query, List dataset, + Consumer configure, + Consumer verify, ValueType valueType) throws IOException { + executeTestCase(false, query, dataset, configure, verify, valueType); + } + + private void testSearchAndReduceCase(Query query, List dataset, + Consumer configure, + Consumer verify, ValueType valueType) throws IOException { + executeTestCase(true, query, dataset, configure, verify, valueType); + } + + private void testBothCases(Query query, List dataset, + Consumer configure, + Consumer verify, ValueType valueType) throws IOException { + testSearchCase(query, dataset, configure, verify, valueType); + testSearchAndReduceCase(query, dataset, configure, verify, valueType); + } + + private void executeTestCase(boolean reduced, Query query, List dataset, + Consumer configure, + Consumer verify, ValueType valueType) throws IOException { + + try (Directory directory = newDirectory()) { + try (RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory)) { + Document document = new Document(); + for (String value : dataset) { + if (frequently()) { + indexWriter.commit(); + } + + document.add(new SortedSetDocValuesField(KEYWORD_FIELD, new BytesRef(value))); + indexWriter.addDocument(document); + document.clear(); + } + } + + try (IndexReader indexReader = DirectoryReader.open(directory)) { + IndexSearcher indexSearcher = newIndexSearcher(indexReader); + + TermsAggregationBuilder aggregationBuilder = new TermsAggregationBuilder("_name"); + if (valueType != null) { + aggregationBuilder.userValueTypeHint(valueType); + } + if (configure != null) { + configure.accept(aggregationBuilder); + } + + MappedFieldType keywordFieldType = new KeywordFieldMapper.KeywordFieldType(); + keywordFieldType.setName(KEYWORD_FIELD); + keywordFieldType.setHasDocValues(true); + + InternalMappedTerms rareTerms; + if (reduced) { + rareTerms = searchAndReduce(indexSearcher, query, aggregationBuilder, keywordFieldType); + } else { + rareTerms = search(indexSearcher, query, aggregationBuilder, keywordFieldType); + } + verify.accept(rareTerms); + } + } + } + +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/NumericTermsAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/NumericTermsAggregatorTests.java new file mode 100644 index 0000000000000..b0790b9fa0413 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/NumericTermsAggregatorTests.java @@ -0,0 +1,190 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.search.aggregations.bucket.terms; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.MatchNoDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.automaton.RegExp; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregatorTestCase; +import org.elasticsearch.search.aggregations.support.ValueType; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Consumer; + +import static org.hamcrest.Matchers.equalTo; + +public class NumericTermsAggregatorTests extends AggregatorTestCase { + private static final String LONG_FIELD = "long"; + + private static final List dataset; + static { + List d = new ArrayList<>(45); + for (int i = 0; i < 10; i++) { + for (int j = 0; j < i; j++) { + d.add((long) i); + } + } + dataset = d; + } + + public void testMatchNoDocs() throws IOException { + + testBothCases(new MatchNoDocsQuery(), dataset, + aggregation -> aggregation.field(LONG_FIELD), + agg -> assertEquals(0, agg.getBuckets().size()), null // without type hint + ); + + testBothCases(new MatchNoDocsQuery(), dataset, + aggregation -> aggregation.field(LONG_FIELD), + agg -> assertEquals(0, agg.getBuckets().size()), ValueType.NUMERIC // with type hint + ); + } + + public void testMatchAllDocs() throws IOException { + Query query = new MatchAllDocsQuery(); + + testBothCases(query, dataset, + aggregation -> aggregation.field(LONG_FIELD), + agg -> { + assertEquals(9, agg.getBuckets().size()); + for (int i = 0; i < 9; i++) { + LongTerms.Bucket bucket = (LongTerms.Bucket) agg.getBuckets().get(i); + assertThat(bucket.getKey(), equalTo(9L - i)); + assertThat(bucket.getDocCount(), equalTo(9L - i)); + } + }, null //without type hint + ); + + testBothCases(query, dataset, + aggregation -> aggregation.field(LONG_FIELD), + agg -> { + assertEquals(9, agg.getBuckets().size()); + for (int i = 0; i < 9; i++) { + LongTerms.Bucket bucket = (LongTerms.Bucket) agg.getBuckets().get(i); + assertThat(bucket.getKey(), equalTo(9L - i)); + assertThat(bucket.getDocCount(), equalTo(9L - i)); + } + }, ValueType.NUMERIC //with type hint + ); + } + + public void testBadIncludeExclude() throws IOException { + IncludeExclude includeExclude = new IncludeExclude(new RegExp("foo"), null); + + // Numerics don't support any regex include/exclude, so should fail no matter what we do + + AggregationExecutionException e = expectThrows(AggregationExecutionException.class, + () -> testBothCases(new MatchNoDocsQuery(), dataset, + aggregation -> aggregation.field(LONG_FIELD).includeExclude(includeExclude).format("yyyy-MM-dd"), + agg -> fail("test should have failed with exception"), null + )); + assertThat(e.getMessage(), equalTo("Aggregation [_name] cannot support regular expression style " + + "include/exclude settings as they can only be applied to string fields. Use an array of numeric " + + "values for include/exclude clauses used to filter numeric fields")); + + e = expectThrows(AggregationExecutionException.class, + () -> testBothCases(new MatchNoDocsQuery(), dataset, + aggregation -> aggregation.field(LONG_FIELD).includeExclude(includeExclude).format("yyyy-MM-dd"), + agg -> fail("test should have failed with exception"), ValueType.NUMERIC // with type hint + )); + assertThat(e.getMessage(), equalTo("Aggregation [_name] cannot support regular expression style " + + "include/exclude settings as they can only be applied to string fields. Use an array of numeric " + + "values for include/exclude clauses used to filter numeric fields")); + + } + + private void testSearchCase(Query query, List dataset, + Consumer configure, + Consumer verify, ValueType valueType) throws IOException { + executeTestCase(false, query, dataset, configure, verify, valueType); + } + + private void testSearchAndReduceCase(Query query, List dataset, + Consumer configure, + Consumer verify, ValueType valueType) throws IOException { + executeTestCase(true, query, dataset, configure, verify, valueType); + } + + private void testBothCases(Query query, List dataset, + Consumer configure, + Consumer verify, ValueType valueType) throws IOException { + testSearchCase(query, dataset, configure, verify, valueType); + testSearchAndReduceCase(query, dataset, configure, verify, valueType); + } + + private void executeTestCase(boolean reduced, Query query, List dataset, + Consumer configure, + Consumer verify, ValueType valueType) throws IOException { + + try (Directory directory = newDirectory()) { + try (RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory)) { + Document document = new Document(); + for (Long value : dataset) { + if (frequently()) { + indexWriter.commit(); + } + + document.add(new SortedNumericDocValuesField(LONG_FIELD, value)); + document.add(new LongPoint(LONG_FIELD, value)); + indexWriter.addDocument(document); + document.clear(); + } + } + + try (IndexReader indexReader = DirectoryReader.open(directory)) { + IndexSearcher indexSearcher = newIndexSearcher(indexReader); + + TermsAggregationBuilder aggregationBuilder = new TermsAggregationBuilder("_name"); + if (valueType != null) { + aggregationBuilder.userValueTypeHint(valueType); + } + if (configure != null) { + configure.accept(aggregationBuilder); + } + + MappedFieldType longFieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG); + longFieldType.setName(LONG_FIELD); + longFieldType.setHasDocValues(true); + + InternalMappedTerms rareTerms; + if (reduced) { + rareTerms = searchAndReduce(indexSearcher, query, aggregationBuilder, longFieldType); + } else { + rareTerms = search(indexSearcher, query, aggregationBuilder, longFieldType); + } + verify.accept(rareTerms); + } + } + } + +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java index b4aedb3061f0b..b2ce863a9df57 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java @@ -920,8 +920,7 @@ public void testRangeField() throws Exception { IndexSearcher indexSearcher = newIndexSearcher(indexReader); TermsAggregationBuilder aggregationBuilder = new TermsAggregationBuilder("_name") .field(fieldName); - // Note - other places we throw IllegalArgumentException - expectThrows(AggregationExecutionException.class, () -> { + expectThrows(IllegalArgumentException.class, () -> { createAggregator(aggregationBuilder, indexSearcher, fieldType); }); } @@ -944,8 +943,7 @@ public void testGeoPointField() throws Exception { IndexSearcher indexSearcher = newIndexSearcher(indexReader); TermsAggregationBuilder aggregationBuilder = new TermsAggregationBuilder("_name") .field(field); - // Note - other places we throw IllegalArgumentException - expectThrows(AggregationExecutionException.class, () -> { + expectThrows(IllegalArgumentException.class, () -> { createAggregator(aggregationBuilder, indexSearcher, fieldType); }); }