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 d929bfd099e..d9ea43c2345 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 @@ -107,7 +107,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase { public void testFactoryHtmlStripClassicFolding() throws Exception { CustomAnalyzer a = CustomAnalyzer.builder() - .withDefaultMatchVersion(Version.LUCENE_6_0_0) + .withDefaultMatchVersion(Version.LUCENE_7_0_0) .addCharFilter(HTMLStripCharFilterFactory.class) .withTokenizer(ClassicTokenizerFactory.class) .addTokenFilter(ASCIIFoldingFilterFactory.class, "preserveOriginal", "true") @@ -126,7 +126,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase { assertSame(LowerCaseFilterFactory.class, tokenFilters.get(1).getClass()); assertEquals(100, a.getPositionIncrementGap("dummy")); assertEquals(1000, a.getOffsetGap("dummy")); - assertSame(Version.LUCENE_6_0_0, a.getVersion()); + assertSame(Version.LUCENE_7_0_0, a.getVersion()); assertAnalyzesTo(a, "

foo bar

FOO BAR", new String[] { "foo", "bar", "foo", "bar" }, @@ -139,7 +139,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase { public void testHtmlStripClassicFolding() throws Exception { CustomAnalyzer a = CustomAnalyzer.builder() - .withDefaultMatchVersion(Version.LUCENE_6_0_0) + .withDefaultMatchVersion(Version.LUCENE_7_0_0) .addCharFilter("htmlstrip") .withTokenizer("classic") .addTokenFilter("asciifolding", "preserveOriginal", "true") @@ -158,7 +158,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase { assertSame(LowerCaseFilterFactory.class, tokenFilters.get(1).getClass()); assertEquals(100, a.getPositionIncrementGap("dummy")); assertEquals(1000, a.getOffsetGap("dummy")); - assertSame(Version.LUCENE_6_0_0, a.getVersion()); + assertSame(Version.LUCENE_7_0_0, a.getVersion()); assertAnalyzesTo(a, "

foo bar

FOO BAR", new String[] { "foo", "bar", "foo", "bar" }, diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java deleted file mode 100644 index d2a384e975f..00000000000 --- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java +++ /dev/null @@ -1,90 +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.codecs.lucene50; - - -import java.io.IOException; -import java.util.Map; -import java.util.Set; - -import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.codecs.SegmentInfoFormat; -import org.apache.lucene.index.CorruptIndexException; -import org.apache.lucene.index.IndexFileNames; -import org.apache.lucene.index.SegmentInfo; // javadocs -import org.apache.lucene.store.ChecksumIndexInput; -import org.apache.lucene.store.Directory; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.util.Version; - -/** - * Lucene 5.0 Segment info format. - * @deprecated Only for reading old 5.0-6.0 segments - */ -@Deprecated -public class Lucene50SegmentInfoFormat extends SegmentInfoFormat { - - /** Sole constructor. */ - public Lucene50SegmentInfoFormat() { - } - - @Override - public SegmentInfo read(Directory dir, String segment, byte[] segmentID, IOContext context) throws IOException { - final String fileName = IndexFileNames.segmentFileName(segment, "", Lucene50SegmentInfoFormat.SI_EXTENSION); - try (ChecksumIndexInput input = dir.openChecksumInput(fileName, context)) { - Throwable priorE = null; - SegmentInfo si = null; - try { - CodecUtil.checkIndexHeader(input, Lucene50SegmentInfoFormat.CODEC_NAME, - Lucene50SegmentInfoFormat.VERSION_START, - Lucene50SegmentInfoFormat.VERSION_CURRENT, - segmentID, ""); - final Version version = Version.fromBits(input.readInt(), input.readInt(), input.readInt()); - - final int docCount = input.readInt(); - if (docCount < 0) { - throw new CorruptIndexException("invalid docCount: " + docCount, input); - } - final boolean isCompoundFile = input.readByte() == SegmentInfo.YES; - - final Map diagnostics = input.readMapOfStrings(); - final Set files = input.readSetOfStrings(); - final Map attributes = input.readMapOfStrings(); - - si = new SegmentInfo(dir, version, null, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, null); - si.setFiles(files); - } catch (Throwable exception) { - priorE = exception; - } finally { - CodecUtil.checkFooter(input, priorE); - } - return si; - } - } - - @Override - public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException { - throw new UnsupportedOperationException("this codec can only be used for reading"); - } - - /** File extension used to store {@link SegmentInfo}. */ - public final static String SI_EXTENSION = "si"; - static final String CODEC_NAME = "Lucene50SegmentInfo"; - static final int VERSION_SAFE_MAPS = 1; - static final int VERSION_START = VERSION_SAFE_MAPS; - static final int VERSION_CURRENT = VERSION_SAFE_MAPS; -} diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/package.html b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/package.html deleted file mode 100644 index 352cef4bf9a..00000000000 --- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/package.html +++ /dev/null @@ -1,25 +0,0 @@ - - - - - - - -Lucene 5.0 file format. - - diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsFormat.java deleted file mode 100644 index 1f7928f2b29..00000000000 --- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsFormat.java +++ /dev/null @@ -1,91 +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.codecs.lucene53; - - -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 5.3 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: - *

    - *
  1. .nvd: Norms data
  2. - *
  3. .nvm: Norms metadata
  4. - *
- *
    - *
  1. - *

    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,BytesPerValue, Address
    • - *
    • FieldNumber --> {@link DataOutput#writeVInt vInt}
    • - *
    • BytesPerValue --> {@link DataOutput#writeByte byte}
    • - *
    • Offset --> {@link DataOutput#writeLong Int64}
    • - *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter}
    • - *
    - *

    FieldNumber of -1 indicates the end of metadata.

    - *

    Offset is the pointer to the start of the data in the norms data (.nvd), or the singleton value - * when BytesPerValue = 0

    - *
  2. - *

    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}
    • - *
    • Data --> {@link DataOutput#writeByte(byte) byte}MaxDoc * BytesPerValue
    • - *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter}
    • - *
    - *
- * @lucene.experimental - */ -public class Lucene53NormsFormat extends NormsFormat { - - /** Sole Constructor */ - public Lucene53NormsFormat() {} - - @Override - public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException { - throw new UnsupportedOperationException("This format can only be used for reading"); - } - - @Override - public NormsProducer normsProducer(SegmentReadState state) throws IOException { - return new Lucene53NormsProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION); - } - - static final String DATA_CODEC = "Lucene53NormsData"; - static final String DATA_EXTENSION = "nvd"; - static final String METADATA_CODEC = "Lucene53NormsMetadata"; - 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/codecs/lucene53/Lucene53NormsProducer.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java deleted file mode 100644 index 718fcd651b9..00000000000 --- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java +++ /dev/null @@ -1,236 +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.codecs.lucene53; - - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.codecs.NormsProducer; -import org.apache.lucene.index.CorruptIndexException; -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; - -import static org.apache.lucene.codecs.lucene53.Lucene53NormsFormat.VERSION_CURRENT; -import static org.apache.lucene.codecs.lucene53.Lucene53NormsFormat.VERSION_START; - -/** - * Reader for {@link Lucene53NormsFormat} - */ -class Lucene53NormsProducer extends NormsProducer { - // metadata maps (just file pointers and minimal stuff) - private final Map norms = new HashMap<>(); - private final IndexInput data; - private final int maxDoc; - - Lucene53NormsProducer(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 = state.directory.openChecksumInput(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 = state.directory.openInput(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); - } - } - } - - private void readFields(IndexInput meta, FieldInfos infos) throws IOException { - int fieldNumber = meta.readVInt(); - while (fieldNumber != -1) { - 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.bytesPerValue = meta.readByte(); - switch (entry.bytesPerValue) { - case 0: case 1: case 2: case 4: case 8: - break; - default: - throw new CorruptIndexException("Invalid bytesPerValue: " + entry.bytesPerValue + ", field: " + info.name, meta); - } - entry.offset = meta.readLong(); - norms.put(info.number, entry); - fieldNumber = meta.readVInt(); - } - } - - @Override - public NumericDocValues getNorms(FieldInfo field) throws IOException { - final NormsEntry entry = norms.get(field.number); - - if (entry.bytesPerValue == 0) { - final long value = entry.offset; - return new NormsIterator(maxDoc) { - @Override - public long longValue() { - return value; - } - }; - } else { - RandomAccessInput slice; - synchronized (data) { - switch (entry.bytesPerValue) { - case 1: - slice = data.randomAccessSlice(entry.offset, maxDoc); - return new NormsIterator(maxDoc) { - @Override - public long longValue() throws IOException { - return slice.readByte(docID); - } - }; - case 2: - slice = data.randomAccessSlice(entry.offset, maxDoc * 2L); - return new NormsIterator(maxDoc) { - @Override - public long longValue() throws IOException { - return slice.readShort(((long)docID) << 1L); - } - }; - case 4: - slice = data.randomAccessSlice(entry.offset, maxDoc * 4L); - return new NormsIterator(maxDoc) { - @Override - public long longValue() throws IOException { - return slice.readInt(((long)docID) << 2L); - } - }; - case 8: - slice = data.randomAccessSlice(entry.offset, maxDoc * 8L); - return new NormsIterator(maxDoc) { - @Override - public long longValue() throws IOException { - return slice.readLong(((long)docID) << 3L); - } - }; - default: - throw new AssertionError(); - } - } - } - } - - @Override - public void close() throws IOException { - data.close(); - } - - @Override - public long ramBytesUsed() { - return 64L * norms.size(); // good enough - } - - @Override - public void checkIntegrity() throws IOException { - CodecUtil.checksumEntireFile(data); - } - - static class NormsEntry { - byte bytesPerValue; - long offset; - } - - @Override - public String toString() { - return getClass().getSimpleName() + "(fields=" + norms.size() + ")"; - } - - private static abstract class NormsIterator extends NumericDocValues { - private final int maxDoc; - protected int docID = -1; - - public NormsIterator(int maxDoc) { - this.maxDoc = maxDoc; - } - - @Override - public int docID() { - return docID; - } - - @Override - public int nextDoc() { - docID++; - if (docID == maxDoc) { - docID = NO_MORE_DOCS; - } - return docID; - } - - @Override - public int advance(int target) { - docID = target; - if (docID >= maxDoc) { - docID = NO_MORE_DOCS; - } - return docID; - } - - @Override - public boolean advanceExact(int target) throws IOException { - docID = target; - return true; - } - - @Override - public long cost() { - // TODO - return 0; - } - } -} diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package-info.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package-info.java deleted file mode 100644 index 93fefb8448a..00000000000 --- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package-info.java +++ /dev/null @@ -1,23 +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. - */ - -/** - * Components from the Lucene 5.3 index format - * See {@link org.apache.lucene.codecs.lucene53} for an overview - * of the index format. - */ -package org.apache.lucene.codecs.lucene53; diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java deleted file mode 100644 index 217cd4318fd..00000000000 --- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java +++ /dev/null @@ -1,797 +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.codecs.lucene54; - - -import java.io.Closeable; // javadocs -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.Map; -import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; -import java.util.stream.StreamSupport; - -import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.codecs.DocValuesConsumer; -import org.apache.lucene.codecs.DocValuesProducer; -import org.apache.lucene.codecs.LegacyDocValuesIterables; -import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.IndexFileNames; -import org.apache.lucene.index.SegmentWriteState; -import org.apache.lucene.store.IndexOutput; -import org.apache.lucene.store.RAMOutputStream; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.BytesRefBuilder; -import org.apache.lucene.util.IOUtils; -import org.apache.lucene.util.LongsRef; -import org.apache.lucene.util.MathUtil; -import org.apache.lucene.util.PagedBytes.PagedBytesDataInput; -import org.apache.lucene.util.PagedBytes; -import org.apache.lucene.util.StringHelper; -import org.apache.lucene.util.packed.DirectMonotonicWriter; -import org.apache.lucene.util.packed.DirectWriter; -import org.apache.lucene.util.packed.MonotonicBlockPackedWriter; -import org.apache.lucene.util.packed.PackedInts; - -import static org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat.*; - -/** writer for {@link Lucene54DocValuesFormat} */ -final class Lucene54DocValuesConsumer extends DocValuesConsumer implements Closeable { - - enum NumberType { - /** Dense ordinals */ - ORDINAL, - /** Random long values */ - VALUE; - } - - IndexOutput data, meta; - final int maxDoc; - - /** expert: Creates a new writer */ - public Lucene54DocValuesConsumer(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 = state.directory.createOutput(dataName, state.context); - CodecUtil.writeIndexHeader(data, dataCodec, Lucene54DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); - String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension); - meta = state.directory.createOutput(metaName, state.context); - CodecUtil.writeIndexHeader(meta, metaCodec, Lucene54DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); - maxDoc = state.segmentInfo.maxDoc(); - success = true; - } finally { - if (!success) { - IOUtils.closeWhileHandlingException(this); - } - } - } - - @Override - public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { - addNumericField(field, LegacyDocValuesIterables.numericIterable(field, valuesProducer, maxDoc), NumberType.VALUE); - } - - void addNumericField(FieldInfo field, Iterable values, NumberType numberType) throws IOException { - long count = 0; - long minValue = Long.MAX_VALUE; - long maxValue = Long.MIN_VALUE; - long gcd = 0; - long missingCount = 0; - long zeroCount = 0; - // TODO: more efficient? - HashSet uniqueValues = null; - long missingOrdCount = 0; - if (numberType == NumberType.VALUE) { - uniqueValues = new HashSet<>(); - - for (Number nv : values) { - final long v; - if (nv == null) { - v = 0; - missingCount++; - zeroCount++; - } else { - v = nv.longValue(); - if (v == 0) { - zeroCount++; - } - } - - 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 (count != 0) { // minValue needs to be set first - gcd = MathUtil.gcd(gcd, v - minValue); - } - } - - minValue = Math.min(minValue, v); - maxValue = Math.max(maxValue, v); - - if (uniqueValues != null) { - if (uniqueValues.add(v)) { - if (uniqueValues.size() > 256) { - uniqueValues = null; - } - } - } - - ++count; - } - } else { - for (Number nv : values) { - long v = nv.longValue(); - if (v == -1L) { - missingOrdCount++; - } - minValue = Math.min(minValue, v); - maxValue = Math.max(maxValue, v); - ++count; - } - } - - final long delta = maxValue - minValue; - final int deltaBitsRequired = DirectWriter.unsignedBitsRequired(delta); - final int tableBitsRequired = uniqueValues == null - ? Integer.MAX_VALUE - : DirectWriter.bitsRequired(uniqueValues.size() - 1); - - final boolean sparse; // 1% of docs or less have a value - switch (numberType) { - case VALUE: - sparse = (double) missingCount / count >= 0.99; - break; - case ORDINAL: - sparse = (double) missingOrdCount / count >= 0.99; - break; - default: - throw new AssertionError(); - } - - final int format; - if (uniqueValues != null - && count <= Integer.MAX_VALUE - && (uniqueValues.size() == 1 - || (uniqueValues.size() == 2 && missingCount > 0 && zeroCount == missingCount))) { - // either one unique value C or two unique values: "missing" and C - format = CONST_COMPRESSED; - } else if (sparse && count >= 1024) { - // require at least 1024 docs to avoid flipping back and forth when doing NRT search - format = SPARSE_COMPRESSED; - } else if (uniqueValues != null && tableBitsRequired < deltaBitsRequired) { - format = TABLE_COMPRESSED; - } else if (gcd != 0 && gcd != 1) { - final long gcdDelta = (maxValue - minValue) / gcd; - final long gcdBitsRequired = DirectWriter.unsignedBitsRequired(gcdDelta); - format = gcdBitsRequired < deltaBitsRequired ? GCD_COMPRESSED : DELTA_COMPRESSED; - } else { - format = DELTA_COMPRESSED; - } - meta.writeVInt(field.number); - meta.writeByte(Lucene54DocValuesFormat.NUMERIC); - meta.writeVInt(format); - if (format == SPARSE_COMPRESSED) { - meta.writeLong(data.getFilePointer()); - final long numDocsWithValue; - switch (numberType) { - case VALUE: - numDocsWithValue = count - missingCount; - break; - case ORDINAL: - numDocsWithValue = count - missingOrdCount; - break; - default: - throw new AssertionError(); - } - final long maxDoc = writeSparseMissingBitset(values, numberType, numDocsWithValue); - assert maxDoc == count; - } else if (missingCount == 0) { - meta.writeLong(ALL_LIVE); - } else if (missingCount == count) { - meta.writeLong(ALL_MISSING); - } else { - meta.writeLong(data.getFilePointer()); - writeMissingBitset(values); - } - meta.writeLong(data.getFilePointer()); - meta.writeVLong(count); - - switch (format) { - case CONST_COMPRESSED: - // write the constant (nonzero value in the n=2 case, singleton value otherwise) - meta.writeLong(minValue < 0 ? Collections.min(uniqueValues) : Collections.max(uniqueValues)); - break; - case GCD_COMPRESSED: - meta.writeLong(minValue); - meta.writeLong(gcd); - final long maxDelta = (maxValue - minValue) / gcd; - final int bits = DirectWriter.unsignedBitsRequired(maxDelta); - meta.writeVInt(bits); - final DirectWriter quotientWriter = DirectWriter.getInstance(data, count, bits); - for (Number nv : values) { - long value = nv == null ? 0 : nv.longValue(); - quotientWriter.add((value - minValue) / gcd); - } - quotientWriter.finish(); - break; - case DELTA_COMPRESSED: - final long minDelta = delta < 0 ? 0 : minValue; - meta.writeLong(minDelta); - meta.writeVInt(deltaBitsRequired); - final DirectWriter writer = DirectWriter.getInstance(data, count, deltaBitsRequired); - for (Number nv : values) { - long v = nv == null ? 0 : nv.longValue(); - writer.add(v - minDelta); - } - writer.finish(); - break; - case TABLE_COMPRESSED: - final Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]); - Arrays.sort(decode); - final HashMap encode = new HashMap<>(); - meta.writeVInt(decode.length); - for (int i = 0; i < decode.length; i++) { - meta.writeLong(decode[i]); - encode.put(decode[i], i); - } - meta.writeVInt(tableBitsRequired); - final DirectWriter ordsWriter = DirectWriter.getInstance(data, count, tableBitsRequired); - for (Number nv : values) { - ordsWriter.add(encode.get(nv == null ? 0 : nv.longValue())); - } - ordsWriter.finish(); - break; - case SPARSE_COMPRESSED: - final Iterable filteredMissingValues; - switch (numberType) { - case VALUE: - meta.writeByte((byte) 0); - filteredMissingValues = new Iterable() { - @Override - public Iterator iterator() { - return StreamSupport - .stream(values.spliterator(), false) - .filter(value -> value != null) - .iterator(); - } - }; - break; - case ORDINAL: - meta.writeByte((byte) 1); - filteredMissingValues = new Iterable() { - @Override - public Iterator iterator() { - return StreamSupport - .stream(values.spliterator(), false) - .filter(value -> value.longValue() != -1L) - .iterator(); - } - }; - break; - default: - throw new AssertionError(); - } - // Write non-missing values as a numeric field - addNumericField(field, filteredMissingValues, numberType); - break; - default: - throw new AssertionError(); - } - meta.writeLong(data.getFilePointer()); - } - - // TODO: in some cases representing missing with minValue-1 wouldn't take up additional space and so on, - // but this is very simple, and algorithms only check this for values of 0 anyway (doesnt slow down normal decode) - void writeMissingBitset(Iterable values) throws IOException { - byte bits = 0; - int count = 0; - for (Object v : values) { - if (count == 8) { - data.writeByte(bits); - count = 0; - bits = 0; - } - if (v != null) { - bits |= 1 << (count & 7); - } - count++; - } - if (count > 0) { - data.writeByte(bits); - } - } - - long writeSparseMissingBitset(Iterable values, NumberType numberType, long numDocsWithValue) throws IOException { - meta.writeVLong(numDocsWithValue); - - // Write doc IDs that have a value - meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT); - final DirectMonotonicWriter docIdsWriter = DirectMonotonicWriter.getInstance(meta, data, numDocsWithValue, DIRECT_MONOTONIC_BLOCK_SHIFT); - long docID = 0; - for (Number nv : values) { - switch (numberType) { - case VALUE: - if (nv != null) { - docIdsWriter.add(docID); - } - break; - case ORDINAL: - if (nv.longValue() != -1L) { - docIdsWriter.add(docID); - } - break; - default: - throw new AssertionError(); - } - docID++; - } - docIdsWriter.finish(); - return docID; - } - - @Override - public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { - addBinaryField(field, LegacyDocValuesIterables.binaryIterable(field, valuesProducer, maxDoc)); - } - - private void addBinaryField(FieldInfo field, Iterable values) throws IOException { - // write the byte[] data - meta.writeVInt(field.number); - meta.writeByte(Lucene54DocValuesFormat.BINARY); - int minLength = Integer.MAX_VALUE; - int maxLength = Integer.MIN_VALUE; - final long startFP = data.getFilePointer(); - long count = 0; - long missingCount = 0; - for(BytesRef v : values) { - final int length; - if (v == null) { - length = 0; - missingCount++; - } else { - length = v.length; - } - minLength = Math.min(minLength, length); - maxLength = Math.max(maxLength, length); - if (v != null) { - data.writeBytes(v.bytes, v.offset, v.length); - } - count++; - } - meta.writeVInt(minLength == maxLength ? BINARY_FIXED_UNCOMPRESSED : BINARY_VARIABLE_UNCOMPRESSED); - if (missingCount == 0) { - meta.writeLong(ALL_LIVE); - } else if (missingCount == count) { - meta.writeLong(ALL_MISSING); - } else { - meta.writeLong(data.getFilePointer()); - writeMissingBitset(values); - } - meta.writeVInt(minLength); - meta.writeVInt(maxLength); - meta.writeVLong(count); - meta.writeLong(startFP); - - // if minLength == maxLength, it's a fixed-length byte[], we are done (the addresses are implicit) - // otherwise, we need to record the length fields... - if (minLength != maxLength) { - meta.writeLong(data.getFilePointer()); - meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT); - - final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, count + 1, DIRECT_MONOTONIC_BLOCK_SHIFT); - long addr = 0; - writer.add(addr); - for (BytesRef v : values) { - if (v != null) { - addr += v.length; - } - writer.add(addr); - } - writer.finish(); - meta.writeLong(data.getFilePointer()); - } - } - - /** expert: writes a value dictionary for a sorted/sortedset field */ - private void addTermsDict(FieldInfo field, final Iterable values) throws IOException { - // first check if it's a "fixed-length" terms dict, and compressibility if so - int minLength = Integer.MAX_VALUE; - int maxLength = Integer.MIN_VALUE; - long numValues = 0; - BytesRefBuilder previousValue = new BytesRefBuilder(); - long prefixSum = 0; // only valid for fixed-width data, as we have a choice there - for (BytesRef v : values) { - minLength = Math.min(minLength, v.length); - maxLength = Math.max(maxLength, v.length); - if (minLength == maxLength) { - int termPosition = (int) (numValues & INTERVAL_MASK); - if (termPosition == 0) { - // first term in block, save it away to compare against the last term later - previousValue.copyBytes(v); - } else if (termPosition == INTERVAL_COUNT - 1) { - // last term in block, accumulate shared prefix against first term - prefixSum += StringHelper.bytesDifference(previousValue.get(), v); - } - } - numValues++; - } - // for fixed width data, look at the avg(shared prefix) before deciding how to encode: - // prefix compression "costs" worst case 2 bytes per term because we must store suffix lengths. - // so if we share at least 3 bytes on average, always compress. - if (minLength == maxLength && prefixSum <= 3*(numValues >> INTERVAL_SHIFT)) { - // no index needed: not very compressible, direct addressing by mult - addBinaryField(field, values); - } else if (numValues < REVERSE_INTERVAL_COUNT) { - // low cardinality: waste a few KB of ram, but can't really use fancy index etc - addBinaryField(field, values); - } else { - assert numValues > 0; // we don't have to handle the empty case - // header - meta.writeVInt(field.number); - meta.writeByte(Lucene54DocValuesFormat.BINARY); - meta.writeVInt(BINARY_PREFIX_COMPRESSED); - meta.writeLong(-1L); - // now write the bytes: sharing prefixes within a block - final long startFP = data.getFilePointer(); - // currently, we have to store the delta from expected for every 1/nth term - // we could avoid this, but it's not much and less overall RAM than the previous approach! - RAMOutputStream addressBuffer = new RAMOutputStream(); - MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, MONOTONIC_BLOCK_SIZE); - // buffers up 16 terms - RAMOutputStream bytesBuffer = new RAMOutputStream(); - // buffers up block header - RAMOutputStream headerBuffer = new RAMOutputStream(); - BytesRefBuilder lastTerm = new BytesRefBuilder(); - lastTerm.grow(maxLength); - long count = 0; - int suffixDeltas[] = new int[INTERVAL_COUNT]; - for (BytesRef v : values) { - int termPosition = (int) (count & INTERVAL_MASK); - if (termPosition == 0) { - termAddresses.add(data.getFilePointer() - startFP); - // abs-encode first term - headerBuffer.writeVInt(v.length); - headerBuffer.writeBytes(v.bytes, v.offset, v.length); - lastTerm.copyBytes(v); - } else { - // prefix-code: we only share at most 255 characters, to encode the length as a single - // byte and have random access. Larger terms just get less compression. - int sharedPrefix = Math.min(255, StringHelper.bytesDifference(lastTerm.get(), v)); - bytesBuffer.writeByte((byte) sharedPrefix); - bytesBuffer.writeBytes(v.bytes, v.offset + sharedPrefix, v.length - sharedPrefix); - // we can encode one smaller, because terms are unique. - suffixDeltas[termPosition] = v.length - sharedPrefix - 1; - } - - count++; - // flush block - if ((count & INTERVAL_MASK) == 0) { - flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas); - } - } - // flush trailing crap - int leftover = (int) (count & INTERVAL_MASK); - if (leftover > 0) { - Arrays.fill(suffixDeltas, leftover, suffixDeltas.length, 0); - flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas); - } - final long indexStartFP = data.getFilePointer(); - // write addresses of indexed terms - termAddresses.finish(); - addressBuffer.writeTo(data); - addressBuffer = null; - termAddresses = null; - meta.writeVInt(minLength); - meta.writeVInt(maxLength); - meta.writeVLong(count); - meta.writeLong(startFP); - meta.writeLong(indexStartFP); - meta.writeVInt(PackedInts.VERSION_CURRENT); - meta.writeVInt(MONOTONIC_BLOCK_SIZE); - addReverseTermIndex(field, values, maxLength); - } - } - // writes term dictionary "block" - // first term is absolute encoded as vint length + bytes. - // lengths of subsequent N terms are encoded as either N bytes or N shorts. - // in the double-byte case, the first byte is indicated with -1. - // subsequent terms are encoded as byte suffixLength + bytes. - private void flushTermsDictBlock(RAMOutputStream headerBuffer, RAMOutputStream bytesBuffer, int suffixDeltas[]) throws IOException { - boolean twoByte = false; - for (int i = 1; i < suffixDeltas.length; i++) { - if (suffixDeltas[i] > 254) { - twoByte = true; - } - } - if (twoByte) { - headerBuffer.writeByte((byte)255); - for (int i = 1; i < suffixDeltas.length; i++) { - headerBuffer.writeShort((short) suffixDeltas[i]); - } - } else { - for (int i = 1; i < suffixDeltas.length; i++) { - headerBuffer.writeByte((byte) suffixDeltas[i]); - } - } - headerBuffer.writeTo(data); - headerBuffer.reset(); - bytesBuffer.writeTo(data); - bytesBuffer.reset(); - } - - // writes reverse term index: used for binary searching a term into a range of 64 blocks - // for every 64 blocks (1024 terms) we store a term, trimming any suffix unnecessary for comparison - // terms are written as a contiguous byte[], but never spanning 2^15 byte boundaries. - private void addReverseTermIndex(FieldInfo field, final Iterable values, int maxLength) throws IOException { - long count = 0; - BytesRefBuilder priorTerm = new BytesRefBuilder(); - priorTerm.grow(maxLength); - BytesRef indexTerm = new BytesRef(); - long startFP = data.getFilePointer(); - PagedBytes pagedBytes = new PagedBytes(15); - MonotonicBlockPackedWriter addresses = new MonotonicBlockPackedWriter(data, MONOTONIC_BLOCK_SIZE); - - for (BytesRef b : values) { - int termPosition = (int) (count & REVERSE_INTERVAL_MASK); - if (termPosition == 0) { - int len = StringHelper.sortKeyLength(priorTerm.get(), b); - indexTerm.bytes = b.bytes; - indexTerm.offset = b.offset; - indexTerm.length = len; - addresses.add(pagedBytes.copyUsingLengthPrefix(indexTerm)); - } else if (termPosition == REVERSE_INTERVAL_MASK) { - priorTerm.copyBytes(b); - } - count++; - } - addresses.finish(); - long numBytes = pagedBytes.getPointer(); - pagedBytes.freeze(true); - PagedBytesDataInput in = pagedBytes.getDataInput(); - meta.writeLong(startFP); - data.writeVLong(numBytes); - data.copyBytes(in, numBytes); - } - - @Override - public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { - meta.writeVInt(field.number); - meta.writeByte(Lucene54DocValuesFormat.SORTED); - addTermsDict(field, LegacyDocValuesIterables.valuesIterable(valuesProducer.getSorted(field))); - addNumericField(field, LegacyDocValuesIterables.sortedOrdIterable(valuesProducer, field, maxDoc), NumberType.ORDINAL); - } - - private void addSortedField(FieldInfo field, Iterable values, Iterable ords) throws IOException { - meta.writeVInt(field.number); - meta.writeByte(Lucene54DocValuesFormat.SORTED); - addTermsDict(field, values); - addNumericField(field, ords, NumberType.ORDINAL); - } - - @Override - public void addSortedNumericField(FieldInfo field, final DocValuesProducer valuesProducer) throws IOException { - - final Iterable docToValueCount = LegacyDocValuesIterables.sortedNumericToDocCount(valuesProducer, field, maxDoc); - final Iterable values = LegacyDocValuesIterables.sortedNumericToValues(valuesProducer, field); - - meta.writeVInt(field.number); - meta.writeByte(Lucene54DocValuesFormat.SORTED_NUMERIC); - if (isSingleValued(docToValueCount)) { - meta.writeVInt(SORTED_SINGLE_VALUED); - // The field is single-valued, we can encode it as NUMERIC - addNumericField(field, singletonView(docToValueCount, values, null), NumberType.VALUE); - } else { - final SortedSet uniqueValueSets = uniqueValueSets(docToValueCount, values); - if (uniqueValueSets != null) { - meta.writeVInt(SORTED_SET_TABLE); - - // write the set_id -> values mapping - writeDictionary(uniqueValueSets); - - // write the doc -> set_id as a numeric field - addNumericField(field, docToSetId(uniqueValueSets, docToValueCount, values), NumberType.ORDINAL); - } else { - meta.writeVInt(SORTED_WITH_ADDRESSES); - // write the stream of values as a numeric field - addNumericField(field, values, NumberType.VALUE); - // write the doc -> ord count as a absolute index to the stream - addOrdIndex(field, docToValueCount); - } - } - } - - @Override - public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { - - Iterable values = LegacyDocValuesIterables.valuesIterable(valuesProducer.getSortedSet(field)); - Iterable docToOrdCount = LegacyDocValuesIterables.sortedSetOrdCountIterable(valuesProducer, field, maxDoc); - Iterable ords = LegacyDocValuesIterables.sortedSetOrdsIterable(valuesProducer, field); - - meta.writeVInt(field.number); - meta.writeByte(Lucene54DocValuesFormat.SORTED_SET); - - if (isSingleValued(docToOrdCount)) { - meta.writeVInt(SORTED_SINGLE_VALUED); - // The field is single-valued, we can encode it as SORTED - addSortedField(field, values, singletonView(docToOrdCount, ords, -1L)); - } else { - final SortedSet uniqueValueSets = uniqueValueSets(docToOrdCount, ords); - if (uniqueValueSets != null) { - meta.writeVInt(SORTED_SET_TABLE); - - // write the set_id -> ords mapping - writeDictionary(uniqueValueSets); - - // write the ord -> byte[] as a binary field - addTermsDict(field, values); - - // write the doc -> set_id as a numeric field - addNumericField(field, docToSetId(uniqueValueSets, docToOrdCount, ords), NumberType.ORDINAL); - } else { - meta.writeVInt(SORTED_WITH_ADDRESSES); - - // write the ord -> byte[] as a binary field - addTermsDict(field, values); - - // write the stream of ords as a numeric field - // NOTE: we could return an iterator that delta-encodes these within a doc - addNumericField(field, ords, NumberType.ORDINAL); - - // write the doc -> ord count as a absolute index to the stream - addOrdIndex(field, docToOrdCount); - } - } - } - - private SortedSet uniqueValueSets(Iterable docToValueCount, Iterable values) { - Set uniqueValueSet = new HashSet<>(); - LongsRef docValues = new LongsRef(256); - - Iterator valueCountIterator = docToValueCount.iterator(); - Iterator valueIterator = values.iterator(); - int totalDictSize = 0; - while (valueCountIterator.hasNext()) { - docValues.length = valueCountIterator.next().intValue(); - if (docValues.length > 256) { - return null; - } - for (int i = 0; i < docValues.length; ++i) { - docValues.longs[i] = valueIterator.next().longValue(); - } - if (uniqueValueSet.contains(docValues)) { - continue; - } - totalDictSize += docValues.length; - if (totalDictSize > 256) { - return null; - } - uniqueValueSet.add(new LongsRef(Arrays.copyOf(docValues.longs, docValues.length), 0, docValues.length)); - } - assert valueIterator.hasNext() == false; - return new TreeSet<>(uniqueValueSet); - } - - private void writeDictionary(SortedSet uniqueValueSets) throws IOException { - int lengthSum = 0; - for (LongsRef longs : uniqueValueSets) { - lengthSum += longs.length; - } - - meta.writeInt(lengthSum); - for (LongsRef valueSet : uniqueValueSets) { - for (int i = 0; i < valueSet.length; ++i) { - meta.writeLong(valueSet.longs[valueSet.offset + i]); - } - } - - meta.writeInt(uniqueValueSets.size()); - for (LongsRef valueSet : uniqueValueSets) { - meta.writeInt(valueSet.length); - } - } - - private Iterable docToSetId(SortedSet uniqueValueSets, Iterable docToValueCount, Iterable values) { - final Map setIds = new HashMap<>(); - int i = 0; - for (LongsRef set : uniqueValueSets) { - setIds.put(set, i++); - } - assert i == uniqueValueSets.size(); - - return new Iterable() { - - @Override - public Iterator iterator() { - final Iterator valueCountIterator = docToValueCount.iterator(); - final Iterator valueIterator = values.iterator(); - final LongsRef docValues = new LongsRef(256); - return new Iterator() { - - @Override - public boolean hasNext() { - return valueCountIterator.hasNext(); - } - - @Override - public Number next() { - docValues.length = valueCountIterator.next().intValue(); - for (int i = 0; i < docValues.length; ++i) { - docValues.longs[i] = valueIterator.next().longValue(); - } - final Integer id = setIds.get(docValues); - assert id != null; - return id; - } - - }; - - } - }; - } - - // writes addressing information as MONOTONIC_COMPRESSED integer - private void addOrdIndex(FieldInfo field, Iterable values) throws IOException { - meta.writeVInt(field.number); - meta.writeByte(Lucene54DocValuesFormat.NUMERIC); - meta.writeVInt(MONOTONIC_COMPRESSED); - meta.writeLong(-1L); - meta.writeLong(data.getFilePointer()); - meta.writeVLong(maxDoc); - meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT); - - final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, maxDoc + 1, DIRECT_MONOTONIC_BLOCK_SHIFT); - long addr = 0; - writer.add(addr); - for (Number v : values) { - addr += v.longValue(); - writer.add(addr); - } - writer.finish(); - meta.writeLong(data.getFilePointer()); - } - - @Override - public void close() throws IOException { - boolean success = false; - try { - if (meta != null) { - meta.writeVInt(-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; - } - } -} diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java deleted file mode 100644 index 91ccfe26f7f..00000000000 --- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java +++ /dev/null @@ -1,186 +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.codecs.lucene54; - - -import java.io.IOException; - -import org.apache.lucene.codecs.DocValuesConsumer; -import org.apache.lucene.codecs.DocValuesFormat; -import org.apache.lucene.codecs.DocValuesProducer; -import org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat; -import org.apache.lucene.index.DocValuesType; -import org.apache.lucene.index.SegmentReadState; -import org.apache.lucene.index.SegmentWriteState; -import org.apache.lucene.util.SmallFloat; -import org.apache.lucene.util.packed.DirectWriter; - -/** - * Lucene 5.4 DocValues format. - *

- * Encodes the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) with these 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 DirectWriter}. - *
  • 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 DirectWriter}). - *
  • 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 non-missing value, only the missing - * bitset is encoded. - *
  • Sparse-compressed: only documents with a value are stored, and lookups are performed - * using binary search. - *
- *

- * {@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 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 table: when there are few unique sets of values (< 256) then each set is assigned - * an id, a lookup table is written and the mapping from document to set id is written using the - * numeric strategies above. - *
  • 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. - *
  • SortedSet table: when there are few unique sets of values (< 256) then each set is assigned - * an id, a lookup table is written and the mapping from document to set id is written using the - * numeric strategies above. - *
  • SortedNumeric: a value list and per-document index into this list are written using the numeric - * strategies above. - *
- *

- * Files: - *

    - *
  1. .dvd: DocValues data
  2. - *
  3. .dvm: DocValues metadata
  4. - *
- * @lucene.experimental - * @deprecated Use {@link Lucene70DocValuesFormat}. - */ -@Deprecated -public final class Lucene54DocValuesFormat extends DocValuesFormat { - - /** Sole Constructor */ - public Lucene54DocValuesFormat() { - super("Lucene54"); - } - - @Override - public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - return new Lucene54DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION); - } - - @Override - public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException { - return new Lucene54DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION); - } - - static final String DATA_CODEC = "Lucene54DocValuesData"; - static final String DATA_EXTENSION = "dvd"; - static final String META_CODEC = "Lucene54DocValuesMetadata"; - 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; - - // address terms in blocks of 16 terms - static final int INTERVAL_SHIFT = 4; - static final int INTERVAL_COUNT = 1 << INTERVAL_SHIFT; - static final int INTERVAL_MASK = INTERVAL_COUNT - 1; - - // build reverse index from every 1024th term - static final int REVERSE_INTERVAL_SHIFT = 10; - static final int REVERSE_INTERVAL_COUNT = 1 << REVERSE_INTERVAL_SHIFT; - static final int REVERSE_INTERVAL_MASK = REVERSE_INTERVAL_COUNT - 1; - - // for conversion from reverse index to block - static final int BLOCK_INTERVAL_SHIFT = REVERSE_INTERVAL_SHIFT - INTERVAL_SHIFT; - static final int BLOCK_INTERVAL_COUNT = 1 << BLOCK_INTERVAL_SHIFT; - static final int BLOCK_INTERVAL_MASK = BLOCK_INTERVAL_COUNT - 1; - - /** Compressed using packed blocks of ints. */ - static final int DELTA_COMPRESSED = 0; - /** Compressed by computing the GCD. */ - static final int GCD_COMPRESSED = 1; - /** Compressed by giving IDs to unique values. */ - static final int TABLE_COMPRESSED = 2; - /** Compressed with monotonically increasing values */ - static final int MONOTONIC_COMPRESSED = 3; - /** Compressed with constant value (uses only missing bitset) */ - static final int CONST_COMPRESSED = 4; - /** Compressed with sparse arrays. */ - static final int SPARSE_COMPRESSED = 5; - - /** Uncompressed binary, written directly (fixed length). */ - static final int BINARY_FIXED_UNCOMPRESSED = 0; - /** Uncompressed binary, written directly (variable length). */ - static final int BINARY_VARIABLE_UNCOMPRESSED = 1; - /** Compressed binary with shared prefixes */ - static final int BINARY_PREFIX_COMPRESSED = 2; - - /** Standard storage for sorted set values with 1 level of indirection: - * {@code docId -> address -> ord}. */ - static final int SORTED_WITH_ADDRESSES = 0; - /** Single-valued sorted set values, encoded as sorted values, so no level - * of indirection: {@code docId -> ord}. */ - static final int SORTED_SINGLE_VALUED = 1; - /** Compressed giving IDs to unique sets of values: - * {@code docId -> setId -> ords} */ - static final int SORTED_SET_TABLE = 2; - - /** placeholder for missing offset that means there are no missing values */ - static final int ALL_LIVE = -1; - /** placeholder for missing offset that means all values are missing */ - static final int ALL_MISSING = -2; - - // addressing uses 16k blocks - static final int MONOTONIC_BLOCK_SIZE = 16384; - static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16; -} diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java deleted file mode 100644 index a35f503cb98..00000000000 --- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java +++ /dev/null @@ -1,1803 +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.codecs.lucene54; - -import java.io.Closeable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.codecs.DocValuesProducer; -import org.apache.lucene.codecs.lucene54.Lucene54DocValuesConsumer.NumberType; -import org.apache.lucene.index.*; -import org.apache.lucene.index.NumericDocValues; -import org.apache.lucene.store.ChecksumIndexInput; -import org.apache.lucene.store.IndexInput; -import org.apache.lucene.store.RandomAccessInput; -import org.apache.lucene.util.Accountable; -import org.apache.lucene.util.Accountables; -import org.apache.lucene.util.Bits; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.IOUtils; -import org.apache.lucene.util.LongValues; -import org.apache.lucene.util.PagedBytes; -import org.apache.lucene.util.RamUsageEstimator; -import org.apache.lucene.util.packed.DirectMonotonicReader; -import org.apache.lucene.util.packed.DirectReader; -import org.apache.lucene.util.packed.MonotonicBlockPackedReader; - -import static org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat.*; - -/** reader for {@link Lucene54DocValuesFormat} */ -final class Lucene54DocValuesProducer extends DocValuesProducer implements Closeable { - private final Map numerics = new HashMap<>(); - private final Map binaries = new HashMap<>(); - private final Map sortedSets = new HashMap<>(); - private final Map sortedNumerics = new HashMap<>(); - private final Map ords = new HashMap<>(); - private final Map ordIndexes = new HashMap<>(); - private final int numFields; - private final AtomicLong ramBytesUsed; - private final IndexInput data; - private final int maxDoc; - - // memory-resident structures - private final Map addressInstances = new HashMap<>(); - private final Map reverseIndexInstances = new HashMap<>(); - private final Map directAddressesMeta = new HashMap<>(); - - private final boolean merging; - - // clone for merge: when merging we don't do any instances.put()s - Lucene54DocValuesProducer(Lucene54DocValuesProducer original) throws IOException { - assert Thread.holdsLock(original); - numerics.putAll(original.numerics); - binaries.putAll(original.binaries); - sortedSets.putAll(original.sortedSets); - sortedNumerics.putAll(original.sortedNumerics); - ords.putAll(original.ords); - ordIndexes.putAll(original.ordIndexes); - numFields = original.numFields; - ramBytesUsed = new AtomicLong(original.ramBytesUsed.get()); - data = original.data.clone(); - maxDoc = original.maxDoc; - - addressInstances.putAll(original.addressInstances); - reverseIndexInstances.putAll(original.reverseIndexInstances); - merging = true; - } - - /** expert: instantiates a new reader */ - Lucene54DocValuesProducer(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(); - merging = false; - ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass())); - - int version = -1; - int numFields = -1; - - // read in the entries from the metadata file. - try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) { - Throwable priorE = null; - try { - version = CodecUtil.checkIndexHeader(in, metaCodec, - Lucene54DocValuesFormat.VERSION_START, - Lucene54DocValuesFormat.VERSION_CURRENT, - state.segmentInfo.getId(), - state.segmentSuffix); - numFields = readFields(in, state.fieldInfos); - } catch (Throwable exception) { - priorE = exception; - } finally { - CodecUtil.checkFooter(in, priorE); - } - } - - this.numFields = numFields; - String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension); - this.data = state.directory.openInput(dataName, state.context); - boolean success = false; - try { - final int version2 = CodecUtil.checkIndexHeader(data, dataCodec, - Lucene54DocValuesFormat.VERSION_START, - Lucene54DocValuesFormat.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 readSortedField(FieldInfo info, IndexInput meta) throws IOException { - // sorted = binary + numeric - if (meta.readVInt() != info.number) { - throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta); - } - if (meta.readByte() != Lucene54DocValuesFormat.BINARY) { - throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta); - } - BinaryEntry b = readBinaryEntry(info, meta); - binaries.put(info.name, b); - - if (meta.readVInt() != info.number) { - throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta); - } - if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) { - throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta); - } - NumericEntry n = readNumericEntry(info, meta); - ords.put(info.name, n); - } - - private void readSortedSetFieldWithAddresses(FieldInfo info, IndexInput meta) throws IOException { - // sortedset = binary + numeric (addresses) + ordIndex - if (meta.readVInt() != info.number) { - throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta); - } - if (meta.readByte() != Lucene54DocValuesFormat.BINARY) { - throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta); - } - BinaryEntry b = readBinaryEntry(info, meta); - binaries.put(info.name, b); - - if (meta.readVInt() != info.number) { - throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta); - } - if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) { - throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta); - } - NumericEntry n1 = readNumericEntry(info, meta); - ords.put(info.name, n1); - - if (meta.readVInt() != info.number) { - throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta); - } - if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) { - throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta); - } - NumericEntry n2 = readNumericEntry(info, meta); - ordIndexes.put(info.name, n2); - } - - private void readSortedSetFieldWithTable(FieldInfo info, IndexInput meta) throws IOException { - // sortedset table = binary + ordset table + ordset index - if (meta.readVInt() != info.number) { - throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta); - } - if (meta.readByte() != Lucene54DocValuesFormat.BINARY) { - throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta); - } - - BinaryEntry b = readBinaryEntry(info, meta); - binaries.put(info.name, b); - - if (meta.readVInt() != info.number) { - throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta); - } - if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) { - throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta); - } - NumericEntry n = readNumericEntry(info, meta); - ords.put(info.name, n); - } - - private int readFields(IndexInput meta, FieldInfos infos) throws IOException { - int numFields = 0; - int fieldNumber = meta.readVInt(); - while (fieldNumber != -1) { - numFields++; - FieldInfo info = infos.fieldInfo(fieldNumber); - if (info == null) { - // trickier to validate more: because we use multiple entries for "composite" types like sortedset, etc. - throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta); - } - byte type = meta.readByte(); - if (type == Lucene54DocValuesFormat.NUMERIC) { - numerics.put(info.name, readNumericEntry(info, meta)); - } else if (type == Lucene54DocValuesFormat.BINARY) { - BinaryEntry b = readBinaryEntry(info, meta); - binaries.put(info.name, b); - } else if (type == Lucene54DocValuesFormat.SORTED) { - readSortedField(info, meta); - } else if (type == Lucene54DocValuesFormat.SORTED_SET) { - SortedSetEntry ss = readSortedSetEntry(meta); - sortedSets.put(info.name, ss); - if (ss.format == SORTED_WITH_ADDRESSES) { - readSortedSetFieldWithAddresses(info, meta); - } else if (ss.format == SORTED_SET_TABLE) { - readSortedSetFieldWithTable(info, meta); - } else if (ss.format == SORTED_SINGLE_VALUED) { - if (meta.readVInt() != fieldNumber) { - throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta); - } - if (meta.readByte() != Lucene54DocValuesFormat.SORTED) { - throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta); - } - readSortedField(info, meta); - } else { - throw new AssertionError(); - } - } else if (type == Lucene54DocValuesFormat.SORTED_NUMERIC) { - SortedSetEntry ss = readSortedSetEntry(meta); - sortedNumerics.put(info.name, ss); - if (ss.format == SORTED_WITH_ADDRESSES) { - if (meta.readVInt() != fieldNumber) { - throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta); - } - if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) { - throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta); - } - numerics.put(info.name, readNumericEntry(info, meta)); - if (meta.readVInt() != fieldNumber) { - throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta); - } - if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) { - throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta); - } - NumericEntry ordIndex = readNumericEntry(info, meta); - ordIndexes.put(info.name, ordIndex); - } else if (ss.format == SORTED_SET_TABLE) { - if (meta.readVInt() != info.number) { - throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta); - } - if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) { - throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta); - } - NumericEntry n = readNumericEntry(info, meta); - ords.put(info.name, n); - } else if (ss.format == SORTED_SINGLE_VALUED) { - if (meta.readVInt() != fieldNumber) { - throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta); - } - if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) { - throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta); - } - numerics.put(info.name, readNumericEntry(info, meta)); - } else { - throw new AssertionError(); - } - } else { - throw new CorruptIndexException("invalid type: " + type, meta); - } - fieldNumber = meta.readVInt(); - } - return numFields; - } - - private NumericEntry readNumericEntry(FieldInfo info, IndexInput meta) throws IOException { - NumericEntry entry = new NumericEntry(); - entry.format = meta.readVInt(); - entry.missingOffset = meta.readLong(); - if (entry.format == SPARSE_COMPRESSED) { - // sparse bits need a bit more metadata - entry.numDocsWithValue = meta.readVLong(); - final int blockShift = meta.readVInt(); - entry.monotonicMeta = DirectMonotonicReader.loadMeta(meta, entry.numDocsWithValue, blockShift); - ramBytesUsed.addAndGet(entry.monotonicMeta.ramBytesUsed()); - directAddressesMeta.put(info.name, entry.monotonicMeta); - } - entry.offset = meta.readLong(); - entry.count = meta.readVLong(); - switch(entry.format) { - case CONST_COMPRESSED: - entry.minValue = meta.readLong(); - if (entry.count > Integer.MAX_VALUE) { - // currently just a limitation e.g. of bits interface and so on. - throw new CorruptIndexException("illegal CONST_COMPRESSED count: " + entry.count, meta); - } - break; - case GCD_COMPRESSED: - entry.minValue = meta.readLong(); - entry.gcd = meta.readLong(); - entry.bitsPerValue = meta.readVInt(); - break; - case TABLE_COMPRESSED: - final int uniqueValues = meta.readVInt(); - if (uniqueValues > 256) { - throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, got=" + uniqueValues, meta); - } - entry.table = new long[uniqueValues]; - for (int i = 0; i < uniqueValues; ++i) { - entry.table[i] = meta.readLong(); - } - ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table)); - entry.bitsPerValue = meta.readVInt(); - break; - case DELTA_COMPRESSED: - entry.minValue = meta.readLong(); - entry.bitsPerValue = meta.readVInt(); - break; - case MONOTONIC_COMPRESSED: - final int blockShift = meta.readVInt(); - entry.monotonicMeta = DirectMonotonicReader.loadMeta(meta, maxDoc + 1, blockShift); - ramBytesUsed.addAndGet(entry.monotonicMeta.ramBytesUsed()); - directAddressesMeta.put(info.name, entry.monotonicMeta); - break; - case SPARSE_COMPRESSED: - final byte numberType = meta.readByte(); - switch (numberType) { - case 0: - entry.numberType = NumberType.VALUE; - break; - case 1: - entry.numberType = NumberType.ORDINAL; - break; - default: - throw new CorruptIndexException("Number type can only be 0 or 1, got=" + numberType, meta); - } - - // now read the numeric entry for non-missing values - final int fieldNumber = meta.readVInt(); - if (fieldNumber != info.number) { - throw new CorruptIndexException("Field numbers mistmatch: " + fieldNumber + " != " + info.number, meta); - } - final int dvFormat = meta.readByte(); - if (dvFormat != NUMERIC) { - throw new CorruptIndexException("Formats mistmatch: " + dvFormat + " != " + NUMERIC, meta); - } - entry.nonMissingValues = readNumericEntry(info, meta); - break; - default: - throw new CorruptIndexException("Unknown format: " + entry.format + ", input=", meta); - } - entry.endOffset = meta.readLong(); - return entry; - } - - private BinaryEntry readBinaryEntry(FieldInfo info, IndexInput meta) throws IOException { - BinaryEntry entry = new BinaryEntry(); - entry.format = meta.readVInt(); - entry.missingOffset = meta.readLong(); - entry.minLength = meta.readVInt(); - entry.maxLength = meta.readVInt(); - entry.count = meta.readVLong(); - entry.offset = meta.readLong(); - switch(entry.format) { - case BINARY_FIXED_UNCOMPRESSED: - break; - case BINARY_PREFIX_COMPRESSED: - entry.addressesOffset = meta.readLong(); - entry.packedIntsVersion = meta.readVInt(); - entry.blockSize = meta.readVInt(); - entry.reverseIndexOffset = meta.readLong(); - break; - case BINARY_VARIABLE_UNCOMPRESSED: - entry.addressesOffset = meta.readLong(); - final int blockShift = meta.readVInt(); - entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, entry.count + 1, blockShift); - ramBytesUsed.addAndGet(entry.addressesMeta.ramBytesUsed()); - directAddressesMeta.put(info.name, entry.addressesMeta); - entry.addressesEndOffset = meta.readLong(); - break; - default: - throw new CorruptIndexException("Unknown format: " + entry.format, meta); - } - return entry; - } - - SortedSetEntry readSortedSetEntry(IndexInput meta) throws IOException { - SortedSetEntry entry = new SortedSetEntry(); - entry.format = meta.readVInt(); - if (entry.format == SORTED_SET_TABLE) { - final int totalTableLength = meta.readInt(); - if (totalTableLength > 256) { - throw new CorruptIndexException("SORTED_SET_TABLE cannot have more than 256 values in its dictionary, got=" + totalTableLength, meta); - } - entry.table = new long[totalTableLength]; - for (int i = 0; i < totalTableLength; ++i) { - entry.table[i] = meta.readLong(); - } - ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table)); - final int tableSize = meta.readInt(); - if (tableSize > totalTableLength + 1) { // +1 because of the empty set - throw new CorruptIndexException("SORTED_SET_TABLE cannot have more set ids than ords in its dictionary, got " + totalTableLength + " ords and " + tableSize + " sets", meta); - } - entry.tableOffsets = new int[tableSize + 1]; - for (int i = 1; i < entry.tableOffsets.length; ++i) { - entry.tableOffsets[i] = entry.tableOffsets[i - 1] + meta.readInt(); - } - ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.tableOffsets)); - } else if (entry.format != SORTED_SINGLE_VALUED && entry.format != SORTED_WITH_ADDRESSES) { - throw new CorruptIndexException("Unknown format: " + entry.format, meta); - } - return entry; - } - - @Override - public NumericDocValues getNumeric(FieldInfo field) throws IOException { - NumericEntry entry = numerics.get(field.name); - Bits docsWithField; - - if (entry.format == SPARSE_COMPRESSED) { - return getSparseNumericDocValues(entry); - } else { - if (entry.missingOffset == ALL_MISSING) { - return DocValues.emptyNumeric(); - } else if (entry.missingOffset == ALL_LIVE) { - LongValues values = getNumeric(entry); - return new NumericDocValues() { - private int docID = -1; - - @Override - public int docID() { - return docID; - } - - @Override - public int nextDoc() { - docID++; - if (docID == maxDoc) { - docID = NO_MORE_DOCS; - } - return docID; - } - - @Override - public int advance(int target) { - if (target >= maxDoc) { - docID = NO_MORE_DOCS; - } else { - docID = target; - } - return docID; - } - - @Override - public boolean advanceExact(int target) throws IOException { - docID = target; - return true; - } - - @Override - public long cost() { - // TODO - return 0; - } - - @Override - public long longValue() { - return values.get(docID); - } - }; - } else { - docsWithField = getLiveBits(entry.missingOffset, maxDoc); - } - } - final LongValues values = getNumeric(entry); - return new NumericDocValues() { - - int doc = -1; - long value; - - @Override - public long longValue() throws IOException { - return value; - } - - @Override - public int docID() { - return doc; - } - - @Override - public int nextDoc() throws IOException { - return advance(doc + 1); - } - - @Override - public int advance(int target) throws IOException { - for (int doc = target; doc < maxDoc; ++doc) { - value = values.get(doc); - if (value != 0 || docsWithField.get(doc)) { - return this.doc = doc; - } - } - return doc = NO_MORE_DOCS; - } - - @Override - public boolean advanceExact(int target) throws IOException { - doc = target; - value = values.get(doc); - return value != 0 || docsWithField.get(doc); - } - - @Override - public long cost() { - return maxDoc; - } - - }; - } - - @Override - public long ramBytesUsed() { - return ramBytesUsed.get(); - } - - @Override - public synchronized Collection getChildResources() { - List resources = new ArrayList<>(); - resources.addAll(Accountables.namedAccountables("addresses field", addressInstances)); - resources.addAll(Accountables.namedAccountables("reverse index field", reverseIndexInstances)); - resources.addAll(Accountables.namedAccountables("direct addresses meta field", directAddressesMeta)); - return Collections.unmodifiableList(resources); - } - - @Override - public void checkIntegrity() throws IOException { - CodecUtil.checksumEntireFile(data); - } - - @Override - public String toString() { - return getClass().getSimpleName() + "(fields=" + numFields + ")"; - } - - LongValues getNumeric(NumericEntry entry) throws IOException { - switch (entry.format) { - case CONST_COMPRESSED: { - final long constant = entry.minValue; - final Bits live = getLiveBits(entry.missingOffset, (int)entry.count); - return new LongValues() { - @Override - public long get(long index) { - return live.get((int)index) ? constant : 0; - } - }; - } - case DELTA_COMPRESSED: { - RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset); - final long delta = entry.minValue; - final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue, 0); - return new LongValues() { - @Override - public long get(long id) { - return delta + values.get(id); - } - }; - } - case GCD_COMPRESSED: { - RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset); - final long min = entry.minValue; - final long mult = entry.gcd; - final LongValues quotientReader = DirectReader.getInstance(slice, entry.bitsPerValue, 0); - return new LongValues() { - @Override - public long get(long id) { - return min + mult * quotientReader.get(id); - } - }; - } - case TABLE_COMPRESSED: { - RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset); - final long table[] = entry.table; - final LongValues ords = DirectReader.getInstance(slice, entry.bitsPerValue, 0); - return new LongValues() { - @Override - public long get(long id) { - return table[(int) ords.get(id)]; - } - }; - } - case SPARSE_COMPRESSED: - final SparseNumericDocValues values = getSparseNumericDocValues(entry); - final long missingValue; - switch (entry.numberType) { - case ORDINAL: - missingValue = -1L; - break; - case VALUE: - missingValue = 0L; - break; - default: - throw new AssertionError(); - } - return new SparseNumericDocValuesRandomAccessWrapper(values, missingValue); - default: - throw new AssertionError(); - } - } - - static final class SparseNumericDocValues extends NumericDocValues { - - final int docIDsLength; - final LongValues docIds, values; - - int index, doc; - - SparseNumericDocValues(int docIDsLength, LongValues docIDs, LongValues values) { - this.docIDsLength = docIDsLength; - this.docIds = docIDs; - this.values = values; - reset(); - } - - void reset() { - index = -1; - doc = -1; - } - - @Override - public int docID() { - return doc; - } - - @Override - public int nextDoc() throws IOException { - if (index >= docIDsLength - 1) { - index = docIDsLength; - return doc = NO_MORE_DOCS; - } - return doc = (int) docIds.get(++index); - } - - @Override - public int advance(int target) throws IOException { - long loIndex = index; - long step = 1; - long hiIndex; - int hiDoc; - - // gallop forward by exponentially growing the interval - // in order to find an interval so that the target doc - // is in ]lo, hi]. Compared to a regular binary search, - // this optimizes the case that the caller performs many - // advance calls by small deltas - do { - hiIndex = index + step; - if (hiIndex >= docIDsLength) { - hiIndex = docIDsLength; - hiDoc = NO_MORE_DOCS; - break; - } - hiDoc = (int) docIds.get(hiIndex); - if (hiDoc >= target) { - break; - } - step <<= 1; - } while (true); - - // now binary search - while (loIndex + 1 < hiIndex) { - final long midIndex = (loIndex + 1 + hiIndex) >>> 1; - final int midDoc = (int) docIds.get(midIndex); - if (midDoc >= target) { - hiIndex = midIndex; - hiDoc = midDoc; - } else { - loIndex = midIndex; - } - } - - index = (int) hiIndex; - return doc = hiDoc; - } - - @Override - public boolean advanceExact(int target) throws IOException { - if (advance(target) == target) { - return true; - } - --index; - doc = target; - return index >= 0 && docIds.get(index) == target; - } - - @Override - public long longValue() { - assert index >= 0; - assert index < docIDsLength; - return values.get(index); - } - - @Override - public long cost() { - return docIDsLength; - } - } - - static class SparseNumericDocValuesRandomAccessWrapper extends LongValues { - - final SparseNumericDocValues values; - final long missingValue; - - SparseNumericDocValuesRandomAccessWrapper(SparseNumericDocValues values, long missingValue) { - this.values = values; - this.missingValue = missingValue; - } - - @Override - public long get(long longIndex) { - final int index = Math.toIntExact(longIndex); - int doc = values.docID(); - if (doc >= index) { - values.reset(); - } - assert values.docID() < index; - try { - doc = values.advance(index); - } catch (IOException e) { - throw new RuntimeException(e); - } - if (doc == index) { - return values.longValue(); - } else { - return missingValue; - } - } - - } - - LegacyBinaryDocValues getLegacyBinary(FieldInfo field) throws IOException { - BinaryEntry bytes = binaries.get(field.name); - switch(bytes.format) { - case BINARY_FIXED_UNCOMPRESSED: - return getFixedBinary(field, bytes); - case BINARY_VARIABLE_UNCOMPRESSED: - return getVariableBinary(field, bytes); - case BINARY_PREFIX_COMPRESSED: - return getCompressedBinary(field, bytes); - default: - throw new AssertionError(); - } - } - - @Override - public BinaryDocValues getBinary(FieldInfo field) throws IOException { - BinaryEntry be = binaries.get(field.name); - return new LegacyBinaryDocValuesWrapper(getLiveBits(be.missingOffset, maxDoc), getLegacyBinary(field)); - } - - private LegacyBinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException { - final IndexInput data = this.data.slice("fixed-binary", bytes.offset, bytes.count * bytes.maxLength); - - final BytesRef term = new BytesRef(bytes.maxLength); - final byte[] buffer = term.bytes; - final int length = term.length = bytes.maxLength; - - return new LongBinaryDocValues() { - @Override - public BytesRef get(long id) { - try { - data.seek(id * length); - data.readBytes(buffer, 0, buffer.length); - return term; - } catch (IOException e) { - throw new RuntimeException(e); - } - } - }; - } - - private LegacyBinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException { - final RandomAccessInput addressesData = this.data.randomAccessSlice(bytes.addressesOffset, bytes.addressesEndOffset - bytes.addressesOffset); - final LongValues addresses = DirectMonotonicReader.getInstance(bytes.addressesMeta, addressesData); - - final IndexInput data = this.data.slice("var-binary", bytes.offset, bytes.addressesOffset - bytes.offset); - final BytesRef term = new BytesRef(Math.max(0, bytes.maxLength)); - final byte buffer[] = term.bytes; - - return new LongBinaryDocValues() { - @Override - public BytesRef get(long id) { - long startAddress = addresses.get(id); - long endAddress = addresses.get(id+1); - int length = (int) (endAddress - startAddress); - try { - data.seek(startAddress); - data.readBytes(buffer, 0, length); - term.length = length; - return term; - } catch (IOException e) { - throw new RuntimeException(e); - } - } - }; - } - - /** returns an address instance for prefix-compressed binary values. */ - private synchronized MonotonicBlockPackedReader getIntervalInstance(FieldInfo field, BinaryEntry bytes) throws IOException { - MonotonicBlockPackedReader addresses = addressInstances.get(field.name); - if (addresses == null) { - data.seek(bytes.addressesOffset); - final long size = (bytes.count + INTERVAL_MASK) >>> INTERVAL_SHIFT; - addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false); - if (!merging) { - addressInstances.put(field.name, addresses); - ramBytesUsed.addAndGet(addresses.ramBytesUsed() + Integer.BYTES); - } - } - return addresses; - } - - /** returns a reverse lookup instance for prefix-compressed binary values. */ - private synchronized ReverseTermsIndex getReverseIndexInstance(FieldInfo field, BinaryEntry bytes) throws IOException { - ReverseTermsIndex index = reverseIndexInstances.get(field.name); - if (index == null) { - index = new ReverseTermsIndex(); - data.seek(bytes.reverseIndexOffset); - long size = (bytes.count + REVERSE_INTERVAL_MASK) >>> REVERSE_INTERVAL_SHIFT; - index.termAddresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false); - long dataSize = data.readVLong(); - PagedBytes pagedBytes = new PagedBytes(15); - pagedBytes.copy(data, dataSize); - index.terms = pagedBytes.freeze(true); - if (!merging) { - reverseIndexInstances.put(field.name, index); - ramBytesUsed.addAndGet(index.ramBytesUsed()); - } - } - return index; - } - - private LegacyBinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException { - final MonotonicBlockPackedReader addresses = getIntervalInstance(field, bytes); - final ReverseTermsIndex index = getReverseIndexInstance(field, bytes); - assert addresses.size() > 0; // we don't have to handle empty case - IndexInput slice = data.slice("terms", bytes.offset, bytes.addressesOffset - bytes.offset); - return new CompressedBinaryDocValues(bytes, addresses, index, slice); - } - - @Override - public SortedDocValues getSorted(FieldInfo field) throws IOException { - final int valueCount = (int) binaries.get(field.name).count; - final LegacyBinaryDocValues binary = getLegacyBinary(field); - NumericEntry entry = ords.get(field.name); - final LongValues ordinals = getNumeric(entry); - if (entry.format == SPARSE_COMPRESSED) { - final SparseNumericDocValues sparseValues = ((SparseNumericDocValuesRandomAccessWrapper) ordinals).values; - return new SortedDocValues() { - - @Override - public int ordValue() { - return (int) sparseValues.longValue(); - } - - @Override - public BytesRef lookupOrd(int ord) { - return binary.get(ord); - } - - @Override - public int getValueCount() { - return valueCount; - } - - @Override - public int docID() { - return sparseValues.docID(); - } - - @Override - public int nextDoc() throws IOException { - return sparseValues.nextDoc(); - } - - @Override - public int advance(int target) throws IOException { - return sparseValues.advance(target); - } - - @Override - public boolean advanceExact(int target) throws IOException { - return sparseValues.advanceExact(target); - } - - @Override - public long cost() { - return sparseValues.cost(); - } - - }; - } - return new SortedDocValues() { - private int docID = -1; - private int ord; - - @Override - public int docID() { - return docID; - } - - @Override - public int nextDoc() throws IOException { - assert docID != NO_MORE_DOCS; - while (true) { - docID++; - if (docID == maxDoc) { - docID = NO_MORE_DOCS; - break; - } - ord = (int) ordinals.get(docID); - if (ord != -1) { - break; - } - } - return docID; - } - - @Override - public int advance(int target) throws IOException { - if (target >= maxDoc) { - docID = NO_MORE_DOCS; - return docID; - } else { - docID = target-1; - return nextDoc(); - } - } - - @Override - public boolean advanceExact(int target) throws IOException { - docID = target; - ord = (int) ordinals.get(target); - return ord != -1; - } - - @Override - public int ordValue() { - return ord; - } - - @Override - public long cost() { - // TODO - return 0; - } - - @Override - public BytesRef lookupOrd(int ord) { - return binary.get(ord); - } - - @Override - public int getValueCount() { - return valueCount; - } - - @Override - public int lookupTerm(BytesRef key) throws IOException { - if (binary instanceof CompressedBinaryDocValues) { - return (int) ((CompressedBinaryDocValues)binary).lookupTerm(key); - } else { - return super.lookupTerm(key); - } - } - - @Override - public TermsEnum termsEnum() throws IOException { - if (binary instanceof CompressedBinaryDocValues) { - return ((CompressedBinaryDocValues)binary).getTermsEnum(); - } else { - return super.termsEnum(); - } - } - }; - } - - /** returns an address instance for sortedset ordinal lists */ - private LongValues getOrdIndexInstance(FieldInfo field, NumericEntry entry) throws IOException { - RandomAccessInput data = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset); - return DirectMonotonicReader.getInstance(entry.monotonicMeta, data); - } - - @Override - public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { - SortedSetEntry ss = sortedNumerics.get(field.name); - if (ss.format == SORTED_SINGLE_VALUED) { - NumericEntry numericEntry = numerics.get(field.name); - final LongValues values = getNumeric(numericEntry); - if (numericEntry.format == SPARSE_COMPRESSED) { - SparseNumericDocValues sparseValues = ((SparseNumericDocValuesRandomAccessWrapper) values).values; - return new SortedNumericDocValues() { - - @Override - public long nextValue() throws IOException { - return sparseValues.longValue(); - } - - @Override - public int docValueCount() { - return 1; - } - - @Override - public int docID() { - return sparseValues.docID(); - } - - @Override - public int nextDoc() throws IOException { - return sparseValues.nextDoc(); - } - - @Override - public int advance(int target) throws IOException { - return sparseValues.advance(target); - } - - @Override - public boolean advanceExact(int target) throws IOException { - return sparseValues.advanceExact(target); - } - - @Override - public long cost() { - return sparseValues.cost(); - } - - }; - } - final Bits docsWithField = getLiveBits(numericEntry.missingOffset, maxDoc); - return new SortedNumericDocValues() { - int docID = -1; - - @Override - public int docID() { - return docID; - } - - @Override - public int nextDoc() { - while (true) { - docID++; - if (docID == maxDoc) { - docID = NO_MORE_DOCS; - break; - } - - if (docsWithField.get(docID)) { - // TODO: use .nextSetBit here, at least!! - break; - } - } - return docID; - } - - @Override - public int advance(int target) { - if (target >= maxDoc) { - docID = NO_MORE_DOCS; - return docID; - } else { - docID = target-1; - return nextDoc(); - } - } - - @Override - public boolean advanceExact(int target) throws IOException { - docID = target; - return docsWithField.get(docID); - } - - @Override - public long cost() { - // TODO - return 0; - } - - @Override - public int docValueCount() { - return 1; - } - - @Override - public long nextValue() { - return values.get(docID); - } - }; - } else if (ss.format == SORTED_WITH_ADDRESSES) { - NumericEntry numericEntry = numerics.get(field.name); - final LongValues values = getNumeric(numericEntry); - final LongValues ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name)); - - return new SortedNumericDocValues() { - long startOffset; - long endOffset; - int docID = -1; - long upto; - - @Override - public int docID() { - return docID; - } - - @Override - public int nextDoc() { - while (true) { - docID++; - if (docID == maxDoc) { - docID = NO_MORE_DOCS; - return docID; - } - startOffset = ordIndex.get(docID); - endOffset = ordIndex.get(docID+1L); - if (endOffset > startOffset) { - break; - } - } - upto = startOffset; - return docID; - } - - @Override - public int advance(int target) { - if (target >= maxDoc) { - docID = NO_MORE_DOCS; - return docID; - } else { - docID = target-1; - return nextDoc(); - } - } - - @Override - public boolean advanceExact(int target) throws IOException { - docID = target; - startOffset = ordIndex.get(docID); - endOffset = ordIndex.get(docID+1L); - return endOffset > startOffset; - } - - @Override - public long cost() { - // TODO - return 0; - } - - @Override - public int docValueCount() { - return (int) (endOffset - startOffset); - } - - @Override - public long nextValue() { - return values.get(upto++); - } - }; - } else if (ss.format == SORTED_SET_TABLE) { - NumericEntry entry = ords.get(field.name); - final LongValues ordinals = getNumeric(entry); - - final long[] table = ss.table; - final int[] offsets = ss.tableOffsets; - return new SortedNumericDocValues() { - int startOffset; - int endOffset; - int docID = -1; - int upto; - - @Override - public int docID() { - return docID; - } - - @Override - public int nextDoc() { - while (true) { - docID++; - if (docID == maxDoc) { - docID = NO_MORE_DOCS; - return docID; - } - int ord = (int) ordinals.get(docID); - startOffset = offsets[ord]; - endOffset = offsets[ord+1]; - if (endOffset > startOffset) { - break; - } - } - upto = startOffset; - return docID; - } - - @Override - public int advance(int target) { - if (target >= maxDoc) { - docID = NO_MORE_DOCS; - return docID; - } else { - docID = target-1; - return nextDoc(); - } - } - - @Override - public boolean advanceExact(int target) throws IOException { - docID = target; - int ord = (int) ordinals.get(docID); - startOffset = offsets[ord]; - endOffset = offsets[ord+1]; - return endOffset > startOffset; - } - - @Override - public long cost() { - // TODO - return 0; - } - - @Override - public int docValueCount() { - return endOffset - startOffset; - } - - @Override - public long nextValue() { - return table[upto++]; - } - }; - } else { - throw new AssertionError(); - } - } - - @Override - public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { - SortedSetEntry ss = sortedSets.get(field.name); - switch (ss.format) { - case SORTED_SINGLE_VALUED: - return DocValues.singleton(getSorted(field)); - case SORTED_WITH_ADDRESSES: - return getSortedSetWithAddresses(field); - case SORTED_SET_TABLE: - return getSortedSetTable(field, ss); - default: - throw new AssertionError(); - } - } - - private SortedSetDocValues getSortedSetWithAddresses(FieldInfo field) throws IOException { - final long valueCount = binaries.get(field.name).count; - // we keep the byte[]s and list of ords on disk, these could be large - final LongBinaryDocValues binary = (LongBinaryDocValues) getLegacyBinary(field); - final LongValues ordinals = getNumeric(ords.get(field.name)); - // but the addresses to the ord stream are in RAM - final LongValues ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name)); - - return new LegacySortedSetDocValuesWrapper(new LegacySortedSetDocValues() { - long startOffset; - long offset; - long endOffset; - - @Override - public long nextOrd() { - if (offset == endOffset) { - return NO_MORE_ORDS; - } else { - long ord = ordinals.get(offset); - offset++; - return ord; - } - } - - @Override - public void setDocument(int docID) { - startOffset = offset = ordIndex.get(docID); - endOffset = ordIndex.get(docID+1L); - } - - @Override - public BytesRef lookupOrd(long ord) { - return binary.get(ord); - } - - @Override - public long getValueCount() { - return valueCount; - } - - @Override - public long lookupTerm(BytesRef key) { - if (binary instanceof CompressedBinaryDocValues) { - return ((CompressedBinaryDocValues)binary).lookupTerm(key); - } else { - return super.lookupTerm(key); - } - } - - @Override - public TermsEnum termsEnum() throws IOException { - if (binary instanceof CompressedBinaryDocValues) { - return ((CompressedBinaryDocValues)binary).getTermsEnum(); - } else { - return super.termsEnum(); - } - } - }, maxDoc); - } - - private SortedSetDocValues getSortedSetTable(FieldInfo field, SortedSetEntry ss) throws IOException { - final long valueCount = binaries.get(field.name).count; - final LongBinaryDocValues binary = (LongBinaryDocValues) getLegacyBinary(field); - final NumericEntry ordinalsEntry = ords.get(field.name); - final LongValues ordinals = getNumeric(ordinalsEntry); - - final long[] table = ss.table; - final int[] offsets = ss.tableOffsets; - - return new LegacySortedSetDocValuesWrapper(new LegacySortedSetDocValues() { - - int offset, startOffset, endOffset; - - @Override - public void setDocument(int docID) { - final int ord = (int) ordinals.get(docID); - offset = startOffset = offsets[ord]; - endOffset = offsets[ord + 1]; - } - - @Override - public long nextOrd() { - if (offset == endOffset) { - return NO_MORE_ORDS; - } else { - return table[offset++]; - } - } - - @Override - public BytesRef lookupOrd(long ord) { - return binary.get(ord); - } - - @Override - public long getValueCount() { - return valueCount; - } - - @Override - public long lookupTerm(BytesRef key) { - if (binary instanceof CompressedBinaryDocValues) { - return ((CompressedBinaryDocValues) binary).lookupTerm(key); - } else { - return super.lookupTerm(key); - } - } - - @Override - public TermsEnum termsEnum() throws IOException { - if (binary instanceof CompressedBinaryDocValues) { - return ((CompressedBinaryDocValues) binary).getTermsEnum(); - } else { - return super.termsEnum(); - } - } - }, maxDoc); - } - - private Bits getLiveBits(final long offset, final int count) throws IOException { - if (offset == ALL_MISSING) { - return new Bits.MatchNoBits(count); - } else if (offset == ALL_LIVE) { - return new Bits.MatchAllBits(count); - } else { - int length = (int) ((count + 7L) >>> 3); - final RandomAccessInput in = data.randomAccessSlice(offset, length); - return new Bits() { - @Override - public boolean get(int index) { - try { - return (in.readByte(index >> 3) & (1 << (index & 7))) != 0; - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public int length() { - return count; - } - }; - } - } - - private SparseNumericDocValues getSparseNumericDocValues(NumericEntry entry) throws IOException { - final RandomAccessInput docIdsData = this.data.randomAccessSlice(entry.missingOffset, entry.offset - entry.missingOffset); - final LongValues docIDs = DirectMonotonicReader.getInstance(entry.monotonicMeta, docIdsData); - final LongValues values = getNumeric(entry.nonMissingValues); // cannot be sparse - return new SparseNumericDocValues(Math.toIntExact(entry.numDocsWithValue), docIDs, values); - } - - @Override - public synchronized DocValuesProducer getMergeInstance() throws IOException { - return new Lucene54DocValuesProducer(this); - } - - @Override - public void close() throws IOException { - data.close(); - } - - /** metadata entry for a numeric docvalues field */ - static class NumericEntry { - private NumericEntry() {} - /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */ - long missingOffset; - /** offset to the actual numeric values */ - public long offset; - /** end offset to the actual numeric values */ - public long endOffset; - /** bits per value used to pack the numeric values */ - public int bitsPerValue; - - int format; - /** count of values written */ - public long count; - - /** monotonic meta */ - public DirectMonotonicReader.Meta monotonicMeta; - - long minValue; - long gcd; - long table[]; - - /** for sparse compression */ - long numDocsWithValue; - NumericEntry nonMissingValues; - NumberType numberType; - - } - - /** metadata entry for a binary docvalues field */ - static class BinaryEntry { - private BinaryEntry() {} - /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */ - long missingOffset; - /** offset to the actual binary values */ - long offset; - - int format; - /** count of values written */ - public long count; - int minLength; - int maxLength; - /** offset to the addressing data that maps a value to its slice of the byte[] */ - public long addressesOffset, addressesEndOffset; - /** meta data for addresses */ - public DirectMonotonicReader.Meta addressesMeta; - /** offset to the reverse index */ - public long reverseIndexOffset; - /** packed ints version used to encode addressing information */ - public int packedIntsVersion; - /** packed ints blocksize */ - public int blockSize; - } - - /** metadata entry for a sorted-set docvalues field */ - static class SortedSetEntry { - private SortedSetEntry() {} - int format; - - long[] table; - int[] tableOffsets; - } - - // internally we compose complex dv (sorted/sortedset) from other ones - static abstract class LongBinaryDocValues extends LegacyBinaryDocValues { - @Override - public final BytesRef get(int docID) { - return get((long)docID); - } - - abstract BytesRef get(long id); - } - - // used for reverse lookup to a small range of blocks - static class ReverseTermsIndex implements Accountable { - public MonotonicBlockPackedReader termAddresses; - public PagedBytes.Reader terms; - - @Override - public long ramBytesUsed() { - return termAddresses.ramBytesUsed() + terms.ramBytesUsed(); - } - - @Override - public Collection getChildResources() { - List resources = new ArrayList<>(); - resources.add(Accountables.namedAccountable("term bytes", terms)); - resources.add(Accountables.namedAccountable("term addresses", termAddresses)); - return Collections.unmodifiableList(resources); - } - - @Override - public String toString() { - return getClass().getSimpleName() + "(size=" + termAddresses.size() + ")"; - } - } - - //in the compressed case, we add a few additional operations for - //more efficient reverse lookup and enumeration - static final class CompressedBinaryDocValues extends LongBinaryDocValues { - final long numValues; - final long numIndexValues; - final int maxTermLength; - final MonotonicBlockPackedReader addresses; - final IndexInput data; - final CompressedBinaryTermsEnum termsEnum; - final PagedBytes.Reader reverseTerms; - final MonotonicBlockPackedReader reverseAddresses; - final long numReverseIndexValues; - - public CompressedBinaryDocValues(BinaryEntry bytes, MonotonicBlockPackedReader addresses, ReverseTermsIndex index, IndexInput data) throws IOException { - this.maxTermLength = bytes.maxLength; - this.numValues = bytes.count; - this.addresses = addresses; - this.numIndexValues = addresses.size(); - this.data = data; - this.reverseTerms = index.terms; - this.reverseAddresses = index.termAddresses; - this.numReverseIndexValues = reverseAddresses.size(); - this.termsEnum = getTermsEnum(data); - } - - @Override - public BytesRef get(long id) { - try { - termsEnum.seekExact(id); - return termsEnum.term(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - long lookupTerm(BytesRef key) { - try { - switch (termsEnum.seekCeil(key)) { - case FOUND: return termsEnum.ord(); - case NOT_FOUND: return -termsEnum.ord()-1; - default: return -numValues-1; - } - } catch (IOException bogus) { - throw new RuntimeException(bogus); - } - } - - TermsEnum getTermsEnum() throws IOException { - return getTermsEnum(data.clone()); - } - - private CompressedBinaryTermsEnum getTermsEnum(IndexInput input) throws IOException { - return new CompressedBinaryTermsEnum(input); - } - - class CompressedBinaryTermsEnum extends TermsEnum { - private long currentOrd = -1; - // offset to the start of the current block - private long currentBlockStart; - private final IndexInput input; - // delta from currentBlockStart to start of each term - private final int offsets[] = new int[INTERVAL_COUNT]; - private final byte buffer[] = new byte[2*INTERVAL_COUNT-1]; - - private final BytesRef term = new BytesRef(maxTermLength); - private final BytesRef firstTerm = new BytesRef(maxTermLength); - private final BytesRef scratch = new BytesRef(); - - CompressedBinaryTermsEnum(IndexInput input) throws IOException { - this.input = input; - input.seek(0); - } - - private void readHeader() throws IOException { - firstTerm.length = input.readVInt(); - input.readBytes(firstTerm.bytes, 0, firstTerm.length); - input.readBytes(buffer, 0, INTERVAL_COUNT-1); - if (buffer[0] == -1) { - readShortAddresses(); - } else { - readByteAddresses(); - } - currentBlockStart = input.getFilePointer(); - } - - // read single byte addresses: each is delta - 2 - // (shared prefix byte and length > 0 are both implicit) - private void readByteAddresses() throws IOException { - int addr = 0; - for (int i = 1; i < offsets.length; i++) { - addr += 2 + (buffer[i-1] & 0xFF); - offsets[i] = addr; - } - } - - // read double byte addresses: each is delta - 2 - // (shared prefix byte and length > 0 are both implicit) - private void readShortAddresses() throws IOException { - input.readBytes(buffer, INTERVAL_COUNT-1, INTERVAL_COUNT); - int addr = 0; - for (int i = 1; i < offsets.length; i++) { - int x = i<<1; - addr += 2 + ((buffer[x-1] << 8) | (buffer[x] & 0xFF)); - offsets[i] = addr; - } - } - - // set term to the first term - private void readFirstTerm() throws IOException { - term.length = firstTerm.length; - System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, term.length); - } - - // read term at offset, delta encoded from first term - private void readTerm(int offset) throws IOException { - int start = input.readByte() & 0xFF; - System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, start); - int suffix = offsets[offset] - offsets[offset-1] - 1; - input.readBytes(term.bytes, start, suffix); - term.length = start + suffix; - } - - @Override - public BytesRef next() throws IOException { - currentOrd++; - if (currentOrd >= numValues) { - return null; - } else { - int offset = (int) (currentOrd & INTERVAL_MASK); - if (offset == 0) { - // switch to next block - readHeader(); - readFirstTerm(); - } else { - readTerm(offset); - } - return term; - } - } - - // binary search reverse index to find smaller - // range of blocks to search - long binarySearchIndex(BytesRef text) throws IOException { - long low = 0; - long high = numReverseIndexValues - 1; - while (low <= high) { - long mid = (low + high) >>> 1; - reverseTerms.fill(scratch, reverseAddresses.get(mid)); - int cmp = scratch.compareTo(text); - - if (cmp < 0) { - low = mid + 1; - } else if (cmp > 0) { - high = mid - 1; - } else { - return mid; - } - } - return high; - } - - // binary search against first term in block range - // to find term's block - long binarySearchBlock(BytesRef text, long low, long high) throws IOException { - while (low <= high) { - long mid = (low + high) >>> 1; - input.seek(addresses.get(mid)); - term.length = input.readVInt(); - input.readBytes(term.bytes, 0, term.length); - int cmp = term.compareTo(text); - - if (cmp < 0) { - low = mid + 1; - } else if (cmp > 0) { - high = mid - 1; - } else { - return mid; - } - } - return high; - } - - @Override - public SeekStatus seekCeil(BytesRef text) throws IOException { - // locate block: narrow to block range with index, then search blocks - final long block; - long indexPos = binarySearchIndex(text); - if (indexPos < 0) { - block = 0; - } else { - long low = indexPos << BLOCK_INTERVAL_SHIFT; - long high = Math.min(numIndexValues - 1, low + BLOCK_INTERVAL_MASK); - block = Math.max(low, binarySearchBlock(text, low, high)); - } - - // position before block, then scan to term. - input.seek(addresses.get(block)); - currentOrd = (block << INTERVAL_SHIFT) - 1; - - while (next() != null) { - int cmp = term.compareTo(text); - if (cmp == 0) { - return SeekStatus.FOUND; - } else if (cmp > 0) { - return SeekStatus.NOT_FOUND; - } - } - return SeekStatus.END; - } - - @Override - public void seekExact(long ord) throws IOException { - long block = ord >>> INTERVAL_SHIFT; - if (block != currentOrd >>> INTERVAL_SHIFT) { - // switch to different block - input.seek(addresses.get(block)); - readHeader(); - } - - currentOrd = ord; - - int offset = (int) (ord & INTERVAL_MASK); - if (offset == 0) { - readFirstTerm(); - } else { - input.seek(currentBlockStart + offsets[offset-1]); - readTerm(offset); - } - } - - @Override - public BytesRef term() throws IOException { - return term; - } - - @Override - public long ord() throws IOException { - return currentOrd; - } - - @Override - public int docFreq() throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public long totalTermFreq() throws IOException { - return -1; - } - - @Override - public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException { - throw new UnsupportedOperationException(); - } - - } - } -} diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/package-info.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/package-info.java deleted file mode 100644 index 2c192227b42..00000000000 --- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/package-info.java +++ /dev/null @@ -1,403 +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. - */ - -/** - * Lucene 5.4 file format. - * - *

Apache Lucene - Index File Formats

- * - * - *

Introduction

- *
- *

This document defines the index file formats used in this version of Lucene. - * If you are using a different version of Lucene, please consult the copy of - * docs/ that was distributed with - * the version you are using.

- *

Apache Lucene is written in Java, but several efforts are underway to write - * versions of - * Lucene in other programming languages. If these versions are to remain - * compatible with Apache Lucene, then a language-independent definition of the - * Lucene index format is required. This document thus attempts to provide a - * complete and independent definition of the Apache Lucene file formats.

- *

As Lucene evolves, this document should evolve. Versions of Lucene in - * different programming languages should endeavor to agree on file formats, and - * generate new versions of this document.

- *
- * - *

Definitions

- *
- *

The fundamental concepts in Lucene are index, document, field and term.

- *

An index contains a sequence of documents.

- *
    - *
  • A document is a sequence of fields.
  • - *
  • A field is a named sequence of terms.
  • - *
  • A term is a sequence of bytes.
  • - *
- *

The same sequence of bytes in two different fields is considered a different - * term. Thus terms are represented as a pair: the string naming the field, and the - * bytes within the field.

- * - *

Inverted Indexing

- *

The index stores statistics about terms in order to make term-based search - * more efficient. Lucene's index falls into the family of indexes known as an - * inverted index. This is because it can list, for a term, the documents - * that contain it. This is the inverse of the natural relationship, in which - * documents list terms.

- * - *

Types of Fields

- *

In Lucene, fields may be stored, in which case their text is stored - * in the index literally, in a non-inverted manner. Fields that are inverted are - * called indexed. A field may be both stored and indexed.

- *

The text of a field may be tokenized into terms to be indexed, or the - * text of a field may be used literally as a term to be indexed. Most fields are - * tokenized, but sometimes it is useful for certain identifier fields to be - * indexed literally.

- *

See the {@link org.apache.lucene.document.Field Field} - * java docs for more information on Fields.

- * - *

Segments

- *

Lucene indexes may be composed of multiple sub-indexes, or segments. - * Each segment is a fully independent index, which could be searched separately. - * Indexes evolve by:

- *
    - *
  1. Creating new segments for newly added documents.
  2. - *
  3. Merging existing segments.
  4. - *
- *

Searches may involve multiple segments and/or multiple indexes, each index - * potentially composed of a set of segments.

- * - *

Document Numbers

- *

Internally, Lucene refers to documents by an integer document number. - * The first document added to an index is numbered zero, and each subsequent - * document added gets a number one greater than the previous.

- *

Note that a document's number may change, so caution should be taken when - * storing these numbers outside of Lucene. In particular, numbers may change in - * the following situations:

- *
    - *
  • - *

    The numbers stored in each segment are unique only within the segment, and - * must be converted before they can be used in a larger context. The standard - * technique is to allocate each segment a range of values, based on the range of - * numbers used in that segment. To convert a document number from a segment to an - * external value, the segment's base document number is added. To convert - * an external value back to a segment-specific value, the segment is identified - * by the range that the external value is in, and the segment's base value is - * subtracted. For example two five document segments might be combined, so that - * the first segment has a base value of zero, and the second of five. Document - * three from the second segment would have an external value of eight.

    - *
  • - *
  • - *

    When documents are deleted, gaps are created in the numbering. These are - * eventually removed as the index evolves through merging. Deleted documents are - * dropped when segments are merged. A freshly-merged segment thus has no gaps in - * its numbering.

    - *
  • - *
- *
- * - *

Index Structure Overview

- *
- *

Each segment index maintains the following:

- *
    - *
  • - * {@link org.apache.lucene.codecs.lucene62.Lucene62SegmentInfoFormat Segment info}. - * This contains metadata about a segment, such as the number of documents, - * what files it uses, - *
  • - *
  • - * {@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Field names}. - * This contains the set of field names used in the index. - *
  • - *
  • - * {@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Stored Field values}. - * This contains, for each document, a list of attribute-value pairs, where the attributes - * are field names. These are used to store auxiliary information about the document, such as - * its title, url, or an identifier to access a database. The set of stored fields are what is - * returned for each hit when searching. This is keyed by document number. - *
  • - *
  • - * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term dictionary}. - * A dictionary containing all of the terms used in all of the - * indexed fields of all of the documents. The dictionary also contains the number - * of documents which contain the term, and pointers to the term's frequency and - * proximity data. - *
  • - *
  • - * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Frequency data}. - * For each term in the dictionary, the numbers of all the - * documents that contain that term, and the frequency of the term in that - * document, unless frequencies are omitted (IndexOptions.DOCS_ONLY) - *
  • - *
  • - * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Proximity data}. - * For each term in the dictionary, the positions that the - * term occurs in each document. Note that this will not exist if all fields in - * all documents omit position data. - *
  • - *
  • - * {@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Normalization factors}. - * For each field in each document, a value is stored - * that is multiplied into the score for hits on that field. - *
  • - *
  • - * {@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vectors}. - * For each field in each document, the term vector (sometimes - * called document vector) may be stored. A term vector consists of term text and - * term frequency. To add Term Vectors to your index see the - * {@link org.apache.lucene.document.Field Field} constructors - *
  • - *
  • - * {@link org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat Per-document values}. - * Like stored values, these are also keyed by document - * number, but are generally intended to be loaded into main memory for fast - * access. Whereas stored values are generally intended for summary results from - * searches, per-document values are useful for things like scoring factors. - *
  • - *
  • - * {@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live documents}. - * An optional file indicating which documents are live. - *
  • - *
