mirror of https://github.com/apache/lucene.git
LUCENE-9705: Create Lucene90TermVectorsFormat (#2334)
This commit is contained in:
parent
84a35dfaea
commit
f8be421ae1
|
@ -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() {
|
|
@ -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();
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
||||
}
|
|
@ -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
|
||||
+ ")";
|
||||
}
|
||||
}
|
|
@ -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
|
|
@ -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;
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,
|
File diff suppressed because it is too large
Load Diff
|
@ -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();
|
|
@ -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:
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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) --> <Header>, PackedIntsVersion, ChunkSize,
|
||||
* <Chunk><sup>ChunkCount</sup>, ChunkCount, DirtyChunkCount, Footer
|
||||
* <li>Header --> {@link CodecUtil#writeIndexHeader IndexHeader}
|
||||
* <li>PackedIntsVersion --> {@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 --> DocBase, ChunkDocs, < NumFields >, < FieldNums >, <
|
||||
* FieldNumOffs >, < Flags >, < NumTerms >, < TermLengths >, <
|
||||
* TermFreqs >, < Positions >, < StartOffsets >, < Lengths >, <
|
||||
* PayloadLengths >, < TermAndPayloads >
|
||||
* <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 --> 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 --> FieldNumDelta<sup>TotalDistincFields</sup>, a delta-encoded list of
|
||||
* the sorted unique field numbers present in the chunk
|
||||
* <li>FieldNumOffs --> 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 --> Bit < FieldFlags >
|
||||
* <li>Bit is a single bit which when true means that fields have the same options for every
|
||||
* document in the chunk
|
||||
* <li>FieldFlags --> 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 --> FieldNumTerms<sup>TotalFields</sup>
|
||||
* <li>FieldNumTerms: the number of terms for each field, using {@link BlockPackedWriter
|
||||
* blocks of 64 packed ints}
|
||||
* <li>TermLengths --> 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 --> TermFreqMinus1<sup>TotalTerms</sup>
|
||||
* <li>TermFreqMinus1: (frequency - 1) for each term using {@link BlockPackedWriter blocks
|
||||
* of 64 packed ints}
|
||||
* <li>Positions --> 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 --> (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 --> LengthMinusTermLength<sup>TotalOffsets</sup>
|
||||
* <li>LengthMinusTermLength: (endOffset - startOffset - termLength) using {@link
|
||||
* BlockPackedWriter blocks of 64 packed ints}
|
||||
* <li>PayloadLengths --> 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 --> LZ4-compressed representation of < FieldTermsAndPayLoads
|
||||
* ><sup>TotalFields</sup>
|
||||
* <li>FieldTermsAndPayLoads --> Terms (Payloads)
|
||||
* <li>Terms: term bytes
|
||||
* <li>Payloads: payload bytes (if the field has payloads)
|
||||
* <li>ChunkCount --> the number of chunks in this file
|
||||
* <li>DirtyChunkCount --> the number of prematurely flushed chunks in this file
|
||||
* <li>Footer --> {@link CodecUtil#writeFooter CodecFooter}
|
||||
* </ul>
|
||||
* <li><a id="vector_index"></a>
|
||||
* <p>An index file (extension <code>.tvx</code>).
|
||||
* <ul>
|
||||
* <li>VectorIndex (.tvx) --> <Header>, <ChunkIndex>, Footer
|
||||
* <li>Header --> {@link CodecUtil#writeIndexHeader IndexHeader}
|
||||
* <li>ChunkIndex: See {@link FieldsIndexWriter}
|
||||
* <li>Footer --> {@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);
|
||||
}
|
||||
}
|
|
@ -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>
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
Loading…
Reference in New Issue