diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java index fa90e34ef3dd0..c72e9d22dc0ae 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java @@ -19,6 +19,10 @@ package org.elasticsearch.search.aggregations.bucket.histogram; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.search.DocIdSetIterator; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.rounding.DateTimeUnit; @@ -27,8 +31,13 @@ import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.fielddata.AtomicNumericFieldData; +import org.elasticsearch.index.fielddata.IndexNumericFieldData; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.MappedFieldType.Relation; +import org.elasticsearch.index.query.QueryShardContext; +import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.AggregationBuilder; -import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.BucketOrder; @@ -44,6 +53,8 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper; import org.elasticsearch.search.aggregations.support.ValuesSourceType; import org.elasticsearch.search.internal.SearchContext; +import org.joda.time.DateTimeField; +import org.joda.time.DateTimeZone; import java.io.IOException; import java.util.HashMap; @@ -351,36 +362,121 @@ public String getType() { return NAME; } + /* + * NOTE: this can't be done in rewrite() because the timezone is then also used on the + * coordinating node in order to generate missing buckets, which may cross a transition + * even though data on the shards doesn't. + */ + DateTimeZone rewriteTimeZone(QueryShardContext context) throws IOException { + final DateTimeZone tz = timeZone(); + if (field() != null && + tz != null && + tz.isFixed() == false && + field() != null && + script() == null) { + final MappedFieldType ft = context.fieldMapper(field()); + final IndexReader reader = context.getIndexReader(); + if (ft != null && reader != null) { + Long anyInstant = null; + final IndexNumericFieldData fieldData = context.getForField(ft); + for (LeafReaderContext ctx : reader.leaves()) { + AtomicNumericFieldData leafFD = ((IndexNumericFieldData) fieldData).load(ctx); + SortedNumericDocValues values = leafFD.getLongValues(); + if (values.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { + anyInstant = values.nextValue(); + break; + } + } + + if (anyInstant != null) { + final long prevTransition = tz.previousTransition(anyInstant); + final long nextTransition = tz.nextTransition(anyInstant); + + // We need all not only values but also rounded values to be within + // [prevTransition, nextTransition]. + final long low; + DateTimeUnit intervalAsUnit = getIntervalAsDateTimeUnit(); + if (intervalAsUnit != null) { + final DateTimeField dateTimeField = intervalAsUnit.field(tz); + low = dateTimeField.roundCeiling(prevTransition); + } else { + final TimeValue intervalAsMillis = getIntervalAsTimeValue(); + low = Math.addExact(prevTransition, intervalAsMillis.millis()); + } + // rounding rounds down, so 'nextTransition' is a good upper bound + final long high = nextTransition; + + final DocValueFormat format = ft.docValueFormat(null, null); + final String formattedLow = format.format(low); + final String formattedHigh = format.format(high); + if (ft.isFieldWithinQuery(reader, formattedLow, formattedHigh, + true, false, tz, null, context) == Relation.WITHIN) { + // All values in this reader have the same offset despite daylight saving times. + // This is very common for location-based timezones such as Europe/Paris in + // combination with time-based indices. + return DateTimeZone.forOffsetMillis(tz.getOffset(anyInstant)); + } + } + } + } + return tz; + } + @Override protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { - Rounding rounding = createRounding(); + final DateTimeZone tz = timeZone(); + final Rounding rounding = createRounding(tz); + final DateTimeZone rewrittenTimeZone = rewriteTimeZone(context.getQueryShardContext()); + final Rounding shardRounding; + if (tz == rewrittenTimeZone) { + shardRounding = rounding; + } else { + shardRounding = createRounding(rewrittenTimeZone); + } + ExtendedBounds roundedBounds = null; if (this.extendedBounds != null) { // parse any string bounds to longs and round roundedBounds = this.extendedBounds.parseAndValidate(name, context, config.format()).round(rounding); } - return new DateHistogramAggregatorFactory(name, config, interval, dateHistogramInterval, offset, order, keyed, minDocCount, - rounding, roundedBounds, context, parent, subFactoriesBuilder, metaData); + return new DateHistogramAggregatorFactory(name, config, offset, order, keyed, minDocCount, + rounding, shardRounding, roundedBounds, context, parent, subFactoriesBuilder, metaData); } - private Rounding createRounding() { - Rounding.Builder tzRoundingBuilder; + /** Return the interval as a date time unit if applicable. If this returns + * {@code null} then it means that the interval is expressed as a fixed + * {@link TimeValue} and may be accessed via + * {@link #getIntervalAsTimeValue()}. */ + private DateTimeUnit getIntervalAsDateTimeUnit() { if (dateHistogramInterval != null) { - DateTimeUnit dateTimeUnit = DATE_FIELD_UNITS.get(dateHistogramInterval.toString()); - if (dateTimeUnit != null) { - tzRoundingBuilder = Rounding.builder(dateTimeUnit); - } else { - // the interval is a time value? - tzRoundingBuilder = Rounding.builder( - TimeValue.parseTimeValue(dateHistogramInterval.toString(), null, getClass().getSimpleName() + ".interval")); - } + return DATE_FIELD_UNITS.get(dateHistogramInterval.toString()); + } + return null; + } + + /** + * Get the interval as a {@link TimeValue}. Should only be called if + * {@link #getIntervalAsDateTimeUnit()} returned {@code null}. + */ + private TimeValue getIntervalAsTimeValue() { + if (dateHistogramInterval != null) { + return TimeValue.parseTimeValue(dateHistogramInterval.toString(), null, getClass().getSimpleName() + ".interval"); + } else { + return TimeValue.timeValueMillis(interval); + } + } + + private Rounding createRounding(DateTimeZone timeZone) { + Rounding.Builder tzRoundingBuilder; + DateTimeUnit intervalAsUnit = getIntervalAsDateTimeUnit(); + if (intervalAsUnit != null) { + tzRoundingBuilder = Rounding.builder(intervalAsUnit); } else { - // the interval is an integer time value in millis? - tzRoundingBuilder = Rounding.builder(TimeValue.timeValueMillis(interval)); + tzRoundingBuilder = Rounding.builder(getIntervalAsTimeValue()); } - if (timeZone() != null) { - tzRoundingBuilder.timeZone(timeZone()); + if (timeZone != null) { + tzRoundingBuilder.timeZone(timeZone); } Rounding rounding = tzRoundingBuilder.build(); return rounding; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java index c32cedb4427e8..94dc18eae63e2 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java @@ -55,6 +55,7 @@ class DateHistogramAggregator extends BucketsAggregator { private final ValuesSource.Numeric valuesSource; private final DocValueFormat formatter; private final Rounding rounding; + private final Rounding shardRounding; private final BucketOrder order; private final boolean keyed; @@ -64,14 +65,15 @@ class DateHistogramAggregator extends BucketsAggregator { private final LongHash bucketOrds; private long offset; - DateHistogramAggregator(String name, AggregatorFactories factories, Rounding rounding, long offset, BucketOrder order, - boolean keyed, + DateHistogramAggregator(String name, AggregatorFactories factories, Rounding rounding, Rounding shardRounding, + long offset, BucketOrder order, boolean keyed, long minDocCount, @Nullable ExtendedBounds extendedBounds, @Nullable ValuesSource.Numeric valuesSource, DocValueFormat formatter, SearchContext aggregationContext, Aggregator parent, List pipelineAggregators, Map metaData) throws IOException { super(name, factories, aggregationContext, parent, pipelineAggregators, metaData); this.rounding = rounding; + this.shardRounding = shardRounding; this.offset = offset; this.order = InternalOrder.validate(order, this);; this.keyed = keyed; @@ -105,7 +107,9 @@ public void collect(int doc, long bucket) throws IOException { long previousRounded = Long.MIN_VALUE; for (int i = 0; i < valuesCount; ++i) { long value = values.nextValue(); - long rounded = rounding.round(value - offset) + offset; + // We can use shardRounding here, which is sometimes more efficient + // if daylight saving times are involved. + long rounded = shardRounding.round(value - offset) + offset; assert rounded >= previousRounded; if (rounded == previousRounded) { continue; @@ -138,6 +142,7 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator(this)); // value source will be null for unmapped fields + // Important: use `rounding` here, not `shardRounding` InternalDateHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new InternalDateHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations(), extendedBounds) : null; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java index a64e018288879..c7ad6de7e0d72 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java @@ -38,28 +38,27 @@ public final class DateHistogramAggregatorFactory extends ValuesSourceAggregatorFactory { - private final DateHistogramInterval dateHistogramInterval; - private final long interval; private final long offset; private final BucketOrder order; private final boolean keyed; private final long minDocCount; private final ExtendedBounds extendedBounds; - private Rounding rounding; + private final Rounding rounding; + private final Rounding shardRounding; - public DateHistogramAggregatorFactory(String name, ValuesSourceConfig config, long interval, - DateHistogramInterval dateHistogramInterval, long offset, BucketOrder order, boolean keyed, long minDocCount, - Rounding rounding, ExtendedBounds extendedBounds, SearchContext context, AggregatorFactory parent, - AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { + public DateHistogramAggregatorFactory(String name, ValuesSourceConfig config, + long offset, BucketOrder order, boolean keyed, long minDocCount, + Rounding rounding, Rounding shardRounding, ExtendedBounds extendedBounds, SearchContext context, + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, + Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); - this.interval = interval; - this.dateHistogramInterval = dateHistogramInterval; this.offset = offset; this.order = order; this.keyed = keyed; this.minDocCount = minDocCount; this.extendedBounds = extendedBounds; this.rounding = rounding; + this.shardRounding = shardRounding; } public long minDocCount() { @@ -77,8 +76,8 @@ protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, Aggrega private Aggregator createAggregator(ValuesSource.Numeric valuesSource, Aggregator parent, List pipelineAggregators, Map metaData) throws IOException { - return new DateHistogramAggregator(name, factories, rounding, offset, order, keyed, minDocCount, extendedBounds, valuesSource, - config.format(), context, parent, pipelineAggregators, metaData); + return new DateHistogramAggregator(name, factories, rounding, shardRounding, offset, order, keyed, minDocCount, extendedBounds, + valuesSource, config.format(), context, parent, pipelineAggregators, metaData); } @Override diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramTests.java similarity index 52% rename from server/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramTests.java rename to server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramTests.java index e86b3a553e9c4..1f83842eab24f 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramTests.java @@ -17,14 +17,27 @@ * under the License. */ -package org.elasticsearch.search.aggregations.bucket; +package org.elasticsearch.search.aggregations.bucket.histogram; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.store.Directory; +import org.elasticsearch.common.joda.FormatDateTimeFormatter; +import org.elasticsearch.common.joda.Joda; +import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.search.aggregations.BaseAggregationTestCase; +import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBoundsTests; -import org.elasticsearch.search.aggregations.BucketOrder; +import org.joda.time.DateTimeZone; +import org.junit.Assume; +import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -120,4 +133,73 @@ private List randomOrder() { return orders; } + private static Document documentForDate(String field, long millis) { + Document doc = new Document(); + doc.add(new LongPoint(field, millis)); + doc.add(new SortedNumericDocValuesField(field, millis)); + return doc; + } + + public void testRewriteTimeZone() throws IOException { + Assume.assumeTrue(getCurrentTypes().length > 0); // we need mappings + FormatDateTimeFormatter format = Joda.forPattern("strict_date_optional_time"); + + try (Directory dir = newDirectory(); + IndexWriter w = new IndexWriter(dir, newIndexWriterConfig())) { + + w.addDocument(documentForDate(DATE_FIELD_NAME, format.parser().parseDateTime("2018-03-11T11:55:00").getMillis())); + w.addDocument(documentForDate(DATE_FIELD_NAME, format.parser().parseDateTime("2017-10-30T18:13:00").getMillis())); + + try (IndexReader readerThatDoesntCross = DirectoryReader.open(w)) { + + w.addDocument(documentForDate(DATE_FIELD_NAME, format.parser().parseDateTime("2018-03-25T02:44:00").getMillis())); + + try (IndexReader readerThatCrosses = DirectoryReader.open(w)) { + + QueryShardContext shardContextThatDoesntCross = createShardContext(readerThatDoesntCross); + QueryShardContext shardContextThatCrosses = createShardContext(readerThatCrosses); + + DateHistogramAggregationBuilder builder = new DateHistogramAggregationBuilder("my_date_histo"); + builder.field(DATE_FIELD_NAME); + builder.dateHistogramInterval(DateHistogramInterval.DAY); + + // no timeZone => no rewrite + assertNull(builder.rewriteTimeZone(shardContextThatDoesntCross)); + assertNull(builder.rewriteTimeZone(shardContextThatCrosses)); + + // fixed timeZone => no rewrite + DateTimeZone tz = DateTimeZone.forOffsetHours(1); + builder.timeZone(tz); + assertSame(tz, builder.rewriteTimeZone(shardContextThatDoesntCross)); + assertSame(tz, builder.rewriteTimeZone(shardContextThatCrosses)); + + // daylight-saving-times => rewrite if doesn't cross + tz = DateTimeZone.forID("Europe/Paris"); + builder.timeZone(tz); + assertEquals(DateTimeZone.forOffsetHours(1), builder.rewriteTimeZone(shardContextThatDoesntCross)); + assertSame(tz, builder.rewriteTimeZone(shardContextThatCrosses)); + + // Rounded values are no longer all within the same transitions => no rewrite + builder.dateHistogramInterval(DateHistogramInterval.MONTH); + assertSame(tz, builder.rewriteTimeZone(shardContextThatDoesntCross)); + assertSame(tz, builder.rewriteTimeZone(shardContextThatCrosses)); + + builder = new DateHistogramAggregationBuilder("my_date_histo"); + builder.field(DATE_FIELD_NAME); + builder.timeZone(tz); + + builder.interval(1000L * 60 * 60 * 24); // ~ 1 day + assertEquals(DateTimeZone.forOffsetHours(1), builder.rewriteTimeZone(shardContextThatDoesntCross)); + assertSame(tz, builder.rewriteTimeZone(shardContextThatCrosses)); + + // Because the interval is large, rounded values are not + // within the same transitions as the values => no rewrite + builder.interval(1000L * 60 * 60 * 24 * 30); // ~ 1 month + assertSame(tz, builder.rewriteTimeZone(shardContextThatDoesntCross)); + assertSame(tz, builder.rewriteTimeZone(shardContextThatCrosses)); + } + } + } + } + } diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/BaseAggregationTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/BaseAggregationTestCase.java index 3213ce526ce12..de5e238199693 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/BaseAggregationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/BaseAggregationTestCase.java @@ -38,6 +38,7 @@ import org.elasticsearch.plugins.SearchPlugin; import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; +import org.elasticsearch.test.AbstractBuilderTestCase; import org.elasticsearch.test.AbstractQueryTestCase; import org.elasticsearch.test.ESTestCase; @@ -50,60 +51,12 @@ import static org.elasticsearch.test.EqualsHashCodeTestUtils.checkEqualsAndHashCode; import static org.hamcrest.Matchers.hasSize; -public abstract class BaseAggregationTestCase> extends ESTestCase { +public abstract class BaseAggregationTestCase> extends AbstractBuilderTestCase { - protected static final String STRING_FIELD_NAME = "mapped_string"; - protected static final String INT_FIELD_NAME = "mapped_int"; - protected static final String DOUBLE_FIELD_NAME = "mapped_double"; - protected static final String BOOLEAN_FIELD_NAME = "mapped_boolean"; - protected static final String DATE_FIELD_NAME = "mapped_date"; protected static final String IP_FIELD_NAME = "mapped_ip"; - private String[] currentTypes; - - protected String[] getCurrentTypes() { - return currentTypes; - } - - private NamedWriteableRegistry namedWriteableRegistry; - private NamedXContentRegistry xContentRegistry; protected abstract AB createTestAggregatorBuilder(); - protected Collection> getPlugins() { - return Collections.emptyList(); - } - - /** - * Setup for the whole base test class. - */ - @Override - public void setUp() throws Exception { - super.setUp(); - Settings settings = Settings.builder() - .put("node.name", AbstractQueryTestCase.class.toString()) - .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()) - .build(); - IndicesModule indicesModule = new IndicesModule(Collections.emptyList()); - PluginsService pluginsService = new PluginsService(settings, null, null, null, getPlugins()); - SearchModule searchModule = new SearchModule(settings, false, pluginsService.filterPlugins(SearchPlugin.class)); - List entries = new ArrayList<>(); - entries.addAll(indicesModule.getNamedWriteables()); - entries.addAll(searchModule.getNamedWriteables()); - namedWriteableRegistry = new NamedWriteableRegistry(entries); - xContentRegistry = new NamedXContentRegistry(searchModule.getNamedXContents()); - //create some random type with some default field, those types will stick around for all of the subclasses - currentTypes = new String[randomIntBetween(0, 5)]; - for (int i = 0; i < currentTypes.length; i++) { - String type = randomAlphaOfLengthBetween(1, 10); - currentTypes[i] = type; - } - } - - @Override - protected NamedXContentRegistry xContentRegistry() { - return xContentRegistry; - } - /** * Generic test that creates new AggregatorFactory from the test * AggregatorFactory and checks both for equality and asserts equality on @@ -157,7 +110,7 @@ public void testSerialization() throws IOException { AB testAgg = createTestAggregatorBuilder(); try (BytesStreamOutput output = new BytesStreamOutput()) { output.writeNamedWriteable(testAgg); - try (StreamInput in = new NamedWriteableAwareStreamInput(output.bytes().streamInput(), namedWriteableRegistry)) { + try (StreamInput in = new NamedWriteableAwareStreamInput(output.bytes().streamInput(), namedWriteableRegistry())) { AggregationBuilder deserialized = in.readNamedWriteable(AggregationBuilder.class); assertEquals(testAgg, deserialized); assertEquals(testAgg.hashCode(), deserialized.hashCode()); @@ -181,12 +134,12 @@ public void testShallowCopy() { // we use the streaming infra to create a copy of the query provided as // argument - private AB copyAggregation(AB agg) throws IOException { + protected AB copyAggregation(AB agg) throws IOException { try (BytesStreamOutput output = new BytesStreamOutput()) { agg.writeTo(output); - try (StreamInput in = new NamedWriteableAwareStreamInput(output.bytes().streamInput(), namedWriteableRegistry)) { + try (StreamInput in = new NamedWriteableAwareStreamInput(output.bytes().streamInput(), namedWriteableRegistry())) { @SuppressWarnings("unchecked") - AB secondAgg = (AB) namedWriteableRegistry.getReader(AggregationBuilder.class, agg.getWriteableName()).read(in); + AB secondAgg = (AB) namedWriteableRegistry().getReader(AggregationBuilder.class, agg.getWriteableName()).read(in); return secondAgg; } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java new file mode 100644 index 0000000000000..e50e93ec672b8 --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java @@ -0,0 +1,399 @@ +/* + * 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.test; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.util.Accountable; +import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.action.termvectors.MultiTermVectorsRequest; +import org.elasticsearch.action.termvectors.MultiTermVectorsResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.compress.CompressedXContent; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.settings.IndexScopedSettings; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.settings.SettingsModule; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.env.Environment; +import org.elasticsearch.env.TestEnvironment; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.analysis.IndexAnalyzers; +import org.elasticsearch.index.cache.bitset.BitsetFilterCache; +import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.IndexFieldDataCache; +import org.elasticsearch.index.fielddata.IndexFieldDataService; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.query.QueryShardContext; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.similarity.SimilarityService; +import org.elasticsearch.indices.IndicesModule; +import org.elasticsearch.indices.analysis.AnalysisModule; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; +import org.elasticsearch.indices.mapper.MapperRegistry; +import org.elasticsearch.node.InternalSettingsPreparer; +import org.elasticsearch.plugins.MapperPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.PluginsService; +import org.elasticsearch.plugins.ScriptPlugin; +import org.elasticsearch.plugins.SearchPlugin; +import org.elasticsearch.script.ScriptModule; +import org.elasticsearch.script.ScriptService; +import org.elasticsearch.search.SearchModule; +import org.elasticsearch.search.internal.SearchContext; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; + +import java.io.Closeable; +import java.io.IOException; +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.function.Function; +import java.util.stream.Stream; + +import static java.util.Collections.emptyList; +import static java.util.stream.Collectors.toList; + +public abstract class AbstractBuilderTestCase extends ESTestCase { + + public static final String STRING_FIELD_NAME = "mapped_string"; + protected static final String STRING_FIELD_NAME_2 = "mapped_string_2"; + protected static final String INT_FIELD_NAME = "mapped_int"; + protected static final String INT_RANGE_FIELD_NAME = "mapped_int_range"; + protected static final String DOUBLE_FIELD_NAME = "mapped_double"; + protected static final String BOOLEAN_FIELD_NAME = "mapped_boolean"; + protected static final String DATE_FIELD_NAME = "mapped_date"; + protected static final String DATE_RANGE_FIELD_NAME = "mapped_date_range"; + protected static final String OBJECT_FIELD_NAME = "mapped_object"; + protected static final String GEO_POINT_FIELD_NAME = "mapped_geo_point"; + protected static final String GEO_SHAPE_FIELD_NAME = "mapped_geo_shape"; + protected static final String[] MAPPED_FIELD_NAMES = new String[]{STRING_FIELD_NAME, INT_FIELD_NAME, INT_RANGE_FIELD_NAME, + DOUBLE_FIELD_NAME, BOOLEAN_FIELD_NAME, DATE_FIELD_NAME, DATE_RANGE_FIELD_NAME, OBJECT_FIELD_NAME, GEO_POINT_FIELD_NAME, + GEO_SHAPE_FIELD_NAME}; + protected static final String[] MAPPED_LEAF_FIELD_NAMES = new String[]{STRING_FIELD_NAME, INT_FIELD_NAME, INT_RANGE_FIELD_NAME, + DOUBLE_FIELD_NAME, BOOLEAN_FIELD_NAME, DATE_FIELD_NAME, DATE_RANGE_FIELD_NAME, GEO_POINT_FIELD_NAME, }; + + protected static Version indexVersionCreated; + + private static ServiceHolder serviceHolder; + private static int queryNameId = 0; + private static Settings nodeSettings; + private static Index index; + private static String[] currentTypes; + protected static String[] randomTypes; + + protected static Index getIndex() { + return index; + } + + protected static String[] getCurrentTypes() { + return currentTypes; + } + + protected Collection> getPlugins() { + return Collections.emptyList(); + } + + protected void initializeAdditionalMappings(MapperService mapperService) throws IOException { + } + + @BeforeClass + public static void beforeClass() { + nodeSettings = Settings.builder() + .put("node.name", AbstractQueryTestCase.class.toString()) + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()) + .build(); + + index = new Index(randomAlphaOfLengthBetween(1, 10), "_na_"); + + // Set a single type in the index + switch (random().nextInt(3)) { + case 0: + currentTypes = new String[0]; // no types + break; + default: + currentTypes = new String[] { "_doc" }; + break; + } + randomTypes = getRandomTypes(); + } + + private static String[] getRandomTypes() { + String[] types; + if (currentTypes.length > 0 && randomBoolean()) { + int numberOfQueryTypes = randomIntBetween(1, currentTypes.length); + types = new String[numberOfQueryTypes]; + for (int i = 0; i < numberOfQueryTypes; i++) { + types[i] = randomFrom(currentTypes); + } + } else { + if (randomBoolean()) { + types = new String[]{MetaData.ALL}; + } else { + types = new String[0]; + } + } + return types; + } + + @Override + protected NamedXContentRegistry xContentRegistry() { + return serviceHolder.xContentRegistry; + } + + protected NamedWriteableRegistry namedWriteableRegistry() { + return serviceHolder.namedWriteableRegistry; + } + + /** + * make sure query names are unique by suffixing them with increasing counter + */ + protected static String createUniqueRandomName() { + String queryName = randomAlphaOfLengthBetween(1, 10) + queryNameId; + queryNameId++; + return queryName; + } + + protected Settings indexSettings() { + // we have to prefer CURRENT since with the range of versions we support it's rather unlikely to get the current actually. + indexVersionCreated = randomBoolean() ? Version.CURRENT + : VersionUtils.randomVersionBetween(random(), null, Version.CURRENT); + return Settings.builder() + .put(IndexMetaData.SETTING_VERSION_CREATED, indexVersionCreated) + .build(); + } + + @AfterClass + public static void afterClass() throws Exception { + IOUtils.close(serviceHolder); + serviceHolder = null; + } + + @Before + public void beforeTest() throws IOException { + if (serviceHolder == null) { + serviceHolder = new ServiceHolder(nodeSettings, indexSettings(), getPlugins(), this); + } + serviceHolder.clientInvocationHandler.delegate = this; + } + + protected static SearchContext getSearchContext(String[] types, QueryShardContext context) { + TestSearchContext testSearchContext = new TestSearchContext(context) { + @Override + public MapperService mapperService() { + return serviceHolder.mapperService; // need to build / parse inner hits sort fields + } + + @Override + public > IFD getForField(MappedFieldType fieldType) { + return serviceHolder.indexFieldDataService.getForField(fieldType); // need to build / parse inner hits sort fields + } + + }; + testSearchContext.getQueryShardContext().setTypes(types); + return testSearchContext; + } + + @After + public void afterTest() { + serviceHolder.clientInvocationHandler.delegate = null; + } + + /** + * Override this to handle {@link Client#get(GetRequest)} calls from parsers / builders + */ + protected GetResponse executeGet(GetRequest getRequest) { + throw new UnsupportedOperationException("this test can't handle GET requests"); + } + + /** + * Override this to handle {@link Client#get(GetRequest)} calls from parsers / builders + */ + protected MultiTermVectorsResponse executeMultiTermVectors(MultiTermVectorsRequest mtvRequest) { + throw new UnsupportedOperationException("this test can't handle MultiTermVector requests"); + } + + /** + * @return a new {@link QueryShardContext} with the provided reader + */ + protected static QueryShardContext createShardContext(IndexReader reader) { + return serviceHolder.createShardContext(reader); + } + + /** + * @return a new {@link QueryShardContext} based on the base test index and queryParserService + */ + protected static QueryShardContext createShardContext() { + return createShardContext(null); + } + + private static class ClientInvocationHandler implements InvocationHandler { + AbstractBuilderTestCase delegate; + + @Override + public Object invoke(Object proxy, Method method, Object[] args) throws Throwable { + if (method.equals(Client.class.getMethod("get", GetRequest.class, ActionListener.class))){ + GetResponse getResponse = delegate.executeGet((GetRequest) args[0]); + ActionListener listener = (ActionListener) args[1]; + if (randomBoolean()) { + listener.onResponse(getResponse); + } else { + new Thread(() -> listener.onResponse(getResponse)).start(); + } + return null; + } else if (method.equals(Client.class.getMethod + ("multiTermVectors", MultiTermVectorsRequest.class))) { + return new PlainActionFuture() { + @Override + public MultiTermVectorsResponse get() throws InterruptedException, ExecutionException { + return delegate.executeMultiTermVectors((MultiTermVectorsRequest) args[0]); + } + }; + } else if (method.equals(Object.class.getMethod("toString"))) { + return "MockClient"; + } + throw new UnsupportedOperationException("this test can't handle calls to: " + method); + } + + } + + private static class ServiceHolder implements Closeable { + private final IndexFieldDataService indexFieldDataService; + private final SearchModule searchModule; + private final NamedWriteableRegistry namedWriteableRegistry; + private final NamedXContentRegistry xContentRegistry; + private final ClientInvocationHandler clientInvocationHandler = new ClientInvocationHandler(); + private final IndexSettings idxSettings; + private final SimilarityService similarityService; + private final MapperService mapperService; + private final BitsetFilterCache bitsetFilterCache; + private final ScriptService scriptService; + private final Client client; + private final long nowInMillis = randomNonNegativeLong(); + + ServiceHolder(Settings nodeSettings, Settings indexSettings, + Collection> plugins, AbstractBuilderTestCase testCase) throws IOException { + Environment env = InternalSettingsPreparer.prepareEnvironment(nodeSettings); + PluginsService pluginsService; + pluginsService = new PluginsService(nodeSettings, null, env.modulesFile(), env.pluginsFile(), plugins); + + client = (Client) Proxy.newProxyInstance( + Client.class.getClassLoader(), + new Class[]{Client.class}, + clientInvocationHandler); + ScriptModule scriptModule = createScriptModule(pluginsService.filterPlugins(ScriptPlugin.class)); + List> additionalSettings = pluginsService.getPluginSettings(); + additionalSettings.add(InternalSettingsPlugin.VERSION_CREATED); + SettingsModule settingsModule = new SettingsModule(nodeSettings, additionalSettings, pluginsService.getPluginSettingsFilter()); + searchModule = new SearchModule(nodeSettings, false, pluginsService.filterPlugins(SearchPlugin.class)); + IndicesModule indicesModule = new IndicesModule(pluginsService.filterPlugins(MapperPlugin.class)); + List entries = new ArrayList<>(); + entries.addAll(indicesModule.getNamedWriteables()); + entries.addAll(searchModule.getNamedWriteables()); + namedWriteableRegistry = new NamedWriteableRegistry(entries); + xContentRegistry = new NamedXContentRegistry(Stream.of( + searchModule.getNamedXContents().stream() + ).flatMap(Function.identity()).collect(toList())); + IndexScopedSettings indexScopedSettings = settingsModule.getIndexScopedSettings(); + idxSettings = IndexSettingsModule.newIndexSettings(index, indexSettings, indexScopedSettings); + AnalysisModule analysisModule = new AnalysisModule(TestEnvironment.newEnvironment(nodeSettings), emptyList()); + IndexAnalyzers indexAnalyzers = analysisModule.getAnalysisRegistry().build(idxSettings); + scriptService = scriptModule.getScriptService(); + similarityService = new SimilarityService(idxSettings, null, Collections.emptyMap()); + MapperRegistry mapperRegistry = indicesModule.getMapperRegistry(); + mapperService = new MapperService(idxSettings, indexAnalyzers, xContentRegistry, similarityService, mapperRegistry, + () -> createShardContext(null)); + IndicesFieldDataCache indicesFieldDataCache = new IndicesFieldDataCache(nodeSettings, new IndexFieldDataCache.Listener() { + }); + indexFieldDataService = new IndexFieldDataService(idxSettings, indicesFieldDataCache, + new NoneCircuitBreakerService(), mapperService); + bitsetFilterCache = new BitsetFilterCache(idxSettings, new BitsetFilterCache.Listener() { + @Override + public void onCache(ShardId shardId, Accountable accountable) { + + } + + @Override + public void onRemoval(ShardId shardId, Accountable accountable) { + + } + }); + + for (String type : currentTypes) { + mapperService.merge(type, new CompressedXContent(Strings.toString(PutMappingRequest.buildFromSimplifiedDef(type, + STRING_FIELD_NAME, "type=text", + STRING_FIELD_NAME_2, "type=keyword", + INT_FIELD_NAME, "type=integer", + INT_RANGE_FIELD_NAME, "type=integer_range", + DOUBLE_FIELD_NAME, "type=double", + BOOLEAN_FIELD_NAME, "type=boolean", + DATE_FIELD_NAME, "type=date", + DATE_RANGE_FIELD_NAME, "type=date_range", + OBJECT_FIELD_NAME, "type=object", + GEO_POINT_FIELD_NAME, "type=geo_point", + GEO_SHAPE_FIELD_NAME, "type=geo_shape" + ))), MapperService.MergeReason.MAPPING_UPDATE); + // also add mappings for two inner field in the object field + mapperService.merge(type, new CompressedXContent("{\"properties\":{\"" + OBJECT_FIELD_NAME + "\":{\"type\":\"object\"," + + "\"properties\":{\"" + DATE_FIELD_NAME + "\":{\"type\":\"date\"},\"" + + INT_FIELD_NAME + "\":{\"type\":\"integer\"}}}}}"), + MapperService.MergeReason.MAPPING_UPDATE); + } + testCase.initializeAdditionalMappings(mapperService); + } + + @Override + public void close() throws IOException { + } + + QueryShardContext createShardContext(IndexReader reader) { + return new QueryShardContext(0, idxSettings, bitsetFilterCache, indexFieldDataService::getForField, mapperService, + similarityService, scriptService, xContentRegistry, namedWriteableRegistry, this.client, reader, () -> nowInMillis, null); + } + + ScriptModule createScriptModule(List scriptPlugins) { + if (scriptPlugins == null || scriptPlugins.isEmpty()) { + return newTestScriptModule(); + } + return new ScriptModule(Settings.EMPTY, scriptPlugins); + } + } + +} diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java index 1c02f960143ce..d2f3a56aebe3d 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java @@ -25,33 +25,17 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.spans.SpanBoostQuery; -import org.apache.lucene.util.Accountable; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.Version; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; -import org.elasticsearch.action.get.GetRequest; -import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.support.PlainActionFuture; -import org.elasticsearch.action.termvectors.MultiTermVectorsRequest; -import org.elasticsearch.action.termvectors.MultiTermVectorsResponse; -import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; -import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.Writeable.Reader; -import org.elasticsearch.common.settings.IndexScopedSettings; -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.settings.SettingsModule; import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.common.xcontent.DeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; @@ -64,55 +48,18 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.env.Environment; -import org.elasticsearch.env.TestEnvironment; -import org.elasticsearch.index.Index; -import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.analysis.IndexAnalyzers; -import org.elasticsearch.index.cache.bitset.BitsetFilterCache; -import org.elasticsearch.index.fielddata.IndexFieldData; -import org.elasticsearch.index.fielddata.IndexFieldDataCache; -import org.elasticsearch.index.fielddata.IndexFieldDataService; -import org.elasticsearch.index.mapper.MappedFieldType; -import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.query.AbstractQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryRewriteContext; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.query.Rewriteable; import org.elasticsearch.index.query.support.QueryParsers; -import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.index.similarity.SimilarityService; -import org.elasticsearch.indices.IndicesModule; -import org.elasticsearch.indices.analysis.AnalysisModule; -import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; -import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; -import org.elasticsearch.indices.mapper.MapperRegistry; -import org.elasticsearch.node.InternalSettingsPreparer; -import org.elasticsearch.plugins.MapperPlugin; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.plugins.PluginsService; -import org.elasticsearch.plugins.ScriptPlugin; -import org.elasticsearch.plugins.SearchPlugin; -import org.elasticsearch.script.ScriptModule; -import org.elasticsearch.script.ScriptService; -import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.internal.SearchContext; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import java.io.Closeable; import java.io.IOException; -import java.lang.reflect.InvocationHandler; -import java.lang.reflect.Method; -import java.lang.reflect.Proxy; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.Deque; import java.util.HashSet; @@ -121,12 +68,7 @@ import java.util.Locale; import java.util.Map; import java.util.Set; -import java.util.concurrent.ExecutionException; -import java.util.function.Function; -import java.util.stream.Stream; -import static java.util.Collections.emptyList; -import static java.util.stream.Collectors.toList; import static org.elasticsearch.index.query.AbstractQueryBuilder.parseInnerQueryBuilder; import static org.elasticsearch.test.EqualsHashCodeTestUtils.checkEqualsAndHashCode; import static org.hamcrest.CoreMatchers.equalTo; @@ -136,115 +78,9 @@ import static org.hamcrest.Matchers.instanceOf; -public abstract class AbstractQueryTestCase> extends ESTestCase { - - public static final String STRING_FIELD_NAME = "mapped_string"; - protected static final String STRING_FIELD_NAME_2 = "mapped_string_2"; - protected static final String INT_FIELD_NAME = "mapped_int"; - protected static final String INT_RANGE_FIELD_NAME = "mapped_int_range"; - protected static final String DOUBLE_FIELD_NAME = "mapped_double"; - protected static final String BOOLEAN_FIELD_NAME = "mapped_boolean"; - protected static final String DATE_FIELD_NAME = "mapped_date"; - protected static final String DATE_RANGE_FIELD_NAME = "mapped_date_range"; - protected static final String OBJECT_FIELD_NAME = "mapped_object"; - protected static final String GEO_POINT_FIELD_NAME = "mapped_geo_point"; - protected static final String GEO_SHAPE_FIELD_NAME = "mapped_geo_shape"; - protected static final String[] MAPPED_FIELD_NAMES = new String[]{STRING_FIELD_NAME, INT_FIELD_NAME, INT_RANGE_FIELD_NAME, - DOUBLE_FIELD_NAME, BOOLEAN_FIELD_NAME, DATE_FIELD_NAME, DATE_RANGE_FIELD_NAME, OBJECT_FIELD_NAME, GEO_POINT_FIELD_NAME, - GEO_SHAPE_FIELD_NAME}; - private static final String[] MAPPED_LEAF_FIELD_NAMES = new String[]{STRING_FIELD_NAME, INT_FIELD_NAME, INT_RANGE_FIELD_NAME, - DOUBLE_FIELD_NAME, BOOLEAN_FIELD_NAME, DATE_FIELD_NAME, DATE_RANGE_FIELD_NAME, GEO_POINT_FIELD_NAME, }; - private static final int NUMBER_OF_TESTQUERIES = 20; - - protected static Version indexVersionCreated; - - private static ServiceHolder serviceHolder; - private static int queryNameId = 0; - private static Settings nodeSettings; - private static Index index; - private static String[] currentTypes; - private static String[] randomTypes; - - protected static Index getIndex() { - return index; - } - - protected static String[] getCurrentTypes() { - return currentTypes; - } - - protected Collection> getPlugins() { - return Collections.emptyList(); - } - - protected void initializeAdditionalMappings(MapperService mapperService) throws IOException { - } - - @BeforeClass - public static void beforeClass() { - nodeSettings = Settings.builder() - .put("node.name", AbstractQueryTestCase.class.toString()) - .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()) - .build(); - - index = new Index(randomAlphaOfLengthBetween(1, 10), "_na_"); - - // Set a single type in the index - switch (random().nextInt(3)) { - case 0: - currentTypes = new String[0]; // no types - break; - default: - currentTypes = new String[] { "_doc" }; - break; - } - randomTypes = getRandomTypes(); - } - - protected Settings indexSettings() { - // we have to prefer CURRENT since with the range of versions we support it's rather unlikely to get the current actually. - indexVersionCreated = randomBoolean() ? Version.CURRENT - : VersionUtils.randomVersionBetween(random(), null, Version.CURRENT); - return Settings.builder() - .put(IndexMetaData.SETTING_VERSION_CREATED, indexVersionCreated) - .build(); - } - - @AfterClass - public static void afterClass() throws Exception { - IOUtils.close(serviceHolder); - serviceHolder = null; - } - - @Before - public void beforeTest() throws IOException { - if (serviceHolder == null) { - serviceHolder = new ServiceHolder(nodeSettings, indexSettings(), getPlugins(), this); - } - serviceHolder.clientInvocationHandler.delegate = this; - } +public abstract class AbstractQueryTestCase> extends AbstractBuilderTestCase { - private static SearchContext getSearchContext(String[] types, QueryShardContext context) { - TestSearchContext testSearchContext = new TestSearchContext(context) { - @Override - public MapperService mapperService() { - return serviceHolder.mapperService; // need to build / parse inner hits sort fields - } - - @Override - public > IFD getForField(MappedFieldType fieldType) { - return serviceHolder.indexFieldDataService.getForField(fieldType); // need to build / parse inner hits sort fields - } - - }; - testSearchContext.getQueryShardContext().setTypes(types); - return testSearchContext; - } - - @After - public void afterTest() { - serviceHolder.clientInvocationHandler.delegate = null; - } + private static final int NUMBER_OF_TESTQUERIES = 20; public final QB createTestQueryBuilder() { QB query = doCreateTestQueryBuilder(); @@ -260,15 +96,6 @@ public final QB createTestQueryBuilder() { return query; } - /** - * make sure query names are unique by suffixing them with increasing counter - */ - private static String createUniqueRandomName() { - String queryName = randomAlphaOfLengthBetween(1, 10) + queryNameId; - queryNameId++; - return queryName; - } - /** * Create the query that is being tested */ @@ -717,18 +544,18 @@ public void testSerialization() throws IOException { } } - protected static QueryBuilder assertSerialization(QueryBuilder testQuery) throws IOException { + protected QueryBuilder assertSerialization(QueryBuilder testQuery) throws IOException { return assertSerialization(testQuery, Version.CURRENT); } /** * Serialize the given query builder and asserts that both are equal */ - protected static QueryBuilder assertSerialization(QueryBuilder testQuery, Version version) throws IOException { + protected QueryBuilder assertSerialization(QueryBuilder testQuery, Version version) throws IOException { try (BytesStreamOutput output = new BytesStreamOutput()) { output.setVersion(version); output.writeNamedWriteable(testQuery); - try (StreamInput in = new NamedWriteableAwareStreamInput(output.bytes().streamInput(), serviceHolder.namedWriteableRegistry)) { + try (StreamInput in = new NamedWriteableAwareStreamInput(output.bytes().streamInput(), namedWriteableRegistry())) { in.setVersion(version); QueryBuilder deserializedQuery = in.readNamedWriteable(QueryBuilder.class); assertEquals(testQuery, deserializedQuery); @@ -780,15 +607,8 @@ protected QB changeNameOrBoost(QB original) throws IOException { //we use the streaming infra to create a copy of the query provided as argument @SuppressWarnings("unchecked") private QB copyQuery(QB query) throws IOException { - Reader reader = (Reader) serviceHolder.namedWriteableRegistry.getReader(QueryBuilder.class, query.getWriteableName()); - return copyWriteable(query, serviceHolder.namedWriteableRegistry, reader); - } - - /** - * @return a new {@link QueryShardContext} based on the base test index and queryParserService - */ - protected static QueryShardContext createShardContext() { - return serviceHolder.createShardContext(); + Reader reader = (Reader) namedWriteableRegistry().getReader(QueryBuilder.class, query.getWriteableName()); + return copyWriteable(query, namedWriteableRegistry(), reader); } /** @@ -840,7 +660,7 @@ protected static String getRandomQueryText() { */ protected static String getRandomFieldName() { // if no type is set then return a random field name - if (currentTypes.length == 0 || randomBoolean()) { + if (getCurrentTypes().length == 0 || randomBoolean()) { return randomAlphaOfLengthBetween(1, 10); } return randomFrom(MAPPED_LEAF_FIELD_NAMES); @@ -863,24 +683,6 @@ protected static String getRandomRewriteMethod() { return rewrite; } - private static String[] getRandomTypes() { - String[] types; - if (currentTypes.length > 0 && randomBoolean()) { - int numberOfQueryTypes = randomIntBetween(1, currentTypes.length); - types = new String[numberOfQueryTypes]; - for (int i = 0; i < numberOfQueryTypes; i++) { - types[i] = randomFrom(currentTypes); - } - } else { - if (randomBoolean()) { - types = new String[]{MetaData.ALL}; - } else { - types = new String[0]; - } - } - return types; - } - protected static Fuzziness randomFuzziness(String fieldName) { switch (fieldName) { case INT_FIELD_NAME: @@ -905,50 +707,6 @@ protected static String randomMinimumShouldMatch() { return randomFrom("1", "-1", "75%", "-25%", "2<75%", "2<-25%"); } - private static class ClientInvocationHandler implements InvocationHandler { - AbstractQueryTestCase delegate; - - @Override - public Object invoke(Object proxy, Method method, Object[] args) throws Throwable { - if (method.equals(Client.class.getMethod("get", GetRequest.class, ActionListener.class))){ - GetResponse getResponse = delegate.executeGet((GetRequest) args[0]); - ActionListener listener = (ActionListener) args[1]; - if (randomBoolean()) { - listener.onResponse(getResponse); - } else { - new Thread(() -> listener.onResponse(getResponse)).start(); - } - return null; - } else if (method.equals(Client.class.getMethod - ("multiTermVectors", MultiTermVectorsRequest.class))) { - return new PlainActionFuture() { - @Override - public MultiTermVectorsResponse get() throws InterruptedException, ExecutionException { - return delegate.executeMultiTermVectors((MultiTermVectorsRequest) args[0]); - } - }; - } else if (method.equals(Object.class.getMethod("toString"))) { - return "MockClient"; - } - throw new UnsupportedOperationException("this test can't handle calls to: " + method); - } - - } - - /** - * Override this to handle {@link Client#get(GetRequest)} calls from parsers / builders - */ - protected GetResponse executeGet(GetRequest getRequest) { - throw new UnsupportedOperationException("this test can't handle GET requests"); - } - - /** - * Override this to handle {@link Client#get(GetRequest)} calls from parsers / builders - */ - protected MultiTermVectorsResponse executeMultiTermVectors(MultiTermVectorsRequest mtvRequest) { - throw new UnsupportedOperationException("this test can't handle MultiTermVector requests"); - } - /** * Call this method to check a valid json string representing the query under test against * it's generated json. @@ -1015,113 +773,6 @@ protected Query rewrite(Query query) throws IOException { return query; } - @Override - protected NamedXContentRegistry xContentRegistry() { - return serviceHolder.xContentRegistry; - } - - private static class ServiceHolder implements Closeable { - private final IndexFieldDataService indexFieldDataService; - private final SearchModule searchModule; - private final NamedWriteableRegistry namedWriteableRegistry; - private final NamedXContentRegistry xContentRegistry; - private final ClientInvocationHandler clientInvocationHandler = new ClientInvocationHandler(); - private final IndexSettings idxSettings; - private final SimilarityService similarityService; - private final MapperService mapperService; - private final BitsetFilterCache bitsetFilterCache; - private final ScriptService scriptService; - private final Client client; - private final long nowInMillis = randomNonNegativeLong(); - - ServiceHolder(Settings nodeSettings, Settings indexSettings, - Collection> plugins, AbstractQueryTestCase testCase) throws IOException { - Environment env = InternalSettingsPreparer.prepareEnvironment(nodeSettings); - PluginsService pluginsService; - pluginsService = new PluginsService(nodeSettings, null, env.modulesFile(), env.pluginsFile(), plugins); - - client = (Client) Proxy.newProxyInstance( - Client.class.getClassLoader(), - new Class[]{Client.class}, - clientInvocationHandler); - ScriptModule scriptModule = createScriptModule(pluginsService.filterPlugins(ScriptPlugin.class)); - List> additionalSettings = pluginsService.getPluginSettings(); - additionalSettings.add(InternalSettingsPlugin.VERSION_CREATED); - SettingsModule settingsModule = new SettingsModule(nodeSettings, additionalSettings, pluginsService.getPluginSettingsFilter()); - searchModule = new SearchModule(nodeSettings, false, pluginsService.filterPlugins(SearchPlugin.class)); - IndicesModule indicesModule = new IndicesModule(pluginsService.filterPlugins(MapperPlugin.class)); - List entries = new ArrayList<>(); - entries.addAll(indicesModule.getNamedWriteables()); - entries.addAll(searchModule.getNamedWriteables()); - namedWriteableRegistry = new NamedWriteableRegistry(entries); - xContentRegistry = new NamedXContentRegistry(Stream.of( - searchModule.getNamedXContents().stream() - ).flatMap(Function.identity()).collect(toList())); - IndexScopedSettings indexScopedSettings = settingsModule.getIndexScopedSettings(); - idxSettings = IndexSettingsModule.newIndexSettings(index, indexSettings, indexScopedSettings); - AnalysisModule analysisModule = new AnalysisModule(TestEnvironment.newEnvironment(nodeSettings), emptyList()); - IndexAnalyzers indexAnalyzers = analysisModule.getAnalysisRegistry().build(idxSettings); - scriptService = scriptModule.getScriptService(); - similarityService = new SimilarityService(idxSettings, null, Collections.emptyMap()); - MapperRegistry mapperRegistry = indicesModule.getMapperRegistry(); - mapperService = new MapperService(idxSettings, indexAnalyzers, xContentRegistry, similarityService, mapperRegistry, - this::createShardContext); - IndicesFieldDataCache indicesFieldDataCache = new IndicesFieldDataCache(nodeSettings, new IndexFieldDataCache.Listener() { - }); - indexFieldDataService = new IndexFieldDataService(idxSettings, indicesFieldDataCache, - new NoneCircuitBreakerService(), mapperService); - bitsetFilterCache = new BitsetFilterCache(idxSettings, new BitsetFilterCache.Listener() { - @Override - public void onCache(ShardId shardId, Accountable accountable) { - - } - - @Override - public void onRemoval(ShardId shardId, Accountable accountable) { - - } - }); - - for (String type : currentTypes) { - mapperService.merge(type, new CompressedXContent(Strings.toString(PutMappingRequest.buildFromSimplifiedDef(type, - STRING_FIELD_NAME, "type=text", - STRING_FIELD_NAME_2, "type=keyword", - INT_FIELD_NAME, "type=integer", - INT_RANGE_FIELD_NAME, "type=integer_range", - DOUBLE_FIELD_NAME, "type=double", - BOOLEAN_FIELD_NAME, "type=boolean", - DATE_FIELD_NAME, "type=date", - DATE_RANGE_FIELD_NAME, "type=date_range", - OBJECT_FIELD_NAME, "type=object", - GEO_POINT_FIELD_NAME, "type=geo_point", - GEO_SHAPE_FIELD_NAME, "type=geo_shape" - ))), MapperService.MergeReason.MAPPING_UPDATE); - // also add mappings for two inner field in the object field - mapperService.merge(type, new CompressedXContent("{\"properties\":{\"" + OBJECT_FIELD_NAME + "\":{\"type\":\"object\"," - + "\"properties\":{\"" + DATE_FIELD_NAME + "\":{\"type\":\"date\"},\"" + - INT_FIELD_NAME + "\":{\"type\":\"integer\"}}}}}"), - MapperService.MergeReason.MAPPING_UPDATE); - } - testCase.initializeAdditionalMappings(mapperService); - } - - @Override - public void close() throws IOException { - } - - QueryShardContext createShardContext() { - return new QueryShardContext(0, idxSettings, bitsetFilterCache, indexFieldDataService::getForField, mapperService, - similarityService, scriptService, xContentRegistry, namedWriteableRegistry, this.client, null, () -> nowInMillis, null); - } - - ScriptModule createScriptModule(List scriptPlugins) { - if (scriptPlugins == null || scriptPlugins.isEmpty()) { - return newTestScriptModule(); - } - return new ScriptModule(Settings.EMPTY, scriptPlugins); - } - } - protected QueryBuilder rewriteAndFetch(QueryBuilder builder, QueryRewriteContext context) throws IOException { PlainActionFuture future = new PlainActionFuture<>(); Rewriteable.rewriteAndFetch(builder, context, future);