LUCENE-9705: Create Lucene90TermVectorsFormat (#2334)

This commit is contained in:
Ignacio Vera 2021-02-24 11:15:11 +01:00 committed by GitHub
parent 84a35dfaea
commit f8be421ae1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
32 changed files with 3377 additions and 78 deletions

View File

@ -14,11 +14,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene50;
package org.apache.lucene.backward_codecs.lucene50;
import org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingTermVectorsFormat;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.compressing.CompressingTermVectorsFormat;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.compressing.FieldsIndexWriter;
import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat;
@ -151,7 +151,7 @@ import org.apache.lucene.util.packed.PackedInts;
*
* @lucene.experimental
*/
public final class Lucene50TermVectorsFormat extends CompressingTermVectorsFormat {
public final class Lucene50TermVectorsFormat extends Lucene50CompressingTermVectorsFormat {
/** Sole constructor. */
public Lucene50TermVectorsFormat() {

View File

@ -0,0 +1,33 @@
/*
* 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.lucene50.compressing;
import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.util.Accountable;
abstract class FieldsIndex implements Accountable, Cloneable, Closeable {
/** Get the start pointer for the block that contains the given docID. */
abstract long getStartPointer(int docID);
/** Check the integrity of the index. */
abstract void checkIntegrity() throws IOException;
@Override
public abstract FieldsIndex clone();
}

View File

@ -0,0 +1,155 @@
/*
* 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.lucene50.compressing;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.Objects;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.RandomAccessInput;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.DirectMonotonicReader;
final class FieldsIndexReader extends FieldsIndex {
static final int VERSION_START = 0;
static final int VERSION_CURRENT = 0;
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(FieldsIndexReader.class);
private final int maxDoc;
private final int blockShift;
private final int numChunks;
private final DirectMonotonicReader.Meta docsMeta;
private final DirectMonotonicReader.Meta startPointersMeta;
private final IndexInput indexInput;
private final long docsStartPointer,
docsEndPointer,
startPointersStartPointer,
startPointersEndPointer;
private final DirectMonotonicReader docs, startPointers;
private final long maxPointer;
FieldsIndexReader(
Directory dir,
String name,
String suffix,
String extension,
String codecName,
byte[] id,
IndexInput metaIn)
throws IOException {
maxDoc = metaIn.readInt();
blockShift = metaIn.readInt();
numChunks = metaIn.readInt();
docsStartPointer = metaIn.readLong();
docsMeta = DirectMonotonicReader.loadMeta(metaIn, numChunks, blockShift);
docsEndPointer = startPointersStartPointer = metaIn.readLong();
startPointersMeta = DirectMonotonicReader.loadMeta(metaIn, numChunks, blockShift);
startPointersEndPointer = metaIn.readLong();
maxPointer = metaIn.readLong();
indexInput =
dir.openInput(IndexFileNames.segmentFileName(name, suffix, extension), IOContext.READ);
boolean success = false;
try {
CodecUtil.checkIndexHeader(
indexInput, codecName + "Idx", VERSION_START, VERSION_CURRENT, id, suffix);
CodecUtil.retrieveChecksum(indexInput);
success = true;
} finally {
if (success == false) {
indexInput.close();
}
}
final RandomAccessInput docsSlice =
indexInput.randomAccessSlice(docsStartPointer, docsEndPointer - docsStartPointer);
final RandomAccessInput startPointersSlice =
indexInput.randomAccessSlice(
startPointersStartPointer, startPointersEndPointer - startPointersStartPointer);
docs = DirectMonotonicReader.getInstance(docsMeta, docsSlice);
startPointers = DirectMonotonicReader.getInstance(startPointersMeta, startPointersSlice);
}
private FieldsIndexReader(FieldsIndexReader other) throws IOException {
maxDoc = other.maxDoc;
numChunks = other.numChunks;
blockShift = other.blockShift;
docsMeta = other.docsMeta;
startPointersMeta = other.startPointersMeta;
indexInput = other.indexInput.clone();
docsStartPointer = other.docsStartPointer;
docsEndPointer = other.docsEndPointer;
startPointersStartPointer = other.startPointersStartPointer;
startPointersEndPointer = other.startPointersEndPointer;
maxPointer = other.maxPointer;
final RandomAccessInput docsSlice =
indexInput.randomAccessSlice(docsStartPointer, docsEndPointer - docsStartPointer);
final RandomAccessInput startPointersSlice =
indexInput.randomAccessSlice(
startPointersStartPointer, startPointersEndPointer - startPointersStartPointer);
docs = DirectMonotonicReader.getInstance(docsMeta, docsSlice);
startPointers = DirectMonotonicReader.getInstance(startPointersMeta, startPointersSlice);
}
@Override
public long ramBytesUsed() {
return BASE_RAM_BYTES_USED
+ docsMeta.ramBytesUsed()
+ startPointersMeta.ramBytesUsed()
+ docs.ramBytesUsed()
+ startPointers.ramBytesUsed();
}
@Override
public void close() throws IOException {
indexInput.close();
}
@Override
long getStartPointer(int docID) {
Objects.checkIndex(docID, maxDoc);
long blockIndex = docs.binarySearch(0, numChunks, docID);
if (blockIndex < 0) {
blockIndex = -2 - blockIndex;
}
return startPointers.get(blockIndex);
}
@Override
public FieldsIndex clone() {
try {
return new FieldsIndexReader(this);
} catch (IOException e) {
throw new UncheckedIOException(e);
}
}
public long getMaxPointer() {
return maxPointer;
}
@Override
void checkIntegrity() throws IOException {
CodecUtil.checksumEntireFile(indexInput);
}
}

View File

@ -0,0 +1,233 @@
/*
* 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.lucene50.compressing;
import static org.apache.lucene.util.BitUtil.zigZagDecode;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.PackedInts;
final class LegacyFieldsIndexReader extends FieldsIndex {
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(LegacyFieldsIndexReader.class);
final int maxDoc;
final int[] docBases;
final long[] startPointers;
final int[] avgChunkDocs;
final long[] avgChunkSizes;
final PackedInts.Reader[] docBasesDeltas; // delta from the avg
final PackedInts.Reader[] startPointersDeltas; // delta from the avg
// It is the responsibility of the caller to close fieldsIndexIn after this constructor
// has been called
LegacyFieldsIndexReader(IndexInput fieldsIndexIn, SegmentInfo si) throws IOException {
maxDoc = si.maxDoc();
int[] docBases = new int[16];
long[] startPointers = new long[16];
int[] avgChunkDocs = new int[16];
long[] avgChunkSizes = new long[16];
PackedInts.Reader[] docBasesDeltas = new PackedInts.Reader[16];
PackedInts.Reader[] startPointersDeltas = new PackedInts.Reader[16];
final int packedIntsVersion = fieldsIndexIn.readVInt();
int blockCount = 0;
for (; ; ) {
final int numChunks = fieldsIndexIn.readVInt();
if (numChunks == 0) {
break;
}
if (blockCount == docBases.length) {
final int newSize = ArrayUtil.oversize(blockCount + 1, 8);
docBases = ArrayUtil.growExact(docBases, newSize);
startPointers = ArrayUtil.growExact(startPointers, newSize);
avgChunkDocs = ArrayUtil.growExact(avgChunkDocs, newSize);
avgChunkSizes = ArrayUtil.growExact(avgChunkSizes, newSize);
docBasesDeltas = ArrayUtil.growExact(docBasesDeltas, newSize);
startPointersDeltas = ArrayUtil.growExact(startPointersDeltas, newSize);
}
// doc bases
docBases[blockCount] = fieldsIndexIn.readVInt();
avgChunkDocs[blockCount] = fieldsIndexIn.readVInt();
final int bitsPerDocBase = fieldsIndexIn.readVInt();
if (bitsPerDocBase > 32) {
throw new CorruptIndexException(
"Corrupted bitsPerDocBase: " + bitsPerDocBase, fieldsIndexIn);
}
docBasesDeltas[blockCount] =
PackedInts.getReaderNoHeader(
fieldsIndexIn,
PackedInts.Format.PACKED,
packedIntsVersion,
numChunks,
bitsPerDocBase);
// start pointers
startPointers[blockCount] = fieldsIndexIn.readVLong();
avgChunkSizes[blockCount] = fieldsIndexIn.readVLong();
final int bitsPerStartPointer = fieldsIndexIn.readVInt();
if (bitsPerStartPointer > 64) {
throw new CorruptIndexException(
"Corrupted bitsPerStartPointer: " + bitsPerStartPointer, fieldsIndexIn);
}
startPointersDeltas[blockCount] =
PackedInts.getReaderNoHeader(
fieldsIndexIn,
PackedInts.Format.PACKED,
packedIntsVersion,
numChunks,
bitsPerStartPointer);
++blockCount;
}
this.docBases = ArrayUtil.copyOfSubArray(docBases, 0, blockCount);
this.startPointers = ArrayUtil.copyOfSubArray(startPointers, 0, blockCount);
this.avgChunkDocs = ArrayUtil.copyOfSubArray(avgChunkDocs, 0, blockCount);
this.avgChunkSizes = ArrayUtil.copyOfSubArray(avgChunkSizes, 0, blockCount);
this.docBasesDeltas = ArrayUtil.copyOfSubArray(docBasesDeltas, 0, blockCount);
this.startPointersDeltas = ArrayUtil.copyOfSubArray(startPointersDeltas, 0, blockCount);
}
private int block(int docID) {
int lo = 0, hi = docBases.length - 1;
while (lo <= hi) {
final int mid = (lo + hi) >>> 1;
final int midValue = docBases[mid];
if (midValue == docID) {
return mid;
} else if (midValue < docID) {
lo = mid + 1;
} else {
hi = mid - 1;
}
}
return hi;
}
private int relativeDocBase(int block, int relativeChunk) {
final int expected = avgChunkDocs[block] * relativeChunk;
final long delta = zigZagDecode(docBasesDeltas[block].get(relativeChunk));
return expected + (int) delta;
}
private long relativeStartPointer(int block, int relativeChunk) {
final long expected = avgChunkSizes[block] * relativeChunk;
final long delta = zigZagDecode(startPointersDeltas[block].get(relativeChunk));
return expected + delta;
}
private int relativeChunk(int block, int relativeDoc) {
int lo = 0, hi = docBasesDeltas[block].size() - 1;
while (lo <= hi) {
final int mid = (lo + hi) >>> 1;
final int midValue = relativeDocBase(block, mid);
if (midValue == relativeDoc) {
return mid;
} else if (midValue < relativeDoc) {
lo = mid + 1;
} else {
hi = mid - 1;
}
}
return hi;
}
long getStartPointer(int docID) {
if (docID < 0 || docID >= maxDoc) {
throw new IllegalArgumentException("docID out of range [0-" + maxDoc + "]: " + docID);
}
final int block = block(docID);
final int relativeChunk = relativeChunk(block, docID - docBases[block]);
return startPointers[block] + relativeStartPointer(block, relativeChunk);
}
@Override
public LegacyFieldsIndexReader clone() {
return this;
}
@Override
public long ramBytesUsed() {
long res = BASE_RAM_BYTES_USED;
res += RamUsageEstimator.shallowSizeOf(docBasesDeltas);
for (PackedInts.Reader r : docBasesDeltas) {
res += r.ramBytesUsed();
}
res += RamUsageEstimator.shallowSizeOf(startPointersDeltas);
for (PackedInts.Reader r : startPointersDeltas) {
res += r.ramBytesUsed();
}
res += RamUsageEstimator.sizeOf(docBases);
res += RamUsageEstimator.sizeOf(startPointers);
res += RamUsageEstimator.sizeOf(avgChunkDocs);
res += RamUsageEstimator.sizeOf(avgChunkSizes);
return res;
}
@Override
public Collection<Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
long docBaseDeltaBytes = RamUsageEstimator.shallowSizeOf(docBasesDeltas);
for (PackedInts.Reader r : docBasesDeltas) {
docBaseDeltaBytes += r.ramBytesUsed();
}
resources.add(Accountables.namedAccountable("doc base deltas", docBaseDeltaBytes));
long startPointerDeltaBytes = RamUsageEstimator.shallowSizeOf(startPointersDeltas);
for (PackedInts.Reader r : startPointersDeltas) {
startPointerDeltaBytes += r.ramBytesUsed();
}
resources.add(Accountables.namedAccountable("start pointer deltas", startPointerDeltaBytes));
return Collections.unmodifiableList(resources);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(blocks=" + docBases.length + ")";
}
@Override
public void close() throws IOException {
// nothing to do
}
@Override
void checkIntegrity() throws IOException {
// nothing to do, the index is checked at open time
}
}

View File

@ -0,0 +1,118 @@
/*
* 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.lucene50.compressing;
import java.io.IOException;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/**
* A {@link TermVectorsFormat} that compresses chunks of documents together in order to improve the
* compression ratio.
*
* @lucene.experimental
*/
public class Lucene50CompressingTermVectorsFormat extends TermVectorsFormat {
/** format name */
protected final String formatName;
/** segment suffix */
protected final String segmentSuffix;
/** compression mode */
protected final CompressionMode compressionMode;
/** chunk size */
protected final int chunkSize;
/** block size */
protected final int blockSize;
/**
* Create a new {@link Lucene50CompressingTermVectorsFormat}.
*
* <p><code>formatName</code> is the name of the format. This name will be used in the file
* formats to perform {@link CodecUtil#checkIndexHeader codec header checks}.
*
* <p>The <code>compressionMode</code> parameter allows you to choose between compression
* algorithms that have various compression and decompression speeds so that you can pick the one
* that best fits your indexing and searching throughput. You should never instantiate two {@link
* Lucene50CompressingTermVectorsFormat}s that have the same name but different {@link
* CompressionMode}s.
*
* <p><code>chunkSize</code> is the minimum byte size of a chunk of documents. Higher values of
* <code>chunkSize</code> should improve the compression ratio but will require more memory at
* indexing time and might make document loading a little slower (depending on the size of your OS
* cache compared to the size of your index).
*
* @param formatName the name of the {@link StoredFieldsFormat}
* @param segmentSuffix a suffix to append to files created by this format
* @param compressionMode the {@link CompressionMode} to use
* @param chunkSize the minimum number of bytes of a single chunk of stored documents
* @param blockSize the number of chunks to store in an index block.
* @see CompressionMode
*/
public Lucene50CompressingTermVectorsFormat(
String formatName,
String segmentSuffix,
CompressionMode compressionMode,
int chunkSize,
int blockSize) {
this.formatName = formatName;
this.segmentSuffix = segmentSuffix;
this.compressionMode = compressionMode;
if (chunkSize < 1) {
throw new IllegalArgumentException("chunkSize must be >= 1");
}
this.chunkSize = chunkSize;
if (blockSize < 1) {
throw new IllegalArgumentException("blockSize must be >= 1");
}
this.blockSize = blockSize;
}
@Override
public final TermVectorsReader vectorsReader(
Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context)
throws IOException {
return new Lucene50CompressingTermVectorsReader(
directory, segmentInfo, segmentSuffix, fieldInfos, context, formatName, compressionMode);
}
@Override
public TermVectorsWriter vectorsWriter(
Directory directory, SegmentInfo segmentInfo, IOContext context) throws IOException {
throw new UnsupportedOperationException("Old formats can't be used for writing");
}
@Override
public String toString() {
return getClass().getSimpleName()
+ "(compressionMode="
+ compressionMode
+ ", chunkSize="
+ chunkSize
+ ", blockSize="
+ blockSize
+ ")";
}
}

View File

@ -14,22 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.compressing;
import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.FLAGS_BITS;
import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.META_VERSION_START;
import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.OFFSETS;
import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.PACKED_BLOCK_SIZE;
import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.PAYLOADS;
import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.POSITIONS;
import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VECTORS_EXTENSION;
import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VECTORS_INDEX_CODEC_NAME;
import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VECTORS_INDEX_EXTENSION;
import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VECTORS_META_EXTENSION;
import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VERSION_CURRENT;
import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VERSION_META;
import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VERSION_OFFHEAP_INDEX;
import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VERSION_START;
package org.apache.lucene.backward_codecs.lucene50.compressing;
import java.io.Closeable;
import java.io.IOException;
@ -39,6 +24,8 @@ import java.util.Iterator;
import java.util.NoSuchElementException;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.compressing.Decompressor;
import org.apache.lucene.index.BaseTermsEnum;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
@ -67,11 +54,35 @@ import org.apache.lucene.util.packed.BlockPackedReaderIterator;
import org.apache.lucene.util.packed.PackedInts;
/**
* {@link TermVectorsReader} for {@link CompressingTermVectorsFormat}.
* {@link TermVectorsReader} for {@link Lucene50CompressingTermVectorsFormat}.
*
* @lucene.experimental
*/
public final class CompressingTermVectorsReader extends TermVectorsReader implements Closeable {
public final class Lucene50CompressingTermVectorsReader extends TermVectorsReader
implements Closeable {
// hard limit on the maximum number of documents per chunk
static final int MAX_DOCUMENTS_PER_CHUNK = 128;
static final String VECTORS_EXTENSION = "tvd";
static final String VECTORS_INDEX_EXTENSION = "tvx";
static final String VECTORS_META_EXTENSION = "tvm";
static final String VECTORS_INDEX_CODEC_NAME = "Lucene85TermVectorsIndex";
static final int VERSION_START = 1;
static final int VERSION_OFFHEAP_INDEX = 2;
/** Version where all metadata were moved to the meta file. */
static final int VERSION_META = 3;
static final int VERSION_CURRENT = VERSION_META;
static final int META_VERSION_START = 0;
static final int PACKED_BLOCK_SIZE = 64;
static final int POSITIONS = 0x01;
static final int OFFSETS = 0x02;
static final int PAYLOADS = 0x04;
static final int FLAGS_BITS = PackedInts.bitsRequired(POSITIONS | OFFSETS | PAYLOADS);
private final FieldInfos fieldInfos;
final FieldsIndex indexReader;
@ -89,7 +100,7 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
private final long maxPointer; // end of the data section
// used by clone
private CompressingTermVectorsReader(CompressingTermVectorsReader reader) {
private Lucene50CompressingTermVectorsReader(Lucene50CompressingTermVectorsReader reader) {
this.fieldInfos = reader.fieldInfos;
this.vectorsStream = reader.vectorsStream.clone();
this.indexReader = reader.indexReader.clone();
@ -108,7 +119,7 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
}
/** Sole constructor. */
public CompressingTermVectorsReader(
public Lucene50CompressingTermVectorsReader(
Directory d,
SegmentInfo si,
String segmentSuffix,
@ -318,7 +329,7 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
@Override
public TermVectorsReader clone() {
return new CompressingTermVectorsReader(this);
return new Lucene50CompressingTermVectorsReader(this);
}
@Override

View File

@ -15,8 +15,5 @@
* limitations under the License.
*/
/**
* Components from the Lucene 5.0 index format See {@link org.apache.lucene.codecs.lucene90} for an
* overview of the index format.
*/
package org.apache.lucene.codecs.lucene50;
/** Lucene 5.0 compressing format. */
package org.apache.lucene.backward_codecs.lucene50.compressing;

View File

@ -20,6 +20,7 @@ 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;
@ -35,7 +36,6 @@ import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.VectorFormat;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
@ -87,7 +87,7 @@ public class Lucene70Codec extends Codec {
}
@Override
public final TermVectorsFormat termVectorsFormat() {
public TermVectorsFormat termVectorsFormat() {
return vectorsFormat;
}

View File

@ -19,6 +19,7 @@ package org.apache.lucene.backward_codecs.lucene80;
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.Lucene50TermVectorsFormat;
import org.apache.lucene.backward_codecs.lucene60.Lucene60FieldInfosFormat;
import org.apache.lucene.backward_codecs.lucene60.Lucene60PointsFormat;
import org.apache.lucene.backward_codecs.lucene70.Lucene70SegmentInfoFormat;
@ -34,7 +35,6 @@ import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.VectorFormat;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
@ -84,7 +84,7 @@ public class Lucene80Codec extends Codec {
}
@Override
public final TermVectorsFormat termVectorsFormat() {
public TermVectorsFormat termVectorsFormat() {
return vectorsFormat;
}

View File

@ -21,6 +21,7 @@ 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.backward_codecs.lucene70.Lucene70SegmentInfoFormat;
@ -38,7 +39,6 @@ import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.VectorFormat;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
@ -99,7 +99,7 @@ public class Lucene84Codec extends Codec {
}
@Override
public final TermVectorsFormat termVectorsFormat() {
public TermVectorsFormat termVectorsFormat() {
return vectorsFormat;
}

View File

@ -21,6 +21,7 @@ import java.util.Objects;
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.Lucene50TermVectorsFormat;
import org.apache.lucene.backward_codecs.lucene60.Lucene60FieldInfosFormat;
import org.apache.lucene.backward_codecs.lucene80.Lucene80NormsFormat;
import org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat;
@ -37,7 +38,6 @@ import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.VectorFormat;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat;
import org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
@ -101,7 +101,7 @@ public class Lucene86Codec extends Codec {
}
@Override
public final TermVectorsFormat termVectorsFormat() {
public TermVectorsFormat termVectorsFormat() {
return vectorsFormat;
}

View File

@ -20,6 +20,7 @@ package org.apache.lucene.backward_codecs.lucene87;
import java.util.Objects;
import org.apache.lucene.backward_codecs.lucene50.Lucene50CompoundFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50LiveDocsFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.backward_codecs.lucene60.Lucene60FieldInfosFormat;
import org.apache.lucene.backward_codecs.lucene80.Lucene80DocValuesFormat;
import org.apache.lucene.backward_codecs.lucene80.Lucene80NormsFormat;
@ -37,7 +38,6 @@ import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.VectorFormat;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat;
import org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat;
import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat;
@ -113,7 +113,7 @@ public class Lucene87Codec extends Codec {
}
@Override
public final TermVectorsFormat termVectorsFormat() {
public TermVectorsFormat termVectorsFormat() {
return vectorsFormat;
}

View File

@ -0,0 +1,29 @@
/*
* 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.lucene50;
import org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50RWCompressingTermVectorsFormat;
import org.apache.lucene.codecs.compressing.CompressionMode;
/** RW impersonation of Lucene50StoredFieldsFormat. */
public final class Lucene50RWTermVectorsFormat extends Lucene50RWCompressingTermVectorsFormat {
/** Sole constructor. */
public Lucene50RWTermVectorsFormat() {
super("Lucene50TermVectorsData", "", CompressionMode.FAST, 1 << 12, 10);
}
}

View File

@ -14,15 +14,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene50;
package org.apache.lucene.backward_codecs.lucene50;
import org.apache.lucene.backward_codecs.lucene87.Lucene87RWCodec;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BaseTermVectorsFormatTestCase;
import org.apache.lucene.util.TestUtil;
public class TestLucene50TermVectorsFormat extends BaseTermVectorsFormatTestCase {
@Override
protected Codec getCodec() {
return TestUtil.getDefaultCodec();
return new Lucene87RWCodec();
}
}

View File

@ -0,0 +1,202 @@
/*
* 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.lucene50.compressing;
import static org.apache.lucene.backward_codecs.lucene50.compressing.FieldsIndexReader.VERSION_CURRENT;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.DirectMonotonicReader;
import org.apache.lucene.util.packed.DirectMonotonicWriter;
/**
* Efficient index format for block-based {@link Codec}s.
*
* <p>For each block of compressed stored fields, this stores the first document of the block and
* the start pointer of the block in a {@link DirectMonotonicWriter}. At read time, the docID is
* binary-searched in the {@link DirectMonotonicReader} that records doc IDS, and the returned index
* is used to look up the start pointer in the {@link DirectMonotonicReader} that records start
* pointers.
*
* @lucene.internal
*/
public final class FieldsIndexWriter implements Closeable {
private final Directory dir;
private final String name;
private final String suffix;
private final String extension;
private final String codecName;
private final byte[] id;
private final int blockShift;
private final IOContext ioContext;
private IndexOutput docsOut;
private IndexOutput filePointersOut;
private int totalDocs;
private int totalChunks;
private long previousFP;
FieldsIndexWriter(
Directory dir,
String name,
String suffix,
String extension,
String codecName,
byte[] id,
int blockShift,
IOContext ioContext)
throws IOException {
this.dir = dir;
this.name = name;
this.suffix = suffix;
this.extension = extension;
this.codecName = codecName;
this.id = id;
this.blockShift = blockShift;
this.ioContext = ioContext;
this.docsOut = dir.createTempOutput(name, codecName + "-doc_ids", ioContext);
boolean success = false;
try {
CodecUtil.writeHeader(docsOut, codecName + "Docs", VERSION_CURRENT);
filePointersOut = dir.createTempOutput(name, codecName + "file_pointers", ioContext);
CodecUtil.writeHeader(filePointersOut, codecName + "FilePointers", VERSION_CURRENT);
success = true;
} finally {
if (success == false) {
close();
}
}
}
void writeIndex(int numDocs, long startPointer) throws IOException {
assert startPointer >= previousFP;
docsOut.writeVInt(numDocs);
filePointersOut.writeVLong(startPointer - previousFP);
previousFP = startPointer;
totalDocs += numDocs;
totalChunks++;
}
void finish(int numDocs, long maxPointer, IndexOutput metaOut) throws IOException {
if (numDocs != totalDocs) {
throw new IllegalStateException("Expected " + numDocs + " docs, but got " + totalDocs);
}
CodecUtil.writeFooter(docsOut);
CodecUtil.writeFooter(filePointersOut);
IOUtils.close(docsOut, filePointersOut);
try (IndexOutput dataOut =
dir.createOutput(IndexFileNames.segmentFileName(name, suffix, extension), ioContext)) {
CodecUtil.writeIndexHeader(dataOut, codecName + "Idx", VERSION_CURRENT, id, suffix);
metaOut.writeInt(numDocs);
metaOut.writeInt(blockShift);
metaOut.writeInt(totalChunks + 1);
metaOut.writeLong(dataOut.getFilePointer());
try (ChecksumIndexInput docsIn =
dir.openChecksumInput(docsOut.getName(), IOContext.READONCE)) {
CodecUtil.checkHeader(docsIn, codecName + "Docs", VERSION_CURRENT, VERSION_CURRENT);
Throwable priorE = null;
try {
final DirectMonotonicWriter docs =
DirectMonotonicWriter.getInstance(metaOut, dataOut, totalChunks + 1, blockShift);
long doc = 0;
docs.add(doc);
for (int i = 0; i < totalChunks; ++i) {
doc += docsIn.readVInt();
docs.add(doc);
}
docs.finish();
if (doc != totalDocs) {
throw new CorruptIndexException("Docs don't add up", docsIn);
}
} catch (Throwable e) {
priorE = e;
} finally {
CodecUtil.checkFooter(docsIn, priorE);
}
}
dir.deleteFile(docsOut.getName());
docsOut = null;
metaOut.writeLong(dataOut.getFilePointer());
try (ChecksumIndexInput filePointersIn =
dir.openChecksumInput(filePointersOut.getName(), IOContext.READONCE)) {
CodecUtil.checkHeader(
filePointersIn, codecName + "FilePointers", VERSION_CURRENT, VERSION_CURRENT);
Throwable priorE = null;
try {
final DirectMonotonicWriter filePointers =
DirectMonotonicWriter.getInstance(metaOut, dataOut, totalChunks + 1, blockShift);
long fp = 0;
for (int i = 0; i < totalChunks; ++i) {
fp += filePointersIn.readVLong();
filePointers.add(fp);
}
if (maxPointer < fp) {
throw new CorruptIndexException("File pointers don't add up", filePointersIn);
}
filePointers.add(maxPointer);
filePointers.finish();
} catch (Throwable e) {
priorE = e;
} finally {
CodecUtil.checkFooter(filePointersIn, priorE);
}
}
dir.deleteFile(filePointersOut.getName());
filePointersOut = null;
metaOut.writeLong(dataOut.getFilePointer());
metaOut.writeLong(maxPointer);
CodecUtil.writeFooter(dataOut);
}
}
@Override
public void close() throws IOException {
try {
IOUtils.close(docsOut, filePointersOut);
} finally {
List<String> fileNames = new ArrayList<>();
if (docsOut != null) {
fileNames.add(docsOut.getName());
}
if (filePointersOut != null) {
fileNames.add(filePointersOut.getName());
}
try {
IOUtils.deleteFiles(dir, fileNames);
} finally {
docsOut = filePointersOut = null;
}
}
}
}

View File

@ -0,0 +1,953 @@
/*
* 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.lucene50.compressing;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingTermVectorsReader.FLAGS_BITS;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingTermVectorsReader.MAX_DOCUMENTS_PER_CHUNK;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingTermVectorsReader.OFFSETS;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingTermVectorsReader.PACKED_BLOCK_SIZE;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingTermVectorsReader.PAYLOADS;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingTermVectorsReader.POSITIONS;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingTermVectorsReader.VECTORS_EXTENSION;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingTermVectorsReader.VECTORS_INDEX_CODEC_NAME;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingTermVectorsReader.VECTORS_INDEX_EXTENSION;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingTermVectorsReader.VECTORS_META_EXTENSION;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingTermVectorsReader.VERSION_CURRENT;
import java.io.IOException;
import java.util.ArrayDeque;
import java.util.Arrays;
import java.util.Collection;
import java.util.Deque;
import java.util.Iterator;
import java.util.List;
import java.util.SortedSet;
import java.util.TreeSet;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.compressing.Compressor;
import org.apache.lucene.codecs.compressing.MatchingReaders;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.packed.BlockPackedWriter;
import org.apache.lucene.util.packed.PackedInts;
/**
* {@link TermVectorsWriter} for {@link Lucene50CompressingTermVectorsFormat}.
*
* @lucene.experimental
*/
public final class Lucene50CompressingTermVectorsWriter extends TermVectorsWriter {
private final String segment;
private FieldsIndexWriter indexWriter;
private IndexOutput metaStream, vectorsStream;
private final CompressionMode compressionMode;
private final Compressor compressor;
private final int chunkSize;
private long numDirtyChunks; // number of incomplete compressed blocks written
private long numDirtyDocs; // cumulative number of missing docs in incomplete chunks
/** a pending doc */
private class DocData {
final int numFields;
final Deque<FieldData> fields;
final int posStart, offStart, payStart;
DocData(int numFields, int posStart, int offStart, int payStart) {
this.numFields = numFields;
this.fields = new ArrayDeque<>(numFields);
this.posStart = posStart;
this.offStart = offStart;
this.payStart = payStart;
}
FieldData addField(
int fieldNum, int numTerms, boolean positions, boolean offsets, boolean payloads) {
final FieldData field;
if (fields.isEmpty()) {
field =
new FieldData(
fieldNum, numTerms, positions, offsets, payloads, posStart, offStart, payStart);
} else {
final FieldData last = fields.getLast();
final int posStart = last.posStart + (last.hasPositions ? last.totalPositions : 0);
final int offStart = last.offStart + (last.hasOffsets ? last.totalPositions : 0);
final int payStart = last.payStart + (last.hasPayloads ? last.totalPositions : 0);
field =
new FieldData(
fieldNum, numTerms, positions, offsets, payloads, posStart, offStart, payStart);
}
fields.add(field);
return field;
}
}
private DocData addDocData(int numVectorFields) {
FieldData last = null;
for (Iterator<DocData> it = pendingDocs.descendingIterator(); it.hasNext(); ) {
final DocData doc = it.next();
if (!doc.fields.isEmpty()) {
last = doc.fields.getLast();
break;
}
}
final DocData doc;
if (last == null) {
doc = new DocData(numVectorFields, 0, 0, 0);
} else {
final int posStart = last.posStart + (last.hasPositions ? last.totalPositions : 0);
final int offStart = last.offStart + (last.hasOffsets ? last.totalPositions : 0);
final int payStart = last.payStart + (last.hasPayloads ? last.totalPositions : 0);
doc = new DocData(numVectorFields, posStart, offStart, payStart);
}
pendingDocs.add(doc);
return doc;
}
/** a pending field */
private class FieldData {
final boolean hasPositions, hasOffsets, hasPayloads;
final int fieldNum, flags, numTerms;
final int[] freqs, prefixLengths, suffixLengths;
final int posStart, offStart, payStart;
int totalPositions;
int ord;
FieldData(
int fieldNum,
int numTerms,
boolean positions,
boolean offsets,
boolean payloads,
int posStart,
int offStart,
int payStart) {
this.fieldNum = fieldNum;
this.numTerms = numTerms;
this.hasPositions = positions;
this.hasOffsets = offsets;
this.hasPayloads = payloads;
this.flags =
(positions ? POSITIONS : 0) | (offsets ? OFFSETS : 0) | (payloads ? PAYLOADS : 0);
this.freqs = new int[numTerms];
this.prefixLengths = new int[numTerms];
this.suffixLengths = new int[numTerms];
this.posStart = posStart;
this.offStart = offStart;
this.payStart = payStart;
totalPositions = 0;
ord = 0;
}
void addTerm(int freq, int prefixLength, int suffixLength) {
freqs[ord] = freq;
prefixLengths[ord] = prefixLength;
suffixLengths[ord] = suffixLength;
++ord;
}
void addPosition(int position, int startOffset, int length, int payloadLength) {
if (hasPositions) {
if (posStart + totalPositions == positionsBuf.length) {
positionsBuf = ArrayUtil.grow(positionsBuf);
}
positionsBuf[posStart + totalPositions] = position;
}
if (hasOffsets) {
if (offStart + totalPositions == startOffsetsBuf.length) {
final int newLength = ArrayUtil.oversize(offStart + totalPositions, 4);
startOffsetsBuf = ArrayUtil.growExact(startOffsetsBuf, newLength);
lengthsBuf = ArrayUtil.growExact(lengthsBuf, newLength);
}
startOffsetsBuf[offStart + totalPositions] = startOffset;
lengthsBuf[offStart + totalPositions] = length;
}
if (hasPayloads) {
if (payStart + totalPositions == payloadLengthsBuf.length) {
payloadLengthsBuf = ArrayUtil.grow(payloadLengthsBuf);
}
payloadLengthsBuf[payStart + totalPositions] = payloadLength;
}
++totalPositions;
}
}
private int numDocs; // total number of docs seen
private final Deque<DocData> pendingDocs; // pending docs
private DocData curDoc; // current document
private FieldData curField; // current field
private final BytesRef lastTerm;
private int[] positionsBuf, startOffsetsBuf, lengthsBuf, payloadLengthsBuf;
private final ByteBuffersDataOutput termSuffixes; // buffered term suffixes
private final ByteBuffersDataOutput payloadBytes; // buffered term payloads
private final BlockPackedWriter writer;
/** Sole constructor. */
Lucene50CompressingTermVectorsWriter(
Directory directory,
SegmentInfo si,
String segmentSuffix,
IOContext context,
String formatName,
CompressionMode compressionMode,
int chunkSize,
int blockShift)
throws IOException {
assert directory != null;
this.segment = si.name;
this.compressionMode = compressionMode;
this.compressor = compressionMode.newCompressor();
this.chunkSize = chunkSize;
numDocs = 0;
pendingDocs = new ArrayDeque<>();
termSuffixes = ByteBuffersDataOutput.newResettableInstance();
payloadBytes = ByteBuffersDataOutput.newResettableInstance();
lastTerm = new BytesRef(ArrayUtil.oversize(30, 1));
boolean success = false;
try {
metaStream =
directory.createOutput(
IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_META_EXTENSION),
context);
CodecUtil.writeIndexHeader(
metaStream,
VECTORS_INDEX_CODEC_NAME + "Meta",
VERSION_CURRENT,
si.getId(),
segmentSuffix);
assert CodecUtil.indexHeaderLength(VECTORS_INDEX_CODEC_NAME + "Meta", segmentSuffix)
== metaStream.getFilePointer();
vectorsStream =
directory.createOutput(
IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_EXTENSION), context);
CodecUtil.writeIndexHeader(
vectorsStream, formatName, VERSION_CURRENT, si.getId(), segmentSuffix);
assert CodecUtil.indexHeaderLength(formatName, segmentSuffix)
== vectorsStream.getFilePointer();
indexWriter =
new FieldsIndexWriter(
directory,
segment,
segmentSuffix,
VECTORS_INDEX_EXTENSION,
VECTORS_INDEX_CODEC_NAME,
si.getId(),
blockShift,
context);
metaStream.writeVInt(PackedInts.VERSION_CURRENT);
metaStream.writeVInt(chunkSize);
writer = new BlockPackedWriter(vectorsStream, PACKED_BLOCK_SIZE);
positionsBuf = new int[1024];
startOffsetsBuf = new int[1024];
lengthsBuf = new int[1024];
payloadLengthsBuf = new int[1024];
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(metaStream, vectorsStream, indexWriter, indexWriter);
}
}
}
@Override
public void close() throws IOException {
try {
IOUtils.close(metaStream, vectorsStream, indexWriter);
} finally {
metaStream = null;
vectorsStream = null;
indexWriter = null;
}
}
@Override
public void startDocument(int numVectorFields) throws IOException {
curDoc = addDocData(numVectorFields);
}
@Override
public void finishDocument() throws IOException {
// append the payload bytes of the doc after its terms
payloadBytes.copyTo(termSuffixes);
payloadBytes.reset();
++numDocs;
if (triggerFlush()) {
flush();
}
curDoc = null;
}
@Override
public void startField(
FieldInfo info, int numTerms, boolean positions, boolean offsets, boolean payloads)
throws IOException {
curField = curDoc.addField(info.number, numTerms, positions, offsets, payloads);
lastTerm.length = 0;
}
@Override
public void finishField() throws IOException {
curField = null;
}
@Override
public void startTerm(BytesRef term, int freq) throws IOException {
assert freq >= 1;
final int prefix;
if (lastTerm.length == 0) {
// no previous term: no bytes to write
prefix = 0;
} else {
prefix = StringHelper.bytesDifference(lastTerm, term);
}
curField.addTerm(freq, prefix, term.length - prefix);
termSuffixes.writeBytes(term.bytes, term.offset + prefix, term.length - prefix);
// copy last term
if (lastTerm.bytes.length < term.length) {
lastTerm.bytes = new byte[ArrayUtil.oversize(term.length, 1)];
}
lastTerm.offset = 0;
lastTerm.length = term.length;
System.arraycopy(term.bytes, term.offset, lastTerm.bytes, 0, term.length);
}
@Override
public void addPosition(int position, int startOffset, int endOffset, BytesRef payload)
throws IOException {
assert curField.flags != 0;
curField.addPosition(
position, startOffset, endOffset - startOffset, payload == null ? 0 : payload.length);
if (curField.hasPayloads && payload != null) {
payloadBytes.writeBytes(payload.bytes, payload.offset, payload.length);
}
}
private boolean triggerFlush() {
return termSuffixes.size() >= chunkSize || pendingDocs.size() >= MAX_DOCUMENTS_PER_CHUNK;
}
private void flush() throws IOException {
final int chunkDocs = pendingDocs.size();
assert chunkDocs > 0 : chunkDocs;
// write the index file
indexWriter.writeIndex(chunkDocs, vectorsStream.getFilePointer());
final int docBase = numDocs - chunkDocs;
vectorsStream.writeVInt(docBase);
vectorsStream.writeVInt(chunkDocs);
// total number of fields of the chunk
final int totalFields = flushNumFields(chunkDocs);
if (totalFields > 0) {
// unique field numbers (sorted)
final int[] fieldNums = flushFieldNums();
// offsets in the array of unique field numbers
flushFields(totalFields, fieldNums);
// flags (does the field have positions, offsets, payloads?)
flushFlags(totalFields, fieldNums);
// number of terms of each field
flushNumTerms(totalFields);
// prefix and suffix lengths for each field
flushTermLengths();
// term freqs - 1 (because termFreq is always >=1) for each term
flushTermFreqs();
// positions for all terms, when enabled
flushPositions();
// offsets for all terms, when enabled
flushOffsets(fieldNums);
// payload lengths for all terms, when enabled
flushPayloadLengths();
// compress terms and payloads and write them to the output
//
// TODO: We could compress in the slices we already have in the buffer (min/max slice
// can be set on the buffer itself).
byte[] content = termSuffixes.toArrayCopy();
compressor.compress(content, 0, content.length, vectorsStream);
}
// reset
pendingDocs.clear();
curDoc = null;
curField = null;
termSuffixes.reset();
}
private int flushNumFields(int chunkDocs) throws IOException {
if (chunkDocs == 1) {
final int numFields = pendingDocs.getFirst().numFields;
vectorsStream.writeVInt(numFields);
return numFields;
} else {
writer.reset(vectorsStream);
int totalFields = 0;
for (DocData dd : pendingDocs) {
writer.add(dd.numFields);
totalFields += dd.numFields;
}
writer.finish();
return totalFields;
}
}
/** Returns a sorted array containing unique field numbers */
private int[] flushFieldNums() throws IOException {
SortedSet<Integer> fieldNums = new TreeSet<>();
for (DocData dd : pendingDocs) {
for (FieldData fd : dd.fields) {
fieldNums.add(fd.fieldNum);
}
}
final int numDistinctFields = fieldNums.size();
assert numDistinctFields > 0;
final int bitsRequired = PackedInts.bitsRequired(fieldNums.last());
final int token = (Math.min(numDistinctFields - 1, 0x07) << 5) | bitsRequired;
vectorsStream.writeByte((byte) token);
if (numDistinctFields - 1 >= 0x07) {
vectorsStream.writeVInt(numDistinctFields - 1 - 0x07);
}
final PackedInts.Writer writer =
PackedInts.getWriterNoHeader(
vectorsStream, PackedInts.Format.PACKED, fieldNums.size(), bitsRequired, 1);
for (Integer fieldNum : fieldNums) {
writer.add(fieldNum);
}
writer.finish();
int[] fns = new int[fieldNums.size()];
int i = 0;
for (Integer key : fieldNums) {
fns[i++] = key;
}
return fns;
}
private void flushFields(int totalFields, int[] fieldNums) throws IOException {
final PackedInts.Writer writer =
PackedInts.getWriterNoHeader(
vectorsStream,
PackedInts.Format.PACKED,
totalFields,
PackedInts.bitsRequired(fieldNums.length - 1),
1);
for (DocData dd : pendingDocs) {
for (FieldData fd : dd.fields) {
final int fieldNumIndex = Arrays.binarySearch(fieldNums, fd.fieldNum);
assert fieldNumIndex >= 0;
writer.add(fieldNumIndex);
}
}
writer.finish();
}
private void flushFlags(int totalFields, int[] fieldNums) throws IOException {
// check if fields always have the same flags
boolean nonChangingFlags = true;
int[] fieldFlags = new int[fieldNums.length];
Arrays.fill(fieldFlags, -1);
outer:
for (DocData dd : pendingDocs) {
for (FieldData fd : dd.fields) {
final int fieldNumOff = Arrays.binarySearch(fieldNums, fd.fieldNum);
assert fieldNumOff >= 0;
if (fieldFlags[fieldNumOff] == -1) {
fieldFlags[fieldNumOff] = fd.flags;
} else if (fieldFlags[fieldNumOff] != fd.flags) {
nonChangingFlags = false;
break outer;
}
}
}
if (nonChangingFlags) {
// write one flag per field num
vectorsStream.writeVInt(0);
final PackedInts.Writer writer =
PackedInts.getWriterNoHeader(
vectorsStream, PackedInts.Format.PACKED, fieldFlags.length, FLAGS_BITS, 1);
for (int flags : fieldFlags) {
assert flags >= 0;
writer.add(flags);
}
assert writer.ord() == fieldFlags.length - 1;
writer.finish();
} else {
// write one flag for every field instance
vectorsStream.writeVInt(1);
final PackedInts.Writer writer =
PackedInts.getWriterNoHeader(
vectorsStream, PackedInts.Format.PACKED, totalFields, FLAGS_BITS, 1);
for (DocData dd : pendingDocs) {
for (FieldData fd : dd.fields) {
writer.add(fd.flags);
}
}
assert writer.ord() == totalFields - 1;
writer.finish();
}
}
private void flushNumTerms(int totalFields) throws IOException {
int maxNumTerms = 0;
for (DocData dd : pendingDocs) {
for (FieldData fd : dd.fields) {
maxNumTerms |= fd.numTerms;
}
}
final int bitsRequired = PackedInts.bitsRequired(maxNumTerms);
vectorsStream.writeVInt(bitsRequired);
final PackedInts.Writer writer =
PackedInts.getWriterNoHeader(
vectorsStream, PackedInts.Format.PACKED, totalFields, bitsRequired, 1);
for (DocData dd : pendingDocs) {
for (FieldData fd : dd.fields) {
writer.add(fd.numTerms);
}
}
assert writer.ord() == totalFields - 1;
writer.finish();
}
private void flushTermLengths() throws IOException {
writer.reset(vectorsStream);
for (DocData dd : pendingDocs) {
for (FieldData fd : dd.fields) {
for (int i = 0; i < fd.numTerms; ++i) {
writer.add(fd.prefixLengths[i]);
}
}
}
writer.finish();
writer.reset(vectorsStream);
for (DocData dd : pendingDocs) {
for (FieldData fd : dd.fields) {
for (int i = 0; i < fd.numTerms; ++i) {
writer.add(fd.suffixLengths[i]);
}
}
}
writer.finish();
}
private void flushTermFreqs() throws IOException {
writer.reset(vectorsStream);
for (DocData dd : pendingDocs) {
for (FieldData fd : dd.fields) {
for (int i = 0; i < fd.numTerms; ++i) {
writer.add(fd.freqs[i] - 1);
}
}
}
writer.finish();
}
private void flushPositions() throws IOException {
writer.reset(vectorsStream);
for (DocData dd : pendingDocs) {
for (FieldData fd : dd.fields) {
if (fd.hasPositions) {
int pos = 0;
for (int i = 0; i < fd.numTerms; ++i) {
int previousPosition = 0;
for (int j = 0; j < fd.freqs[i]; ++j) {
final int position = positionsBuf[fd.posStart + pos++];
writer.add(position - previousPosition);
previousPosition = position;
}
}
assert pos == fd.totalPositions;
}
}
}
writer.finish();
}
private void flushOffsets(int[] fieldNums) throws IOException {
boolean hasOffsets = false;
long[] sumPos = new long[fieldNums.length];
long[] sumOffsets = new long[fieldNums.length];
for (DocData dd : pendingDocs) {
for (FieldData fd : dd.fields) {
hasOffsets |= fd.hasOffsets;
if (fd.hasOffsets && fd.hasPositions) {
final int fieldNumOff = Arrays.binarySearch(fieldNums, fd.fieldNum);
int pos = 0;
for (int i = 0; i < fd.numTerms; ++i) {
sumPos[fieldNumOff] += positionsBuf[fd.posStart + fd.freqs[i] - 1 + pos];
sumOffsets[fieldNumOff] += startOffsetsBuf[fd.offStart + fd.freqs[i] - 1 + pos];
pos += fd.freqs[i];
}
assert pos == fd.totalPositions;
}
}
}
if (!hasOffsets) {
// nothing to do
return;
}
final float[] charsPerTerm = new float[fieldNums.length];
for (int i = 0; i < fieldNums.length; ++i) {
charsPerTerm[i] =
(sumPos[i] <= 0 || sumOffsets[i] <= 0) ? 0 : (float) ((double) sumOffsets[i] / sumPos[i]);
}
// start offsets
for (int i = 0; i < fieldNums.length; ++i) {
vectorsStream.writeInt(Float.floatToRawIntBits(charsPerTerm[i]));
}
writer.reset(vectorsStream);
for (DocData dd : pendingDocs) {
for (FieldData fd : dd.fields) {
if ((fd.flags & OFFSETS) != 0) {
final int fieldNumOff = Arrays.binarySearch(fieldNums, fd.fieldNum);
final float cpt = charsPerTerm[fieldNumOff];
int pos = 0;
for (int i = 0; i < fd.numTerms; ++i) {
int previousPos = 0;
int previousOff = 0;
for (int j = 0; j < fd.freqs[i]; ++j) {
final int position = fd.hasPositions ? positionsBuf[fd.posStart + pos] : 0;
final int startOffset = startOffsetsBuf[fd.offStart + pos];
writer.add(startOffset - previousOff - (int) (cpt * (position - previousPos)));
previousPos = position;
previousOff = startOffset;
++pos;
}
}
}
}
}
writer.finish();
// lengths
writer.reset(vectorsStream);
for (DocData dd : pendingDocs) {
for (FieldData fd : dd.fields) {
if ((fd.flags & OFFSETS) != 0) {
int pos = 0;
for (int i = 0; i < fd.numTerms; ++i) {
for (int j = 0; j < fd.freqs[i]; ++j) {
writer.add(
lengthsBuf[fd.offStart + pos++] - fd.prefixLengths[i] - fd.suffixLengths[i]);
}
}
assert pos == fd.totalPositions;
}
}
}
writer.finish();
}
private void flushPayloadLengths() throws IOException {
writer.reset(vectorsStream);
for (DocData dd : pendingDocs) {
for (FieldData fd : dd.fields) {
if (fd.hasPayloads) {
for (int i = 0; i < fd.totalPositions; ++i) {
writer.add(payloadLengthsBuf[fd.payStart + i]);
}
}
}
}
writer.finish();
}
@Override
public void finish(FieldInfos fis, int numDocs) throws IOException {
if (!pendingDocs.isEmpty()) {
numDirtyChunks++; // incomplete: we had to force this flush
final long expectedChunkDocs =
Math.min(
MAX_DOCUMENTS_PER_CHUNK,
(long) ((double) chunkSize / termSuffixes.size() * pendingDocs.size()));
numDirtyDocs += expectedChunkDocs - pendingDocs.size();
flush();
}
if (numDocs != this.numDocs) {
throw new RuntimeException(
"Wrote " + this.numDocs + " docs, finish called with numDocs=" + numDocs);
}
indexWriter.finish(numDocs, vectorsStream.getFilePointer(), metaStream);
metaStream.writeVLong(numDirtyChunks);
metaStream.writeVLong(numDirtyDocs);
CodecUtil.writeFooter(metaStream);
CodecUtil.writeFooter(vectorsStream);
}
@Override
public void addProx(int numProx, DataInput positions, DataInput offsets) throws IOException {
assert (curField.hasPositions) == (positions != null);
assert (curField.hasOffsets) == (offsets != null);
if (curField.hasPositions) {
final int posStart = curField.posStart + curField.totalPositions;
if (posStart + numProx > positionsBuf.length) {
positionsBuf = ArrayUtil.grow(positionsBuf, posStart + numProx);
}
int position = 0;
if (curField.hasPayloads) {
final int payStart = curField.payStart + curField.totalPositions;
if (payStart + numProx > payloadLengthsBuf.length) {
payloadLengthsBuf = ArrayUtil.grow(payloadLengthsBuf, payStart + numProx);
}
for (int i = 0; i < numProx; ++i) {
final int code = positions.readVInt();
if ((code & 1) != 0) {
// This position has a payload
final int payloadLength = positions.readVInt();
payloadLengthsBuf[payStart + i] = payloadLength;
payloadBytes.copyBytes(positions, payloadLength);
} else {
payloadLengthsBuf[payStart + i] = 0;
}
position += code >>> 1;
positionsBuf[posStart + i] = position;
}
} else {
for (int i = 0; i < numProx; ++i) {
position += (positions.readVInt() >>> 1);
positionsBuf[posStart + i] = position;
}
}
}
if (curField.hasOffsets) {
final int offStart = curField.offStart + curField.totalPositions;
if (offStart + numProx > startOffsetsBuf.length) {
final int newLength = ArrayUtil.oversize(offStart + numProx, 4);
startOffsetsBuf = ArrayUtil.growExact(startOffsetsBuf, newLength);
lengthsBuf = ArrayUtil.growExact(lengthsBuf, newLength);
}
int lastOffset = 0, startOffset, endOffset;
for (int i = 0; i < numProx; ++i) {
startOffset = lastOffset + offsets.readVInt();
endOffset = startOffset + offsets.readVInt();
lastOffset = endOffset;
startOffsetsBuf[offStart + i] = startOffset;
lengthsBuf[offStart + i] = endOffset - startOffset;
}
}
curField.totalPositions += numProx;
}
// bulk merge is scary: its caused corruption bugs in the past.
// we try to be extra safe with this impl, but add an escape hatch to
// have a workaround for undiscovered bugs.
static final String BULK_MERGE_ENABLED_SYSPROP =
Lucene50CompressingTermVectorsWriter.class.getName() + ".enableBulkMerge";
static final boolean BULK_MERGE_ENABLED;
static {
boolean v = true;
try {
v = Boolean.parseBoolean(System.getProperty(BULK_MERGE_ENABLED_SYSPROP, "true"));
} catch (SecurityException ignored) {
}
BULK_MERGE_ENABLED = v;
}
@Override
public int merge(MergeState mergeState) throws IOException {
if (mergeState.needsIndexSort) {
// TODO: can we gain back some optos even if index is sorted? E.g. if sort results in large
// chunks of contiguous docs from one sub
// being copied over...?
return super.merge(mergeState);
}
int docCount = 0;
int numReaders = mergeState.maxDocs.length;
MatchingReaders matching = new MatchingReaders(mergeState);
for (int readerIndex = 0; readerIndex < numReaders; readerIndex++) {
Lucene50CompressingTermVectorsReader matchingVectorsReader = null;
final TermVectorsReader vectorsReader = mergeState.termVectorsReaders[readerIndex];
if (matching.matchingReaders[readerIndex]) {
// we can only bulk-copy if the matching reader is also a CompressingTermVectorsReader
if (vectorsReader != null
&& vectorsReader instanceof Lucene50CompressingTermVectorsReader) {
matchingVectorsReader = (Lucene50CompressingTermVectorsReader) vectorsReader;
}
}
final int maxDoc = mergeState.maxDocs[readerIndex];
final Bits liveDocs = mergeState.liveDocs[readerIndex];
if (matchingVectorsReader != null
&& matchingVectorsReader.getCompressionMode() == compressionMode
&& matchingVectorsReader.getChunkSize() == chunkSize
&& matchingVectorsReader.getVersion() == VERSION_CURRENT
&& matchingVectorsReader.getPackedIntsVersion() == PackedInts.VERSION_CURRENT
&& BULK_MERGE_ENABLED
&& liveDocs == null
&& !tooDirty(matchingVectorsReader)) {
// optimized merge, raw byte copy
// its not worth fine-graining this if there are deletions.
matchingVectorsReader.checkIntegrity();
// flush any pending chunks
if (!pendingDocs.isEmpty()) {
flush();
numDirtyChunks++; // incomplete: we had to force this flush
}
// iterate over each chunk. we use the vectors index to find chunk boundaries,
// read the docstart + doccount from the chunk header (we write a new header, since doc
// numbers will change),
// and just copy the bytes directly.
IndexInput rawDocs = matchingVectorsReader.getVectorsStream();
FieldsIndex index = matchingVectorsReader.getIndexReader();
rawDocs.seek(index.getStartPointer(0));
int docID = 0;
while (docID < maxDoc) {
// read header
int base = rawDocs.readVInt();
if (base != docID) {
throw new CorruptIndexException(
"invalid state: base=" + base + ", docID=" + docID, rawDocs);
}
int bufferedDocs = rawDocs.readVInt();
// write a new index entry and new header for this chunk.
indexWriter.writeIndex(bufferedDocs, vectorsStream.getFilePointer());
vectorsStream.writeVInt(docCount); // rebase
vectorsStream.writeVInt(bufferedDocs);
docID += bufferedDocs;
docCount += bufferedDocs;
numDocs += bufferedDocs;
if (docID > maxDoc) {
throw new CorruptIndexException(
"invalid state: base=" + base + ", count=" + bufferedDocs + ", maxDoc=" + maxDoc,
rawDocs);
}
// copy bytes until the next chunk boundary (or end of chunk data).
// using the stored fields index for this isn't the most efficient, but fast enough
// and is a source of redundancy for detecting bad things.
final long end;
if (docID == maxDoc) {
end = matchingVectorsReader.getMaxPointer();
} else {
end = index.getStartPointer(docID);
}
vectorsStream.copyBytes(rawDocs, end - rawDocs.getFilePointer());
}
if (rawDocs.getFilePointer() != matchingVectorsReader.getMaxPointer()) {
throw new CorruptIndexException(
"invalid state: pos="
+ rawDocs.getFilePointer()
+ ", max="
+ matchingVectorsReader.getMaxPointer(),
rawDocs);
}
// since we bulk merged all chunks, we inherit any dirty ones from this segment.
numDirtyChunks += matchingVectorsReader.getNumDirtyChunks();
numDirtyDocs += matchingVectorsReader.getNumDirtyDocs();
} else {
// naive merge...
if (vectorsReader != null) {
vectorsReader.checkIntegrity();
}
for (int i = 0; i < maxDoc; i++) {
if (liveDocs != null && liveDocs.get(i) == false) {
continue;
}
Fields vectors;
if (vectorsReader == null) {
vectors = null;
} else {
vectors = vectorsReader.get(i);
}
addAllDocVectors(vectors, mergeState);
++docCount;
}
}
}
finish(mergeState.mergeFieldInfos, docCount);
return docCount;
}
/**
* Returns true if we should recompress this reader, even though we could bulk merge compressed
* data
*
* <p>The last chunk written for a segment is typically incomplete, so without recompressing, in
* some worst-case situations (e.g. frequent reopen with tiny flushes), over time the compression
* ratio can degrade. This is a safety switch.
*/
boolean tooDirty(Lucene50CompressingTermVectorsReader candidate) {
// more than 1% dirty, or more than hard limit of 1024 dirty chunks
return candidate.getNumDirtyChunks() > 1024
|| candidate.getNumDirtyDocs() * 100 > candidate.getNumDocs();
}
@Override
public long ramBytesUsed() {
return positionsBuf.length
+ startOffsetsBuf.length
+ lengthsBuf.length
+ payloadLengthsBuf.length
+ termSuffixes.ramBytesUsed()
+ payloadBytes.ramBytesUsed()
+ lastTerm.bytes.length;
}
@Override
public Collection<Accountable> getChildResources() {
return List.of(termSuffixes, payloadBytes);
}
}

View File

@ -0,0 +1,52 @@
/*
* 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.lucene50.compressing;
import java.io.IOException;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/** RW impersonation of Lucene50CompressingTermVectorsFormat. */
public class Lucene50RWCompressingTermVectorsFormat extends Lucene50CompressingTermVectorsFormat {
/** Sole constructor. */
public Lucene50RWCompressingTermVectorsFormat(
String formatName,
String segmentSuffix,
CompressionMode compressionMode,
int chunkSize,
int blockSize) {
super(formatName, segmentSuffix, compressionMode, chunkSize, blockSize);
}
@Override
public final TermVectorsWriter vectorsWriter(
Directory directory, SegmentInfo segmentInfo, IOContext context) throws IOException {
return new Lucene50CompressingTermVectorsWriter(
directory,
segmentInfo,
segmentSuffix,
context,
formatName,
compressionMode,
chunkSize,
blockSize);
}
}

View File

@ -19,11 +19,13 @@ 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}. */
@ -65,4 +67,9 @@ public final class Lucene70RWCodec extends Lucene70Codec {
public CompoundFormat compoundFormat() {
return new Lucene50RWCompoundFormat();
}
@Override
public TermVectorsFormat termVectorsFormat() {
return new Lucene50RWTermVectorsFormat();
}
}

View File

@ -18,6 +18,7 @@ package org.apache.lucene.backward_codecs.lucene84;
import org.apache.lucene.backward_codecs.lucene50.Lucene50RWCompoundFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50RWStoredFieldsFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50RWTermVectorsFormat;
import org.apache.lucene.backward_codecs.lucene60.Lucene60RWPointsFormat;
import org.apache.lucene.backward_codecs.lucene70.Lucene70RWSegmentInfoFormat;
import org.apache.lucene.backward_codecs.lucene80.Lucene80RWNormsFormat;
@ -27,6 +28,7 @@ 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.PerFieldPostingsFormat;
/** RW impersonation of {@link Lucene84Codec}. */
@ -70,4 +72,9 @@ public class Lucene84RWCodec extends Lucene84Codec {
public final CompoundFormat compoundFormat() {
return new Lucene50RWCompoundFormat();
}
@Override
public TermVectorsFormat termVectorsFormat() {
return new Lucene50RWTermVectorsFormat();
}
}

View File

@ -18,6 +18,7 @@ package org.apache.lucene.backward_codecs.lucene86;
import org.apache.lucene.backward_codecs.lucene50.Lucene50RWCompoundFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50RWStoredFieldsFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50RWTermVectorsFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50StoredFieldsFormat;
import org.apache.lucene.backward_codecs.lucene80.Lucene80RWNormsFormat;
import org.apache.lucene.backward_codecs.lucene84.Lucene84RWPostingsFormat;
@ -25,6 +26,7 @@ import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
/** RW impersonation of {@link Lucene86Codec}. */
@ -69,4 +71,9 @@ public class Lucene86RWCodec extends Lucene86Codec {
public final CompoundFormat compoundFormat() {
return new Lucene50RWCompoundFormat();
}
@Override
public TermVectorsFormat termVectorsFormat() {
return new Lucene50RWTermVectorsFormat();
}
}

View File

@ -17,11 +17,13 @@
package org.apache.lucene.backward_codecs.lucene87;
import org.apache.lucene.backward_codecs.lucene50.Lucene50RWCompoundFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50RWTermVectorsFormat;
import org.apache.lucene.backward_codecs.lucene80.Lucene80RWNormsFormat;
import org.apache.lucene.backward_codecs.lucene84.Lucene84RWPostingsFormat;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
/** RW impersonation of {@link Lucene87Codec}. */
@ -50,4 +52,9 @@ public class Lucene87RWCodec extends Lucene87Codec {
public PostingsFormat postingsFormat() {
return postingsFormat;
}
@Override
public TermVectorsFormat termVectorsFormat() {
return new Lucene50RWTermVectorsFormat();
}
}

View File

@ -33,7 +33,7 @@ import org.apache.lucene.store.IOContext;
*
* @lucene.experimental
*/
public class CompressingTermVectorsFormat extends TermVectorsFormat {
public class Lucene90CompressingTermVectorsFormat extends TermVectorsFormat {
private final String formatName;
private final String segmentSuffix;
@ -42,7 +42,7 @@ public class CompressingTermVectorsFormat extends TermVectorsFormat {
private final int blockSize;
/**
* Create a new {@link CompressingTermVectorsFormat}.
* Create a new {@link Lucene90CompressingTermVectorsFormat}.
*
* <p><code>formatName</code> is the name of the format. This name will be used in the file
* formats to perform {@link CodecUtil#checkIndexHeader codec header checks}.
@ -50,7 +50,8 @@ public class CompressingTermVectorsFormat extends TermVectorsFormat {
* <p>The <code>compressionMode</code> parameter allows you to choose between compression
* algorithms that have various compression and decompression speeds so that you can pick the one
* that best fits your indexing and searching throughput. You should never instantiate two {@link
* CompressingTermVectorsFormat}s that have the same name but different {@link CompressionMode}s.
* Lucene90CompressingTermVectorsFormat}s that have the same name but different {@link
* CompressionMode}s.
*
* <p><code>chunkSize</code> is the minimum byte size of a chunk of documents. Higher values of
* <code>chunkSize</code> should improve the compression ratio but will require more memory at
@ -64,7 +65,7 @@ public class CompressingTermVectorsFormat extends TermVectorsFormat {
* @param blockSize the number of chunks to store in an index block.
* @see CompressionMode
*/
public CompressingTermVectorsFormat(
public Lucene90CompressingTermVectorsFormat(
String formatName,
String segmentSuffix,
CompressionMode compressionMode,
@ -87,14 +88,14 @@ public class CompressingTermVectorsFormat extends TermVectorsFormat {
public final TermVectorsReader vectorsReader(
Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context)
throws IOException {
return new CompressingTermVectorsReader(
return new Lucene90CompressingTermVectorsReader(
directory, segmentInfo, segmentSuffix, fieldInfos, context, formatName, compressionMode);
}
@Override
public final TermVectorsWriter vectorsWriter(
Directory directory, SegmentInfo segmentInfo, IOContext context) throws IOException {
return new CompressingTermVectorsWriter(
return new Lucene90CompressingTermVectorsWriter(
directory,
segmentInfo,
segmentSuffix,

View File

@ -51,11 +51,11 @@ import org.apache.lucene.util.packed.BlockPackedWriter;
import org.apache.lucene.util.packed.PackedInts;
/**
* {@link TermVectorsWriter} for {@link CompressingTermVectorsFormat}.
* {@link TermVectorsWriter} for {@link Lucene90CompressingTermVectorsFormat}.
*
* @lucene.experimental
*/
public final class CompressingTermVectorsWriter extends TermVectorsWriter {
public final class Lucene90CompressingTermVectorsWriter extends TermVectorsWriter {
// hard limit on the maximum number of documents per chunk
static final int MAX_DOCUMENTS_PER_CHUNK = 128;
@ -63,14 +63,10 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
static final String VECTORS_EXTENSION = "tvd";
static final String VECTORS_INDEX_EXTENSION = "tvx";
static final String VECTORS_META_EXTENSION = "tvm";
static final String VECTORS_INDEX_CODEC_NAME = "Lucene85TermVectorsIndex";
static final String VECTORS_INDEX_CODEC_NAME = "Lucene90TermVectorsIndex";
static final int VERSION_START = 1;
static final int VERSION_OFFHEAP_INDEX = 2;
/** Version where all metadata were moved to the meta file. */
static final int VERSION_META = 3;
static final int VERSION_CURRENT = VERSION_META;
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
static final int META_VERSION_START = 0;
static final int PACKED_BLOCK_SIZE = 64;
@ -227,7 +223,7 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
private final BlockPackedWriter writer;
/** Sole constructor. */
CompressingTermVectorsWriter(
Lucene90CompressingTermVectorsWriter(
Directory directory,
SegmentInfo si,
String segmentSuffix,
@ -792,7 +788,7 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
// we try to be extra safe with this impl, but add an escape hatch to
// have a workaround for undiscovered bugs.
static final String BULK_MERGE_ENABLED_SYSPROP =
CompressingTermVectorsWriter.class.getName() + ".enableBulkMerge";
Lucene90CompressingTermVectorsWriter.class.getName() + ".enableBulkMerge";
static final boolean BULK_MERGE_ENABLED;
static {
@ -818,12 +814,13 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
MatchingReaders matching = new MatchingReaders(mergeState);
for (int readerIndex = 0; readerIndex < numReaders; readerIndex++) {
CompressingTermVectorsReader matchingVectorsReader = null;
Lucene90CompressingTermVectorsReader matchingVectorsReader = null;
final TermVectorsReader vectorsReader = mergeState.termVectorsReaders[readerIndex];
if (matching.matchingReaders[readerIndex]) {
// we can only bulk-copy if the matching reader is also a CompressingTermVectorsReader
if (vectorsReader != null && vectorsReader instanceof CompressingTermVectorsReader) {
matchingVectorsReader = (CompressingTermVectorsReader) vectorsReader;
if (vectorsReader != null
&& vectorsReader instanceof Lucene90CompressingTermVectorsReader) {
matchingVectorsReader = (Lucene90CompressingTermVectorsReader) vectorsReader;
}
}
@ -936,7 +933,7 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
* some worst-case situations (e.g. frequent reopen with tiny flushes), over time the compression
* ratio can degrade. This is a safety switch.
*/
boolean tooDirty(CompressingTermVectorsReader candidate) {
boolean tooDirty(Lucene90CompressingTermVectorsReader candidate) {
// more than 1% dirty, or more than hard limit of 1024 dirty chunks
return candidate.getNumDirtyChunks() > 1024
|| candidate.getNumDirtyDocs() * 100 > candidate.getNumDocs();

View File

@ -23,19 +23,21 @@ import org.apache.lucene.index.SegmentReader;
/**
* Computes which segments have identical field name to number mappings, which allows stored fields
* and term vectors in this codec to be bulk-merged.
*
* @lucene.internal
*/
class MatchingReaders {
public class MatchingReaders {
/**
* {@link SegmentReader}s that have identical field name/number mapping, so their stored fields
* and term vectors may be bulk merged.
*/
final boolean[] matchingReaders;
public final boolean[] matchingReaders;
/** How many {@link #matchingReaders} are set. */
final int count;
MatchingReaders(MergeState mergeState) {
public MatchingReaders(MergeState mergeState) {
// If the i'th reader is a SegmentReader and has
// identical fieldName -> number mapping, then this
// array will be non-null at position i:

View File

@ -30,7 +30,6 @@ import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.VectorFormat;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat;
import org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat;
import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat;
@ -65,7 +64,7 @@ public class Lucene90Codec extends Codec {
}
}
private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene90TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene90FieldInfosFormat();
private final SegmentInfoFormat segmentInfosFormat = new Lucene86SegmentInfoFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene90LiveDocsFormat();

View File

@ -0,0 +1,160 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene90;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.compressing.FieldsIndexWriter;
import org.apache.lucene.codecs.compressing.Lucene90CompressingTermVectorsFormat;
import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.packed.BlockPackedWriter;
import org.apache.lucene.util.packed.PackedInts;
/**
* Lucene 9.0 {@link TermVectorsFormat term vectors format}.
*
* <p>Very similarly to {@link Lucene87StoredFieldsFormat}, this format is based on compressed
* chunks of data, with document-level granularity so that a document can never span across distinct
* chunks. Moreover, data is made as compact as possible:
*
* <ul>
* <li>textual data is compressed using the very light, <a
* href="http://code.google.com/p/lz4/">LZ4</a> compression algorithm,
* <li>binary data is written using fixed-size blocks of {@link PackedInts packed ints}.
* </ul>
*
* <p>Term vectors are stored using two files
*
* <ul>
* <li>a data file where terms, frequencies, positions, offsets and payloads are stored,
* <li>an index file, loaded into memory, used to locate specific documents in the data file.
* </ul>
*
* Looking up term vectors for any document requires at most 1 disk seek.
*
* <p><b>File formats</b>
*
* <ol>
* <li><a id="vector_data"></a>
* <p>A vector data file (extension <code>.tvd</code>). This file stores terms, frequencies,
* positions, offsets and payloads for every document. Upon writing a new segment, it
* accumulates data into memory until the buffer used to store terms and payloads grows beyond
* 4KB. Then it flushes all metadata, terms and positions to disk using <a
* href="http://code.google.com/p/lz4/">LZ4</a> compression for terms and payloads and {@link
* BlockPackedWriter blocks of packed ints} for positions.
* <p>Here is a more detailed description of the field data file format:
* <ul>
* <li>VectorData (.tvd) --&gt; &lt;Header&gt;, PackedIntsVersion, ChunkSize,
* &lt;Chunk&gt;<sup>ChunkCount</sup>, ChunkCount, DirtyChunkCount, Footer
* <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}
* <li>PackedIntsVersion --&gt; {@link PackedInts#VERSION_CURRENT} as a {@link
* DataOutput#writeVInt VInt}
* <li>ChunkSize is the number of bytes of terms to accumulate before flushing, as a {@link
* DataOutput#writeVInt VInt}
* <li>ChunkCount is not known in advance and is the number of chunks necessary to store all
* document of the segment
* <li>Chunk --&gt; DocBase, ChunkDocs, &lt; NumFields &gt;, &lt; FieldNums &gt;, &lt;
* FieldNumOffs &gt;, &lt; Flags &gt;, &lt; NumTerms &gt;, &lt; TermLengths &gt;, &lt;
* TermFreqs &gt;, &lt; Positions &gt;, &lt; StartOffsets &gt;, &lt; Lengths &gt;, &lt;
* PayloadLengths &gt;, &lt; TermAndPayloads &gt;
* <li>DocBase is the ID of the first doc of the chunk as a {@link DataOutput#writeVInt
* VInt}
* <li>ChunkDocs is the number of documents in the chunk
* <li>NumFields --&gt; DocNumFields<sup>ChunkDocs</sup>
* <li>DocNumFields is the number of fields for each doc, written as a {@link
* DataOutput#writeVInt VInt} if ChunkDocs==1 and as a {@link PackedInts} array
* otherwise
* <li>FieldNums --&gt; FieldNumDelta<sup>TotalDistincFields</sup>, a delta-encoded list of
* the sorted unique field numbers present in the chunk
* <li>FieldNumOffs --&gt; FieldNumOff<sup>TotalFields</sup>, as a {@link PackedInts} array
* <li>FieldNumOff is the offset of the field number in FieldNums
* <li>TotalFields is the total number of fields (sum of the values of NumFields)
* <li>Flags --&gt; Bit &lt; FieldFlags &gt;
* <li>Bit is a single bit which when true means that fields have the same options for every
* document in the chunk
* <li>FieldFlags --&gt; if Bit==1: Flag<sup>TotalDistinctFields</sup> else
* Flag<sup>TotalFields</sup>
* <li>Flag: a 3-bits int where:
* <ul>
* <li>the first bit means that the field has positions
* <li>the second bit means that the field has offsets
* <li>the third bit means that the field has payloads
* </ul>
* <li>NumTerms --&gt; FieldNumTerms<sup>TotalFields</sup>
* <li>FieldNumTerms: the number of terms for each field, using {@link BlockPackedWriter
* blocks of 64 packed ints}
* <li>TermLengths --&gt; PrefixLength<sup>TotalTerms</sup>
* SuffixLength<sup>TotalTerms</sup>
* <li>TotalTerms: total number of terms (sum of NumTerms)
* <li>PrefixLength: 0 for the first term of a field, the common prefix with the previous
* term otherwise using {@link BlockPackedWriter blocks of 64 packed ints}
* <li>SuffixLength: length of the term minus PrefixLength for every term using {@link
* BlockPackedWriter blocks of 64 packed ints}
* <li>TermFreqs --&gt; TermFreqMinus1<sup>TotalTerms</sup>
* <li>TermFreqMinus1: (frequency - 1) for each term using {@link BlockPackedWriter blocks
* of 64 packed ints}
* <li>Positions --&gt; PositionDelta<sup>TotalPositions</sup>
* <li>TotalPositions is the sum of frequencies of terms of all fields that have positions
* <li>PositionDelta: the absolute position for the first position of a term, and the
* difference with the previous positions for following positions using {@link
* BlockPackedWriter blocks of 64 packed ints}
* <li>StartOffsets --&gt; (AvgCharsPerTerm<sup>TotalDistinctFields</sup>)
* StartOffsetDelta<sup>TotalOffsets</sup>
* <li>TotalOffsets is the sum of frequencies of terms of all fields that have offsets
* <li>AvgCharsPerTerm: average number of chars per term, encoded as a float on 4 bytes.
* They are not present if no field has both positions and offsets enabled.
* <li>StartOffsetDelta: (startOffset - previousStartOffset - AvgCharsPerTerm *
* PositionDelta). previousStartOffset is 0 for the first offset and AvgCharsPerTerm is
* 0 if the field has no positions using {@link BlockPackedWriter blocks of 64 packed
* ints}
* <li>Lengths --&gt; LengthMinusTermLength<sup>TotalOffsets</sup>
* <li>LengthMinusTermLength: (endOffset - startOffset - termLength) using {@link
* BlockPackedWriter blocks of 64 packed ints}
* <li>PayloadLengths --&gt; PayloadLength<sup>TotalPayloads</sup>
* <li>TotalPayloads is the sum of frequencies of terms of all fields that have payloads
* <li>PayloadLength is the payload length encoded using {@link BlockPackedWriter blocks of
* 64 packed ints}
* <li>TermAndPayloads --&gt; LZ4-compressed representation of &lt; FieldTermsAndPayLoads
* &gt;<sup>TotalFields</sup>
* <li>FieldTermsAndPayLoads --&gt; Terms (Payloads)
* <li>Terms: term bytes
* <li>Payloads: payload bytes (if the field has payloads)
* <li>ChunkCount --&gt; the number of chunks in this file
* <li>DirtyChunkCount --&gt; the number of prematurely flushed chunks in this file
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}
* </ul>
* <li><a id="vector_index"></a>
* <p>An index file (extension <code>.tvx</code>).
* <ul>
* <li>VectorIndex (.tvx) --&gt; &lt;Header&gt;, &lt;ChunkIndex&gt;, Footer
* <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}
* <li>ChunkIndex: See {@link FieldsIndexWriter}
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}
* </ul>
* </ol>
*
* @lucene.experimental
*/
public final class Lucene90TermVectorsFormat extends Lucene90CompressingTermVectorsFormat {
/** Sole constructor. */
public Lucene90TermVectorsFormat() {
super("Lucene90TermVectorsData", "", CompressionMode.FAST, 1 << 12, 10);
}
}

View File

@ -165,7 +165,7 @@
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Normalization factors}. For
* each field in each document, a value is stored that is multiplied into the score for hits
* on that field.
* <li>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vectors}. For each
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vectors}. For each
* field in each document, the term vector (sometimes called document vector) may be stored. A
* term vector consists of term text and term frequency. To add Term Vectors to your index see
* the {@link org.apache.lucene.document.Field Field} constructors
@ -290,12 +290,12 @@
* <td>Encodes additional scoring factors or other per-document information.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Index}</td>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Index}</td>
* <td>.tvx</td>
* <td>Stores offset into the document data file</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Data}</td>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Data}</td>
* <td>.tvd</td>
* <td>Contains term vector data.</td>
* </tr>

View File

@ -24,7 +24,7 @@ import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.codecs.compressing.CompressingTermVectorsFormat;
import org.apache.lucene.codecs.compressing.Lucene90CompressingTermVectorsFormat;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FlushInfo;
@ -37,7 +37,7 @@ import org.apache.lucene.util.IntBlockPool;
final class SortingTermVectorsConsumer extends TermVectorsConsumer {
private static final TermVectorsFormat TEMP_TERM_VECTORS_FORMAT =
new CompressingTermVectorsFormat(
new Lucene90CompressingTermVectorsFormat(
"TempTermVectors", "", SortingStoredFieldsConsumer.NO_COMPRESSION, 8 * 1024, 10);
TrackingTmpOutputDirectoryWrapper tmpDirectory;

View File

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

View File

@ -102,7 +102,7 @@ public abstract class CompressingCodec extends FilterCodec {
}
private final CompressingStoredFieldsFormat storedFieldsFormat;
private final CompressingTermVectorsFormat termVectorsFormat;
private final Lucene90CompressingTermVectorsFormat termVectorsFormat;
/** Creates a compressing codec with a given segment suffix */
public CompressingCodec(
@ -117,7 +117,7 @@ public abstract class CompressingCodec extends FilterCodec {
new CompressingStoredFieldsFormat(
name, segmentSuffix, compressionMode, chunkSize, maxDocsPerChunk, blockShift);
this.termVectorsFormat =
new CompressingTermVectorsFormat(
new Lucene90CompressingTermVectorsFormat(
name, segmentSuffix, compressionMode, chunkSize, blockShift);
}

View File

@ -100,8 +100,8 @@ public class TestCompressingTermVectorsFormat extends BaseTermVectorsFormatTestC
// examine dirty counts:
for (LeafReaderContext leaf : ir2.leaves()) {
CodecReader sr = (CodecReader) leaf.reader();
CompressingTermVectorsReader reader =
(CompressingTermVectorsReader) sr.getTermVectorsReader();
Lucene90CompressingTermVectorsReader reader =
(Lucene90CompressingTermVectorsReader) sr.getTermVectorsReader();
assertTrue(reader.getNumDirtyDocs() > 0);
assertEquals(1, reader.getNumDirtyChunks());
}
@ -113,7 +113,8 @@ public class TestCompressingTermVectorsFormat extends BaseTermVectorsFormatTestC
ir.close();
ir = ir2;
CodecReader sr = (CodecReader) getOnlyLeafReader(ir);
CompressingTermVectorsReader reader = (CompressingTermVectorsReader) sr.getTermVectorsReader();
Lucene90CompressingTermVectorsReader reader =
(Lucene90CompressingTermVectorsReader) sr.getTermVectorsReader();
// we could get lucky, and have zero, but typically one.
assertTrue(reader.getNumDirtyChunks() <= 1);
ir.close();