args) {
super(args);
- Version luceneMatchVersion = getLuceneMatchVersion();
- @SuppressWarnings("deprecation")
- Version LUCENE_8_4_0 = Version.LUCENE_8_4_0;
- if (luceneMatchVersion.onOrAfter(LUCENE_8_4_0)) {
- tokenSeparator =
- getCharacter(args, "tokenSeparator", ConcatenateGraphFilter.DEFAULT_TOKEN_SEPARATOR);
- } else {
- boolean preserveSep =
- getBoolean(args, "preserveSep", ConcatenateGraphFilter.DEFAULT_PRESERVE_SEP);
- tokenSeparator = (preserveSep) ? ConcatenateGraphFilter.DEFAULT_TOKEN_SEPARATOR : null;
- }
+ tokenSeparator =
+ getCharacter(args, "tokenSeparator", ConcatenateGraphFilter.DEFAULT_TOKEN_SEPARATOR);
preservePositionIncrements =
getBoolean(
args,
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
index 66e74d5dc7d..f87551279f6 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
@@ -49,7 +49,7 @@ import org.apache.lucene.util.Version;
public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
@SuppressWarnings("deprecation")
- private static final Version LUCENE_8_0_0 = Version.LUCENE_8_0_0;
+ private static final Version LUCENE_9_0_0 = Version.LUCENE_9_0_0;
// Test some examples (TODO: we only check behavior, we may need something like
// TestRandomChains...)
@@ -111,7 +111,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
public void testVersionAwareFilter() throws Exception {
CustomAnalyzer a =
CustomAnalyzer.builder()
- .withDefaultMatchVersion(Version.LUCENE_8_0_0)
+ .withDefaultMatchVersion(LUCENE_9_0_0)
.withTokenizer(StandardTokenizerFactory.class)
.addTokenFilter(DummyVersionAwareTokenFilterFactory.class)
.build();
@@ -128,7 +128,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
public void testFactoryHtmlStripClassicFolding() throws Exception {
CustomAnalyzer a =
CustomAnalyzer.builder()
- .withDefaultMatchVersion(LUCENE_8_0_0)
+ .withDefaultMatchVersion(LUCENE_9_0_0)
.addCharFilter(HTMLStripCharFilterFactory.class)
.withTokenizer(ClassicTokenizerFactory.class)
.addTokenFilter(ASCIIFoldingFilterFactory.class, "preserveOriginal", "true")
@@ -164,7 +164,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
public void testHtmlStripClassicFolding() throws Exception {
CustomAnalyzer a =
CustomAnalyzer.builder()
- .withDefaultMatchVersion(LUCENE_8_0_0)
+ .withDefaultMatchVersion(LUCENE_9_0_0)
.addCharFilter("htmlstrip")
.withTokenizer("classic")
.addTokenFilter("asciifolding", "preserveOriginal", "true")
@@ -507,7 +507,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
@Override
public TokenStream create(TokenStream input) {
- if (luceneMatchVersion.equals(Version.LUCENE_8_0_0)) {
+ if (luceneMatchVersion.equals(LUCENE_9_0_0)) {
return input;
}
return new LowerCaseFilter(input);
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConcatenateGraphFilterFactory.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConcatenateGraphFilterFactory.java
index 25236d62553..3b60c2374f5 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConcatenateGraphFilterFactory.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConcatenateGraphFilterFactory.java
@@ -23,7 +23,6 @@ import org.apache.lucene.analysis.BaseTokenStreamFactoryTestCase;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.StopFilter;
import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.util.Version;
public class TestConcatenateGraphFilterFactory extends BaseTokenStreamFactoryTestCase {
public void test() throws Exception {
@@ -52,21 +51,6 @@ public class TestConcatenateGraphFilterFactory extends BaseTokenStreamFactoryTes
assertTokenStreamContents(stream, new String[] {output});
}
- public void testPreserveSep() throws Exception {
- final String input = "A1 B2 A1 D4 C3";
- final String output = "A1A1D4C3";
- Reader reader = new StringReader(input);
- MockTokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
- tokenizer.setReader(reader);
- TokenStream stream = tokenizer;
- stream = new StopFilter(stream, StopFilter.makeStopSet("B2"));
- @SuppressWarnings("deprecation")
- Version LUCENE_8_0_0 = Version.LUCENE_8_0_0;
- stream =
- tokenFilterFactory("ConcatenateGraph", LUCENE_8_0_0, "preserveSep", "false").create(stream);
- assertTokenStreamContents(stream, new String[] {output});
- }
-
public void testPreservePositionIncrements() throws Exception {
final String input = "A1 B2 A1 D4 C3";
final String output = "A1 A1 D4 C3";
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70Codec.java
deleted file mode 100644
index a30f7d061de..00000000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70Codec.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.backward_codecs.lucene70;
-
-import org.apache.lucene.backward_codecs.lucene50.Lucene50CompoundFormat;
-import org.apache.lucene.backward_codecs.lucene50.Lucene50LiveDocsFormat;
-import org.apache.lucene.backward_codecs.lucene50.Lucene50StoredFieldsFormat;
-import org.apache.lucene.backward_codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
-import org.apache.lucene.backward_codecs.lucene50.Lucene50TermVectorsFormat;
-import org.apache.lucene.backward_codecs.lucene60.Lucene60FieldInfosFormat;
-import org.apache.lucene.backward_codecs.lucene60.Lucene60PointsFormat;
-import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.CompoundFormat;
-import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.FieldInfosFormat;
-import org.apache.lucene.codecs.FilterCodec;
-import org.apache.lucene.codecs.KnnVectorsFormat;
-import org.apache.lucene.codecs.LiveDocsFormat;
-import org.apache.lucene.codecs.NormsFormat;
-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.TermVectorsFormat;
-import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
-import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
-
-/**
- * Implements the Lucene 7.0 index format, with configurable per-field postings and docvalues
- * formats.
- *
- * If you want to reuse functionality of this codec in another codec, extend {@link FilterCodec}.
- *
- * @see org.apache.lucene.backward_codecs.lucene70 package documentation for file format details.
- * @lucene.experimental
- */
-public class Lucene70Codec extends Codec {
- private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
- private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
- private final SegmentInfoFormat segmentInfosFormat = new Lucene70SegmentInfoFormat();
- private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
- private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
- private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene70");
-
- private final PostingsFormat postingsFormat =
- new PerFieldPostingsFormat() {
- @Override
- public PostingsFormat getPostingsFormatForField(String field) {
- throw new IllegalStateException(
- "This codec should only be used for reading, not writing");
- }
- };
-
- private final DocValuesFormat docValuesFormat =
- new PerFieldDocValuesFormat() {
- @Override
- public DocValuesFormat getDocValuesFormatForField(String field) {
- return defaultDVFormat;
- }
- };
-
- private final StoredFieldsFormat storedFieldsFormat =
- new Lucene50StoredFieldsFormat(Mode.BEST_SPEED);
-
- /** Instantiates a new codec. */
- public Lucene70Codec() {
- super("Lucene70");
- }
-
- @Override
- public StoredFieldsFormat storedFieldsFormat() {
- return storedFieldsFormat;
- }
-
- @Override
- public TermVectorsFormat termVectorsFormat() {
- return vectorsFormat;
- }
-
- @Override
- public PostingsFormat postingsFormat() {
- return postingsFormat;
- }
-
- @Override
- public final FieldInfosFormat fieldInfosFormat() {
- return fieldInfosFormat;
- }
-
- @Override
- public SegmentInfoFormat segmentInfoFormat() {
- return segmentInfosFormat;
- }
-
- @Override
- public final LiveDocsFormat liveDocsFormat() {
- return liveDocsFormat;
- }
-
- @Override
- public CompoundFormat compoundFormat() {
- return compoundFormat;
- }
-
- @Override
- public final PointsFormat pointsFormat() {
- return new Lucene60PointsFormat();
- }
-
- @Override
- public KnnVectorsFormat knnVectorsFormat() {
- return KnnVectorsFormat.EMPTY;
- }
-
- @Override
- public final DocValuesFormat docValuesFormat() {
- return docValuesFormat;
- }
-
- private final NormsFormat normsFormat = new Lucene70NormsFormat();
-
- @Override
- public NormsFormat normsFormat() {
- return normsFormat;
- }
-}
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesConsumer.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesConsumer.java
deleted file mode 100644
index ec8909bac71..00000000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesConsumer.java
+++ /dev/null
@@ -1,704 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.backward_codecs.lucene70;
-
-import static org.apache.lucene.backward_codecs.lucene70.Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
-import static org.apache.lucene.backward_codecs.lucene70.Lucene70DocValuesFormat.NUMERIC_BLOCK_SHIFT;
-import static org.apache.lucene.backward_codecs.lucene70.Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import org.apache.lucene.backward_codecs.packed.LegacyDirectMonotonicWriter;
-import org.apache.lucene.backward_codecs.packed.LegacyDirectWriter;
-import org.apache.lucene.backward_codecs.store.EndiannessReverserUtil;
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.EmptyDocValuesProducer;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedNumericDocValues;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.SortedSetSelector;
-import org.apache.lucene.store.ByteBuffersDataOutput;
-import org.apache.lucene.store.ByteBuffersIndexOutput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.MathUtil;
-import org.apache.lucene.util.StringHelper;
-
-/** writer for {@link Lucene70DocValuesFormat} */
-final class Lucene70DocValuesConsumer extends DocValuesConsumer {
-
- IndexOutput data, meta;
- final int maxDoc;
-
- /** expert: Creates a new writer */
- public Lucene70DocValuesConsumer(
- SegmentWriteState state,
- String dataCodec,
- String dataExtension,
- String metaCodec,
- String metaExtension)
- throws IOException {
- boolean success = false;
- try {
- String dataName =
- IndexFileNames.segmentFileName(
- state.segmentInfo.name, state.segmentSuffix, dataExtension);
- data = EndiannessReverserUtil.createOutput(state.directory, dataName, state.context);
- CodecUtil.writeIndexHeader(
- data,
- dataCodec,
- Lucene70DocValuesFormat.VERSION_CURRENT,
- state.segmentInfo.getId(),
- state.segmentSuffix);
- String metaName =
- IndexFileNames.segmentFileName(
- state.segmentInfo.name, state.segmentSuffix, metaExtension);
- meta = EndiannessReverserUtil.createOutput(state.directory, metaName, state.context);
- CodecUtil.writeIndexHeader(
- meta,
- metaCodec,
- Lucene70DocValuesFormat.VERSION_CURRENT,
- state.segmentInfo.getId(),
- state.segmentSuffix);
- maxDoc = state.segmentInfo.maxDoc();
- success = true;
- } finally {
- if (!success) {
- IOUtils.closeWhileHandlingException(this);
- }
- }
- }
-
- @Override
- public void close() throws IOException {
- boolean success = false;
- try {
- if (meta != null) {
- meta.writeInt(-1); // write EOF marker
- CodecUtil.writeFooter(meta); // write checksum
- }
- if (data != null) {
- CodecUtil.writeFooter(data); // write checksum
- }
- success = true;
- } finally {
- if (success) {
- IOUtils.close(data, meta);
- } else {
- IOUtils.closeWhileHandlingException(data, meta);
- }
- meta = data = null;
- }
- }
-
- @Override
- public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer)
- throws IOException {
- meta.writeInt(field.number);
- meta.writeByte(Lucene70DocValuesFormat.NUMERIC);
-
- writeValues(
- field,
- new EmptyDocValuesProducer() {
- @Override
- public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
- return DocValues.singleton(valuesProducer.getNumeric(field));
- }
- });
- }
-
- private static class MinMaxTracker {
- long min, max, numValues, spaceInBits;
-
- MinMaxTracker() {
- reset();
- spaceInBits = 0;
- }
-
- private void reset() {
- min = Long.MAX_VALUE;
- max = Long.MIN_VALUE;
- numValues = 0;
- }
-
- /** Accumulate a new value. */
- void update(long v) {
- min = Math.min(min, v);
- max = Math.max(max, v);
- ++numValues;
- }
-
- /** Update the required space. */
- void finish() {
- if (max > min) {
- spaceInBits += LegacyDirectWriter.unsignedBitsRequired(max - min) * numValues;
- }
- }
-
- /** Update space usage and get ready for accumulating values for the next block. */
- void nextBlock() {
- finish();
- reset();
- }
- }
-
- private long[] writeValues(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
- SortedNumericDocValues values = valuesProducer.getSortedNumeric(field);
- int numDocsWithValue = 0;
- MinMaxTracker minMax = new MinMaxTracker();
- MinMaxTracker blockMinMax = new MinMaxTracker();
- long gcd = 0;
- Set uniqueValues = new HashSet<>();
- for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
- for (int i = 0, count = values.docValueCount(); i < count; ++i) {
- long v = values.nextValue();
-
- if (gcd != 1) {
- if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
- // in that case v - minValue might overflow and make the GCD computation return
- // wrong results. Since these extreme values are unlikely, we just discard
- // GCD computation for them
- gcd = 1;
- } else if (minMax.numValues != 0) { // minValue needs to be set first
- gcd = MathUtil.gcd(gcd, v - minMax.min);
- }
- }
-
- minMax.update(v);
- blockMinMax.update(v);
- if (blockMinMax.numValues == NUMERIC_BLOCK_SIZE) {
- blockMinMax.nextBlock();
- }
-
- if (uniqueValues != null && uniqueValues.add(v) && uniqueValues.size() > 256) {
- uniqueValues = null;
- }
- }
-
- numDocsWithValue++;
- }
-
- minMax.finish();
- blockMinMax.finish();
-
- final long numValues = minMax.numValues;
- long min = minMax.min;
- final long max = minMax.max;
- assert blockMinMax.spaceInBits <= minMax.spaceInBits;
-
- if (numDocsWithValue == 0) {
- meta.writeLong(-2);
- meta.writeLong(0L);
- } else if (numDocsWithValue == maxDoc) {
- meta.writeLong(-1);
- meta.writeLong(0L);
- } else {
- long offset = data.getFilePointer();
- meta.writeLong(offset);
- values = valuesProducer.getSortedNumeric(field);
- IndexedDISI.writeBitSet(values, data);
- meta.writeLong(data.getFilePointer() - offset);
- }
-
- meta.writeLong(numValues);
- final int numBitsPerValue;
- boolean doBlocks = false;
- Map encode = null;
- if (min >= max) {
- numBitsPerValue = 0;
- meta.writeInt(-1);
- } else {
- if (uniqueValues != null
- && uniqueValues.size() > 1
- && LegacyDirectWriter.unsignedBitsRequired(uniqueValues.size() - 1)
- < LegacyDirectWriter.unsignedBitsRequired((max - min) / gcd)) {
- numBitsPerValue = LegacyDirectWriter.unsignedBitsRequired(uniqueValues.size() - 1);
- final Long[] sortedUniqueValues = uniqueValues.toArray(new Long[0]);
- Arrays.sort(sortedUniqueValues);
- meta.writeInt(sortedUniqueValues.length);
- for (Long v : sortedUniqueValues) {
- meta.writeLong(v);
- }
- encode = new HashMap<>();
- for (int i = 0; i < sortedUniqueValues.length; ++i) {
- encode.put(sortedUniqueValues[i], i);
- }
- min = 0;
- gcd = 1;
- } else {
- uniqueValues = null;
- // we do blocks if that appears to save 10+% storage
- doBlocks =
- minMax.spaceInBits > 0 && (double) blockMinMax.spaceInBits / minMax.spaceInBits <= 0.9;
- if (doBlocks) {
- numBitsPerValue = 0xFF;
- meta.writeInt(-2 - NUMERIC_BLOCK_SHIFT);
- } else {
- numBitsPerValue = LegacyDirectWriter.unsignedBitsRequired((max - min) / gcd);
- if (gcd == 1
- && min > 0
- && LegacyDirectWriter.unsignedBitsRequired(max)
- == LegacyDirectWriter.unsignedBitsRequired(max - min)) {
- min = 0;
- }
- meta.writeInt(-1);
- }
- }
- }
-
- meta.writeByte((byte) numBitsPerValue);
- meta.writeLong(min);
- meta.writeLong(gcd);
- long startOffset = data.getFilePointer();
- meta.writeLong(startOffset);
- if (doBlocks) {
- writeValuesMultipleBlocks(valuesProducer.getSortedNumeric(field), gcd);
- } else if (numBitsPerValue != 0) {
- writeValuesSingleBlock(
- valuesProducer.getSortedNumeric(field), numValues, numBitsPerValue, min, gcd, encode);
- }
- meta.writeLong(data.getFilePointer() - startOffset);
-
- return new long[] {numDocsWithValue, numValues};
- }
-
- private void writeValuesSingleBlock(
- SortedNumericDocValues values,
- long numValues,
- int numBitsPerValue,
- long min,
- long gcd,
- Map encode)
- throws IOException {
- LegacyDirectWriter writer = LegacyDirectWriter.getInstance(data, numValues, numBitsPerValue);
- for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
- for (int i = 0, count = values.docValueCount(); i < count; ++i) {
- long v = values.nextValue();
- if (encode == null) {
- writer.add((v - min) / gcd);
- } else {
- writer.add(encode.get(v));
- }
- }
- }
- writer.finish();
- }
-
- private void writeValuesMultipleBlocks(SortedNumericDocValues values, long gcd)
- throws IOException {
- final long[] buffer = new long[NUMERIC_BLOCK_SIZE];
- final ByteBuffersDataOutput encodeBuffer = ByteBuffersDataOutput.newResettableInstance();
- int upTo = 0;
- for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
- for (int i = 0, count = values.docValueCount(); i < count; ++i) {
- buffer[upTo++] = values.nextValue();
- if (upTo == NUMERIC_BLOCK_SIZE) {
- writeBlock(buffer, NUMERIC_BLOCK_SIZE, gcd, encodeBuffer);
- upTo = 0;
- }
- }
- }
- if (upTo > 0) {
- writeBlock(buffer, upTo, gcd, encodeBuffer);
- }
- }
-
- private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutput buffer)
- throws IOException {
- assert length > 0;
- long min = values[0];
- long max = values[0];
- for (int i = 1; i < length; ++i) {
- final long v = values[i];
- assert Math.floorMod(values[i] - min, gcd) == 0;
- min = Math.min(min, v);
- max = Math.max(max, v);
- }
- if (min == max) {
- data.writeByte((byte) 0);
- data.writeLong(min);
- } else {
- final int bitsPerValue = LegacyDirectWriter.unsignedBitsRequired(max - min);
- buffer.reset();
- assert buffer.size() == 0;
- final LegacyDirectWriter w = LegacyDirectWriter.getInstance(buffer, length, bitsPerValue);
- for (int i = 0; i < length; ++i) {
- w.add((values[i] - min) / gcd);
- }
- w.finish();
- data.writeByte((byte) bitsPerValue);
- data.writeLong(min);
- data.writeInt(Math.toIntExact(buffer.size()));
- buffer.copyTo(data);
- }
- }
-
- @Override
- public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
- meta.writeInt(field.number);
- meta.writeByte(Lucene70DocValuesFormat.BINARY);
-
- BinaryDocValues values = valuesProducer.getBinary(field);
- long start = data.getFilePointer();
- meta.writeLong(start);
- int numDocsWithField = 0;
- int minLength = Integer.MAX_VALUE;
- int maxLength = 0;
- for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
- numDocsWithField++;
- BytesRef v = values.binaryValue();
- int length = v.length;
- data.writeBytes(v.bytes, v.offset, v.length);
- minLength = Math.min(length, minLength);
- maxLength = Math.max(length, maxLength);
- }
- assert numDocsWithField <= maxDoc;
- meta.writeLong(data.getFilePointer() - start);
-
- if (numDocsWithField == 0) {
- meta.writeLong(-2);
- meta.writeLong(0L);
- } else if (numDocsWithField == maxDoc) {
- meta.writeLong(-1);
- meta.writeLong(0L);
- } else {
- long offset = data.getFilePointer();
- meta.writeLong(offset);
- values = valuesProducer.getBinary(field);
- IndexedDISI.writeBitSet(values, data);
- meta.writeLong(data.getFilePointer() - offset);
- }
-
- meta.writeInt(numDocsWithField);
- meta.writeInt(minLength);
- meta.writeInt(maxLength);
- if (maxLength > minLength) {
- start = data.getFilePointer();
- meta.writeLong(start);
- meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
-
- final LegacyDirectMonotonicWriter writer =
- LegacyDirectMonotonicWriter.getInstance(
- meta, data, numDocsWithField + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
- long addr = 0;
- writer.add(addr);
- values = valuesProducer.getBinary(field);
- for (int doc = values.nextDoc();
- doc != DocIdSetIterator.NO_MORE_DOCS;
- doc = values.nextDoc()) {
- addr += values.binaryValue().length;
- writer.add(addr);
- }
- writer.finish();
- meta.writeLong(data.getFilePointer() - start);
- }
- }
-
- @Override
- public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
- meta.writeInt(field.number);
- meta.writeByte(Lucene70DocValuesFormat.SORTED);
- doAddSortedField(field, valuesProducer);
- }
-
- private void doAddSortedField(FieldInfo field, DocValuesProducer valuesProducer)
- throws IOException {
- SortedDocValues values = valuesProducer.getSorted(field);
- int numDocsWithField = 0;
- for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
- numDocsWithField++;
- }
-
- if (numDocsWithField == 0) {
- meta.writeLong(-2);
- meta.writeLong(0L);
- } else if (numDocsWithField == maxDoc) {
- meta.writeLong(-1);
- meta.writeLong(0L);
- } else {
- long offset = data.getFilePointer();
- meta.writeLong(offset);
- values = valuesProducer.getSorted(field);
- IndexedDISI.writeBitSet(values, data);
- meta.writeLong(data.getFilePointer() - offset);
- }
-
- meta.writeInt(numDocsWithField);
- if (values.getValueCount() <= 1) {
- meta.writeByte((byte) 0);
- meta.writeLong(0L);
- meta.writeLong(0L);
- } else {
- int numberOfBitsPerOrd = LegacyDirectWriter.unsignedBitsRequired(values.getValueCount() - 1);
- meta.writeByte((byte) numberOfBitsPerOrd);
- long start = data.getFilePointer();
- meta.writeLong(start);
- LegacyDirectWriter writer =
- LegacyDirectWriter.getInstance(data, numDocsWithField, numberOfBitsPerOrd);
- values = valuesProducer.getSorted(field);
- for (int doc = values.nextDoc();
- doc != DocIdSetIterator.NO_MORE_DOCS;
- doc = values.nextDoc()) {
- writer.add(values.ordValue());
- }
- writer.finish();
- meta.writeLong(data.getFilePointer() - start);
- }
-
- addTermsDict(DocValues.singleton(valuesProducer.getSorted(field)));
- }
-
- private void addTermsDict(SortedSetDocValues values) throws IOException {
- final long size = values.getValueCount();
- meta.writeVLong(size);
- meta.writeInt(Lucene70DocValuesFormat.TERMS_DICT_BLOCK_SHIFT);
-
- ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput();
- ByteBuffersIndexOutput addressIndexOut =
- new ByteBuffersIndexOutput(addressBuffer, "temp", "temp");
- meta.writeInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
- long numBlocks =
- (size + Lucene70DocValuesFormat.TERMS_DICT_BLOCK_MASK)
- >>> Lucene70DocValuesFormat.TERMS_DICT_BLOCK_SHIFT;
- LegacyDirectMonotonicWriter writer =
- LegacyDirectMonotonicWriter.getInstance(
- meta, addressIndexOut, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT);
-
- BytesRefBuilder previous = new BytesRefBuilder();
- long ord = 0;
- long start = data.getFilePointer();
- int maxLength = 0;
- TermsEnum iterator = values.termsEnum();
- for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
- if ((ord & Lucene70DocValuesFormat.TERMS_DICT_BLOCK_MASK) == 0) {
- writer.add(data.getFilePointer() - start);
- data.writeVInt(term.length);
- data.writeBytes(term.bytes, term.offset, term.length);
- } else {
- final int prefixLength = StringHelper.bytesDifference(previous.get(), term);
- final int suffixLength = term.length - prefixLength;
- assert suffixLength > 0; // terms are unique
-
- data.writeByte((byte) (Math.min(prefixLength, 15) | (Math.min(15, suffixLength - 1) << 4)));
- if (prefixLength >= 15) {
- data.writeVInt(prefixLength - 15);
- }
- if (suffixLength >= 16) {
- data.writeVInt(suffixLength - 16);
- }
- data.writeBytes(term.bytes, term.offset + prefixLength, term.length - prefixLength);
- }
- maxLength = Math.max(maxLength, term.length);
- previous.copyBytes(term);
- ++ord;
- }
- writer.finish();
- meta.writeInt(maxLength);
- meta.writeLong(start);
- meta.writeLong(data.getFilePointer() - start);
- start = data.getFilePointer();
- addressBuffer.copyTo(data);
- meta.writeLong(start);
- meta.writeLong(data.getFilePointer() - start);
-
- // Now write the reverse terms index
- writeTermsIndex(values);
- }
-
- private void writeTermsIndex(SortedSetDocValues values) throws IOException {
- final long size = values.getValueCount();
- meta.writeInt(Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_SHIFT);
- long start = data.getFilePointer();
-
- long numBlocks =
- 1L
- + ((size + Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK)
- >>> Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_SHIFT);
- ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput();
- ByteBuffersIndexOutput addressIndexOut =
- new ByteBuffersIndexOutput(addressBuffer, "temp", "temp");
- LegacyDirectMonotonicWriter writer =
- LegacyDirectMonotonicWriter.getInstance(
- meta, addressIndexOut, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT);
-
- TermsEnum iterator = values.termsEnum();
- BytesRefBuilder previous = new BytesRefBuilder();
- long offset = 0;
- long ord = 0;
- for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
- if ((ord & Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) == 0) {
- writer.add(offset);
- final int sortKeyLength;
- if (ord == 0) {
- // no previous term: no bytes to write
- sortKeyLength = 0;
- } else {
- sortKeyLength = StringHelper.sortKeyLength(previous.get(), term);
- }
- offset += sortKeyLength;
- data.writeBytes(term.bytes, term.offset, sortKeyLength);
- } else if ((ord & Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK)
- == Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) {
- previous.copyBytes(term);
- }
- ++ord;
- }
- writer.add(offset);
- writer.finish();
- meta.writeLong(start);
- meta.writeLong(data.getFilePointer() - start);
- start = data.getFilePointer();
- addressBuffer.copyTo(data);
- meta.writeLong(start);
- meta.writeLong(data.getFilePointer() - start);
- }
-
- @Override
- public void addSortedNumericField(FieldInfo field, DocValuesProducer valuesProducer)
- throws IOException {
- meta.writeInt(field.number);
- meta.writeByte(Lucene70DocValuesFormat.SORTED_NUMERIC);
-
- long[] stats = writeValues(field, valuesProducer);
- int numDocsWithField = Math.toIntExact(stats[0]);
- long numValues = stats[1];
- assert numValues >= numDocsWithField;
-
- meta.writeInt(numDocsWithField);
- if (numValues > numDocsWithField) {
- long start = data.getFilePointer();
- meta.writeLong(start);
- meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
-
- final LegacyDirectMonotonicWriter addressesWriter =
- LegacyDirectMonotonicWriter.getInstance(
- meta, data, numDocsWithField + 1L, DIRECT_MONOTONIC_BLOCK_SHIFT);
- long addr = 0;
- addressesWriter.add(addr);
- SortedNumericDocValues values = valuesProducer.getSortedNumeric(field);
- for (int doc = values.nextDoc();
- doc != DocIdSetIterator.NO_MORE_DOCS;
- doc = values.nextDoc()) {
- addr += values.docValueCount();
- addressesWriter.add(addr);
- }
- addressesWriter.finish();
- meta.writeLong(data.getFilePointer() - start);
- }
- }
-
- @Override
- public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer)
- throws IOException {
- meta.writeInt(field.number);
- meta.writeByte(Lucene70DocValuesFormat.SORTED_SET);
-
- SortedSetDocValues values = valuesProducer.getSortedSet(field);
- int numDocsWithField = 0;
- long numOrds = 0;
- for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
- numDocsWithField++;
- for (long ord = values.nextOrd();
- ord != SortedSetDocValues.NO_MORE_ORDS;
- ord = values.nextOrd()) {
- numOrds++;
- }
- }
-
- if (numDocsWithField == numOrds) {
- meta.writeByte((byte) 0);
- doAddSortedField(
- field,
- new EmptyDocValuesProducer() {
- @Override
- public SortedDocValues getSorted(FieldInfo field) throws IOException {
- return SortedSetSelector.wrap(
- valuesProducer.getSortedSet(field), SortedSetSelector.Type.MIN);
- }
- });
- return;
- }
- meta.writeByte((byte) 1);
-
- assert numDocsWithField != 0;
- if (numDocsWithField == maxDoc) {
- meta.writeLong(-1);
- meta.writeLong(0L);
- } else {
- long offset = data.getFilePointer();
- meta.writeLong(offset);
- values = valuesProducer.getSortedSet(field);
- IndexedDISI.writeBitSet(values, data);
- meta.writeLong(data.getFilePointer() - offset);
- }
-
- int numberOfBitsPerOrd = LegacyDirectWriter.unsignedBitsRequired(values.getValueCount() - 1);
- meta.writeByte((byte) numberOfBitsPerOrd);
- long start = data.getFilePointer();
- meta.writeLong(start);
- LegacyDirectWriter writer = LegacyDirectWriter.getInstance(data, numOrds, numberOfBitsPerOrd);
- values = valuesProducer.getSortedSet(field);
- for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
- for (long ord = values.nextOrd();
- ord != SortedSetDocValues.NO_MORE_ORDS;
- ord = values.nextOrd()) {
- writer.add(ord);
- }
- }
- writer.finish();
- meta.writeLong(data.getFilePointer() - start);
-
- meta.writeInt(numDocsWithField);
- start = data.getFilePointer();
- meta.writeLong(start);
- meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
-
- final LegacyDirectMonotonicWriter addressesWriter =
- LegacyDirectMonotonicWriter.getInstance(
- meta, data, numDocsWithField + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
- long addr = 0;
- addressesWriter.add(addr);
- values = valuesProducer.getSortedSet(field);
- for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
- values.nextOrd();
- addr++;
- while (values.nextOrd() != SortedSetDocValues.NO_MORE_ORDS) {
- addr++;
- }
- addressesWriter.add(addr);
- }
- addressesWriter.finish();
- meta.writeLong(data.getFilePointer() - start);
-
- addTermsDict(values);
- }
-}
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesFormat.java
deleted file mode 100644
index f061dfc4d13..00000000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesFormat.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.backward_codecs.lucene70;
-
-import java.io.IOException;
-import org.apache.lucene.backward_codecs.packed.LegacyDirectWriter;
-import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.index.DocValuesType;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.SmallFloat;
-
-/**
- * Lucene 7.0 DocValues format.
- *
- * Documents that have a value for the field are encoded in a way that it is always possible to
- * know the ordinal of the current document in the set of documents that have a value. For instance,
- * say the set of documents that have a value for the field is {1, 5, 6, 11}
. When the
- * iterator is on 6
, it knows that this is the 3rd item of the set. This way, values
- * can be stored densely and accessed based on their index at search time. If all documents in a
- * segment have a value for the field, the index is the same as the doc ID, so this case is encoded
- * implicitly and is very fast at query time. On the other hand if some documents are missing a
- * value for the field then the set of documents that have a value is encoded into blocks. All doc
- * IDs that share the same upper 16 bits are encoded into the same block with the following
- * strategies:
- *
- *
- * - SPARSE: This strategy is used when a block contains at most 4095 documents. The lower 16
- * bits of doc IDs are stored as {@link DataOutput#writeShort(short) shorts} while the upper
- * 16 bits are given by the block ID.
- *
- DENSE: This strategy is used when a block contains between 4096 and 65535 documents. The
- * lower bits of doc IDs are stored in a bit set. Advancing is performed using {@link
- * Long#numberOfTrailingZeros(long) ntz} operations while the index is computed by
- * accumulating the {@link Long#bitCount(long) bit counts} of the visited longs.
- *
- ALL: This strategy is used when a block contains exactly 65536 documents, meaning that the
- * block is full. In that case doc IDs do not need to be stored explicitly. This is typically
- * faster than both SPARSE and DENSE which is a reason why it is preferable to have all
- * documents that have a value for a field using contiguous doc IDs, for instance by using
- * {@link IndexWriterConfig#setIndexSort(org.apache.lucene.search.Sort) index sorting}.
- *
- *
- * Then the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) are
- * encoded using the following strategies:
- *
- *
{@link DocValuesType#NUMERIC NUMERIC}:
- *
- *
- * - Delta-compressed: per-document integers written as deltas from the minimum value,
- * compressed with bitpacking. For more information, see {@link LegacyDirectWriter}.
- *
- Table-compressed: when the number of unique values is very small (< 256), and when there
- * are unused "gaps" in the range of values used (such as {@link SmallFloat}), a lookup table
- * is written instead. Each per-document entry is instead the ordinal to this table, and those
- * ordinals are compressed with bitpacking ({@link LegacyDirectWriter}).
- *
- GCD-compressed: when all numbers share a common divisor, such as dates, the greatest common
- * denominator (GCD) is computed, and quotients are stored using Delta-compressed Numerics.
- *
- Monotonic-compressed: when all numbers are monotonically increasing offsets, they are
- * written as blocks of bitpacked integers, encoding the deviation from the expected delta.
- *
- Const-compressed: when there is only one possible value, no per-document data is needed and
- * this value is encoded alone.
- *
- *
- * {@link DocValuesType#BINARY BINARY}:
- *
- *
- * - Fixed-width Binary: one large concatenated byte[] is written, along with the fixed length.
- * Each document's value can be addressed directly with multiplication ({@code docID *
- * length}).
- *
- Variable-width Binary: one large concatenated byte[] is written, along with end addresses
- * for each document. The addresses are written as Monotonic-compressed numerics.
- *
- Prefix-compressed Binary: values are written in chunks of 16, with the first value written
- * completely and other values sharing prefixes. chunk addresses are written as
- * Monotonic-compressed numerics. A reverse lookup index is written from a portion of every
- * 1024th term.
- *
- *
- * {@link DocValuesType#SORTED SORTED}:
- *
- *
- * - Sorted: a mapping of ordinals to deduplicated terms is written as Prefix-compressed Binary,
- * along with the per-document ordinals written using one of the numeric strategies above.
- *
- *
- * {@link DocValuesType#SORTED_SET SORTED_SET}:
- *
- *
- * - Single: if all documents have 0 or 1 value, then data are written like SORTED.
- *
- SortedSet: a mapping of ordinals to deduplicated terms is written as Binary, an ordinal
- * list and per-document index into this list are written using the numeric strategies above.
- *
- *
- * {@link DocValuesType#SORTED_NUMERIC SORTED_NUMERIC}:
- *
- *
- * - Single: if all documents have 0 or 1 value, then data are written like NUMERIC.
- *
- SortedNumeric: a value list and per-document index into this list are written using the
- * numeric strategies above.
- *
- *
- * Files:
- *
- *
- * .dvd
: DocValues data
- * .dvm
: DocValues metadata
- *
- *
- * @lucene.experimental
- */
-public final class Lucene70DocValuesFormat extends DocValuesFormat {
-
- /** Sole Constructor */
- public Lucene70DocValuesFormat() {
- super("Lucene70");
- }
-
- @Override
- public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
- return new Lucene70DocValuesConsumer(
- state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
- }
-
- @Override
- public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
- return new Lucene70DocValuesProducer(
- state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
- }
-
- static final String DATA_CODEC = "Lucene70DocValuesData";
- static final String DATA_EXTENSION = "dvd";
- static final String META_CODEC = "Lucene70DocValuesMetadata";
- static final String META_EXTENSION = "dvm";
- static final int VERSION_START = 0;
- static final int VERSION_CURRENT = VERSION_START;
-
- // indicates docvalues type
- static final byte NUMERIC = 0;
- static final byte BINARY = 1;
- static final byte SORTED = 2;
- static final byte SORTED_SET = 3;
- static final byte SORTED_NUMERIC = 4;
-
- static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16;
-
- static final int NUMERIC_BLOCK_SHIFT = 14;
- static final int NUMERIC_BLOCK_SIZE = 1 << NUMERIC_BLOCK_SHIFT;
-
- static final int TERMS_DICT_BLOCK_SHIFT = 4;
- static final int TERMS_DICT_BLOCK_SIZE = 1 << TERMS_DICT_BLOCK_SHIFT;
- static final int TERMS_DICT_BLOCK_MASK = TERMS_DICT_BLOCK_SIZE - 1;
-
- static final int TERMS_DICT_REVERSE_INDEX_SHIFT = 10;
- static final int TERMS_DICT_REVERSE_INDEX_SIZE = 1 << TERMS_DICT_REVERSE_INDEX_SHIFT;
- static final int TERMS_DICT_REVERSE_INDEX_MASK = TERMS_DICT_REVERSE_INDEX_SIZE - 1;
-}
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesProducer.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesProducer.java
deleted file mode 100644
index dbe98037ea6..00000000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesProducer.java
+++ /dev/null
@@ -1,1476 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.backward_codecs.lucene70;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.lucene.backward_codecs.packed.LegacyDirectMonotonicReader;
-import org.apache.lucene.backward_codecs.packed.LegacyDirectReader;
-import org.apache.lucene.backward_codecs.store.EndiannessReverserUtil;
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.index.BaseTermsEnum;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.ImpactsEnum;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.PostingsEnum;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedNumericDocValues;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.TermsEnum.SeekStatus;
-import org.apache.lucene.store.ChecksumIndexInput;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.RandomAccessInput;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LongValues;
-
-/** reader for {@link Lucene70DocValuesFormat} */
-final class Lucene70DocValuesProducer extends DocValuesProducer {
- private final Map numerics = new HashMap<>();
- private final Map binaries = new HashMap<>();
- private final Map sorted = new HashMap<>();
- private final Map sortedSets = new HashMap<>();
- private final Map sortedNumerics = new HashMap<>();
- private final IndexInput data;
- private final int maxDoc;
-
- /** expert: instantiates a new reader */
- Lucene70DocValuesProducer(
- SegmentReadState state,
- String dataCodec,
- String dataExtension,
- String metaCodec,
- String metaExtension)
- throws IOException {
- String metaName =
- IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
- this.maxDoc = state.segmentInfo.maxDoc();
-
- int version = -1;
-
- // read in the entries from the metadata file.
- try (ChecksumIndexInput in =
- EndiannessReverserUtil.openChecksumInput(state.directory, metaName, state.context)) {
- Throwable priorE = null;
- try {
- version =
- CodecUtil.checkIndexHeader(
- in,
- metaCodec,
- Lucene70DocValuesFormat.VERSION_START,
- Lucene70DocValuesFormat.VERSION_CURRENT,
- state.segmentInfo.getId(),
- state.segmentSuffix);
- readFields(in, state.fieldInfos);
- } catch (Throwable exception) {
- priorE = exception;
- } finally {
- CodecUtil.checkFooter(in, priorE);
- }
- }
-
- String dataName =
- IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
- this.data = EndiannessReverserUtil.openInput(state.directory, dataName, state.context);
- boolean success = false;
- try {
- final int version2 =
- CodecUtil.checkIndexHeader(
- data,
- dataCodec,
- Lucene70DocValuesFormat.VERSION_START,
- Lucene70DocValuesFormat.VERSION_CURRENT,
- state.segmentInfo.getId(),
- state.segmentSuffix);
- if (version != version2) {
- throw new CorruptIndexException(
- "Format versions mismatch: meta=" + version + ", data=" + version2, data);
- }
-
- // 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(data);
-
- success = true;
- } finally {
- if (!success) {
- IOUtils.closeWhileHandlingException(this.data);
- }
- }
- }
-
- private void readFields(ChecksumIndexInput meta, FieldInfos infos) throws IOException {
- for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
- FieldInfo info = infos.fieldInfo(fieldNumber);
- if (info == null) {
- throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
- }
- byte type = meta.readByte();
- if (type == Lucene70DocValuesFormat.NUMERIC) {
- numerics.put(info.name, readNumeric(meta));
- } else if (type == Lucene70DocValuesFormat.BINARY) {
- binaries.put(info.name, readBinary(meta));
- } else if (type == Lucene70DocValuesFormat.SORTED) {
- sorted.put(info.name, readSorted(meta));
- } else if (type == Lucene70DocValuesFormat.SORTED_SET) {
- sortedSets.put(info.name, readSortedSet(meta));
- } else if (type == Lucene70DocValuesFormat.SORTED_NUMERIC) {
- sortedNumerics.put(info.name, readSortedNumeric(meta));
- } else {
- throw new CorruptIndexException("invalid type: " + type, meta);
- }
- }
- }
-
- private NumericEntry readNumeric(ChecksumIndexInput meta) throws IOException {
- NumericEntry entry = new NumericEntry();
- readNumeric(meta, entry);
- return entry;
- }
-
- private void readNumeric(ChecksumIndexInput meta, NumericEntry entry) throws IOException {
- entry.docsWithFieldOffset = meta.readLong();
- entry.docsWithFieldLength = meta.readLong();
- entry.numValues = meta.readLong();
- int tableSize = meta.readInt();
- if (tableSize > 256) {
- throw new CorruptIndexException("invalid table size: " + tableSize, meta);
- }
- if (tableSize >= 0) {
- entry.table = new long[tableSize];
- for (int i = 0; i < tableSize; ++i) {
- entry.table[i] = meta.readLong();
- }
- }
- if (tableSize < -1) {
- entry.blockShift = -2 - tableSize;
- } else {
- entry.blockShift = -1;
- }
- entry.bitsPerValue = meta.readByte();
- entry.minValue = meta.readLong();
- entry.gcd = meta.readLong();
- entry.valuesOffset = meta.readLong();
- entry.valuesLength = meta.readLong();
- }
-
- private BinaryEntry readBinary(ChecksumIndexInput meta) throws IOException {
- BinaryEntry entry = new BinaryEntry();
- entry.dataOffset = meta.readLong();
- entry.dataLength = meta.readLong();
- entry.docsWithFieldOffset = meta.readLong();
- entry.docsWithFieldLength = meta.readLong();
- entry.numDocsWithField = meta.readInt();
- entry.minLength = meta.readInt();
- entry.maxLength = meta.readInt();
- if (entry.minLength < entry.maxLength) {
- entry.addressesOffset = meta.readLong();
- final int blockShift = meta.readVInt();
- entry.addressesMeta =
- LegacyDirectMonotonicReader.loadMeta(meta, entry.numDocsWithField + 1L, blockShift);
- entry.addressesLength = meta.readLong();
- }
- return entry;
- }
-
- private SortedEntry readSorted(ChecksumIndexInput meta) throws IOException {
- SortedEntry entry = new SortedEntry();
- entry.docsWithFieldOffset = meta.readLong();
- entry.docsWithFieldLength = meta.readLong();
- entry.numDocsWithField = meta.readInt();
- entry.bitsPerValue = meta.readByte();
- entry.ordsOffset = meta.readLong();
- entry.ordsLength = meta.readLong();
- readTermDict(meta, entry);
- return entry;
- }
-
- private SortedSetEntry readSortedSet(ChecksumIndexInput meta) throws IOException {
- SortedSetEntry entry = new SortedSetEntry();
- byte multiValued = meta.readByte();
- switch (multiValued) {
- case 0: // singlevalued
- entry.singleValueEntry = readSorted(meta);
- return entry;
- case 1: // multivalued
- break;
- default:
- throw new CorruptIndexException("Invalid multiValued flag: " + multiValued, meta);
- }
- entry.docsWithFieldOffset = meta.readLong();
- entry.docsWithFieldLength = meta.readLong();
- entry.bitsPerValue = meta.readByte();
- entry.ordsOffset = meta.readLong();
- entry.ordsLength = meta.readLong();
- entry.numDocsWithField = meta.readInt();
- entry.addressesOffset = meta.readLong();
- final int blockShift = meta.readVInt();
- entry.addressesMeta =
- LegacyDirectMonotonicReader.loadMeta(meta, entry.numDocsWithField + 1, blockShift);
- entry.addressesLength = meta.readLong();
- readTermDict(meta, entry);
- return entry;
- }
-
- private static void readTermDict(ChecksumIndexInput meta, TermsDictEntry entry)
- throws IOException {
- entry.termsDictSize = meta.readVLong();
- entry.termsDictBlockShift = meta.readInt();
- final int blockShift = meta.readInt();
- final long addressesSize =
- (entry.termsDictSize + (1L << entry.termsDictBlockShift) - 1) >>> entry.termsDictBlockShift;
- entry.termsAddressesMeta =
- LegacyDirectMonotonicReader.loadMeta(meta, addressesSize, blockShift);
- entry.maxTermLength = meta.readInt();
- entry.termsDataOffset = meta.readLong();
- entry.termsDataLength = meta.readLong();
- entry.termsAddressesOffset = meta.readLong();
- entry.termsAddressesLength = meta.readLong();
- entry.termsDictIndexShift = meta.readInt();
- final long indexSize =
- (entry.termsDictSize + (1L << entry.termsDictIndexShift) - 1) >>> entry.termsDictIndexShift;
- entry.termsIndexAddressesMeta =
- LegacyDirectMonotonicReader.loadMeta(meta, 1 + indexSize, blockShift);
- entry.termsIndexOffset = meta.readLong();
- entry.termsIndexLength = meta.readLong();
- entry.termsIndexAddressesOffset = meta.readLong();
- entry.termsIndexAddressesLength = meta.readLong();
- }
-
- private SortedNumericEntry readSortedNumeric(ChecksumIndexInput meta) throws IOException {
- SortedNumericEntry entry = new SortedNumericEntry();
- readNumeric(meta, entry);
- entry.numDocsWithField = meta.readInt();
- if (entry.numDocsWithField != entry.numValues) {
- entry.addressesOffset = meta.readLong();
- final int blockShift = meta.readVInt();
- entry.addressesMeta =
- LegacyDirectMonotonicReader.loadMeta(meta, entry.numDocsWithField + 1, blockShift);
- entry.addressesLength = meta.readLong();
- }
- return entry;
- }
-
- @Override
- public void close() throws IOException {
- data.close();
- }
-
- private static class NumericEntry {
- long[] table;
- int blockShift;
- byte bitsPerValue;
- long docsWithFieldOffset;
- long docsWithFieldLength;
- long numValues;
- long minValue;
- long gcd;
- long valuesOffset;
- long valuesLength;
- }
-
- private static class BinaryEntry {
- long dataOffset;
- long dataLength;
- long docsWithFieldOffset;
- long docsWithFieldLength;
- int numDocsWithField;
- int minLength;
- int maxLength;
- long addressesOffset;
- long addressesLength;
- LegacyDirectMonotonicReader.Meta addressesMeta;
- }
-
- private static class TermsDictEntry {
- long termsDictSize;
- int termsDictBlockShift;
- LegacyDirectMonotonicReader.Meta termsAddressesMeta;
- int maxTermLength;
- long termsDataOffset;
- long termsDataLength;
- long termsAddressesOffset;
- long termsAddressesLength;
- int termsDictIndexShift;
- LegacyDirectMonotonicReader.Meta termsIndexAddressesMeta;
- long termsIndexOffset;
- long termsIndexLength;
- long termsIndexAddressesOffset;
- long termsIndexAddressesLength;
- }
-
- private static class SortedEntry extends TermsDictEntry {
- long docsWithFieldOffset;
- long docsWithFieldLength;
- int numDocsWithField;
- byte bitsPerValue;
- long ordsOffset;
- long ordsLength;
- }
-
- private static class SortedSetEntry extends TermsDictEntry {
- SortedEntry singleValueEntry;
- long docsWithFieldOffset;
- long docsWithFieldLength;
- int numDocsWithField;
- byte bitsPerValue;
- long ordsOffset;
- long ordsLength;
- LegacyDirectMonotonicReader.Meta addressesMeta;
- long addressesOffset;
- long addressesLength;
- }
-
- private static class SortedNumericEntry extends NumericEntry {
- int numDocsWithField;
- LegacyDirectMonotonicReader.Meta addressesMeta;
- long addressesOffset;
- long addressesLength;
- }
-
- @Override
- public NumericDocValues getNumeric(FieldInfo field) throws IOException {
- NumericEntry entry = numerics.get(field.name);
- return getNumeric(entry);
- }
-
- private abstract static class DenseNumericDocValues extends NumericDocValues {
-
- final int maxDoc;
- int doc = -1;
-
- DenseNumericDocValues(int maxDoc) {
- this.maxDoc = maxDoc;
- }
-
- @Override
- public int docID() {
- return doc;
- }
-
- @Override
- public int nextDoc() throws IOException {
- return advance(doc + 1);
- }
-
- @Override
- public int advance(int target) throws IOException {
- if (target >= maxDoc) {
- return doc = NO_MORE_DOCS;
- }
- return doc = target;
- }
-
- @Override
- public boolean advanceExact(int target) {
- doc = target;
- return true;
- }
-
- @Override
- public long cost() {
- return maxDoc;
- }
- }
-
- private abstract static class SparseNumericDocValues extends NumericDocValues {
-
- final IndexedDISI disi;
-
- SparseNumericDocValues(IndexedDISI disi) {
- this.disi = disi;
- }
-
- @Override
- public int advance(int target) throws IOException {
- return disi.advance(target);
- }
-
- @Override
- public boolean advanceExact(int target) throws IOException {
- return disi.advanceExact(target);
- }
-
- @Override
- public int nextDoc() throws IOException {
- return disi.nextDoc();
- }
-
- @Override
- public int docID() {
- return disi.docID();
- }
-
- @Override
- public long cost() {
- return disi.cost();
- }
- }
-
- private NumericDocValues getNumeric(NumericEntry entry) throws IOException {
- if (entry.docsWithFieldOffset == -2) {
- // empty
- return DocValues.emptyNumeric();
- } else if (entry.docsWithFieldOffset == -1) {
- // dense
- if (entry.bitsPerValue == 0) {
- return new DenseNumericDocValues(maxDoc) {
- @Override
- public long longValue() throws IOException {
- return entry.minValue;
- }
- };
- } else {
- final RandomAccessInput slice =
- data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
- if (entry.blockShift >= 0) {
- // dense but split into blocks of different bits per value
- final int shift = entry.blockShift;
- final long mul = entry.gcd;
- final int mask = (1 << shift) - 1;
- return new DenseNumericDocValues(maxDoc) {
- int block = -1;
- long delta;
- long offset;
- long blockEndOffset;
- LongValues values;
-
- @Override
- public long longValue() throws IOException {
- final int block = doc >>> shift;
- if (this.block != block) {
- int bitsPerValue;
- do {
- offset = blockEndOffset;
- bitsPerValue = slice.readByte(offset++);
- delta = slice.readLong(offset);
- offset += Long.BYTES;
- if (bitsPerValue == 0) {
- blockEndOffset = offset;
- } else {
- final int length = slice.readInt(offset);
- offset += Integer.BYTES;
- blockEndOffset = offset + length;
- }
- this.block++;
- } while (this.block != block);
- values =
- bitsPerValue == 0
- ? LongValues.ZEROES
- : LegacyDirectReader.getInstance(slice, bitsPerValue, offset);
- }
- return mul * values.get(doc & mask) + delta;
- }
- };
- } else {
- final LongValues values = LegacyDirectReader.getInstance(slice, entry.bitsPerValue);
- if (entry.table != null) {
- final long[] table = entry.table;
- return new DenseNumericDocValues(maxDoc) {
- @Override
- public long longValue() throws IOException {
- return table[(int) values.get(doc)];
- }
- };
- } else {
- final long mul = entry.gcd;
- final long delta = entry.minValue;
- return new DenseNumericDocValues(maxDoc) {
- @Override
- public long longValue() throws IOException {
- return mul * values.get(doc) + delta;
- }
- };
- }
- }
- }
- } else {
- // sparse
- final IndexedDISI disi =
- new IndexedDISI(
- data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numValues);
- if (entry.bitsPerValue == 0) {
- return new SparseNumericDocValues(disi) {
- @Override
- public long longValue() throws IOException {
- return entry.minValue;
- }
- };
- } else {
- final RandomAccessInput slice =
- data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
- if (entry.blockShift >= 0) {
- // sparse and split into blocks of different bits per value
- final int shift = entry.blockShift;
- final long mul = entry.gcd;
- final int mask = (1 << shift) - 1;
- return new SparseNumericDocValues(disi) {
- int block = -1;
- long delta;
- long offset;
- long blockEndOffset;
- LongValues values;
-
- @Override
- public long longValue() throws IOException {
- final int index = disi.index();
- final int block = index >>> shift;
- if (this.block != block) {
- int bitsPerValue;
- do {
- offset = blockEndOffset;
- bitsPerValue = slice.readByte(offset++);
- delta = slice.readLong(offset);
- offset += Long.BYTES;
- if (bitsPerValue == 0) {
- blockEndOffset = offset;
- } else {
- final int length = slice.readInt(offset);
- offset += Integer.BYTES;
- blockEndOffset = offset + length;
- }
- this.block++;
- } while (this.block != block);
- values =
- bitsPerValue == 0
- ? LongValues.ZEROES
- : LegacyDirectReader.getInstance(slice, bitsPerValue, offset);
- }
- return mul * values.get(index & mask) + delta;
- }
- };
- } else {
- final LongValues values = LegacyDirectReader.getInstance(slice, entry.bitsPerValue);
- if (entry.table != null) {
- final long[] table = entry.table;
- return new SparseNumericDocValues(disi) {
- @Override
- public long longValue() throws IOException {
- return table[(int) values.get(disi.index())];
- }
- };
- } else {
- final long mul = entry.gcd;
- final long delta = entry.minValue;
- return new SparseNumericDocValues(disi) {
- @Override
- public long longValue() throws IOException {
- return mul * values.get(disi.index()) + delta;
- }
- };
- }
- }
- }
- }
- }
-
- private LongValues getNumericValues(NumericEntry entry) throws IOException {
- if (entry.bitsPerValue == 0) {
- return new LongValues() {
- @Override
- public long get(long index) {
- return entry.minValue;
- }
- };
- } else {
- final RandomAccessInput slice =
- data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
- if (entry.blockShift >= 0) {
- final int shift = entry.blockShift;
- final long mul = entry.gcd;
- final long mask = (1L << shift) - 1;
- return new LongValues() {
- long block = -1;
- long delta;
- long offset;
- long blockEndOffset;
- LongValues values;
-
- @Override
- public long get(long index) {
- final long block = index >>> shift;
- if (this.block != block) {
- assert block > this.block
- : "Reading backwards is illegal: " + this.block + " < " + block;
- int bitsPerValue;
- do {
- offset = blockEndOffset;
- try {
- bitsPerValue = slice.readByte(offset++);
- delta = slice.readLong(offset);
- offset += Long.BYTES;
- if (bitsPerValue == 0) {
- blockEndOffset = offset;
- } else {
- final int length = slice.readInt(offset);
- offset += Integer.BYTES;
- blockEndOffset = offset + length;
- }
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- this.block++;
- } while (this.block != block);
- values =
- bitsPerValue == 0
- ? LongValues.ZEROES
- : LegacyDirectReader.getInstance(slice, bitsPerValue, offset);
- }
- return mul * values.get(index & mask) + delta;
- }
- };
- } else {
- final LongValues values = LegacyDirectReader.getInstance(slice, entry.bitsPerValue);
- if (entry.table != null) {
- final long[] table = entry.table;
- return new LongValues() {
- @Override
- public long get(long index) {
- return table[(int) values.get(index)];
- }
- };
- } else if (entry.gcd != 1) {
- final long gcd = entry.gcd;
- final long minValue = entry.minValue;
- return new LongValues() {
- @Override
- public long get(long index) {
- return values.get(index) * gcd + minValue;
- }
- };
- } else if (entry.minValue != 0) {
- final long minValue = entry.minValue;
- return new LongValues() {
- @Override
- public long get(long index) {
- return values.get(index) + minValue;
- }
- };
- } else {
- return values;
- }
- }
- }
- }
-
- private abstract static class DenseBinaryDocValues extends BinaryDocValues {
-
- final int maxDoc;
- int doc = -1;
-
- DenseBinaryDocValues(int maxDoc) {
- this.maxDoc = maxDoc;
- }
-
- @Override
- public int nextDoc() throws IOException {
- return advance(doc + 1);
- }
-
- @Override
- public int docID() {
- return doc;
- }
-
- @Override
- public long cost() {
- return maxDoc;
- }
-
- @Override
- public int advance(int target) throws IOException {
- if (target >= maxDoc) {
- return doc = NO_MORE_DOCS;
- }
- return doc = target;
- }
-
- @Override
- public boolean advanceExact(int target) throws IOException {
- doc = target;
- return true;
- }
- }
-
- private abstract static class SparseBinaryDocValues extends BinaryDocValues {
-
- final IndexedDISI disi;
-
- SparseBinaryDocValues(IndexedDISI disi) {
- this.disi = disi;
- }
-
- @Override
- public int nextDoc() throws IOException {
- return disi.nextDoc();
- }
-
- @Override
- public int docID() {
- return disi.docID();
- }
-
- @Override
- public long cost() {
- return disi.cost();
- }
-
- @Override
- public int advance(int target) throws IOException {
- return disi.advance(target);
- }
-
- @Override
- public boolean advanceExact(int target) throws IOException {
- return disi.advanceExact(target);
- }
- }
-
- @Override
- public BinaryDocValues getBinary(FieldInfo field) throws IOException {
- BinaryEntry entry = binaries.get(field.name);
- if (entry.docsWithFieldOffset == -2) {
- return DocValues.emptyBinary();
- }
-
- final IndexInput bytesSlice = data.slice("fixed-binary", entry.dataOffset, entry.dataLength);
-
- if (entry.docsWithFieldOffset == -1) {
- // dense
- if (entry.minLength == entry.maxLength) {
- // fixed length
- final int length = entry.maxLength;
- return new DenseBinaryDocValues(maxDoc) {
- final BytesRef bytes = new BytesRef(new byte[length], 0, length);
-
- @Override
- public BytesRef binaryValue() throws IOException {
- bytesSlice.seek((long) doc * length);
- bytesSlice.readBytes(bytes.bytes, 0, length);
- return bytes;
- }
- };
- } else {
- // variable length
- final RandomAccessInput addressesData =
- this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength);
- final LongValues addresses =
- LegacyDirectMonotonicReader.getInstance(entry.addressesMeta, addressesData);
- return new DenseBinaryDocValues(maxDoc) {
- final BytesRef bytes = new BytesRef(new byte[entry.maxLength], 0, entry.maxLength);
-
- @Override
- public BytesRef binaryValue() throws IOException {
- long startOffset = addresses.get(doc);
- bytes.length = (int) (addresses.get(doc + 1L) - startOffset);
- bytesSlice.seek(startOffset);
- bytesSlice.readBytes(bytes.bytes, 0, bytes.length);
- return bytes;
- }
- };
- }
- } else {
- // sparse
- final IndexedDISI disi =
- new IndexedDISI(
- data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
- if (entry.minLength == entry.maxLength) {
- // fixed length
- final int length = entry.maxLength;
- return new SparseBinaryDocValues(disi) {
- final BytesRef bytes = new BytesRef(new byte[length], 0, length);
-
- @Override
- public BytesRef binaryValue() throws IOException {
- bytesSlice.seek((long) disi.index() * length);
- bytesSlice.readBytes(bytes.bytes, 0, length);
- return bytes;
- }
- };
- } else {
- // variable length
- final RandomAccessInput addressesData =
- this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength);
- final LongValues addresses =
- LegacyDirectMonotonicReader.getInstance(entry.addressesMeta, addressesData);
- return new SparseBinaryDocValues(disi) {
- final BytesRef bytes = new BytesRef(new byte[entry.maxLength], 0, entry.maxLength);
-
- @Override
- public BytesRef binaryValue() throws IOException {
- final int index = disi.index();
- long startOffset = addresses.get(index);
- bytes.length = (int) (addresses.get(index + 1L) - startOffset);
- bytesSlice.seek(startOffset);
- bytesSlice.readBytes(bytes.bytes, 0, bytes.length);
- return bytes;
- }
- };
- }
- }
- }
-
- @Override
- public SortedDocValues getSorted(FieldInfo field) throws IOException {
- SortedEntry entry = sorted.get(field.name);
- return getSorted(entry);
- }
-
- private SortedDocValues getSorted(SortedEntry entry) throws IOException {
- if (entry.docsWithFieldOffset == -2) {
- return DocValues.emptySorted();
- }
-
- final LongValues ords;
- if (entry.bitsPerValue == 0) {
- ords =
- new LongValues() {
- @Override
- public long get(long index) {
- return 0L;
- }
- };
- } else {
- final RandomAccessInput slice = data.randomAccessSlice(entry.ordsOffset, entry.ordsLength);
- ords = LegacyDirectReader.getInstance(slice, entry.bitsPerValue);
- }
-
- if (entry.docsWithFieldOffset == -1) {
- // dense
- return new BaseSortedDocValues(entry, data) {
-
- int doc = -1;
-
- @Override
- public int nextDoc() throws IOException {
- return advance(doc + 1);
- }
-
- @Override
- public int docID() {
- return doc;
- }
-
- @Override
- public long cost() {
- return maxDoc;
- }
-
- @Override
- public int advance(int target) throws IOException {
- if (target >= maxDoc) {
- return doc = NO_MORE_DOCS;
- }
- return doc = target;
- }
-
- @Override
- public boolean advanceExact(int target) {
- doc = target;
- return true;
- }
-
- @Override
- public int ordValue() {
- return (int) ords.get(doc);
- }
- };
- } else {
- // sparse
- final IndexedDISI disi =
- new IndexedDISI(
- data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
- return new BaseSortedDocValues(entry, data) {
-
- @Override
- public int nextDoc() throws IOException {
- return disi.nextDoc();
- }
-
- @Override
- public int docID() {
- return disi.docID();
- }
-
- @Override
- public long cost() {
- return disi.cost();
- }
-
- @Override
- public int advance(int target) throws IOException {
- return disi.advance(target);
- }
-
- @Override
- public boolean advanceExact(int target) throws IOException {
- return disi.advanceExact(target);
- }
-
- @Override
- public int ordValue() {
- return (int) ords.get(disi.index());
- }
- };
- }
- }
-
- private abstract static class BaseSortedDocValues extends SortedDocValues {
-
- final SortedEntry entry;
- final IndexInput data;
- final TermsEnum termsEnum;
-
- BaseSortedDocValues(SortedEntry entry, IndexInput data) throws IOException {
- this.entry = entry;
- this.data = data;
- this.termsEnum = termsEnum();
- }
-
- @Override
- public int getValueCount() {
- return Math.toIntExact(entry.termsDictSize);
- }
-
- @Override
- public BytesRef lookupOrd(int ord) throws IOException {
- termsEnum.seekExact(ord);
- return termsEnum.term();
- }
-
- @Override
- public int lookupTerm(BytesRef key) throws IOException {
- SeekStatus status = termsEnum.seekCeil(key);
- switch (status) {
- case FOUND:
- return Math.toIntExact(termsEnum.ord());
- case NOT_FOUND:
- case END:
- default:
- return Math.toIntExact(-1L - termsEnum.ord());
- }
- }
-
- @Override
- public TermsEnum termsEnum() throws IOException {
- return new TermsDict(entry, data);
- }
- }
-
- private abstract static class BaseSortedSetDocValues extends SortedSetDocValues {
-
- final SortedSetEntry entry;
- final IndexInput data;
- final TermsEnum termsEnum;
-
- BaseSortedSetDocValues(SortedSetEntry entry, IndexInput data) throws IOException {
- this.entry = entry;
- this.data = data;
- this.termsEnum = termsEnum();
- }
-
- @Override
- public long getValueCount() {
- return entry.termsDictSize;
- }
-
- @Override
- public BytesRef lookupOrd(long ord) throws IOException {
- termsEnum.seekExact(ord);
- return termsEnum.term();
- }
-
- @Override
- public long lookupTerm(BytesRef key) throws IOException {
- SeekStatus status = termsEnum.seekCeil(key);
- switch (status) {
- case FOUND:
- return termsEnum.ord();
- case NOT_FOUND:
- case END:
- default:
- return -1L - termsEnum.ord();
- }
- }
-
- @Override
- public TermsEnum termsEnum() throws IOException {
- return new TermsDict(entry, data);
- }
- }
-
- private static class TermsDict extends BaseTermsEnum {
-
- final TermsDictEntry entry;
- final LongValues blockAddresses;
- final IndexInput bytes;
- final long blockMask;
- final LongValues indexAddresses;
- final IndexInput indexBytes;
- final BytesRef term;
- long ord = -1;
-
- TermsDict(TermsDictEntry entry, IndexInput data) throws IOException {
- this.entry = entry;
- RandomAccessInput addressesSlice =
- data.randomAccessSlice(entry.termsAddressesOffset, entry.termsAddressesLength);
- blockAddresses =
- LegacyDirectMonotonicReader.getInstance(entry.termsAddressesMeta, addressesSlice);
- bytes = data.slice("terms", entry.termsDataOffset, entry.termsDataLength);
- blockMask = (1L << entry.termsDictBlockShift) - 1;
- RandomAccessInput indexAddressesSlice =
- data.randomAccessSlice(entry.termsIndexAddressesOffset, entry.termsIndexAddressesLength);
- indexAddresses =
- LegacyDirectMonotonicReader.getInstance(
- entry.termsIndexAddressesMeta, indexAddressesSlice);
- indexBytes = data.slice("terms-index", entry.termsIndexOffset, entry.termsIndexLength);
- term = new BytesRef(entry.maxTermLength);
- }
-
- @Override
- public BytesRef next() throws IOException {
- if (++ord >= entry.termsDictSize) {
- return null;
- }
- if ((ord & blockMask) == 0L) {
- term.length = bytes.readVInt();
- bytes.readBytes(term.bytes, 0, term.length);
- } else {
- final int token = Byte.toUnsignedInt(bytes.readByte());
- int prefixLength = token & 0x0F;
- int suffixLength = 1 + (token >>> 4);
- if (prefixLength == 15) {
- prefixLength += bytes.readVInt();
- }
- if (suffixLength == 16) {
- suffixLength += bytes.readVInt();
- }
- term.length = prefixLength + suffixLength;
- bytes.readBytes(term.bytes, prefixLength, suffixLength);
- }
- return term;
- }
-
- @Override
- public void seekExact(long ord) throws IOException {
- if (ord < 0 || ord >= entry.termsDictSize) {
- throw new IndexOutOfBoundsException();
- }
- final long blockIndex = ord >>> entry.termsDictBlockShift;
- final long blockAddress = blockAddresses.get(blockIndex);
- bytes.seek(blockAddress);
- this.ord = (blockIndex << entry.termsDictBlockShift) - 1;
- do {
- next();
- } while (this.ord < ord);
- }
-
- private BytesRef getTermFromIndex(long index) throws IOException {
- assert index >= 0 && index <= (entry.termsDictSize - 1) >>> entry.termsDictIndexShift;
- final long start = indexAddresses.get(index);
- term.length = (int) (indexAddresses.get(index + 1) - start);
- indexBytes.seek(start);
- indexBytes.readBytes(term.bytes, 0, term.length);
- return term;
- }
-
- private long seekTermsIndex(BytesRef text) throws IOException {
- long lo = 0L;
- long hi = (entry.termsDictSize - 1) >>> entry.termsDictIndexShift;
- while (lo <= hi) {
- final long mid = (lo + hi) >>> 1;
- getTermFromIndex(mid);
- final int cmp = term.compareTo(text);
- if (cmp <= 0) {
- lo = mid + 1;
- } else {
- hi = mid - 1;
- }
- }
-
- assert hi < 0 || getTermFromIndex(hi).compareTo(text) <= 0;
- assert hi == ((entry.termsDictSize - 1) >>> entry.termsDictIndexShift)
- || getTermFromIndex(hi + 1).compareTo(text) > 0;
-
- return hi;
- }
-
- private BytesRef getFirstTermFromBlock(long block) throws IOException {
- assert block >= 0 && block <= (entry.termsDictSize - 1) >>> entry.termsDictBlockShift;
- final long blockAddress = blockAddresses.get(block);
- bytes.seek(blockAddress);
- term.length = bytes.readVInt();
- bytes.readBytes(term.bytes, 0, term.length);
- return term;
- }
-
- private long seekBlock(BytesRef text) throws IOException {
- long index = seekTermsIndex(text);
- if (index == -1L) {
- return -1L;
- }
-
- long ordLo = index << entry.termsDictIndexShift;
- long ordHi = Math.min(entry.termsDictSize, ordLo + (1L << entry.termsDictIndexShift)) - 1L;
-
- long blockLo = ordLo >>> entry.termsDictBlockShift;
- long blockHi = ordHi >>> entry.termsDictBlockShift;
-
- while (blockLo <= blockHi) {
- final long blockMid = (blockLo + blockHi) >>> 1;
- getFirstTermFromBlock(blockMid);
- final int cmp = term.compareTo(text);
- if (cmp <= 0) {
- blockLo = blockMid + 1;
- } else {
- blockHi = blockMid - 1;
- }
- }
-
- assert blockHi < 0 || getFirstTermFromBlock(blockHi).compareTo(text) <= 0;
- assert blockHi == ((entry.termsDictSize - 1) >>> entry.termsDictBlockShift)
- || getFirstTermFromBlock(blockHi + 1).compareTo(text) > 0;
-
- return blockHi;
- }
-
- @Override
- public SeekStatus seekCeil(BytesRef text) throws IOException {
- final long block = seekBlock(text);
- if (block == -1) {
- // before the first term
- seekExact(0L);
- return SeekStatus.NOT_FOUND;
- }
- final long blockAddress = blockAddresses.get(block);
- this.ord = block << entry.termsDictBlockShift;
- bytes.seek(blockAddress);
- term.length = bytes.readVInt();
- bytes.readBytes(term.bytes, 0, term.length);
- while (true) {
- int cmp = term.compareTo(text);
- if (cmp == 0) {
- return SeekStatus.FOUND;
- } else if (cmp > 0) {
- return SeekStatus.NOT_FOUND;
- }
- if (next() == null) {
- return SeekStatus.END;
- }
- }
- }
-
- @Override
- public BytesRef term() throws IOException {
- return term;
- }
-
- @Override
- public long ord() throws IOException {
- return ord;
- }
-
- @Override
- public long totalTermFreq() throws IOException {
- return -1L;
- }
-
- @Override
- public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public ImpactsEnum impacts(int flags) throws IOException {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public int docFreq() throws IOException {
- throw new UnsupportedOperationException();
- }
- }
-
- @Override
- public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
- SortedNumericEntry entry = sortedNumerics.get(field.name);
- if (entry.numValues == entry.numDocsWithField) {
- return DocValues.singleton(getNumeric(entry));
- }
-
- final RandomAccessInput addressesInput =
- data.randomAccessSlice(entry.addressesOffset, entry.addressesLength);
- final LongValues addresses =
- LegacyDirectMonotonicReader.getInstance(entry.addressesMeta, addressesInput);
-
- final LongValues values = getNumericValues(entry);
-
- if (entry.docsWithFieldOffset == -1) {
- // dense
- return new SortedNumericDocValues() {
-
- int doc = -1;
- long start, end;
- int count;
-
- @Override
- public int nextDoc() throws IOException {
- return advance(doc + 1);
- }
-
- @Override
- public int docID() {
- return doc;
- }
-
- @Override
- public long cost() {
- return maxDoc;
- }
-
- @Override
- public int advance(int target) throws IOException {
- if (target >= maxDoc) {
- return doc = NO_MORE_DOCS;
- }
- start = addresses.get(target);
- end = addresses.get(target + 1L);
- count = (int) (end - start);
- return doc = target;
- }
-
- @Override
- public boolean advanceExact(int target) throws IOException {
- start = addresses.get(target);
- end = addresses.get(target + 1L);
- count = (int) (end - start);
- doc = target;
- return true;
- }
-
- @Override
- public long nextValue() throws IOException {
- return values.get(start++);
- }
-
- @Override
- public int docValueCount() {
- return count;
- }
- };
- } else {
- // sparse
- final IndexedDISI disi =
- new IndexedDISI(
- data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
- return new SortedNumericDocValues() {
-
- boolean set;
- long start, end;
- int count;
-
- @Override
- public int nextDoc() throws IOException {
- set = false;
- return disi.nextDoc();
- }
-
- @Override
- public int docID() {
- return disi.docID();
- }
-
- @Override
- public long cost() {
- return disi.cost();
- }
-
- @Override
- public int advance(int target) throws IOException {
- set = false;
- return disi.advance(target);
- }
-
- @Override
- public boolean advanceExact(int target) throws IOException {
- set = false;
- return disi.advanceExact(target);
- }
-
- @Override
- public long nextValue() throws IOException {
- set();
- return values.get(start++);
- }
-
- @Override
- public int docValueCount() {
- set();
- return count;
- }
-
- private void set() {
- if (set == false) {
- final int index = disi.index();
- start = addresses.get(index);
- end = addresses.get(index + 1L);
- count = (int) (end - start);
- set = true;
- }
- }
- };
- }
- }
-
- @Override
- public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
- SortedSetEntry entry = sortedSets.get(field.name);
- if (entry.singleValueEntry != null) {
- return DocValues.singleton(getSorted(entry.singleValueEntry));
- }
-
- final RandomAccessInput slice = data.randomAccessSlice(entry.ordsOffset, entry.ordsLength);
- final LongValues ords = LegacyDirectReader.getInstance(slice, entry.bitsPerValue);
-
- final RandomAccessInput addressesInput =
- data.randomAccessSlice(entry.addressesOffset, entry.addressesLength);
- final LongValues addresses =
- LegacyDirectMonotonicReader.getInstance(entry.addressesMeta, addressesInput);
-
- if (entry.docsWithFieldOffset == -1) {
- // dense
- return new BaseSortedSetDocValues(entry, data) {
-
- int doc = -1;
- long start;
- long end;
-
- @Override
- public int nextDoc() throws IOException {
- return advance(doc + 1);
- }
-
- @Override
- public int docID() {
- return doc;
- }
-
- @Override
- public long cost() {
- return maxDoc;
- }
-
- @Override
- public int advance(int target) throws IOException {
- if (target >= maxDoc) {
- return doc = NO_MORE_DOCS;
- }
- start = addresses.get(target);
- end = addresses.get(target + 1L);
- return doc = target;
- }
-
- @Override
- public boolean advanceExact(int target) throws IOException {
- start = addresses.get(target);
- end = addresses.get(target + 1L);
- doc = target;
- return true;
- }
-
- @Override
- public long nextOrd() throws IOException {
- if (start == end) {
- return NO_MORE_ORDS;
- }
- return ords.get(start++);
- }
- };
- } else {
- // sparse
- final IndexedDISI disi =
- new IndexedDISI(
- data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
- return new BaseSortedSetDocValues(entry, data) {
-
- boolean set;
- long start;
- long end = 0;
-
- @Override
- public int nextDoc() throws IOException {
- set = false;
- return disi.nextDoc();
- }
-
- @Override
- public int docID() {
- return disi.docID();
- }
-
- @Override
- public long cost() {
- return disi.cost();
- }
-
- @Override
- public int advance(int target) throws IOException {
- set = false;
- return disi.advance(target);
- }
-
- @Override
- public boolean advanceExact(int target) throws IOException {
- set = false;
- return disi.advanceExact(target);
- }
-
- @Override
- public long nextOrd() throws IOException {
- if (set == false) {
- final int index = disi.index();
- final long start = addresses.get(index);
- this.start = start + 1;
- end = addresses.get(index + 1L);
- set = true;
- return ords.get(start);
- } else if (start == end) {
- return NO_MORE_ORDS;
- } else {
- return ords.get(start++);
- }
- }
- };
- }
- }
-
- @Override
- public void checkIntegrity() throws IOException {
- CodecUtil.checksumEntireFile(data);
- }
-}
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsFormat.java
deleted file mode 100644
index a5fdf906ab7..00000000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsFormat.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.backward_codecs.lucene70;
-
-import java.io.IOException;
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.NormsConsumer;
-import org.apache.lucene.codecs.NormsFormat;
-import org.apache.lucene.codecs.NormsProducer;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.store.DataOutput;
-
-/**
- * Lucene 7.0 Score normalization format.
- *
- * Encodes normalization values by encoding each value with the minimum number of bytes needed to
- * represent the range (which can be zero).
- *
- *
Files:
- *
- *
- * .nvd
: Norms data
- * .nvm
: Norms metadata
- *
- *
- *
- * -
- *
The Norms metadata or .nvm file.
- *
For each norms field, this stores metadata, such as the offset into the Norms data
- * (.nvd)
- *
Norms metadata (.dvm) --> Header,<Entry>NumFields,Footer
- *
- * - Header --> {@link CodecUtil#writeIndexHeader IndexHeader}
- *
- Entry --> FieldNumber, DocsWithFieldAddress, DocsWithFieldLength,
- * NumDocsWithField, BytesPerNorm, NormsAddress
- *
- FieldNumber --> {@link DataOutput#writeInt Int32}
- *
- DocsWithFieldAddress --> {@link DataOutput#writeLong Int64}
- *
- DocsWithFieldLength --> {@link DataOutput#writeLong Int64}
- *
- NumDocsWithField --> {@link DataOutput#writeInt Int32}
- *
- BytesPerNorm --> {@link DataOutput#writeByte byte}
- *
- NormsAddress --> {@link DataOutput#writeLong Int64}
- *
- Footer --> {@link CodecUtil#writeFooter CodecFooter}
- *
- * FieldNumber of -1 indicates the end of metadata.
- *
NormsAddress is the pointer to the start of the data in the norms data (.nvd), or the
- * singleton value when BytesPerValue = 0. If BytesPerValue is different from 0 then there are
- * NumDocsWithField values to read at that offset.
- *
DocsWithFieldAddress is the pointer to the start of the bit set containing documents
- * that have a norm in the norms data (.nvd), or -2 if no documents have a norm value, or -1
- * if all documents have a norm value.
- *
DocsWithFieldLength is the number of bytes used to encode the set of documents that have
- * a norm.
- *
-
- *
The Norms data or .nvd file.
- *
For each Norms field, this stores the actual per-document data (the heavy-lifting)
- *
Norms data (.nvd) --> Header,< Data >NumFields,Footer
- *
- * - Header --> {@link CodecUtil#writeIndexHeader IndexHeader}
- *
- DocsWithFieldData --> Bit set of MaxDoc bits
- *
- NormsData --> {@link DataOutput#writeByte(byte) byte}NumDocsWithField *
- * BytesPerValue
- *
- Footer --> {@link CodecUtil#writeFooter CodecFooter}
- *
- *
- *
- * @lucene.experimental
- */
-public class Lucene70NormsFormat extends NormsFormat {
-
- /** Sole Constructor */
- public Lucene70NormsFormat() {}
-
- @Override
- public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
- throw new UnsupportedOperationException("Old codecs may only be used for reading");
- }
-
- @Override
- public NormsProducer normsProducer(SegmentReadState state) throws IOException {
- return new Lucene70NormsProducer(
- state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
- }
-
- static final String DATA_CODEC = "Lucene70NormsData";
- static final String DATA_EXTENSION = "nvd";
- static final String METADATA_CODEC = "Lucene70NormsMetadata";
- static final String METADATA_EXTENSION = "nvm";
- static final int VERSION_START = 0;
- static final int VERSION_CURRENT = VERSION_START;
-}
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsProducer.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsProducer.java
deleted file mode 100644
index 94dc70b0b14..00000000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsProducer.java
+++ /dev/null
@@ -1,447 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.backward_codecs.lucene70;
-
-import static org.apache.lucene.backward_codecs.lucene70.Lucene70NormsFormat.VERSION_CURRENT;
-import static org.apache.lucene.backward_codecs.lucene70.Lucene70NormsFormat.VERSION_START;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.lucene.backward_codecs.store.EndiannessReverserUtil;
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.NormsProducer;
-import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.store.ChecksumIndexInput;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.RandomAccessInput;
-import org.apache.lucene.util.IOUtils;
-
-/** Reader for {@link Lucene70NormsFormat} */
-final class Lucene70NormsProducer extends NormsProducer implements Cloneable {
- // metadata maps (just file pointers and minimal stuff)
- private final Map norms = new HashMap<>();
- private final int maxDoc;
- private IndexInput data;
- private boolean merging;
- private Map disiInputs;
- private Map dataInputs;
-
- Lucene70NormsProducer(
- SegmentReadState state,
- String dataCodec,
- String dataExtension,
- String metaCodec,
- String metaExtension)
- throws IOException {
- maxDoc = state.segmentInfo.maxDoc();
- String metaName =
- IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
- int version = -1;
-
- // read in the entries from the metadata file.
- try (ChecksumIndexInput in =
- EndiannessReverserUtil.openChecksumInput(state.directory, metaName, state.context)) {
- Throwable priorE = null;
- try {
- version =
- CodecUtil.checkIndexHeader(
- in,
- metaCodec,
- VERSION_START,
- VERSION_CURRENT,
- state.segmentInfo.getId(),
- state.segmentSuffix);
- readFields(in, state.fieldInfos);
- } catch (Throwable exception) {
- priorE = exception;
- } finally {
- CodecUtil.checkFooter(in, priorE);
- }
- }
-
- String dataName =
- IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
- data = EndiannessReverserUtil.openInput(state.directory, dataName, state.context);
- boolean success = false;
- try {
- final int version2 =
- CodecUtil.checkIndexHeader(
- data,
- dataCodec,
- VERSION_START,
- VERSION_CURRENT,
- state.segmentInfo.getId(),
- state.segmentSuffix);
- if (version != version2) {
- throw new CorruptIndexException(
- "Format versions mismatch: meta=" + version + ",data=" + version2, data);
- }
-
- // 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(data);
-
- success = true;
- } finally {
- if (!success) {
- IOUtils.closeWhileHandlingException(this.data);
- }
- }
- }
-
- @Override
- public NormsProducer getMergeInstance() {
- Lucene70NormsProducer clone;
- try {
- clone = (Lucene70NormsProducer) super.clone();
- } catch (CloneNotSupportedException e) {
- // cannot happen
- throw new RuntimeException(e);
- }
- clone.data = data.clone();
- clone.dataInputs = new HashMap<>();
- clone.disiInputs = new HashMap<>();
- clone.merging = true;
- return clone;
- }
-
- static class NormsEntry {
- byte bytesPerNorm;
- long docsWithFieldOffset;
- long docsWithFieldLength;
- int numDocsWithField;
- long normsOffset;
- }
-
- abstract static class DenseNormsIterator extends NumericDocValues {
-
- final int maxDoc;
- int doc = -1;
-
- DenseNormsIterator(int maxDoc) {
- this.maxDoc = maxDoc;
- }
-
- @Override
- public int docID() {
- return doc;
- }
-
- @Override
- public int nextDoc() throws IOException {
- return advance(doc + 1);
- }
-
- @Override
- public int advance(int target) throws IOException {
- if (target >= maxDoc) {
- return doc = NO_MORE_DOCS;
- }
- return doc = target;
- }
-
- @Override
- public boolean advanceExact(int target) throws IOException {
- this.doc = target;
- return true;
- }
-
- @Override
- public long cost() {
- return maxDoc;
- }
- }
-
- abstract static class SparseNormsIterator extends NumericDocValues {
-
- final IndexedDISI disi;
-
- SparseNormsIterator(IndexedDISI disi) {
- this.disi = disi;
- }
-
- @Override
- public int docID() {
- return disi.docID();
- }
-
- @Override
- public int nextDoc() throws IOException {
- return disi.nextDoc();
- }
-
- @Override
- public int advance(int target) throws IOException {
- return disi.advance(target);
- }
-
- @Override
- public boolean advanceExact(int target) throws IOException {
- return disi.advanceExact(target);
- }
-
- @Override
- public long cost() {
- return disi.cost();
- }
- }
-
- private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
- for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
- FieldInfo info = infos.fieldInfo(fieldNumber);
- if (info == null) {
- throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
- } else if (!info.hasNorms()) {
- throw new CorruptIndexException("Invalid field: " + info.name, meta);
- }
- NormsEntry entry = new NormsEntry();
- entry.docsWithFieldOffset = meta.readLong();
- entry.docsWithFieldLength = meta.readLong();
- entry.numDocsWithField = meta.readInt();
- entry.bytesPerNorm = meta.readByte();
- switch (entry.bytesPerNorm) {
- case 0:
- case 1:
- case 2:
- case 4:
- case 8:
- break;
- default:
- throw new CorruptIndexException(
- "Invalid bytesPerValue: " + entry.bytesPerNorm + ", field: " + info.name, meta);
- }
- entry.normsOffset = meta.readLong();
- norms.put(info.number, entry);
- }
- }
-
- private RandomAccessInput getDataInput(FieldInfo field, NormsEntry entry) throws IOException {
- RandomAccessInput slice = null;
- if (merging) {
- slice = dataInputs.get(field.number);
- }
- if (slice == null) {
- slice =
- data.randomAccessSlice(
- entry.normsOffset, entry.numDocsWithField * (long) entry.bytesPerNorm);
- if (merging) {
- dataInputs.put(field.number, slice);
- }
- }
- return slice;
- }
-
- private IndexInput getDisiInput(FieldInfo field, NormsEntry entry) throws IOException {
- if (merging == false) {
- return data.slice("docs", entry.docsWithFieldOffset, entry.docsWithFieldLength);
- }
-
- IndexInput in = disiInputs.get(field.number);
- if (in == null) {
- in = data.slice("docs", entry.docsWithFieldOffset, entry.docsWithFieldLength);
- disiInputs.put(field.number, in);
- }
-
- final IndexInput inF = in; // same as in but final
-
- // Wrap so that reads can be interleaved from the same thread if two
- // norms instances are pulled and consumed in parallel. Merging usually
- // doesn't need this feature but CheckIndex might, plus we need merge
- // instances to behave well and not be trappy.
- return new IndexInput("docs") {
-
- long offset = 0;
-
- @Override
- public void readBytes(byte[] b, int off, int len) throws IOException {
- inF.seek(offset);
- offset += len;
- inF.readBytes(b, off, len);
- }
-
- @Override
- public byte readByte() throws IOException {
- throw new UnsupportedOperationException("Unused by IndexedDISI");
- }
-
- @Override
- public IndexInput slice(String sliceDescription, long offset, long length)
- throws IOException {
- throw new UnsupportedOperationException("Unused by IndexedDISI");
- }
-
- @Override
- public short readShort() throws IOException {
- inF.seek(offset);
- offset += Short.BYTES;
- return inF.readShort();
- }
-
- @Override
- public long readLong() throws IOException {
- inF.seek(offset);
- offset += Long.BYTES;
- return inF.readLong();
- }
-
- @Override
- public void seek(long pos) throws IOException {
- offset = pos;
- }
-
- @Override
- public long length() {
- throw new UnsupportedOperationException("Unused by IndexedDISI");
- }
-
- @Override
- public long getFilePointer() {
- return offset;
- }
-
- @Override
- public void close() throws IOException {
- throw new UnsupportedOperationException("Unused by IndexedDISI");
- }
- };
- }
-
- @Override
- public NumericDocValues getNorms(FieldInfo field) throws IOException {
- final NormsEntry entry = norms.get(field.number);
- if (entry.docsWithFieldOffset == -2) {
- // empty
- return DocValues.emptyNumeric();
- } else if (entry.docsWithFieldOffset == -1) {
- // dense
- if (entry.bytesPerNorm == 0) {
- return new DenseNormsIterator(maxDoc) {
- @Override
- public long longValue() throws IOException {
- return entry.normsOffset;
- }
- };
- }
- final RandomAccessInput slice = getDataInput(field, entry);
- switch (entry.bytesPerNorm) {
- case 1:
- return new DenseNormsIterator(maxDoc) {
- @Override
- public long longValue() throws IOException {
- return slice.readByte(doc);
- }
- };
- case 2:
- return new DenseNormsIterator(maxDoc) {
- @Override
- public long longValue() throws IOException {
- return slice.readShort(((long) doc) << 1);
- }
- };
- case 4:
- return new DenseNormsIterator(maxDoc) {
- @Override
- public long longValue() throws IOException {
- return slice.readInt(((long) doc) << 2);
- }
- };
- case 8:
- return new DenseNormsIterator(maxDoc) {
- @Override
- public long longValue() throws IOException {
- return slice.readLong(((long) doc) << 3);
- }
- };
- default:
- // should not happen, we already validate bytesPerNorm in readFields
- throw new AssertionError();
- }
- } else {
- // sparse
- final IndexInput disiInput = getDisiInput(field, entry);
- final IndexedDISI disi = new IndexedDISI(disiInput, entry.numDocsWithField);
- if (entry.bytesPerNorm == 0) {
- return new SparseNormsIterator(disi) {
- @Override
- public long longValue() throws IOException {
- return entry.normsOffset;
- }
- };
- }
- final RandomAccessInput slice =
- data.randomAccessSlice(
- entry.normsOffset, entry.numDocsWithField * (long) entry.bytesPerNorm);
- switch (entry.bytesPerNorm) {
- case 1:
- return new SparseNormsIterator(disi) {
- @Override
- public long longValue() throws IOException {
- return slice.readByte(disi.index());
- }
- };
- case 2:
- return new SparseNormsIterator(disi) {
- @Override
- public long longValue() throws IOException {
- return slice.readShort(((long) disi.index()) << 1);
- }
- };
- case 4:
- return new SparseNormsIterator(disi) {
- @Override
- public long longValue() throws IOException {
- return slice.readInt(((long) disi.index()) << 2);
- }
- };
- case 8:
- return new SparseNormsIterator(disi) {
- @Override
- public long longValue() throws IOException {
- return slice.readLong(((long) disi.index()) << 3);
- }
- };
- default:
- // should not happen, we already validate bytesPerNorm in readFields
- throw new AssertionError();
- }
- }
- }
-
- @Override
- public void close() throws IOException {
- data.close();
- }
-
- @Override
- public void checkIntegrity() throws IOException {
- CodecUtil.checksumEntireFile(data);
- }
-
- @Override
- public String toString() {
- return getClass().getSimpleName() + "(fields=" + norms.size() + ")";
- }
-}
diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
index 5ff3ebfdeee..c9a6322f0b1 100644
--- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
+++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
@@ -17,4 +17,3 @@ org.apache.lucene.backward_codecs.lucene80.Lucene80Codec
org.apache.lucene.backward_codecs.lucene84.Lucene84Codec
org.apache.lucene.backward_codecs.lucene86.Lucene86Codec
org.apache.lucene.backward_codecs.lucene87.Lucene87Codec
-org.apache.lucene.backward_codecs.lucene70.Lucene70Codec
diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
index f058ccd9ffb..0d59eca00b9 100644
--- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
+++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
@@ -13,5 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-org.apache.lucene.backward_codecs.lucene70.Lucene70DocValuesFormat
org.apache.lucene.backward_codecs.lucene80.Lucene80DocValuesFormat
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsConsumer.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsConsumer.java
deleted file mode 100644
index 05dc6618480..00000000000
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsConsumer.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.backward_codecs.lucene70;
-
-import static org.apache.lucene.backward_codecs.lucene70.Lucene70NormsFormat.VERSION_CURRENT;
-
-import java.io.IOException;
-import org.apache.lucene.backward_codecs.store.EndiannessReverserUtil;
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.NormsConsumer;
-import org.apache.lucene.codecs.NormsProducer;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.IOUtils;
-
-/** Writer for {@link Lucene70NormsFormat} */
-final class Lucene70NormsConsumer extends NormsConsumer {
- IndexOutput data, meta;
- final int maxDoc;
-
- Lucene70NormsConsumer(
- SegmentWriteState state,
- String dataCodec,
- String dataExtension,
- String metaCodec,
- String metaExtension)
- throws IOException {
- boolean success = false;
- try {
- String dataName =
- IndexFileNames.segmentFileName(
- state.segmentInfo.name, state.segmentSuffix, dataExtension);
- data = EndiannessReverserUtil.createOutput(state.directory, dataName, state.context);
- CodecUtil.writeIndexHeader(
- data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
- String metaName =
- IndexFileNames.segmentFileName(
- state.segmentInfo.name, state.segmentSuffix, metaExtension);
- meta = EndiannessReverserUtil.createOutput(state.directory, metaName, state.context);
- CodecUtil.writeIndexHeader(
- meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
- maxDoc = state.segmentInfo.maxDoc();
- success = true;
- } finally {
- if (!success) {
- IOUtils.closeWhileHandlingException(this);
- }
- }
- }
-
- @Override
- public void close() throws IOException {
- boolean success = false;
- try {
- if (meta != null) {
- meta.writeInt(-1); // write EOF marker
- CodecUtil.writeFooter(meta); // write checksum
- }
- if (data != null) {
- CodecUtil.writeFooter(data); // write checksum
- }
- success = true;
- } finally {
- if (success) {
- IOUtils.close(data, meta);
- } else {
- IOUtils.closeWhileHandlingException(data, meta);
- }
- meta = data = null;
- }
- }
-
- @Override
- public void addNormsField(FieldInfo field, NormsProducer normsProducer) throws IOException {
- NumericDocValues values = normsProducer.getNorms(field);
- int numDocsWithValue = 0;
- long min = Long.MAX_VALUE;
- long max = Long.MIN_VALUE;
- for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
- numDocsWithValue++;
- long v = values.longValue();
- min = Math.min(min, v);
- max = Math.max(max, v);
- }
- assert numDocsWithValue <= maxDoc;
-
- meta.writeInt(field.number);
-
- if (numDocsWithValue == 0) {
- meta.writeLong(-2);
- meta.writeLong(0L);
- } else if (numDocsWithValue == maxDoc) {
- meta.writeLong(-1);
- meta.writeLong(0L);
- } else {
- long offset = data.getFilePointer();
- meta.writeLong(offset);
- values = normsProducer.getNorms(field);
- IndexedDISI.writeBitSet(values, data);
- meta.writeLong(data.getFilePointer() - offset);
- }
-
- meta.writeInt(numDocsWithValue);
- int numBytesPerValue = numBytesPerValue(min, max);
-
- meta.writeByte((byte) numBytesPerValue);
- if (numBytesPerValue == 0) {
- meta.writeLong(min);
- } else {
- meta.writeLong(data.getFilePointer());
- values = normsProducer.getNorms(field);
- writeValues(values, numBytesPerValue, data);
- }
- }
-
- private int numBytesPerValue(long min, long max) {
- if (min >= max) {
- return 0;
- } else if (min >= Byte.MIN_VALUE && max <= Byte.MAX_VALUE) {
- return 1;
- } else if (min >= Short.MIN_VALUE && max <= Short.MAX_VALUE) {
- return 2;
- } else if (min >= Integer.MIN_VALUE && max <= Integer.MAX_VALUE) {
- return 4;
- } else {
- return 8;
- }
- }
-
- private void writeValues(NumericDocValues values, int numBytesPerValue, IndexOutput out)
- throws IOException, AssertionError {
- for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
- long value = values.longValue();
- switch (numBytesPerValue) {
- case 1:
- out.writeByte((byte) value);
- break;
- case 2:
- out.writeShort((short) value);
- break;
- case 4:
- out.writeInt((int) value);
- break;
- case 8:
- out.writeLong(value);
- break;
- default:
- throw new AssertionError();
- }
- }
- }
-}
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/Lucene70RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/Lucene70RWCodec.java
deleted file mode 100644
index 5a38dd4f652..00000000000
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/Lucene70RWCodec.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.backward_codecs.lucene70;
-
-import org.apache.lucene.backward_codecs.lucene50.Lucene50RWCompoundFormat;
-import org.apache.lucene.backward_codecs.lucene50.Lucene50RWPostingsFormat;
-import org.apache.lucene.backward_codecs.lucene50.Lucene50RWStoredFieldsFormat;
-import org.apache.lucene.backward_codecs.lucene50.Lucene50RWTermVectorsFormat;
-import org.apache.lucene.codecs.CompoundFormat;
-import org.apache.lucene.codecs.NormsFormat;
-import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.SegmentInfoFormat;
-import org.apache.lucene.codecs.StoredFieldsFormat;
-import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
-
-/** RW impersonation of {@link Lucene70Codec}. */
-public final class Lucene70RWCodec extends Lucene70Codec {
-
- private final PostingsFormat defaultPF = new Lucene50RWPostingsFormat();
- private final PostingsFormat postingsFormat =
- new PerFieldPostingsFormat() {
- @Override
- public PostingsFormat getPostingsFormatForField(String field) {
- return defaultPF;
- }
- };
-
- /** Sole constructor. */
- public Lucene70RWCodec() {}
-
- @Override
- public SegmentInfoFormat segmentInfoFormat() {
- return new Lucene70RWSegmentInfoFormat();
- }
-
- @Override
- public NormsFormat normsFormat() {
- return new Lucene70RWNormsFormat();
- }
-
- @Override
- public StoredFieldsFormat storedFieldsFormat() {
- return new Lucene50RWStoredFieldsFormat();
- }
-
- @Override
- public PostingsFormat postingsFormat() {
- return postingsFormat;
- }
-
- @Override
- public CompoundFormat compoundFormat() {
- return new Lucene50RWCompoundFormat();
- }
-
- @Override
- public TermVectorsFormat termVectorsFormat() {
- return new Lucene50RWTermVectorsFormat();
- }
-}
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/Lucene70RWNormsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/Lucene70RWNormsFormat.java
deleted file mode 100644
index 9cdde9e97fd..00000000000
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/Lucene70RWNormsFormat.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.backward_codecs.lucene70;
-
-import java.io.IOException;
-import org.apache.lucene.codecs.NormsConsumer;
-import org.apache.lucene.index.SegmentWriteState;
-
-public class Lucene70RWNormsFormat extends Lucene70NormsFormat {
-
- @Override
- public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
- return new Lucene70NormsConsumer(
- state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
- }
-}
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestLucene70DocValuesFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestLucene70DocValuesFormat.java
deleted file mode 100644
index 894b2c151f4..00000000000
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestLucene70DocValuesFormat.java
+++ /dev/null
@@ -1,696 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.backward_codecs.lucene70;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.function.LongSupplier;
-import java.util.function.Supplier;
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.asserting.AssertingCodec;
-import org.apache.lucene.document.BinaryDocValuesField;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.NumericDocValuesField;
-import org.apache.lucene.document.SortedDocValuesField;
-import org.apache.lucene.document.SortedNumericDocValuesField;
-import org.apache.lucene.document.SortedSetDocValuesField;
-import org.apache.lucene.document.StoredField;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.SerialMergeScheduler;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedNumericDocValues;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.TermsEnum.SeekStatus;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.store.ByteBuffersDataInput;
-import org.apache.lucene.store.ByteBuffersDataOutput;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.TestUtil;
-
-/** Tests Lucene70DocValuesFormat */
-public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
- private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene70DocValuesFormat());
-
- @Override
- protected Codec getCodec() {
- return codec;
- }
-
- // TODO: these big methods can easily blow up some of the other ram-hungry codecs...
- // for now just keep them here, as we want to test this for this format.
-
- @Slow
- public void testSortedSetVariableLengthBigVsStoredFields() throws Exception {
- int numIterations = atLeast(1);
- for (int i = 0; i < numIterations; i++) {
- int numDocs = TEST_NIGHTLY ? atLeast(100) : atLeast(10);
- doTestSortedSetVsStoredFields(numDocs, 1, 32766, 16, 100);
- }
- }
-
- @Nightly
- public void testSortedSetVariableLengthManyVsStoredFields() throws Exception {
- int numIterations = atLeast(1);
- for (int i = 0; i < numIterations; i++) {
- doTestSortedSetVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500, 16, 100);
- }
- }
-
- @Slow
- public void testSortedVariableLengthBigVsStoredFields() throws Exception {
- int numIterations = atLeast(1);
- for (int i = 0; i < numIterations; i++) {
- doTestSortedVsStoredFields(atLeast(100), 1d, 1, 32766);
- }
- }
-
- @Nightly
- public void testSortedVariableLengthManyVsStoredFields() throws Exception {
- int numIterations = atLeast(1);
- for (int i = 0; i < numIterations; i++) {
- doTestSortedVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1d, 1, 500);
- }
- }
-
- @Nightly
- public void testTermsEnumFixedWidth() throws Exception {
- int numIterations = atLeast(1);
- for (int i = 0; i < numIterations; i++) {
- doTestTermsEnumRandom(
- TestUtil.nextInt(random(), 1025, 5121),
- () -> TestUtil.randomSimpleString(random(), 10, 10));
- }
- }
-
- @Nightly
- public void testTermsEnumVariableWidth() throws Exception {
- int numIterations = atLeast(1);
- for (int i = 0; i < numIterations; i++) {
- doTestTermsEnumRandom(
- TestUtil.nextInt(random(), 1025, 5121),
- () -> TestUtil.randomSimpleString(random(), 1, 500));
- }
- }
-
- @Nightly
- public void testTermsEnumRandomMany() throws Exception {
- int numIterations = atLeast(1);
- for (int i = 0; i < numIterations; i++) {
- doTestTermsEnumRandom(
- TestUtil.nextInt(random(), 1025, 8121),
- () -> TestUtil.randomSimpleString(random(), 1, 500));
- }
- }
-
- @Nightly
- public void testTermsEnumLongSharedPrefixes() throws Exception {
- int numIterations = atLeast(1);
- for (int i = 0; i < numIterations; i++) {
- doTestTermsEnumRandom(
- TestUtil.nextInt(random(), 1025, 5121),
- () -> {
- char[] chars = new char[random().nextInt(500)];
- Arrays.fill(chars, 'a');
- if (chars.length > 0) {
- chars[random().nextInt(chars.length)] = 'b';
- }
- return new String(chars);
- });
- }
- }
-
- @Slow
- public void testSparseDocValuesVsStoredFields() throws Exception {
- int numIterations = atLeast(1);
- for (int i = 0; i < numIterations; i++) {
- doTestSparseDocValuesVsStoredFields();
- }
- }
-
- private void doTestSparseDocValuesVsStoredFields() throws Exception {
- final long[] values = new long[TestUtil.nextInt(random(), 1, 500)];
- for (int i = 0; i < values.length; ++i) {
- values[i] = random().nextLong();
- }
-
- Directory dir = newFSDirectory(createTempDir());
- IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
- conf.setMergeScheduler(new SerialMergeScheduler());
- RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
-
- // sparse compression is only enabled if less than 1% of docs have a value
- final int avgGap = 100;
-
- final int numDocs = atLeast(200);
- for (int i = random().nextInt(avgGap * 2); i >= 0; --i) {
- writer.addDocument(new Document());
- }
- final int maxNumValuesPerDoc = random().nextBoolean() ? 1 : TestUtil.nextInt(random(), 2, 5);
- for (int i = 0; i < numDocs; ++i) {
- Document doc = new Document();
-
- // single-valued
- long docValue = values[random().nextInt(values.length)];
- doc.add(new NumericDocValuesField("numeric", docValue));
- doc.add(new SortedDocValuesField("sorted", new BytesRef(Long.toString(docValue))));
- doc.add(new BinaryDocValuesField("binary", new BytesRef(Long.toString(docValue))));
- doc.add(new StoredField("value", docValue));
-
- // multi-valued
- final int numValues = TestUtil.nextInt(random(), 1, maxNumValuesPerDoc);
- for (int j = 0; j < numValues; ++j) {
- docValue = values[random().nextInt(values.length)];
- doc.add(new SortedNumericDocValuesField("sorted_numeric", docValue));
- doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef(Long.toString(docValue))));
- doc.add(new StoredField("values", docValue));
- }
-
- writer.addDocument(doc);
-
- // add a gap
- for (int j = TestUtil.nextInt(random(), 0, avgGap * 2); j >= 0; --j) {
- writer.addDocument(new Document());
- }
- }
-
- if (random().nextBoolean()) {
- writer.forceMerge(1);
- }
-
- final IndexReader indexReader = writer.getReader();
- writer.close();
-
- for (LeafReaderContext context : indexReader.leaves()) {
- final LeafReader reader = context.reader();
- final NumericDocValues numeric = DocValues.getNumeric(reader, "numeric");
-
- final SortedDocValues sorted = DocValues.getSorted(reader, "sorted");
-
- final BinaryDocValues binary = DocValues.getBinary(reader, "binary");
-
- final SortedNumericDocValues sortedNumeric =
- DocValues.getSortedNumeric(reader, "sorted_numeric");
-
- final SortedSetDocValues sortedSet = DocValues.getSortedSet(reader, "sorted_set");
-
- for (int i = 0; i < reader.maxDoc(); ++i) {
- final Document doc = reader.document(i);
- final IndexableField valueField = doc.getField("value");
- final Long value = valueField == null ? null : valueField.numericValue().longValue();
-
- if (value == null) {
- assertTrue(numeric.docID() + " vs " + i, numeric.docID() < i);
- } else {
- assertEquals(i, numeric.nextDoc());
- assertEquals(i, binary.nextDoc());
- assertEquals(i, sorted.nextDoc());
- assertEquals(value.longValue(), numeric.longValue());
- assertTrue(sorted.ordValue() >= 0);
- assertEquals(new BytesRef(Long.toString(value)), sorted.lookupOrd(sorted.ordValue()));
- assertEquals(new BytesRef(Long.toString(value)), binary.binaryValue());
- }
-
- final IndexableField[] valuesFields = doc.getFields("values");
- if (valuesFields.length == 0) {
- assertTrue(sortedNumeric.docID() + " vs " + i, sortedNumeric.docID() < i);
- } else {
- final Set valueSet = new HashSet<>();
- for (IndexableField sf : valuesFields) {
- valueSet.add(sf.numericValue().longValue());
- }
-
- assertEquals(i, sortedNumeric.nextDoc());
- assertEquals(valuesFields.length, sortedNumeric.docValueCount());
- for (int j = 0; j < sortedNumeric.docValueCount(); ++j) {
- assertTrue(valueSet.contains(sortedNumeric.nextValue()));
- }
- assertEquals(i, sortedSet.nextDoc());
- int sortedSetCount = 0;
- while (true) {
- long ord = sortedSet.nextOrd();
- if (ord == SortedSetDocValues.NO_MORE_ORDS) {
- break;
- }
- assertTrue(valueSet.contains(Long.parseLong(sortedSet.lookupOrd(ord).utf8ToString())));
- sortedSetCount++;
- }
- assertEquals(valueSet.size(), sortedSetCount);
- }
- }
- }
-
- indexReader.close();
- dir.close();
- }
-
- // TODO: try to refactor this and some termsenum tests into the base class.
- // to do this we need to fix the test class to get a DVF not a Codec so we can setup
- // the postings format correctly.
- private void doTestTermsEnumRandom(int numDocs, Supplier valuesProducer)
- throws Exception {
- Directory dir = newFSDirectory(createTempDir());
- IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
- conf.setMergeScheduler(new SerialMergeScheduler());
- // set to duel against a codec which has ordinals:
- final PostingsFormat pf = TestUtil.getPostingsFormatWithOrds(random());
- final DocValuesFormat dv = new Lucene70DocValuesFormat();
- conf.setCodec(
- new AssertingCodec() {
- @Override
- public PostingsFormat getPostingsFormatForField(String field) {
- return pf;
- }
-
- @Override
- public DocValuesFormat getDocValuesFormatForField(String field) {
- return dv;
- }
- });
- RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
-
- // index some docs
- for (int i = 0; i < numDocs; i++) {
- Document doc = new Document();
- Field idField = new StringField("id", Integer.toString(i), Field.Store.NO);
- doc.add(idField);
- int numValues = random().nextInt(17);
- // create a random list of strings
- List values = new ArrayList<>();
- for (int v = 0; v < numValues; v++) {
- values.add(valuesProducer.get());
- }
-
- // add in any order to the indexed field
- ArrayList unordered = new ArrayList<>(values);
- Collections.shuffle(unordered, random());
- for (String v : values) {
- doc.add(newStringField("indexed", v, Field.Store.NO));
- }
-
- // add in any order to the dv field
- ArrayList unordered2 = new ArrayList<>(values);
- Collections.shuffle(unordered2, random());
- for (String v : unordered2) {
- doc.add(new SortedSetDocValuesField("dv", new BytesRef(v)));
- }
-
- writer.addDocument(doc);
- if (random().nextInt(31) == 0) {
- writer.commit();
- }
- }
-
- // delete some docs
- int numDeletions = random().nextInt(numDocs / 10);
- for (int i = 0; i < numDeletions; i++) {
- int id = random().nextInt(numDocs);
- writer.deleteDocuments(new Term("id", Integer.toString(id)));
- }
-
- // compare per-segment
- DirectoryReader ir = writer.getReader();
- for (LeafReaderContext context : ir.leaves()) {
- LeafReader r = context.reader();
- Terms terms = r.terms("indexed");
- if (terms != null) {
- SortedSetDocValues ssdv = r.getSortedSetDocValues("dv");
- assertEquals(terms.size(), ssdv.getValueCount());
- TermsEnum expected = terms.iterator();
- TermsEnum actual = r.getSortedSetDocValues("dv").termsEnum();
- assertEquals(terms.size(), expected, actual);
-
- doTestSortedSetEnumAdvanceIndependently(ssdv);
- }
- }
- ir.close();
-
- writer.forceMerge(1);
-
- // now compare again after the merge
- ir = writer.getReader();
- LeafReader ar = getOnlyLeafReader(ir);
- Terms terms = ar.terms("indexed");
- if (terms != null) {
- assertEquals(terms.size(), ar.getSortedSetDocValues("dv").getValueCount());
- TermsEnum expected = terms.iterator();
- TermsEnum actual = ar.getSortedSetDocValues("dv").termsEnum();
- assertEquals(terms.size(), expected, actual);
- }
- ir.close();
-
- writer.close();
- dir.close();
- }
-
- private void assertEquals(long numOrds, TermsEnum expected, TermsEnum actual) throws Exception {
- BytesRef ref;
-
- // sequential next() through all terms
- while ((ref = expected.next()) != null) {
- assertEquals(ref, actual.next());
- assertEquals(expected.ord(), actual.ord());
- assertEquals(expected.term(), actual.term());
- }
- assertNull(actual.next());
-
- // sequential seekExact(ord) through all terms
- for (long i = 0; i < numOrds; i++) {
- expected.seekExact(i);
- actual.seekExact(i);
- assertEquals(expected.ord(), actual.ord());
- assertEquals(expected.term(), actual.term());
- }
-
- // sequential seekExact(BytesRef) through all terms
- for (long i = 0; i < numOrds; i++) {
- expected.seekExact(i);
- assertTrue(actual.seekExact(expected.term()));
- assertEquals(expected.ord(), actual.ord());
- assertEquals(expected.term(), actual.term());
- }
-
- // sequential seekCeil(BytesRef) through all terms
- for (long i = 0; i < numOrds; i++) {
- expected.seekExact(i);
- assertEquals(SeekStatus.FOUND, actual.seekCeil(expected.term()));
- assertEquals(expected.ord(), actual.ord());
- assertEquals(expected.term(), actual.term());
- }
-
- // random seekExact(ord)
- for (long i = 0; i < numOrds; i++) {
- long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
- expected.seekExact(randomOrd);
- actual.seekExact(randomOrd);
- assertEquals(expected.ord(), actual.ord());
- assertEquals(expected.term(), actual.term());
- }
-
- // random seekExact(BytesRef)
- for (long i = 0; i < numOrds; i++) {
- long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
- expected.seekExact(randomOrd);
- actual.seekExact(expected.term());
- assertEquals(expected.ord(), actual.ord());
- assertEquals(expected.term(), actual.term());
- }
-
- // random seekCeil(BytesRef)
- for (long i = 0; i < numOrds; i++) {
- BytesRef target = new BytesRef(TestUtil.randomUnicodeString(random()));
- SeekStatus expectedStatus = expected.seekCeil(target);
- assertEquals(expectedStatus, actual.seekCeil(target));
- if (expectedStatus != SeekStatus.END) {
- assertEquals(expected.ord(), actual.ord());
- assertEquals(expected.term(), actual.term());
- }
- }
- }
-
- @Nightly
- public void testSortedSetAroundBlockSize() throws IOException {
- final int frontier = 1 << Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
- for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
- final Directory dir = newDirectory();
- IndexWriter w =
- new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
- ByteBuffersDataOutput out = new ByteBuffersDataOutput();
- Document doc = new Document();
- SortedSetDocValuesField field1 = new SortedSetDocValuesField("sset", new BytesRef());
- doc.add(field1);
- SortedSetDocValuesField field2 = new SortedSetDocValuesField("sset", new BytesRef());
- doc.add(field2);
- for (int i = 0; i < maxDoc; ++i) {
- BytesRef s1 = new BytesRef(TestUtil.randomSimpleString(random(), 2));
- BytesRef s2 = new BytesRef(TestUtil.randomSimpleString(random(), 2));
- field1.setBytesValue(s1);
- field2.setBytesValue(s2);
- w.addDocument(doc);
- Set set = new TreeSet<>(Arrays.asList(s1, s2));
- out.writeVInt(set.size());
- for (BytesRef ref : set) {
- out.writeVInt(ref.length);
- out.writeBytes(ref.bytes, ref.offset, ref.length);
- }
- }
- w.forceMerge(1);
- DirectoryReader r = DirectoryReader.open(w);
- w.close();
- LeafReader sr = getOnlyLeafReader(r);
- assertEquals(maxDoc, sr.maxDoc());
- SortedSetDocValues values = sr.getSortedSetDocValues("sset");
- assertNotNull(values);
- ByteBuffersDataInput in = new ByteBuffersDataInput(out.toBufferList());
- BytesRefBuilder b = new BytesRefBuilder();
- for (int i = 0; i < maxDoc; ++i) {
- assertEquals(i, values.nextDoc());
- final int numValues = in.readVInt();
-
- for (int j = 0; j < numValues; ++j) {
- b.setLength(in.readVInt());
- b.grow(b.length());
- in.readBytes(b.bytes(), 0, b.length());
- assertEquals(b.get(), values.lookupOrd(values.nextOrd()));
- }
-
- assertEquals(SortedSetDocValues.NO_MORE_ORDS, values.nextOrd());
- }
- r.close();
- dir.close();
- }
- }
-
- @Nightly
- public void testSortedNumericAroundBlockSize() throws IOException {
- final int frontier = 1 << Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
- for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
- final Directory dir = newDirectory();
- IndexWriter w =
- new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
- ByteBuffersDataOutput out = new ByteBuffersDataOutput();
- Document doc = new Document();
- SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("snum", 0L);
- doc.add(field1);
- SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("snum", 0L);
- doc.add(field2);
- for (int i = 0; i < maxDoc; ++i) {
- long s1 = random().nextInt(100);
- long s2 = random().nextInt(100);
- field1.setLongValue(s1);
- field2.setLongValue(s2);
- w.addDocument(doc);
- out.writeVLong(Math.min(s1, s2));
- out.writeVLong(Math.max(s1, s2));
- }
- w.forceMerge(1);
- DirectoryReader r = DirectoryReader.open(w);
- w.close();
- LeafReader sr = getOnlyLeafReader(r);
- assertEquals(maxDoc, sr.maxDoc());
- SortedNumericDocValues values = sr.getSortedNumericDocValues("snum");
- assertNotNull(values);
- ByteBuffersDataInput in = new ByteBuffersDataInput(out.toBufferList());
- for (int i = 0; i < maxDoc; ++i) {
- assertEquals(i, values.nextDoc());
- assertEquals(2, values.docValueCount());
- assertEquals(in.readVLong(), values.nextValue());
- assertEquals(in.readVLong(), values.nextValue());
- }
- r.close();
- dir.close();
- }
- }
-
- @Nightly
- public void testSortedNumericBlocksOfVariousBitsPerValue() throws Exception {
- doTestSortedNumericBlocksOfVariousBitsPerValue(() -> TestUtil.nextInt(random(), 1, 3));
- }
-
- @Nightly
- public void testSparseSortedNumericBlocksOfVariousBitsPerValue() throws Exception {
- doTestSortedNumericBlocksOfVariousBitsPerValue(() -> TestUtil.nextInt(random(), 0, 2));
- }
-
- @Nightly
- public void testNumericBlocksOfVariousBitsPerValue() throws Exception {
- doTestSparseNumericBlocksOfVariousBitsPerValue(1);
- }
-
- @Nightly
- public void testSparseNumericBlocksOfVariousBitsPerValue() throws Exception {
- doTestSparseNumericBlocksOfVariousBitsPerValue(random().nextDouble());
- }
-
- private static LongSupplier blocksOfVariousBPV() {
- final long mul = TestUtil.nextInt(random(), 1, 100);
- final long min = random().nextInt();
- return new LongSupplier() {
- int i = Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE;
- int maxDelta;
-
- @Override
- public long getAsLong() {
- if (i == Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE) {
- maxDelta = 1 << random().nextInt(5);
- i = 0;
- }
- i++;
- return min + mul * random().nextInt(maxDelta);
- }
- };
- }
-
- private void doTestSortedNumericBlocksOfVariousBitsPerValue(LongSupplier counts)
- throws Exception {
- Directory dir = newDirectory();
- IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
- conf.setMaxBufferedDocs(atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE));
- conf.setRAMBufferSizeMB(-1);
- conf.setMergePolicy(newLogMergePolicy(random().nextBoolean()));
- IndexWriter writer = new IndexWriter(dir, conf);
-
- final int numDocs = atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE * 3);
- final LongSupplier values = blocksOfVariousBPV();
- for (int i = 0; i < numDocs; i++) {
- Document doc = new Document();
-
- int valueCount = (int) counts.getAsLong();
- long[] valueArray = new long[valueCount];
- for (int j = 0; j < valueCount; j++) {
- long value = values.getAsLong();
- valueArray[j] = value;
- doc.add(new SortedNumericDocValuesField("dv", value));
- }
- Arrays.sort(valueArray);
- for (int j = 0; j < valueCount; j++) {
- doc.add(new StoredField("stored", Long.toString(valueArray[j])));
- }
- writer.addDocument(doc);
- if (random().nextInt(31) == 0) {
- writer.commit();
- }
- }
- writer.forceMerge(1);
-
- writer.close();
-
- // compare
- DirectoryReader ir = DirectoryReader.open(dir);
- TestUtil.checkReader(ir);
- for (LeafReaderContext context : ir.leaves()) {
- LeafReader r = context.reader();
- SortedNumericDocValues docValues = DocValues.getSortedNumeric(r, "dv");
- for (int i = 0; i < r.maxDoc(); i++) {
- if (i > docValues.docID()) {
- docValues.nextDoc();
- }
- String[] expected = r.document(i).getValues("stored");
- if (i < docValues.docID()) {
- assertEquals(0, expected.length);
- } else {
- String[] actual = new String[docValues.docValueCount()];
- for (int j = 0; j < actual.length; j++) {
- actual[j] = Long.toString(docValues.nextValue());
- }
- assertArrayEquals(expected, actual);
- }
- }
- }
- ir.close();
- dir.close();
- }
-
- private void doTestSparseNumericBlocksOfVariousBitsPerValue(double density) throws Exception {
- Directory dir = newDirectory();
- IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
- conf.setMaxBufferedDocs(atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE));
- conf.setRAMBufferSizeMB(-1);
- conf.setMergePolicy(newLogMergePolicy(random().nextBoolean()));
- IndexWriter writer = new IndexWriter(dir, conf);
- Document doc = new Document();
- Field storedField = newStringField("stored", "", Field.Store.YES);
- Field dvField = new NumericDocValuesField("dv", 0);
- doc.add(storedField);
- doc.add(dvField);
-
- final int numDocs = atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE * 3);
- final LongSupplier longs = blocksOfVariousBPV();
- for (int i = 0; i < numDocs; i++) {
- if (random().nextDouble() > density) {
- writer.addDocument(new Document());
- continue;
- }
- long value = longs.getAsLong();
- storedField.setStringValue(Long.toString(value));
- dvField.setLongValue(value);
- writer.addDocument(doc);
- }
-
- writer.forceMerge(1);
-
- writer.close();
-
- // compare
- DirectoryReader ir = DirectoryReader.open(dir);
- TestUtil.checkReader(ir);
- for (LeafReaderContext context : ir.leaves()) {
- LeafReader r = context.reader();
- NumericDocValues docValues = DocValues.getNumeric(r, "dv");
- docValues.nextDoc();
- for (int i = 0; i < r.maxDoc(); i++) {
- String storedValue = r.document(i).get("stored");
- if (storedValue == null) {
- assertTrue(docValues.docID() > i);
- } else {
- assertEquals(i, docValues.docID());
- assertEquals(Long.parseLong(storedValue), docValues.longValue());
- docValues.nextDoc();
- }
- }
- assertEquals(DocIdSetIterator.NO_MORE_DOCS, docValues.docID());
- }
- ir.close();
- dir.close();
- }
-}
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestLucene70NormsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestLucene70NormsFormat.java
deleted file mode 100644
index 55302352e08..00000000000
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestLucene70NormsFormat.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.backward_codecs.lucene70;
-
-import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.index.BaseNormsFormatTestCase;
-
-/** Tests Lucene70NormsFormat */
-public class TestLucene70NormsFormat extends BaseNormsFormatTestCase {
- @Override
- protected Codec getCodec() {
- return new Lucene70RWCodec();
- }
-}
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestLucene70SegmentInfoFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestLucene70SegmentInfoFormat.java
index ecc5d34417e..9a50ab7714f 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestLucene70SegmentInfoFormat.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestLucene70SegmentInfoFormat.java
@@ -22,12 +22,11 @@ import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BaseSegmentInfoFormatTestCase;
import org.apache.lucene.util.Version;
-@SuppressWarnings("deprecation")
public class TestLucene70SegmentInfoFormat extends BaseSegmentInfoFormatTestCase {
@Override
protected Version[] getVersions() {
- return new Version[] {Version.LUCENE_8_4_0};
+ return new Version[] {Version.fromBits(8, 4, 0)};
}
@Override
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene86/TestLucene86SegmentInfoFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene86/TestLucene86SegmentInfoFormat.java
index a88bb57efd6..e013fdfc1cb 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene86/TestLucene86SegmentInfoFormat.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene86/TestLucene86SegmentInfoFormat.java
@@ -27,7 +27,7 @@ public class TestLucene86SegmentInfoFormat extends BaseSegmentInfoFormatTestCase
@Override
protected Version[] getVersions() {
- return new Version[] {Version.LUCENE_8_8_1};
+ return new Version[] {Version.fromBits(8, 8, 1)};
}
@Override
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestBackwardsCompatibility.java
index ecc3b2ae911..ff6b55cea4b 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestBackwardsCompatibility.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestBackwardsCompatibility.java
@@ -332,81 +332,13 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
}
}
- static final String[] oldNames = {
- "8.0.0-cfs",
- "8.0.0-nocfs",
- "8.1.0-cfs",
- "8.1.0-nocfs",
- "8.1.1-cfs",
- "8.1.1-nocfs",
- "8.2.0-cfs",
- "8.2.0-nocfs",
- "8.3.0-cfs",
- "8.3.0-nocfs",
- "8.3.1-cfs",
- "8.3.1-nocfs",
- "8.4.0-cfs",
- "8.4.0-nocfs",
- "8.4.1-cfs",
- "8.4.1-nocfs",
- "8.5.0-cfs",
- "8.5.0-nocfs",
- "8.5.1-cfs",
- "8.5.1-nocfs",
- "8.5.2-cfs",
- "8.5.2-nocfs",
- "8.6.0-cfs",
- "8.6.0-nocfs",
- "8.6.1-cfs",
- "8.6.1-nocfs",
- "8.6.2-cfs",
- "8.6.2-nocfs",
- "8.6.3-cfs",
- "8.6.3-nocfs",
- "8.7.0-cfs",
- "8.7.0-nocfs",
- "8.8.0-cfs",
- "8.8.0-nocfs",
- "8.8.1-cfs",
- "8.8.1-nocfs",
- "8.8.2-cfs",
- "8.8.2-nocfs",
- "8.9.0-cfs",
- "8.9.0-nocfs",
- "8.10.0-cfs",
- "8.10.0-nocfs",
- "8.10.1-cfs",
- "8.10.1-nocfs"
- };
+ static final String[] oldNames = {};
public static String[] getOldNames() {
return oldNames;
}
- static final String[] oldSortedNames = {
- "sorted.8.0.0",
- "sorted.8.1.0",
- "sorted.8.1.1",
- "sorted.8.10.0",
- "sorted.8.2.0",
- "sorted.8.3.0",
- "sorted.8.3.1",
- "sorted.8.4.0",
- "sorted.8.4.1",
- "sorted.8.5.0",
- "sorted.8.5.1",
- "sorted.8.5.2",
- "sorted.8.6.0",
- "sorted.8.6.1",
- "sorted.8.6.2",
- "sorted.8.6.3",
- "sorted.8.7.0",
- "sorted.8.8.0",
- "sorted.8.8.1",
- "sorted.8.8.2",
- "sorted.8.9.0",
- "sorted.8.10.1"
- };
+ static final String[] oldSortedNames = {};
public static String[] getOldSortedNames() {
return oldSortedNames;
@@ -609,7 +541,51 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
"7.7.2-cfs",
"7.7.2-nocfs",
"7.7.3-cfs",
- "7.7.3-nocfs"
+ "7.7.3-nocfs",
+ "8.0.0-cfs",
+ "8.0.0-nocfs",
+ "8.1.0-cfs",
+ "8.1.0-nocfs",
+ "8.1.1-cfs",
+ "8.1.1-nocfs",
+ "8.2.0-cfs",
+ "8.2.0-nocfs",
+ "8.3.0-cfs",
+ "8.3.0-nocfs",
+ "8.3.1-cfs",
+ "8.3.1-nocfs",
+ "8.4.0-cfs",
+ "8.4.0-nocfs",
+ "8.4.1-cfs",
+ "8.4.1-nocfs",
+ "8.5.0-cfs",
+ "8.5.0-nocfs",
+ "8.5.1-cfs",
+ "8.5.1-nocfs",
+ "8.5.2-cfs",
+ "8.5.2-nocfs",
+ "8.6.0-cfs",
+ "8.6.0-nocfs",
+ "8.6.1-cfs",
+ "8.6.1-nocfs",
+ "8.6.2-cfs",
+ "8.6.2-nocfs",
+ "8.6.3-cfs",
+ "8.6.3-nocfs",
+ "8.7.0-cfs",
+ "8.7.0-nocfs",
+ "8.8.0-cfs",
+ "8.8.0-nocfs",
+ "8.8.1-cfs",
+ "8.8.1-nocfs",
+ "8.8.2-cfs",
+ "8.8.2-nocfs",
+ "8.9.0-cfs",
+ "8.9.0-nocfs",
+ "8.10.0-cfs",
+ "8.10.0-nocfs",
+ "8.10.1-cfs",
+ "8.10.1-nocfs"
};
static final int MIN_BINARY_SUPPORTED_MAJOR = Version.MIN_SUPPORTED_MAJOR - 1;
@@ -904,12 +880,17 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
CheckIndex checker = new CheckIndex(dir);
checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8));
CheckIndex.Status indexStatus = checker.checkIndex();
- if (unsupportedNames[i].startsWith("7.")) {
+ if (unsupportedNames[i].startsWith("8.")) {
assertTrue(indexStatus.clean);
} else {
assertFalse(indexStatus.clean);
- assertTrue(
- bos.toString(IOUtils.UTF_8).contains(IndexFormatTooOldException.class.getName()));
+ // CheckIndex doesn't enforce a minimum version, so we either get an
+ // IndexFormatTooOldException
+ // or an IllegalArgumentException saying that the codec doesn't exist.
+ boolean formatTooOld =
+ bos.toString(IOUtils.UTF_8).contains(IndexFormatTooOldException.class.getName());
+ boolean missingCodec = bos.toString(IOUtils.UTF_8).contains("Could not load codec");
+ assertTrue(formatTooOld || missingCodec);
}
checker.close();
@@ -982,7 +963,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
.filter( // depending on the MergePolicy we might see these segments merged away
sci ->
sci.getId() != null
- && sci.info.getVersion().onOrAfter(Version.LUCENE_8_6_0) == false)
+ && sci.info.getVersion().onOrAfter(Version.fromBits(8, 6, 0)) == false)
.findAny()
.orElse(null));
if (VERBOSE) {
@@ -1589,7 +1570,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
Directory dir = oldIndexDirs.get(name);
SegmentInfos infos = SegmentInfos.readLatestCommit(dir);
for (SegmentCommitInfo info : infos) {
- if (info.info.getVersion().onOrAfter(Version.LUCENE_8_6_0)) {
+ if (info.info.getVersion().onOrAfter(Version.fromBits(8, 6, 0))) {
assertNotNull(info.toString(), info.getId());
} else {
assertNull(info.toString(), info.getId());
@@ -1767,6 +1748,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
public static final String emptyIndex = "empty.8.0.0.zip";
+ @LuceneTestCase.AwaitsFix(bugUrl = "Unavailable until 9.0.0 is released")
public void testUpgradeEmptyOldIndex() throws Exception {
Path oldIndexDir = createTempDir("emptyIndex");
TestUtil.unzip(getDataInputStream(emptyIndex), oldIndexDir);
@@ -1779,8 +1761,9 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
dir.close();
}
- public static final String moreTermsIndex = "moreterms.8.0.0.zip";
+ public static final String moreTermsIndex = "moreterms.9.0.0.zip";
+ @LuceneTestCase.AwaitsFix(bugUrl = "Unavailable until 9.0.0 is released")
public void testMoreTerms() throws Exception {
Path oldIndexDir = createTempDir("moreterms");
TestUtil.unzip(getDataInputStream(moreTermsIndex), oldIndexDir);
@@ -1795,7 +1778,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
dir.close();
}
- public static final String dvUpdatesIndex = "dvupdates.8.0.0.zip";
+ public static final String dvUpdatesIndex = "dvupdates.9.0.0.zip";
private void assertNumericDocValues(LeafReader r, String f, String cf) throws IOException {
NumericDocValues ndvf = r.getNumericDocValues(f);
@@ -1829,6 +1812,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
reader.close();
}
+ @LuceneTestCase.AwaitsFix(bugUrl = "Unavailable until 9.0.0 is released")
public void testDocValuesUpdates() throws Exception {
Path oldIndexDir = createTempDir("dvupdates");
TestUtil.unzip(getDataInputStream(dvUpdatesIndex), oldIndexDir);
@@ -1857,6 +1841,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
dir.close();
}
+ @LuceneTestCase.AwaitsFix(bugUrl = "Unavailable until 9.0.0 is released")
public void testDeletes() throws Exception {
Path oldIndexDir = createTempDir("dvupdates");
TestUtil.unzip(getDataInputStream(dvUpdatesIndex), oldIndexDir);
@@ -1880,6 +1865,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
dir.close();
}
+ @LuceneTestCase.AwaitsFix(bugUrl = "Unavailable until 9.0.0 is released")
public void testSoftDeletes() throws Exception {
Path oldIndexDir = createTempDir("dvupdates");
TestUtil.unzip(getDataInputStream(dvUpdatesIndex), oldIndexDir);
@@ -1901,6 +1887,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
dir.close();
}
+ @LuceneTestCase.AwaitsFix(bugUrl = "Unavailable until 9.0.0 is released")
public void testDocValuesUpdatesWithNewField() throws Exception {
Path oldIndexDir = createTempDir("dvupdates");
TestUtil.unzip(getDataInputStream(dvUpdatesIndex), oldIndexDir);
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.0.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.0.0-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.0.0-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.0.0-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.0.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.0.0-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.0.0-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.0.0-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.1.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.1.0-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.1.0-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.1.0-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.1.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.1.0-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.1.0-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.1.0-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.1.1-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.1.1-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.1.1-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.1.1-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.1.1-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.1.1-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.1.1-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.1.1-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.10.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.10.0-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.10.0-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.10.0-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.10.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.10.0-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.10.0-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.10.0-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.10.1-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.10.1-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.10.1-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.10.1-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.10.1-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.10.1-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.10.1-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.10.1-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.2.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.2.0-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.2.0-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.2.0-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.2.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.2.0-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.2.0-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.2.0-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.3.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.3.0-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.3.0-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.3.0-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.3.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.3.0-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.3.0-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.3.0-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.3.1-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.3.1-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.3.1-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.3.1-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.3.1-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.3.1-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.3.1-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.3.1-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.4.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.4.0-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.4.0-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.4.0-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.4.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.4.0-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.4.0-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.4.0-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.4.1-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.4.1-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.4.1-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.4.1-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.4.1-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.4.1-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.4.1-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.4.1-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.5.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.5.0-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.5.0-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.5.0-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.5.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.5.0-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.5.0-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.5.0-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.5.1-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.5.1-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.5.1-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.5.1-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.5.1-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.5.1-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.5.1-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.5.1-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.5.2-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.5.2-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.5.2-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.5.2-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.5.2-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.5.2-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.5.2-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.5.2-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.6.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.6.0-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.6.0-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.6.0-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.6.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.6.0-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.6.0-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.6.0-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.6.1-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.6.1-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.6.1-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.6.1-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.6.1-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.6.1-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.6.1-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.6.1-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.6.2-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.6.2-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.6.2-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.6.2-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.6.2-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.6.2-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.6.2-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.6.2-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.6.3-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.6.3-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.6.3-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.6.3-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.6.3-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.6.3-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.6.3-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.6.3-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.7.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.7.0-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.7.0-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.7.0-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.7.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.7.0-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.7.0-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.7.0-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.8.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.8.0-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.8.0-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.8.0-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.8.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.8.0-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.8.0-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.8.0-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.8.1-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.8.1-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.8.1-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.8.1-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.8.1-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.8.1-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.8.1-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.8.1-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.8.2-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.8.2-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.8.2-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.8.2-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.8.2-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.8.2-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.8.2-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.8.2-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.9.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.9.0-cfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.9.0-cfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.9.0-cfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.9.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.9.0-nocfs.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.8.9.0-nocfs.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.8.9.0-nocfs.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.0.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.0.0.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.0.0.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.0.0.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.1.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.1.0.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.1.0.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.1.0.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.1.1.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.1.1.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.1.1.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.1.1.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.10.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.10.0.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.10.0.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.10.0.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.10.1.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.10.1.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.10.1.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.10.1.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.2.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.2.0.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.2.0.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.2.0.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.3.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.3.0.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.3.0.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.3.0.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.3.1.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.3.1.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.3.1.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.3.1.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.4.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.4.0.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.4.0.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.4.0.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.4.1.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.4.1.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.4.1.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.4.1.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.5.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.5.0.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.5.0.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.5.0.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.5.1.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.5.1.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.5.1.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.5.1.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.5.2.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.5.2.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.5.2.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.5.2.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.6.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.6.0.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.6.0.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.6.0.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.6.1.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.6.1.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.6.1.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.6.1.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.6.2.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.6.2.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.6.2.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.6.2.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.6.3.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.6.3.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.6.3.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.6.3.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.7.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.7.0.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.7.0.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.7.0.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.8.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.8.0.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.8.0.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.8.0.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.8.1.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.8.1.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.8.1.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.8.1.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.8.2.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.8.2.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.8.2.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.8.2.zip
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.9.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.9.0.zip
similarity index 100%
rename from lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.8.9.0.zip
rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_index/unsupported.sorted.8.9.0.zip
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
index 8650d96867d..c1cc07c0ccb 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
@@ -112,14 +112,7 @@ import org.apache.lucene.util.Version;
*/
public final class SegmentInfos implements Cloneable, Iterable {
- /**
- * The version that added information about the Lucene version at the time when the index has been
- * created.
- */
- public static final int VERSION_70 = 7;
- /** The version that updated segment name counter to be long instead of int. */
- public static final int VERSION_72 = 8;
- /** The version that recorded softDelCount */
+ /** The version at the time when 8.0 was released. */
public static final int VERSION_74 = 9;
/** The version that recorded SegmentCommitInfo IDs */
public static final int VERSION_86 = 10;
@@ -324,7 +317,7 @@ public final class SegmentInfos implements Cloneable, Iterable= VERSION_70) { // oldest supported version
+ if (format >= VERSION_74) { // oldest supported version
CodecUtil.checkFooter(input, priorE);
} else {
throw IOUtils.rethrowAlways(priorE);
@@ -379,11 +372,7 @@ public final class SegmentInfos implements Cloneable, Iterable VERSION_72 ? CodecUtil.readBEInt(input) : 0;
+ int softDelCount = CodecUtil.readBEInt(input);
if (softDelCount < 0 || softDelCount > info.maxDoc()) {
throw new CorruptIndexException(
"invalid deletion count: " + softDelCount + " vs maxDoc=" + info.maxDoc(), input);
diff --git a/lucene/core/src/java/org/apache/lucene/util/Version.java b/lucene/core/src/java/org/apache/lucene/util/Version.java
index 18caf930d61..222371f4105 100644
--- a/lucene/core/src/java/org/apache/lucene/util/Version.java
+++ b/lucene/core/src/java/org/apache/lucene/util/Version.java
@@ -28,167 +28,6 @@ import java.util.Locale;
*/
public final class Version {
- /**
- * Match settings and bugs in Lucene's 8.0.0 release.
- *
- * @deprecated (9.0.0) Use latest
- */
- @Deprecated public static final Version LUCENE_8_0_0 = new Version(8, 0, 0);
-
- /**
- * Match settings and bugs in Lucene's 8.1.0 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_1_0 = new Version(8, 1, 0);
-
- /**
- * Match settings and bugs in Lucene's 8.1.1 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_1_1 = new Version(8, 1, 1);
-
- /**
- * Match settings and bugs in Lucene's 8.2.0 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_2_0 = new Version(8, 2, 0);
-
- /**
- * Match settings and bugs in Lucene's 8.3.0 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_3_0 = new Version(8, 3, 0);
-
- /**
- * Match settings and bugs in Lucene's 8.3.1 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_3_1 = new Version(8, 3, 1);
-
- /**
- * Match settings and bugs in Lucene's 8.4.0 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_4_0 = new Version(8, 4, 0);
-
- /**
- * Match settings and bugs in Lucene's 8.4.1 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_4_1 = new Version(8, 4, 1);
-
- /**
- * Match settings and bugs in Lucene's 8.5.0 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_5_0 = new Version(8, 5, 0);
-
- /**
- * Match settings and bugs in Lucene's 8.5.1 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_5_1 = new Version(8, 5, 1);
-
- /**
- * Match settings and bugs in Lucene's 8.5.2 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_5_2 = new Version(8, 5, 2);
-
- /**
- * Match settings and bugs in Lucene's 8.6.0 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_6_0 = new Version(8, 6, 0);
-
- /**
- * Match settings and bugs in Lucene's 8.6.1 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_6_1 = new Version(8, 6, 1);
-
- /**
- * Match settings and bugs in Lucene's 8.6.2 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_6_2 = new Version(8, 6, 2);
-
- /**
- * Match settings and bugs in Lucene's 8.6.3 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_6_3 = new Version(8, 6, 3);
-
- /**
- * Match settings and bugs in Lucene's 8.7.0 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_7_0 = new Version(8, 7, 0);
-
- /**
- * Match settings and bugs in Lucene's 8.8.0 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_8_0 = new Version(8, 8, 0);
-
- /**
- * Match settings and bugs in Lucene's 8.8.1 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_8_1 = new Version(8, 8, 1);
-
- /**
- * Match settings and bugs in Lucene's 8.8.2 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_8_2 = new Version(8, 8, 2);
-
- /**
- * Match settings and bugs in Lucene's 8.9.0 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_9_0 = new Version(8, 9, 0);
-
- /**
- * Match settings and bugs in Lucene's 8.10.0 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_10_0 = new Version(8, 10, 0);
-
- /**
- * Match settings and bugs in Lucene's 8.10.1 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_10_1 = new Version(8, 10, 1);
-
- /**
- * Match settings and bugs in Lucene's 8.10.0 release.
- *
- * @deprecated Use latest
- */
- @Deprecated public static final Version LUCENE_8_11_0 = new Version(8, 11, 0);
-
/** @deprecated (10.0.0) Use latest */
@Deprecated public static final Version LUCENE_9_0_0 = new Version(9, 0, 0);
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java b/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java
index 99b772effdd..f090f89b9ac 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java
@@ -71,8 +71,8 @@ public class TestSegmentInfos extends LuceneTestCase {
SegmentInfo info =
new SegmentInfo(
dir,
- Version.LUCENE_9_0_0,
- Version.LUCENE_9_0_0,
+ Version.LUCENE_10_0_0,
+ Version.LUCENE_10_0_0,
"_0",
1,
false,
@@ -89,7 +89,7 @@ public class TestSegmentInfos extends LuceneTestCase {
sis.add(commitInfo);
sis.commit(dir);
sis = SegmentInfos.readLatestCommit(dir);
- assertEquals(Version.LUCENE_9_0_0, sis.getMinSegmentLuceneVersion());
+ assertEquals(Version.LUCENE_10_0_0, sis.getMinSegmentLuceneVersion());
assertEquals(Version.LATEST, sis.getCommitLuceneVersion());
dir.close();
}
@@ -105,8 +105,8 @@ public class TestSegmentInfos extends LuceneTestCase {
SegmentInfo info =
new SegmentInfo(
dir,
- Version.LUCENE_9_0_0,
- Version.LUCENE_9_0_0,
+ Version.LUCENE_10_0_0,
+ Version.LUCENE_10_0_0,
"_0",
1,
false,
@@ -124,8 +124,8 @@ public class TestSegmentInfos extends LuceneTestCase {
info =
new SegmentInfo(
dir,
- Version.LUCENE_9_0_0,
- Version.LUCENE_9_0_0,
+ Version.LUCENE_10_0_0,
+ Version.LUCENE_10_0_0,
"_1",
1,
false,
@@ -143,7 +143,7 @@ public class TestSegmentInfos extends LuceneTestCase {
byte[] commitInfoId0 = sis.info(0).getId();
byte[] commitInfoId1 = sis.info(1).getId();
sis = SegmentInfos.readLatestCommit(dir);
- assertEquals(Version.LUCENE_9_0_0, sis.getMinSegmentLuceneVersion());
+ assertEquals(Version.LUCENE_10_0_0, sis.getMinSegmentLuceneVersion());
assertEquals(Version.LATEST, sis.getCommitLuceneVersion());
assertEquals(
StringHelper.idToString(commitInfoId0), StringHelper.idToString(sis.info(0).getId()));
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestVersion.java b/lucene/core/src/test/org/apache/lucene/util/TestVersion.java
index 24bcb4486e8..d8f506b3ea6 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestVersion.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestVersion.java
@@ -32,26 +32,25 @@ public class TestVersion extends LuceneTestCase {
assertTrue("LATEST must be always onOrAfter(" + v + ")", Version.LATEST.onOrAfter(v));
}
}
- assertTrue(Version.LUCENE_9_0_0.onOrAfter(Version.LUCENE_8_0_0));
+ assertTrue(Version.LUCENE_10_0_0.onOrAfter(Version.LUCENE_9_0_0));
;
}
public void testToString() {
- assertEquals("8.0.0", Version.LUCENE_8_0_0.toString());
assertEquals("9.0.0", Version.LUCENE_9_0_0.toString());
+ assertEquals("10.0.0", Version.LUCENE_10_0_0.toString());
}
public void testParseLeniently() throws Exception {
+ assertEquals(Version.LUCENE_10_0_0, Version.parseLeniently("10.0"));
+ assertEquals(Version.LUCENE_10_0_0, Version.parseLeniently("10.0.0"));
+ assertEquals(Version.LUCENE_10_0_0, Version.parseLeniently("LUCENE_10_0"));
+ assertEquals(Version.LUCENE_10_0_0, Version.parseLeniently("LUCENE_10_0_0"));
assertEquals(Version.LUCENE_9_0_0, Version.parseLeniently("9.0"));
assertEquals(Version.LUCENE_9_0_0, Version.parseLeniently("9.0.0"));
assertEquals(Version.LUCENE_9_0_0, Version.parseLeniently("LUCENE_90"));
assertEquals(Version.LUCENE_9_0_0, Version.parseLeniently("LUCENE_9_0"));
assertEquals(Version.LUCENE_9_0_0, Version.parseLeniently("LUCENE_9_0_0"));
- assertEquals(Version.LUCENE_8_0_0, Version.parseLeniently("8.0"));
- assertEquals(Version.LUCENE_8_0_0, Version.parseLeniently("8.0.0"));
- assertEquals(Version.LUCENE_8_0_0, Version.parseLeniently("LUCENE_80"));
- assertEquals(Version.LUCENE_8_0_0, Version.parseLeniently("LUCENE_8_0"));
- assertEquals(Version.LUCENE_8_0_0, Version.parseLeniently("LUCENE_8_0_0"));
assertEquals(Version.LATEST, Version.parseLeniently("LATEST"));
assertEquals(Version.LATEST, Version.parseLeniently("latest"));
@@ -108,8 +107,8 @@ public class TestVersion extends LuceneTestCase {
}
public void testParse() throws Exception {
+ assertEquals(Version.LUCENE_10_0_0, Version.parse("10.0.0"));
assertEquals(Version.LUCENE_9_0_0, Version.parse("9.0.0"));
- assertEquals(Version.LUCENE_8_0_0, Version.parse("8.0.0"));
// Version does not pass judgement on the major version:
assertEquals(1, Version.parse("1.0").major);
@@ -117,7 +116,7 @@ public class TestVersion extends LuceneTestCase {
}
public void testForwardsCompatibility() throws Exception {
- assertTrue(Version.parse("8.10.20").onOrAfter(Version.LUCENE_8_0_0));
+ assertTrue(Version.parse("9.10.20").onOrAfter(Version.LUCENE_9_0_0));
}
public void testParseExceptions() {
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java
index 66836536ffa..6454c8e68ec 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java
@@ -26,7 +26,6 @@ import java.util.Map;
import java.util.function.IntUnaryOperator;
import java.util.logging.Level;
import java.util.logging.Logger;
-import org.apache.lucene.document.Document;
import org.apache.lucene.facet.FacetsConfig;
import org.apache.lucene.facet.taxonomy.FacetLabel;
import org.apache.lucene.facet.taxonomy.LRUHashMap;
@@ -338,10 +337,7 @@ public class DirectoryTaxonomyReader extends TaxonomyReader implements Accountab
if (values == null
|| values.advanceExact(ordinal - indexReader.leaves().get(readerIndex).docBase) == false) {
- // The index uses the older StoredField format to store the mapping
- // On recreating the index, the values will be stored using the BinaryDocValuesField format
- Document doc = indexReader.document(ordinal);
- ret = new FacetLabel(FacetsConfig.stringToPath(doc.get(Consts.FULL)));
+ throw new IllegalStateException();
} else {
// The index uses the BinaryDocValuesField format to store the mapping
ret = new FacetLabel(FacetsConfig.stringToPath(values.binaryValue().utf8ToString()));
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
index 90f1d4e6211..7e2decf8918 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
@@ -91,7 +91,6 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
private final Directory dir;
private final IndexWriter indexWriter;
- private final boolean useOlderStoredFieldIndex;
private final TaxonomyWriterCache cache;
private final AtomicInteger cacheMisses = new AtomicInteger(0);
@@ -159,14 +158,10 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
openMode = config.getOpenMode();
if (DirectoryReader.indexExists(directory) == false) {
indexEpoch = 1;
- // no commit exists so we can safely use the new BinaryDocValues field
- useOlderStoredFieldIndex = false;
} else {
String epochStr = null;
SegmentInfos infos = SegmentInfos.readLatestCommit(dir);
- /* a previous commit exists, so check the version of the last commit */
- useOlderStoredFieldIndex = infos.getIndexCreatedVersionMajor() <= 8;
Map commitData = infos.getUserData();
if (commitData != null) {
@@ -184,11 +179,7 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
ft.setOmitNorms(true);
parentStreamField = new Field(Consts.FIELD_PAYLOADS, parentStream, ft);
- if (useOlderStoredFieldIndex) {
- fullPathField = new StringField(Consts.FULL, "", Field.Store.YES);
- } else {
- fullPathField = new StringField(Consts.FULL, "", Field.Store.NO);
- }
+ fullPathField = new StringField(Consts.FULL, "", Field.Store.NO);
nextID = indexWriter.getDocStats().maxDoc;
@@ -482,10 +473,8 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
String fieldPath = FacetsConfig.pathToString(categoryPath.components, categoryPath.length);
fullPathField.setStringValue(fieldPath);
- if (useOlderStoredFieldIndex == false) {
- /* Lucene 9 switches to BinaryDocValuesField for storing taxonomy categories */
- d.add(new BinaryDocValuesField(Consts.FULL, new BytesRef(fieldPath)));
- }
+ /* Lucene 9 switches to BinaryDocValuesField for storing taxonomy categories */
+ d.add(new BinaryDocValuesField(Consts.FULL, new BytesRef(fieldPath)));
d.add(fullPathField);
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestBackwardsCompatibility.java b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestBackwardsCompatibility.java
deleted file mode 100644
index a702ecec723..00000000000
--- a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestBackwardsCompatibility.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.facet.taxonomy.directory;
-
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import org.apache.lucene.facet.taxonomy.FacetLabel;
-import org.apache.lucene.facet.taxonomy.TaxonomyReader;
-import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
-import org.junit.Ignore;
-
-/*
- Verify we can read previous versions' taxonomy indexes, do searches
- against them, and add documents to them.
-*/
-// See: https://issues.apache.org/jira/browse/SOLR-12028 Tests cannot remove files on Windows
-// machines occasionally
-public class TestBackwardsCompatibility extends LuceneTestCase {
-
- // To generate backcompat indexes with the current default codec, run the following gradle
- // command:
- // gradlew test -Dtestcase=TestBackwardsCompatibility -Dtests.bwcdir=/path/to/store/indexes
- // -Dtests.codec=default -Dtests.useSecurityManager=false
- // Also add testmethod with one of the index creation methods below, for example:
- // -Dtestmethod=testCreateOldTaxonomy
- //
- // Zip up the generated indexes:
- //
- // cd /path/to/store/indexes/index.cfs ; zip index.-cfs.zip *
- //
- // Then move the zip file to your trunk checkout and use it in your test cases
-
- public static final String oldTaxonomyIndexName = "taxonomy.8.10.0-cfs";
-
- public void testCreateNewTaxonomy() throws IOException {
- createNewTaxonomyIndex(oldTaxonomyIndexName);
- }
-
- // Opens up a pre-existing old taxonomy index and adds new BinaryDocValues based fields
- private void createNewTaxonomyIndex(String dirName) throws IOException {
- Path indexDir = createTempDir(oldTaxonomyIndexName);
- TestUtil.unzip(getDataInputStream(dirName + ".zip"), indexDir);
- Directory dir = newFSDirectory(indexDir);
-
- DirectoryTaxonomyWriter writer = new DirectoryTaxonomyWriter(dir);
-
- FacetLabel cp_c = new FacetLabel("c");
- writer.addCategory(cp_c);
- writer.getInternalIndexWriter().forceMerge(1);
- writer.commit();
-
- TaxonomyReader reader = new DirectoryTaxonomyReader(writer);
-
- int ord1 = reader.getOrdinal(new FacetLabel("a"));
- assert ord1 != TaxonomyReader.INVALID_ORDINAL;
- // Just asserting ord1 != TaxonomyReader.INVALID_ORDINAL is not enough to check compatibility
- assertNotNull(reader.getPath(ord1));
-
- int ord2 = reader.getOrdinal(new FacetLabel("b"));
- assert ord2 != TaxonomyReader.INVALID_ORDINAL;
- // Just asserting ord2 != TaxonomyReader.INVALID_ORDINAL is not enough to check compatibility
- assertNotNull(reader.getPath(ord2));
-
- int ord3 = reader.getOrdinal(cp_c);
- assert ord3 != TaxonomyReader.INVALID_ORDINAL;
- assertNotNull(reader.getPath(ord3));
-
- reader.close();
- writer.close();
- dir.close();
- }
-
- // Opens up a pre-existing index and tries to run getBulkPath on it
- public void testGetBulkPathOnOlderCodec() throws Exception {
- Path indexDir = createTempDir(oldTaxonomyIndexName);
- TestUtil.unzip(getDataInputStream(oldTaxonomyIndexName + ".zip"), indexDir);
- Directory dir = newFSDirectory(indexDir);
-
- DirectoryTaxonomyWriter writer = new DirectoryTaxonomyWriter(dir);
- DirectoryTaxonomyReader reader = new DirectoryTaxonomyReader(writer);
-
- FacetLabel[] facetLabels = {
- new FacetLabel("a"), new FacetLabel("b"),
- };
- int[] ordinals =
- new int[] {reader.getOrdinal(facetLabels[0]), reader.getOrdinal(facetLabels[1])};
- // The zip file already contains a category "a" and a category "b" stored with the older
- // StoredFields codec
- assertArrayEquals(facetLabels, reader.getBulkPath(ordinals));
- reader.close();
- writer.close();
- dir.close();
- }
-
- // Used to create a fresh taxonomy index with StoredFields
- @Ignore
- public void testCreateOldTaxonomy() throws IOException {
- createOldTaxonomyIndex(oldTaxonomyIndexName);
- }
-
- private void createOldTaxonomyIndex(String dirName) throws IOException {
- Path indexDir = getIndexDir().resolve(dirName);
- Files.deleteIfExists(indexDir);
- Directory dir = newFSDirectory(indexDir);
-
- TaxonomyWriter writer = new DirectoryTaxonomyWriter(dir);
-
- writer.addCategory(new FacetLabel("a"));
- writer.addCategory(new FacetLabel("b"));
- writer.commit();
- writer.close();
- dir.close();
- }
-
- private Path getIndexDir() {
- String path = System.getProperty("tests.bwcdir");
- assumeTrue(
- "backcompat creation tests must be run with -Dtests.bwcdir=/path/to/write/indexes",
- path != null);
- return Paths.get(path);
- }
-}
diff --git a/lucene/luke/src/java/org/apache/lucene/luke/models/util/IndexUtils.java b/lucene/luke/src/java/org/apache/lucene/luke/models/util/IndexUtils.java
index 099326f7602..5841784521f 100644
--- a/lucene/luke/src/java/org/apache/lucene/luke/models/util/IndexUtils.java
+++ b/lucene/luke/src/java/org/apache/lucene/luke/models/util/IndexUtils.java
@@ -314,12 +314,8 @@ public final class IndexUtils {
if (CodecUtil.CODEC_MAGIC == CodecUtil.readBEInt(in)) {
int actualVersion =
CodecUtil.checkHeaderNoMagic(
- in, "segments", SegmentInfos.VERSION_70, Integer.MAX_VALUE);
- if (actualVersion == SegmentInfos.VERSION_70) {
- format = "Lucene 7.0 or later";
- } else if (actualVersion == SegmentInfos.VERSION_72) {
- format = "Lucene 7.2 or later";
- } else if (actualVersion == SegmentInfos.VERSION_74) {
+ in, "segments", SegmentInfos.VERSION_74, Integer.MAX_VALUE);
+ if (actualVersion == SegmentInfos.VERSION_74) {
format = "Lucene 7.4 or later";
} else if (actualVersion == SegmentInfos.VERSION_86) {
format = "Lucene 8.6 or later";
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java
index b74f702c9aa..09447ac3753 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java
@@ -21,7 +21,6 @@ import java.util.Collection;
import java.util.List;
import java.util.NoSuchElementException;
import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.Version;
import org.locationtech.spatial4j.context.SpatialContext;
import org.locationtech.spatial4j.shape.Point;
import org.locationtech.spatial4j.shape.Rectangle;
@@ -59,12 +58,7 @@ public class PackedQuadPrefixTree extends QuadPrefixTree {
public static class Factory extends QuadPrefixTree.Factory {
@Override
protected SpatialPrefixTree newSPT() {
- PackedQuadPrefixTree tree =
- new PackedQuadPrefixTree(ctx, maxLevels != null ? maxLevels : MAX_LEVELS_POSSIBLE);
- @SuppressWarnings("deprecation")
- Version lucene830 = Version.LUCENE_8_3_0;
- tree.robust = getVersion().onOrAfter(lucene830);
- return tree;
+ return new PackedQuadPrefixTree(ctx, maxLevels != null ? maxLevels : MAX_LEVELS_POSSIBLE);
}
}
@@ -95,15 +89,6 @@ public class PackedQuadPrefixTree extends QuadPrefixTree {
@Override
public Cell getCell(Point p, int level) {
- if (!robust) { // old method
- List cells = new ArrayList<>(1);
- buildNotRobustly(
- xmid, ymid, 0, cells, 0x0L, ctx.getShapeFactory().pointXY(p.getX(), p.getY()), level);
- if (!cells.isEmpty()) {
- return cells.get(0); // note cells could be longer if p on edge
- }
- }
-
double currentXmid = xmid;
double currentYmid = ymid;
double xp = p.getX();
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java
index 2e72745d07a..225ca226547 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java
@@ -23,7 +23,6 @@ import java.util.Collection;
import java.util.List;
import java.util.Locale;
import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.Version;
import org.locationtech.spatial4j.context.SpatialContext;
import org.locationtech.spatial4j.shape.Point;
import org.locationtech.spatial4j.shape.Rectangle;
@@ -47,12 +46,7 @@ public class QuadPrefixTree extends LegacyPrefixTree {
@Override
protected SpatialPrefixTree newSPT() {
- QuadPrefixTree tree =
- new QuadPrefixTree(ctx, maxLevels != null ? maxLevels : MAX_LEVELS_POSSIBLE);
- @SuppressWarnings("deprecation")
- Version LUCENE_8_3_0 = Version.LUCENE_8_3_0;
- tree.robust = getVersion().onOrAfter(LUCENE_8_3_0);
- return tree;
+ return new QuadPrefixTree(ctx, maxLevels != null ? maxLevels : MAX_LEVELS_POSSIBLE);
}
}
@@ -72,9 +66,6 @@ public class QuadPrefixTree extends LegacyPrefixTree {
final double[] levelW;
final double[] levelH;
- protected boolean robust =
- true; // for backward compatibility, use the old method if user specified old version.
-
public QuadPrefixTree(SpatialContext ctx, Rectangle bounds, int maxLevels) {
super(ctx, maxLevels);
this.xmin = bounds.getMinX();
@@ -137,21 +128,6 @@ public class QuadPrefixTree extends LegacyPrefixTree {
@Override
public Cell getCell(Point p, int level) {
- if (!robust) { // old method
- List cells = new ArrayList<>(1);
- buildNotRobustly(
- xmid,
- ymid,
- 0,
- cells,
- new BytesRef(maxLevels + 1),
- ctx.getShapeFactory().pointXY(p.getX(), p.getY()),
- level);
- if (!cells.isEmpty()) {
- return cells.get(0); // note cells could be longer if p on edge
- }
- }
-
double currentXmid = xmid;
double currentYmid = ymid;
double xp = p.getX();
| |