Browse Source

Add points metadata support for archive indices (#86655)

Archive indices appear just as regular indices in a cluster, and can be part of index patterns when queried. To allow
searches to quickly skip shards of archive indices that might not have relevant data, we're adding support for skipping
shards of archive indices here that don't have data falling in the time range being queried. This is critical for the Kibana
experience which relies on the date range picker to quickly skip some of the indices in an index pattern.

Doing the actual time-range query on the archive index is much more expensive as on a regular index (as it's using doc
values instead of points to run the query, equating to a full scan of the columnar data). The solution here is to make
points metadata available in archive indices, so that the minimum and maximum value can be retrieved in constant time
(only a tiny fraction of the full points capabilities).
Yannick Welsch 3 years ago
parent
commit
8a58e363f4

+ 35 - 6
server/src/main/java/org/elasticsearch/index/mapper/DateFieldMapper.java

@@ -341,6 +341,7 @@ public final class DateFieldMapper extends FieldMapper {
             DateFieldType ft = new DateFieldType(
                 context.buildFullName(name()),
                 index.getValue() && indexCreatedVersion.isLegacyIndexVersion() == false,
+                index.getValue(),
                 store.getValue(),
                 docValues.getValue(),
                 buildFormatter(),
@@ -387,10 +388,12 @@ public final class DateFieldMapper extends FieldMapper {
         protected final Resolution resolution;
         protected final String nullValue;
         protected final FieldValues<Long> scriptValues;
+        private final boolean pointsMetadataAvailable;
 
         public DateFieldType(
             String name,
             boolean isIndexed,
+            boolean pointsMetadataAvailable,
             boolean isStored,
             boolean hasDocValues,
             DateFormatter dateTimeFormatter,
@@ -405,26 +408,52 @@ public final class DateFieldMapper extends FieldMapper {
             this.resolution = resolution;
             this.nullValue = nullValue;
             this.scriptValues = scriptValues;
+            this.pointsMetadataAvailable = pointsMetadataAvailable;
+        }
+
+        public DateFieldType(
+            String name,
+            boolean isIndexed,
+            boolean isStored,
+            boolean hasDocValues,
+            DateFormatter dateTimeFormatter,
+            Resolution resolution,
+            String nullValue,
+            FieldValues<Long> scriptValues,
+            Map<String, String> meta
+        ) {
+            this(name, isIndexed, isIndexed, isStored, hasDocValues, dateTimeFormatter, resolution, nullValue, scriptValues, meta);
         }
 
         public DateFieldType(String name) {
-            this(name, true, false, true, DEFAULT_DATE_TIME_FORMATTER, Resolution.MILLISECONDS, null, null, Collections.emptyMap());
+            this(name, true, true, false, true, DEFAULT_DATE_TIME_FORMATTER, Resolution.MILLISECONDS, null, null, Collections.emptyMap());
         }
 
         public DateFieldType(String name, boolean isIndexed) {
-            this(name, isIndexed, false, true, DEFAULT_DATE_TIME_FORMATTER, Resolution.MILLISECONDS, null, null, Collections.emptyMap());
+            this(
+                name,
+                isIndexed,
+                isIndexed,
+                false,
+                true,
+                DEFAULT_DATE_TIME_FORMATTER,
+                Resolution.MILLISECONDS,
+                null,
+                null,
+                Collections.emptyMap()
+            );
         }
 
         public DateFieldType(String name, DateFormatter dateFormatter) {
-            this(name, true, false, true, dateFormatter, Resolution.MILLISECONDS, null, null, Collections.emptyMap());
+            this(name, true, true, false, true, dateFormatter, Resolution.MILLISECONDS, null, null, Collections.emptyMap());
         }
 
         public DateFieldType(String name, Resolution resolution) {
-            this(name, true, false, true, DEFAULT_DATE_TIME_FORMATTER, resolution, null, null, Collections.emptyMap());
+            this(name, true, true, false, true, DEFAULT_DATE_TIME_FORMATTER, resolution, null, null, Collections.emptyMap());
         }
 
         public DateFieldType(String name, Resolution resolution, DateFormatter dateFormatter) {
-            this(name, true, false, true, dateFormatter, resolution, null, null, Collections.emptyMap());
+            this(name, true, true, false, true, dateFormatter, resolution, null, null, Collections.emptyMap());
         }
 
         @Override
@@ -646,7 +675,7 @@ public final class DateFieldMapper extends FieldMapper {
             DateMathParser dateParser,
             QueryRewriteContext context
         ) throws IOException {
-            if (isIndexed() == false && hasDocValues()) {
+            if (isIndexed() == false && pointsMetadataAvailable == false && hasDocValues()) {
                 // we don't have a quick way to run this check on doc values, so fall back to default assuming we are within bounds
                 return Relation.INTERSECTS;
             }

+ 5 - 0
x-pack/plugin/old-lucene-versions/src/internalClusterTest/java/org/elasticsearch/xpack/lucene/bwc/AbstractArchiveTestCase.java

@@ -41,6 +41,11 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcke
 @ESIntegTestCase.ClusterScope(supportsDedicatedMasters = false, numClientNodes = 0, scope = ESIntegTestCase.Scope.TEST)
 public abstract class AbstractArchiveTestCase extends AbstractSnapshotIntegTestCase {
 
+    @Override
+    protected boolean addMockInternalEngine() {
+        return false;
+    }
+
     @Override
     protected Collection<Class<? extends Plugin>> nodePlugins() {
         return Arrays.asList(LocalStateOldLuceneVersions.class, TestRepositoryPlugin.class, MockRepository.Plugin.class);

+ 22 - 1
x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/OldLuceneVersions.java

@@ -29,16 +29,21 @@ import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.index.IndexModule;
+import org.elasticsearch.index.IndexSettings;
 import org.elasticsearch.index.engine.Engine;
+import org.elasticsearch.index.engine.EngineFactory;
+import org.elasticsearch.index.engine.ReadOnlyEngine;
 import org.elasticsearch.index.seqno.SequenceNumbers;
 import org.elasticsearch.index.shard.IndexEventListener;
 import org.elasticsearch.index.shard.IndexShard;
+import org.elasticsearch.index.translog.TranslogStats;
 import org.elasticsearch.license.License;
 import org.elasticsearch.license.LicenseUtils;
 import org.elasticsearch.license.LicensedFeature;
 import org.elasticsearch.license.XPackLicenseState;
 import org.elasticsearch.plugins.ActionPlugin;
 import org.elasticsearch.plugins.ClusterPlugin;
+import org.elasticsearch.plugins.EnginePlugin;
 import org.elasticsearch.plugins.IndexStorePlugin;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.plugins.RepositoryPlugin;
@@ -46,6 +51,7 @@ import org.elasticsearch.repositories.RepositoriesService;
 import org.elasticsearch.script.ScriptService;
 import org.elasticsearch.snapshots.Snapshot;
 import org.elasticsearch.snapshots.SnapshotRestoreException;
+import org.elasticsearch.snapshots.sourceonly.SourceOnlySnapshotRepository;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.watcher.ResourceWatcherService;
 import org.elasticsearch.xcontent.NamedXContentRegistry;
@@ -60,10 +66,12 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.function.BiConsumer;
+import java.util.function.Function;
 import java.util.function.Supplier;
 
-public class OldLuceneVersions extends Plugin implements IndexStorePlugin, ClusterPlugin, RepositoryPlugin, ActionPlugin {
+public class OldLuceneVersions extends Plugin implements IndexStorePlugin, ClusterPlugin, RepositoryPlugin, ActionPlugin, EnginePlugin {
 
     public static final LicensedFeature.Momentary ARCHIVE_FEATURE = LicensedFeature.momentary(
         null,
@@ -226,4 +234,17 @@ public class OldLuceneVersions extends Plugin implements IndexStorePlugin, Clust
     public Map<String, DirectoryFactory> getDirectoryFactories() {
         return Map.of();
     }
+
+    @Override
+    public Optional<EngineFactory> getEngineFactory(IndexSettings indexSettings) {
+        if (indexSettings.getIndexVersionCreated().isLegacyIndexVersion()
+            && indexSettings.getIndexMetadata().isSearchableSnapshot() == false
+            && indexSettings.getValue(SourceOnlySnapshotRepository.SOURCE_ONLY) == false) {
+            return Optional.of(
+                engineConfig -> new ReadOnlyEngine(engineConfig, null, new TranslogStats(), true, Function.identity(), true, false)
+            );
+        }
+
+        return Optional.empty();
+    }
 }

+ 4 - 10
x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/BWCCodec.java

@@ -12,7 +12,6 @@ import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.KnnVectorsFormat;
 import org.apache.lucene.codecs.NormsFormat;
-import org.apache.lucene.codecs.PointsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
 import org.apache.lucene.index.FieldInfo;
@@ -45,11 +44,6 @@ public abstract class BWCCodec extends Codec {
         throw new UnsupportedOperationException();
     }
 
-    @Override
-    public PointsFormat pointsFormat() {
-        throw new UnsupportedOperationException();
-    }
-
     @Override
     public KnnVectorsFormat knnVectorsFormat() {
         throw new UnsupportedOperationException();
@@ -85,7 +79,7 @@ public abstract class BWCCodec extends Codec {
         };
     }
 
-    // mark all fields as no term vectors, no norms, no payloads, no points, and no vectors.
+    // mark all fields as no term vectors, no norms, no payloads, and no vectors.
     private static FieldInfos filterFields(FieldInfos fieldInfos) {
         List<FieldInfo> fieldInfoCopy = new ArrayList<>(fieldInfos.size());
         for (FieldInfo fieldInfo : fieldInfos) {
@@ -100,9 +94,9 @@ public abstract class BWCCodec extends Codec {
                     fieldInfo.getDocValuesType(),
                     fieldInfo.getDocValuesGen(),
                     fieldInfo.attributes(),
-                    0,
-                    0,
-                    0,
+                    fieldInfo.getPointDimensionCount(),
+                    fieldInfo.getPointIndexDimensionCount(),
+                    fieldInfo.getPointNumBytes(),
                     0,
                     fieldInfo.getVectorSimilarityFunction(),
                     fieldInfo.isSoftDeletesField()

+ 6 - 0
x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene60/Lucene60Codec.java

@@ -27,6 +27,7 @@ import org.apache.lucene.codecs.CompoundFormat;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.codecs.PointsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
@@ -121,4 +122,9 @@ public class Lucene60Codec extends BWCCodec {
     public PostingsFormat postingsFormat() {
         return postingsFormat;
     }
+
+    @Override
+    public PointsFormat pointsFormat() {
+        return new Lucene60MetadataOnlyPointsFormat();
+    }
 }

+ 62 - 0
x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene60/Lucene60MetadataOnlyPointsFormat.java

@@ -0,0 +1,62 @@
+/*
+ * @notice
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.
+ *
+ * Modifications copyright (C) 2021 Elasticsearch B.V.
+ */
+package org.elasticsearch.xpack.lucene.bwc.codecs.lucene60;
+
+import org.apache.lucene.codecs.PointsFormat;
+import org.apache.lucene.codecs.PointsReader;
+import org.apache.lucene.codecs.PointsWriter;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+
+import java.io.IOException;
+
+/**
+ * Allows reading metadata only from Lucene 6.0 point format
+ **/
+public class Lucene60MetadataOnlyPointsFormat extends PointsFormat {
+
+    static final String DATA_CODEC_NAME = "Lucene60PointsFormatData";
+    static final String META_CODEC_NAME = "Lucene60PointsFormatMeta";
+
+    /** Filename extension for the leaf blocks */
+    public static final String DATA_EXTENSION = "dim";
+
+    /** Filename extension for the index per field */
+    public static final String INDEX_EXTENSION = "dii";
+
+    static final int DATA_VERSION_START = 0;
+    static final int DATA_VERSION_CURRENT = DATA_VERSION_START;
+
+    static final int INDEX_VERSION_START = 0;
+    static final int INDEX_VERSION_CURRENT = INDEX_VERSION_START;
+
+    /** Sole constructor */
+    public Lucene60MetadataOnlyPointsFormat() {}
+
+    @Override
+    public PointsWriter fieldsWriter(SegmentWriteState state) {
+        throw new UnsupportedOperationException("Old codecs may only be used for reading");
+    }
+
+    @Override
+    public PointsReader fieldsReader(SegmentReadState state) throws IOException {
+        return new Lucene60MetadataOnlyPointsReader(state);
+    }
+}

+ 144 - 0
x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene60/Lucene60MetadataOnlyPointsReader.java

@@ -0,0 +1,144 @@
+/*
+ * @notice
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.
+ *
+ * Modifications copyright (C) 2021 Elasticsearch B.V.
+ */
+package org.elasticsearch.xpack.lucene.bwc.codecs.lucene60;
+
+import org.apache.lucene.backward_codecs.store.EndiannessReverserUtil;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.PointsReader;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.elasticsearch.core.IOUtils;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+/** Reads the metadata of point values previously written with Lucene60PointsWriter */
+public class Lucene60MetadataOnlyPointsReader extends PointsReader {
+    final IndexInput dataIn;
+    final SegmentReadState readState;
+    final Map<Integer, PointValues> readers = new HashMap<>();
+
+    /** Sole constructor */
+    public Lucene60MetadataOnlyPointsReader(SegmentReadState readState) throws IOException {
+        this.readState = readState;
+
+        String indexFileName = IndexFileNames.segmentFileName(
+            readState.segmentInfo.name,
+            readState.segmentSuffix,
+            Lucene60MetadataOnlyPointsFormat.INDEX_EXTENSION
+        );
+
+        Map<Integer, Long> fieldToFileOffset = new HashMap<>();
+
+        // Read index file
+        try (ChecksumIndexInput indexIn = EndiannessReverserUtil.openChecksumInput(readState.directory, indexFileName, readState.context)) {
+            Throwable priorE = null;
+            try {
+                CodecUtil.checkIndexHeader(
+                    indexIn,
+                    Lucene60MetadataOnlyPointsFormat.META_CODEC_NAME,
+                    Lucene60MetadataOnlyPointsFormat.INDEX_VERSION_START,
+                    Lucene60MetadataOnlyPointsFormat.INDEX_VERSION_CURRENT,
+                    readState.segmentInfo.getId(),
+                    readState.segmentSuffix
+                );
+                int count = indexIn.readVInt();
+                for (int i = 0; i < count; i++) {
+                    int fieldNumber = indexIn.readVInt();
+                    long fp = indexIn.readVLong();
+                    fieldToFileOffset.put(fieldNumber, fp);
+                }
+            } catch (Throwable t) {
+                priorE = t;
+            } finally {
+                CodecUtil.checkFooter(indexIn, priorE);
+            }
+        }
+
+        String dataFileName = IndexFileNames.segmentFileName(
+            readState.segmentInfo.name,
+            readState.segmentSuffix,
+            Lucene60MetadataOnlyPointsFormat.DATA_EXTENSION
+        );
+        boolean success = false;
+        dataIn = EndiannessReverserUtil.openInput(readState.directory, dataFileName, readState.context);
+        try {
+
+            CodecUtil.checkIndexHeader(
+                dataIn,
+                Lucene60MetadataOnlyPointsFormat.DATA_CODEC_NAME,
+                Lucene60MetadataOnlyPointsFormat.DATA_VERSION_START,
+                Lucene60MetadataOnlyPointsFormat.DATA_VERSION_START,
+                readState.segmentInfo.getId(),
+                readState.segmentSuffix
+            );
+
+            // NOTE: data file is too costly to verify checksum against all the bytes on open,
+            // but for now we at least verify proper structure of the checksum footer: which looks
+            // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
+            // such as file truncation.
+            CodecUtil.retrieveChecksum(dataIn);
+
+            for (Map.Entry<Integer, Long> ent : fieldToFileOffset.entrySet()) {
+                int fieldNumber = ent.getKey();
+                long fp = ent.getValue();
+                dataIn.seek(fp);
+                PointValues reader = new MetadataOnlyBKDReader(dataIn);
+                readers.put(fieldNumber, reader);
+            }
+
+            success = true;
+        } finally {
+            if (success == false) {
+                IOUtils.closeWhileHandlingException(this);
+            }
+        }
+    }
+
+    @Override
+    public PointValues getValues(String fieldName) {
+        FieldInfo fieldInfo = readState.fieldInfos.fieldInfo(fieldName);
+        if (fieldInfo == null) {
+            throw new IllegalArgumentException("field=\"" + fieldName + "\" is unrecognized");
+        }
+        if (fieldInfo.getPointDimensionCount() == 0) {
+            throw new IllegalArgumentException("field=\"" + fieldName + "\" did not index point values");
+        }
+
+        return readers.get(fieldInfo.number);
+    }
+
+    @Override
+    public void checkIntegrity() throws IOException {
+        CodecUtil.checksumEntireFile(dataIn);
+    }
+
+    @Override
+    public void close() throws IOException {
+        dataIn.close();
+        // Free up heap:
+        readers.clear();
+    }
+}

+ 129 - 0
x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene60/MetadataOnlyBKDReader.java

@@ -0,0 +1,129 @@
+/*
+ * @notice
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.
+ *
+ * Modifications copyright (C) 2021 Elasticsearch B.V.
+ */
+package org.elasticsearch.xpack.lucene.bwc.codecs.lucene60;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.bkd.BKDConfig;
+
+import java.io.IOException;
+
+/**
+ * BKD Reader that only provides access to the metadata
+ */
+public class MetadataOnlyBKDReader extends PointValues {
+
+    public static final int VERSION_START = 0;
+    public static final int VERSION_SELECTIVE_INDEXING = 6;
+    public static final int VERSION_META_FILE = 9;
+    public static final int VERSION_CURRENT = VERSION_META_FILE;
+
+    final BKDConfig config;
+    final int numLeaves;
+    final byte[] minPackedValue;
+    final byte[] maxPackedValue;
+    final long pointCount;
+    final int docCount;
+    final int version;
+
+    public MetadataOnlyBKDReader(IndexInput metaIn) throws IOException {
+        version = CodecUtil.checkHeader(metaIn, "BKD", VERSION_START, VERSION_CURRENT);
+        final int numDims = metaIn.readVInt();
+        final int numIndexDims;
+        if (version >= VERSION_SELECTIVE_INDEXING) {
+            numIndexDims = metaIn.readVInt();
+        } else {
+            numIndexDims = numDims;
+        }
+        final int maxPointsInLeafNode = metaIn.readVInt();
+        final int bytesPerDim = metaIn.readVInt();
+        config = new BKDConfig(numDims, numIndexDims, bytesPerDim, maxPointsInLeafNode);
+
+        numLeaves = metaIn.readVInt();
+        assert numLeaves > 0;
+
+        minPackedValue = new byte[config.packedIndexBytesLength];
+        maxPackedValue = new byte[config.packedIndexBytesLength];
+
+        metaIn.readBytes(minPackedValue, 0, config.packedIndexBytesLength);
+        metaIn.readBytes(maxPackedValue, 0, config.packedIndexBytesLength);
+        final ArrayUtil.ByteArrayComparator comparator = ArrayUtil.getUnsignedComparator(config.bytesPerDim);
+        for (int dim = 0; dim < config.numIndexDims; dim++) {
+            if (comparator.compare(minPackedValue, dim * config.bytesPerDim, maxPackedValue, dim * config.bytesPerDim) > 0) {
+                throw new CorruptIndexException(
+                    "minPackedValue "
+                        + new BytesRef(minPackedValue)
+                        + " is > maxPackedValue "
+                        + new BytesRef(maxPackedValue)
+                        + " for dim="
+                        + dim,
+                    metaIn
+                );
+            }
+        }
+
+        pointCount = metaIn.readVLong();
+        docCount = metaIn.readVInt();
+    }
+
+    @Override
+    public PointTree getPointTree() {
+        throw new UnsupportedOperationException("only metadata operations allowed");
+    }
+
+    @Override
+    public byte[] getMinPackedValue() {
+        return minPackedValue;
+    }
+
+    @Override
+    public byte[] getMaxPackedValue() {
+        return maxPackedValue;
+    }
+
+    @Override
+    public int getNumDimensions() {
+        return config.numDims;
+    }
+
+    @Override
+    public int getNumIndexDimensions() {
+        return config.numIndexDims;
+    }
+
+    @Override
+    public int getBytesPerDimension() {
+        return config.bytesPerDim;
+    }
+
+    @Override
+    public long size() {
+        return pointCount;
+    }
+
+    @Override
+    public int getDocCount() {
+        return docCount;
+    }
+}

+ 7 - 0
x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene62/Lucene62Codec.java

@@ -27,6 +27,7 @@ import org.apache.lucene.codecs.CompoundFormat;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.codecs.PointsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
@@ -35,6 +36,7 @@ import org.elasticsearch.xpack.lucene.bwc.codecs.BWCCodec;
 import org.elasticsearch.xpack.lucene.bwc.codecs.LegacyAdaptingPerFieldPostingsFormat;
 import org.elasticsearch.xpack.lucene.bwc.codecs.lucene50.BWCLucene50PostingsFormat;
 import org.elasticsearch.xpack.lucene.bwc.codecs.lucene54.Lucene54DocValuesFormat;
+import org.elasticsearch.xpack.lucene.bwc.codecs.lucene60.Lucene60MetadataOnlyPointsFormat;
 
 import java.util.Objects;
 
@@ -111,4 +113,9 @@ public class Lucene62Codec extends BWCCodec {
     public PostingsFormat postingsFormat() {
         return postingsFormat;
     }
+
+    @Override
+    public PointsFormat pointsFormat() {
+        return new Lucene60MetadataOnlyPointsFormat();
+    }
 }

+ 7 - 0
x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene70/BWCLucene70Codec.java

@@ -16,12 +16,14 @@ import org.apache.lucene.codecs.CompoundFormat;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.codecs.PointsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 import org.elasticsearch.xpack.lucene.bwc.codecs.BWCCodec;
+import org.elasticsearch.xpack.lucene.bwc.codecs.lucene60.Lucene60MetadataOnlyPointsFormat;
 
 public class BWCLucene70Codec extends BWCCodec {
 
@@ -83,4 +85,9 @@ public class BWCLucene70Codec extends BWCCodec {
     public PostingsFormat postingsFormat() {
         return postingsFormat;
     }
+
+    @Override
+    public PointsFormat pointsFormat() {
+        return new Lucene60MetadataOnlyPointsFormat();
+    }
 }

+ 26 - 5
x-pack/qa/repository-old-versions/src/test/java/org/elasticsearch/oldrepos/OldRepositoryAccessIT.java

@@ -45,6 +45,7 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
@@ -280,7 +281,13 @@ public class OldRepositoryAccessIT extends ESRestTestCase {
     }
 
     private static String sourceForDoc(int i) {
-        return "{\"test\":\"test" + i + "\",\"val\":" + i + "}";
+        return "{\"test\":\"test"
+            + i
+            + "\",\"val\":"
+            + i
+            + ",\"create_date\":\"2020-01-"
+            + String.format(Locale.ROOT, "%02d", i + 1)
+            + "\"}";
     }
 
     @SuppressWarnings("removal")
@@ -340,7 +347,7 @@ public class OldRepositoryAccessIT extends ESRestTestCase {
         }
 
         // run a search against the index
-        assertDocs("restored_" + indexName, numDocs, expectedIds, client, sourceOnlyRepository, oldVersion);
+        assertDocs("restored_" + indexName, numDocs, expectedIds, client, sourceOnlyRepository, oldVersion, numberOfShards);
 
         // mount as full copy searchable snapshot
         Request mountRequest = new Request("POST", "/_snapshot/" + repoName + "/" + snapshotName + "/_mount");
@@ -360,7 +367,7 @@ public class OldRepositoryAccessIT extends ESRestTestCase {
         ensureGreen("mounted_full_copy_" + indexName);
 
         // run a search against the index
-        assertDocs("mounted_full_copy_" + indexName, numDocs, expectedIds, client, sourceOnlyRepository, oldVersion);
+        assertDocs("mounted_full_copy_" + indexName, numDocs, expectedIds, client, sourceOnlyRepository, oldVersion, numberOfShards);
 
         // mount as shared cache searchable snapshot
         mountRequest = new Request("POST", "/_snapshot/" + repoName + "/" + snapshotName + "/_mount");
@@ -373,7 +380,7 @@ public class OldRepositoryAccessIT extends ESRestTestCase {
         assertEquals(numberOfShards, (int) mountResponse.evaluate("snapshot.shards.successful"));
 
         // run a search against the index
-        assertDocs("mounted_shared_cache_" + indexName, numDocs, expectedIds, client, sourceOnlyRepository, oldVersion);
+        assertDocs("mounted_shared_cache_" + indexName, numDocs, expectedIds, client, sourceOnlyRepository, oldVersion, numberOfShards);
     }
 
     @SuppressWarnings("removal")
@@ -383,7 +390,8 @@ public class OldRepositoryAccessIT extends ESRestTestCase {
         Set<String> expectedIds,
         RestHighLevelClient client,
         boolean sourceOnlyRepository,
-        Version oldVersion
+        Version oldVersion,
+        int numberOfShards
     ) throws IOException {
         RequestOptions v7RequestOptions = RequestOptions.DEFAULT.toBuilder()
             .addHeader("Content-Type", "application/vnd.elasticsearch+json;compatible-with=7")
@@ -463,6 +471,19 @@ public class OldRepositoryAccessIT extends ESRestTestCase {
                     .getMessage(),
                 containsString("get operations not allowed on a legacy index")
             );
+
+            // check that shards are skipped based on non-matching date
+            searchResponse = client.search(
+                new SearchRequest(index).source(
+                    SearchSourceBuilder.searchSource().query(QueryBuilders.rangeQuery("create_date").from("2020-02-01"))
+                ),
+                randomRequestOptions
+            );
+            logger.info(searchResponse);
+            assertEquals(0, searchResponse.getHits().getTotalHits().value);
+            assertEquals(numberOfShards, searchResponse.getSuccessfulShards());
+            // When all shards are skipped, at least one of them is queried in order to provide a proper search response.
+            assertEquals(numberOfShards - 1, searchResponse.getSkippedShards());
         }
     }