Fix backward-compatibility test and remove unnecessary backward compatibility.

This commit is contained in:
Adrien Grand 2021-11-02 12:34:20 +01:00
parent 20c4a1466d
commit 1ddb01044a
95 changed files with 103 additions and 4563 deletions

View File

@ -19,17 +19,12 @@ package org.apache.lucene.analysis.miscellaneous;
import java.util.Map;
import org.apache.lucene.analysis.TokenFilterFactory;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.util.Version;
import org.apache.lucene.util.automaton.TooComplexToDeterminizeException;
/**
* Factory for {@link ConcatenateGraphFilter}.
*
* <ul>
* <li><code>preserveSep</code>: For lucene versions lesser than {@link
* org.apache.lucene.util.Version#LUCENE_8_4_0} Whether {@link
* ConcatenateGraphFilter#SEP_LABEL} should separate the input tokens in the concatenated
* token
* <li><code>tokenSeparator</code>: Separator to use for concatenation. If not present, {@link
* ConcatenateGraphFilter#DEFAULT_TOKEN_SEPARATOR} will be used. If empty, tokens will be
* concatenated without any separators.
@ -57,17 +52,8 @@ public class ConcatenateGraphFilterFactory extends TokenFilterFactory {
public ConcatenateGraphFilterFactory(Map<String, String> args) {
super(args);
Version luceneMatchVersion = getLuceneMatchVersion();
@SuppressWarnings("deprecation")
Version LUCENE_8_4_0 = Version.LUCENE_8_4_0;
if (luceneMatchVersion.onOrAfter(LUCENE_8_4_0)) {
tokenSeparator =
getCharacter(args, "tokenSeparator", ConcatenateGraphFilter.DEFAULT_TOKEN_SEPARATOR);
} else {
boolean preserveSep =
getBoolean(args, "preserveSep", ConcatenateGraphFilter.DEFAULT_PRESERVE_SEP);
tokenSeparator = (preserveSep) ? ConcatenateGraphFilter.DEFAULT_TOKEN_SEPARATOR : null;
}
tokenSeparator =
getCharacter(args, "tokenSeparator", ConcatenateGraphFilter.DEFAULT_TOKEN_SEPARATOR);
preservePositionIncrements =
getBoolean(
args,

View File

@ -49,7 +49,7 @@ import org.apache.lucene.util.Version;
public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
@SuppressWarnings("deprecation")
private static final Version LUCENE_8_0_0 = Version.LUCENE_8_0_0;
private static final Version LUCENE_9_0_0 = Version.LUCENE_9_0_0;
// Test some examples (TODO: we only check behavior, we may need something like
// TestRandomChains...)
@ -111,7 +111,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
public void testVersionAwareFilter() throws Exception {
CustomAnalyzer a =
CustomAnalyzer.builder()
.withDefaultMatchVersion(Version.LUCENE_8_0_0)
.withDefaultMatchVersion(LUCENE_9_0_0)
.withTokenizer(StandardTokenizerFactory.class)
.addTokenFilter(DummyVersionAwareTokenFilterFactory.class)
.build();
@ -128,7 +128,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
public void testFactoryHtmlStripClassicFolding() throws Exception {
CustomAnalyzer a =
CustomAnalyzer.builder()
.withDefaultMatchVersion(LUCENE_8_0_0)
.withDefaultMatchVersion(LUCENE_9_0_0)
.addCharFilter(HTMLStripCharFilterFactory.class)
.withTokenizer(ClassicTokenizerFactory.class)
.addTokenFilter(ASCIIFoldingFilterFactory.class, "preserveOriginal", "true")
@ -164,7 +164,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
public void testHtmlStripClassicFolding() throws Exception {
CustomAnalyzer a =
CustomAnalyzer.builder()
.withDefaultMatchVersion(LUCENE_8_0_0)
.withDefaultMatchVersion(LUCENE_9_0_0)
.addCharFilter("htmlstrip")
.withTokenizer("classic")
.addTokenFilter("asciifolding", "preserveOriginal", "true")
@ -507,7 +507,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
@Override
public TokenStream create(TokenStream input) {
if (luceneMatchVersion.equals(Version.LUCENE_8_0_0)) {
if (luceneMatchVersion.equals(LUCENE_9_0_0)) {
return input;
}
return new LowerCaseFilter(input);

View File

@ -23,7 +23,6 @@ import org.apache.lucene.analysis.BaseTokenStreamFactoryTestCase;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.StopFilter;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.util.Version;
public class TestConcatenateGraphFilterFactory extends BaseTokenStreamFactoryTestCase {
public void test() throws Exception {
@ -52,21 +51,6 @@ public class TestConcatenateGraphFilterFactory extends BaseTokenStreamFactoryTes
assertTokenStreamContents(stream, new String[] {output});
}
public void testPreserveSep() throws Exception {
final String input = "A1 B2 A1 D4 C3";
final String output = "A1A1D4C3";
Reader reader = new StringReader(input);
MockTokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
tokenizer.setReader(reader);
TokenStream stream = tokenizer;
stream = new StopFilter(stream, StopFilter.makeStopSet("B2"));
@SuppressWarnings("deprecation")
Version LUCENE_8_0_0 = Version.LUCENE_8_0_0;
stream =
tokenFilterFactory("ConcatenateGraph", LUCENE_8_0_0, "preserveSep", "false").create(stream);
assertTokenStreamContents(stream, new String[] {output});
}
public void testPreservePositionIncrements() throws Exception {
final String input = "A1 B2 A1 D4 C3";
final String output = "A1 A1 D4 C3";

View File

@ -1,140 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.backward_codecs.lucene70;
import org.apache.lucene.backward_codecs.lucene50.Lucene50CompoundFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50LiveDocsFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50StoredFieldsFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
import org.apache.lucene.backward_codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.backward_codecs.lucene60.Lucene60FieldInfosFormat;
import org.apache.lucene.backward_codecs.lucene60.Lucene60PointsFormat;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
/**
* Implements the Lucene 7.0 index format, with configurable per-field postings and docvalues
* formats.
*
* <p>If you want to reuse functionality of this codec in another codec, extend {@link FilterCodec}.
*
* @see org.apache.lucene.backward_codecs.lucene70 package documentation for file format details.
* @lucene.experimental
*/
public class Lucene70Codec extends Codec {
private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
private final SegmentInfoFormat segmentInfosFormat = new Lucene70SegmentInfoFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene70");
private final PostingsFormat postingsFormat =
new PerFieldPostingsFormat() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
throw new IllegalStateException(
"This codec should only be used for reading, not writing");
}
};
private final DocValuesFormat docValuesFormat =
new PerFieldDocValuesFormat() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return defaultDVFormat;
}
};
private final StoredFieldsFormat storedFieldsFormat =
new Lucene50StoredFieldsFormat(Mode.BEST_SPEED);
/** Instantiates a new codec. */
public Lucene70Codec() {
super("Lucene70");
}
@Override
public StoredFieldsFormat storedFieldsFormat() {
return storedFieldsFormat;
}
@Override
public TermVectorsFormat termVectorsFormat() {
return vectorsFormat;
}
@Override
public PostingsFormat postingsFormat() {
return postingsFormat;
}
@Override
public final FieldInfosFormat fieldInfosFormat() {
return fieldInfosFormat;
}
@Override
public SegmentInfoFormat segmentInfoFormat() {
return segmentInfosFormat;
}
@Override
public final LiveDocsFormat liveDocsFormat() {
return liveDocsFormat;
}
@Override
public CompoundFormat compoundFormat() {
return compoundFormat;
}
@Override
public final PointsFormat pointsFormat() {
return new Lucene60PointsFormat();
}
@Override
public KnnVectorsFormat knnVectorsFormat() {
return KnnVectorsFormat.EMPTY;
}
@Override
public final DocValuesFormat docValuesFormat() {
return docValuesFormat;
}
private final NormsFormat normsFormat = new Lucene70NormsFormat();
@Override
public NormsFormat normsFormat() {
return normsFormat;
}
}

View File

