Allow efficient can_match phases on frozen indices (#35431)

This change adds a special caching reader that caches all relevant
values for a range query to rewrite correctly in a can_match phase
without actually opening the underlying directory reader. This
allows frozen indices to be filtered with can_match and in-turn
searched with wildcards in a efficient way since it allows us to
exclude shards that won't match based on their date-ranges without
opening their directory readers.

Relates to #34352
Depends on #34357
This commit is contained in:
Simon Willnauer 2018-11-13 14:53:55 +01:00 committed by GitHub
parent e81671dd45
commit 3229dfc4de
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 506 additions and 5 deletions

View File

@ -646,17 +646,17 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
public DefaultSearchContext createSearchContext(ShardSearchRequest request, TimeValue timeout)
throws IOException {
return createSearchContext(request, timeout, true);
return createSearchContext(request, timeout, true, "search");
}
private DefaultSearchContext createSearchContext(ShardSearchRequest request, TimeValue timeout,
boolean assertAsyncActions)
boolean assertAsyncActions, String source)
throws IOException {
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
IndexShard indexShard = indexService.getShard(request.shardId().getId());
SearchShardTarget shardTarget = new SearchShardTarget(clusterService.localNode().getId(),
indexShard.shardId(), request.getClusterAlias(), OriginalIndices.NONE);
Engine.Searcher engineSearcher = indexShard.acquireSearcher("search");
Engine.Searcher engineSearcher = indexShard.acquireSearcher(source);
final DefaultSearchContext searchContext = new DefaultSearchContext(idGenerator.incrementAndGet(), request, shardTarget,
engineSearcher, clusterService, indexService, indexShard, bigArrays, threadPool.estimatedTimeInMillisCounter(), timeout,
@ -1016,7 +1016,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
*/
public boolean canMatch(ShardSearchRequest request) throws IOException {
assert request.searchType() == SearchType.QUERY_THEN_FETCH : "unexpected search type: " + request.searchType();
try (DefaultSearchContext context = createSearchContext(request, defaultSearchTimeout, false)) {
try (DefaultSearchContext context = createSearchContext(request, defaultSearchTimeout, false, "can_match")) {
SearchSourceBuilder source = context.request().source();
if (canRewriteToMatchNone(source)) {
QueryBuilder queryBuilder = source.query();

View File

@ -28,6 +28,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.ReferenceManager;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.elasticsearch.common.SuppressForbidden;
import org.elasticsearch.common.lucene.Lucene;
@ -40,6 +41,7 @@ import org.elasticsearch.transport.TransportRequest;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.function.Function;
/**
@ -66,9 +68,23 @@ public final class FrozenEngine extends ReadOnlyEngine {
public static final Setting<Boolean> INDEX_FROZEN = Setting.boolSetting("index.frozen", false, Setting.Property.IndexScope,
Setting.Property.PrivateIndex);
private volatile DirectoryReader lastOpenedReader;
private final DirectoryReader canMatchReader;
public FrozenEngine(EngineConfig config) {
super(config, null, null, true, Function.identity());
boolean success = false;
Directory directory = store.directory();
try (DirectoryReader reader = DirectoryReader.open(directory)) {
canMatchReader = new RewriteCachingDirectoryReader(directory, reader.leaves());
success = true;
} catch (IOException e) {
throw new UncheckedIOException(e);
} finally {
if (success == false) {
closeNoLock("failed on construction", new CountDownLatch(1));
}
}
}
@Override
@ -193,6 +209,7 @@ public final class FrozenEngine extends ReadOnlyEngine {
case "segments_stats":
case "completion_stats":
case "refresh_needed":
case "can_match": // special case for can_match phase - we use the cached point values reader
maybeOpenReader = false;
break;
default:
@ -205,6 +222,10 @@ public final class FrozenEngine extends ReadOnlyEngine {
// we just hand out a searcher on top of an empty reader that we opened for the ReadOnlyEngine in the #open(IndexCommit)
// method. this is the case when we don't have a reader open right now and we get a stats call any other that falls in
// the category that doesn't trigger a reopen
if ("can_match".equals(source)) {
canMatchReader.incRef();
return new Searcher(source, new IndexSearcher(canMatchReader), canMatchReader::decRef);
}
return super.acquireSearcher(source, scope);
} else {
try {

View File

@ -0,0 +1,267 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.index.engine;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexCommit;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.LeafMetaData;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.PointValues;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.index.Terms;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* This special DirectoryReader is used to handle can_match requests against frozen indices.
* It' caches all relevant point value data for every point value field ie. min/max packed values etc.
* to hold enough information to rewrite a date range query and make a decisions if an index can match or not.
* This allows frozen indices to be searched with wildcards in a very efficient way without opening a reader on them.
*/
final class RewriteCachingDirectoryReader extends DirectoryReader {
RewriteCachingDirectoryReader(Directory directory, List<LeafReaderContext> segmentReaders) throws
IOException {
super(directory, wrap(segmentReaders));
}
private static LeafReader[] wrap(List<LeafReaderContext> readers) throws IOException {
LeafReader[] wrapped = new LeafReader[readers.size()];
int i = 0;
for (LeafReaderContext ctx : readers) {
LeafReader wrap = new RewriteCachingLeafReader(ctx.reader());
wrapped[i++] = wrap;
}
return wrapped;
}
@Override
protected DirectoryReader doOpenIfChanged() {
throw new UnsupportedOperationException();
}
@Override
protected DirectoryReader doOpenIfChanged(IndexCommit commit) {
throw new UnsupportedOperationException();
}
@Override
protected DirectoryReader doOpenIfChanged(IndexWriter writer, boolean applyAllDeletes) {
throw new UnsupportedOperationException();
}
@Override
public long getVersion() {
throw new UnsupportedOperationException();
}
@Override
public boolean isCurrent() {
throw new UnsupportedOperationException();
}
@Override
public IndexCommit getIndexCommit() {
throw new UnsupportedOperationException();
}
@Override
protected void doClose() {
throw new UnsupportedOperationException();
}
@Override
public CacheHelper getReaderCacheHelper() {
throw new UnsupportedOperationException();
}
// except of a couple of selected methods everything else will
// throw a UOE which causes a can_match phase to just move to the actual phase
// later such that we never false exclude a shard if something else is used to rewrite.
private static final class RewriteCachingLeafReader extends LeafReader {
private final int maxDoc;
private final int numDocs;
private final Map<String, PointValues> pointValuesMap;
private final FieldInfos fieldInfos;
private RewriteCachingLeafReader(LeafReader original) throws IOException {
this.maxDoc = original.maxDoc();
this.numDocs = original.numDocs();
fieldInfos = original.getFieldInfos();
Map<String, PointValues> valuesMap = new HashMap<>();
for (FieldInfo info : fieldInfos) {
if (info.getPointIndexDimensionCount() != 0) {
PointValues pointValues = original.getPointValues(info.name);
if (pointValues != null) { // might not be in this reader
byte[] minPackedValue = pointValues.getMinPackedValue();
byte[] maxPackedValue = pointValues.getMaxPackedValue();
int numDimensions = pointValues.getNumIndexDimensions();
int bytesPerDimension = pointValues.getBytesPerDimension();
int numDataDimensions = pointValues.getNumDataDimensions();
long size = pointValues.size();
int docCount = pointValues.getDocCount();
valuesMap.put(info.name, new PointValues() {
@Override
public void intersect(IntersectVisitor visitor) {
throw new UnsupportedOperationException();
}
@Override
public long estimatePointCount(IntersectVisitor visitor) {
throw new UnsupportedOperationException();
}
@Override
public byte[] getMinPackedValue() {
return minPackedValue;
}
@Override
public byte[] getMaxPackedValue() {
return maxPackedValue;
}
@Override
public int getNumDataDimensions() {
return numDataDimensions;
}
@Override
public int getNumIndexDimensions() {
return numDimensions;
}
@Override
public int getBytesPerDimension() {
return bytesPerDimension;
}
@Override
public long size() {
return size;
}
@Override
public int getDocCount() {
return docCount;
}
});
}
}
}
pointValuesMap = valuesMap;
}
@Override
public CacheHelper getCoreCacheHelper() {
throw new UnsupportedOperationException();
}
@Override
public Terms terms(String field) {
throw new UnsupportedOperationException();
}
@Override
public NumericDocValues getNumericDocValues(String field) {
throw new UnsupportedOperationException();
}
@Override
public BinaryDocValues getBinaryDocValues(String field) {
throw new UnsupportedOperationException();
}
@Override
public SortedDocValues getSortedDocValues(String field) {
throw new UnsupportedOperationException();
}
@Override
public SortedNumericDocValues getSortedNumericDocValues(String field) {
throw new UnsupportedOperationException();
}
@Override
public SortedSetDocValues getSortedSetDocValues(String field) {
throw new UnsupportedOperationException();
}
@Override
public NumericDocValues getNormValues(String field) {
throw new UnsupportedOperationException();
}
@Override
public FieldInfos getFieldInfos() {
return fieldInfos;
}
@Override
public Bits getLiveDocs() {
throw new UnsupportedOperationException();
}
@Override
public PointValues getPointValues(String field) {
return pointValuesMap.get(field);
}
@Override
public void checkIntegrity() {
}
@Override
public LeafMetaData getMetaData() {
throw new UnsupportedOperationException();
}
@Override
public Fields getTermVectors(int docID) {
throw new UnsupportedOperationException();
}
@Override
public int numDocs() {
return numDocs;
}
@Override
public int maxDoc() {
return maxDoc;
}
@Override
public void document(int docID, StoredFieldVisitor visitor) {
throw new UnsupportedOperationException();
}
@Override
protected void doClose() {
}
@Override
public CacheHelper getReaderCacheHelper() {
return null;
}
}
}

View File

@ -5,6 +5,8 @@
*/
package org.elasticsearch.index.engine;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.FilterDirectoryReader;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.ReferenceManager;
import org.apache.lucene.search.TopDocs;
@ -19,6 +21,7 @@ import org.elasticsearch.index.seqno.SequenceNumbers;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.hamcrest.Matchers;
import java.io.IOException;
import java.lang.reflect.Method;
@ -279,4 +282,36 @@ public class FrozenEngineTests extends EngineTestCase {
beforeRefresh.set(0);
}
}
public void testCanMatch() throws IOException {
IOUtils.close(engine, store);
final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
try (Store store = createStore()) {
CountingRefreshListener listener = new CountingRefreshListener();
EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), null, listener, null,
globalCheckpoint::get, new NoneCircuitBreakerService());
try (InternalEngine engine = createEngine(config)) {
addDocuments(globalCheckpoint, engine);
engine.flushAndClose();
listener.reset();
try (FrozenEngine frozenEngine = new FrozenEngine(engine.engineConfig)) {
DirectoryReader reader;
try (Engine.Searcher searcher = frozenEngine.acquireSearcher("can_match")) {
reader = searcher.getDirectoryReader();
assertNotEquals(reader, Matchers.instanceOf(FrozenEngine.LazyDirectoryReader.class));
assertEquals(0, listener.afterRefresh.get());
DirectoryReader unwrap = FilterDirectoryReader.unwrap(searcher.getDirectoryReader());
assertThat(unwrap, Matchers.instanceOf(RewriteCachingDirectoryReader.class));
}
try (Engine.Searcher searcher = frozenEngine.acquireSearcher("can_match")) {
assertSame(reader, searcher.getDirectoryReader());
assertNotEquals(reader, Matchers.instanceOf(FrozenEngine.LazyDirectoryReader.class));
assertEquals(0, listener.afterRefresh.get());
DirectoryReader unwrap = FilterDirectoryReader.unwrap(searcher.getDirectoryReader());
assertThat(unwrap, Matchers.instanceOf(RewriteCachingDirectoryReader.class));
}
}
}
}
}
}

View File

@ -9,10 +9,12 @@ import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -20,11 +22,15 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.query.MatchAllQueryBuilder;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardTestCase;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.search.SearchService;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.internal.AliasFilter;
import org.elasticsearch.search.internal.ShardSearchLocalRequest;
import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.xpack.core.XPackClient;
import org.elasticsearch.xpack.core.XPackPlugin;
@ -125,7 +131,8 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
break;
case 1:
client().prepareSearch("index").setIndicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED)
.setSearchType(SearchType.QUERY_THEN_FETCH).execute(listener);
.setSearchType(SearchType.QUERY_THEN_FETCH)
.execute(listener);
// in total 4 refreshes 1x query & 1x fetch per shard (we have 2)
numRefreshes += 3;
break;
@ -197,4 +204,66 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
assertThat(executionException.getCause(), Matchers.instanceOf(IllegalStateException.class));
assertEquals("index [test-idx] is not closed", executionException.getCause().getMessage());
}
public void testCanMatch() throws ExecutionException, InterruptedException, IOException {
createIndex("index");
client().prepareIndex("index", "_doc", "1").setSource("field", "2010-01-05T02:00").setRefreshPolicy(IMMEDIATE).execute()
.actionGet();
client().prepareIndex("index", "_doc", "2").setSource("field", "2010-01-06T02:00").setRefreshPolicy(IMMEDIATE).execute()
.actionGet();
{
IndicesService indexServices = getInstanceFromNode(IndicesService.class);
Index index = resolveIndex("index");
IndexService indexService = indexServices.indexServiceSafe(index);
IndexShard shard = indexService.getShard(0);
assertFalse(indexService.getIndexSettings().isSearchThrottled());
SearchService searchService = getInstanceFromNode(SearchService.class);
assertTrue(searchService.canMatch(new ShardSearchLocalRequest(shard.shardId(), 1, SearchType.QUERY_THEN_FETCH, null,
Strings.EMPTY_ARRAY, false, new AliasFilter(null, Strings.EMPTY_ARRAY), 1f, true, null, null)));
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder();
sourceBuilder.query(QueryBuilders.rangeQuery("field").gte("2010-01-03||+2d").lte("2010-01-04||+2d/d"));
assertTrue(searchService.canMatch(new ShardSearchLocalRequest(shard.shardId(), 1, SearchType.QUERY_THEN_FETCH, sourceBuilder,
Strings.EMPTY_ARRAY, false, new AliasFilter(null, Strings.EMPTY_ARRAY), 1f, true, null, null)));
sourceBuilder = new SearchSourceBuilder();
sourceBuilder.query(QueryBuilders.rangeQuery("field").gt("2010-01-06T02:00").lt("2010-01-07T02:00"));
assertFalse(searchService.canMatch(new ShardSearchLocalRequest(shard.shardId(), 1, SearchType.QUERY_THEN_FETCH, sourceBuilder,
Strings.EMPTY_ARRAY, false, new AliasFilter(null, Strings.EMPTY_ARRAY), 1f, true, null, null)));
}
client().admin().indices().prepareFlush("index").get();
client().admin().indices().prepareClose("index").get();
XPackClient xPackClient = new XPackClient(client());
PlainActionFuture<AcknowledgedResponse> future = new PlainActionFuture<>();
TransportFreezeIndexAction.FreezeRequest request =
new TransportFreezeIndexAction.FreezeRequest("index");
xPackClient.freeze(request, future);
assertAcked(future.get());
assertAcked(client().admin().indices().prepareOpen("index").setWaitForActiveShards(ActiveShardCount.DEFAULT));
{
IndicesService indexServices = getInstanceFromNode(IndicesService.class);
Index index = resolveIndex("index");
IndexService indexService = indexServices.indexServiceSafe(index);
IndexShard shard = indexService.getShard(0);
assertTrue(indexService.getIndexSettings().isSearchThrottled());
SearchService searchService = getInstanceFromNode(SearchService.class);
assertTrue(searchService.canMatch(new ShardSearchLocalRequest(shard.shardId(), 1, SearchType.QUERY_THEN_FETCH, null,
Strings.EMPTY_ARRAY, false, new AliasFilter(null, Strings.EMPTY_ARRAY), 1f, true, null, null)));
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder();
sourceBuilder.query(QueryBuilders.rangeQuery("field").gte("2010-01-03||+2d").lte("2010-01-04||+2d/d"));
assertTrue(searchService.canMatch(new ShardSearchLocalRequest(shard.shardId(), 1, SearchType.QUERY_THEN_FETCH, sourceBuilder,
Strings.EMPTY_ARRAY, false, new AliasFilter(null, Strings.EMPTY_ARRAY), 1f, true, null, null)));
sourceBuilder = new SearchSourceBuilder();
sourceBuilder.query(QueryBuilders.rangeQuery("field").gt("2010-01-06T02:00").lt("2010-01-07T02:00"));
assertFalse(searchService.canMatch(new ShardSearchLocalRequest(shard.shardId(), 1, SearchType.QUERY_THEN_FETCH, sourceBuilder,
Strings.EMPTY_ARRAY, false, new AliasFilter(null, Strings.EMPTY_ARRAY), 1f, true, null, null)));
IndicesStatsResponse response = client().admin().indices().prepareStats("index").clear().setRefresh(true).get();
assertEquals(0, response.getTotal().refresh.getTotal()); // never opened a reader
}
}
}

