Browse Source

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
Simon Willnauer 7 năm trước cách đây
mục cha
commit
3229dfc4de

+ 4 - 4
server/src/main/java/org/elasticsearch/search/SearchService.java

@@ -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();

+ 21 - 0
x-pack/plugin/core/src/main/java/org/elasticsearch/index/engine/FrozenEngine.java

@@ -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 {

+ 267 - 0
x-pack/plugin/core/src/main/java/org/elasticsearch/index/engine/RewriteCachingDirectoryReader.java

@@ -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;
+        }
+    }
+}

+ 35 - 0
x-pack/plugin/core/src/test/java/org/elasticsearch/index/engine/FrozenEngineTests.java

@@ -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));
+                    }
+                }
+            }
+        }
+    }
 }

+ 70 - 1
x-pack/plugin/core/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java

@@ -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
+        }
+    }
 }

+ 109 - 0
x-pack/plugin/core/src/test/java/org/elasticsearch/index/engine/RewriteCachingDirectoryReaderTests.java

@@ -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);
+                }
+            }
+        }
+    }
+}