@ -1,704 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.backward_codecs.lucene70;
import static org.apache.lucene.backward_codecs.lucene70.Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
import static org.apache.lucene.backward_codecs.lucene70.Lucene70DocValuesFormat.NUMERIC_BLOCK_SHIFT;
import static org.apache.lucene.backward_codecs.lucene70.Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.backward_codecs.packed.LegacyDirectMonotonicWriter;
import org.apache.lucene.backward_codecs.packed.LegacyDirectWriter;
import org.apache.lucene.backward_codecs.store.EndiannessReverserUtil;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.EmptyDocValuesProducer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.SortedSetSelector;
import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.ByteBuffersIndexOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.MathUtil;
import org.apache.lucene.util.StringHelper;
/** writer for {@link Lucene70DocValuesFormat} */
final class Lucene70DocValuesConsumer extends DocValuesConsumer {
IndexOutput data, meta;
final int maxDoc;
/** expert: Creates a new writer */
public Lucene70DocValuesConsumer(
SegmentWriteState state,
String dataCodec,
String dataExtension,
String metaCodec,
String metaExtension)
throws IOException {
boolean success = false;
try {
String dataName =
IndexFileNames.segmentFileName(
state.segmentInfo.name, state.segmentSuffix, dataExtension);
data = EndiannessReverserUtil.createOutput(state.directory, dataName, state.context);
CodecUtil.writeIndexHeader(
data,
dataCodec,
Lucene70DocValuesFormat.VERSION_CURRENT,
state.segmentInfo.getId(),
state.segmentSuffix);
String metaName =
IndexFileNames.segmentFileName(
state.segmentInfo.name, state.segmentSuffix, metaExtension);
meta = EndiannessReverserUtil.createOutput(state.directory, metaName, state.context);
CodecUtil.writeIndexHeader(
meta,
metaCodec,
Lucene70DocValuesFormat.VERSION_CURRENT,
state.segmentInfo.getId(),
state.segmentSuffix);
maxDoc = state.segmentInfo.maxDoc();
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(this);
}
}
}
@Override
public void close() throws IOException {
boolean success = false;
try {
if (meta != null) {
meta.writeInt(-1); // write EOF marker
CodecUtil.writeFooter(meta); // write checksum
}
if (data != null) {
CodecUtil.writeFooter(data); // write checksum
}
success = true;
} finally {
if (success) {
IOUtils.close(data, meta);
} else {
IOUtils.closeWhileHandlingException(data, meta);
}
meta = data = null;
}
}
@Override
public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer)
throws IOException {
meta.writeInt(field.number);
meta.writeByte(Lucene70DocValuesFormat.NUMERIC);
writeValues(
field,
new EmptyDocValuesProducer() {
@Override
public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
return DocValues.singleton(valuesProducer.getNumeric(field));
}
});
}
private static class MinMaxTracker {
long min, max, numValues, spaceInBits;
MinMaxTracker() {
reset();
spaceInBits = 0;
}
private void reset() {
min = Long.MAX_VALUE;
max = Long.MIN_VALUE;
numValues = 0;
}
/** Accumulate a new value. */
void update(long v) {
min = Math.min(min, v);
max = Math.max(max, v);
++numValues;
}
/** Update the required space. */
void finish() {
if (max > min) {
spaceInBits += LegacyDirectWriter.unsignedBitsRequired(max - min) * numValues;
}
}
/** Update space usage and get ready for accumulating values for the next block. */
void nextBlock() {
finish();
reset();
}
}
private long[] writeValues(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
SortedNumericDocValues values = valuesProducer.getSortedNumeric(field);
int numDocsWithValue = 0;
MinMaxTracker minMax = new MinMaxTracker();
MinMaxTracker blockMinMax = new MinMaxTracker();
long gcd = 0;
Set<Long> uniqueValues = new HashSet<>();
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
for (int i = 0, count = values.docValueCount(); i < count; ++i) {
long v = values.nextValue();
if (gcd != 1) {
if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
// in that case v - minValue might overflow and make the GCD computation return
// wrong results. Since these extreme values are unlikely, we just discard
// GCD computation for them
gcd = 1;
} else if (minMax.numValues != 0) { // minValue needs to be set first
gcd = MathUtil.gcd(gcd, v - minMax.min);
}
}
minMax.update(v);
blockMinMax.update(v);
if (blockMinMax.numValues == NUMERIC_BLOCK_SIZE) {
blockMinMax.nextBlock();
}
if (uniqueValues != null && uniqueValues.add(v) && uniqueValues.size() > 256) {
uniqueValues = null;
}
}
numDocsWithValue++;
}
minMax.finish();
blockMinMax.finish();
final long numValues = minMax.numValues;
long min = minMax.min;
final long max = minMax.max;
assert blockMinMax.spaceInBits <= minMax.spaceInBits;
if (numDocsWithValue == 0) {
meta.writeLong(-2);
meta.writeLong(0L);
} else if (numDocsWithValue == maxDoc) {
meta.writeLong(-1);
meta.writeLong(0L);
} else {
long offset = data.getFilePointer();
meta.writeLong(offset);
values = valuesProducer.getSortedNumeric(field);
IndexedDISI.writeBitSet(values, data);
meta.writeLong(data.getFilePointer() - offset);
}
meta.writeLong(numValues);
final int numBitsPerValue;
boolean doBlocks = false;
Map<Long, Integer> encode = null;
if (min >= max) {
numBitsPerValue = 0;
meta.writeInt(-1);
} else {
if (uniqueValues != null
&& uniqueValues.size() > 1
&& LegacyDirectWriter.unsignedBitsRequired(uniqueValues.size() - 1)
< LegacyDirectWriter.unsignedBitsRequired((max - min) / gcd)) {
numBitsPerValue = LegacyDirectWriter.unsignedBitsRequired(uniqueValues.size() - 1);
final Long[] sortedUniqueValues = uniqueValues.toArray(new Long[0]);
Arrays.sort(sortedUniqueValues);
meta.writeInt(sortedUniqueValues.length);
for (Long v : sortedUniqueValues) {
meta.writeLong(v);
}
encode = new HashMap<>();
for (int i = 0; i < sortedUniqueValues.length; ++i) {
encode.put(sortedUniqueValues[i], i);
}
min = 0;
gcd = 1;
} else {
uniqueValues = null;
// we do blocks if that appears to save 10+% storage
doBlocks =
minMax.spaceInBits > 0 && (double) blockMinMax.spaceInBits / minMax.spaceInBits <= 0.9;
if (doBlocks) {
numBitsPerValue = 0xFF;
meta.writeInt(-2 - NUMERIC_BLOCK_SHIFT);
} else {
numBitsPerValue = LegacyDirectWriter.unsignedBitsRequired((max - min) / gcd);
if (gcd == 1
&& min > 0
&& LegacyDirectWriter.unsignedBitsRequired(max)
== LegacyDirectWriter.unsignedBitsRequired(max - min)) {
min = 0;
}
meta.writeInt(-1);
}
}
}
meta.writeByte((byte) numBitsPerValue);
meta.writeLong(min);
meta.writeLong(gcd);
long startOffset = data.getFilePointer();
meta.writeLong(startOffset);
if (doBlocks) {
writeValuesMultipleBlocks(valuesProducer.getSortedNumeric(field), gcd);
} else if (numBitsPerValue != 0) {
writeValuesSingleBlock(
valuesProducer.getSortedNumeric(field), numValues, numBitsPerValue, min, gcd, encode);
}
meta.writeLong(data.getFilePointer() - startOffset);
return new long[] {numDocsWithValue, numValues};
}
private void writeValuesSingleBlock(
SortedNumericDocValues values,
long numValues,
int numBitsPerValue,
long min,
long gcd,
Map<Long, Integer> encode)
throws IOException {
LegacyDirectWriter writer = LegacyDirectWriter.getInstance(data, numValues, numBitsPerValue);
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
for (int i = 0, count = values.docValueCount(); i < count; ++i) {
long v = values.nextValue();
if (encode == null) {
writer.add((v - min) / gcd);
} else {
writer.add(encode.get(v));
}
}
}
writer.finish();
}
private void writeValuesMultipleBlocks(SortedNumericDocValues values, long gcd)
throws IOException {
final long[] buffer = new long[NUMERIC_BLOCK_SIZE];
final ByteBuffersDataOutput encodeBuffer = ByteBuffersDataOutput.newResettableInstance();
int upTo = 0;
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
for (int i = 0, count = values.docValueCount(); i < count; ++i) {
buffer[upTo++] = values.nextValue();
if (upTo == NUMERIC_BLOCK_SIZE) {
writeBlock(buffer, NUMERIC_BLOCK_SIZE, gcd, encodeBuffer);
upTo = 0;
}
}
}
if (upTo > 0) {
writeBlock(buffer, upTo, gcd, encodeBuffer);
}
}
private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutput buffer)
throws IOException {
assert length > 0;
long min = values[0];
long max = values[0];
for (int i = 1; i < length; ++i) {
final long v = values[i];
assert Math.floorMod(values[i] - min, gcd) == 0;
min = Math.min(min, v);
max = Math.max(max, v);
}
if (min == max) {
data.writeByte((byte) 0);
data.writeLong(min);
} else {
final int bitsPerValue = LegacyDirectWriter.unsignedBitsRequired(max - min);
buffer.reset();
assert buffer.size() == 0;
final LegacyDirectWriter w = LegacyDirectWriter.getInstance(buffer, length, bitsPerValue);
for (int i = 0; i < length; ++i) {
w.add((values[i] - min) / gcd);
}
w.finish();
data.writeByte((byte) bitsPerValue);
data.writeLong(min);
data.writeInt(Math.toIntExact(buffer.size()));
buffer.copyTo(data);
}
}
@Override
public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
meta.writeInt(field.number);
meta.writeByte(Lucene70DocValuesFormat.BINARY);
BinaryDocValues values = valuesProducer.getBinary(field);
long start = data.getFilePointer();
meta.writeLong(start);
int numDocsWithField = 0;
int minLength = Integer.MAX_VALUE;
int maxLength = 0;
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
numDocsWithField++;
BytesRef v = values.binaryValue();
int length = v.length;
data.writeBytes(v.bytes, v.offset, v.length);
minLength = Math.min(length, minLength);
maxLength = Math.max(length, maxLength);
}
assert numDocsWithField <= maxDoc;
meta.writeLong(data.getFilePointer() - start);
if (numDocsWithField == 0) {
meta.writeLong(-2);
meta.writeLong(0L);
} else if (numDocsWithField == maxDoc) {
meta.writeLong(-1);
meta.writeLong(0L);
} else {
long offset = data.getFilePointer();
meta.writeLong(offset);
values = valuesProducer.getBinary(field);
IndexedDISI.writeBitSet(values, data);
meta.writeLong(data.getFilePointer() - offset);
}
meta.writeInt(numDocsWithField);
meta.writeInt(minLength);
meta.writeInt(maxLength);
if (maxLength > minLength) {
start = data.getFilePointer();
meta.writeLong(start);
meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
final LegacyDirectMonotonicWriter writer =
LegacyDirectMonotonicWriter.getInstance(
meta, data, numDocsWithField + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
long addr = 0;
writer.add(addr);
values = valuesProducer.getBinary(field);
for (int doc = values.nextDoc();
doc != DocIdSetIterator.NO_MORE_DOCS;
doc = values.nextDoc()) {
addr += values.binaryValue().length;
writer.add(addr);
}
writer.finish();
meta.writeLong(data.getFilePointer() - start);
}
}
@Override
public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
meta.writeInt(field.number);
meta.writeByte(Lucene70DocValuesFormat.SORTED);
doAddSortedField(field, valuesProducer);
}
private void doAddSortedField(FieldInfo field, DocValuesProducer valuesProducer)
throws IOException {
SortedDocValues values = valuesProducer.getSorted(field);
int numDocsWithField = 0;
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
numDocsWithField++;
}
if (numDocsWithField == 0) {
meta.writeLong(-2);
meta.writeLong(0L);
} else if (numDocsWithField == maxDoc) {
meta.writeLong(-1);
meta.writeLong(0L);
} else {
long offset = data.getFilePointer();
meta.writeLong(offset);
values = valuesProducer.getSorted(field);
IndexedDISI.writeBitSet(values, data);
meta.writeLong(data.getFilePointer() - offset);
}
meta.writeInt(numDocsWithField);
if (values.getValueCount() <= 1) {
meta.writeByte((byte) 0);
meta.writeLong(0L);
meta.writeLong(0L);
} else {
int numberOfBitsPerOrd = LegacyDirectWriter.unsignedBitsRequired(values.getValueCount() - 1);
meta.writeByte((byte) numberOfBitsPerOrd);
long start = data.getFilePointer();
meta.writeLong(start);
LegacyDirectWriter writer =
LegacyDirectWriter.getInstance(data, numDocsWithField, numberOfBitsPerOrd);
values = valuesProducer.getSorted(field);
for (int doc = values.nextDoc();
doc != DocIdSetIterator.NO_MORE_DOCS;
doc = values.nextDoc()) {
writer.add(values.ordValue());
}
writer.finish();
meta.writeLong(data.getFilePointer() - start);
}
addTermsDict(DocValues.singleton(valuesProducer.getSorted(field)));
}
private void addTermsDict(SortedSetDocValues values) throws IOException {
final long size = values.getValueCount();
meta.writeVLong(size);
meta.writeInt(Lucene70DocValuesFormat.TERMS_DICT_BLOCK_SHIFT);
ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput();
ByteBuffersIndexOutput addressIndexOut =
new ByteBuffersIndexOutput(addressBuffer, "temp", "temp");
meta.writeInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
long numBlocks =
(size + Lucene70DocValuesFormat.TERMS_DICT_BLOCK_MASK)
>>> Lucene70DocValuesFormat.TERMS_DICT_BLOCK_SHIFT;
LegacyDirectMonotonicWriter writer =
LegacyDirectMonotonicWriter.getInstance(
meta, addressIndexOut, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT);
BytesRefBuilder previous = new BytesRefBuilder();
long ord = 0;
long start = data.getFilePointer();
int maxLength = 0;
TermsEnum iterator = values.termsEnum();
for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
if ((ord & Lucene70DocValuesFormat.TERMS_DICT_BLOCK_MASK) == 0) {
writer.add(data.getFilePointer() - start);
data.writeVInt(term.length);
data.writeBytes(term.bytes, term.offset, term.length);
} else {
final int prefixLength = StringHelper.bytesDifference(previous.get(), term);
final int suffixLength = term.length - prefixLength;
assert suffixLength > 0; // terms are unique
data.writeByte((byte) (Math.min(prefixLength, 15) | (Math.min(15, suffixLength - 1) << 4)));
if (prefixLength >= 15) {
data.writeVInt(prefixLength - 15);
}
if (suffixLength >= 16) {
data.writeVInt(suffixLength - 16);
}
data.writeBytes(term.bytes, term.offset + prefixLength, term.length - prefixLength);
}
maxLength = Math.max(maxLength, term.length);
previous.copyBytes(term);
++ord;
}
writer.finish();
meta.writeInt(maxLength);
meta.writeLong(start);
meta.writeLong(data.getFilePointer() - start);
start = data.getFilePointer();
addressBuffer.copyTo(data);
meta.writeLong(start);
meta.writeLong(data.getFilePointer() - start);
// Now write the reverse terms index
writeTermsIndex(values);
}
private void writeTermsIndex(SortedSetDocValues values) throws IOException {
final long size = values.getValueCount();
meta.writeInt(Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_SHIFT);
long start = data.getFilePointer();
long numBlocks =
1L
+ ((size + Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK)
>>> Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_SHIFT);
ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput();
ByteBuffersIndexOutput addressIndexOut =
new ByteBuffersIndexOutput(addressBuffer, "temp", "temp");
LegacyDirectMonotonicWriter writer =
LegacyDirectMonotonicWriter.getInstance(
meta, addressIndexOut, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT);
TermsEnum iterator = values.termsEnum();
BytesRefBuilder previous = new BytesRefBuilder();
long offset = 0;
long ord = 0;
for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
if ((ord & Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) == 0) {
writer.add(offset);
final int sortKeyLength;
if (ord == 0) {
// no previous term: no bytes to write
sortKeyLength = 0;
} else {
sortKeyLength = StringHelper.sortKeyLength(previous.get(), term);
}
offset += sortKeyLength;
data.writeBytes(term.bytes, term.offset, sortKeyLength);
} else if ((ord & Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK)
== Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) {
previous.copyBytes(term);
}
++ord;
}
writer.add(offset);
writer.finish();
meta.writeLong(start);
meta.writeLong(data.getFilePointer() - start);
start = data.getFilePointer();
addressBuffer.copyTo(data);
meta.writeLong(start);
meta.writeLong(data.getFilePointer() - start);
}
@Override
public void addSortedNumericField(FieldInfo field, DocValuesProducer valuesProducer)
throws IOException {
meta.writeInt(field.number);
meta.writeByte(Lucene70DocValuesFormat.SORTED_NUMERIC);
long[] stats = writeValues(field, valuesProducer);
int numDocsWithField = Math.toIntExact(stats[0]);
long numValues = stats[1];
assert numValues >= numDocsWithField;
meta.writeInt(numDocsWithField);
if (numValues > numDocsWithField) {
long start = data.getFilePointer();
meta.writeLong(start);
meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
final LegacyDirectMonotonicWriter addressesWriter =
LegacyDirectMonotonicWriter.getInstance(
meta, data, numDocsWithField + 1L, DIRECT_MONOTONIC_BLOCK_SHIFT);
long addr = 0;
addressesWriter.add(addr);
SortedNumericDocValues values = valuesProducer.getSortedNumeric(field);
for (int doc = values.nextDoc();
doc != DocIdSetIterator.NO_MORE_DOCS;
doc = values.nextDoc()) {
addr += values.docValueCount();
addressesWriter.add(addr);
}
addressesWriter.finish();
meta.writeLong(data.getFilePointer() - start);
}
}
@Override
public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer)
throws IOException {
meta.writeInt(field.number);
meta.writeByte(Lucene70DocValuesFormat.SORTED_SET);
SortedSetDocValues values = valuesProducer.getSortedSet(field);
int numDocsWithField = 0;
long numOrds = 0;
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
numDocsWithField++;
for (long ord = values.nextOrd();
ord != SortedSetDocValues.NO_MORE_ORDS;
ord = values.nextOrd()) {
numOrds++;
}
}
if (numDocsWithField == numOrds) {
meta.writeByte((byte) 0);
doAddSortedField(
field,
new EmptyDocValuesProducer() {
@Override
public SortedDocValues getSorted(FieldInfo field) throws IOException {
return SortedSetSelector.wrap(
valuesProducer.getSortedSet(field), SortedSetSelector.Type.MIN);
}
});
return;
}
meta.writeByte((byte) 1);
assert numDocsWithField != 0;
if (numDocsWithField == maxDoc) {
meta.writeLong(-1);
meta.writeLong(0L);
} else {
long offset = data.getFilePointer();
meta.writeLong(offset);
values = valuesProducer.getSortedSet(field);
IndexedDISI.writeBitSet(values, data);
meta.writeLong(data.getFilePointer() - offset);
}
int numberOfBitsPerOrd = LegacyDirectWriter.unsignedBitsRequired(values.getValueCount() - 1);
meta.writeByte((byte) numberOfBitsPerOrd);
long start = data.getFilePointer();
meta.writeLong(start);
LegacyDirectWriter writer = LegacyDirectWriter.getInstance(data, numOrds, numberOfBitsPerOrd);
values = valuesProducer.getSortedSet(field);
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
for (long ord = values.nextOrd();
ord != SortedSetDocValues.NO_MORE_ORDS;
ord = values.nextOrd()) {
writer.add(ord);
}
}
writer.finish();
meta.writeLong(data.getFilePointer() - start);
meta.writeInt(numDocsWithField);
start = data.getFilePointer();
meta.writeLong(start);
meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
final LegacyDirectMonotonicWriter addressesWriter =
LegacyDirectMonotonicWriter.getInstance(
meta, data, numDocsWithField + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
long addr = 0;
addressesWriter.add(addr);
values = valuesProducer.getSortedSet(field);
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
values.nextOrd();
addr++;
while (values.nextOrd() != SortedSetDocValues.NO_MORE_ORDS) {
addr++;
}
addressesWriter.add(addr);
}
addressesWriter.finish();
meta.writeLong(data.getFilePointer() - start);
addTermsDict(values);
}
}