- *

Details on each of these are provided in their linked pages.

- *
- * - *

File Naming

- *
- *

All files belonging to a segment have the same name with varying extensions. - * The extensions correspond to the different file formats described below. When - * using the Compound File format (default in 1.4 and greater) these files (except - * for the Segment info file, the Lock file, and Deleted documents file) are collapsed - * into a single .cfs file (see below for details)

- *

Typically, all segments in an index are stored in a single directory, - * although this is not required.

- *

As of version 2.1 (lock-less commits), file names are never re-used. - * That is, when any file is saved - * to the Directory it is given a never before used filename. This is achieved - * using a simple generations approach. For example, the first segments file is - * segments_1, then segments_2, etc. The generation is a sequential long integer - * represented in alpha-numeric (base 36) form.

- *
- * - *

Summary of File Extensions

- *
- *

The following table summarizes the names and extensions of the files in - * Lucene:

- * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - *
NameExtensionBrief Description
{@link org.apache.lucene.index.SegmentInfos Segments File}segments_NStores information about a commit point
Lock Filewrite.lockThe Write lock prevents multiple IndexWriters from writing to the same - * file.
{@link org.apache.lucene.codecs.lucene62.Lucene62SegmentInfoFormat Segment Info}.siStores metadata about a segment
{@link org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat Compound File}.cfs, .cfeAn optional "virtual" file consisting of all the other index files for - * systems that frequently run out of file handles.
{@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Fields}.fnmStores information about the fields
{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Index}.fdxContains pointers to field data
{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Data}.fdtThe stored fields for documents
{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Dictionary}.timThe term dictionary, stores term info
{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Index}.tipThe index into the Term Dictionary
{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Frequencies}.docContains the list of docs which contain each term along with frequency
{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Positions}.posStores position information about where a term occurs in the index
{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Payloads}.payStores additional per-position metadata information such as character offsets and user payloads
{@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Norms}.nvd, .nvmEncodes length and boost factors for docs and fields
{@link org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat Per-Document Values}.dvd, .dvmEncodes additional scoring factors or other per-document information.
{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Index}.tvxStores offset into the document data file
{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Documents}.tvdContains information about each document that has term vectors
{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Fields}.tvfThe field level info about term vectors
{@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live Documents}.livInfo about what files are live
- *
- * - *

