From 49ada121a280107799a218e59fc3f17fd2a5310d Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Thu, 16 Jan 2020 16:51:56 -0500 Subject: [PATCH 1/4] Rewire terms agg to use new registry --- .../elasticsearch/search/SearchModule.java | 3 +- .../bucket/terms/TermsAggregationBuilder.java | 9 + .../bucket/terms/TermsAggregatorFactory.java | 269 ++++++++++++------ .../support/TermsAggregatorSupplier.java | 49 ++++ .../terms/StringTermsAggregatorTests.java | 192 +++++++++++++ .../bucket/terms/TermsAggregatorTests.java | 6 +- 6 files changed, 440 insertions(+), 88 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/support/TermsAggregatorSupplier.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregatorTests.java 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 b8398a621e5f7..5fe7160d5b456 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 @@ -41,11 +41,13 @@ 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 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 { @@ -98,6 +100,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..dd82c78b81a98 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 @@ -23,6 +23,14 @@ import org.apache.lucene.search.IndexSearcher; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.logging.DeprecationLogger; +import org.elasticsearch.index.mapper.BinaryFieldMapper; +import org.elasticsearch.index.mapper.BooleanFieldMapper; +import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.index.mapper.IdFieldMapper; +import org.elasticsearch.index.mapper.IndexFieldMapper; +import org.elasticsearch.index.mapper.IpFieldMapper; +import org.elasticsearch.index.mapper.KeywordFieldMapper; +import org.elasticsearch.index.mapper.NumberFieldMapper; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.AggregationExecutionException; @@ -38,9 +46,13 @@ 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.TermsAggregatorSupplier; 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 +71,161 @@ 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, CoreValuesSourceType.BYTES, TermsAggregatorFactory.bytesSupplier(), + (fieldType, indexFieldData) -> fieldType.typeName().equals(KeywordFieldMapper.CONTENT_TYPE) + || fieldType.typeName().equals(BinaryFieldMapper.CONTENT_TYPE) + || fieldType.typeName().equals(IdFieldMapper.CONTENT_TYPE) + || fieldType.typeName().equals(IndexFieldMapper.CONTENT_TYPE) + ); + + valuesSourceRegistry.register(TermsAggregationBuilder.NAME, CoreValuesSourceType.IP, TermsAggregatorFactory.bytesSupplier(), + (fieldType, indexFieldData) -> fieldType.typeName().equals(IpFieldMapper.CONTENT_TYPE) + ); + + valuesSourceRegistry.register(TermsAggregationBuilder.NAME, CoreValuesSourceType.DATE, TermsAggregatorFactory.numericSupplier(), + (fieldType, indexFieldData) -> fieldType.typeName().equals(DateFieldMapper.CONTENT_TYPE) + ); + + valuesSourceRegistry.register(TermsAggregationBuilder.NAME, CoreValuesSourceType.BOOLEAN, TermsAggregatorFactory.numericSupplier(), + (fieldType, indexFieldData) -> fieldType.typeName().equals(BooleanFieldMapper.CONTENT_TYPE) + ); + + valuesSourceRegistry.register(TermsAggregationBuilder.NAME, CoreValuesSourceType.NUMERIC, TermsAggregatorFactory.numericSupplier(), + (fieldType, indexFieldData) -> { + for (NumberFieldMapper.NumberType type : NumberFieldMapper.NumberType.values()) { + if (fieldType.typeName().equals(type.typeName())) { + return true; + } + } + return false; + } + ); + } + + /** + * 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; + 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) { + 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,91 +269,39 @@ 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 // and the cardinality of the field static SubAggCollectionMode subAggCollectionMode(int expectedSize, long maxOrd) { @@ -217,7 +320,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/support/TermsAggregatorSupplier.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/TermsAggregatorSupplier.java new file mode 100644 index 0000000000000..bdba0b9ed66da --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/TermsAggregatorSupplier.java @@ -0,0 +1,49 @@ +/* + * 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.support; + +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.internal.SearchContext; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +public 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/StringTermsAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregatorTests.java new file mode 100644 index 0000000000000..5a48eecf75ea8 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregatorTests.java @@ -0,0 +1,192 @@ +/* + * 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.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.apache.lucene.util.automaton.RegExp; +import org.elasticsearch.index.mapper.KeywordFieldMapper; +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 StringTermsAggregatorTests extends AggregatorTestCase { + private static final String LONG_FIELD = "numeric"; + 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((long) i); + } + } + dataset = d; + } + + public void testMatchNoDocs() throws IOException { + testBothCases(new MatchNoDocsQuery(), dataset, + aggregation -> aggregation.field(KEYWORD_FIELD), + agg -> assertEquals(0, agg.getBuckets().size()), ValueType.STRING + ); + testBothCases(new MatchNoDocsQuery(), dataset, + aggregation -> aggregation.field(LONG_FIELD), + agg -> assertEquals(0, agg.getBuckets().size()), ValueType.NUMERIC + ); + } + + 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)); + } + }, ValueType.NUMERIC + ); + 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 + ); + } + + public void testBadIncludeExclude() throws IOException { + IncludeExclude includeExclude = new IncludeExclude(new RegExp("foo"), null); + + // Bytes fails if the formatter is not RAW. Note this is still on the long field, just hinted as string + 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"), ValueType.STRING + )); + 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")); + + // Numeric cannot use regex at all + e = expectThrows(AggregationExecutionException.class, () -> testBothCases(new MatchNoDocsQuery(), dataset, + aggregation -> aggregation.field(LONG_FIELD).includeExclude(includeExclude), + agg -> fail("test should have failed with exception"), ValueType.NUMERIC + )); + 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)); + document.add(new SortedSetDocValuesField(KEYWORD_FIELD, new BytesRef(Long.toString(value)))); + indexWriter.addDocument(document); + document.clear(); + } + } + + try (IndexReader indexReader = DirectoryReader.open(directory)) { + IndexSearcher indexSearcher = newIndexSearcher(indexReader); + + TermsAggregationBuilder aggregationBuilder = new TermsAggregationBuilder("_name"); + aggregationBuilder.userValueTypeHint(valueType); + if (configure != null) { + configure.accept(aggregationBuilder); + } + + MappedFieldType keywordFieldType = new KeywordFieldMapper.KeywordFieldType(); + keywordFieldType.setName(KEYWORD_FIELD); + keywordFieldType.setHasDocValues(true); + + MappedFieldType longFieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG); + longFieldType.setName(LONG_FIELD); + longFieldType.setHasDocValues(true); + + InternalMappedTerms rareTerms; + if (reduced) { + rareTerms = searchAndReduce(indexSearcher, query, aggregationBuilder, keywordFieldType, longFieldType); + } else { + rareTerms = search(indexSearcher, query, aggregationBuilder, keywordFieldType, 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); }); } From 681233464e2383705cc8b6186199caf9e0842d2b Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Tue, 21 Jan 2020 17:12:08 -0500 Subject: [PATCH 2/4] Fix tests - The only "bytes" that is even accepts a format is Binary, so split that into it's own test for include/exclude regex testing - Keyword test is now basically just match_all/match_none boilerplate, waiting for some industrious soul to fill it out later - Numeric has include/exclude tests for all formatting - with and without user supplied type hints --- .../bucket/terms/TermsAggregatorFactory.java | 1 - .../terms/BinaryTermsAggregatorTests.java | 176 ++++++++++++++++++ .../terms/KeywordTermsAggregatorTests.java | 166 +++++++++++++++++ ....java => NumericTermsAggregatorTests.java} | 61 +++--- 4 files changed, 373 insertions(+), 31 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/BinaryTermsAggregatorTests.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/KeywordTermsAggregatorTests.java rename server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/{StringTermsAggregatorTests.java => NumericTermsAggregatorTests.java} (79%) 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 dd82c78b81a98..638d82e2823db 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 @@ -301,7 +301,6 @@ protected Aggregator doCreateInternal(ValuesSource valuesSource, showTermDocCountError, pipelineAggregators, metaData); } - // return the SubAggCollectionMode that this aggregation should use based on the expected size // and the cardinality of the field static SubAggCollectionMode subAggCollectionMode(int expectedSize, long maxOrd) { 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..06519ff399944 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/KeywordTermsAggregatorTests.java @@ -0,0 +1,166 @@ +/* + * 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.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.apache.lucene.util.automaton.RegExp; +import org.elasticsearch.index.mapper.KeywordFieldMapper; +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 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/StringTermsAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/NumericTermsAggregatorTests.java similarity index 79% rename from server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregatorTests.java rename to server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/NumericTermsAggregatorTests.java index 5a48eecf75ea8..9f09bda6d412b 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/NumericTermsAggregatorTests.java @@ -46,9 +46,8 @@ import static org.hamcrest.Matchers.equalTo; -public class StringTermsAggregatorTests extends AggregatorTestCase { - private static final String LONG_FIELD = "numeric"; - private static final String KEYWORD_FIELD = "keyword"; +public class NumericTermsAggregatorTests extends AggregatorTestCase { + private static final String LONG_FIELD = "long"; private static final List dataset; static { @@ -62,13 +61,15 @@ public class StringTermsAggregatorTests extends AggregatorTestCase { } public void testMatchNoDocs() throws IOException { + testBothCases(new MatchNoDocsQuery(), dataset, - aggregation -> aggregation.field(KEYWORD_FIELD), - agg -> assertEquals(0, agg.getBuckets().size()), ValueType.STRING + 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 + agg -> assertEquals(0, agg.getBuckets().size()), ValueType.NUMERIC // with type hint ); } @@ -84,42 +85,45 @@ public void testMatchAllDocs() throws IOException { assertThat(bucket.getKey(), equalTo(9L - i)); assertThat(bucket.getDocCount(), equalTo(9L - i)); } - }, ValueType.NUMERIC + }, null //without type hint ); + testBothCases(query, dataset, - aggregation -> aggregation.field(KEYWORD_FIELD), + aggregation -> aggregation.field(LONG_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))); + LongTerms.Bucket bucket = (LongTerms.Bucket) agg.getBuckets().get(i); + assertThat(bucket.getKey(), equalTo(9L - i)); assertThat(bucket.getDocCount(), equalTo(9L - i)); } - }, ValueType.STRING + }, ValueType.NUMERIC //with type hint ); } public void testBadIncludeExclude() throws IOException { IncludeExclude includeExclude = new IncludeExclude(new RegExp("foo"), null); - // Bytes fails if the formatter is not RAW. Note this is still on the long field, just hinted as string + // 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"), ValueType.STRING - )); + 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")); - // Numeric cannot use regex at all - e = expectThrows(AggregationExecutionException.class, () -> testBothCases(new MatchNoDocsQuery(), dataset, - aggregation -> aggregation.field(LONG_FIELD).includeExclude(includeExclude), - agg -> fail("test should have failed with exception"), ValueType.NUMERIC - )); - 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, @@ -155,7 +159,6 @@ private void executeTestCase(boolean reduced, Query query, List dataset, document.add(new SortedNumericDocValuesField(LONG_FIELD, value)); document.add(new LongPoint(LONG_FIELD, value)); - document.add(new SortedSetDocValuesField(KEYWORD_FIELD, new BytesRef(Long.toString(value)))); indexWriter.addDocument(document); document.clear(); } @@ -165,24 +168,22 @@ private void executeTestCase(boolean reduced, Query query, List dataset, IndexSearcher indexSearcher = newIndexSearcher(indexReader); TermsAggregationBuilder aggregationBuilder = new TermsAggregationBuilder("_name"); - aggregationBuilder.userValueTypeHint(valueType); + if (valueType != null) { + aggregationBuilder.userValueTypeHint(valueType); + } if (configure != null) { configure.accept(aggregationBuilder); } - MappedFieldType keywordFieldType = new KeywordFieldMapper.KeywordFieldType(); - keywordFieldType.setName(KEYWORD_FIELD); - keywordFieldType.setHasDocValues(true); - MappedFieldType longFieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG); longFieldType.setName(LONG_FIELD); longFieldType.setHasDocValues(true); InternalMappedTerms rareTerms; if (reduced) { - rareTerms = searchAndReduce(indexSearcher, query, aggregationBuilder, keywordFieldType, longFieldType); + rareTerms = searchAndReduce(indexSearcher, query, aggregationBuilder, longFieldType); } else { - rareTerms = search(indexSearcher, query, aggregationBuilder, keywordFieldType, longFieldType); + rareTerms = search(indexSearcher, query, aggregationBuilder, longFieldType); } verify.accept(rareTerms); } From 72a8d40bea1cb0e149a2bf4a3cf4a7eaa24bfa56 Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Tue, 28 Jan 2020 12:57:20 -0500 Subject: [PATCH 3/4] Address review, fix merge conflicts - Removes the need to whitelist due to merging in feature branch - adds some TODOs - Moves the Supplier into the terms package, makes it package-private --- .../bucket/terms/TermsAggregatorFactory.java | 47 ++++--------------- .../terms}/TermsAggregatorSupplier.java | 6 ++- 2 files changed, 13 insertions(+), 40 deletions(-) rename server/src/main/java/org/elasticsearch/search/aggregations/{support => bucket/terms}/TermsAggregatorSupplier.java (88%) 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 638d82e2823db..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 @@ -23,14 +23,6 @@ import org.apache.lucene.search.IndexSearcher; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.logging.DeprecationLogger; -import org.elasticsearch.index.mapper.BinaryFieldMapper; -import org.elasticsearch.index.mapper.BooleanFieldMapper; -import org.elasticsearch.index.mapper.DateFieldMapper; -import org.elasticsearch.index.mapper.IdFieldMapper; -import org.elasticsearch.index.mapper.IndexFieldMapper; -import org.elasticsearch.index.mapper.IpFieldMapper; -import org.elasticsearch.index.mapper.KeywordFieldMapper; -import org.elasticsearch.index.mapper.NumberFieldMapper; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.AggregationExecutionException; @@ -48,7 +40,6 @@ 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.TermsAggregatorSupplier; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; @@ -73,35 +64,13 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory { // TODO: Registration should happen on the actual aggregator classes static void registerAggregators(ValuesSourceRegistry valuesSourceRegistry) { - valuesSourceRegistry.register(TermsAggregationBuilder.NAME, CoreValuesSourceType.BYTES, TermsAggregatorFactory.bytesSupplier(), - (fieldType, indexFieldData) -> fieldType.typeName().equals(KeywordFieldMapper.CONTENT_TYPE) - || fieldType.typeName().equals(BinaryFieldMapper.CONTENT_TYPE) - || fieldType.typeName().equals(IdFieldMapper.CONTENT_TYPE) - || fieldType.typeName().equals(IndexFieldMapper.CONTENT_TYPE) - ); - - valuesSourceRegistry.register(TermsAggregationBuilder.NAME, CoreValuesSourceType.IP, TermsAggregatorFactory.bytesSupplier(), - (fieldType, indexFieldData) -> fieldType.typeName().equals(IpFieldMapper.CONTENT_TYPE) - ); - - valuesSourceRegistry.register(TermsAggregationBuilder.NAME, CoreValuesSourceType.DATE, TermsAggregatorFactory.numericSupplier(), - (fieldType, indexFieldData) -> fieldType.typeName().equals(DateFieldMapper.CONTENT_TYPE) - ); - - valuesSourceRegistry.register(TermsAggregationBuilder.NAME, CoreValuesSourceType.BOOLEAN, TermsAggregatorFactory.numericSupplier(), - (fieldType, indexFieldData) -> fieldType.typeName().equals(BooleanFieldMapper.CONTENT_TYPE) - ); - - valuesSourceRegistry.register(TermsAggregationBuilder.NAME, CoreValuesSourceType.NUMERIC, TermsAggregatorFactory.numericSupplier(), - (fieldType, indexFieldData) -> { - for (NumberFieldMapper.NumberType type : NumberFieldMapper.NumberType.values()) { - if (fieldType.typeName().equals(type.typeName())) { - return true; - } - } - return false; - } - ); + 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()); } /** @@ -140,6 +109,7 @@ public Aggregator build(String name, } if (subAggCollectMode == null) { subAggCollectMode = SubAggCollectionMode.DEPTH_FIRST; + // TODO can we remove concept of AggregatorFactories.EMPTY? if (factories != AggregatorFactories.EMPTY) { subAggCollectMode = subAggCollectionMode(bucketCountThresholds.getShardSize(), maxOrd); } @@ -190,6 +160,7 @@ public Aggregator build(String name, 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 { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/TermsAggregatorSupplier.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorSupplier.java similarity index 88% rename from server/src/main/java/org/elasticsearch/search/aggregations/support/TermsAggregatorSupplier.java rename to server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorSupplier.java index bdba0b9ed66da..ccfb082ac98ce 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/TermsAggregatorSupplier.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorSupplier.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.search.aggregations.support; +package org.elasticsearch.search.aggregations.bucket.terms; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.Aggregator; @@ -25,13 +25,15 @@ 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; -public interface TermsAggregatorSupplier extends AggregatorSupplier { +interface TermsAggregatorSupplier extends AggregatorSupplier { Aggregator build(String name, AggregatorFactories factories, ValuesSource valuesSource, From 4ae490591070f763190872e287ad024016e2d44a Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Tue, 28 Jan 2020 15:04:27 -0500 Subject: [PATCH 4/4] checkstyle --- .../bucket/terms/KeywordTermsAggregatorTests.java | 5 ----- .../bucket/terms/NumericTermsAggregatorTests.java | 3 --- 2 files changed, 8 deletions(-) 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 index 06519ff399944..098e94d3d4450 100644 --- 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 @@ -19,8 +19,6 @@ 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.document.SortedSetDocValuesField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; @@ -31,11 +29,8 @@ 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.index.mapper.KeywordFieldMapper; 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; 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 index 9f09bda6d412b..b0790b9fa0413 100644 --- 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 @@ -21,7 +21,6 @@ import org.apache.lucene.document.Document; import org.apache.lucene.document.LongPoint; import org.apache.lucene.document.SortedNumericDocValuesField; -import org.apache.lucene.document.SortedSetDocValuesField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.RandomIndexWriter; @@ -30,9 +29,7 @@ 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.index.mapper.KeywordFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.NumberFieldMapper; import org.elasticsearch.search.aggregations.AggregationExecutionException;