View File

@ -1,171 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.backward_codecs.lucene70;
import java.io.IOException;
import org.apache.lucene.backward_codecs.packed.LegacyDirectWriter;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.SmallFloat;
/**
* Lucene 7.0 DocValues format.
*
* <p>Documents that have a value for the field are encoded in a way that it is always possible to
* know the ordinal of the current document in the set of documents that have a value. For instance,
* say the set of documents that have a value for the field is <code>{1, 5, 6, 11}</code>. When the
* iterator is on <code>6</code>, it knows that this is the 3rd item of the set. This way, values
* can be stored densely and accessed based on their index at search time. If all documents in a
* segment have a value for the field, the index is the same as the doc ID, so this case is encoded
* implicitly and is very fast at query time. On the other hand if some documents are missing a
* value for the field then the set of documents that have a value is encoded into blocks. All doc
* IDs that share the same upper 16 bits are encoded into the same block with the following
* strategies:
*
* <ul>
* <li>SPARSE: This strategy is used when a block contains at most 4095 documents. The lower 16
* bits of doc IDs are stored as {@link DataOutput#writeShort(short) shorts} while the upper
* 16 bits are given by the block ID.
* <li>DENSE: This strategy is used when a block contains between 4096 and 65535 documents. The
* lower bits of doc IDs are stored in a bit set. Advancing is performed using {@link
* Long#numberOfTrailingZeros(long) ntz} operations while the index is computed by
* accumulating the {@link Long#bitCount(long) bit counts} of the visited longs.
* <li>ALL: This strategy is used when a block contains exactly 65536 documents, meaning that the
* block is full. In that case doc IDs do not need to be stored explicitly. This is typically
* faster than both SPARSE and DENSE which is a reason why it is preferable to have all
* documents that have a value for a field using contiguous doc IDs, for instance by using
* {@link IndexWriterConfig#setIndexSort(org.apache.lucene.search.Sort) index sorting}.
* </ul>
*
* <p>Then the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) are
* encoded using the following strategies:
*
* <p>{@link DocValuesType#NUMERIC NUMERIC}:
*
* <ul>
* <li>Delta-compressed: per-document integers written as deltas from the minimum value,
* compressed with bitpacking. For more information, see {@link LegacyDirectWriter}.
* <li>Table-compressed: when the number of unique values is very small (&lt; 256), and when there
* are unused "gaps" in the range of values used (such as {@link SmallFloat}), a lookup table
* is written instead. Each per-document entry is instead the ordinal to this table, and those
* ordinals are compressed with bitpacking ({@link LegacyDirectWriter}).
* <li>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.
* <li>Monotonic-compressed: when all numbers are monotonically increasing offsets, they are
* written as blocks of bitpacked integers, encoding the deviation from the expected delta.
* <li>Const-compressed: when there is only one possible value, no per-document data is needed and
* this value is encoded alone.
* </ul>
*
* <p>{@link DocValuesType#BINARY BINARY}:
*
* <ul>
* <li>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}).
* <li>Variable-width Binary: one large concatenated byte[] is written, along with end addresses
* for each document. The addresses are written as Monotonic-compressed numerics.
* <li>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.
* </ul>
*
* <p>{@link DocValuesType#SORTED SORTED}:
*
* <ul>
* <li>Sorted: a mapping of ordinals to deduplicated terms is written as Prefix-compressed Binary,
* along with the per-document ordinals written using one of the numeric strategies above.
* </ul>
*
* <p>{@link DocValuesType#SORTED_SET SORTED_SET}:
*
* <ul>
* <li>Single: if all documents have 0 or 1 value, then data are written like SORTED.
* <li>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.
* </ul>
*
* <p>{@link DocValuesType#SORTED_NUMERIC SORTED_NUMERIC}:
*
* <ul>
* <li>Single: if all documents have 0 or 1 value, then data are written like NUMERIC.
* <li>SortedNumeric: a value list and per-document index into this list are written using the
* numeric strategies above.
* </ul>
*
* <p>Files:
*
* <ol>
* <li><code>.dvd</code>: DocValues data
* <li><code>.dvm</code>: DocValues metadata
* </ol>
*
* @lucene.experimental
*/
public final class Lucene70DocValuesFormat extends DocValuesFormat {
/** Sole Constructor */
public Lucene70DocValuesFormat() {
super("Lucene70");
}
@Override
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new Lucene70DocValuesConsumer(
state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
@Override
public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
return new Lucene70DocValuesProducer(
state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
static final String DATA_CODEC = "Lucene70DocValuesData";
static final String DATA_EXTENSION = "dvd";
static final String META_CODEC = "Lucene70DocValuesMetadata";
static final String META_EXTENSION = "dvm";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
// indicates docvalues type
static final byte NUMERIC = 0;
static final byte BINARY = 1;
static final byte SORTED = 2;
static final byte SORTED_SET = 3;
static final byte SORTED_NUMERIC = 4;
static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16;
static final int NUMERIC_BLOCK_SHIFT = 14;
static final int NUMERIC_BLOCK_SIZE = 1 << NUMERIC_BLOCK_SHIFT;
static final int TERMS_DICT_BLOCK_SHIFT = 4;
static final int TERMS_DICT_BLOCK_SIZE = 1 << TERMS_DICT_BLOCK_SHIFT;
static final int TERMS_DICT_BLOCK_MASK = TERMS_DICT_BLOCK_SIZE - 1;
static final int TERMS_DICT_REVERSE_INDEX_SHIFT = 10;
static final int TERMS_DICT_REVERSE_INDEX_SIZE = 1 << TERMS_DICT_REVERSE_INDEX_SHIFT;
static final int TERMS_DICT_REVERSE_INDEX_MASK = TERMS_DICT_REVERSE_INDEX_SIZE - 1;
}

View File

@ -1,105 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.backward_codecs.lucene70;
import java.io.IOException;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.DataOutput;
/**
* Lucene 7.0 Score normalization format.
*
* <p>Encodes normalization values by encoding each value with the minimum number of bytes needed to
* represent the range (which can be zero).
*
* <p>Files:
*
* <ol>
* <li><code>.nvd</code>: Norms data
* <li><code>.nvm</code>: Norms metadata
* </ol>
*
* <ol>
* <li><a id="nvm"></a>
* <p>The Norms metadata or .nvm file.
* <p>For each norms field, this stores metadata, such as the offset into the Norms data
* (.nvd)
* <p>Norms metadata (.dvm) --&gt; Header,&lt;Entry&gt;<sup>NumFields</sup>,Footer
* <ul>
* <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}
* <li>Entry --&gt; FieldNumber, DocsWithFieldAddress, DocsWithFieldLength,
* NumDocsWithField, BytesPerNorm, NormsAddress
* <li>FieldNumber --&gt; {@link DataOutput#writeInt Int32}
* <li>DocsWithFieldAddress --&gt; {@link DataOutput#writeLong Int64}
* <li>DocsWithFieldLength --&gt; {@link DataOutput#writeLong Int64}
* <li>NumDocsWithField --&gt; {@link DataOutput#writeInt Int32}
* <li>BytesPerNorm --&gt; {@link DataOutput#writeByte byte}
* <li>NormsAddress --&gt; {@link DataOutput#writeLong Int64}
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}
* </ul>
* <p>FieldNumber of -1 indicates the end of metadata.
* <p>NormsAddress is the pointer to the start of the data in the norms data (.nvd), or the
* singleton value when BytesPerValue = 0. If BytesPerValue is different from 0 then there are
* NumDocsWithField values to read at that offset.
* <p>DocsWithFieldAddress is the pointer to the start of the bit set containing documents
* that have a norm in the norms data (.nvd), or -2 if no documents have a norm value, or -1
* if all documents have a norm value.
* <p>DocsWithFieldLength is the number of bytes used to encode the set of documents that have
* a norm.
* <li><a id="nvd"></a>
* <p>The Norms data or .nvd file.
* <p>For each Norms field, this stores the actual per-document data (the heavy-lifting)
* <p>Norms data (.nvd) --&gt; Header,&lt; Data &gt;<sup>NumFields</sup>,Footer
* <ul>
* <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}
* <li>DocsWithFieldData --&gt; Bit set of MaxDoc bits
* <li>NormsData --&gt; {@link DataOutput#writeByte(byte) byte}<sup>NumDocsWithField *
* BytesPerValue</sup>
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}
* </ul>
* </ol>
*
* @lucene.experimental
*/
public class Lucene70NormsFormat extends NormsFormat {
/** Sole Constructor */
public Lucene70NormsFormat() {}
@Override
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
throw new UnsupportedOperationException("Old codecs may only be used for reading");
}
@Override
public NormsProducer normsProducer(SegmentReadState state) throws IOException {
return new Lucene70NormsProducer(
state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
}
static final String DATA_CODEC = "Lucene70NormsData";
static final String DATA_EXTENSION = "nvd";
static final String METADATA_CODEC = "Lucene70NormsMetadata";
static final String METADATA_EXTENSION = "nvm";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
}

View File

@ -1,447 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.backward_codecs.lucene70;
import static org.apache.lucene.backward_codecs.lucene70.Lucene70NormsFormat.VERSION_CURRENT;
import static org.apache.lucene.backward_codecs.lucene70.Lucene70NormsFormat.VERSION_START;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.backward_codecs.store.EndiannessReverserUtil;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.RandomAccessInput;
import org.apache.lucene.util.IOUtils;
/** Reader for {@link Lucene70NormsFormat} */
final class Lucene70NormsProducer extends NormsProducer implements Cloneable {
// metadata maps (just file pointers and minimal stuff)
private final Map<Integer, NormsEntry> norms = new HashMap<>();
private final int maxDoc;
private IndexInput data;
private boolean merging;
private Map<Integer, IndexInput> disiInputs;
private Map<Integer, RandomAccessInput> dataInputs;
Lucene70NormsProducer(
SegmentReadState state,
String dataCodec,
String dataExtension,
String metaCodec,
String metaExtension)
throws IOException {
maxDoc = state.segmentInfo.maxDoc();
String metaName =
IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
int version = -1;
// read in the entries from the metadata file.
try (ChecksumIndexInput in =
EndiannessReverserUtil.openChecksumInput(state.directory, metaName, state.context)) {
Throwable priorE = null;
try {
version =
CodecUtil.checkIndexHeader(
in,
metaCodec,
VERSION_START,
VERSION_CURRENT,
state.segmentInfo.getId(),
state.segmentSuffix);
readFields(in, state.fieldInfos);
} catch (Throwable exception) {
priorE = exception;
} finally {
CodecUtil.checkFooter(in, priorE);
}
}
String dataName =
IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
data = EndiannessReverserUtil.openInput(state.directory, dataName, state.context);
boolean success = false;
try {
final int version2 =
CodecUtil.checkIndexHeader(
data,
dataCodec,
VERSION_START,
VERSION_CURRENT,
state.segmentInfo.getId(),
state.segmentSuffix);
if (version != version2) {
throw new CorruptIndexException(
"Format versions mismatch: meta=" + version + ",data=" + version2, data);
}
// NOTE: data file is too costly to verify checksum against all the bytes on open,
// but for now we at least verify proper structure of the checksum footer: which looks
// for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
// such as file truncation.
CodecUtil.retrieveChecksum(data);
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(this.data);
}
}
}
@Override
public NormsProducer getMergeInstance() {
Lucene70NormsProducer clone;
try {
clone = (Lucene70NormsProducer) super.clone();
} catch (CloneNotSupportedException e) {
// cannot happen
throw new RuntimeException(e);
}
clone.data = data.clone();
clone.dataInputs = new HashMap<>();
clone.disiInputs = new HashMap<>();
clone.merging = true;
return clone;
}
static class NormsEntry {
byte bytesPerNorm;
long docsWithFieldOffset;
long docsWithFieldLength;
int numDocsWithField;
long normsOffset;
}
abstract static class DenseNormsIterator extends NumericDocValues {
final int maxDoc;
int doc = -1;
DenseNormsIterator(int maxDoc) {
this.maxDoc = maxDoc;
}
@Override
public int docID() {
return doc;
}
@Override
public int nextDoc() throws IOException {
return advance(doc + 1);
}
@Override
public int advance(int target) throws IOException {
if (target >= maxDoc) {
return doc = NO_MORE_DOCS;
}
return doc = target;
}
@Override
public boolean advanceExact(int target) throws IOException {
this.doc = target;
return true;
}
@Override
public long cost() {
return maxDoc;
}
}
abstract static class SparseNormsIterator extends NumericDocValues {
final IndexedDISI disi;
SparseNormsIterator(IndexedDISI disi) {
this.disi = disi;
}
@Override
public int docID() {
return disi.docID();
}
@Override
public int nextDoc() throws IOException {
return disi.nextDoc();
}
@Override
public int advance(int target) throws IOException {
return disi.advance(target);
}
@Override
public boolean advanceExact(int target) throws IOException {
return disi.advanceExact(target);
}
@Override
public long cost() {
return disi.cost();
}
}
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
FieldInfo info = infos.fieldInfo(fieldNumber);
if (info == null) {
throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
} else if (!info.hasNorms()) {
throw new CorruptIndexException("Invalid field: " + info.name, meta);
}
NormsEntry entry = new NormsEntry();
entry.docsWithFieldOffset = meta.readLong();
entry.docsWithFieldLength = meta.readLong();
entry.numDocsWithField = meta.readInt();
entry.bytesPerNorm = meta.readByte();
switch (entry.bytesPerNorm) {
case 0:
case 1:
case 2:
case 4:
case 8:
break;
default:
throw new CorruptIndexException(
"Invalid bytesPerValue: " + entry.bytesPerNorm + ", field: " + info.name, meta);
}
entry.normsOffset = meta.readLong();
norms.put(info.number, entry);
}
}
private RandomAccessInput getDataInput(FieldInfo field, NormsEntry entry) throws IOException {
RandomAccessInput slice = null;
if (merging) {
slice = dataInputs.get(field.number);
}
if (slice == null) {
slice =
data.randomAccessSlice(
entry.normsOffset, entry.numDocsWithField * (long) entry.bytesPerNorm);
if (merging) {
dataInputs.put(field.number, slice);
}
}
return slice;
}
private IndexInput getDisiInput(FieldInfo field, NormsEntry entry) throws IOException {
if (merging == false) {
return data.slice("docs", entry.docsWithFieldOffset, entry.docsWithFieldLength);
}
IndexInput in = disiInputs.get(field.number);
if (in == null) {
in = data.slice("docs", entry.docsWithFieldOffset, entry.docsWithFieldLength);
disiInputs.put(field.number, in);
}
final IndexInput inF = in; // same as in but final
// Wrap so that reads can be interleaved from the same thread if two
// norms instances are pulled and consumed in parallel. Merging usually
// doesn't need this feature but CheckIndex might, plus we need merge
// instances to behave well and not be trappy.
return new IndexInput("docs") {
long offset = 0;
@Override
public void readBytes(byte[] b, int off, int len) throws IOException {
inF.seek(offset);
offset += len;
inF.readBytes(b, off, len);
}
@Override
public byte readByte() throws IOException {
throw new UnsupportedOperationException("Unused by IndexedDISI");
}
@Override
public IndexInput slice(String sliceDescription, long offset, long length)
throws IOException {
throw new UnsupportedOperationException("Unused by IndexedDISI");
}
@Override
public short readShort() throws IOException {
inF.seek(offset);
offset += Short.BYTES;
return inF.readShort();
}
@Override
public long readLong() throws IOException {
inF.seek(offset);
offset += Long.BYTES;
return inF.readLong();
}
@Override
public void seek(long pos) throws IOException {
offset = pos;
}
@Override
public long length() {
throw new UnsupportedOperationException("Unused by IndexedDISI");
}
@Override
public long getFilePointer() {
return offset;
}
@Override
public void close() throws IOException {
throw new UnsupportedOperationException("Unused by IndexedDISI");
}
};
}
@Override
public NumericDocValues getNorms(FieldInfo field) throws IOException {
final NormsEntry entry = norms.get(field.number);
if (entry.docsWithFieldOffset == -2) {
// empty
return DocValues.emptyNumeric();
} else if (entry.docsWithFieldOffset == -1) {
// dense
if (entry.bytesPerNorm == 0) {
return new DenseNormsIterator(maxDoc) {
@Override
public long longValue() throws IOException {
return entry.normsOffset;
}
};
}
final RandomAccessInput slice = getDataInput(field, entry);
switch (entry.bytesPerNorm) {
case 1:
return new DenseNormsIterator(maxDoc) {
@Override
public long longValue() throws IOException {
return slice.readByte(doc);
}
};
case 2:
return new DenseNormsIterator(maxDoc) {
@Override
public long longValue() throws IOException {
return slice.readShort(((long) doc) << 1);
}
};
case 4:
return new DenseNormsIterator(maxDoc) {
@Override
public long longValue() throws IOException {
return slice.readInt(((long) doc) << 2);
}
};
case 8:
return new DenseNormsIterator(maxDoc) {
@Override
public long longValue() throws IOException {
return slice.readLong(((long) doc) << 3);
}
};
default:
// should not happen, we already validate bytesPerNorm in readFields
throw new AssertionError();
}
} else {
// sparse
final IndexInput disiInput = getDisiInput(field, entry);
final IndexedDISI disi = new IndexedDISI(disiInput, entry.numDocsWithField);
if (entry.bytesPerNorm == 0) {
return new SparseNormsIterator(disi) {
@Override
public long longValue() throws IOException {
return entry.normsOffset;
}
};
}
final RandomAccessInput slice =
data.randomAccessSlice(
entry.normsOffset, entry.numDocsWithField * (long) entry.bytesPerNorm);
switch (entry.bytesPerNorm) {
case 1:
return new SparseNormsIterator(disi) {
@Override
public long longValue() throws IOException {
return slice.readByte(disi.index());
}
};
case 2:
return new SparseNormsIterator(disi) {
@Override
public long longValue() throws IOException {
return slice.readShort(((long) disi.index()) << 1);
}
};
case 4:
return new SparseNormsIterator(disi) {
@Override
public long longValue() throws IOException {
return slice.readInt(((long) disi.index()) << 2);
}
};
case 8:
return new SparseNormsIterator(disi) {
@Override
public long longValue() throws IOException {
return slice.readLong(((long) disi.index()) << 3);
}
};
default:
// should not happen, we already validate bytesPerNorm in readFields
throw new AssertionError();
}
}
}
@Override
public void close() throws IOException {
data.close();
}
@Override
public void checkIntegrity() throws IOException {
CodecUtil.checksumEntireFile(data);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + norms.size() + ")";
}
}

