parent
bf1a60130d
commit
eb0b28dd1d
|
@ -65,6 +65,7 @@ import java.util.List;
|
|||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.LongSupplier;
|
||||
|
||||
import static org.elasticsearch.common.time.DateUtils.toLong;
|
||||
|
@ -470,6 +471,14 @@ public final class DateFieldMapper extends FieldMapper {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<byte[], Number> pointReaderIfPossible() {
|
||||
if (isSearchable()) {
|
||||
return resolution()::parsePointAsMillis;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexFieldData.Builder fielddataBuilder(String fullyQualifiedIndexName) {
|
||||
failIfNoDocValues();
|
||||
|
|
|
@ -53,6 +53,7 @@ import java.time.ZoneId;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* This defines the core properties and functions to operate on a field.
|
||||
|
@ -136,6 +137,17 @@ public abstract class MappedFieldType {
|
|||
return isIndexed;
|
||||
}
|
||||
|
||||
/**
|
||||
* If the field supports using the indexed data to speed up operations related to ordering of data, such as sorting or aggs, return
|
||||
* a function for doing that. If it is unsupported for this field type, there is no need to override this method.
|
||||
*
|
||||
* @return null if the optimization cannot be applied, otherwise a function to use for the optimization
|
||||
*/
|
||||
@Nullable
|
||||
public Function<byte[], Number> pointReaderIfPossible() {
|
||||
return null;
|
||||
}
|
||||
|
||||
/** Returns true if the field is aggregatable.
|
||||
*
|
||||
*/
|
||||
|
|
|
@ -66,6 +66,7 @@ import java.util.Iterator;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.function.Function;
|
||||
|
||||
/** A {@link FieldMapper} for numeric types: byte, short, int, long, float and double. */
|
||||
public class NumberFieldMapper extends FieldMapper {
|
||||
|
@ -961,6 +962,14 @@ public class NumberFieldMapper extends FieldMapper {
|
|||
return query;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<byte[], Number> pointReaderIfPossible() {
|
||||
if (isSearchable()) {
|
||||
return this::parsePoint;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexFieldData.Builder fielddataBuilder(String fullyQualifiedIndexName) {
|
||||
failIfNoDocValues();
|
||||
|
|
|
@ -19,11 +19,13 @@
|
|||
package org.elasticsearch.search.aggregations;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.ScoreMode;
|
||||
import org.elasticsearch.common.breaker.CircuitBreaker;
|
||||
import org.elasticsearch.common.breaker.CircuitBreakingException;
|
||||
import org.elasticsearch.indices.breaker.CircuitBreakerService;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.search.internal.SearchContext.Lifetime;
|
||||
import org.elasticsearch.search.query.QueryPhaseExecutionException;
|
||||
|
@ -34,6 +36,7 @@ import java.util.Arrays;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* Base implementation for concrete aggregators.
|
||||
|
@ -106,6 +109,26 @@ public abstract class AggregatorBase extends Aggregator {
|
|||
addRequestCircuitBreakerBytes(DEFAULT_WEIGHT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a converter for point values if it's safe to use the indexed data instead of
|
||||
* doc values. Generally, this means that the query has no filters or scripts, the aggregation is
|
||||
* top level, and the underlying field is indexed, and the index is sorted in the right order.
|
||||
*
|
||||
* If those conditions aren't met, return <code>null</code> to indicate a point reader cannot
|
||||
* be used in this case.
|
||||
*
|
||||
* @param config The config for the values source metric.
|
||||
*/
|
||||
public final Function<byte[], Number> pointReaderIfAvailable(ValuesSourceConfig config) {
|
||||
if (context.query() != null && context.query().getClass() != MatchAllDocsQuery.class) {
|
||||
return null;
|
||||
}
|
||||
if (parent != null) {
|
||||
return null;
|
||||
}
|
||||
return config.getPointReaderOrNull();
|
||||
}
|
||||
|
||||
/**
|
||||
* Increment or decrement the number of bytes that have been allocated to service
|
||||
* this request and potentially trigger a {@link CircuitBreakingException}. The
|
||||
|
|
|
@ -44,8 +44,6 @@ import java.io.IOException;
|
|||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
|
||||
import static org.elasticsearch.search.aggregations.metrics.MinAggregator.getPointReaderOrNull;
|
||||
|
||||
class MaxAggregator extends NumericMetricsAggregator.SingleValue {
|
||||
|
||||
final ValuesSource.Numeric valuesSource;
|
||||
|
@ -68,7 +66,7 @@ class MaxAggregator extends NumericMetricsAggregator.SingleValue {
|
|||
maxes.fill(0, maxes.size(), Double.NEGATIVE_INFINITY);
|
||||
}
|
||||
this.formatter = config.format();
|
||||
this.pointConverter = getPointReaderOrNull(context, parent, config);
|
||||
this.pointConverter = pointReaderIfAvailable(config);
|
||||
if (pointConverter != null) {
|
||||
pointField = config.fieldContext().field();
|
||||
} else {
|
||||
|
@ -96,7 +94,7 @@ class MaxAggregator extends NumericMetricsAggregator.SingleValue {
|
|||
Number segMax = findLeafMaxValue(ctx.reader(), pointField, pointConverter);
|
||||
if (segMax != null) {
|
||||
/*
|
||||
* There is no parent aggregator (see {@link MinAggregator#getPointReaderOrNull}
|
||||
* There is no parent aggregator (see {@link AggregatorBase#getPointReaderOrNull}
|
||||
* so the ordinal for the bucket is always 0.
|
||||
*/
|
||||
assert maxes.size() == 1;
|
||||
|
|
|
@ -22,7 +22,6 @@ import org.apache.lucene.index.LeafReader;
|
|||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.search.CollectionTerminatedException;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.ScoreMode;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
|
@ -30,9 +29,6 @@ import org.elasticsearch.common.util.BigArrays;
|
|||
import org.elasticsearch.common.util.DoubleArray;
|
||||
import org.elasticsearch.index.fielddata.NumericDoubleValues;
|
||||
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
|
||||
import org.elasticsearch.index.mapper.DateFieldMapper;
|
||||
import org.elasticsearch.index.mapper.MappedFieldType;
|
||||
import org.elasticsearch.index.mapper.NumberFieldMapper;
|
||||
import org.elasticsearch.search.DocValueFormat;
|
||||
import org.elasticsearch.search.MultiValueMode;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
|
@ -71,7 +67,7 @@ class MinAggregator extends NumericMetricsAggregator.SingleValue {
|
|||
mins.fill(0, mins.size(), Double.POSITIVE_INFINITY);
|
||||
}
|
||||
this.format = config.format();
|
||||
this.pointConverter = getPointReaderOrNull(context, parent, config);
|
||||
this.pointConverter = pointReaderIfAvailable(config);
|
||||
if (pointConverter != null) {
|
||||
pointField = config.fieldContext().field();
|
||||
} else {
|
||||
|
@ -159,40 +155,6 @@ class MinAggregator extends NumericMetricsAggregator.SingleValue {
|
|||
}
|
||||
|
||||
|
||||
/**
|
||||
* Returns a converter for point values if early termination is applicable to
|
||||
* the context or <code>null</code> otherwise.
|
||||
*
|
||||
* @param context The {@link SearchContext} of the aggregation.
|
||||
* @param parent The parent aggregator.
|
||||
* @param config The config for the values source metric.
|
||||
*/
|
||||
static Function<byte[], Number> getPointReaderOrNull(SearchContext context, Aggregator parent,
|
||||
ValuesSourceConfig config) {
|
||||
if (context.query() != null &&
|
||||
context.query().getClass() != MatchAllDocsQuery.class) {
|
||||
return null;
|
||||
}
|
||||
if (parent != null) {
|
||||
return null;
|
||||
}
|
||||
if (config.fieldContext() != null && config.script() == null && config.missing() == null) {
|
||||
MappedFieldType fieldType = config.fieldContext().fieldType();
|
||||
if (fieldType == null || fieldType.isSearchable() == false) {
|
||||
return null;
|
||||
}
|
||||
Function<byte[], Number> converter = null;
|
||||
if (fieldType instanceof NumberFieldMapper.NumberFieldType) {
|
||||
converter = ((NumberFieldMapper.NumberFieldType) fieldType)::parsePoint;
|
||||
} else if (fieldType.getClass() == DateFieldMapper.DateFieldType.class) {
|
||||
DateFieldMapper.DateFieldType dft = (DateFieldMapper.DateFieldType) fieldType;
|
||||
converter = dft.resolution()::parsePointAsMillis;
|
||||
}
|
||||
return converter;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the minimum value indexed in the <code>fieldName</code> field or <code>null</code>
|
||||
* if the value cannot be inferred from the indexed {@link PointValues}.
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.elasticsearch.script.Script;
|
|||
import org.elasticsearch.search.DocValueFormat;
|
||||
|
||||
import java.time.ZoneId;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.LongSupplier;
|
||||
|
||||
/**
|
||||
|
@ -371,4 +372,36 @@ public class ValuesSourceConfig {
|
|||
public boolean hasGlobalOrdinals() {
|
||||
return valuesSource.hasGlobalOrdinals();
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is used when an aggregation can optimize by using the indexed data instead of the doc values. We check to see if the
|
||||
* indexed data will match the values source output (meaning there isn't a script or a missing value, since both could modify the
|
||||
* value at read time). If the settings allow for it, we then ask the {@link ValuesSourceType} to build the actual point reader
|
||||
* based on the field type. This allows for a point of extensibility in plugins.
|
||||
*
|
||||
* @return null if we cannot apply the optimization, otherwise the point reader function.
|
||||
*/
|
||||
@Nullable
|
||||
public Function<byte[], Number> getPointReaderOrNull() {
|
||||
MappedFieldType fieldType = fieldType();
|
||||
if (fieldType != null && script() == null && missing() == null) {
|
||||
return fieldType.pointReaderIfPossible();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a human readable description of this values source, for use in error messages and similar.
|
||||
*/
|
||||
public String getDescription() {
|
||||
if (script != null) {
|
||||
return "Script yielding [" + (scriptValueType != null ? scriptValueType.getPreferredName() : "unknown type") + "]";
|
||||
}
|
||||
|
||||
MappedFieldType fieldType = fieldType();
|
||||
if (fieldType != null) {
|
||||
return "Field [" + fieldType.name() + "] of type [" + fieldType.typeName() + "]";
|
||||
}
|
||||
return "unmapped field";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,189 @@
|
|||
/*
|
||||
* 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.apache.lucene.document.LongPoint;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||
import org.elasticsearch.common.breaker.CircuitBreaker;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.mapper.ContentPath;
|
||||
import org.elasticsearch.index.mapper.DateFieldMapper;
|
||||
import org.elasticsearch.index.mapper.MappedFieldType;
|
||||
import org.elasticsearch.index.mapper.Mapper;
|
||||
import org.elasticsearch.index.mapper.NumberFieldMapper;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
import org.elasticsearch.indices.breaker.CircuitBreakerService;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.test.ESSingleNodeTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.Instant;
|
||||
import java.util.Collections;
|
||||
import java.util.function.Function;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class AggregatorBaseTests extends ESSingleNodeTestCase {
|
||||
|
||||
class BogusAggregator extends AggregatorBase {
|
||||
BogusAggregator(SearchContext searchContext, Aggregator parent) throws IOException {
|
||||
super("bogus", AggregatorFactories.EMPTY, searchContext, parent, CardinalityUpperBound.NONE, Collections.emptyMap());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public InternalAggregation buildEmptyAggregation() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
private SearchContext mockSearchContext(Query query) {
|
||||
SearchContext searchContext = mock(SearchContext.class);
|
||||
when(searchContext.query()).thenReturn(query);
|
||||
BigArrays mockBigArrays = mock(BigArrays.class);
|
||||
CircuitBreakerService mockCBS = mock(CircuitBreakerService.class);
|
||||
when(mockCBS.getBreaker(CircuitBreaker.REQUEST)).thenReturn(mock(CircuitBreaker.class));
|
||||
when(mockBigArrays.breakerService()).thenReturn(mockCBS);
|
||||
when(searchContext.bigArrays()).thenReturn(mockBigArrays);
|
||||
return searchContext;
|
||||
}
|
||||
|
||||
private Function<byte[], Number> pointReaderShim(SearchContext context, Aggregator parent, ValuesSourceConfig config)
|
||||
throws IOException {
|
||||
BogusAggregator aggregator = new BogusAggregator(context, parent);
|
||||
return aggregator.pointReaderIfAvailable(config);
|
||||
}
|
||||
|
||||
private Aggregator mockAggregator() {
|
||||
return mock(Aggregator.class);
|
||||
}
|
||||
|
||||
private ValuesSourceConfig getVSConfig(
|
||||
String fieldName,
|
||||
NumberFieldMapper.NumberType numType,
|
||||
boolean indexed,
|
||||
QueryShardContext context
|
||||
) {
|
||||
MappedFieldType ft = new NumberFieldMapper.NumberFieldType(fieldName, numType, indexed, true, Collections.emptyMap());
|
||||
return ValuesSourceConfig.resolveFieldOnly(ft, context);
|
||||
}
|
||||
|
||||
private ValuesSourceConfig getVSConfig(
|
||||
String fieldName,
|
||||
DateFieldMapper.Resolution resolution,
|
||||
boolean indexed,
|
||||
QueryShardContext context
|
||||
) {
|
||||
Mapper.BuilderContext builderContext = new Mapper.BuilderContext(
|
||||
Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT).build(),
|
||||
new ContentPath()
|
||||
);
|
||||
MappedFieldType ft = new DateFieldMapper.Builder(fieldName).index(indexed)
|
||||
.withResolution(resolution)
|
||||
.build(builderContext)
|
||||
.fieldType();
|
||||
return ValuesSourceConfig.resolveFieldOnly(ft, context);
|
||||
}
|
||||
|
||||
public void testShortcutIsApplicable() throws IOException {
|
||||
IndexService indexService = createIndex("index", Settings.EMPTY, "type", "bytes", "type=keyword");
|
||||
|
||||
try (Engine.Searcher searcher = indexService.getShard(0).acquireSearcher("test")) {
|
||||
QueryShardContext context = indexService.newQueryShardContext(0, searcher, () -> 42L, null);
|
||||
|
||||
for (NumberFieldMapper.NumberType type : NumberFieldMapper.NumberType.values()) {
|
||||
assertNotNull(
|
||||
pointReaderShim(mockSearchContext(new MatchAllDocsQuery()), null, getVSConfig("number", type, true, context))
|
||||
);
|
||||
assertNotNull(pointReaderShim(mockSearchContext(null), null, getVSConfig("number", type, true, context)));
|
||||
assertNull(pointReaderShim(mockSearchContext(null), mockAggregator(), getVSConfig("number", type, true, context)));
|
||||
assertNull(
|
||||
pointReaderShim(
|
||||
mockSearchContext(new TermQuery(new Term("foo", "bar"))),
|
||||
null,
|
||||
getVSConfig("number", type, true, context)
|
||||
)
|
||||
);
|
||||
assertNull(pointReaderShim(mockSearchContext(null), mockAggregator(), getVSConfig("number", type, true, context)));
|
||||
assertNull(pointReaderShim(mockSearchContext(null), null, getVSConfig("number", type, false, context)));
|
||||
}
|
||||
for (DateFieldMapper.Resolution resolution : DateFieldMapper.Resolution.values()) {
|
||||
assertNull(
|
||||
pointReaderShim(
|
||||
mockSearchContext(new MatchAllDocsQuery()),
|
||||
mockAggregator(),
|
||||
getVSConfig("number", resolution, true, context)
|
||||
)
|
||||
);
|
||||
assertNull(
|
||||
pointReaderShim(
|
||||
mockSearchContext(new TermQuery(new Term("foo", "bar"))),
|
||||
null,
|
||||
getVSConfig("number", resolution, true, context)
|
||||
)
|
||||
);
|
||||
assertNull(pointReaderShim(mockSearchContext(null), mockAggregator(), getVSConfig("number", resolution, true, context)));
|
||||
assertNull(pointReaderShim(mockSearchContext(null), null, getVSConfig("number", resolution, false, context)));
|
||||
}
|
||||
// Check that we decode a dates "just like" the doc values instance.
|
||||
Instant expected = Instant.from(DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.parse("2020-01-01T00:00:00Z"));
|
||||
byte[] scratch = new byte[8];
|
||||
LongPoint.encodeDimension(DateFieldMapper.Resolution.MILLISECONDS.convert(expected), scratch, 0);
|
||||
assertThat(
|
||||
pointReaderShim(
|
||||
mockSearchContext(new MatchAllDocsQuery()),
|
||||
null,
|
||||
getVSConfig("number", DateFieldMapper.Resolution.MILLISECONDS, true, context)
|
||||
).apply(scratch),
|
||||
equalTo(expected.toEpochMilli())
|
||||
);
|
||||
LongPoint.encodeDimension(DateFieldMapper.Resolution.NANOSECONDS.convert(expected), scratch, 0);
|
||||
assertThat(
|
||||
pointReaderShim(
|
||||
mockSearchContext(new MatchAllDocsQuery()),
|
||||
null,
|
||||
getVSConfig("number", DateFieldMapper.Resolution.NANOSECONDS, true, context)
|
||||
).apply(scratch),
|
||||
equalTo(expected.toEpochMilli())
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -43,22 +43,16 @@ import org.apache.lucene.search.DocValuesFieldExistsQuery;
|
|||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||
import org.elasticsearch.common.CheckedConsumer;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.mapper.ContentPath;
|
||||
import org.elasticsearch.index.mapper.DateFieldMapper;
|
||||
import org.elasticsearch.index.mapper.IpFieldMapper;
|
||||
import org.elasticsearch.index.mapper.KeywordFieldMapper;
|
||||
import org.elasticsearch.index.mapper.MappedFieldType;
|
||||
import org.elasticsearch.index.mapper.Mapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.mapper.NumberFieldMapper;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
|
@ -70,7 +64,6 @@ import org.elasticsearch.script.ScriptModule;
|
|||
import org.elasticsearch.script.ScriptService;
|
||||
import org.elasticsearch.script.ScriptType;
|
||||
import org.elasticsearch.search.aggregations.AggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorTestCase;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.bucket.filter.Filter;
|
||||
|
@ -85,14 +78,10 @@ import org.elasticsearch.search.aggregations.bucket.terms.Terms;
|
|||
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
|
||||
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.FieldContext;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.search.lookup.LeafDocLookup;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.Instant;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -108,8 +97,6 @@ import java.util.function.Supplier;
|
|||
import static java.util.Collections.singleton;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class MinAggregatorTests extends AggregatorTestCase {
|
||||
|
||||
|
@ -692,103 +679,6 @@ public class MinAggregatorTests extends AggregatorTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testShortcutIsApplicable() {
|
||||
for (NumberFieldMapper.NumberType type : NumberFieldMapper.NumberType.values()) {
|
||||
assertNotNull(
|
||||
MinAggregator.getPointReaderOrNull(
|
||||
mockSearchContext(new MatchAllDocsQuery()),
|
||||
null,
|
||||
mockNumericValuesSourceConfig("number", type, true)
|
||||
)
|
||||
);
|
||||
assertNotNull(
|
||||
MinAggregator.getPointReaderOrNull(
|
||||
mockSearchContext(null),
|
||||
null,
|
||||
mockNumericValuesSourceConfig("number", type, true)
|
||||
)
|
||||
);
|
||||
assertNull(
|
||||
MinAggregator.getPointReaderOrNull(
|
||||
mockSearchContext(null),
|
||||
mockAggregator(),
|
||||
mockNumericValuesSourceConfig("number", type, true)
|
||||
)
|
||||
);
|
||||
assertNull(
|
||||
MinAggregator.getPointReaderOrNull(
|
||||
mockSearchContext(new TermQuery(new Term("foo", "bar"))),
|
||||
null,
|
||||
mockNumericValuesSourceConfig("number", type, true)
|
||||
)
|
||||
);
|
||||
assertNull(
|
||||
MinAggregator.getPointReaderOrNull(
|
||||
mockSearchContext(null),
|
||||
mockAggregator(),
|
||||
mockNumericValuesSourceConfig("number", type, true)
|
||||
)
|
||||
);
|
||||
assertNull(
|
||||
MinAggregator.getPointReaderOrNull(
|
||||
mockSearchContext(null),
|
||||
null,
|
||||
mockNumericValuesSourceConfig("number", type, false)
|
||||
)
|
||||
);
|
||||
}
|
||||
for (DateFieldMapper.Resolution resolution : DateFieldMapper.Resolution.values()) {
|
||||
assertNull(
|
||||
MinAggregator.getPointReaderOrNull(
|
||||
mockSearchContext(new MatchAllDocsQuery()),
|
||||
mockAggregator(),
|
||||
mockDateValuesSourceConfig("number", true, resolution)
|
||||
)
|
||||
);
|
||||
assertNull(
|
||||
MinAggregator.getPointReaderOrNull(
|
||||
mockSearchContext(new TermQuery(new Term("foo", "bar"))),
|
||||
null,
|
||||
mockDateValuesSourceConfig("number", true, resolution)
|
||||
)
|
||||
);
|
||||
assertNull(
|
||||
MinAggregator.getPointReaderOrNull(
|
||||
mockSearchContext(null),
|
||||
mockAggregator(),
|
||||
mockDateValuesSourceConfig("number", true, resolution)
|
||||
)
|
||||
);
|
||||
assertNull(
|
||||
MinAggregator.getPointReaderOrNull(
|
||||
mockSearchContext(null),
|
||||
null,
|
||||
mockDateValuesSourceConfig("number", false, resolution)
|
||||
)
|
||||
);
|
||||
}
|
||||
// Check that we decode a dates "just like" the doc values instance.
|
||||
Instant expected = Instant.from(DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.parse("2020-01-01T00:00:00Z"));
|
||||
byte[] scratch = new byte[8];
|
||||
LongPoint.encodeDimension(DateFieldMapper.Resolution.MILLISECONDS.convert(expected), scratch, 0);
|
||||
assertThat(
|
||||
MinAggregator.getPointReaderOrNull(
|
||||
mockSearchContext(new MatchAllDocsQuery()),
|
||||
null,
|
||||
mockDateValuesSourceConfig("number", true, DateFieldMapper.Resolution.MILLISECONDS)
|
||||
).apply(scratch), equalTo(expected.toEpochMilli())
|
||||
);
|
||||
LongPoint.encodeDimension(DateFieldMapper.Resolution.NANOSECONDS.convert(expected), scratch, 0);
|
||||
assertThat(
|
||||
MinAggregator.getPointReaderOrNull(
|
||||
mockSearchContext(new MatchAllDocsQuery()),
|
||||
null,
|
||||
mockDateValuesSourceConfig("number", true, DateFieldMapper.Resolution.NANOSECONDS)
|
||||
).apply(scratch), equalTo(expected.toEpochMilli())
|
||||
);
|
||||
|
||||
}
|
||||
|
||||
public void testMinShortcutRandom() throws Exception {
|
||||
testMinShortcutCase(
|
||||
() -> randomLongBetween(Integer.MIN_VALUE, Integer.MAX_VALUE),
|
||||
|
@ -864,40 +754,6 @@ public class MinAggregatorTests extends AggregatorTestCase {
|
|||
directory.close();
|
||||
}
|
||||
|
||||
private SearchContext mockSearchContext(Query query) {
|
||||
SearchContext searchContext = mock(SearchContext.class);
|
||||
when(searchContext.query()).thenReturn(query);
|
||||
return searchContext;
|
||||
}
|
||||
|
||||
private Aggregator mockAggregator() {
|
||||
return mock(Aggregator.class);
|
||||
}
|
||||
|
||||
private ValuesSourceConfig mockNumericValuesSourceConfig(String fieldName,
|
||||
NumberFieldMapper.NumberType numType,
|
||||
boolean indexed) {
|
||||
ValuesSourceConfig config = mock(ValuesSourceConfig.class);
|
||||
MappedFieldType ft = new NumberFieldMapper.NumberFieldType(fieldName, numType, indexed, true, Collections.emptyMap());
|
||||
when(config.fieldContext()).thenReturn(new FieldContext(fieldName, null, ft));
|
||||
return config;
|
||||
}
|
||||
|
||||
private ValuesSourceConfig mockDateValuesSourceConfig(String fieldName, boolean indexed,
|
||||
DateFieldMapper.Resolution resolution) {
|
||||
ValuesSourceConfig config = mock(ValuesSourceConfig.class);
|
||||
Mapper.BuilderContext builderContext = new Mapper.BuilderContext(
|
||||
Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT).build(),
|
||||
new ContentPath());
|
||||
MappedFieldType ft = new DateFieldMapper.Builder(fieldName)
|
||||
.index(indexed)
|
||||
.withResolution(resolution)
|
||||
.build(builderContext)
|
||||
.fieldType();
|
||||
when(config.fieldContext()).thenReturn(new FieldContext(fieldName, null, ft));
|
||||
return config;
|
||||
}
|
||||
|
||||
private void testCase(Query query,
|
||||
CheckedConsumer<RandomIndexWriter, IOException> buildIndex,
|
||||
Consumer<InternalMin> verify) throws IOException {
|
||||
|
|
Loading…
Reference in New Issue