diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DateFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/DateFieldMapper.java index d5d601fb264c3..672de52624402 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DateFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DateFieldMapper.java @@ -345,6 +345,7 @@ public DateFieldMapper build(MapperBuilderContext context) { DateFieldType ft = new DateFieldType( context.buildFullName(name()), index.getValue() && indexCreatedVersion.isLegacyIndexVersion() == false, + index.getValue(), store.getValue(), docValues.getValue(), buildFormatter(), @@ -391,10 +392,12 @@ public static final class DateFieldType extends MappedFieldType { protected final Resolution resolution; protected final String nullValue; protected final FieldValues scriptValues; + private final boolean pointsMetadataAvailable; public DateFieldType( String name, boolean isIndexed, + boolean pointsMetadataAvailable, boolean isStored, boolean hasDocValues, DateFormatter dateTimeFormatter, @@ -409,26 +412,52 @@ public DateFieldType( 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 scriptValues, + Map 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 @@ -650,7 +679,7 @@ public Relation isFieldWithinQuery( 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; } diff --git a/x-pack/plugin/old-lucene-versions/src/internalClusterTest/java/org/elasticsearch/xpack/lucene/bwc/AbstractArchiveTestCase.java b/x-pack/plugin/old-lucene-versions/src/internalClusterTest/java/org/elasticsearch/xpack/lucene/bwc/AbstractArchiveTestCase.java index dc877b2d90cb5..fb08759587899 100644 --- a/x-pack/plugin/old-lucene-versions/src/internalClusterTest/java/org/elasticsearch/xpack/lucene/bwc/AbstractArchiveTestCase.java +++ b/x-pack/plugin/old-lucene-versions/src/internalClusterTest/java/org/elasticsearch/xpack/lucene/bwc/AbstractArchiveTestCase.java @@ -41,6 +41,11 @@ @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> nodePlugins() { return Arrays.asList(LocalStateOldLuceneVersions.class, TestRepositoryPlugin.class, MockRepository.Plugin.class); diff --git a/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/OldLuceneVersions.java b/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/OldLuceneVersions.java index 86b4b3eb1ef6a..45c68b96e85f7 100644 --- a/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/OldLuceneVersions.java +++ b/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/OldLuceneVersions.java @@ -29,16 +29,21 @@ 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.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.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 @@ private static SegmentInfos convertToNewerLuceneVersion(OldSegmentInfos oldSegme public Map getDirectoryFactories() { return Map.of(); } + + @Override + public Optional 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(); + } } diff --git a/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/BWCCodec.java b/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/BWCCodec.java index 97cf0aa356ec9..ec5d330a68bc2 100644 --- a/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/BWCCodec.java +++ b/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.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 TermVectorsFormat termVectorsFormat() { throw new UnsupportedOperationException(); } - @Override - public PointsFormat pointsFormat() { - throw new UnsupportedOperationException(); - } - @Override public KnnVectorsFormat knnVectorsFormat() { throw new UnsupportedOperationException(); @@ -85,7 +79,7 @@ public void write(Directory directory, SegmentInfo segmentInfo, String segmentSu }; } - // 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 fieldInfoCopy = new ArrayList<>(fieldInfos.size()); for (FieldInfo fieldInfo : fieldInfos) { @@ -100,9 +94,9 @@ private static FieldInfos filterFields(FieldInfos fieldInfos) { fieldInfo.getDocValuesType(), fieldInfo.getDocValuesGen(), fieldInfo.attributes(), - 0, - 0, - 0, + fieldInfo.getPointDimensionCount(), + fieldInfo.getPointIndexDimensionCount(), + fieldInfo.getPointNumBytes(), 0, fieldInfo.getVectorSimilarityFunction(), fieldInfo.isSoftDeletesField() diff --git a/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene60/Lucene60Codec.java b/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene60/Lucene60Codec.java index d507d49907433..7a8a0b59a99ad 100644 --- a/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene60/Lucene60Codec.java +++ b/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.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 DocValuesFormat docValuesFormat() { public PostingsFormat postingsFormat() { return postingsFormat; } + + @Override + public PointsFormat pointsFormat() { + return new Lucene60MetadataOnlyPointsFormat(); + } } diff --git a/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene60/Lucene60MetadataOnlyPointsFormat.java b/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene60/Lucene60MetadataOnlyPointsFormat.java new file mode 100644 index 0000000000000..fc90a3e14b944 --- /dev/null +++ b/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); + } +} diff --git a/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene60/Lucene60MetadataOnlyPointsReader.java b/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene60/Lucene60MetadataOnlyPointsReader.java new file mode 100644 index 0000000000000..96170a946d6b3 --- /dev/null +++ b/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 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 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 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(); + } +} diff --git a/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene60/MetadataOnlyBKDReader.java b/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene60/MetadataOnlyBKDReader.java new file mode 100644 index 0000000000000..f3ce3ea0755e1 --- /dev/null +++ b/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; + } +} diff --git a/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene62/Lucene62Codec.java b/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene62/Lucene62Codec.java index 85084317977b3..d43b306aad9c8 100644 --- a/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene62/Lucene62Codec.java +++ b/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.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.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 DocValuesFormat docValuesFormat() { public PostingsFormat postingsFormat() { return postingsFormat; } + + @Override + public PointsFormat pointsFormat() { + return new Lucene60MetadataOnlyPointsFormat(); + } } diff --git a/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene70/BWCLucene70Codec.java b/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene70/BWCLucene70Codec.java index 8e52baa9a73c5..0e689138acd8f 100644 --- a/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/lucene70/BWCLucene70Codec.java +++ b/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.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 final DocValuesFormat docValuesFormat() { public PostingsFormat postingsFormat() { return postingsFormat; } + + @Override + public PointsFormat pointsFormat() { + return new Lucene60MetadataOnlyPointsFormat(); + } } diff --git a/x-pack/qa/repository-old-versions/src/test/java/org/elasticsearch/oldrepos/OldRepositoryAccessIT.java b/x-pack/qa/repository-old-versions/src/test/java/org/elasticsearch/oldrepos/OldRepositoryAccessIT.java index bd584302da02f..b2ddf99b9c70c 100644 --- a/x-pack/qa/repository-old-versions/src/test/java/org/elasticsearch/oldrepos/OldRepositoryAccessIT.java +++ b/x-pack/qa/repository-old-versions/src/test/java/org/elasticsearch/oldrepos/OldRepositoryAccessIT.java @@ -45,6 +45,7 @@ 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 @@ private String getType(Version oldVersion, String id) { } 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 @@ private void restoreMountAndVerify( } // 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 @@ private void restoreMountAndVerify( 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 @@ private void restoreMountAndVerify( 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 @@ private void assertDocs( Set 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 @@ private void assertDocs( .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()); } }