Lock File

- * The write lock, which is stored in the index directory by default, is named - * "write.lock". If the lock directory is different from the index directory then - * the write lock will be named "XXXX-write.lock" where XXXX is a unique prefix - * derived from the full path to the index directory. When this file is present, a - * writer is currently modifying the index (adding or removing documents). This - * lock file ensures that only one writer is modifying the index at a time. - * - *

History

- *

Compatibility notes are provided in this document, describing how file - * formats have changed from prior versions:

- *
    - *
  • In version 2.1, the file format was changed to allow lock-less commits (ie, - * no more commit lock). The change is fully backwards compatible: you can open a - * pre-2.1 index for searching or adding/deleting of docs. When the new segments - * file is saved (committed), it will be written in the new file format (meaning - * no specific "upgrade" process is needed). But note that once a commit has - * occurred, pre-2.1 Lucene will not be able to read the index.
  • - *
  • In version 2.3, the file format was changed to allow segments to share a - * single set of doc store (vectors & stored fields) files. This allows for - * faster indexing in certain cases. The change is fully backwards compatible (in - * the same way as the lock-less commits change in 2.1).
  • - *
  • In version 2.4, Strings are now written as true UTF-8 byte sequence, not - * Java's modified UTF-8. See - * LUCENE-510 for details.
  • - *
  • In version 2.9, an optional opaque Map<String,String> CommitUserData - * may be passed to IndexWriter's commit methods (and later retrieved), which is - * recorded in the segments_N file. See - * LUCENE-1382 for details. Also, - * diagnostics were added to each segment written recording details about why it - * was written (due to flush, merge; which OS/JRE was used; etc.). See issue - * LUCENE-1654 for details.
  • - *
  • In version 3.0, compressed fields are no longer written to the index (they - * can still be read, but on merge the new segment will write them, uncompressed). - * See issue LUCENE-1960 - * for details.
  • - *
  • In version 3.1, segments records the code version that created them. See - * LUCENE-2720 for details. - * Additionally segments track explicitly whether or not they have term vectors. - * See LUCENE-2811 - * for details.
  • - *
  • In version 3.2, numeric fields are written as natively to stored fields - * file, previously they were stored in text format only.
  • - *
  • In version 3.4, fields can omit position data while still indexing term - * frequencies.
  • - *
  • In version 4.0, the format of the inverted index became extensible via - * the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage - * ({@code DocValues}) was introduced. Normalization factors need no longer be a - * single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}. - * Terms need not be unicode strings, they can be any byte sequence. Term offsets - * can optionally be indexed into the postings lists. Payloads can be stored in the - * term vectors.
  • - *
  • In version 4.1, the format of the postings list changed to use either - * of FOR compression or variable-byte encoding, depending upon the frequency - * of the term. Terms appearing only once were changed to inline directly into - * the term dictionary. Stored fields are compressed by default.
  • - *
  • In version 4.2, term vectors are compressed by default. DocValues has - * a new multi-valued type (SortedSet), that can be used for faceting/grouping/joining - * on multi-valued fields.
  • - *
  • In version 4.5, DocValues were extended to explicitly represent missing values.
  • - *
  • In version 4.6, FieldInfos were extended to support per-field DocValues generation, to - * allow updating NumericDocValues fields.
  • - *
  • In version 4.8, checksum footers were added to the end of each index file - * for improved data integrity. Specifically, the last 8 bytes of every index file - * contain the zlib-crc32 checksum of the file.
  • - *
  • In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric) - * that is suitable for faceting/sorting/analytics. - *
  • In version 5.4, DocValues have been improved to store more information on disk: - * addresses for binary fields and ord indexes for multi-valued fields. - *
  • - *