View File

@ -17,4 +17,3 @@ org.apache.lucene.backward_codecs.lucene80.Lucene80Codec
org.apache.lucene.backward_codecs.lucene84.Lucene84Codec
org.apache.lucene.backward_codecs.lucene86.Lucene86Codec
org.apache.lucene.backward_codecs.lucene87.Lucene87Codec
org.apache.lucene.backward_codecs.lucene70.Lucene70Codec

View File

@ -13,5 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
org.apache.lucene.backward_codecs.lucene70.Lucene70DocValuesFormat
org.apache.lucene.backward_codecs.lucene80.Lucene80DocValuesFormat

View File

@ -1,170 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.backward_codecs.lucene70;
import static org.apache.lucene.backward_codecs.lucene70.Lucene70NormsFormat.VERSION_CURRENT;
import java.io.IOException;
import org.apache.lucene.backward_codecs.store.EndiannessReverserUtil;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.IOUtils;
/** Writer for {@link Lucene70NormsFormat} */
final class Lucene70NormsConsumer extends NormsConsumer {
IndexOutput data, meta;
final int maxDoc;
Lucene70NormsConsumer(
SegmentWriteState state,
String dataCodec,
String dataExtension,
String metaCodec,
String metaExtension)
throws IOException {
boolean success = false;
try {
String dataName =
IndexFileNames.segmentFileName(
state.segmentInfo.name, state.segmentSuffix, dataExtension);
data = EndiannessReverserUtil.createOutput(state.directory, dataName, state.context);
CodecUtil.writeIndexHeader(
data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
String metaName =
IndexFileNames.segmentFileName(
state.segmentInfo.name, state.segmentSuffix, metaExtension);
meta = EndiannessReverserUtil.createOutput(state.directory, metaName, state.context);
CodecUtil.writeIndexHeader(
meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
maxDoc = state.segmentInfo.maxDoc();
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(this);
}
}
}
@Override
public void close() throws IOException {
boolean success = false;
try {
if (meta != null) {
meta.writeInt(-1); // write EOF marker
CodecUtil.writeFooter(meta); // write checksum
}
if (data != null) {
CodecUtil.writeFooter(data); // write checksum
}
success = true;
} finally {
if (success) {
IOUtils.close(data, meta);
} else {
IOUtils.closeWhileHandlingException(data, meta);
}
meta = data = null;
}
}
@Override
public void addNormsField(FieldInfo field, NormsProducer normsProducer) throws IOException {
NumericDocValues values = normsProducer.getNorms(field);
int numDocsWithValue = 0;
long min = Long.MAX_VALUE;
long max = Long.MIN_VALUE;
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
numDocsWithValue++;
long v = values.longValue();
min = Math.min(min, v);
max = Math.max(max, v);
}
assert numDocsWithValue <= maxDoc;
meta.writeInt(field.number);
if (numDocsWithValue == 0) {
meta.writeLong(-2);
meta.writeLong(0L);
} else if (numDocsWithValue == maxDoc) {
meta.writeLong(-1);
meta.writeLong(0L);
} else {
long offset = data.getFilePointer();
meta.writeLong(offset);
values = normsProducer.getNorms(field);
IndexedDISI.writeBitSet(values, data);
meta.writeLong(data.getFilePointer() - offset);
}
meta.writeInt(numDocsWithValue);
int numBytesPerValue = numBytesPerValue(min, max);
meta.writeByte((byte) numBytesPerValue);
if (numBytesPerValue == 0) {
meta.writeLong(min);
} else {
meta.writeLong(data.getFilePointer());
values = normsProducer.getNorms(field);
writeValues(values, numBytesPerValue, data);
}
}
private int numBytesPerValue(long min, long max) {
if (min >= max) {
return 0;
} else if (min >= Byte.MIN_VALUE && max <= Byte.MAX_VALUE) {
return 1;
} else if (min >= Short.MIN_VALUE && max <= Short.MAX_VALUE) {
return 2;
} else if (min >= Integer.MIN_VALUE && max <= Integer.MAX_VALUE) {
return 4;
} else {
return 8;
}
}
private void writeValues(NumericDocValues values, int numBytesPerValue, IndexOutput out)
throws IOException, AssertionError {
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
long value = values.longValue();
switch (numBytesPerValue) {
case 1:
out.writeByte((byte) value);
break;
case 2:
out.writeShort((short) value);
break;
case 4:
out.writeInt((int) value);
break;
case 8:
out.writeLong(value);
break;
default:
throw new AssertionError();
}
}
}
}

View File

