summaryrefslogtreecommitdiff
path: root/core/src/test/java/org/elasticsearch/search/aggregations
diff options
context:
space:
mode:
authorTanguy Leroux <tlrx.dev@gmail.com>2017-05-10 13:40:39 +0200
committerTanguy Leroux <tlrx.dev@gmail.com>2017-05-10 13:40:39 +0200
commitbf718a686fa9ba51a8292178740c15670d0f0ec5 (patch)
tree13056b1cf9dbaf25f87ad94e66231d4d60b41081 /core/src/test/java/org/elasticsearch/search/aggregations
parent7c97296d774d397a0c311cb70485ea985b34c49d (diff)
parent3e4406f044676d50aab0e7135d2a289a929ea84b (diff)
Merge remote-tracking branch 'origin/master' into feature/client_aggs_parsing
# Conflicts: # core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTermsTestCase.java # core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalExtendedStatsTests.java # core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalMaxTests.java # core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalStatsTests.java # core/src/test/java/org/elasticsearch/search/aggregations/metrics/avg/InternalAvgTests.java # core/src/test/java/org/elasticsearch/search/aggregations/metrics/cardinality/InternalCardinalityTests.java # core/src/test/java/org/elasticsearch/search/aggregations/metrics/geobounds/InternalGeoBoundsTests.java # core/src/test/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroidTests.java # core/src/test/java/org/elasticsearch/search/aggregations/metrics/min/InternalMinTests.java # core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/InternalPercentilesTestCase.java # core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/hdr/InternalHDRPercentilesRanksTests.java # core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/tdigest/InternalTDigestPercentilesRanksTests.java # core/src/test/java/org/elasticsearch/search/aggregations/metrics/sum/InternalSumTests.java # core/src/test/java/org/elasticsearch/search/aggregations/metrics/valuecount/InternalValueCountTests.java # core/src/test/java/org/elasticsearch/search/aggregations/pipeline/InternalSimpleValueTests.java # core/src/test/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/percentile/InternalPercentilesBucketTests.java # core/src/test/java/org/elasticsearch/search/aggregations/pipeline/derivative/InternalDerivativeTests.java # test/framework/src/main/java/org/elasticsearch/test/InternalAggregationTestCase.java
Diffstat (limited to 'core/src/test/java/org/elasticsearch/search/aggregations')
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/InternalAggregationTestCase.java301
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregationTestCase.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/InternalGeoHashGridTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogramTests.java1
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogramTests.java1
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/InternalBinaryRangeTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/InternalRangeTestCase.java68
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/InternalRangeTests.java82
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/date/InternalDateRangeTests.java97
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/InternalGeoDistanceTests.java81
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/significant/InternalSignificantTermsTestCase.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTermsTestCase.java1
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/AvgIT.java10
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalExtendedStatsTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalMaxTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalStatsTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/SumIT.java10
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/ValueCountIT.java6
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/avg/InternalAvgTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/cardinality/InternalCardinalityTests.java3
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/geobounds/InternalGeoBoundsTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroidTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/min/InternalMinTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/InternalPercentilesTestCase.java3
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/hdr/InternalHDRPercentilesRanksTests.java1
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/tdigest/InternalTDigestPercentilesRanksTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/InternalScriptedMetricTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/sum/InternalSumTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/tophits/InternalTopHitsTests.java6
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/valuecount/InternalValueCountTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/pipeline/InternalSimpleValueTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/percentile/InternalPercentilesBucketTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/pipeline/derivative/InternalDerivativeTests.java2
33 files changed, 371 insertions, 336 deletions
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/InternalAggregationTestCase.java b/core/src/test/java/org/elasticsearch/search/aggregations/InternalAggregationTestCase.java
deleted file mode 100644
index a766dcbf5c..0000000000
--- a/core/src/test/java/org/elasticsearch/search/aggregations/InternalAggregationTestCase.java
+++ /dev/null
@@ -1,301 +0,0 @@
-/*
- * 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;
-
-import org.elasticsearch.common.ParseField;
-import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.util.MockBigArrays;
-import org.elasticsearch.common.xcontent.ContextParser;
-import org.elasticsearch.common.xcontent.NamedXContentRegistry;
-import org.elasticsearch.common.xcontent.ToXContent;
-import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.common.xcontent.XContentParserUtils;
-import org.elasticsearch.common.xcontent.XContentType;
-import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
-import org.elasticsearch.rest.action.search.RestSearchAction;
-import org.elasticsearch.script.ScriptService;
-import org.elasticsearch.search.DocValueFormat;
-import org.elasticsearch.search.SearchModule;
-import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.histogram.ParsedDateHistogram;
-import org.elasticsearch.search.aggregations.bucket.histogram.ParsedHistogram;
-import org.elasticsearch.search.aggregations.bucket.terms.DoubleTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.LongTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.ParsedDoubleTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.ParsedLongTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.ParsedStringTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.StringTerms;
-import org.elasticsearch.search.aggregations.metrics.avg.AvgAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.avg.ParsedAvg;
-import org.elasticsearch.search.aggregations.metrics.cardinality.CardinalityAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.cardinality.ParsedCardinality;
-import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBoundsAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.geobounds.ParsedGeoBounds;
-import org.elasticsearch.search.aggregations.metrics.geocentroid.GeoCentroidAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.geocentroid.ParsedGeoCentroid;
-import org.elasticsearch.search.aggregations.metrics.max.MaxAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.max.ParsedMax;
-import org.elasticsearch.search.aggregations.metrics.min.MinAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.min.ParsedMin;
-import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.InternalHDRPercentileRanks;
-import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.InternalHDRPercentiles;
-import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.ParsedHDRPercentileRanks;
-import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.ParsedHDRPercentiles;
-import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.InternalTDigestPercentileRanks;
-import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.InternalTDigestPercentiles;
-import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.ParsedTDigestPercentileRanks;
-import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.ParsedTDigestPercentiles;
-import org.elasticsearch.search.aggregations.metrics.stats.ParsedStats;
-import org.elasticsearch.search.aggregations.metrics.stats.StatsAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStatsAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.stats.extended.ParsedExtendedStats;
-import org.elasticsearch.search.aggregations.metrics.sum.ParsedSum;
-import org.elasticsearch.search.aggregations.metrics.sum.SumAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.valuecount.ParsedValueCount;
-import org.elasticsearch.search.aggregations.metrics.valuecount.ValueCountAggregationBuilder;
-import org.elasticsearch.search.aggregations.pipeline.InternalSimpleValue;
-import org.elasticsearch.search.aggregations.pipeline.ParsedSimpleValue;
-import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
-import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.InternalBucketMetricValue;
-import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.ParsedBucketMetricValue;
-import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.percentile.ParsedPercentilesBucket;
-import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.percentile.PercentilesBucketPipelineAggregationBuilder;
-import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.ParsedStatsBucket;
-import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.StatsBucketPipelineAggregationBuilder;
-import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.extended.ExtendedStatsBucketPipelineAggregationBuilder;
-import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.extended.ParsedExtendedStatsBucket;
-import org.elasticsearch.search.aggregations.pipeline.derivative.DerivativePipelineAggregationBuilder;
-import org.elasticsearch.search.aggregations.pipeline.derivative.ParsedDerivative;
-import org.elasticsearch.test.AbstractWireSerializingTestCase;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import static java.util.Collections.emptyList;
-import static java.util.Collections.singletonMap;
-import static org.elasticsearch.common.xcontent.XContentHelper.toXContent;
-import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent;
-
-public abstract class InternalAggregationTestCase<T extends InternalAggregation> extends AbstractWireSerializingTestCase<T> {
-
- private final NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(
- new SearchModule(Settings.EMPTY, false, emptyList()).getNamedWriteables());
-
- private final NamedXContentRegistry namedXContentRegistry = new NamedXContentRegistry(getNamedXContents());
-
- static List<NamedXContentRegistry.Entry> getNamedXContents() {
- Map<String, ContextParser<Object, ? extends Aggregation>> namedXContents = new HashMap<>();
- namedXContents.put(CardinalityAggregationBuilder.NAME, (p, c) -> ParsedCardinality.fromXContent(p, (String) c));
- namedXContents.put(InternalHDRPercentiles.NAME, (p, c) -> ParsedHDRPercentiles.fromXContent(p, (String) c));
- namedXContents.put(InternalHDRPercentileRanks.NAME, (p, c) -> ParsedHDRPercentileRanks.fromXContent(p, (String) c));
- namedXContents.put(InternalTDigestPercentiles.NAME, (p, c) -> ParsedTDigestPercentiles.fromXContent(p, (String) c));
- namedXContents.put(InternalTDigestPercentileRanks.NAME, (p, c) -> ParsedTDigestPercentileRanks.fromXContent(p, (String) c));
- namedXContents.put(PercentilesBucketPipelineAggregationBuilder.NAME, (p, c) -> ParsedPercentilesBucket.fromXContent(p, (String) c));
- namedXContents.put(MinAggregationBuilder.NAME, (p, c) -> ParsedMin.fromXContent(p, (String) c));
- namedXContents.put(MaxAggregationBuilder.NAME, (p, c) -> ParsedMax.fromXContent(p, (String) c));
- namedXContents.put(SumAggregationBuilder.NAME, (p, c) -> ParsedSum.fromXContent(p, (String) c));
- namedXContents.put(AvgAggregationBuilder.NAME, (p, c) -> ParsedAvg.fromXContent(p, (String) c));
- namedXContents.put(ValueCountAggregationBuilder.NAME, (p, c) -> ParsedValueCount.fromXContent(p, (String) c));
- namedXContents.put(InternalSimpleValue.NAME, (p, c) -> ParsedSimpleValue.fromXContent(p, (String) c));
- namedXContents.put(DerivativePipelineAggregationBuilder.NAME, (p, c) -> ParsedDerivative.fromXContent(p, (String) c));
- namedXContents.put(InternalBucketMetricValue.NAME, (p, c) -> ParsedBucketMetricValue.fromXContent(p, (String) c));
- namedXContents.put(StatsAggregationBuilder.NAME, (p, c) -> ParsedStats.fromXContent(p, (String) c));
- namedXContents.put(StatsBucketPipelineAggregationBuilder.NAME, (p, c) -> ParsedStatsBucket.fromXContent(p, (String) c));
- namedXContents.put(ExtendedStatsAggregationBuilder.NAME, (p, c) -> ParsedExtendedStats.fromXContent(p, (String) c));
- namedXContents.put(ExtendedStatsBucketPipelineAggregationBuilder.NAME,
- (p, c) -> ParsedExtendedStatsBucket.fromXContent(p, (String) c));
- namedXContents.put(GeoBoundsAggregationBuilder.NAME, (p, c) -> ParsedGeoBounds.fromXContent(p, (String) c));
- namedXContents.put(GeoCentroidAggregationBuilder.NAME, (p, c) -> ParsedGeoCentroid.fromXContent(p, (String) c));
- namedXContents.put(HistogramAggregationBuilder.NAME, (p, c) -> ParsedHistogram.fromXContent(p, (String) c));
- namedXContents.put(DateHistogramAggregationBuilder.NAME, (p, c) -> ParsedDateHistogram.fromXContent(p, (String) c));
- namedXContents.put(StringTerms.NAME, (p, c) -> ParsedStringTerms.fromXContent(p, (String) c));
- namedXContents.put(LongTerms.NAME, (p, c) -> ParsedLongTerms.fromXContent(p, (String) c));
- namedXContents.put(DoubleTerms.NAME, (p, c) -> ParsedDoubleTerms.fromXContent(p, (String) c));
-
- return namedXContents.entrySet().stream()
- .map(entry -> new NamedXContentRegistry.Entry(Aggregation.class, new ParseField(entry.getKey()), entry.getValue()))
- .collect(Collectors.toList());
- }
-
- protected abstract T createTestInstance(String name, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData);
-
- /** Return an instance on an unmapped field. */
- protected T createUnmappedInstance(String name,
- List<PipelineAggregator> pipelineAggregators,
- Map<String, Object> metaData) {
- // For most impls, we use the same instance in the unmapped case and in the mapped case
- return createTestInstance(name, pipelineAggregators, metaData);
- }
-
- public void testReduceRandom() {
- String name = randomAlphaOfLength(5);
- List<T> inputs = new ArrayList<>();
- List<InternalAggregation> toReduce = new ArrayList<>();
- int toReduceSize = between(1, 200);
- for (int i = 0; i < toReduceSize; i++) {
- T t = randomBoolean() ? createUnmappedInstance(name) : createTestInstance(name);
- inputs.add(t);
- toReduce.add(t);
- }
- ScriptService mockScriptService = mockScriptService();
- MockBigArrays bigArrays = new MockBigArrays(Settings.EMPTY, new NoneCircuitBreakerService());
- if (randomBoolean() && toReduce.size() > 1) {
- // sometimes do an incremental reduce
- Collections.shuffle(toReduce, random());
- int r = randomIntBetween(1, toReduceSize);
- List<InternalAggregation> internalAggregations = toReduce.subList(0, r);
- InternalAggregation.ReduceContext context =
- new InternalAggregation.ReduceContext(bigArrays, mockScriptService, false);
- @SuppressWarnings("unchecked")
- T reduced = (T) inputs.get(0).reduce(internalAggregations, context);
- toReduce = new ArrayList<>(toReduce.subList(r, toReduceSize));
- toReduce.add(reduced);
- }
- InternalAggregation.ReduceContext context =
- new InternalAggregation.ReduceContext(bigArrays, mockScriptService, true);
- @SuppressWarnings("unchecked")
- T reduced = (T) inputs.get(0).reduce(toReduce, context);
- assertReduced(reduced, inputs);
- }
-
- /**
- * overwrite in tests that need it
- */
- protected ScriptService mockScriptService() {
- return null;
- }
-
- protected abstract void assertReduced(T reduced, List<T> inputs);
-
- @Override
- protected final T createTestInstance() {
- return createTestInstance(randomAlphaOfLength(5));
- }
-
- private T createTestInstance(String name) {
- List<PipelineAggregator> pipelineAggregators = new ArrayList<>();
- // TODO populate pipelineAggregators
- Map<String, Object> metaData = null;
- if (randomBoolean()) {
- metaData = new HashMap<>();
- int metaDataCount = between(0, 10);
- while (metaData.size() < metaDataCount) {
- metaData.put(randomAlphaOfLength(5), randomAlphaOfLength(5));
- }
- }
- return createTestInstance(name, pipelineAggregators, metaData);
- }
-
- /** Return an instance on an unmapped field. */
- protected final T createUnmappedInstance(String name) {
- List<PipelineAggregator> pipelineAggregators = new ArrayList<>();
- // TODO populate pipelineAggregators
- Map<String, Object> metaData = new HashMap<>();
- int metaDataCount = randomBoolean() ? 0 : between(1, 10);
- while (metaData.size() < metaDataCount) {
- metaData.put(randomAlphaOfLength(5), randomAlphaOfLength(5));
- }
- return createUnmappedInstance(name, pipelineAggregators, metaData);
- }
-
- @Override
- protected NamedWriteableRegistry getNamedWriteableRegistry() {
- return namedWriteableRegistry;
- }
-
- @Override
- protected NamedXContentRegistry xContentRegistry() {
- return namedXContentRegistry;
- }
-
- public final void testFromXContent() throws IOException {
- final T aggregation = createTestInstance();
-
- //norelease Remove this assumption when all aggregations can be parsed back.
- assumeTrue("This test does not support the aggregation type yet",
- getNamedXContents().stream().filter(entry -> entry.name.match(aggregation.getType())).count() > 0);
-
- final Aggregation parsedAggregation = parseAndAssert(aggregation, randomBoolean());
- assertFromXContent(aggregation, (ParsedAggregation) parsedAggregation);
- }
-
- //norelease TODO make abstract
- protected void assertFromXContent(T aggregation, ParsedAggregation parsedAggregation) {
- }
-
- @SuppressWarnings("unchecked")
- protected <P extends ParsedAggregation> P parseAndAssert(final InternalAggregation aggregation,
- final boolean shuffled) throws IOException {
-
- final ToXContent.Params params = new ToXContent.MapParams(singletonMap(RestSearchAction.TYPED_KEYS_PARAM, "true"));
- final XContentType xContentType = randomFrom(XContentType.values());
- final boolean humanReadable = randomBoolean();
-
- final BytesReference originalBytes;
- if (shuffled) {
- originalBytes = toShuffledXContent(aggregation, xContentType, params, humanReadable);
- } else {
- originalBytes = toXContent(aggregation, xContentType, params, humanReadable);
- }
-
- Aggregation parsedAggregation;
- try (XContentParser parser = createParser(xContentType.xContent(), originalBytes)) {
- assertEquals(XContentParser.Token.START_OBJECT, parser.nextToken());
- assertEquals(XContentParser.Token.FIELD_NAME, parser.nextToken());
-
- parsedAggregation = XContentParserUtils.parseTypedKeysObject(parser, Aggregation.TYPED_KEYS_DELIMITER, Aggregation.class);
-
- assertEquals(XContentParser.Token.END_OBJECT, parser.currentToken());
- assertEquals(XContentParser.Token.END_OBJECT, parser.nextToken());
- assertNull(parser.nextToken());
-
- assertEquals(aggregation.getName(), parsedAggregation.getName());
- assertEquals(aggregation.getMetaData(), parsedAggregation.getMetaData());
-
- assertTrue(parsedAggregation instanceof ParsedAggregation);
- assertEquals(aggregation.getType(), ((ParsedAggregation) parsedAggregation).getType());
- }
-
- BytesReference parsedBytes = toXContent(parsedAggregation, xContentType, params, humanReadable);
- assertToXContentEquivalent(originalBytes, parsedBytes, xContentType);
-
- return (P) parsedAggregation;
- }
-
- /**
- * @return a random {@link DocValueFormat} that can be used in aggregations which
- * compute numbers.
- */
- protected static DocValueFormat randomNumericDocValueFormat() {
- final List<Supplier<DocValueFormat>> formats = new ArrayList<>(3);
- formats.add(() -> DocValueFormat.RAW);
- formats.add(() -> new DocValueFormat.Decimal(randomFrom("###.##", "###,###.##")));
- return randomFrom(formats).get();
- }
-}
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregationTestCase.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregationTestCase.java
index a31b28e7fd..f84a364ab1 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregationTestCase.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregationTestCase.java
@@ -20,11 +20,11 @@
package org.elasticsearch.search.aggregations.bucket;
import org.elasticsearch.search.aggregations.InternalAggregation;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.metrics.max.InternalMax;
import org.elasticsearch.search.aggregations.metrics.min.InternalMin;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.util.ArrayList;
import java.util.List;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/InternalGeoHashGridTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/InternalGeoHashGridTests.java
index ace22b244a..a27b47946f 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/InternalGeoHashGridTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/InternalGeoHashGridTests.java
@@ -21,9 +21,9 @@ package org.elasticsearch.search.aggregations.bucket.geogrid;
import org.apache.lucene.index.IndexWriter;
import org.elasticsearch.common.geo.GeoHashUtils;
import org.elasticsearch.common.io.stream.Writeable;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.util.ArrayList;
import java.util.HashMap;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogramTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogramTests.java
index 0b71d138a5..50cb453049 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogramTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogramTests.java
@@ -25,6 +25,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.InternalMultiBucketAggregationTestCase;
import org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import org.joda.time.DateTime;
import java.util.ArrayList;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogramTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogramTests.java
index 4f0fea87d6..b4ecf828e7 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogramTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogramTests.java
@@ -26,6 +26,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.InternalMultiBucketAggregationTestCase;
import org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.util.ArrayList;
import java.util.List;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/InternalBinaryRangeTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/InternalBinaryRangeTests.java
index 2f8d4f9e65..6d83b74c82 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/InternalBinaryRangeTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/InternalBinaryRangeTests.java
@@ -22,9 +22,9 @@ import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.search.DocValueFormat;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import org.junit.Before;
import java.util.ArrayList;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/InternalRangeTestCase.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/InternalRangeTestCase.java
new file mode 100644
index 0000000000..1edaea2f8e
--- /dev/null
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/InternalRangeTestCase.java
@@ -0,0 +1,68 @@
+/*
+ * 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.range;
+
+import org.elasticsearch.search.aggregations.InternalAggregation;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
+import org.junit.Before;
+
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+public abstract class InternalRangeTestCase<T extends InternalAggregation & Range> extends InternalAggregationTestCase<T> {
+
+ private boolean keyed;
+
+ @Override
+ @Before
+ public void setUp() throws Exception {
+ super.setUp();
+ keyed = randomBoolean();
+ }
+
+ @Override
+ protected T createTestInstance(String name, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
+ return createTestInstance(name, pipelineAggregators, metaData, keyed);
+ }
+
+ protected abstract T createTestInstance(String name,
+ List<PipelineAggregator> pipelineAggregators,
+ Map<String, Object> metaData,
+ boolean keyed);
+ @Override
+ protected void assertReduced(T reduced, List<T> inputs) {
+ final Map<String, Long> expectedCounts = new TreeMap<>();
+ for (T input : inputs) {
+ for (Range.Bucket bucket : input.getBuckets()) {
+ expectedCounts.compute(bucket.getKeyAsString(),
+ (key, oldValue) -> (oldValue == null ? 0 : oldValue) + bucket.getDocCount());
+
+ }
+ }
+ final Map<String, Long> actualCounts = new TreeMap<>();
+ for (Range.Bucket bucket : reduced.getBuckets()) {
+ actualCounts.compute(bucket.getKeyAsString(),
+ (key, oldValue) -> (oldValue == null ? 0 : oldValue) + bucket.getDocCount());
+ }
+ assertEquals(expectedCounts, actualCounts);
+ }
+}
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/InternalRangeTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/InternalRangeTests.java
new file mode 100644
index 0000000000..9264028d07
--- /dev/null
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/InternalRangeTests.java
@@ -0,0 +1,82 @@
+/*
+ * 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.range;
+
+import org.elasticsearch.common.collect.Tuple;
+import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.InternalAggregations;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.junit.Before;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class InternalRangeTests extends InternalRangeTestCase<InternalRange> {
+
+ private DocValueFormat format;
+ private List<Tuple<Double, Double>> ranges;
+
+ @Override
+ @Before
+ public void setUp() throws Exception {
+ super.setUp();
+ format = randomNumericDocValueFormat();
+
+ final int interval = randomFrom(1, 5, 10, 25, 50, 100);
+ final int numRanges = 1;//randomIntBetween(1, 10);
+
+ List<Tuple<Double, Double>> listOfRanges = new ArrayList<>(numRanges);
+ for (int i = 0; i < numRanges; i++) {
+ double from = i * interval;
+ double to = from + interval;
+ listOfRanges.add(Tuple.tuple(from, to));
+ }
+ if (randomBoolean()) {
+ // Add some overlapping ranges
+ double max = (double) numRanges * interval;
+ listOfRanges.add(Tuple.tuple(0.0, max));
+ listOfRanges.add(Tuple.tuple(0.0, max / 2));
+ listOfRanges.add(Tuple.tuple(max / 3, max / 3 * 2));
+ }
+ ranges = Collections.unmodifiableList(listOfRanges);
+ }
+
+ @Override
+ protected InternalRange createTestInstance(String name, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData,
+ boolean keyed) {
+ final List<InternalRange.Bucket> buckets = new ArrayList<>();
+ for (int i = 0; i < ranges.size(); ++i) {
+ Tuple<Double, Double> range = ranges.get(i);
+ int docCount = randomIntBetween(0, 1000);
+ double from = range.v1();
+ double to = range.v2();
+ buckets.add( new InternalRange.Bucket("range_" + i, from, to, docCount, InternalAggregations.EMPTY, keyed, format));
+ }
+ return new InternalRange<>(name, buckets, format, keyed, pipelineAggregators, Collections.emptyMap());
+ }
+
+ @Override
+ protected Writeable.Reader<InternalRange> instanceReader() {
+ return InternalRange::new;
+ }
+}
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/date/InternalDateRangeTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/date/InternalDateRangeTests.java
new file mode 100644
index 0000000000..bbfcdf7463
--- /dev/null
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/date/InternalDateRangeTests.java
@@ -0,0 +1,97 @@
+/*
+ * 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.range.date;
+
+import org.elasticsearch.common.collect.Tuple;
+import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.InternalAggregations;
+import org.elasticsearch.search.aggregations.bucket.range.InternalRangeTestCase;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
+import org.junit.Before;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+public class InternalDateRangeTests extends InternalRangeTestCase<InternalDateRange> {
+
+ private DocValueFormat format;
+ private List<Tuple<Double, Double>> dateRanges;
+
+ @Override
+ @Before
+ public void setUp() throws Exception {
+ super.setUp();
+ format = randomNumericDocValueFormat();
+
+ Function<DateTime, DateTime> interval = randomFrom(dateTime -> dateTime.plusSeconds(1), dateTime -> dateTime.plusMinutes(1),
+ dateTime -> dateTime.plusHours(1), dateTime -> dateTime.plusDays(1), dateTime -> dateTime.plusMonths(1), dateTime ->
+ dateTime.plusYears(1));
+
+ final int numRanges = randomIntBetween(1, 10);
+ final List<Tuple<Double, Double>> listOfRanges = new ArrayList<>(numRanges);
+
+ DateTime date = new DateTime(DateTimeZone.UTC);
+ double start = date.getMillis();
+ double end = 0;
+ for (int i = 0; i < numRanges; i++) {
+ double from = date.getMillis();
+ date = interval.apply(date);
+ double to = date.getMillis();
+ listOfRanges.add(Tuple.tuple(from, to));
+ if (to > end) {
+ end = to;
+ }
+ }
+ if (randomBoolean()) {
+ final int randomOverlaps = randomIntBetween(1, 5);
+ for (int i = 0; i < randomOverlaps; i++) {
+ listOfRanges.add(Tuple.tuple(start, randomDoubleBetween(start, end, false)));
+ }
+ }
+ dateRanges = Collections.unmodifiableList(listOfRanges);
+ }
+
+ @Override
+ protected InternalDateRange createTestInstance(String name,
+ List<PipelineAggregator> pipelineAggregators,
+ Map<String, Object> metaData,
+ boolean keyed) {
+ final List<InternalDateRange.Bucket> buckets = new ArrayList<>();
+ for (int i = 0; i < dateRanges.size(); ++i) {
+ Tuple<Double, Double> range = dateRanges.get(i);
+ int docCount = randomIntBetween(0, 1000);
+ double from = range.v1();
+ double to = range.v2();
+ buckets.add( new InternalDateRange.Bucket("range_" + i, from, to, docCount, InternalAggregations.EMPTY, keyed, format));
+ }
+ return new InternalDateRange(name, buckets, format, keyed, pipelineAggregators, metaData);
+ }
+
+ @Override
+ protected Writeable.Reader<InternalDateRange> instanceReader() {
+ return InternalDateRange::new;
+ }
+}
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/InternalGeoDistanceTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/InternalGeoDistanceTests.java
new file mode 100644
index 0000000000..9dd2a7a67c
--- /dev/null
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/InternalGeoDistanceTests.java
@@ -0,0 +1,81 @@
+/*
+ * 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.range.geodistance;
+
+import org.elasticsearch.common.collect.Tuple;
+import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.search.aggregations.InternalAggregations;
+import org.elasticsearch.search.aggregations.bucket.range.InternalRangeTestCase;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.junit.Before;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class InternalGeoDistanceTests extends InternalRangeTestCase<InternalGeoDistance> {
+
+ private List<Tuple<Double, Double>> geoDistanceRanges;
+
+ @Override
+ @Before
+ public void setUp() throws Exception {
+ super.setUp();
+
+ final int interval = randomFrom(1, 5, 10, 25, 50, 100);
+ final int numRanges = randomIntBetween(1, 10);
+
+ List<Tuple<Double, Double>> listOfRanges = new ArrayList<>(numRanges);
+ for (int i = 0; i < numRanges; i++) {
+ double from = i * interval;
+ double to = from + interval;
+ listOfRanges.add(Tuple.tuple(from, to));
+ }
+ if (randomBoolean()) {
+ // Add some overlapping ranges
+ double max = (double) numRanges * interval;
+ listOfRanges.add(Tuple.tuple(0.0, max));
+ listOfRanges.add(Tuple.tuple(0.0, max / 2));
+ listOfRanges.add(Tuple.tuple(max / 3, max / 3 * 2));
+ }
+ geoDistanceRanges = Collections.unmodifiableList(listOfRanges);
+ }
+ @Override
+ protected Writeable.Reader<InternalGeoDistance> instanceReader() {
+ return InternalGeoDistance::new;
+ }
+
+ @Override
+ protected InternalGeoDistance createTestInstance(String name,
+ List<PipelineAggregator> pipelineAggregators,
+ Map<String, Object> metaData,
+ boolean keyed) {
+ final List<InternalGeoDistance.Bucket> buckets = new ArrayList<>();
+ for (int i = 0; i < geoDistanceRanges.size(); ++i) {
+ Tuple<Double, Double> range = geoDistanceRanges.get(i);
+ int docCount = randomIntBetween(0, 1000);
+ double from = range.v1();
+ double to = range.v2();
+ buckets.add(new InternalGeoDistance.Bucket("range_" + i, from, to, docCount, InternalAggregations.EMPTY, keyed));
+ }
+ return new InternalGeoDistance(name, buckets, keyed, pipelineAggregators, metaData);
+ }
+}
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/significant/InternalSignificantTermsTestCase.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/significant/InternalSignificantTermsTestCase.java
index dde1562e1a..f1c54c4b50 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/significant/InternalSignificantTermsTestCase.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/significant/InternalSignificantTermsTestCase.java
@@ -19,8 +19,8 @@
package org.elasticsearch.search.aggregations.bucket.significant;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.util.Arrays;
import java.util.HashMap;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTermsTestCase.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTermsTestCase.java
index cfd7a82da1..b8c44d5c9b 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTermsTestCase.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTermsTestCase.java
@@ -23,6 +23,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.InternalMultiBucketAggregationTestCase;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.junit.Before;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.util.HashMap;
import java.util.List;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/AvgIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/AvgIT.java
index b7739d6c81..c952f43eb3 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/AvgIT.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/AvgIT.java
@@ -28,7 +28,7 @@ import org.elasticsearch.script.CompiledScript;
import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.LeafSearchScript;
import org.elasticsearch.script.Script;
-import org.elasticsearch.script.ScriptEngineService;
+import org.elasticsearch.script.ScriptEngine;
import org.elasticsearch.script.ScriptType;
import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.aggregations.InternalAggregation;
@@ -397,7 +397,7 @@ public class AvgIT extends AbstractNumericTestCase {
*/
public static class ExtractFieldScriptPlugin extends Plugin implements ScriptPlugin {
@Override
- public ScriptEngineService getScriptEngineService(Settings settings) {
+ public ScriptEngine getScriptEngine(Settings settings) {
return new ExtractFieldScriptEngine();
}
}
@@ -405,7 +405,7 @@ public class AvgIT extends AbstractNumericTestCase {
/**
* This mock script returns the field that is specified by name in the script body
*/
- public static class ExtractFieldScriptEngine implements ScriptEngineService {
+ public static class ExtractFieldScriptEngine implements ScriptEngine {
public static final String NAME = "extract_field";
@@ -502,7 +502,7 @@ public class AvgIT extends AbstractNumericTestCase {
*/
public static class FieldValueScriptPlugin extends Plugin implements ScriptPlugin {
@Override
- public ScriptEngineService getScriptEngineService(Settings settings) {
+ public ScriptEngine getScriptEngine(Settings settings) {
return new FieldValueScriptEngine();
}
}
@@ -510,7 +510,7 @@ public class AvgIT extends AbstractNumericTestCase {
/**
* This mock script returns the field value and adds one month to the returned date
*/
- public static class FieldValueScriptEngine implements ScriptEngineService {
+ public static class FieldValueScriptEngine implements ScriptEngine {
public static final String NAME = "field_value";
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalExtendedStatsTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalExtendedStatsTests.java
index 6459f5a5ae..8e90b88176 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalExtendedStatsTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalExtendedStatsTests.java
@@ -21,12 +21,12 @@ package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.search.DocValueFormat;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
import org.elasticsearch.search.aggregations.ParsedAggregation;
import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats.Bounds;
import org.elasticsearch.search.aggregations.metrics.stats.extended.InternalExtendedStats;
import org.elasticsearch.search.aggregations.metrics.stats.extended.ParsedExtendedStats;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import org.junit.Before;
import java.util.List;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalMaxTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalMaxTests.java
index 1d3a8b45a1..e2114fbebb 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalMaxTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalMaxTests.java
@@ -21,11 +21,11 @@ package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.search.DocValueFormat;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
import org.elasticsearch.search.aggregations.ParsedAggregation;
import org.elasticsearch.search.aggregations.metrics.max.InternalMax;
import org.elasticsearch.search.aggregations.metrics.max.ParsedMax;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.util.List;
import java.util.Map;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalStatsTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalStatsTests.java
index 6bfcfdd1ee..45bc4a8a9b 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalStatsTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/InternalStatsTests.java
@@ -20,11 +20,11 @@ package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.search.DocValueFormat;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
import org.elasticsearch.search.aggregations.ParsedAggregation;
import org.elasticsearch.search.aggregations.metrics.stats.InternalStats;
import org.elasticsearch.search.aggregations.metrics.stats.ParsedStats;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.util.List;
import java.util.Map;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/SumIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/SumIT.java
index 16d345c7b8..227ffc7251 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/SumIT.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/SumIT.java
@@ -28,7 +28,7 @@ import org.elasticsearch.script.CompiledScript;
import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.LeafSearchScript;
import org.elasticsearch.script.Script;
-import org.elasticsearch.script.ScriptEngineService;
+import org.elasticsearch.script.ScriptEngine;
import org.elasticsearch.script.ScriptType;
import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.aggregations.InternalAggregation;
@@ -396,7 +396,7 @@ public class SumIT extends AbstractNumericTestCase {
*/
public static class ExtractFieldScriptPlugin extends Plugin implements ScriptPlugin {
@Override
- public ScriptEngineService getScriptEngineService(Settings settings) {
+ public ScriptEngine getScriptEngine(Settings settings) {
return new ExtractFieldScriptEngine();
}
}
@@ -405,7 +405,7 @@ public class SumIT extends AbstractNumericTestCase {
* This mock script returns the field that is specified by name in the
* script body
*/
- public static class ExtractFieldScriptEngine implements ScriptEngineService {
+ public static class ExtractFieldScriptEngine implements ScriptEngine {
public static final String NAME = "extract_field";
@@ -508,7 +508,7 @@ public class SumIT extends AbstractNumericTestCase {
*/
public static class FieldValueScriptPlugin extends Plugin implements ScriptPlugin {
@Override
- public ScriptEngineService getScriptEngineService(Settings settings) {
+ public ScriptEngine getScriptEngine(Settings settings) {
return new FieldValueScriptEngine();
}
}
@@ -517,7 +517,7 @@ public class SumIT extends AbstractNumericTestCase {
* This mock script returns the field value and adds one to the returned
* value
*/
- public static class FieldValueScriptEngine implements ScriptEngineService {
+ public static class FieldValueScriptEngine implements ScriptEngine {
public static final String NAME = "field_value";
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/ValueCountIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/ValueCountIT.java
index 784635bb1d..2cfb344310 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/ValueCountIT.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/ValueCountIT.java
@@ -28,7 +28,7 @@ import org.elasticsearch.script.CompiledScript;
import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.LeafSearchScript;
import org.elasticsearch.script.Script;
-import org.elasticsearch.script.ScriptEngineService;
+import org.elasticsearch.script.ScriptEngine;
import org.elasticsearch.script.ScriptType;
import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.aggregations.InternalAggregation;
@@ -251,7 +251,7 @@ public class ValueCountIT extends ESIntegTestCase {
*/
public static class FieldValueScriptPlugin extends Plugin implements ScriptPlugin {
@Override
- public ScriptEngineService getScriptEngineService(Settings settings) {
+ public ScriptEngine getScriptEngine(Settings settings) {
return new FieldValueScriptEngine();
}
}
@@ -259,7 +259,7 @@ public class ValueCountIT extends ESIntegTestCase {
/**
* This mock script returns the field value. If the parameter map contains a parameter "s", the corresponding is used as field name.
*/
- public static class FieldValueScriptEngine implements ScriptEngineService {
+ public static class FieldValueScriptEngine implements ScriptEngine {
public static final String NAME = "field_value";
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/avg/InternalAvgTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/avg/InternalAvgTests.java
index 965f06e9e6..4e05b1bd68 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/avg/InternalAvgTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/avg/InternalAvgTests.java
@@ -22,8 +22,8 @@ package org.elasticsearch.search.aggregations.metrics.avg;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
-import org.elasticsearch.search.aggregations.ParsedAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.util.List;
import java.util.Map;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/cardinality/InternalCardinalityTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/cardinality/InternalCardinalityTests.java
index 60f0179d9a..1b46e0cdaf 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/cardinality/InternalCardinalityTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/cardinality/InternalCardinalityTests.java
@@ -24,9 +24,10 @@ import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.MockBigArrays;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
+import org.elasticsearch.test.InternalAggregationTestCase;
import org.elasticsearch.search.aggregations.ParsedAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import org.junit.After;
import java.util.ArrayList;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/geobounds/InternalGeoBoundsTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/geobounds/InternalGeoBoundsTests.java
index 611178d1bf..133b948219 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/geobounds/InternalGeoBoundsTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/geobounds/InternalGeoBoundsTests.java
@@ -20,9 +20,9 @@
package org.elasticsearch.search.aggregations.metrics.geobounds;
import org.elasticsearch.common.io.stream.Writeable;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
import org.elasticsearch.search.aggregations.ParsedAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.util.Collections;
import java.util.List;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroidTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroidTests.java
index 52474c4ae3..4e48649b07 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroidTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroidTests.java
@@ -21,9 +21,9 @@ package org.elasticsearch.search.aggregations.metrics.geocentroid;
import org.apache.lucene.geo.GeoEncodingUtils;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.io.stream.Writeable;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
import org.elasticsearch.search.aggregations.ParsedAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import org.elasticsearch.test.geo.RandomGeoGenerator;
import java.util.Collections;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/min/InternalMinTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/min/InternalMinTests.java
index 6230c80132..f1cfa0a63b 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/min/InternalMinTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/min/InternalMinTests.java
@@ -21,9 +21,9 @@ package org.elasticsearch.search.aggregations.metrics.min;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.search.DocValueFormat;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
import org.elasticsearch.search.aggregations.ParsedAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.util.List;
import java.util.Map;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/InternalPercentilesTestCase.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/InternalPercentilesTestCase.java
index be105f2af8..8e06926ea0 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/InternalPercentilesTestCase.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/InternalPercentilesTestCase.java
@@ -21,6 +21,9 @@ package org.elasticsearch.search.aggregations.metrics.percentiles;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.ParsedAggregation;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
+import org.junit.Before;
import java.util.List;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/hdr/InternalHDRPercentilesRanksTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/hdr/InternalHDRPercentilesRanksTests.java
index d9379edefe..728ddf6afa 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/hdr/InternalHDRPercentilesRanksTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/hdr/InternalHDRPercentilesRanksTests.java
@@ -25,6 +25,7 @@ import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.metrics.percentiles.InternalPercentilesRanksTestCase;
import org.elasticsearch.search.aggregations.metrics.percentiles.ParsedPercentiles;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.util.Arrays;
import java.util.List;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/tdigest/InternalTDigestPercentilesRanksTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/tdigest/InternalTDigestPercentilesRanksTests.java
index f8698fda2c..189091433a 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/tdigest/InternalTDigestPercentilesRanksTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/tdigest/InternalTDigestPercentilesRanksTests.java
@@ -21,9 +21,9 @@ package org.elasticsearch.search.aggregations.metrics.percentiles.tdigest;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.search.DocValueFormat;
-import org.elasticsearch.search.aggregations.metrics.percentiles.InternalPercentilesRanksTestCase;
import org.elasticsearch.search.aggregations.metrics.percentiles.ParsedPercentiles;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.util.Arrays;
import java.util.List;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/InternalScriptedMetricTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/InternalScriptedMetricTests.java
index 9d8ff94c47..75975d5a39 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/InternalScriptedMetricTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/InternalScriptedMetricTests.java
@@ -30,8 +30,8 @@ import org.elasticsearch.script.ScriptEngineRegistry;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.script.ScriptSettings;
import org.elasticsearch.script.ScriptType;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.io.IOException;
import java.util.Collections;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/sum/InternalSumTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/sum/InternalSumTests.java
index 6fb61257a9..1e5b8057cb 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/sum/InternalSumTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/sum/InternalSumTests.java
@@ -20,9 +20,9 @@ package org.elasticsearch.search.aggregations.metrics.sum;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.search.DocValueFormat;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
import org.elasticsearch.search.aggregations.ParsedAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.util.List;
import java.util.Map;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/tophits/InternalTopHitsTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/tophits/InternalTopHitsTests.java
index 22479eb434..db68e8537d 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/tophits/InternalTopHitsTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/tophits/InternalTopHitsTests.java
@@ -30,11 +30,11 @@ import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.common.text.Text;
-import org.elasticsearch.search.SearchHitField;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
-import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.SearchHit;
+import org.elasticsearch.search.SearchHitField;
import org.elasticsearch.search.SearchHits;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.util.ArrayList;
import java.util.Arrays;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/valuecount/InternalValueCountTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/valuecount/InternalValueCountTests.java
index 17df4b89dc..e7483cad91 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/valuecount/InternalValueCountTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/valuecount/InternalValueCountTests.java
@@ -20,9 +20,9 @@
package org.elasticsearch.search.aggregations.metrics.valuecount;
import org.elasticsearch.common.io.stream.Writeable;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
import org.elasticsearch.search.aggregations.ParsedAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.util.List;
import java.util.Map;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/InternalSimpleValueTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/InternalSimpleValueTests.java
index ffd5fd1bf8..a996bde059 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/InternalSimpleValueTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/InternalSimpleValueTests.java
@@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.pipeline;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.search.DocValueFormat;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
+import org.elasticsearch.test.InternalAggregationTestCase;
import org.elasticsearch.search.aggregations.ParsedAggregation;
import java.util.Collections;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/percentile/InternalPercentilesBucketTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/percentile/InternalPercentilesBucketTests.java
index 3f29caae84..ed4fc00776 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/percentile/InternalPercentilesBucketTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/percentile/InternalPercentilesBucketTests.java
@@ -21,10 +21,10 @@ package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.percentile;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.search.DocValueFormat;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
import org.elasticsearch.search.aggregations.ParsedAggregation;
import org.elasticsearch.search.aggregations.metrics.percentiles.Percentile;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.io.IOException;
import java.util.Collections;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/derivative/InternalDerivativeTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/derivative/InternalDerivativeTests.java
index 229c98b4ad..58115a7fbb 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/derivative/InternalDerivativeTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/derivative/InternalDerivativeTests.java
@@ -21,9 +21,9 @@ package org.elasticsearch.search.aggregations.pipeline.derivative;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.search.DocValueFormat;
-import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
import org.elasticsearch.search.aggregations.ParsedAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.test.InternalAggregationTestCase;
import java.util.Collections;
import java.util.List;