View File

@ -0,0 +1,109 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.index.engine;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.LongPoint;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.PointValues;
import org.apache.lucene.store.Directory;
import org.elasticsearch.index.mapper.DateFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.query.QueryRewriteContext;
import org.elasticsearch.test.ESTestCase;
import org.joda.time.DateTimeZone;
import java.io.IOException;
public class RewriteCachingDirectoryReaderTests extends ESTestCase {
public void testGetMinMaxPackedValue() throws IOException {
try (Directory dir = newDirectory()) {
try (IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig())) {
int numDocs = randomIntBetween(10, 100);
boolean rarely = false;
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
if (i > 0 && rarely()) {
rarely = true;
doc.add(new LongPoint("rarely", 1));
} else {
long value = randomLongBetween(0, 10000);
doc.add(new LongPoint("test", value));
doc.add(new LongPoint("test_const", 1));
}
writer.addDocument(doc);
}
try (DirectoryReader reader = DirectoryReader.open(writer)) {
RewriteCachingDirectoryReader cachingDirectoryReader = new RewriteCachingDirectoryReader(dir, reader.leaves());
if (rarely) {
assertArrayEquals(PointValues.getMaxPackedValue(reader, "rarely"),
PointValues.getMaxPackedValue(cachingDirectoryReader, "rarely"));
assertArrayEquals(PointValues.getMinPackedValue(reader, "rarely"),
PointValues.getMinPackedValue(cachingDirectoryReader, "rarely"));
assertEquals(PointValues.size(reader, "rarely"),
PointValues.size(cachingDirectoryReader, "rarely"));
}
assertArrayEquals(PointValues.getMaxPackedValue(reader, "test"),
PointValues.getMaxPackedValue(cachingDirectoryReader, "test"));
assertArrayEquals(PointValues.getMaxPackedValue(reader, "test_const"),
PointValues.getMaxPackedValue(cachingDirectoryReader, "test_const"));
assertArrayEquals(PointValues.getMinPackedValue(reader, "test"),
PointValues.getMinPackedValue(cachingDirectoryReader, "test"));
assertArrayEquals(PointValues.getMinPackedValue(reader, "test_const"),
PointValues.getMinPackedValue(cachingDirectoryReader, "test_const"));
assertEquals(PointValues.size(reader, "test"),
PointValues.size(cachingDirectoryReader, "test"));
assertEquals(PointValues.size(reader, "test_const"),
PointValues.size(cachingDirectoryReader, "test_const"));
}
}
}
}
public void testIsWithinQuery() throws IOException {
try (Directory dir = newDirectory()) {
try (IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig())) {
Document doc = new Document();
doc.add(new LongPoint("test", 5));
writer.addDocument(doc);
if (randomBoolean()) {
writer.flush();
}
doc = new Document();
doc.add(new LongPoint("test", 0));
writer.addDocument(doc);
if (randomBoolean()) {
writer.flush();
}
doc = new Document();
doc.add(new LongPoint("test", 10));
writer.addDocument(doc);
try (DirectoryReader reader = DirectoryReader.open(writer)) {
RewriteCachingDirectoryReader cachingDirectoryReader = new RewriteCachingDirectoryReader(dir, reader.leaves());
DateFieldMapper.Builder b = new DateFieldMapper.Builder("test");
DateFieldMapper.DateFieldType dateFieldType = b.fieldType();
dateFieldType.setName("test");
QueryRewriteContext context = new QueryRewriteContext(xContentRegistry(), writableRegistry(), null, () -> 0);
MappedFieldType.Relation relation = dateFieldType.isFieldWithinQuery(cachingDirectoryReader, 0, 10,
true, true, DateTimeZone.UTC, null, context);
assertEquals(relation, MappedFieldType.Relation.WITHIN);
relation = dateFieldType.isFieldWithinQuery(cachingDirectoryReader, 3, 11,
true, true, DateTimeZone.UTC, null, context);
assertEquals(relation, MappedFieldType.Relation.INTERSECTS);
relation = dateFieldType.isFieldWithinQuery(cachingDirectoryReader, 10, 11,
false, true, DateTimeZone.UTC, null, context);
assertEquals(relation, MappedFieldType.Relation.DISJOINT);
}
}
}
}
}