@ -1,75 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.backward_codecs.lucene70;
import org.apache.lucene.backward_codecs.lucene50.Lucene50RWCompoundFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50RWPostingsFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50RWStoredFieldsFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50RWTermVectorsFormat;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
/** RW impersonation of {@link Lucene70Codec}. */
public final class Lucene70RWCodec extends Lucene70Codec {
private final PostingsFormat defaultPF = new Lucene50RWPostingsFormat();
private final PostingsFormat postingsFormat =
new PerFieldPostingsFormat() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return defaultPF;
}
};
/** Sole constructor. */
public Lucene70RWCodec() {}
@Override
public SegmentInfoFormat segmentInfoFormat() {
return new Lucene70RWSegmentInfoFormat();
}
@Override
public NormsFormat normsFormat() {
return new Lucene70RWNormsFormat();
}
@Override
public StoredFieldsFormat storedFieldsFormat() {
return new Lucene50RWStoredFieldsFormat();
}
@Override
public PostingsFormat postingsFormat() {
return postingsFormat;
}
@Override
public CompoundFormat compoundFormat() {
return new Lucene50RWCompoundFormat();
}
@Override
public TermVectorsFormat termVectorsFormat() {
return new Lucene50RWTermVectorsFormat();
}
}

View File

@ -1,30 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.backward_codecs.lucene70;
import java.io.IOException;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.index.SegmentWriteState;
public class Lucene70RWNormsFormat extends Lucene70NormsFormat {
@Override
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
return new Lucene70NormsConsumer(
state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
}
}

View File

@ -1,696 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.backward_codecs.lucene70;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.TreeSet;
import java.util.function.LongSupplier;
import java.util.function.Supplier;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.asserting.AssertingCodec;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SerialMergeScheduler;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.ByteBuffersDataInput;
import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.TestUtil;
/** Tests Lucene70DocValuesFormat */
public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene70DocValuesFormat());
@Override
protected Codec getCodec() {
return codec;
}
// TODO: these big methods can easily blow up some of the other ram-hungry codecs...
// for now just keep them here, as we want to test this for this format.
@Slow
public void testSortedSetVariableLengthBigVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
int numDocs = TEST_NIGHTLY ? atLeast(100) : atLeast(10);
doTestSortedSetVsStoredFields(numDocs, 1, 32766, 16, 100);
}
}
@Nightly
public void testSortedSetVariableLengthManyVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestSortedSetVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500, 16, 100);
}
}
@Slow
public void testSortedVariableLengthBigVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestSortedVsStoredFields(atLeast(100), 1d, 1, 32766);
}
}
@Nightly
public void testSortedVariableLengthManyVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestSortedVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1d, 1, 500);
}
}
@Nightly
public void testTermsEnumFixedWidth() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestTermsEnumRandom(
TestUtil.nextInt(random(), 1025, 5121),
() -> TestUtil.randomSimpleString(random(), 10, 10));
}
}
@Nightly
public void testTermsEnumVariableWidth() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestTermsEnumRandom(
TestUtil.nextInt(random(), 1025, 5121),
() -> TestUtil.randomSimpleString(random(), 1, 500));
}
}
@Nightly
public void testTermsEnumRandomMany() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestTermsEnumRandom(
TestUtil.nextInt(random(), 1025, 8121),
() -> TestUtil.randomSimpleString(random(), 1, 500));
}
}
@Nightly
public void testTermsEnumLongSharedPrefixes() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestTermsEnumRandom(
TestUtil.nextInt(random(), 1025, 5121),
() -> {
char[] chars = new char[random().nextInt(500)];
Arrays.fill(chars, 'a');
if (chars.length > 0) {
chars[random().nextInt(chars.length)] = 'b';
}
return new String(chars);
});
}
}
@Slow
public void testSparseDocValuesVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestSparseDocValuesVsStoredFields();
}
}
private void doTestSparseDocValuesVsStoredFields() throws Exception {
final long[] values = new long[TestUtil.nextInt(random(), 1, 500)];
for (int i = 0; i < values.length; ++i) {
values[i] = random().nextLong();
}
Directory dir = newFSDirectory(createTempDir());
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
conf.setMergeScheduler(new SerialMergeScheduler());
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
// sparse compression is only enabled if less than 1% of docs have a value
final int avgGap = 100;
final int numDocs = atLeast(200);
for (int i = random().nextInt(avgGap * 2); i >= 0; --i) {
writer.addDocument(new Document());
}
final int maxNumValuesPerDoc = random().nextBoolean() ? 1 : TestUtil.nextInt(random(), 2, 5);
for (int i = 0; i < numDocs; ++i) {
Document doc = new Document();
// single-valued
long docValue = values[random().nextInt(values.length)];
doc.add(new NumericDocValuesField("numeric", docValue));
doc.add(new SortedDocValuesField("sorted", new BytesRef(Long.toString(docValue))));
doc.add(new BinaryDocValuesField("binary", new BytesRef(Long.toString(docValue))));
doc.add(new StoredField("value", docValue));
// multi-valued
final int numValues = TestUtil.nextInt(random(), 1, maxNumValuesPerDoc);
for (int j = 0; j < numValues; ++j) {
docValue = values[random().nextInt(values.length)];
doc.add(new SortedNumericDocValuesField("sorted_numeric", docValue));
doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef(Long.toString(docValue))));
doc.add(new StoredField("values", docValue));
}
writer.addDocument(doc);
// add a gap
for (int j = TestUtil.nextInt(random(), 0, avgGap * 2); j >= 0; --j) {
writer.addDocument(new Document());
}
}
if (random().nextBoolean()) {
writer.forceMerge(1);
}
final IndexReader indexReader = writer.getReader();
writer.close();
for (LeafReaderContext context : indexReader.leaves()) {
final LeafReader reader = context.reader();
final NumericDocValues numeric = DocValues.getNumeric(reader, "numeric");
final SortedDocValues sorted = DocValues.getSorted(reader, "sorted");
final BinaryDocValues binary = DocValues.getBinary(reader, "binary");
final SortedNumericDocValues sortedNumeric =
DocValues.getSortedNumeric(reader, "sorted_numeric");
final SortedSetDocValues sortedSet = DocValues.getSortedSet(reader, "sorted_set");
for (int i = 0; i < reader.maxDoc(); ++i) {
final Document doc = reader.document(i);
final IndexableField valueField = doc.getField("value");
final Long value = valueField == null ? null : valueField.numericValue().longValue();
if (value == null) {
assertTrue(numeric.docID() + " vs " + i, numeric.docID() < i);
} else {
assertEquals(i, numeric.nextDoc());
assertEquals(i, binary.nextDoc());
assertEquals(i, sorted.nextDoc());
assertEquals(value.longValue(), numeric.longValue());
assertTrue(sorted.ordValue() >= 0);
assertEquals(new BytesRef(Long.toString(value)), sorted.lookupOrd(sorted.ordValue()));
assertEquals(new BytesRef(Long.toString(value)), binary.binaryValue());
}
final IndexableField[] valuesFields = doc.getFields("values");
if (valuesFields.length == 0) {
assertTrue(sortedNumeric.docID() + " vs " + i, sortedNumeric.docID() < i);
} else {
final Set<Long> valueSet = new HashSet<>();
for (IndexableField sf : valuesFields) {
valueSet.add(sf.numericValue().longValue());
}
assertEquals(i, sortedNumeric.nextDoc());
assertEquals(valuesFields.length, sortedNumeric.docValueCount());
for (int j = 0; j < sortedNumeric.docValueCount(); ++j) {
assertTrue(valueSet.contains(sortedNumeric.nextValue()));
}
assertEquals(i, sortedSet.nextDoc());
int sortedSetCount = 0;
while (true) {
long ord = sortedSet.nextOrd();
if (ord == SortedSetDocValues.NO_MORE_ORDS) {
break;
}
assertTrue(valueSet.contains(Long.parseLong(sortedSet.lookupOrd(ord).utf8ToString())));
sortedSetCount++;
}
assertEquals(valueSet.size(), sortedSetCount);
}
}
}
indexReader.close();
dir.close();
}
// TODO: try to refactor this and some termsenum tests into the base class.
// to do this we need to fix the test class to get a DVF not a Codec so we can setup
// the postings format correctly.
private void doTestTermsEnumRandom(int numDocs, Supplier<String> valuesProducer)
throws Exception {
Directory dir = newFSDirectory(createTempDir());
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
conf.setMergeScheduler(new SerialMergeScheduler());
// set to duel against a codec which has ordinals:
final PostingsFormat pf = TestUtil.getPostingsFormatWithOrds(random());
final DocValuesFormat dv = new Lucene70DocValuesFormat();
conf.setCodec(
new AssertingCodec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return pf;
}
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return dv;
}
});
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
// index some docs
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
Field idField = new StringField("id", Integer.toString(i), Field.Store.NO);
doc.add(idField);
int numValues = random().nextInt(17);
// create a random list of strings
List<String> values = new ArrayList<>();
for (int v = 0; v < numValues; v++) {
values.add(valuesProducer.get());
}
// add in any order to the indexed field
ArrayList<String> 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<String> unordered2 = new ArrayList<>(values);
Collections.shuffle(unordered2, random());
for (String v : unordered2) {
doc.add(new SortedSetDocValuesField("dv", new BytesRef(v)));
}
writer.addDocument(doc);
if (random().nextInt(31) == 0) {
writer.commit();
}
}
// delete some docs
int numDeletions = random().nextInt(numDocs / 10);
for (int i = 0; i < numDeletions; i++) {
int id = random().nextInt(numDocs);
writer.deleteDocuments(new Term("id", Integer.toString(id)));
}
// compare per-segment
DirectoryReader ir = writer.getReader();
for (LeafReaderContext context : ir.leaves()) {
LeafReader r = context.reader();
Terms terms = r.terms("indexed");
if (terms != null) {
SortedSetDocValues ssdv = r.getSortedSetDocValues("dv");
assertEquals(terms.size(), ssdv.getValueCount());
TermsEnum expected = terms.iterator();
TermsEnum actual = r.getSortedSetDocValues("dv").termsEnum();
assertEquals(terms.size(), expected, actual);
doTestSortedSetEnumAdvanceIndependently(ssdv);
}
}
ir.close();
writer.forceMerge(1);
// now compare again after the merge
ir = writer.getReader();
LeafReader ar = getOnlyLeafReader(ir);
Terms terms = ar.terms("indexed");
if (terms != null) {
assertEquals(terms.size(), ar.getSortedSetDocValues("dv").getValueCount());
TermsEnum expected = terms.iterator();
TermsEnum actual = ar.getSortedSetDocValues("dv").termsEnum();
assertEquals(terms.size(), expected, actual);
}
ir.close();
writer.close();
dir.close();
}
private void assertEquals(long numOrds, TermsEnum expected, TermsEnum actual) throws Exception {
BytesRef ref;
// sequential next() through all terms
while ((ref = expected.next()) != null) {
assertEquals(ref, actual.next());
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
assertNull(actual.next());
// sequential seekExact(ord) through all terms
for (long i = 0; i < numOrds; i++) {
expected.seekExact(i);
actual.seekExact(i);
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
// sequential seekExact(BytesRef) through all terms
for (long i = 0; i < numOrds; i++) {
expected.seekExact(i);
assertTrue(actual.seekExact(expected.term()));
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
// sequential seekCeil(BytesRef) through all terms
for (long i = 0; i < numOrds; i++) {
expected.seekExact(i);
assertEquals(SeekStatus.FOUND, actual.seekCeil(expected.term()));
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
// random seekExact(ord)
for (long i = 0; i < numOrds; i++) {
long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
expected.seekExact(randomOrd);
actual.seekExact(randomOrd);
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
// random seekExact(BytesRef)
for (long i = 0; i < numOrds; i++) {
long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
expected.seekExact(randomOrd);
actual.seekExact(expected.term());
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
// random seekCeil(BytesRef)
for (long i = 0; i < numOrds; i++) {
BytesRef target = new BytesRef(TestUtil.randomUnicodeString(random()));
SeekStatus expectedStatus = expected.seekCeil(target);
assertEquals(expectedStatus, actual.seekCeil(target));
if (expectedStatus != SeekStatus.END) {
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
}
}
@Nightly
public void testSortedSetAroundBlockSize() throws IOException {
final int frontier = 1 << Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
final Directory dir = newDirectory();
IndexWriter w =
new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
ByteBuffersDataOutput out = new ByteBuffersDataOutput();
Document doc = new Document();
SortedSetDocValuesField field1 = new SortedSetDocValuesField("sset", new BytesRef());
doc.add(field1);
SortedSetDocValuesField field2 = new SortedSetDocValuesField("sset", new BytesRef());
doc.add(field2);
for (int i = 0; i < maxDoc; ++i) {
BytesRef s1 = new BytesRef(TestUtil.randomSimpleString(random(), 2));
BytesRef s2 = new BytesRef(TestUtil.randomSimpleString(random(), 2));
field1.setBytesValue(s1);
field2.setBytesValue(s2);
w.addDocument(doc);
Set<BytesRef> set = new TreeSet<>(Arrays.asList(s1, s2));
out.writeVInt(set.size());
for (BytesRef ref : set) {
out.writeVInt(ref.length);
out.writeBytes(ref.bytes, ref.offset, ref.length);
}
}
w.forceMerge(1);
DirectoryReader r = DirectoryReader.open(w);
w.close();
LeafReader sr = getOnlyLeafReader(r);
assertEquals(maxDoc, sr.maxDoc());
SortedSetDocValues values = sr.getSortedSetDocValues("sset");
assertNotNull(values);
ByteBuffersDataInput in = new ByteBuffersDataInput(out.toBufferList());
BytesRefBuilder b = new BytesRefBuilder();
for (int i = 0; i < maxDoc; ++i) {
assertEquals(i, values.nextDoc());
final int numValues = in.readVInt();
for (int j = 0; j < numValues; ++j) {
b.setLength(in.readVInt());
b.grow(b.length());
in.readBytes(b.bytes(), 0, b.length());
assertEquals(b.get(), values.lookupOrd(values.nextOrd()));
}
assertEquals(SortedSetDocValues.NO_MORE_ORDS, values.nextOrd());
}
r.close();
dir.close();
}
}
@Nightly
public void testSortedNumericAroundBlockSize() throws IOException {
final int frontier = 1 << Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
final Directory dir = newDirectory();
IndexWriter w =
new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
ByteBuffersDataOutput out = new ByteBuffersDataOutput();
Document doc = new Document();
SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("snum", 0L);
doc.add(field1);
SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("snum", 0L);
doc.add(field2);
for (int i = 0; i < maxDoc; ++i) {
long s1 = random().nextInt(100);
long s2 = random().nextInt(100);
field1.setLongValue(s1);
field2.setLongValue(s2);
w.addDocument(doc);
out.writeVLong(Math.min(s1, s2));
out.writeVLong(Math.max(s1, s2));
}
w.forceMerge(1);
DirectoryReader r = DirectoryReader.open(w);
w.close();
LeafReader sr = getOnlyLeafReader(r);
assertEquals(maxDoc, sr.maxDoc());
SortedNumericDocValues values = sr.getSortedNumericDocValues("snum");
assertNotNull(values);
ByteBuffersDataInput in = new ByteBuffersDataInput(out.toBufferList());
for (int i = 0; i < maxDoc; ++i) {
assertEquals(i, values.nextDoc());
assertEquals(2, values.docValueCount());
assertEquals(in.readVLong(), values.nextValue());
assertEquals(in.readVLong(), values.nextValue());
}
r.close();
dir.close();
}
}
@Nightly
public void testSortedNumericBlocksOfVariousBitsPerValue() throws Exception {
doTestSortedNumericBlocksOfVariousBitsPerValue(() -> TestUtil.nextInt(random(), 1, 3));
}
@Nightly
public void testSparseSortedNumericBlocksOfVariousBitsPerValue() throws Exception {
doTestSortedNumericBlocksOfVariousBitsPerValue(() -> TestUtil.nextInt(random(), 0, 2));
}
@Nightly
public void testNumericBlocksOfVariousBitsPerValue() throws Exception {
doTestSparseNumericBlocksOfVariousBitsPerValue(1);
}
@Nightly
public void testSparseNumericBlocksOfVariousBitsPerValue() throws Exception {
doTestSparseNumericBlocksOfVariousBitsPerValue(random().nextDouble());
}
private static LongSupplier blocksOfVariousBPV() {
final long mul = TestUtil.nextInt(random(), 1, 100);
final long min = random().nextInt();
return new LongSupplier() {
int i = Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE;
int maxDelta;
@Override
public long getAsLong() {
if (i == Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE) {
maxDelta = 1 << random().nextInt(5);
i = 0;
}
i++;
return min + mul * random().nextInt(maxDelta);
}
};
}
private void doTestSortedNumericBlocksOfVariousBitsPerValue(LongSupplier counts)
throws Exception {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
conf.setMaxBufferedDocs(atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE));
conf.setRAMBufferSizeMB(-1);
conf.setMergePolicy(newLogMergePolicy(random().nextBoolean()));
IndexWriter writer = new IndexWriter(dir, conf);
final int numDocs = atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE * 3);
final LongSupplier values = blocksOfVariousBPV();
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
int valueCount = (int) counts.getAsLong();
long[] valueArray = new long[valueCount];
for (int j = 0; j < valueCount; j++) {
long value = values.getAsLong();
valueArray[j] = value;
doc.add(new SortedNumericDocValuesField("dv", value));
}
Arrays.sort(valueArray);
for (int j = 0; j < valueCount; j++) {
doc.add(new StoredField("stored", Long.toString(valueArray[j])));
}
writer.addDocument(doc);
if (random().nextInt(31) == 0) {
writer.commit();
}
}
writer.forceMerge(1);
writer.close();
// compare
DirectoryReader ir = DirectoryReader.open(dir);
TestUtil.checkReader(ir);
for (LeafReaderContext context : ir.leaves()) {
LeafReader r = context.reader();
SortedNumericDocValues docValues = DocValues.getSortedNumeric(r, "dv");
for (int i = 0; i < r.maxDoc(); i++) {
if (i > docValues.docID()) {
docValues.nextDoc();
}
String[] expected = r.document(i).getValues("stored");
if (i < docValues.docID()) {
assertEquals(0, expected.length);
} else {
String[] actual = new String[docValues.docValueCount()];
for (int j = 0; j < actual.length; j++) {
actual[j] = Long.toString(docValues.nextValue());
}
assertArrayEquals(expected, actual);
}
}
}
ir.close();
dir.close();
}
private void doTestSparseNumericBlocksOfVariousBitsPerValue(double density) throws Exception {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
conf.setMaxBufferedDocs(atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE));
conf.setRAMBufferSizeMB(-1);
conf.setMergePolicy(newLogMergePolicy(random().nextBoolean()));
IndexWriter writer = new IndexWriter(dir, conf);
Document doc = new Document();
Field storedField = newStringField("stored", "", Field.Store.YES);
Field dvField = new NumericDocValuesField("dv", 0);
doc.add(storedField);
doc.add(dvField);
final int numDocs = atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE * 3);
final LongSupplier longs = blocksOfVariousBPV();
for (int i = 0; i < numDocs; i++) {
if (random().nextDouble() > density) {
writer.addDocument(new Document());
continue;
}
long value = longs.getAsLong();
storedField.setStringValue(Long.toString(value));
dvField.setLongValue(value);
writer.addDocument(doc);
}
writer.forceMerge(1);
writer.close();
// compare
DirectoryReader ir = DirectoryReader.open(dir);
TestUtil.checkReader(ir);
for (LeafReaderContext context : ir.leaves()) {
LeafReader r = context.reader();
NumericDocValues docValues = DocValues.getNumeric(r, "dv");
docValues.nextDoc();
for (int i = 0; i < r.maxDoc(); i++) {
String storedValue = r.document(i).get("stored");
if (storedValue == null) {
assertTrue(docValues.docID() > i);
} else {
assertEquals(i, docValues.docID());
assertEquals(Long.parseLong(storedValue), docValues.longValue());
docValues.nextDoc();
}
}
assertEquals(DocIdSetIterator.NO_MORE_DOCS, docValues.docID());
}
ir.close();
dir.close();
}
}

