diff --git a/docs/changelog/99963.yaml b/docs/changelog/99963.yaml new file mode 100644 index 0000000000000..4f03dceeb22aa --- /dev/null +++ b/docs/changelog/99963.yaml @@ -0,0 +1,5 @@ +pr: 99963 +summary: Aggs error codes part 1 +area: Aggregations +type: bug +issues: [] diff --git a/modules/aggregations/src/main/java/org/elasticsearch/aggregations/metric/MatrixStatsAggregatorFactory.java b/modules/aggregations/src/main/java/org/elasticsearch/aggregations/metric/MatrixStatsAggregatorFactory.java index 2418aada6c260..8802e8d53497d 100644 --- a/modules/aggregations/src/main/java/org/elasticsearch/aggregations/metric/MatrixStatsAggregatorFactory.java +++ b/modules/aggregations/src/main/java/org/elasticsearch/aggregations/metric/MatrixStatsAggregatorFactory.java @@ -9,7 +9,7 @@ import org.elasticsearch.common.util.Maps; import org.elasticsearch.search.MultiValueMode; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactory; @@ -53,9 +53,7 @@ protected Aggregator doCreateInternal( Map typedValuesSources = Maps.newMapWithExpectedSize(valuesSources.size()); for (Map.Entry entry : valuesSources.entrySet()) { if (entry.getValue() instanceof ValuesSource.Numeric == false) { - throw new AggregationExecutionException( - "ValuesSource type [" + entry.getValue().toString() + "] is not supported for aggregation [" + this.name() + "]" - ); + throw AggregationErrors.unsupportedValuesSourceType(entry.getValue(), this.name()); } // TODO: There must be a better option than this. typedValuesSources.put(entry.getKey(), (ValuesSource.Numeric) entry.getValue()); diff --git a/modules/aggregations/src/yamlRestTest/resources/rest-api-spec/test/aggregations/terms.yml b/modules/aggregations/src/yamlRestTest/resources/rest-api-spec/test/aggregations/terms.yml index faedbbc997e11..7deee31920cc8 100644 --- a/modules/aggregations/src/yamlRestTest/resources/rest-api-spec/test/aggregations/terms.yml +++ b/modules/aggregations/src/yamlRestTest/resources/rest-api-spec/test/aggregations/terms.yml @@ -79,6 +79,10 @@ setup: --- "IP test": + - skip: + version: " - 8.11.99" + reason: "Changed rest status for error in 8.12" + - do: index: index: test_1 @@ -144,7 +148,7 @@ setup: - match: { aggregations.ip_terms.buckets.0.key: "::1" } - do: - catch: request + catch: bad_request search: rest_total_hits_as_int: true index: test_1 diff --git a/modules/aggregations/src/yamlRestTest/resources/rest-api-spec/test/aggregations/weighted_avg.yml b/modules/aggregations/src/yamlRestTest/resources/rest-api-spec/test/aggregations/weighted_avg.yml index e988f62082565..3563e0da18e59 100644 --- a/modules/aggregations/src/yamlRestTest/resources/rest-api-spec/test/aggregations/weighted_avg.yml +++ b/modules/aggregations/src/yamlRestTest/resources/rest-api-spec/test/aggregations/weighted_avg.yml @@ -184,9 +184,11 @@ setup: --- "Multi weight field not allowed": - skip: + version: " - 8.11.99" + reason: "Changed status code in 8.12" features: close_to - do: - catch: request + catch: bad_request search: body: aggs: diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregatorFactory.java b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregatorFactory.java index fc797be044513..05316353387f5 100644 --- a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregatorFactory.java +++ b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregatorFactory.java @@ -9,7 +9,7 @@ package org.elasticsearch.join.aggregations; import org.apache.lucene.search.Query; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactory; @@ -63,9 +63,7 @@ protected Aggregator doCreateInternal(Aggregator parent, CardinalityUpperBound c throws IOException { ValuesSource rawValuesSource = config.getValuesSource(); if (rawValuesSource instanceof WithOrdinals == false) { - throw new AggregationExecutionException( - "ValuesSource type " + rawValuesSource.toString() + "is not supported for aggregation " + this.name() - ); + throw AggregationErrors.unsupportedValuesSourceType(rawValuesSource, this.name()); } WithOrdinals valuesSource = (WithOrdinals) rawValuesSource; long maxOrd = valuesSource.globalMaxOrd(context.searcher().getIndexReader()); diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentAggregatorFactory.java b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentAggregatorFactory.java index f531ac67b98f3..89c5b40fee9a7 100644 --- a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentAggregatorFactory.java +++ b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentAggregatorFactory.java @@ -9,7 +9,7 @@ package org.elasticsearch.join.aggregations; import org.apache.lucene.search.Query; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactory; @@ -63,9 +63,7 @@ protected Aggregator doCreateInternal(Aggregator children, CardinalityUpperBound throws IOException { ValuesSource rawValuesSource = config.getValuesSource(); if (rawValuesSource instanceof WithOrdinals == false) { - throw new AggregationExecutionException( - "ValuesSource type " + rawValuesSource.toString() + "is not supported for aggregation " + this.name() - ); + throw AggregationErrors.unsupportedValuesSourceType(rawValuesSource, this.name()); } WithOrdinals valuesSource = (WithOrdinals) rawValuesSource; long maxOrd = valuesSource.globalMaxOrd(context.searcher().getIndexReader()); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/pipeline/BucketScriptIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/pipeline/BucketScriptIT.java index eb606f6746c88..5c4caf5f242a6 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/pipeline/BucketScriptIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/pipeline/BucketScriptIT.java @@ -17,7 +17,6 @@ import org.elasticsearch.script.MockScriptPlugin; import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptType; -import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; import org.elasticsearch.search.aggregations.bucket.range.Range; import org.elasticsearch.search.aggregations.metrics.Percentiles; @@ -856,10 +855,10 @@ public void testInlineScriptWithMultiValueAggregationIllegalBucketsPaths() { } else if (cause instanceof SearchPhaseExecutionException) { SearchPhaseExecutionException spee = (SearchPhaseExecutionException) e; Throwable rootCause = spee.getRootCause(); - if ((rootCause instanceof AggregationExecutionException) == false) { + if ((rootCause instanceof IllegalArgumentException) == false) { throw e; } - } else if ((cause instanceof AggregationExecutionException) == false) { + } else if ((cause instanceof IllegalArgumentException) == false) { throw e; } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationErrors.java b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationErrors.java index 9a5395d2dc476..5ff0c90687adf 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationErrors.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationErrors.java @@ -8,10 +8,16 @@ package org.elasticsearch.search.aggregations; +import org.elasticsearch.search.aggregations.support.ValuesSource; + +import java.util.Optional; + /** * Collection of helper methods for what to throw in common aggregation error scenarios. */ public class AggregationErrors { + private static String parentType; + private AggregationErrors() {} /** @@ -21,4 +27,89 @@ private AggregationErrors() {} public static IllegalArgumentException invalidPathElement(String element, String path) { return new IllegalArgumentException("Invalid path element [" + element + "] in path [" + path + "]"); } + + /** + * This error indicates that an aggregation is being used on a value type that it is not capable of processing, such as taking + * the sum of a keyword. It is a 400 class error and should not be retried. + * + * This is a legacy version of this error; in general, we should aim to use the + * {@link org.elasticsearch.search.aggregations.support.ValuesSourceType} version below + * + * @param valuesSource The values source we resolved from the query + * @param name The name of the aggregation + * @return an appropriate exception type + */ + public static IllegalArgumentException unsupportedValuesSourceType(ValuesSource valuesSource, String name) { + return new IllegalArgumentException( + "ValuesSource type [" + valuesSource.toString() + "] is not supported for aggregation [" + name + "]" + ); + } + + /** + * This error indicates that a rate aggregation is being invoked without a single Date Histogram parent, as is required. This is a + * 400 class error and should not be retried. + * + * @param name the name of the rate aggregation + * @return an appropriate exception + */ + public static RuntimeException rateWithoutDateHistogram(String name) { + return new IllegalArgumentException( + "aggregation [" + + name + + "] does not have exactly one date_histogram value source; exactly one is required when using with rate aggregation" + ); + } + + /** + * This error indicates that the backing indices for a field have different, incompatible, types (e.g. IP and Keyword). This + * causes a failure at reduce time, and is not retryable (and thus should be a 400 class error) + * + * @param aggregationName - The name of the aggregation + * @param position - optional, for multisource aggregations. Indicates the position of the field causing the problem. + * @return - an appropriate exception + */ + public static RuntimeException reduceTypeMissmatch(String aggregationName, Optional position) { + String fieldString; + if (position.isPresent()) { + fieldString = "the field in position" + position.get().toString(); + } else { + fieldString = "the field you gave"; + } + return new IllegalArgumentException( + "Merging/Reducing the aggregations failed when computing the aggregation [" + + aggregationName + + "] because " + + fieldString + + " in the aggregation query existed as two different " + + "types in two different indices" + ); + } + + public static RuntimeException valuesSourceDoesNotSupportScritps(String typeName) { + return new IllegalArgumentException("value source of type [" + typeName + "] is not supported by scripts"); + } + + public static RuntimeException unsupportedScriptValue(String actual) { + return new IllegalArgumentException("Unsupported script value [" + actual + "], expected a number, date, or boolean"); + } + + /** + * Indicates that a multivalued field was found where we only support a single valued field + * @return an appropriate exception + */ + public static RuntimeException unsupportedMultivalue() { + return new IllegalArgumentException( + "Encountered more than one value for a " + + "single document. Use a script to combine multiple values per doc into a single value." + ); + } + + /** + * Indicates an attempt to use date rounding on a non-date values source + * @param typeName - name of the type we're attempting to round + * @return an appropriate exception + */ + public static RuntimeException unsupportedRounding(String typeName) { + return new IllegalArgumentException("can't round a [" + typeName + "]"); + } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java index 1094db38cb14d..c9c30914da33a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java @@ -40,8 +40,8 @@ import org.elasticsearch.logging.Logger; import org.elasticsearch.lucene.queries.SearchAfterSortedDocQuery; import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.aggregations.AggregationExecutionContext; -import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.BucketCollector; @@ -610,11 +610,7 @@ public void collect(int doc, long zeroBucket) throws IOException { @Override public double bucketSize(long bucket, Rounding.DateTimeUnit unit) { if (innerSizedBucketAggregators.length != 1) { - throw new AggregationExecutionException( - "aggregation [" - + name() - + "] does not have exactly one date_histogram value source; exactly one is required when using with rate aggregation" - ); + throw AggregationErrors.rateWithoutDateHistogram(name()); } return innerSizedBucketAggregators[0].bucketSize(bucket, unit); } @@ -622,11 +618,7 @@ public double bucketSize(long bucket, Rounding.DateTimeUnit unit) { @Override public double bucketSize(Rounding.DateTimeUnit unit) { if (innerSizedBucketAggregators.length != 1) { - throw new AggregationExecutionException( - "aggregation [" - + name() - + "] does not have exactly one date_histogram value source; exactly one is required when using with rate aggregation" - ); + throw AggregationErrors.rateWithoutDateHistogram(name()); } return innerSizedBucketAggregators[0].bucketSize(unit); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregatorFactory.java index cd9a79a2c0cff..2d63d3727c3e7 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregatorFactory.java @@ -29,7 +29,7 @@ public GlobalAggregatorFactory( ) throws IOException { super(name, context, parent, subFactories, metadata); if (subFactories.isInSortOrderExecutionRequired()) { - throw new AggregationExecutionException("Time series aggregations cannot be used inside global aggregation."); + throw new IllegalArgumentException("Time series aggregations cannot be used inside global aggregation."); } } @@ -37,7 +37,7 @@ public GlobalAggregatorFactory( public Aggregator createInternal(Aggregator parent, CardinalityUpperBound cardinality, Map metadata) throws IOException { if (parent != null) { - throw new AggregationExecutionException( + throw new IllegalArgumentException( "Aggregation [" + parent.name() + "] cannot have a global " diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java index b5d7a45e9367f..538be7de0a022 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java @@ -13,7 +13,6 @@ import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.core.Releasables; import org.elasticsearch.search.aggregations.AggregationExecutionContext; -import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.InternalAggregation; @@ -232,7 +231,7 @@ public InternalAggregation buildEmptyAggregation() { @Override protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException { if (bdd == null) { - throw new AggregationExecutionException("Sampler aggregation must be used with child aggregations."); + throw new IllegalArgumentException("Sampler aggregation must be used with child aggregations."); } return bdd.getLeafCollector(aggCtx); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractInternalTerms.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractInternalTerms.java index d055802761243..f3c7430283112 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractInternalTerms.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractInternalTerms.java @@ -10,7 +10,7 @@ import org.apache.lucene.util.PriorityQueue; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.aggregations.AggregationReduceContext; import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.DelayedBucket; @@ -29,6 +29,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.function.Function; import static org.elasticsearch.search.aggregations.InternalOrder.isKeyAsc; @@ -269,12 +270,7 @@ public InternalAggregation reduce(List aggregations, Aggreg if (referenceTerms != null && referenceTerms.getClass().equals(terms.getClass()) == false && terms.canLeadReduction()) { // control gets into this loop when the same field name against which the query is executed // is of different types in different indices. - throw new AggregationExecutionException( - "Merging/Reducing the aggregations failed when computing the aggregation [" - + referenceTerms.getName() - + "] because the field you gave in the aggregation query existed as two different " - + "types in two different indices" - ); + throw AggregationErrors.reduceTypeMissmatch(referenceTerms.getName(), Optional.empty()); } otherDocCount[0] += terms.getSumOfOtherDocCounts(); final long thisAggDocCountError = getDocCountError(terms); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalMappedRareTerms.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalMappedRareTerms.java index 61f15978bc154..23c773d764920 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalMappedRareTerms.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalMappedRareTerms.java @@ -15,7 +15,7 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.util.SetBackedScalingCuckooFilter; import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.aggregations.AggregationReduceContext; import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.InternalAggregation; @@ -29,6 +29,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.function.Function; import java.util.stream.Collectors; @@ -106,12 +107,7 @@ public InternalAggregation reduce(List aggregations, Aggreg && terms.getClass().equals(UnmappedRareTerms.class) == false) { // control gets into this loop when the same field name against which the query is executed // is of different types in different indices. - throw new AggregationExecutionException( - "Merging/Reducing the aggregations failed when computing the aggregation [" - + referenceTerms.getName() - + "] because the field you gave in the aggregation query existed as two different " - + "types in two different indices" - ); + throw AggregationErrors.reduceTypeMissmatch(referenceTerms.getName(), Optional.empty()); } for (B bucket : terms.getBuckets()) { List bucketList = buckets.computeIfAbsent(bucket.getKey(), k -> new ArrayList<>()); 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 e13ee43df5746..31c6a4a7e0430 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 @@ -17,7 +17,6 @@ import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode; import org.elasticsearch.search.aggregations.AggregatorFactories; @@ -121,7 +120,7 @@ private static TermsAggregatorSupplier bytesSupplier() { if ((includeExclude != null) && (includeExclude.isRegexBased()) && valuesSourceConfig.format() != DocValueFormat.RAW) { // TODO this exception message is not really accurate for the string case. It's really disallowing regex + formatter - throw new AggregationExecutionException( + throw new IllegalArgumentException( "Aggregation [" + name + "] cannot support regular expression style " @@ -170,7 +169,7 @@ private static TermsAggregatorSupplier numericSupplier() { metadata) -> { if ((includeExclude != null) && (includeExclude.isRegexBased())) { - throw new AggregationExecutionException( + throw new IllegalArgumentException( "Aggregation [" + name + "] cannot support regular expression style " diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregatorFactory.java index 0549dac2b964b..6962bf50da73d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregatorFactory.java @@ -8,7 +8,6 @@ package org.elasticsearch.search.aggregations.metrics; -import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactory; @@ -67,7 +66,7 @@ class TopHitsAggregatorFactory extends AggregatorFactory { ) throws IOException { super(name, context, parent, subFactories, metadata); if (context.isInSortOrderExecutionRequired()) { - throw new AggregationExecutionException("Top hits aggregations cannot be used together with time series aggregations"); + throw new IllegalArgumentException("Top hits aggregations cannot be used together with time series aggregations"); } this.from = from; this.size = size; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregator.java index 7409f10c3983a..1a1ffc46946eb 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregator.java @@ -13,7 +13,6 @@ import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.AggregationExecutionContext; -import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; @@ -77,7 +76,7 @@ public void collect(int doc, long bucket) throws IOException { if (docValues.advanceExact(doc) && docWeights.advanceExact(doc)) { if (docWeights.docValueCount() > 1) { - throw new AggregationExecutionException( + throw new IllegalArgumentException( "Encountered more than one weight for a " + "single document. Use a script to combine multiple weights-per-doc into a single value." ); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpers.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpers.java index d1a10f07641f8..b4c2a3a7cb370 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpers.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpers.java @@ -13,7 +13,6 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; -import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; import org.elasticsearch.search.aggregations.InvalidAggregationPathException; import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; @@ -198,7 +197,7 @@ public static Double resolveBucketValue( Object propertyValue = bucket.getProperty(agg.getName(), aggPathAsList); if (propertyValue == null) { - throw new AggregationExecutionException( + throw new IllegalArgumentException( AbstractPipelineAggregationBuilder.BUCKETS_PATH_FIELD.getPreferredName() + " must reference either a number value or a single value numeric metric aggregation" ); @@ -233,11 +232,7 @@ public static Double resolveBucketValue( /** * Inspects where we are in the agg tree and tries to format a helpful error */ - private static AggregationExecutionException formatResolutionError( - MultiBucketsAggregation agg, - List aggPathAsList, - Object propertyValue - ) { + private static RuntimeException formatResolutionError(MultiBucketsAggregation agg, List aggPathAsList, Object propertyValue) { String currentAggName; Object currentAgg; if (aggPathAsList.isEmpty()) { @@ -248,14 +243,14 @@ private static AggregationExecutionException formatResolutionError( currentAgg = propertyValue; } if (currentAgg instanceof InternalNumericMetricsAggregation.MultiValue) { - return new AggregationExecutionException( + return new IllegalArgumentException( AbstractPipelineAggregationBuilder.BUCKETS_PATH_FIELD.getPreferredName() + " must reference either a number value or a single value numeric metric aggregation, but [" + currentAggName + "] contains multiple values. Please specify which to use." ); } else { - return new AggregationExecutionException( + return new IllegalArgumentException( AbstractPipelineAggregationBuilder.BUCKETS_PATH_FIELD.getPreferredName() + " must reference either a number value or a single value numeric metric aggregation, got: [" + propertyValue.getClass().getSimpleName() diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/CoreValuesSourceType.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/CoreValuesSourceType.java index 7369fd9bc5b6f..fd4fbac82334e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/CoreValuesSourceType.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/CoreValuesSourceType.java @@ -31,7 +31,7 @@ import org.elasticsearch.index.mapper.RangeFieldMapper; import org.elasticsearch.script.AggregationScript; import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import java.io.IOException; import java.time.ZoneId; @@ -156,7 +156,7 @@ public ValuesSource getEmpty() { @Override public ValuesSource getScript(AggregationScript.LeafFactory script, ValueType scriptValueType) { - throw new AggregationExecutionException("value source of type [" + this.value() + "] is not supported by scripts"); + throw AggregationErrors.valuesSourceDoesNotSupportScritps(this.value()); } @Override @@ -194,7 +194,7 @@ public ValuesSource getEmpty() { @Override public ValuesSource getScript(AggregationScript.LeafFactory script, ValueType scriptValueType) { - throw new AggregationExecutionException("value source of type [" + this.value() + "] is not supported by scripts"); + throw AggregationErrors.valuesSourceDoesNotSupportScritps(this.value()); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java index 5a9367ec058d0..ca76f27ef7471 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java @@ -10,7 +10,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.elasticsearch.common.util.Maps; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import java.io.IOException; import java.util.Map; @@ -28,9 +28,7 @@ public NumericMultiValuesSource(Map valuesSourceConf for (Map.Entry entry : valuesSourceConfigs.entrySet()) { final ValuesSource valuesSource = entry.getValue().getValuesSource(); if (valuesSource instanceof ValuesSource.Numeric == false) { - throw new AggregationExecutionException( - "ValuesSource type " + valuesSource.toString() + "is not supported for multi-valued aggregation" - ); + throw AggregationErrors.unsupportedValuesSourceType(valuesSource, "multi-value"); } values.put(entry.getKey(), (ValuesSource.Numeric) valuesSource); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/TimeSeriesValuesSourceType.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/TimeSeriesValuesSourceType.java index d3a7b05919e1b..ae720528b680f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/TimeSeriesValuesSourceType.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/TimeSeriesValuesSourceType.java @@ -11,7 +11,7 @@ import org.elasticsearch.index.fielddata.IndexNumericFieldData; import org.elasticsearch.script.AggregationScript; import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import java.util.Locale; @@ -64,7 +64,7 @@ public ValuesSource getEmpty() { @Override public ValuesSource getScript(AggregationScript.LeafFactory script, ValueType scriptValueType) { - throw new AggregationExecutionException("value source of type [" + this.value() + "] is not supported by scripts"); + throw AggregationErrors.valuesSourceDoesNotSupportScritps(this.value()); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSource.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSource.java index ca9fa237f7fc6..9903232e5cd19 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSource.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSource.java @@ -35,7 +35,7 @@ import org.elasticsearch.index.mapper.RangeType; import org.elasticsearch.script.AggregationScript; import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.bucket.geogrid.GeoTileCellIdSource; import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator; @@ -116,7 +116,7 @@ public DocValueBits docsWithValue(LeafReaderContext context) throws IOException @Override public final Function roundingPreparer(AggregationContext context) throws IOException { - throw new AggregationExecutionException("can't round a [BYTES]"); + throw AggregationErrors.unsupportedRounding("BYTES"); } /** @@ -723,7 +723,7 @@ public DocValueBits docsWithValue(LeafReaderContext context) throws IOException @Override public final Function roundingPreparer(AggregationContext context) throws IOException { - throw new AggregationExecutionException("can't round a [GEO_POINT]"); + throw AggregationErrors.unsupportedRounding("GEO_POINT"); } /** diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/values/ScriptDoubleValues.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/values/ScriptDoubleValues.java index 1baaa433dc7e4..0e122162e5e87 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/values/ScriptDoubleValues.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/values/ScriptDoubleValues.java @@ -11,7 +11,7 @@ import org.elasticsearch.common.lucene.ScorerAware; import org.elasticsearch.index.fielddata.SortingNumericDoubleValues; import org.elasticsearch.script.AggregationScript; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import java.io.IOException; import java.lang.reflect.Array; @@ -83,7 +83,7 @@ private static double toDoubleValue(Object o) { // make sure of using true/false in the key_as_string field return ((Boolean) o).booleanValue() ? 1.0 : 0.0; } else { - throw new AggregationExecutionException("Unsupported script value [" + o + "], expected a number, date, or boolean"); + throw AggregationErrors.unsupportedScriptValue(o == null ? "null" : o.toString()); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/values/ScriptLongValues.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/values/ScriptLongValues.java index 6a3c143722bf6..f702be71c49f3 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/values/ScriptLongValues.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/values/ScriptLongValues.java @@ -12,7 +12,7 @@ import org.elasticsearch.common.lucene.ScorerAware; import org.elasticsearch.index.fielddata.AbstractSortingNumericDocValues; import org.elasticsearch.script.AggregationScript; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import java.io.IOException; import java.lang.reflect.Array; @@ -85,7 +85,7 @@ private static long toLongValue(Object o) { // make sure of using true/false in the key_as_string field return ((Boolean) o).booleanValue() ? 1L : 0L; } else { - throw new AggregationExecutionException("Unsupported script value [" + o + "], expected a number, date, or boolean"); + throw AggregationErrors.unsupportedScriptValue(o == null ? "null" : o.toString()); } } 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 index 368a0197ab137..0441151e174ce 100644 --- 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 @@ -19,7 +19,6 @@ 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; @@ -73,8 +72,8 @@ public void testBadIncludeExclude() throws IOException { IncludeExclude includeExclude = new IncludeExclude("foo", null, null, null); // Make sure the include/exclude fails regardless of how the user tries to type hint the agg - AggregationExecutionException e = expectThrows( - AggregationExecutionException.class, + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, () -> testSearchCase( new MatchNoDocsQuery(), dataset, @@ -92,7 +91,7 @@ public void testBadIncludeExclude() throws IOException { ); e = expectThrows( - AggregationExecutionException.class, + IllegalArgumentException.class, () -> testSearchCase( new MatchNoDocsQuery(), dataset, 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 a48c764a94eb1..8188af7482576 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 @@ -18,7 +18,6 @@ import org.apache.lucene.tests.index.RandomIndexWriter; 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; @@ -93,8 +92,8 @@ public void testBadIncludeExclude() throws IOException { // Numerics don't support any regex include/exclude, so should fail no matter what we do - AggregationExecutionException e = expectThrows( - AggregationExecutionException.class, + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, () -> testSearchCase( new MatchNoDocsQuery(), dataset, @@ -113,7 +112,7 @@ public void testBadIncludeExclude() throws IOException { ); e = expectThrows( - AggregationExecutionException.class, + IllegalArgumentException.class, () -> testSearchCase( new MatchNoDocsQuery(), dataset, diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregatorTests.java index e974144395a62..2a2fbe8e37523 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregatorTests.java @@ -19,7 +19,6 @@ import org.elasticsearch.core.CheckedConsumer; 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.AggregationInspectionHelper; import org.elasticsearch.search.aggregations.support.MultiValuesSourceFieldConfig; @@ -392,8 +391,8 @@ public void testMultiWeight() throws IOException { WeightedAvgAggregationBuilder aggregationBuilder = new WeightedAvgAggregationBuilder("_name").value(valueConfig) .weight(weightConfig); - AggregationExecutionException e = expectThrows( - AggregationExecutionException.class, + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, () -> testCase(new MatchAllDocsQuery(), aggregationBuilder, iw -> { iw.addDocument( Arrays.asList( diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpersTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpersTests.java index 8c1feae3d649d..d4c004a6e776f 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpersTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpersTests.java @@ -9,7 +9,6 @@ package org.elasticsearch.search.aggregations.pipeline; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.Aggregations; import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; @@ -92,8 +91,8 @@ public Object getProperty(String containingAggName, List path) { } }; - AggregationExecutionException e = expectThrows( - AggregationExecutionException.class, + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, () -> BucketHelpers.resolveBucketValue(agg, bucket, "foo>bar", BucketHelpers.GapPolicy.SKIP) ); diff --git a/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/support/AnalyticsValuesSourceType.java b/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/support/AnalyticsValuesSourceType.java index 7132a0537e0ae..9ff3024cfa59e 100644 --- a/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/support/AnalyticsValuesSourceType.java +++ b/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/support/AnalyticsValuesSourceType.java @@ -10,7 +10,7 @@ import org.elasticsearch.index.fielddata.IndexHistogramFieldData; import org.elasticsearch.script.AggregationScript; import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.FieldContext; import org.elasticsearch.search.aggregations.support.ValueType; @@ -29,7 +29,7 @@ public ValuesSource getEmpty() { @Override public ValuesSource getScript(AggregationScript.LeafFactory script, ValueType scriptValueType) { - throw new AggregationExecutionException("value source of type [" + this.value() + "] is not supported by scripts"); + throw AggregationErrors.valuesSourceDoesNotSupportScritps(this.value()); } @Override diff --git a/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/support/HistogramValuesSource.java b/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/support/HistogramValuesSource.java index 2c395e11e47c5..b62ce59b14c1d 100644 --- a/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/support/HistogramValuesSource.java +++ b/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/support/HistogramValuesSource.java @@ -14,7 +14,7 @@ import org.elasticsearch.index.fielddata.HistogramValues; import org.elasticsearch.index.fielddata.IndexHistogramFieldData; import org.elasticsearch.index.fielddata.SortedBinaryDocValues; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; @@ -27,7 +27,7 @@ public abstract static class Histogram extends org.elasticsearch.search.aggregat @Override public Function roundingPreparer(AggregationContext context) throws IOException { - throw new AggregationExecutionException("can't round a [histogram]"); + throw AggregationErrors.unsupportedRounding("histogram"); } public static class Fielddata extends Histogram { diff --git a/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/movingPercentiles/MovingPercentilesPipelineAggregator.java b/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/movingPercentiles/MovingPercentilesPipelineAggregator.java index 72a0dd4b6cb07..6e9c9812e9e95 100644 --- a/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/movingPercentiles/MovingPercentilesPipelineAggregator.java +++ b/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/movingPercentiles/MovingPercentilesPipelineAggregator.java @@ -8,7 +8,6 @@ import org.HdrHistogram.DoubleHistogram; import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.AggregationReduceContext; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregations; @@ -55,7 +54,7 @@ public InternalAggregation reduce(InternalAggregation aggregation, AggregationRe switch (config.method) { case TDIGEST -> reduceTDigest(buckets, histo, newBuckets, factory, config); case HDR -> reduceHDR(buckets, histo, newBuckets, factory, config); - default -> throw new AggregationExecutionException( + default -> throw new IllegalArgumentException( AbstractPipelineAggregationBuilder.BUCKETS_PATH_FIELD.getPreferredName() + " references an unknown percentile aggregation method: [" + config.method diff --git a/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/multiterms/InternalMultiTerms.java b/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/multiterms/InternalMultiTerms.java index c098d75b29a7b..ea864ddd761f7 100644 --- a/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/multiterms/InternalMultiTerms.java +++ b/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/multiterms/InternalMultiTerms.java @@ -11,6 +11,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.AggregationReduceContext; import org.elasticsearch.search.aggregations.Aggregations; @@ -28,6 +29,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import static org.elasticsearch.search.aggregations.bucket.terms.InternalTerms.DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME; @@ -201,11 +203,7 @@ public int compare(List thisTerms, List otherTerms) { try { res = ((Comparable) thisTerms.get(i)).compareTo(otherTerms.get(i)); } catch (ClassCastException ex) { - throw new AggregationExecutionException( - "Merging/Reducing the multi_term aggregations failed when computing " - + "the aggregation because one of the field you gave in the aggregation query existed as two different " - + "types in two different indices" - ); + throw AggregationErrors.reduceTypeMissmatch("MultiTerms", Optional.empty()); } if (res != 0) { return res; @@ -466,14 +464,7 @@ private boolean[] needsPromotionToDouble(List aggregations) } } if (hasNonNumber && (hasDouble || hasUnsignedLong || hasLong)) { - throw new AggregationExecutionException( - "Merging/Reducing the multi_term aggregations failed when computing the aggregation " - + name - + " because the field in the position " - + (i + 1) - + " in the aggregation has two different types in two " - + " different indices" - ); + throw AggregationErrors.reduceTypeMissmatch(name, Optional.of(i + 1)); } // Promotion to double is required if at least 2 of these 3 conditions are true. if ((hasDouble ? 1 : 0) + (hasUnsignedLong ? 1 : 0) + (hasLong ? 1 : 0) > 1) { diff --git a/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/ttest/PairedTTestAggregator.java b/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/ttest/PairedTTestAggregator.java index b817bea06dc56..4786f0bd00947 100644 --- a/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/ttest/PairedTTestAggregator.java +++ b/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/ttest/PairedTTestAggregator.java @@ -10,8 +10,8 @@ import org.elasticsearch.core.Releasables; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.aggregations.AggregationExecutionContext; -import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; @@ -71,10 +71,7 @@ public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, public void collect(int doc, long bucket) throws IOException { if (docAValues.advanceExact(doc) && docBValues.advanceExact(doc)) { if (docAValues.docValueCount() > 1 || docBValues.docValueCount() > 1) { - throw new AggregationExecutionException( - "Encountered more than one value for a " - + "single document. Use a script to combine multiple values per doc into a single value." - ); + throw AggregationErrors.unsupportedMultivalue(); } statsBuilder.grow(bigArrays(), bucket + 1); // There should always be one value if advanceExact lands us here, either diff --git a/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/rate/RateAggregatorTests.java b/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/rate/RateAggregatorTests.java index e648576e31f26..548069b83b4f8 100644 --- a/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/rate/RateAggregatorTests.java +++ b/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/rate/RateAggregatorTests.java @@ -37,7 +37,6 @@ import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptType; import org.elasticsearch.search.aggregations.AbstractAggregationBuilder; -import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.AggregatorTestCase; import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; @@ -309,7 +308,7 @@ public void testCompositeAggregationWithNoDateHistogramValueSources() { CompositeAggregationBuilder compositeAggregationBuilder = new CompositeAggregationBuilder("my_buckets", valuesSourceBuilders) .subAggregation(rateAggregationBuilder); - AggregationExecutionException ex = expectThrows(AggregationExecutionException.class, () -> testCase(iw -> { + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> testCase(iw -> { iw.addDocument(doc("2010-03-12T01:07:45", new NumericDocValuesField("val", 1))); iw.addDocument(doc("2010-04-01T03:43:34", new NumericDocValuesField("val", 3))); iw.addDocument(doc("2010-04-27T03:43:34", new NumericDocValuesField("val", 4))); diff --git a/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/ttest/TTestAggregatorTests.java b/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/ttest/TTestAggregatorTests.java index c59cb5d5c8ce5..da77790e6493c 100644 --- a/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/ttest/TTestAggregatorTests.java +++ b/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/ttest/TTestAggregatorTests.java @@ -31,7 +31,6 @@ import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptType; import org.elasticsearch.search.aggregations.AggregationBuilder; -import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.AggregatorTestCase; import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.global.InternalGlobal; @@ -177,8 +176,8 @@ public void testMatchesSortedNumericDocValues() throws IOException { } public void testMultiplePairedValues() { - AggregationExecutionException ex = expectThrows( - AggregationExecutionException.class, + IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, () -> testCase(new MatchAllDocsQuery(), TTestType.PAIRED, iw -> { iw.addDocument( asList( diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/support/AggregateMetricsValuesSource.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/support/AggregateMetricsValuesSource.java index 09e5ce70e2262..a964573dbb5d9 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/support/AggregateMetricsValuesSource.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/support/AggregateMetricsValuesSource.java @@ -11,7 +11,7 @@ import org.elasticsearch.index.fielddata.DocValueBits; import org.elasticsearch.index.fielddata.SortedBinaryDocValues; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.xpack.aggregatemetric.fielddata.IndexAggregateDoubleMetricFieldData; import org.elasticsearch.xpack.aggregatemetric.mapper.AggregateDoubleMetricFieldMapper; @@ -51,7 +51,7 @@ public boolean advanceExact(int doc) throws IOException { @Override protected Function roundingPreparer(AggregationContext context) throws IOException { - throw new AggregationExecutionException("Can't round an [" + AggregateDoubleMetricFieldMapper.CONTENT_TYPE + "]"); + throw AggregationErrors.unsupportedRounding(AggregateDoubleMetricFieldMapper.CONTENT_TYPE); } public SortedNumericDoubleValues getAggregateMetricValues(LeafReaderContext context, Metric metric) throws IOException { diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/support/AggregateMetricsValuesSourceType.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/support/AggregateMetricsValuesSourceType.java index 4e3455217aa96..4d5b572583bd1 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/support/AggregateMetricsValuesSourceType.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/support/AggregateMetricsValuesSourceType.java @@ -9,7 +9,7 @@ import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.script.AggregationScript; import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.aggregations.UnsupportedAggregationOnDownsampledIndex; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.FieldContext; @@ -36,7 +36,7 @@ public ValuesSource getEmpty() { @Override public ValuesSource getScript(AggregationScript.LeafFactory script, ValueType scriptValueType) { - throw new AggregationExecutionException("Value source of type [" + this.value() + "] is not supported by scripts"); + throw AggregationErrors.valuesSourceDoesNotSupportScritps(this.value()); } @Override diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java index 83f4c40c4ec28..13881f3900da3 100644 --- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java +++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java @@ -18,7 +18,7 @@ import org.elasticsearch.index.fielddata.MultiGeoPointValues; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.sort.BucketedSort; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.xpack.core.common.search.aggregations.MissingHelper; @@ -148,10 +148,7 @@ public BucketedSort.Leaf forLeaf(LeafReaderContext ctx) throws IOException { protected boolean advanceExact(int doc) throws IOException { if (docSortValues.advanceExact(doc)) { if (docSortValues.docValueCount() > 1) { - throw new AggregationExecutionException( - "Encountered more than one sort value for a " - + "single document. Use a script to combine multiple sort-values-per-doc into a single value." - ); + throw AggregationErrors.unsupportedMultivalue(); } // There should always be one weight if advanceExact lands us here, either @@ -211,10 +208,7 @@ public Loader loader(LeafReaderContext ctx) { } if (docGeoPointValues.docValueCount() > 1) { - throw new AggregationExecutionException( - "Encountered more than one geo_point value for a " - + "single document. Use a script to combine multiple geo_point-values-per-doc into a single value." - ); + throw AggregationErrors.unsupportedMultivalue(); } if (index >= values.size()) { diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/TimeSeriesGeoLineBuckets.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/TimeSeriesGeoLineBuckets.java index 08605e36229f0..9685f9047e4b4 100644 --- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/TimeSeriesGeoLineBuckets.java +++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/TimeSeriesGeoLineBuckets.java @@ -21,8 +21,8 @@ import org.elasticsearch.geometry.utils.WellKnownText; import org.elasticsearch.index.fielddata.MultiGeoPointValues; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.aggregations.AggregationExecutionContext; -import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.xpack.spatial.search.aggregations.support.GeoLineMultiValuesSource; @@ -288,10 +288,7 @@ private boolean loadSortField(int doc) throws IOException { if (docSortValues.advanceExact(doc)) { // If we get here from TSDB `position` metric, this assertion should have been made during indexing if (docSortValues.docValueCount() > 1) { - throw new AggregationExecutionException( - "Encountered more than one sort value for a " - + "single document. Use a script to combine multiple sort-values-per-doc into a single value." - ); + throw AggregationErrors.unsupportedMultivalue(); } assert docSortValues.docValueCount() == 1; simplifier.currentSortValue = docSortValues.nextValue(); @@ -306,10 +303,7 @@ private void loadPointField(int doc) throws IOException { } if (docGeoPointValues.docValueCount() > 1) { - throw new AggregationExecutionException( - "Encountered more than one geo_point value for a " - + "single document. Use a script to combine multiple geo_point-values-per-doc into a single value." - ); + throw AggregationErrors.unsupportedMultivalue(); } final GeoPoint point = docGeoPointValues.nextValue(); diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/support/CartesianPointValuesSource.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/support/CartesianPointValuesSource.java index f24be5050c016..5d916b6c5cead 100644 --- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/support/CartesianPointValuesSource.java +++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/support/CartesianPointValuesSource.java @@ -16,7 +16,7 @@ import org.elasticsearch.index.fielddata.MultiPointValues; import org.elasticsearch.index.fielddata.PointValues; import org.elasticsearch.index.fielddata.SortedBinaryDocValues; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.xcontent.ToXContentFragment; @@ -51,7 +51,7 @@ public DocValueBits docsWithValue(LeafReaderContext context) { @Override public final Function roundingPreparer(AggregationContext context) { - throw new AggregationExecutionException("can't round a [POINT]"); + throw AggregationErrors.unsupportedRounding("POINT"); } /** diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/support/CartesianShapeValuesSource.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/support/CartesianShapeValuesSource.java index 02777c1b05ad3..6403250d938f5 100644 --- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/support/CartesianShapeValuesSource.java +++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/support/CartesianShapeValuesSource.java @@ -10,7 +10,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.elasticsearch.common.Rounding; import org.elasticsearch.index.fielddata.SortedBinaryDocValues; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.xpack.spatial.index.fielddata.CartesianShapeValues; import org.elasticsearch.xpack.spatial.index.fielddata.IndexShapeFieldData; @@ -28,7 +28,7 @@ public CartesianShapeValues shapeValues(LeafReaderContext context) { @Override protected Function roundingPreparer(AggregationContext context) { - throw new AggregationExecutionException("can't round a [shape]"); + throw AggregationErrors.unsupportedRounding("shape"); } public static class Fielddata extends CartesianShapeValuesSource { diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/support/GeoShapeValuesSource.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/support/GeoShapeValuesSource.java index ae271d4470c68..84f231cbb2667 100644 --- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/support/GeoShapeValuesSource.java +++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/support/GeoShapeValuesSource.java @@ -10,7 +10,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.elasticsearch.common.Rounding; import org.elasticsearch.index.fielddata.SortedBinaryDocValues; -import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationErrors; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.xpack.spatial.index.fielddata.GeoShapeValues; import org.elasticsearch.xpack.spatial.index.fielddata.IndexShapeFieldData; @@ -28,7 +28,7 @@ public GeoShapeValues shapeValues(LeafReaderContext context) { @Override protected Function roundingPreparer(AggregationContext context) { - throw new AggregationExecutionException("can't round a [geo_shape]"); + throw AggregationErrors.unsupportedRounding("geo_shape"); } public static class Fielddata extends GeoShapeValuesSource {