Skip to content

Commit

Permalink
Aggs error codes part 1 (elastic#99963)
Browse files Browse the repository at this point in the history
As part of our effort to increase the supportability of Elasticsearch,
this PR changes many aggregations errors from being 500 class (which is
the default for `AggregationExecutionException`) to 400 class (which is
the default for `IllegalArgumentException`).  All of these cases are
errors which should not be retried, as they are failing directly related
to the content of the request and/or state of the index.

There are definitely more cases where we are returning an incorrect
error code, but for this PR I focused on just changing the low hanging
fruit.
  • Loading branch information
not-napoleon authored Oct 4, 2023
1 parent 60daef3 commit 6660503
Show file tree
Hide file tree
Showing 41 changed files with 186 additions and 149 deletions.
5 changes: 5 additions & 0 deletions docs/changelog/99963.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
pr: 99963
summary: Aggs error codes part 1
area: Aggregations
type: bug
issues: []
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -53,9 +53,7 @@ protected Aggregator doCreateInternal(
Map<String, ValuesSource.Numeric> typedValuesSources = Maps.newMapWithExpectedSize(valuesSources.size());
for (Map.Entry<String, ValuesSource> 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());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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;
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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() {}

/**
Expand All @@ -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<Integer> 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 + "]");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -610,23 +610,15 @@ 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);
}

@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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,15 +29,15 @@ 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.");
}
}

@Override
public Aggregator createInternal(Aggregator parent, CardinalityUpperBound cardinality, Map<String, Object> metadata)
throws IOException {
if (parent != null) {
throw new AggregationExecutionException(
throw new IllegalArgumentException(
"Aggregation ["
+ parent.name()
+ "] cannot have a global "
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -269,12 +270,7 @@ public InternalAggregation reduce(List<InternalAggregation> 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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;

Expand Down Expand Up @@ -106,12 +107,7 @@ public InternalAggregation reduce(List<InternalAggregation> 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<B> bucketList = buckets.computeIfAbsent(bucket.getKey(), k -> new ArrayList<>());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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 "
Expand Down Expand Up @@ -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 "
Expand Down
Loading

0 comments on commit 6660503

Please sign in to comment.