View File

@ -1,28 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.backward_codecs.lucene70;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BaseNormsFormatTestCase;
/** Tests Lucene70NormsFormat */
public class TestLucene70NormsFormat extends BaseNormsFormatTestCase {
@Override
protected Codec getCodec() {
return new Lucene70RWCodec();
}
}

View File

@ -22,12 +22,11 @@ import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BaseSegmentInfoFormatTestCase;
import org.apache.lucene.util.Version;
@SuppressWarnings("deprecation")
public class TestLucene70SegmentInfoFormat extends BaseSegmentInfoFormatTestCase {
@Override
protected Version[] getVersions() {
return new Version[] {Version.LUCENE_8_4_0};
return new Version[] {Version.fromBits(8, 4, 0)};
}
@Override

View File

@ -27,7 +27,7 @@ public class TestLucene86SegmentInfoFormat extends BaseSegmentInfoFormatTestCase
@Override
protected Version[] getVersions() {
return new Version[] {Version.LUCENE_8_8_1};
return new Version[] {Version.fromBits(8, 8, 1)};
}
@Override

View File

@ -332,81 +332,13 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
}
}
static final String[] oldNames = {
"8.0.0-cfs",
"8.0.0-nocfs",
"8.1.0-cfs",
"8.1.0-nocfs",
"8.1.1-cfs",
"8.1.1-nocfs",
"8.2.0-cfs",
"8.2.0-nocfs",
"8.3.0-cfs",
"8.3.0-nocfs",
"8.3.1-cfs",
"8.3.1-nocfs",
"8.4.0-cfs",
"8.4.0-nocfs",
"8.4.1-cfs",
"8.4.1-nocfs",
"8.5.0-cfs",
"8.5.0-nocfs",
"8.5.1-cfs",
"8.5.1-nocfs",
"8.5.2-cfs",
"8.5.2-nocfs",
"8.6.0-cfs",
"8.6.0-nocfs",
"8.6.1-cfs",
"8.6.1-nocfs",
"8.6.2-cfs",
"8.6.2-nocfs",
"8.6.3-cfs",
"8.6.3-nocfs",
"8.7.0-cfs",
"8.7.0-nocfs",
"8.8.0-cfs",
"8.8.0-nocfs",
"8.8.1-cfs",
"8.8.1-nocfs",
"8.8.2-cfs",
"8.8.2-nocfs",
"8.9.0-cfs",
"8.9.0-nocfs",
"8.10.0-cfs",
"8.10.0-nocfs",
"8.10.1-cfs",
"8.10.1-nocfs"
};
static final String[] oldNames = {};
public static String[] getOldNames() {
return oldNames;
}
static final String[] oldSortedNames = {
"sorted.8.0.0",
"sorted.8.1.0",
"sorted.8.1.1",
"sorted.8.10.0",
"sorted.8.2.0",
"sorted.8.3.0",
"sorted.8.3.1",
"sorted.8.4.0",
"sorted.8.4.1",
"sorted.8.5.0",
"sorted.8.5.1",
"sorted.8.5.2",
"sorted.8.6.0",
"sorted.8.6.1",
"sorted.8.6.2",
"sorted.8.6.3",
"sorted.8.7.0",
"sorted.8.8.0",
"sorted.8.8.1",
"sorted.8.8.2",
"sorted.8.9.0",
"sorted.8.10.1"
};
static final String[] oldSortedNames = {};
public static String[] getOldSortedNames() {
return oldSortedNames;
@ -609,7 +541,51 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
"7.7.2-cfs",
"7.7.2-nocfs",
"7.7.3-cfs",
"7.7.3-nocfs"
"7.7.3-nocfs",
"8.0.0-cfs",
"8.0.0-nocfs",
"8.1.0-cfs",
"8.1.0-nocfs",
"8.1.1-cfs",
"8.1.1-nocfs",
"8.2.0-cfs",
"8.2.0-nocfs",
"8.3.0-cfs",
"8.3.0-nocfs",
"8.3.1-cfs",
"8.3.1-nocfs",
"8.4.0-cfs",
"8.4.0-nocfs",
"8.4.1-cfs",
"8.4.1-nocfs",
"8.5.0-cfs",
"8.5.0-nocfs",
"8.5.1-cfs",
"8.5.1-nocfs",
"8.5.2-cfs",
"8.5.2-nocfs",
"8.6.0-cfs",
"8.6.0-nocfs",
"8.6.1-cfs",
"8.6.1-nocfs",
"8.6.2-cfs",
"8.6.2-nocfs",
"8.6.3-cfs",
"8.6.3-nocfs",
"8.7.0-cfs",
"8.7.0-nocfs",
"8.8.0-cfs",
"8.8.0-nocfs",
"8.8.1-cfs",
"8.8.1-nocfs",
"8.8.2-cfs",
"8.8.2-nocfs",
"8.9.0-cfs",
"8.9.0-nocfs",
"8.10.0-cfs",
"8.10.0-nocfs",
"8.10.1-cfs",
"8.10.1-nocfs"
};
static final int MIN_BINARY_SUPPORTED_MAJOR = Version.MIN_SUPPORTED_MAJOR - 1;
@ -904,12 +880,17 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
CheckIndex checker = new CheckIndex(dir);
checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8));
CheckIndex.Status indexStatus = checker.checkIndex();
if (unsupportedNames[i].startsWith("7.")) {
if (unsupportedNames[i].startsWith("8.")) {
assertTrue(indexStatus.clean);
} else {
assertFalse(indexStatus.clean);
assertTrue(
bos.toString(IOUtils.UTF_8).contains(IndexFormatTooOldException.class.getName()));
// CheckIndex doesn't enforce a minimum version, so we either get an
// IndexFormatTooOldException
// or an IllegalArgumentException saying that the codec doesn't exist.
boolean formatTooOld =
bos.toString(IOUtils.UTF_8).contains(IndexFormatTooOldException.class.getName());
boolean missingCodec = bos.toString(IOUtils.UTF_8).contains("Could not load codec");
assertTrue(formatTooOld || missingCodec);
}
checker.close();
@ -982,7 +963,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
.filter( // depending on the MergePolicy we might see these segments merged away
sci ->
sci.getId() != null
&& sci.info.getVersion().onOrAfter(Version.LUCENE_8_6_0) == false)
&& sci.info.getVersion().onOrAfter(Version.fromBits(8, 6, 0)) == false)
.findAny()
.orElse(null));
if (VERBOSE) {
@ -1589,7 +1570,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
Directory dir = oldIndexDirs.get(name);
SegmentInfos infos = SegmentInfos.readLatestCommit(dir);
for (SegmentCommitInfo info : infos) {
if (info.info.getVersion().onOrAfter(Version.LUCENE_8_6_0)) {
if (info.info.getVersion().onOrAfter(Version.fromBits(8, 6, 0))) {
assertNotNull(info.toString(), info.getId());
} else {
assertNull(info.toString(), info.getId());
@ -1767,6 +1748,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
public static final String emptyIndex = "empty.8.0.0.zip";
@LuceneTestCase.AwaitsFix(bugUrl = "Unavailable until 9.0.0 is released")
public void testUpgradeEmptyOldIndex() throws Exception {
Path oldIndexDir = createTempDir("emptyIndex");
TestUtil.unzip(getDataInputStream(emptyIndex), oldIndexDir);
@ -1779,8 +1761,9 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
dir.close();
}
public static final String moreTermsIndex = "moreterms.8.0.0.zip";
public static final String moreTermsIndex = "moreterms.9.0.0.zip";
@LuceneTestCase.AwaitsFix(bugUrl = "Unavailable until 9.0.0 is released")
public void testMoreTerms() throws Exception {
Path oldIndexDir = createTempDir("moreterms");
TestUtil.unzip(getDataInputStream(moreTermsIndex), oldIndexDir);
@ -1795,7 +1778,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
dir.close();
}
public static final String dvUpdatesIndex = "dvupdates.8.0.0.zip";
public static final String dvUpdatesIndex = "dvupdates.9.0.0.zip";
private void assertNumericDocValues(LeafReader r, String f, String cf) throws IOException {
NumericDocValues ndvf = r.getNumericDocValues(f);
@ -1829,6 +1812,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
reader.close();
}
@LuceneTestCase.AwaitsFix(bugUrl = "Unavailable until 9.0.0 is released")
public void testDocValuesUpdates() throws Exception {
Path oldIndexDir = createTempDir("dvupdates");
TestUtil.unzip(getDataInputStream(dvUpdatesIndex), oldIndexDir);
@ -1857,6 +1841,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
dir.close();
}
@LuceneTestCase.AwaitsFix(bugUrl = "Unavailable until 9.0.0 is released")
public void testDeletes() throws Exception {
Path oldIndexDir = createTempDir("dvupdates");
TestUtil.unzip(getDataInputStream(dvUpdatesIndex), oldIndexDir);
@ -1880,6 +1865,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
dir.close();
}
@LuceneTestCase.AwaitsFix(bugUrl = "Unavailable until 9.0.0 is released")
public void testSoftDeletes() throws Exception {
Path oldIndexDir = createTempDir("dvupdates");
TestUtil.unzip(getDataInputStream(dvUpdatesIndex), oldIndexDir);
@ -1901,6 +1887,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
dir.close();
}
@LuceneTestCase.AwaitsFix(bugUrl = "Unavailable until 9.0.0 is released")
public void testDocValuesUpdatesWithNewField() throws Exception {
Path oldIndexDir = createTempDir("dvupdates");
TestUtil.unzip(getDataInputStream(dvUpdatesIndex), oldIndexDir);

View File

@ -112,14 +112,7 @@ import org.apache.lucene.util.Version;
*/
public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo> {
/**
* The version that added information about the Lucene version at the time when the index has been
* created.
*/
public static final int VERSION_70 = 7;
/** The version that updated segment name counter to be long instead of int. */
public static final int VERSION_72 = 8;
/** The version that recorded softDelCount */
/** The version at the time when 8.0 was released. */
public static final int VERSION_74 = 9;
/** The version that recorded SegmentCommitInfo IDs */
public static final int VERSION_86 = 10;
@ -324,7 +317,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
throw new IndexFormatTooOldException(
input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
}
format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_70, VERSION_CURRENT);
format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_74, VERSION_CURRENT);
byte[] id = new byte[StringHelper.ID_LENGTH];
input.readBytes(id, 0, id.length);
CodecUtil.checkIndexHeaderSuffix(input, Long.toString(generation, Character.MAX_RADIX));
@ -366,7 +359,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
} catch (Throwable t) {
priorE = t;
} finally {
if (format >= VERSION_70) { // oldest supported version
if (format >= VERSION_74) { // oldest supported version
CodecUtil.checkFooter(input, priorE);
} else {
throw IOUtils.rethrowAlways(priorE);
@ -379,11 +372,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
Directory directory, DataInput input, SegmentInfos infos, int format) throws IOException {
infos.version = CodecUtil.readBELong(input);
// System.out.println("READ sis version=" + infos.version);
if (format > VERSION_70) {
infos.counter = input.readVLong();
} else {
infos.counter = CodecUtil.readBEInt(input);
}
infos.counter = input.readVLong();
int numSegments = CodecUtil.readBEInt(input);
if (numSegments < 0) {
throw new CorruptIndexException("invalid segment count: " + numSegments, input);
@ -414,7 +403,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
}
long fieldInfosGen = CodecUtil.readBELong(input);
long dvGen = CodecUtil.readBELong(input);
int softDelCount = format > VERSION_72 ? CodecUtil.readBEInt(input) : 0;
int softDelCount = CodecUtil.readBEInt(input);
if (softDelCount < 0 || softDelCount > info.maxDoc()) {
throw new CorruptIndexException(
"invalid deletion count: " + softDelCount + " vs maxDoc=" + info.maxDoc(), input);

View File

@ -28,167 +28,6 @@ import java.util.Locale;
*/
public final class Version {
/**
* Match settings and bugs in Lucene's 8.0.0 release.
*
* @deprecated (9.0.0) Use latest
*/
@Deprecated public static final Version LUCENE_8_0_0 = new Version(8, 0, 0);
/**
* Match settings and bugs in Lucene's 8.1.0 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_1_0 = new Version(8, 1, 0);
/**
* Match settings and bugs in Lucene's 8.1.1 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_1_1 = new Version(8, 1, 1);
/**
* Match settings and bugs in Lucene's 8.2.0 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_2_0 = new Version(8, 2, 0);
/**
* Match settings and bugs in Lucene's 8.3.0 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_3_0 = new Version(8, 3, 0);
/**
* Match settings and bugs in Lucene's 8.3.1 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_3_1 = new Version(8, 3, 1);
/**
* Match settings and bugs in Lucene's 8.4.0 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_4_0 = new Version(8, 4, 0);
/**
* Match settings and bugs in Lucene's 8.4.1 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_4_1 = new Version(8, 4, 1);
/**
* Match settings and bugs in Lucene's 8.5.0 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_5_0 = new Version(8, 5, 0);
/**
* Match settings and bugs in Lucene's 8.5.1 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_5_1 = new Version(8, 5, 1);
/**
* Match settings and bugs in Lucene's 8.5.2 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_5_2 = new Version(8, 5, 2);
/**
* Match settings and bugs in Lucene's 8.6.0 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_6_0 = new Version(8, 6, 0);
/**
* Match settings and bugs in Lucene's 8.6.1 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_6_1 = new Version(8, 6, 1);
/**
* Match settings and bugs in Lucene's 8.6.2 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_6_2 = new Version(8, 6, 2);
/**
* Match settings and bugs in Lucene's 8.6.3 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_6_3 = new Version(8, 6, 3);
/**
* Match settings and bugs in Lucene's 8.7.0 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_7_0 = new Version(8, 7, 0);
/**
* Match settings and bugs in Lucene's 8.8.0 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_8_0 = new Version(8, 8, 0);
/**
* Match settings and bugs in Lucene's 8.8.1 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_8_1 = new Version(8, 8, 1);
/**
* Match settings and bugs in Lucene's 8.8.2 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_8_2 = new Version(8, 8, 2);
/**
* Match settings and bugs in Lucene's 8.9.0 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_9_0 = new Version(8, 9, 0);
/**
* Match settings and bugs in Lucene's 8.10.0 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_10_0 = new Version(8, 10, 0);
/**
* Match settings and bugs in Lucene's 8.10.1 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_10_1 = new Version(8, 10, 1);
/**
* Match settings and bugs in Lucene's 8.10.0 release.
*
* @deprecated Use latest
*/
@Deprecated public static final Version LUCENE_8_11_0 = new Version(8, 11, 0);
/** @deprecated (10.0.0) Use latest */
@Deprecated public static final Version LUCENE_9_0_0 = new Version(9, 0, 0);

View File

@ -71,8 +71,8 @@ public class TestSegmentInfos extends LuceneTestCase {
SegmentInfo info =
new SegmentInfo(
dir,
Version.LUCENE_9_0_0,
Version.LUCENE_9_0_0,
Version.LUCENE_10_0_0,
Version.LUCENE_10_0_0,
"_0",
1,
false,
@ -89,7 +89,7 @@ public class TestSegmentInfos extends LuceneTestCase {
sis.add(commitInfo);
sis.commit(dir);
sis = SegmentInfos.readLatestCommit(dir);
assertEquals(Version.LUCENE_9_0_0, sis.getMinSegmentLuceneVersion());
assertEquals(Version.LUCENE_10_0_0, sis.getMinSegmentLuceneVersion());
assertEquals(Version.LATEST, sis.getCommitLuceneVersion());
dir.close();
}
@ -105,8 +105,8 @@ public class TestSegmentInfos extends LuceneTestCase {
SegmentInfo info =
new SegmentInfo(
dir,
Version.LUCENE_9_0_0,
Version.LUCENE_9_0_0,
Version.LUCENE_10_0_0,
Version.LUCENE_10_0_0,
"_0",
1,
false,
@ -124,8 +124,8 @@ public class TestSegmentInfos extends LuceneTestCase {
info =
new SegmentInfo(
dir,
Version.LUCENE_9_0_0,
Version.LUCENE_9_0_0,
Version.LUCENE_10_0_0,
Version.LUCENE_10_0_0,
"_1",
1,
false,
@ -143,7 +143,7 @@ public class TestSegmentInfos extends LuceneTestCase {
byte[] commitInfoId0 = sis.info(0).getId();
byte[] commitInfoId1 = sis.info(1).getId();
sis = SegmentInfos.readLatestCommit(dir);
assertEquals(Version.LUCENE_9_0_0, sis.getMinSegmentLuceneVersion());
assertEquals(Version.LUCENE_10_0_0, sis.getMinSegmentLuceneVersion());
assertEquals(Version.LATEST, sis.getCommitLuceneVersion());
assertEquals(
StringHelper.idToString(commitInfoId0), StringHelper.idToString(sis.info(0).getId()));

View File

@ -32,26 +32,25 @@ public class TestVersion extends LuceneTestCase {
assertTrue("LATEST must be always onOrAfter(" + v + ")", Version.LATEST.onOrAfter(v));
}
}
assertTrue(Version.LUCENE_9_0_0.onOrAfter(Version.LUCENE_8_0_0));
assertTrue(Version.LUCENE_10_0_0.onOrAfter(Version.LUCENE_9_0_0));
;
}
public void testToString() {
assertEquals("8.0.0", Version.LUCENE_8_0_0.toString());
assertEquals("9.0.0", Version.LUCENE_9_0_0.toString());
assertEquals("10.0.0", Version.LUCENE_10_0_0.toString());
}
public void testParseLeniently() throws Exception {
assertEquals(Version.LUCENE_10_0_0, Version.parseLeniently("10.0"));
assertEquals(Version.LUCENE_10_0_0, Version.parseLeniently("10.0.0"));
assertEquals(Version.LUCENE_10_0_0, Version.parseLeniently("LUCENE_10_0"));
assertEquals(Version.LUCENE_10_0_0, Version.parseLeniently("LUCENE_10_0_0"));
assertEquals(Version.LUCENE_9_0_0, Version.parseLeniently("9.0"));
assertEquals(Version.LUCENE_9_0_0, Version.parseLeniently("9.0.0"));
assertEquals(Version.LUCENE_9_0_0, Version.parseLeniently("LUCENE_90"));
assertEquals(Version.LUCENE_9_0_0, Version.parseLeniently("LUCENE_9_0"));
assertEquals(Version.LUCENE_9_0_0, Version.parseLeniently("LUCENE_9_0_0"));
assertEquals(Version.LUCENE_8_0_0, Version.parseLeniently("8.0"));
assertEquals(Version.LUCENE_8_0_0, Version.parseLeniently("8.0.0"));
assertEquals(Version.LUCENE_8_0_0, Version.parseLeniently("LUCENE_80"));
assertEquals(Version.LUCENE_8_0_0, Version.parseLeniently("LUCENE_8_0"));
assertEquals(Version.LUCENE_8_0_0, Version.parseLeniently("LUCENE_8_0_0"));
assertEquals(Version.LATEST, Version.parseLeniently("LATEST"));
assertEquals(Version.LATEST, Version.parseLeniently("latest"));
@ -108,8 +107,8 @@ public class TestVersion extends LuceneTestCase {
}
public void testParse() throws Exception {
assertEquals(Version.LUCENE_10_0_0, Version.parse("10.0.0"));
assertEquals(Version.LUCENE_9_0_0, Version.parse("9.0.0"));
assertEquals(Version.LUCENE_8_0_0, Version.parse("8.0.0"));
// Version does not pass judgement on the major version:
assertEquals(1, Version.parse("1.0").major);
@ -117,7 +116,7 @@ public class TestVersion extends LuceneTestCase {
}
public void testForwardsCompatibility() throws Exception {
assertTrue(Version.parse("8.10.20").onOrAfter(Version.LUCENE_8_0_0));
assertTrue(Version.parse("9.10.20").onOrAfter(Version.LUCENE_9_0_0));
}
public void testParseExceptions() {

View File

@ -26,7 +26,6 @@ import java.util.Map;
import java.util.function.IntUnaryOperator;
import java.util.logging.Level;
import java.util.logging.Logger;
import org.apache.lucene.document.Document;
import org.apache.lucene.facet.FacetsConfig;
import org.apache.lucene.facet.taxonomy.FacetLabel;
import org.apache.lucene.facet.taxonomy.LRUHashMap;
@ -338,10 +337,7 @@ public class DirectoryTaxonomyReader extends TaxonomyReader implements Accountab
if (values == null
|| values.advanceExact(ordinal - indexReader.leaves().get(readerIndex).docBase) == false) {
// The index uses the older StoredField format to store the mapping
// On recreating the index, the values will be stored using the BinaryDocValuesField format
Document doc = indexReader.document(ordinal);
ret = new FacetLabel(FacetsConfig.stringToPath(doc.get(Consts.FULL)));
throw new IllegalStateException();
} else {
// The index uses the BinaryDocValuesField format to store the mapping
ret = new FacetLabel(FacetsConfig.stringToPath(values.binaryValue().utf8ToString()));

View File

@ -91,7 +91,6 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
private final Directory dir;
private final IndexWriter indexWriter;
private final boolean useOlderStoredFieldIndex;
private final TaxonomyWriterCache cache;
private final AtomicInteger cacheMisses = new AtomicInteger(0);
@ -159,14 +158,10 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
openMode = config.getOpenMode();
if (DirectoryReader.indexExists(directory) == false) {
indexEpoch = 1;
// no commit exists so we can safely use the new BinaryDocValues field
useOlderStoredFieldIndex = false;
} else {
String epochStr = null;
SegmentInfos infos = SegmentInfos.readLatestCommit(dir);
/* a previous commit exists, so check the version of the last commit */
useOlderStoredFieldIndex = infos.getIndexCreatedVersionMajor() <= 8;
Map<String, String> commitData = infos.getUserData();
if (commitData != null) {
@ -184,11 +179,7 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
ft.setOmitNorms(true);
parentStreamField = new Field(Consts.FIELD_PAYLOADS, parentStream, ft);
if (useOlderStoredFieldIndex) {
fullPathField = new StringField(Consts.FULL, "", Field.Store.YES);
} else {
fullPathField = new StringField(Consts.FULL, "", Field.Store.NO);
}
fullPathField = new StringField(Consts.FULL, "", Field.Store.NO);
nextID = indexWriter.getDocStats().maxDoc;
@ -482,10 +473,8 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
String fieldPath = FacetsConfig.pathToString(categoryPath.components, categoryPath.length);
fullPathField.setStringValue(fieldPath);
if (useOlderStoredFieldIndex == false) {
/* Lucene 9 switches to BinaryDocValuesField for storing taxonomy categories */
d.add(new BinaryDocValuesField(Consts.FULL, new BytesRef(fieldPath)));
}
/* Lucene 9 switches to BinaryDocValuesField for storing taxonomy categories */
d.add(new BinaryDocValuesField(Consts.FULL, new BytesRef(fieldPath)));
d.add(fullPathField);

View File

@ -1,141 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.facet.taxonomy.directory;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import org.apache.lucene.facet.taxonomy.FacetLabel;
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.junit.Ignore;
/*
Verify we can read previous versions' taxonomy indexes, do searches
against them, and add documents to them.
*/
// See: https://issues.apache.org/jira/browse/SOLR-12028 Tests cannot remove files on Windows
// machines occasionally
public class TestBackwardsCompatibility extends LuceneTestCase {
// To generate backcompat indexes with the current default codec, run the following gradle
// command:
// gradlew test -Dtestcase=TestBackwardsCompatibility -Dtests.bwcdir=/path/to/store/indexes
// -Dtests.codec=default -Dtests.useSecurityManager=false
// Also add testmethod with one of the index creation methods below, for example:
// -Dtestmethod=testCreateOldTaxonomy
//
// Zip up the generated indexes:
//
// cd /path/to/store/indexes/index.cfs ; zip index.<VERSION>-cfs.zip *
//
// Then move the zip file to your trunk checkout and use it in your test cases
public static final String oldTaxonomyIndexName = "taxonomy.8.10.0-cfs";
public void testCreateNewTaxonomy() throws IOException {
createNewTaxonomyIndex(oldTaxonomyIndexName);
}
// Opens up a pre-existing old taxonomy index and adds new BinaryDocValues based fields
private void createNewTaxonomyIndex(String dirName) throws IOException {
Path indexDir = createTempDir(oldTaxonomyIndexName);
TestUtil.unzip(getDataInputStream(dirName + ".zip"), indexDir);
Directory dir = newFSDirectory(indexDir);
DirectoryTaxonomyWriter writer = new DirectoryTaxonomyWriter(dir);
FacetLabel cp_c = new FacetLabel("c");
writer.addCategory(cp_c);
writer.getInternalIndexWriter().forceMerge(1);
writer.commit();
TaxonomyReader reader = new DirectoryTaxonomyReader(writer);
int ord1 = reader.getOrdinal(new FacetLabel("a"));
assert ord1 != TaxonomyReader.INVALID_ORDINAL;
// Just asserting ord1 != TaxonomyReader.INVALID_ORDINAL is not enough to check compatibility
assertNotNull(reader.getPath(ord1));
int ord2 = reader.getOrdinal(new FacetLabel("b"));
assert ord2 != TaxonomyReader.INVALID_ORDINAL;
// Just asserting ord2 != TaxonomyReader.INVALID_ORDINAL is not enough to check compatibility
assertNotNull(reader.getPath(ord2));
int ord3 = reader.getOrdinal(cp_c);
assert ord3 != TaxonomyReader.INVALID_ORDINAL;
assertNotNull(reader.getPath(ord3));
reader.close();
writer.close();
dir.close();
}
// Opens up a pre-existing index and tries to run getBulkPath on it
public void testGetBulkPathOnOlderCodec() throws Exception {
Path indexDir = createTempDir(oldTaxonomyIndexName);
TestUtil.unzip(getDataInputStream(oldTaxonomyIndexName + ".zip"), indexDir);
Directory dir = newFSDirectory(indexDir);
DirectoryTaxonomyWriter writer = new DirectoryTaxonomyWriter(dir);
DirectoryTaxonomyReader reader = new DirectoryTaxonomyReader(writer);
FacetLabel[] facetLabels = {
new FacetLabel("a"), new FacetLabel("b"),
};
int[] ordinals =
new int[] {reader.getOrdinal(facetLabels[0]), reader.getOrdinal(facetLabels[1])};
// The zip file already contains a category "a" and a category "b" stored with the older
// StoredFields codec
assertArrayEquals(facetLabels, reader.getBulkPath(ordinals));
reader.close();
writer.close();
dir.close();
}
// Used to create a fresh taxonomy index with StoredFields
@Ignore
public void testCreateOldTaxonomy() throws IOException {
createOldTaxonomyIndex(oldTaxonomyIndexName);
}
private void createOldTaxonomyIndex(String dirName) throws IOException {
Path indexDir = getIndexDir().resolve(dirName);
Files.deleteIfExists(indexDir);
Directory dir = newFSDirectory(indexDir);
TaxonomyWriter writer = new DirectoryTaxonomyWriter(dir);
writer.addCategory(new FacetLabel("a"));
writer.addCategory(new FacetLabel("b"));
writer.commit();
writer.close();
dir.close();
}
private Path getIndexDir() {
String path = System.getProperty("tests.bwcdir");
assumeTrue(
"backcompat creation tests must be run with -Dtests.bwcdir=/path/to/write/indexes",
path != null);
return Paths.get(path);
}
}

View File

@ -314,12 +314,8 @@ public final class IndexUtils {
if (CodecUtil.CODEC_MAGIC == CodecUtil.readBEInt(in)) {
int actualVersion =
CodecUtil.checkHeaderNoMagic(
in, "segments", SegmentInfos.VERSION_70, Integer.MAX_VALUE);
if (actualVersion == SegmentInfos.VERSION_70) {
format = "Lucene 7.0 or later";
} else if (actualVersion == SegmentInfos.VERSION_72) {
format = "Lucene 7.2 or later";
} else if (actualVersion == SegmentInfos.VERSION_74) {
in, "segments", SegmentInfos.VERSION_74, Integer.MAX_VALUE);
if (actualVersion == SegmentInfos.VERSION_74) {
format = "Lucene 7.4 or later";
} else if (actualVersion == SegmentInfos.VERSION_86) {
format = "Lucene 8.6 or later";

View File

@ -21,7 +21,6 @@ import java.util.Collection;
import java.util.List;
import java.util.NoSuchElementException;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Version;
import org.locationtech.spatial4j.context.SpatialContext;
import org.locationtech.spatial4j.shape.Point;
import org.locationtech.spatial4j.shape.Rectangle;
@ -59,12 +58,7 @@ public class PackedQuadPrefixTree extends QuadPrefixTree {
public static class Factory extends QuadPrefixTree.Factory {
@Override
protected SpatialPrefixTree newSPT() {
PackedQuadPrefixTree tree =
new PackedQuadPrefixTree(ctx, maxLevels != null ? maxLevels : MAX_LEVELS_POSSIBLE);
@SuppressWarnings("deprecation")
Version lucene830 = Version.LUCENE_8_3_0;
tree.robust = getVersion().onOrAfter(lucene830);
return tree;
return new PackedQuadPrefixTree(ctx, maxLevels != null ? maxLevels : MAX_LEVELS_POSSIBLE);
}
}
@ -95,15 +89,6 @@ public class PackedQuadPrefixTree extends QuadPrefixTree {
@Override
public Cell getCell(Point p, int level) {
if (!robust) { // old method
List<Cell> cells = new ArrayList<>(1);
buildNotRobustly(
xmid, ymid, 0, cells, 0x0L, ctx.getShapeFactory().pointXY(p.getX(), p.getY()), level);
if (!cells.isEmpty()) {
return cells.get(0); // note cells could be longer if p on edge
}
}
double currentXmid = xmid;
double currentYmid = ymid;
double xp = p.getX();

View File

@ -23,7 +23,6 @@ import java.util.Collection;
import java.util.List;
import java.util.Locale;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Version;
import org.locationtech.spatial4j.context.SpatialContext;
import org.locationtech.spatial4j.shape.Point;
import org.locationtech.spatial4j.shape.Rectangle;
@ -47,12 +46,7 @@ public class QuadPrefixTree extends LegacyPrefixTree {
@Override
protected SpatialPrefixTree newSPT() {
QuadPrefixTree tree =
new QuadPrefixTree(ctx, maxLevels != null ? maxLevels : MAX_LEVELS_POSSIBLE);
@SuppressWarnings("deprecation")
Version LUCENE_8_3_0 = Version.LUCENE_8_3_0;
tree.robust = getVersion().onOrAfter(LUCENE_8_3_0);
return tree;
return new QuadPrefixTree(ctx, maxLevels != null ? maxLevels : MAX_LEVELS_POSSIBLE);
}
}
@ -72,9 +66,6 @@ public class QuadPrefixTree extends LegacyPrefixTree {
final double[] levelW;
final double[] levelH;
protected boolean robust =
true; // for backward compatibility, use the old method if user specified old version.
public QuadPrefixTree(SpatialContext ctx, Rectangle bounds, int maxLevels) {
super(ctx, maxLevels);
this.xmin = bounds.getMinX();
@ -137,21 +128,6 @@ public class QuadPrefixTree extends LegacyPrefixTree {
@Override
public Cell getCell(Point p, int level) {
if (!robust) { // old method
List<Cell> cells = new ArrayList<>(1);
buildNotRobustly(
xmid,
ymid,
0,
cells,
new BytesRef(maxLevels + 1),
ctx.getShapeFactory().pointXY(p.getX(), p.getY()),
level);
if (!cells.isEmpty()) {
return cells.get(0); // note cells could be longer if p on edge
}
}
double currentXmid = xmid;
double currentYmid = ymid;
double xp = p.getX();