- * - *

Limitations

- *
- *

Lucene uses a Java int to refer to - * document numbers, and the index file format uses an Int32 - * on-disk to store document numbers. This is a limitation - * of both the index file format and the current implementation. Eventually these - * should be replaced with either UInt64 values, or - * better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.

- *
- */ -package org.apache.lucene.codecs.lucene54; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/Lucene50RWSegmentInfoFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/Lucene50RWSegmentInfoFormat.java deleted file mode 100644 index 4bed31106dc..00000000000 --- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/Lucene50RWSegmentInfoFormat.java +++ /dev/null @@ -1,125 +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.codecs.lucene50; - - -import java.io.IOException; -import java.util.Map; -import java.util.Set; - -import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.index.CorruptIndexException; -import org.apache.lucene.index.IndexFileNames; -import org.apache.lucene.index.SegmentInfo; // javadocs -import org.apache.lucene.store.ChecksumIndexInput; -import org.apache.lucene.store.Directory; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.IndexOutput; -import org.apache.lucene.util.Version; - -/** - * Read-write version of 5.0 SegmentInfoFormat for testing - * @deprecated for test purposes only - */ -@Deprecated -public class Lucene50RWSegmentInfoFormat extends Lucene50SegmentInfoFormat { - - /** Sole constructor. */ - public Lucene50RWSegmentInfoFormat() { - } - - @Override - public SegmentInfo read(Directory dir, String segment, byte[] segmentID, IOContext context) throws IOException { - final String fileName = IndexFileNames.segmentFileName(segment, "", Lucene50SegmentInfoFormat.SI_EXTENSION); - try (ChecksumIndexInput input = dir.openChecksumInput(fileName, context)) { - Throwable priorE = null; - SegmentInfo si = null; - try { - CodecUtil.checkIndexHeader(input, Lucene50SegmentInfoFormat.CODEC_NAME, - Lucene50SegmentInfoFormat.VERSION_START, - Lucene50SegmentInfoFormat.VERSION_CURRENT, - segmentID, ""); - final Version version = Version.fromBits(input.readInt(), input.readInt(), input.readInt()); - - final int docCount = input.readInt(); - if (docCount < 0) { - throw new CorruptIndexException("invalid docCount: " + docCount, input); - } - final boolean isCompoundFile = input.readByte() == SegmentInfo.YES; - - final Map diagnostics = input.readMapOfStrings(); - final Set files = input.readSetOfStrings(); - final Map attributes = input.readMapOfStrings(); - - si = new SegmentInfo(dir, version, null, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, null); - si.setFiles(files); - } catch (Throwable exception) { - priorE = exception; - } finally { - CodecUtil.checkFooter(input, priorE); - } - return si; - } - } - - @Override - public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException { - final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene50SegmentInfoFormat.SI_EXTENSION); - - assert si.getIndexSort() == null; - - try (IndexOutput output = dir.createOutput(fileName, ioContext)) { - // Only add the file once we've successfully created it, else IFD assert can trip: - si.addFile(fileName); - CodecUtil.writeIndexHeader(output, - Lucene50SegmentInfoFormat.CODEC_NAME, - Lucene50SegmentInfoFormat.VERSION_CURRENT, - si.getId(), - ""); - Version version = si.getVersion(); - if (version.major < 5) { - throw new IllegalArgumentException("invalid major version: should be >= 5 but got: " + version.major + " segment=" + si); - } - // Write the Lucene version that created this segment, since 3.1 - output.writeInt(version.major); - output.writeInt(version.minor); - output.writeInt(version.bugfix); - assert version.prerelease == 0; - output.writeInt(si.maxDoc()); - - output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO)); - output.writeMapOfStrings(si.getDiagnostics()); - Set files = si.files(); - for (String file : files) { - if (!IndexFileNames.parseSegmentName(file).equals(si.name)) { - throw new IllegalArgumentException("invalid files: expected segment=" + si.name + ", got=" + files); - } - } - output.writeSetOfStrings(files); - output.writeMapOfStrings(si.getAttributes()); - - CodecUtil.writeFooter(output); - } - } - - /** File extension used to store {@link SegmentInfo}. */ - public final static String SI_EXTENSION = "si"; - static final String CODEC_NAME = "Lucene50SegmentInfo"; - static final int VERSION_SAFE_MAPS = 1; - static final int VERSION_START = VERSION_SAFE_MAPS; - static final int VERSION_CURRENT = VERSION_SAFE_MAPS; -} diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50SegmentInfoFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50SegmentInfoFormat.java deleted file mode 100644 index 0a9bf799dba..00000000000 --- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50SegmentInfoFormat.java +++ /dev/null @@ -1,50 +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.codecs.lucene50; - -import org.apache.lucene.codecs.Codec; -import org.apache.lucene.codecs.lucene60.Lucene60RWCodec; -import org.apache.lucene.index.BaseSegmentInfoFormatTestCase; -import org.apache.lucene.util.Version; - -public class TestLucene50SegmentInfoFormat extends BaseSegmentInfoFormatTestCase { - - @Override - protected Codec getCodec() { - return new Lucene60RWCodec(); - } - - @Override - protected int getCreatedVersionMajor() { - return Version.LUCENE_6_0_0.major; - } - - @Override - protected Version[] getVersions() { - return new Version[] { Version.LUCENE_6_0_0 }; - } - - @Override - protected boolean supportsIndexSort() { - return false; - } - - @Override - protected boolean supportsMinVersion() { - return false; - } -} diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53NormsConsumer.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53NormsConsumer.java deleted file mode 100644 index 833500c1930..00000000000 --- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53NormsConsumer.java +++ /dev/null @@ -1,159 +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.codecs.lucene53; - - -import java.io.IOException; - -import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.codecs.LegacyDocValuesIterables; -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.SegmentWriteState; -import org.apache.lucene.store.IndexOutput; -import org.apache.lucene.util.IOUtils; - -import static org.apache.lucene.codecs.lucene53.Lucene53NormsFormat.VERSION_CURRENT; - -/** - * Writer for {@link Lucene53NormsFormat} - */ -class Lucene53NormsConsumer extends NormsConsumer { - IndexOutput data, meta; - final int maxDoc; - - Lucene53NormsConsumer(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 = state.directory.createOutput(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 = state.directory.createOutput(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 addNormsField(FieldInfo field, NormsProducer normsProducer) throws IOException { - addNormsField(field, LegacyDocValuesIterables.normsIterable(field, normsProducer, maxDoc)); - } - - private void addNormsField(FieldInfo field, Iterable values) throws IOException { - meta.writeVInt(field.number); - long minValue = Long.MAX_VALUE; - long maxValue = Long.MIN_VALUE; - int count = 0; - - for (Number nv : values) { - if (nv == null) { - throw new IllegalStateException("illegal norms data for field " + field.name + ", got null for value: " + count); - } - final long v = nv.longValue(); - minValue = Math.min(minValue, v); - maxValue = Math.max(maxValue, v); - count++; - } - - if (count != maxDoc) { - throw new IllegalStateException("illegal norms data for field " + field.name + ", expected count=" + maxDoc + ", got=" + count); - } - - if (minValue == maxValue) { - addConstant(minValue); - } else if (minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) { - addByte1(values); - } else if (minValue >= Short.MIN_VALUE && maxValue <= Short.MAX_VALUE) { - addByte2(values); - } else if (minValue >= Integer.MIN_VALUE && maxValue <= Integer.MAX_VALUE) { - addByte4(values); - } else { - addByte8(values); - } - } - - private void addConstant(long constant) throws IOException { - meta.writeByte((byte) 0); - meta.writeLong(constant); - } - - private void addByte1(Iterable values) throws IOException { - meta.writeByte((byte) 1); - meta.writeLong(data.getFilePointer()); - - for (Number value : values) { - data.writeByte(value.byteValue()); - } - } - - private void addByte2(Iterable values) throws IOException { - meta.writeByte((byte) 2); - meta.writeLong(data.getFilePointer()); - - for (Number value : values) { - data.writeShort(value.shortValue()); - } - } - - private void addByte4(Iterable values) throws IOException { - meta.writeByte((byte) 4); - meta.writeLong(data.getFilePointer()); - - for (Number value : values) { - data.writeInt(value.intValue()); - } - } - - private void addByte8(Iterable values) throws IOException { - meta.writeByte((byte) 8); - meta.writeLong(data.getFilePointer()); - - for (Number value : values) { - data.writeLong(value.longValue()); - } - } - - @Override - public void close() throws IOException { - boolean success = false; - try { - if (meta != null) { - meta.writeVInt(-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; - } - } -} diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53RWNormsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53RWNormsFormat.java deleted file mode 100644 index 86a2b6a509b..00000000000 --- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53RWNormsFormat.java +++ /dev/null @@ -1,31 +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.codecs.lucene53; - -import java.io.IOException; - -import org.apache.lucene.codecs.NormsConsumer; -import org.apache.lucene.index.SegmentWriteState; - -public class Lucene53RWNormsFormat extends Lucene53NormsFormat { - - @Override - public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException { - return new Lucene53NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION); - } - -} diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java deleted file mode 100644 index 7d37b45e56d..00000000000 --- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java +++ /dev/null @@ -1,44 +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.codecs.lucene53; - -import org.apache.lucene.codecs.Codec; -import org.apache.lucene.codecs.lucene62.Lucene62RWCodec; -import org.apache.lucene.index.BaseNormsFormatTestCase; -import org.apache.lucene.util.Version; - -/** - * Tests Lucene53NormsFormat - */ -public class TestLucene53NormsFormat extends BaseNormsFormatTestCase { - private final Codec codec = new Lucene62RWCodec(); - - @Override - protected int getCreatedVersionMajor() { - return Version.LUCENE_6_2_0.major; - } - - @Override - protected Codec getCodec() { - return codec; - } - - @Override - protected boolean codecSupportsSparsity() { - return false; - } -} \ No newline at end of file diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java deleted file mode 100644 index a761dfcec15..00000000000 --- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java +++ /dev/null @@ -1,640 +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.codecs.lucene54; - - -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 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.codecs.lucene54.Lucene54DocValuesProducer.SparseNumericDocValues; -import org.apache.lucene.codecs.lucene54.Lucene54DocValuesProducer.SparseNumericDocValuesRandomAccessWrapper; -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.SeekStatus; -import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.index.TermsEnum; -import org.apache.lucene.store.Directory; -import org.apache.lucene.store.RAMFile; -import org.apache.lucene.store.RAMInputStream; -import org.apache.lucene.store.RAMOutputStream; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.BytesRefBuilder; -import org.apache.lucene.util.LongValues; -import org.apache.lucene.util.TestUtil; - -/** - * Tests Lucene54DocValuesFormat - */ -public class TestLucene54DocValuesFormat extends BaseCompressingDocValuesFormatTestCase { - private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene54DocValuesFormat()); - - @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++) { - doTestSortedSetVsStoredFields(atLeast(300), 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(300), 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); - } - } - - @Slow - public void testTermsEnumFixedWidth() throws Exception { - int numIterations = atLeast(1); - for (int i = 0; i < numIterations; i++) { - doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 10, 10); - } - } - - @Slow - public void testTermsEnumVariableWidth() throws Exception { - int numIterations = atLeast(1); - for (int i = 0; i < numIterations; i++) { - doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 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), 1, 500); - } - } - - @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(); - TestUtil.checkReader(indexReader); - 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, int minLength, int maxLength) 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 Lucene54DocValuesFormat(); - 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); - final int length = TestUtil.nextInt(random(), minLength, maxLength); - int numValues = random().nextInt(17); - // create a random list of strings - List values = new ArrayList<>(); - for (int v = 0; v < numValues; v++) { - values.add(TestUtil.randomSimpleString(random(), minLength, length)); - } - - // 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()); - } - } - } - - public void testSparseLongValues() throws IOException { - final int iters = atLeast(5); - for (int iter = 0; iter < iters; ++iter) { - final int numDocs = TestUtil.nextInt(random(), 0, 100); - final int[] docIds = new int[numDocs]; - final long[] values = new long[numDocs]; - final int maxDoc; - if (numDocs == 0) { - maxDoc = 1 + random().nextInt(10); - } else { - docIds[0] = random().nextInt(10); - for (int i = 1; i < docIds.length; ++i) { - docIds[i] = docIds[i - 1] + 1 + random().nextInt(100); - } - maxDoc = docIds[numDocs - 1] + 1 + random().nextInt(10); - } - for (int i = 0; i < values.length; ++i) { - values[i] = random().nextLong(); - } - final long missingValue = random().nextLong(); - final LongValues docIdsValues = new LongValues() { - @Override - public long get(long index) { - return docIds[Math.toIntExact(index)]; - } - }; - final LongValues valuesValues = new LongValues() { - @Override - public long get(long index) { - return values[Math.toIntExact(index)]; - } - }; - final SparseNumericDocValues sparseValues = new SparseNumericDocValues(numDocs, docIdsValues, valuesValues); - - // sequential access - assertEquals(-1, sparseValues.docID()); - for (int i = 0; i < docIds.length; ++i) { - assertEquals(docIds[i], sparseValues.nextDoc()); - } - assertEquals(DocIdSetIterator.NO_MORE_DOCS, sparseValues.nextDoc()); - - // advance - for (int i = 0; i < 2000; ++i) { - final int target = TestUtil.nextInt(random(), 0, maxDoc); - int index = Arrays.binarySearch(docIds, target); - if (index < 0) { - index = -1 - index; - } - sparseValues.reset(); - if (index > 0) { - assertEquals(docIds[index - 1], sparseValues.advance(Math.toIntExact(docIds[index - 1]))); - } - if (index == docIds.length) { - assertEquals(DocIdSetIterator.NO_MORE_DOCS, sparseValues.advance(target)); - } else { - assertEquals(docIds[index], sparseValues.advance(target)); - } - } - - // advanceExact - for (int i = 0; i < 2000; ++i) { - sparseValues.reset(); - if (random().nextBoolean() && docIds.length > 0) { - sparseValues.advance(docIds[TestUtil.nextInt(random(), 0, docIds.length - 1)]); - } - - final int target = TestUtil.nextInt(random(), Math.max(0, sparseValues.docID()), maxDoc - 1); - final boolean exists = sparseValues.advanceExact(target); - - final int index = Arrays.binarySearch(docIds, target); - assertEquals(index >= 0, exists); - assertEquals(target, sparseValues.docID()); - - final boolean exists2 = sparseValues.advanceExact(target); - assertEquals(index >= 0, exists2); - assertEquals(target, sparseValues.docID()); - - final int nextIndex = index >= 0 ? index + 1 : -1 - index; - if (nextIndex >= docIds.length) { - assertEquals(DocIdSetIterator.NO_MORE_DOCS, sparseValues.nextDoc()); - } else { - assertEquals(docIds[nextIndex], sparseValues.nextDoc()); - } - } - - - final SparseNumericDocValuesRandomAccessWrapper raWrapper = new SparseNumericDocValuesRandomAccessWrapper(sparseValues, missingValue); - - // random-access - for (int i = 0; i < 2000; ++i) { - final int docId = TestUtil.nextInt(random(), 0, maxDoc - 1); - final int idx = Arrays.binarySearch(docIds, docId); - final long value = raWrapper.get(docId); - if (idx >= 0) { - assertEquals(values[idx], value); - } else { - assertEquals(missingValue, value); - } - } - - // sequential access - for (int docId = 0; docId < maxDoc; docId += random().nextInt(3)) { - final int idx = Arrays.binarySearch(docIds, docId); - final long value = raWrapper.get(docId); - if (idx >= 0) { - assertEquals(values[idx], value); - } else { - assertEquals(missingValue, value); - } - } - } - } - - @Slow - public void testSortedSetAroundBlockSize() throws IOException { - final int frontier = 1 << Lucene54DocValuesFormat.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())); - RAMFile buffer = new RAMFile(); - RAMOutputStream out = new RAMOutputStream(buffer, false); - 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); - } - } - out.close(); - 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); - RAMInputStream in = new RAMInputStream("", buffer); - 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(); - } - } - - @Slow - public void testSortedNumericAroundBlockSize() throws IOException { - final int frontier = 1 << Lucene54DocValuesFormat.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())); - RAMFile buffer = new RAMFile(); - RAMOutputStream out = new RAMOutputStream(buffer, false); - 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)); - } - out.close(); - 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); - RAMInputStream in = new RAMInputStream("", buffer); - 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(); - } - } -} diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexFormatTooOldException.java b/lucene/core/src/java/org/apache/lucene/index/IndexFormatTooOldException.java index b7c02f5299e..1fd79539adb 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexFormatTooOldException.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexFormatTooOldException.java @@ -42,7 +42,7 @@ public class IndexFormatTooOldException extends IOException { * @lucene.internal */ public IndexFormatTooOldException(String resourceDescription, String reason) { super("Format version is not supported (resource " + resourceDescription + "): " + - reason + ". This version of Lucene only supports indexes created with release 6.0 and later."); + reason + ". This version of Lucene only supports indexes created with release 7.0 and later."); this.resourceDescription = resourceDescription; this.reason = reason; this.version = null; 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 0b10d95312f..634caee610a 100644 --- a/lucene/core/src/java/org/apache/lucene/util/Version.java +++ b/lucene/core/src/java/org/apache/lucene/util/Version.java @@ -38,62 +38,6 @@ public final class Version { @Deprecated public static final Version LUCENE_6_0_0 = new Version(6, 0, 0); - /** - * Match settings and bugs in Lucene's 6.0.1 release. - * @deprecated Use latest - */ - @Deprecated - public static final Version LUCENE_6_0_1 = new Version(6, 0, 1); - - /** - * Match settings and bugs in Lucene's 6.1.0 release. - * @deprecated Use latest - */ - @Deprecated - public static final Version LUCENE_6_1_0 = new Version(6, 1, 0); - - /** - * Match settings and bugs in Lucene's 6.2.0 release. - * @deprecated Use latest - */ - @Deprecated - public static final Version LUCENE_6_2_0 = new Version(6, 2, 0); - - /** - * Match settings and bugs in Lucene's 6.2.1 release. - * @deprecated Use latest - */ - @Deprecated - public static final Version LUCENE_6_2_1 = new Version(6, 2, 1); - - /** - * Match settings and bugs in Lucene's 6.3.0 release. - * @deprecated Use latest - */ - @Deprecated - public static final Version LUCENE_6_3_0 = new Version(6, 3, 0); - - /** - * Match settings and bugs in Lucene's 6.4.0 release. - * @deprecated Use latest - */ - @Deprecated - public static final Version LUCENE_6_4_0 = new Version(6, 4, 0); - - /** - * Match settings and bugs in Lucene's 6.4.1 release. - * @deprecated Use latest - */ - @Deprecated - public static final Version LUCENE_6_4_1 = new Version(6, 4, 1); - - /** - * Match settings and bugs in Lucene's 6.4.2 release. - * @deprecated Use latest - */ - @Deprecated - public static final Version LUCENE_6_4_2 = new Version(6, 4, 2); - /** * Match settings and bugs in Lucene's 6.5.0 release. * @deprecated Use latest @@ -101,27 +45,6 @@ public final class Version { @Deprecated public static final Version LUCENE_6_5_0 = new Version(6, 5, 0); - /** - * Match settings and bugs in Lucene's 6.5.1 release. - * @deprecated Use latest - */ - @Deprecated - public static final Version LUCENE_6_5_1 = new Version(6, 5, 1); - - /** - * Match settings and bugs in Lucene's 6.6.0 release. - * @deprecated Use latest - */ - @Deprecated - public static final Version LUCENE_6_6_0 = new Version(6, 6, 0); - - /** - * Match settings and bugs in Lucene's 6.7.0 release. - * @deprecated Use latest - */ - @Deprecated - public static final Version LUCENE_6_7_0 = new Version(6, 7, 0); - /** * Match settings and bugs in Lucene's 7.0.0 release. * @deprecated (8.0.0) Use latest 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 673aaae488a..cc30a5d3bf0 100644 --- a/lucene/core/src/test/org/apache/lucene/util/TestVersion.java +++ b/lucene/core/src/test/org/apache/lucene/util/TestVersion.java @@ -32,25 +32,26 @@ public class TestVersion extends LuceneTestCase { assertTrue("LATEST must be always onOrAfter("+v+")", Version.LATEST.onOrAfter(v)); } } - assertTrue(Version.LUCENE_7_0_0.onOrAfter(Version.LUCENE_6_0_0));; + assertTrue(Version.LUCENE_8_0_0.onOrAfter(Version.LUCENE_7_0_0));; } public void testToString() { - assertEquals("6.0.0", Version.LUCENE_6_0_0.toString()); assertEquals("7.0.0", Version.LUCENE_7_0_0.toString()); + assertEquals("8.0.0", Version.LUCENE_8_0_0.toString()); } public void testParseLeniently() throws Exception { - assertEquals(Version.LUCENE_6_0_0, Version.parseLeniently("6.0")); - assertEquals(Version.LUCENE_6_0_0, Version.parseLeniently("6.0.0")); - assertEquals(Version.LUCENE_6_0_0, Version.parseLeniently("LUCENE_60")); - assertEquals(Version.LUCENE_6_0_0, Version.parseLeniently("LUCENE_6_0")); - assertEquals(Version.LUCENE_6_0_0, Version.parseLeniently("LUCENE_6_0_0")); assertEquals(Version.LUCENE_7_0_0, Version.parseLeniently("7.0")); assertEquals(Version.LUCENE_7_0_0, Version.parseLeniently("7.0.0")); assertEquals(Version.LUCENE_7_0_0, Version.parseLeniently("LUCENE_70")); assertEquals(Version.LUCENE_7_0_0, Version.parseLeniently("LUCENE_7_0")); assertEquals(Version.LUCENE_7_0_0, Version.parseLeniently("LUCENE_7_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")); assertEquals(Version.LATEST, Version.parseLeniently("LUCENE_CURRENT")); @@ -74,9 +75,9 @@ public class TestVersion extends LuceneTestCase { assertTrue(expected.getMessage().contains("LUCENE61")); expected = expectThrows(ParseException.class, () -> { - Version.parseLeniently("LUCENE_6.0.0"); + Version.parseLeniently("LUCENE_7.0.0"); }); - assertTrue(expected.getMessage().contains("LUCENE_6.0.0")); + assertTrue(expected.getMessage().contains("LUCENE_7.0.0")); } public void testParseLenientlyOnAllConstants() throws Exception { @@ -94,8 +95,8 @@ public class TestVersion extends LuceneTestCase { } public void testParse() throws Exception { - assertEquals(Version.LUCENE_6_0_0, Version.parse("6.0.0")); assertEquals(Version.LUCENE_7_0_0, Version.parse("7.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); @@ -103,69 +104,69 @@ public class TestVersion extends LuceneTestCase { } public void testForwardsCompatibility() throws Exception { - assertTrue(Version.parse("6.10.20").onOrAfter(Version.LUCENE_6_0_0)); + assertTrue(Version.parse("7.10.20").onOrAfter(Version.LUCENE_7_0_0)); } public void testParseExceptions() { ParseException expected = expectThrows(ParseException.class, () -> { - Version.parse("LUCENE_6_0_0"); + Version.parse("LUCENE_7_0_0"); }); - assertTrue(expected.getMessage().contains("LUCENE_6_0_0")); + assertTrue(expected.getMessage().contains("LUCENE_7_0_0")); expected = expectThrows(ParseException.class, () -> { - Version.parse("6.256"); + Version.parse("7.256"); }); - assertTrue(expected.getMessage().contains("6.256")); + assertTrue(expected.getMessage().contains("7.256")); expected = expectThrows(ParseException.class, () -> { - Version.parse("6.-1"); + Version.parse("7.-1"); }); - assertTrue(expected.getMessage().contains("6.-1")); + assertTrue(expected.getMessage().contains("7.-1")); expected = expectThrows(ParseException.class, () -> { - Version.parse("6.1.256"); + Version.parse("7.1.256"); }); - assertTrue(expected.getMessage().contains("6.1.256")); + assertTrue(expected.getMessage().contains("7.1.256")); expected = expectThrows(ParseException.class, () -> { - Version.parse("6.1.-1"); + Version.parse("7.1.-1"); }); - assertTrue(expected.getMessage().contains("6.1.-1")); + assertTrue(expected.getMessage().contains("7.1.-1")); expected = expectThrows(ParseException.class, () -> { - Version.parse("6.1.1.3"); + Version.parse("7.1.1.3"); }); - assertTrue(expected.getMessage().contains("6.1.1.3")); + assertTrue(expected.getMessage().contains("7.1.1.3")); expected = expectThrows(ParseException.class, () -> { - Version.parse("6.1.1.-1"); + Version.parse("7.1.1.-1"); }); - assertTrue(expected.getMessage().contains("6.1.1.-1")); + assertTrue(expected.getMessage().contains("7.1.1.-1")); expected = expectThrows(ParseException.class, () -> { - Version.parse("6.1.1.1"); + Version.parse("7.1.1.1"); }); - assertTrue(expected.getMessage().contains("6.1.1.1")); + assertTrue(expected.getMessage().contains("7.1.1.1")); expected = expectThrows(ParseException.class, () -> { - Version.parse("6.1.1.2"); + Version.parse("7.1.1.2"); }); - assertTrue(expected.getMessage().contains("6.1.1.2")); + assertTrue(expected.getMessage().contains("7.1.1.2")); expected = expectThrows(ParseException.class, () -> { - Version.parse("6.0.0.0"); + Version.parse("7.0.0.0"); }); - assertTrue(expected.getMessage().contains("6.0.0.0")); + assertTrue(expected.getMessage().contains("7.0.0.0")); expected = expectThrows(ParseException.class, () -> { - Version.parse("6.0.0.1.42"); + Version.parse("7.0.0.1.42"); }); - assertTrue(expected.getMessage().contains("6.0.0.1.42")); + assertTrue(expected.getMessage().contains("7.0.0.1.42")); expected = expectThrows(ParseException.class, () -> { - Version.parse("6..0.1"); + Version.parse("7..0.1"); }); - assertTrue(expected.getMessage().contains("6..0.1")); + assertTrue(expected.getMessage().contains("7..0.1")); } public void testDeprecations() throws Exception {