mirror of https://github.com/apache/lucene.git
LUCENE-5969: copy over cruft for back compat
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5969@1628019 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
35db73c39e
commit
1de80597a1
|
@ -0,0 +1,55 @@
|
|||
package org.apache.lucene.codecs.lucene41;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
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;
|
||||
|
||||
/**
|
||||
* Lucene 4.1 stored fields format.
|
||||
*/
|
||||
@Deprecated
|
||||
public class Lucene41StoredFieldsFormat extends StoredFieldsFormat {
|
||||
static final String FORMAT_NAME = "Lucene41StoredFields";
|
||||
static final String SEGMENT_SUFFIX = "";
|
||||
static final CompressionMode COMPRESSION_MODE = CompressionMode.FAST;
|
||||
static final int CHUNK_SIZE = 1 << 14;
|
||||
|
||||
@Override
|
||||
public final StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
|
||||
return new Lucene41StoredFieldsReader(directory, si, SEGMENT_SUFFIX, fn, context, FORMAT_NAME, COMPRESSION_MODE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context) throws IOException {
|
||||
throw new UnsupportedOperationException("this codec can only be used for reading");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getClass().getSimpleName() + "(compressionMode=" + COMPRESSION_MODE + ", chunkSize=" + CHUNK_SIZE + ")";
|
||||
}
|
||||
}
|
|
@ -0,0 +1,214 @@
|
|||
package org.apache.lucene.codecs.lucene41;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import static org.apache.lucene.util.BitUtil.zigZagDecode;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
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;
|
||||
|
||||
/**
|
||||
* Random-access reader for {@code Lucene41CompressingStoredFieldsIndexWriter}.
|
||||
* @deprecated only for reading old segments
|
||||
*/
|
||||
@Deprecated
|
||||
public final class Lucene41StoredFieldsIndexReader implements Cloneable, Accountable {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Lucene41StoredFieldsIndexReader.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
|
||||
public Lucene41StoredFieldsIndexReader(IndexInput fieldsIndexIn, SegmentInfo si) throws IOException {
|
||||
maxDoc = si.getDocCount();
|
||||
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 = Arrays.copyOf(docBases, newSize);
|
||||
startPointers = Arrays.copyOf(startPointers, newSize);
|
||||
avgChunkDocs = Arrays.copyOf(avgChunkDocs, newSize);
|
||||
avgChunkSizes = Arrays.copyOf(avgChunkSizes, newSize);
|
||||
docBasesDeltas = Arrays.copyOf(docBasesDeltas, newSize);
|
||||
startPointersDeltas = Arrays.copyOf(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 = Arrays.copyOf(docBases, blockCount);
|
||||
this.startPointers = Arrays.copyOf(startPointers, blockCount);
|
||||
this.avgChunkDocs = Arrays.copyOf(avgChunkDocs, blockCount);
|
||||
this.avgChunkSizes = Arrays.copyOf(avgChunkSizes, blockCount);
|
||||
this.docBasesDeltas = Arrays.copyOf(docBasesDeltas, blockCount);
|
||||
this.startPointersDeltas = Arrays.copyOf(startPointersDeltas, 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;
|
||||
}
|
||||
|
||||
public 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 Lucene41StoredFieldsIndexReader 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 Iterable<? extends 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 resources;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getClass().getSimpleName() + "(blocks=" + docBases.length + ")";
|
||||
}
|
||||
}
|
|
@ -0,0 +1,417 @@
|
|||
package org.apache.lucene.codecs.lucene41;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Collections;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.compressing.CompressionMode;
|
||||
import org.apache.lucene.codecs.compressing.Decompressor;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.StoredFieldVisitor;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.ByteArrayDataInput;
|
||||
import org.apache.lucene.store.ChecksumIndexInput;
|
||||
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.util.Accountable;
|
||||
import org.apache.lucene.util.Accountables;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
* {@link StoredFieldsReader} impl for {@code Lucene41StoredFieldsFormat}.
|
||||
* @deprecated only for reading old segments
|
||||
*/
|
||||
@Deprecated
|
||||
final class Lucene41StoredFieldsReader extends StoredFieldsReader {
|
||||
|
||||
// Do not reuse the decompression buffer when there is more than 32kb to decompress
|
||||
private static final int BUFFER_REUSE_THRESHOLD = 1 << 15;
|
||||
|
||||
static final int STRING = 0x00;
|
||||
static final int BYTE_ARR = 0x01;
|
||||
static final int NUMERIC_INT = 0x02;
|
||||
static final int NUMERIC_FLOAT = 0x03;
|
||||
static final int NUMERIC_LONG = 0x04;
|
||||
static final int NUMERIC_DOUBLE = 0x05;
|
||||
|
||||
static final String CODEC_SFX_IDX = "Index";
|
||||
static final String CODEC_SFX_DAT = "Data";
|
||||
|
||||
static final int TYPE_BITS = PackedInts.bitsRequired(NUMERIC_DOUBLE);
|
||||
static final int TYPE_MASK = (int) PackedInts.maxValue(TYPE_BITS);
|
||||
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_BIG_CHUNKS = 1;
|
||||
static final int VERSION_CHECKSUM = 2;
|
||||
static final int VERSION_CURRENT = VERSION_CHECKSUM;
|
||||
|
||||
/** Extension of stored fields file */
|
||||
public static final String FIELDS_EXTENSION = "fdt";
|
||||
|
||||
/** Extension of stored fields index file */
|
||||
public static final String FIELDS_INDEX_EXTENSION = "fdx";
|
||||
|
||||
private final int version;
|
||||
private final FieldInfos fieldInfos;
|
||||
private final Lucene41StoredFieldsIndexReader indexReader;
|
||||
private final long maxPointer;
|
||||
private final IndexInput fieldsStream;
|
||||
private final int chunkSize;
|
||||
private final int packedIntsVersion;
|
||||
private final CompressionMode compressionMode;
|
||||
private final Decompressor decompressor;
|
||||
private final BytesRef bytes;
|
||||
private final int numDocs;
|
||||
private boolean closed;
|
||||
|
||||
// used by clone
|
||||
private Lucene41StoredFieldsReader(Lucene41StoredFieldsReader reader) {
|
||||
this.version = reader.version;
|
||||
this.fieldInfos = reader.fieldInfos;
|
||||
this.fieldsStream = reader.fieldsStream.clone();
|
||||
this.indexReader = reader.indexReader.clone();
|
||||
this.maxPointer = reader.maxPointer;
|
||||
this.chunkSize = reader.chunkSize;
|
||||
this.packedIntsVersion = reader.packedIntsVersion;
|
||||
this.compressionMode = reader.compressionMode;
|
||||
this.decompressor = reader.decompressor.clone();
|
||||
this.numDocs = reader.numDocs;
|
||||
this.bytes = new BytesRef(reader.bytes.bytes.length);
|
||||
this.closed = false;
|
||||
}
|
||||
|
||||
/** Sole constructor. */
|
||||
public Lucene41StoredFieldsReader(Directory d, SegmentInfo si, String segmentSuffix, FieldInfos fn,
|
||||
IOContext context, String formatName, CompressionMode compressionMode) throws IOException {
|
||||
this.compressionMode = compressionMode;
|
||||
final String segment = si.name;
|
||||
boolean success = false;
|
||||
fieldInfos = fn;
|
||||
numDocs = si.getDocCount();
|
||||
ChecksumIndexInput indexStream = null;
|
||||
try {
|
||||
final String indexStreamFN = IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_INDEX_EXTENSION);
|
||||
final String fieldsStreamFN = IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_EXTENSION);
|
||||
// Load the index into memory
|
||||
indexStream = d.openChecksumInput(indexStreamFN, context);
|
||||
final String codecNameIdx = formatName + CODEC_SFX_IDX;
|
||||
version = CodecUtil.checkHeader(indexStream, codecNameIdx, VERSION_START, VERSION_CURRENT);
|
||||
assert CodecUtil.headerLength(codecNameIdx) == indexStream.getFilePointer();
|
||||
indexReader = new Lucene41StoredFieldsIndexReader(indexStream, si);
|
||||
|
||||
long maxPointer = -1;
|
||||
|
||||
if (version >= VERSION_CHECKSUM) {
|
||||
maxPointer = indexStream.readVLong();
|
||||
CodecUtil.checkFooter(indexStream);
|
||||
} else {
|
||||
CodecUtil.checkEOF(indexStream);
|
||||
}
|
||||
indexStream.close();
|
||||
indexStream = null;
|
||||
|
||||
// Open the data file and read metadata
|
||||
fieldsStream = d.openInput(fieldsStreamFN, context);
|
||||
if (version >= VERSION_CHECKSUM) {
|
||||
if (maxPointer + CodecUtil.footerLength() != fieldsStream.length()) {
|
||||
throw new CorruptIndexException("Invalid fieldsStream maxPointer (file truncated?): maxPointer=" + maxPointer + ", length=" + fieldsStream.length(), fieldsStream);
|
||||
}
|
||||
} else {
|
||||
maxPointer = fieldsStream.length();
|
||||
}
|
||||
this.maxPointer = maxPointer;
|
||||
final String codecNameDat = formatName + CODEC_SFX_DAT;
|
||||
final int fieldsVersion = CodecUtil.checkHeader(fieldsStream, codecNameDat, VERSION_START, VERSION_CURRENT);
|
||||
if (version != fieldsVersion) {
|
||||
throw new CorruptIndexException("Version mismatch between stored fields index and data: " + version + " != " + fieldsVersion, fieldsStream);
|
||||
}
|
||||
assert CodecUtil.headerLength(codecNameDat) == fieldsStream.getFilePointer();
|
||||
|
||||
if (version >= VERSION_BIG_CHUNKS) {
|
||||
chunkSize = fieldsStream.readVInt();
|
||||
} else {
|
||||
chunkSize = -1;
|
||||
}
|
||||
packedIntsVersion = fieldsStream.readVInt();
|
||||
decompressor = compressionMode.newDecompressor();
|
||||
this.bytes = new BytesRef();
|
||||
|
||||
if (version >= VERSION_CHECKSUM) {
|
||||
// NOTE: data file is too costly to verify checksum against all the bytes on open,
|
||||
// but for now we at least verify proper structure of the checksum footer: which looks
|
||||
// for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
|
||||
// such as file truncation.
|
||||
CodecUtil.retrieveChecksum(fieldsStream);
|
||||
}
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(this, indexStream);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @throws AlreadyClosedException if this FieldsReader is closed
|
||||
*/
|
||||
private void ensureOpen() throws AlreadyClosedException {
|
||||
if (closed) {
|
||||
throw new AlreadyClosedException("this FieldsReader is closed");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Close the underlying {@link IndexInput}s.
|
||||
*/
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (!closed) {
|
||||
IOUtils.close(fieldsStream);
|
||||
closed = true;
|
||||
}
|
||||
}
|
||||
|
||||
private static void readField(DataInput in, StoredFieldVisitor visitor, FieldInfo info, int bits) throws IOException {
|
||||
switch (bits & TYPE_MASK) {
|
||||
case BYTE_ARR:
|
||||
int length = in.readVInt();
|
||||
byte[] data = new byte[length];
|
||||
in.readBytes(data, 0, length);
|
||||
visitor.binaryField(info, data);
|
||||
break;
|
||||
case STRING:
|
||||
length = in.readVInt();
|
||||
data = new byte[length];
|
||||
in.readBytes(data, 0, length);
|
||||
visitor.stringField(info, new String(data, StandardCharsets.UTF_8));
|
||||
break;
|
||||
case NUMERIC_INT:
|
||||
visitor.intField(info, in.readInt());
|
||||
break;
|
||||
case NUMERIC_FLOAT:
|
||||
visitor.floatField(info, Float.intBitsToFloat(in.readInt()));
|
||||
break;
|
||||
case NUMERIC_LONG:
|
||||
visitor.longField(info, in.readLong());
|
||||
break;
|
||||
case NUMERIC_DOUBLE:
|
||||
visitor.doubleField(info, Double.longBitsToDouble(in.readLong()));
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError("Unknown type flag: " + Integer.toHexString(bits));
|
||||
}
|
||||
}
|
||||
|
||||
private static void skipField(DataInput in, int bits) throws IOException {
|
||||
switch (bits & TYPE_MASK) {
|
||||
case BYTE_ARR:
|
||||
case STRING:
|
||||
final int length = in.readVInt();
|
||||
in.skipBytes(length);
|
||||
break;
|
||||
case NUMERIC_INT:
|
||||
case NUMERIC_FLOAT:
|
||||
in.readInt();
|
||||
break;
|
||||
case NUMERIC_LONG:
|
||||
case NUMERIC_DOUBLE:
|
||||
in.readLong();
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError("Unknown type flag: " + Integer.toHexString(bits));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visitDocument(int docID, StoredFieldVisitor visitor)
|
||||
throws IOException {
|
||||
fieldsStream.seek(indexReader.getStartPointer(docID));
|
||||
|
||||
final int docBase = fieldsStream.readVInt();
|
||||
final int chunkDocs = fieldsStream.readVInt();
|
||||
if (docID < docBase
|
||||
|| docID >= docBase + chunkDocs
|
||||
|| docBase + chunkDocs > numDocs) {
|
||||
throw new CorruptIndexException("Corrupted: docID=" + docID
|
||||
+ ", docBase=" + docBase + ", chunkDocs=" + chunkDocs
|
||||
+ ", numDocs=" + numDocs, fieldsStream);
|
||||
}
|
||||
|
||||
final int numStoredFields, offset, length, totalLength;
|
||||
if (chunkDocs == 1) {
|
||||
numStoredFields = fieldsStream.readVInt();
|
||||
offset = 0;
|
||||
length = fieldsStream.readVInt();
|
||||
totalLength = length;
|
||||
} else {
|
||||
final int bitsPerStoredFields = fieldsStream.readVInt();
|
||||
if (bitsPerStoredFields == 0) {
|
||||
numStoredFields = fieldsStream.readVInt();
|
||||
} else if (bitsPerStoredFields > 31) {
|
||||
throw new CorruptIndexException("bitsPerStoredFields=" + bitsPerStoredFields, fieldsStream);
|
||||
} else {
|
||||
final long filePointer = fieldsStream.getFilePointer();
|
||||
final PackedInts.Reader reader = PackedInts.getDirectReaderNoHeader(fieldsStream, PackedInts.Format.PACKED, packedIntsVersion, chunkDocs, bitsPerStoredFields);
|
||||
numStoredFields = (int) (reader.get(docID - docBase));
|
||||
fieldsStream.seek(filePointer + PackedInts.Format.PACKED.byteCount(packedIntsVersion, chunkDocs, bitsPerStoredFields));
|
||||
}
|
||||
|
||||
final int bitsPerLength = fieldsStream.readVInt();
|
||||
if (bitsPerLength == 0) {
|
||||
length = fieldsStream.readVInt();
|
||||
offset = (docID - docBase) * length;
|
||||
totalLength = chunkDocs * length;
|
||||
} else if (bitsPerStoredFields > 31) {
|
||||
throw new CorruptIndexException("bitsPerLength=" + bitsPerLength, fieldsStream);
|
||||
} else {
|
||||
final PackedInts.ReaderIterator it = PackedInts.getReaderIteratorNoHeader(fieldsStream, PackedInts.Format.PACKED, packedIntsVersion, chunkDocs, bitsPerLength, 1);
|
||||
int off = 0;
|
||||
for (int i = 0; i < docID - docBase; ++i) {
|
||||
off += it.next();
|
||||
}
|
||||
offset = off;
|
||||
length = (int) it.next();
|
||||
off += length;
|
||||
for (int i = docID - docBase + 1; i < chunkDocs; ++i) {
|
||||
off += it.next();
|
||||
}
|
||||
totalLength = off;
|
||||
}
|
||||
}
|
||||
|
||||
if ((length == 0) != (numStoredFields == 0)) {
|
||||
throw new CorruptIndexException("length=" + length + ", numStoredFields=" + numStoredFields, fieldsStream);
|
||||
}
|
||||
if (numStoredFields == 0) {
|
||||
// nothing to do
|
||||
return;
|
||||
}
|
||||
|
||||
final DataInput documentInput;
|
||||
if (version >= VERSION_BIG_CHUNKS && totalLength >= 2 * chunkSize) {
|
||||
assert chunkSize > 0;
|
||||
assert offset < chunkSize;
|
||||
|
||||
decompressor.decompress(fieldsStream, chunkSize, offset, Math.min(length, chunkSize - offset), bytes);
|
||||
documentInput = new DataInput() {
|
||||
|
||||
int decompressed = bytes.length;
|
||||
|
||||
void fillBuffer() throws IOException {
|
||||
assert decompressed <= length;
|
||||
if (decompressed == length) {
|
||||
throw new EOFException();
|
||||
}
|
||||
final int toDecompress = Math.min(length - decompressed, chunkSize);
|
||||
decompressor.decompress(fieldsStream, toDecompress, 0, toDecompress, bytes);
|
||||
decompressed += toDecompress;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte readByte() throws IOException {
|
||||
if (bytes.length == 0) {
|
||||
fillBuffer();
|
||||
}
|
||||
--bytes.length;
|
||||
return bytes.bytes[bytes.offset++];
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readBytes(byte[] b, int offset, int len) throws IOException {
|
||||
while (len > bytes.length) {
|
||||
System.arraycopy(bytes.bytes, bytes.offset, b, offset, bytes.length);
|
||||
len -= bytes.length;
|
||||
offset += bytes.length;
|
||||
fillBuffer();
|
||||
}
|
||||
System.arraycopy(bytes.bytes, bytes.offset, b, offset, len);
|
||||
bytes.offset += len;
|
||||
bytes.length -= len;
|
||||
}
|
||||
|
||||
};
|
||||
} else {
|
||||
final BytesRef bytes = totalLength <= BUFFER_REUSE_THRESHOLD ? this.bytes : new BytesRef();
|
||||
decompressor.decompress(fieldsStream, totalLength, offset, length, bytes);
|
||||
assert bytes.length == length;
|
||||
documentInput = new ByteArrayDataInput(bytes.bytes, bytes.offset, bytes.length);
|
||||
}
|
||||
|
||||
for (int fieldIDX = 0; fieldIDX < numStoredFields; fieldIDX++) {
|
||||
final long infoAndBits = documentInput.readVLong();
|
||||
final int fieldNumber = (int) (infoAndBits >>> TYPE_BITS);
|
||||
final FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
|
||||
|
||||
final int bits = (int) (infoAndBits & TYPE_MASK);
|
||||
assert bits <= NUMERIC_DOUBLE: "bits=" + Integer.toHexString(bits);
|
||||
|
||||
switch(visitor.needsField(fieldInfo)) {
|
||||
case YES:
|
||||
readField(documentInput, visitor, fieldInfo, bits);
|
||||
break;
|
||||
case NO:
|
||||
skipField(documentInput, bits);
|
||||
break;
|
||||
case STOP:
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public StoredFieldsReader clone() {
|
||||
ensureOpen();
|
||||
return new Lucene41StoredFieldsReader(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return indexReader.ramBytesUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
return Collections.singleton(Accountables.namedAccountable("stored field index", indexReader));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void checkIntegrity() throws IOException {
|
||||
if (version >= VERSION_CHECKSUM) {
|
||||
CodecUtil.checksumEntireFile(fieldsStream);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getClass().getSimpleName() + "(mode=" + compressionMode + ",chunksize=" + chunkSize + ")";
|
||||
}
|
||||
}
|
|
@ -80,12 +80,12 @@ public class Lucene42Codec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public final StoredFieldsFormat storedFieldsFormat() {
|
||||
public StoredFieldsFormat storedFieldsFormat() {
|
||||
return fieldsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final TermVectorsFormat termVectorsFormat() {
|
||||
public TermVectorsFormat termVectorsFormat() {
|
||||
return vectorsFormat;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,57 @@
|
|||
package org.apache.lucene.codecs.lucene42;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
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;
|
||||
|
||||
/**
|
||||
* Lucene 4.2 {@link TermVectorsFormat term vectors format}.
|
||||
* @deprecated only for reading old segments
|
||||
*/
|
||||
@Deprecated
|
||||
public class Lucene42TermVectorsFormat extends TermVectorsFormat {
|
||||
// this is actually what 4.2 TVF wrote!
|
||||
static final String FORMAT_NAME = "Lucene41StoredFields";
|
||||
static final String SEGMENT_SUFFIX = "";
|
||||
static final CompressionMode COMPRESSION_MODE = CompressionMode.FAST;
|
||||
static final int CHUNK_SIZE = 1 << 12;
|
||||
|
||||
@Override
|
||||
public final TermVectorsReader vectorsReader(Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) throws IOException {
|
||||
return new Lucene42TermVectorsReader(directory, segmentInfo, SEGMENT_SUFFIX, fieldInfos, context, FORMAT_NAME, COMPRESSION_MODE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo segmentInfo, IOContext context) throws IOException {
|
||||
throw new UnsupportedOperationException("this codec can only be used for reading");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getClass().getSimpleName() + "(compressionMode=" + COMPRESSION_MODE + ", chunkSize=" + CHUNK_SIZE + ")";
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -83,12 +83,12 @@ public class Lucene45Codec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public final StoredFieldsFormat storedFieldsFormat() {
|
||||
public StoredFieldsFormat storedFieldsFormat() {
|
||||
return fieldsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final TermVectorsFormat termVectorsFormat() {
|
||||
public TermVectorsFormat termVectorsFormat() {
|
||||
return vectorsFormat;
|
||||
}
|
||||
|
||||
|
|
|
@ -80,12 +80,12 @@ public class Lucene46Codec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public final StoredFieldsFormat storedFieldsFormat() {
|
||||
public StoredFieldsFormat storedFieldsFormat() {
|
||||
return fieldsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final TermVectorsFormat termVectorsFormat() {
|
||||
public TermVectorsFormat termVectorsFormat() {
|
||||
return vectorsFormat;
|
||||
}
|
||||
|
||||
|
|
|
@ -79,12 +79,12 @@ public class Lucene49Codec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public final StoredFieldsFormat storedFieldsFormat() {
|
||||
public StoredFieldsFormat storedFieldsFormat() {
|
||||
return fieldsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final TermVectorsFormat termVectorsFormat() {
|
||||
public TermVectorsFormat termVectorsFormat() {
|
||||
return vectorsFormat;
|
||||
}
|
||||
|
||||
|
|
|
@ -15,7 +15,6 @@ import org.apache.lucene.codecs.lucene40.Lucene40RWDocValuesFormat;
|
|||
import org.apache.lucene.codecs.lucene40.Lucene40RWNormsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40RWSegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40RWTermVectorsFormat;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -39,7 +38,7 @@ import org.apache.lucene.util.LuceneTestCase;
|
|||
*/
|
||||
@SuppressWarnings("deprecation")
|
||||
public class Lucene41RWCodec extends Lucene41Codec {
|
||||
private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
|
||||
private final StoredFieldsFormat fieldsFormat = new Lucene41RWStoredFieldsFormat();
|
||||
private final FieldInfosFormat fieldInfos = new Lucene40FieldInfosFormat() {
|
||||
@Override
|
||||
public FieldInfosWriter getFieldInfosWriter() throws IOException {
|
||||
|
|
|
@ -0,0 +1,33 @@
|
|||
package org.apache.lucene.codecs.lucene41;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
||||
/** read-write version of Lucene41StoredsFieldsFormat for testing */
|
||||
public class Lucene41RWStoredFieldsFormat extends Lucene41StoredFieldsFormat {
|
||||
@Override
|
||||
public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context) throws IOException {
|
||||
return new Lucene41StoredFieldsWriter(directory, si, SEGMENT_SUFFIX, context, FORMAT_NAME, COMPRESSION_MODE, CHUNK_SIZE);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,169 @@
|
|||
package org.apache.lucene.codecs.lucene41;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import static org.apache.lucene.util.BitUtil.zigZagEncode;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
* writer for lucene 4.x stored fields/vectors index for testing
|
||||
*/
|
||||
public final class Lucene41StoredFieldsIndexWriter implements Closeable {
|
||||
|
||||
static final int BLOCK_SIZE = 1024; // number of chunks to serialize at once
|
||||
|
||||
final IndexOutput fieldsIndexOut;
|
||||
int totalDocs;
|
||||
int blockDocs;
|
||||
int blockChunks;
|
||||
long firstStartPointer;
|
||||
long maxStartPointer;
|
||||
final int[] docBaseDeltas;
|
||||
final long[] startPointerDeltas;
|
||||
|
||||
public Lucene41StoredFieldsIndexWriter(IndexOutput indexOutput) throws IOException {
|
||||
this.fieldsIndexOut = indexOutput;
|
||||
reset();
|
||||
totalDocs = 0;
|
||||
docBaseDeltas = new int[BLOCK_SIZE];
|
||||
startPointerDeltas = new long[BLOCK_SIZE];
|
||||
fieldsIndexOut.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
}
|
||||
|
||||
private void reset() {
|
||||
blockChunks = 0;
|
||||
blockDocs = 0;
|
||||
firstStartPointer = -1; // means unset
|
||||
}
|
||||
|
||||
private void writeBlock() throws IOException {
|
||||
assert blockChunks > 0;
|
||||
fieldsIndexOut.writeVInt(blockChunks);
|
||||
|
||||
// The trick here is that we only store the difference from the average start
|
||||
// pointer or doc base, this helps save bits per value.
|
||||
// And in order to prevent a few chunks that would be far from the average to
|
||||
// raise the number of bits per value for all of them, we only encode blocks
|
||||
// of 1024 chunks at once
|
||||
// See LUCENE-4512
|
||||
|
||||
// doc bases
|
||||
final int avgChunkDocs;
|
||||
if (blockChunks == 1) {
|
||||
avgChunkDocs = 0;
|
||||
} else {
|
||||
avgChunkDocs = Math.round((float) (blockDocs - docBaseDeltas[blockChunks - 1]) / (blockChunks - 1));
|
||||
}
|
||||
fieldsIndexOut.writeVInt(totalDocs - blockDocs); // docBase
|
||||
fieldsIndexOut.writeVInt(avgChunkDocs);
|
||||
int docBase = 0;
|
||||
long maxDelta = 0;
|
||||
for (int i = 0; i < blockChunks; ++i) {
|
||||
final int delta = docBase - avgChunkDocs * i;
|
||||
maxDelta |= zigZagEncode(delta);
|
||||
docBase += docBaseDeltas[i];
|
||||
}
|
||||
|
||||
final int bitsPerDocBase = PackedInts.bitsRequired(maxDelta);
|
||||
fieldsIndexOut.writeVInt(bitsPerDocBase);
|
||||
PackedInts.Writer writer = PackedInts.getWriterNoHeader(fieldsIndexOut,
|
||||
PackedInts.Format.PACKED, blockChunks, bitsPerDocBase, 1);
|
||||
docBase = 0;
|
||||
for (int i = 0; i < blockChunks; ++i) {
|
||||
final long delta = docBase - avgChunkDocs * i;
|
||||
assert PackedInts.bitsRequired(zigZagEncode(delta)) <= writer.bitsPerValue();
|
||||
writer.add(zigZagEncode(delta));
|
||||
docBase += docBaseDeltas[i];
|
||||
}
|
||||
writer.finish();
|
||||
|
||||
// start pointers
|
||||
fieldsIndexOut.writeVLong(firstStartPointer);
|
||||
final long avgChunkSize;
|
||||
if (blockChunks == 1) {
|
||||
avgChunkSize = 0;
|
||||
} else {
|
||||
avgChunkSize = (maxStartPointer - firstStartPointer) / (blockChunks - 1);
|
||||
}
|
||||
fieldsIndexOut.writeVLong(avgChunkSize);
|
||||
long startPointer = 0;
|
||||
maxDelta = 0;
|
||||
for (int i = 0; i < blockChunks; ++i) {
|
||||
startPointer += startPointerDeltas[i];
|
||||
final long delta = startPointer - avgChunkSize * i;
|
||||
maxDelta |= zigZagEncode(delta);
|
||||
}
|
||||
|
||||
final int bitsPerStartPointer = PackedInts.bitsRequired(maxDelta);
|
||||
fieldsIndexOut.writeVInt(bitsPerStartPointer);
|
||||
writer = PackedInts.getWriterNoHeader(fieldsIndexOut, PackedInts.Format.PACKED,
|
||||
blockChunks, bitsPerStartPointer, 1);
|
||||
startPointer = 0;
|
||||
for (int i = 0; i < blockChunks; ++i) {
|
||||
startPointer += startPointerDeltas[i];
|
||||
final long delta = startPointer - avgChunkSize * i;
|
||||
assert PackedInts.bitsRequired(zigZagEncode(delta)) <= writer.bitsPerValue();
|
||||
writer.add(zigZagEncode(delta));
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
|
||||
public void writeIndex(int numDocs, long startPointer) throws IOException {
|
||||
if (blockChunks == BLOCK_SIZE) {
|
||||
writeBlock();
|
||||
reset();
|
||||
}
|
||||
|
||||
if (firstStartPointer == -1) {
|
||||
firstStartPointer = maxStartPointer = startPointer;
|
||||
}
|
||||
assert firstStartPointer > 0 && startPointer >= firstStartPointer;
|
||||
|
||||
docBaseDeltas[blockChunks] = numDocs;
|
||||
startPointerDeltas[blockChunks] = startPointer - maxStartPointer;
|
||||
|
||||
++blockChunks;
|
||||
blockDocs += numDocs;
|
||||
totalDocs += numDocs;
|
||||
maxStartPointer = startPointer;
|
||||
}
|
||||
|
||||
public void finish(int numDocs, long maxPointer) throws IOException {
|
||||
if (numDocs != totalDocs) {
|
||||
throw new IllegalStateException("Expected " + numDocs + " docs, but got " + totalDocs);
|
||||
}
|
||||
if (blockChunks > 0) {
|
||||
writeBlock();
|
||||
}
|
||||
fieldsIndexOut.writeVInt(0); // end marker
|
||||
fieldsIndexOut.writeVLong(maxPointer);
|
||||
CodecUtil.writeFooter(fieldsIndexOut);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
fieldsIndexOut.close();
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,313 @@
|
|||
package org.apache.lucene.codecs.lucene41;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.BYTE_ARR;
|
||||
import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.CODEC_SFX_DAT;
|
||||
import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.CODEC_SFX_IDX;
|
||||
import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.FIELDS_EXTENSION;
|
||||
import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.FIELDS_INDEX_EXTENSION;
|
||||
import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.NUMERIC_DOUBLE;
|
||||
import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.NUMERIC_FLOAT;
|
||||
import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.NUMERIC_INT;
|
||||
import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.NUMERIC_LONG;
|
||||
import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.STRING;
|
||||
import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.TYPE_BITS;
|
||||
import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.VERSION_CURRENT;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.codecs.compressing.CompressionMode;
|
||||
import org.apache.lucene.codecs.compressing.Compressor;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.StorableField;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.GrowableByteArrayDataOutput;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
* writer for Lucene 4.1 stored fields for testing
|
||||
*/
|
||||
final class Lucene41StoredFieldsWriter extends StoredFieldsWriter {
|
||||
|
||||
// hard limit on the maximum number of documents per chunk
|
||||
static final int MAX_DOCUMENTS_PER_CHUNK = 128;
|
||||
|
||||
private final Directory directory;
|
||||
private final String segment;
|
||||
private final String segmentSuffix;
|
||||
private Lucene41StoredFieldsIndexWriter indexWriter;
|
||||
private IndexOutput fieldsStream;
|
||||
|
||||
private final Compressor compressor;
|
||||
private final int chunkSize;
|
||||
|
||||
private final GrowableByteArrayDataOutput bufferedDocs;
|
||||
private int[] numStoredFields; // number of stored fields
|
||||
private int[] endOffsets; // end offsets in bufferedDocs
|
||||
private int docBase; // doc ID at the beginning of the chunk
|
||||
private int numBufferedDocs; // docBase + numBufferedDocs == current doc ID
|
||||
|
||||
/** Sole constructor. */
|
||||
public Lucene41StoredFieldsWriter(Directory directory, SegmentInfo si, String segmentSuffix, IOContext context,
|
||||
String formatName, CompressionMode compressionMode, int chunkSize) throws IOException {
|
||||
assert directory != null;
|
||||
this.directory = directory;
|
||||
this.segment = si.name;
|
||||
this.segmentSuffix = segmentSuffix;
|
||||
this.compressor = compressionMode.newCompressor();
|
||||
this.chunkSize = chunkSize;
|
||||
this.docBase = 0;
|
||||
this.bufferedDocs = new GrowableByteArrayDataOutput(chunkSize);
|
||||
this.numStoredFields = new int[16];
|
||||
this.endOffsets = new int[16];
|
||||
this.numBufferedDocs = 0;
|
||||
|
||||
boolean success = false;
|
||||
IndexOutput indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_INDEX_EXTENSION),
|
||||
context);
|
||||
try {
|
||||
fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_EXTENSION),
|
||||
context);
|
||||
|
||||
final String codecNameIdx = formatName + CODEC_SFX_IDX;
|
||||
final String codecNameDat = formatName + CODEC_SFX_DAT;
|
||||
CodecUtil.writeHeader(indexStream, codecNameIdx, VERSION_CURRENT);
|
||||
CodecUtil.writeHeader(fieldsStream, codecNameDat, VERSION_CURRENT);
|
||||
assert CodecUtil.headerLength(codecNameDat) == fieldsStream.getFilePointer();
|
||||
assert CodecUtil.headerLength(codecNameIdx) == indexStream.getFilePointer();
|
||||
|
||||
indexWriter = new Lucene41StoredFieldsIndexWriter(indexStream);
|
||||
indexStream = null;
|
||||
|
||||
fieldsStream.writeVInt(chunkSize);
|
||||
fieldsStream.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(indexStream);
|
||||
abort();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
IOUtils.close(fieldsStream, indexWriter);
|
||||
} finally {
|
||||
fieldsStream = null;
|
||||
indexWriter = null;
|
||||
}
|
||||
}
|
||||
|
||||
private int numStoredFieldsInDoc;
|
||||
|
||||
@Override
|
||||
public void startDocument() throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finishDocument() throws IOException {
|
||||
if (numBufferedDocs == this.numStoredFields.length) {
|
||||
final int newLength = ArrayUtil.oversize(numBufferedDocs + 1, 4);
|
||||
this.numStoredFields = Arrays.copyOf(this.numStoredFields, newLength);
|
||||
endOffsets = Arrays.copyOf(endOffsets, newLength);
|
||||
}
|
||||
this.numStoredFields[numBufferedDocs] = numStoredFieldsInDoc;
|
||||
numStoredFieldsInDoc = 0;
|
||||
endOffsets[numBufferedDocs] = bufferedDocs.length;
|
||||
++numBufferedDocs;
|
||||
if (triggerFlush()) {
|
||||
flush();
|
||||
}
|
||||
}
|
||||
|
||||
private static void saveInts(int[] values, int length, DataOutput out) throws IOException {
|
||||
assert length > 0;
|
||||
if (length == 1) {
|
||||
out.writeVInt(values[0]);
|
||||
} else {
|
||||
boolean allEqual = true;
|
||||
for (int i = 1; i < length; ++i) {
|
||||
if (values[i] != values[0]) {
|
||||
allEqual = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (allEqual) {
|
||||
out.writeVInt(0);
|
||||
out.writeVInt(values[0]);
|
||||
} else {
|
||||
long max = 0;
|
||||
for (int i = 0; i < length; ++i) {
|
||||
max |= values[i];
|
||||
}
|
||||
final int bitsRequired = PackedInts.bitsRequired(max);
|
||||
out.writeVInt(bitsRequired);
|
||||
final PackedInts.Writer w = PackedInts.getWriterNoHeader(out, PackedInts.Format.PACKED, length, bitsRequired, 1);
|
||||
for (int i = 0; i < length; ++i) {
|
||||
w.add(values[i]);
|
||||
}
|
||||
w.finish();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void writeHeader(int docBase, int numBufferedDocs, int[] numStoredFields, int[] lengths) throws IOException {
|
||||
// save docBase and numBufferedDocs
|
||||
fieldsStream.writeVInt(docBase);
|
||||
fieldsStream.writeVInt(numBufferedDocs);
|
||||
|
||||
// save numStoredFields
|
||||
saveInts(numStoredFields, numBufferedDocs, fieldsStream);
|
||||
|
||||
// save lengths
|
||||
saveInts(lengths, numBufferedDocs, fieldsStream);
|
||||
}
|
||||
|
||||
private boolean triggerFlush() {
|
||||
return bufferedDocs.length >= chunkSize || // chunks of at least chunkSize bytes
|
||||
numBufferedDocs >= MAX_DOCUMENTS_PER_CHUNK;
|
||||
}
|
||||
|
||||
private void flush() throws IOException {
|
||||
indexWriter.writeIndex(numBufferedDocs, fieldsStream.getFilePointer());
|
||||
|
||||
// transform end offsets into lengths
|
||||
final int[] lengths = endOffsets;
|
||||
for (int i = numBufferedDocs - 1; i > 0; --i) {
|
||||
lengths[i] = endOffsets[i] - endOffsets[i - 1];
|
||||
assert lengths[i] >= 0;
|
||||
}
|
||||
writeHeader(docBase, numBufferedDocs, numStoredFields, lengths);
|
||||
|
||||
// compress stored fields to fieldsStream
|
||||
if (bufferedDocs.length >= 2 * chunkSize) {
|
||||
// big chunk, slice it
|
||||
for (int compressed = 0; compressed < bufferedDocs.length; compressed += chunkSize) {
|
||||
compressor.compress(bufferedDocs.bytes, compressed, Math.min(chunkSize, bufferedDocs.length - compressed), fieldsStream);
|
||||
}
|
||||
} else {
|
||||
compressor.compress(bufferedDocs.bytes, 0, bufferedDocs.length, fieldsStream);
|
||||
}
|
||||
|
||||
// reset
|
||||
docBase += numBufferedDocs;
|
||||
numBufferedDocs = 0;
|
||||
bufferedDocs.length = 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeField(FieldInfo info, StorableField field)
|
||||
throws IOException {
|
||||
|
||||
++numStoredFieldsInDoc;
|
||||
|
||||
int bits = 0;
|
||||
final BytesRef bytes;
|
||||
final String string;
|
||||
|
||||
Number number = field.numericValue();
|
||||
if (number != null) {
|
||||
if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
|
||||
bits = NUMERIC_INT;
|
||||
} else if (number instanceof Long) {
|
||||
bits = NUMERIC_LONG;
|
||||
} else if (number instanceof Float) {
|
||||
bits = NUMERIC_FLOAT;
|
||||
} else if (number instanceof Double) {
|
||||
bits = NUMERIC_DOUBLE;
|
||||
} else {
|
||||
throw new IllegalArgumentException("cannot store numeric type " + number.getClass());
|
||||
}
|
||||
string = null;
|
||||
bytes = null;
|
||||
} else {
|
||||
bytes = field.binaryValue();
|
||||
if (bytes != null) {
|
||||
bits = BYTE_ARR;
|
||||
string = null;
|
||||
} else {
|
||||
bits = STRING;
|
||||
string = field.stringValue();
|
||||
if (string == null) {
|
||||
throw new IllegalArgumentException("field " + field.name() + " is stored but does not have binaryValue, stringValue nor numericValue");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
final long infoAndBits = (((long) info.number) << TYPE_BITS) | bits;
|
||||
bufferedDocs.writeVLong(infoAndBits);
|
||||
|
||||
if (bytes != null) {
|
||||
bufferedDocs.writeVInt(bytes.length);
|
||||
bufferedDocs.writeBytes(bytes.bytes, bytes.offset, bytes.length);
|
||||
} else if (string != null) {
|
||||
bufferedDocs.writeString(field.stringValue());
|
||||
} else {
|
||||
if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
|
||||
bufferedDocs.writeInt(number.intValue());
|
||||
} else if (number instanceof Long) {
|
||||
bufferedDocs.writeLong(number.longValue());
|
||||
} else if (number instanceof Float) {
|
||||
bufferedDocs.writeInt(Float.floatToIntBits(number.floatValue()));
|
||||
} else if (number instanceof Double) {
|
||||
bufferedDocs.writeLong(Double.doubleToLongBits(number.doubleValue()));
|
||||
} else {
|
||||
throw new AssertionError("Cannot get here");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
IOUtils.closeWhileHandlingException(this);
|
||||
IOUtils.deleteFilesIgnoringExceptions(directory,
|
||||
IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_EXTENSION),
|
||||
IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_INDEX_EXTENSION));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(FieldInfos fis, int numDocs) throws IOException {
|
||||
if (numBufferedDocs > 0) {
|
||||
flush();
|
||||
} else {
|
||||
assert bufferedDocs.length == 0;
|
||||
}
|
||||
if (docBase != numDocs) {
|
||||
throw new RuntimeException("Wrote " + docBase + " docs, finish called with numDocs=" + numDocs);
|
||||
}
|
||||
indexWriter.finish(numDocs, fieldsStream.getFilePointer());
|
||||
CodecUtil.writeFooter(fieldsStream);
|
||||
assert bufferedDocs.length == 0;
|
||||
}
|
||||
}
|
|
@ -19,11 +19,10 @@ package org.apache.lucene.codecs.lucene41;
|
|||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
public class TestLucene41StoredFieldsFormat extends BaseStoredFieldsFormatTestCase {
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
return TestUtil.getDefaultCodec();
|
||||
return new Lucene41RWCodec();
|
||||
}
|
||||
}
|
|
@ -24,7 +24,10 @@ import org.apache.lucene.codecs.FieldInfosFormat;
|
|||
import org.apache.lucene.codecs.FieldInfosWriter;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40RWSegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
/**
|
||||
|
@ -35,6 +38,7 @@ public class Lucene42RWCodec extends Lucene42Codec {
|
|||
|
||||
private static final DocValuesFormat dv = new Lucene42RWDocValuesFormat();
|
||||
private static final NormsFormat norms = new Lucene42RWNormsFormat();
|
||||
private static final StoredFieldsFormat storedFields = new Lucene41RWStoredFieldsFormat();
|
||||
|
||||
private final FieldInfosFormat fieldInfosFormat = new Lucene42FieldInfosFormat() {
|
||||
@Override
|
||||
|
@ -64,4 +68,16 @@ public class Lucene42RWCodec extends Lucene42Codec {
|
|||
public SegmentInfoFormat segmentInfoFormat() {
|
||||
return segmentInfos;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StoredFieldsFormat storedFieldsFormat() {
|
||||
return storedFields;
|
||||
}
|
||||
|
||||
private final TermVectorsFormat vectorsFormat = new Lucene42RWTermVectorsFormat();
|
||||
|
||||
@Override
|
||||
public TermVectorsFormat termVectorsFormat() {
|
||||
return vectorsFormat;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,34 @@
|
|||
package org.apache.lucene.codecs.lucene42;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.TermVectorsWriter;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
||||
/** read-write version of lucene 4.2 term vectors for testing */
|
||||
public class Lucene42RWTermVectorsFormat extends Lucene42TermVectorsFormat {
|
||||
|
||||
@Override
|
||||
public TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo segmentInfo, IOContext context) throws IOException {
|
||||
return new Lucene42TermVectorsWriter(directory, segmentInfo, SEGMENT_SUFFIX, context, FORMAT_NAME, COMPRESSION_MODE, CHUNK_SIZE);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,712 @@
|
|||
package org.apache.lucene.codecs.lucene42;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.BLOCK_SIZE;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.CODEC_SFX_DAT;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.CODEC_SFX_IDX;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.FLAGS_BITS;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.OFFSETS;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.PAYLOADS;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.POSITIONS;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.VECTORS_EXTENSION;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.VECTORS_INDEX_EXTENSION;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.VERSION_CURRENT;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.Arrays;
|
||||
import java.util.Deque;
|
||||
import java.util.Iterator;
|
||||
import java.util.SortedSet;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
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.lucene41.Lucene41StoredFieldsIndexWriter;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.DataInput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.GrowableByteArrayDataOutput;
|
||||
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;
|
||||
|
||||
/**
|
||||
* writer for Lucene 4.2 term vectors for testing
|
||||
*/
|
||||
final class Lucene42TermVectorsWriter extends TermVectorsWriter {
|
||||
|
||||
// hard limit on the maximum number of documents per chunk
|
||||
static final int MAX_DOCUMENTS_PER_CHUNK = 128;
|
||||
|
||||
private final Directory directory;
|
||||
private final String segment;
|
||||
private final String segmentSuffix;
|
||||
private Lucene41StoredFieldsIndexWriter indexWriter;
|
||||
private IndexOutput vectorsStream;
|
||||
|
||||
private final Compressor compressor;
|
||||
private final int chunkSize;
|
||||
|
||||
/** 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 = Arrays.copyOf(startOffsetsBuf, newLength);
|
||||
lengthsBuf = Arrays.copyOf(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 GrowableByteArrayDataOutput termSuffixes; // buffered term suffixes
|
||||
private final GrowableByteArrayDataOutput payloadBytes; // buffered term payloads
|
||||
private final BlockPackedWriter writer;
|
||||
|
||||
/** Sole constructor. */
|
||||
public Lucene42TermVectorsWriter(Directory directory, SegmentInfo si, String segmentSuffix, IOContext context,
|
||||
String formatName, CompressionMode compressionMode, int chunkSize) throws IOException {
|
||||
assert directory != null;
|
||||
this.directory = directory;
|
||||
this.segment = si.name;
|
||||
this.segmentSuffix = segmentSuffix;
|
||||
this.compressor = compressionMode.newCompressor();
|
||||
this.chunkSize = chunkSize;
|
||||
|
||||
numDocs = 0;
|
||||
pendingDocs = new ArrayDeque<>();
|
||||
termSuffixes = new GrowableByteArrayDataOutput(ArrayUtil.oversize(chunkSize, 1));
|
||||
payloadBytes = new GrowableByteArrayDataOutput(ArrayUtil.oversize(1, 1));
|
||||
lastTerm = new BytesRef(ArrayUtil.oversize(30, 1));
|
||||
|
||||
boolean success = false;
|
||||
IndexOutput indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_INDEX_EXTENSION),
|
||||
context);
|
||||
try {
|
||||
vectorsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_EXTENSION),
|
||||
context);
|
||||
|
||||
final String codecNameIdx = formatName + CODEC_SFX_IDX;
|
||||
final String codecNameDat = formatName + CODEC_SFX_DAT;
|
||||
CodecUtil.writeHeader(indexStream, codecNameIdx, VERSION_CURRENT);
|
||||
CodecUtil.writeHeader(vectorsStream, codecNameDat, VERSION_CURRENT);
|
||||
assert CodecUtil.headerLength(codecNameDat) == vectorsStream.getFilePointer();
|
||||
assert CodecUtil.headerLength(codecNameIdx) == indexStream.getFilePointer();
|
||||
|
||||
indexWriter = new Lucene41StoredFieldsIndexWriter(indexStream);
|
||||
indexStream = null;
|
||||
|
||||
vectorsStream.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
vectorsStream.writeVInt(chunkSize);
|
||||
writer = new BlockPackedWriter(vectorsStream, 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(indexStream);
|
||||
abort();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
IOUtils.close(vectorsStream, indexWriter);
|
||||
} finally {
|
||||
vectorsStream = null;
|
||||
indexWriter = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
IOUtils.closeWhileHandlingException(this);
|
||||
IOUtils.deleteFilesIgnoringExceptions(directory,
|
||||
IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_EXTENSION),
|
||||
IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_INDEX_EXTENSION));
|
||||
}
|
||||
|
||||
@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
|
||||
termSuffixes.writeBytes(payloadBytes.bytes, payloadBytes.length);
|
||||
payloadBytes.length = 0;
|
||||
++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 = 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.length >= 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
|
||||
compressor.compress(termSuffixes.bytes, 0, termSuffixes.length, vectorsStream);
|
||||
}
|
||||
|
||||
// reset
|
||||
pendingDocs.clear();
|
||||
curDoc = null;
|
||||
curField = null;
|
||||
termSuffixes.length = 0;
|
||||
}
|
||||
|
||||
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) {
|
||||
int previousPos = 0;
|
||||
int previousOff = 0;
|
||||
for (int j = 0; j < fd.freqs[i]; ++j) {
|
||||
final int position = positionsBuf[fd.posStart + pos];
|
||||
final int startOffset = startOffsetsBuf[fd.offStart + pos];
|
||||
sumPos[fieldNumOff] += position - previousPos;
|
||||
sumOffsets[fieldNumOff] += startOffset - previousOff;
|
||||
previousPos = position;
|
||||
previousOff = startOffset;
|
||||
++pos;
|
||||
}
|
||||
}
|
||||
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()) {
|
||||
flush();
|
||||
}
|
||||
if (numDocs != this.numDocs) {
|
||||
throw new RuntimeException("Wrote " + this.numDocs + " docs, finish called with numDocs=" + numDocs);
|
||||
}
|
||||
indexWriter.finish(numDocs, vectorsStream.getFilePointer());
|
||||
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 = Arrays.copyOf(startOffsetsBuf, newLength);
|
||||
lengthsBuf = Arrays.copyOf(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;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,28 @@
|
|||
package org.apache.lucene.codecs.lucene42;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.index.BaseTermVectorsFormatTestCase;
|
||||
|
||||
public class TestLucene42TermVectorsFormat extends BaseTermVectorsFormatTestCase {
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
return new Lucene42RWCodec();
|
||||
}
|
||||
}
|
|
@ -24,10 +24,14 @@ import org.apache.lucene.codecs.FieldInfosFormat;
|
|||
import org.apache.lucene.codecs.FieldInfosWriter;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40RWSegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42FieldInfosWriter;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42RWNormsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
|
||||
|
||||
/**
|
||||
* Read-write version of {@link Lucene45Codec} for testing.
|
||||
|
@ -67,4 +71,18 @@ public class Lucene45RWCodec extends Lucene45Codec {
|
|||
public SegmentInfoFormat segmentInfoFormat() {
|
||||
return segmentInfos;
|
||||
}
|
||||
|
||||
private static final StoredFieldsFormat storedFields = new Lucene41RWStoredFieldsFormat();
|
||||
|
||||
@Override
|
||||
public StoredFieldsFormat storedFieldsFormat() {
|
||||
return storedFields;
|
||||
}
|
||||
|
||||
private final TermVectorsFormat vectorsFormat = new Lucene42RWTermVectorsFormat();
|
||||
|
||||
@Override
|
||||
public TermVectorsFormat termVectorsFormat() {
|
||||
return vectorsFormat;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,7 +21,11 @@ import org.apache.lucene.codecs.DocValuesFormat;
|
|||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoWriter;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42RWNormsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45RWDocValuesFormat;
|
||||
|
||||
/**
|
||||
|
@ -50,4 +54,18 @@ public class Lucene46RWCodec extends Lucene46Codec {
|
|||
public SegmentInfoFormat segmentInfoFormat() {
|
||||
return segmentInfos;
|
||||
}
|
||||
|
||||
private static final StoredFieldsFormat storedFields = new Lucene41RWStoredFieldsFormat();
|
||||
|
||||
@Override
|
||||
public StoredFieldsFormat storedFieldsFormat() {
|
||||
return storedFields;
|
||||
}
|
||||
|
||||
private final TermVectorsFormat vectorsFormat = new Lucene42RWTermVectorsFormat();
|
||||
|
||||
@Override
|
||||
public TermVectorsFormat termVectorsFormat() {
|
||||
return vectorsFormat;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,10 @@ package org.apache.lucene.codecs.lucene49;
|
|||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene46.Lucene46RWSegmentInfoFormat;
|
||||
|
||||
/**
|
||||
|
@ -48,4 +52,18 @@ public class Lucene49RWCodec extends Lucene49Codec {
|
|||
public SegmentInfoFormat segmentInfoFormat() {
|
||||
return segmentInfos;
|
||||
}
|
||||
|
||||
private static final StoredFieldsFormat storedFields = new Lucene41RWStoredFieldsFormat();
|
||||
|
||||
@Override
|
||||
public StoredFieldsFormat storedFieldsFormat() {
|
||||
return storedFields;
|
||||
}
|
||||
|
||||
private final TermVectorsFormat vectorsFormat = new Lucene42RWTermVectorsFormat();
|
||||
|
||||
@Override
|
||||
public TermVectorsFormat termVectorsFormat() {
|
||||
return vectorsFormat;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,25 +0,0 @@
|
|||
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<html>
|
||||
<head>
|
||||
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
|
||||
</head>
|
||||
<body>
|
||||
Lucene 4.2 file format.
|
||||
</body>
|
||||
</html>
|
|
@ -27,8 +27,6 @@ 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.lucene41.Lucene41StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49NormsFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
|
@ -44,8 +42,8 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public class Lucene50Codec extends Codec {
|
||||
private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
|
||||
private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
|
||||
private final StoredFieldsFormat fieldsFormat = new Lucene50StoredFieldsFormat();
|
||||
private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
|
||||
private final FieldInfosFormat fieldInfosFormat = new Lucene50FieldInfosFormat();
|
||||
private final SegmentInfoFormat segmentInfosFormat = new Lucene50SegmentInfoFormat();
|
||||
private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.codecs.lucene41;
|
||||
package org.apache.lucene.codecs.lucene50;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -27,7 +27,7 @@ import org.apache.lucene.store.DataOutput;
|
|||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
* Lucene 4.1 stored fields format.
|
||||
* Lucene 5.0 stored fields format.
|
||||
*
|
||||
* <p><b>Principle</b></p>
|
||||
* <p>This {@link StoredFieldsFormat} compresses blocks of 16KB of documents in
|
||||
|
@ -114,11 +114,11 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
* larger than (<tt>2<sup>31</sup> - 2<sup>14</sup></tt>) bytes.</p>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class Lucene41StoredFieldsFormat extends CompressingStoredFieldsFormat {
|
||||
public final class Lucene50StoredFieldsFormat extends CompressingStoredFieldsFormat {
|
||||
|
||||
/** Sole constructor. */
|
||||
public Lucene41StoredFieldsFormat() {
|
||||
super("Lucene41StoredFields", CompressionMode.FAST, 1 << 14);
|
||||
public Lucene50StoredFieldsFormat() {
|
||||
super("Lucene50StoredFields", CompressionMode.FAST, 1 << 14);
|
||||
}
|
||||
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.codecs.lucene42;
|
||||
package org.apache.lucene.codecs.lucene50;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -22,15 +22,14 @@ import org.apache.lucene.codecs.TermVectorsFormat;
|
|||
import org.apache.lucene.codecs.compressing.CompressingStoredFieldsIndexWriter;
|
||||
import org.apache.lucene.codecs.compressing.CompressingTermVectorsFormat;
|
||||
import org.apache.lucene.codecs.compressing.CompressionMode;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.util.packed.BlockPackedWriter;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
* Lucene 4.2 {@link TermVectorsFormat term vectors format}.
|
||||
* Lucene 5.0 {@link TermVectorsFormat term vectors format}.
|
||||
* <p>
|
||||
* Very similarly to {@link Lucene41StoredFieldsFormat}, this format is based
|
||||
* Very similarly to {@link Lucene50StoredFieldsFormat}, 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>
|
||||
|
@ -122,11 +121,11 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
* </ol>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class Lucene42TermVectorsFormat extends CompressingTermVectorsFormat {
|
||||
public final class Lucene50TermVectorsFormat extends CompressingTermVectorsFormat {
|
||||
|
||||
/** Sole constructor. */
|
||||
public Lucene42TermVectorsFormat() {
|
||||
super("Lucene41StoredFields", "", CompressionMode.FAST, 1 << 12);
|
||||
public Lucene50TermVectorsFormat() {
|
||||
super("Lucene50TermVectors", "", CompressionMode.FAST, 1 << 12);
|
||||
}
|
||||
|
||||
}
|
|
@ -147,7 +147,7 @@ its numbering.</p>
|
|||
This contains the set of field names used in the index.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat Stored Field values}.
|
||||
{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Stored Field values}.
|
||||
This contains, for each document, a list of attribute-value pairs, where the attributes
|
||||
are field names. These are used to store auxiliary information about the document, such as
|
||||
its title, url, or an identifier to access a database. The set of stored fields are what is
|
||||
|
@ -178,7 +178,7 @@ For each field in each document, a value is stored
|
|||
that is multiplied into the score for hits on that field.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vectors}.
|
||||
{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vectors}.
|
||||
For each field in each document, the term vector (sometimes
|
||||
called document vector) may be stored. A term vector consists of term text and
|
||||
term frequency. To add Term Vectors to your index see the
|
||||
|
@ -254,12 +254,12 @@ systems that frequently run out of file handles.</td>
|
|||
<td>Stores information about the fields</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat Field Index}</td>
|
||||
<td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Index}</td>
|
||||
<td>.fdx</td>
|
||||
<td>Contains pointers to field data</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat Field Data}</td>
|
||||
<td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Data}</td>
|
||||
<td>.fdt</td>
|
||||
<td>The stored fields for documents</td>
|
||||
</tr>
|
||||
|
@ -299,17 +299,17 @@ systems that frequently run out of file handles.</td>
|
|||
<td>Encodes additional scoring factors or other per-document information.</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Index}</td>
|
||||
<td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Index}</td>
|
||||
<td>.tvx</td>
|
||||
<td>Stores offset into the document data file</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Documents}</td>
|
||||
<td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Documents}</td>
|
||||
<td>.tvd</td>
|
||||
<td>Contains information about each document that has term vectors</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Fields}</td>
|
||||
<td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Fields}</td>
|
||||
<td>.tvf</td>
|
||||
<td>The field level info about term vectors</td>
|
||||
</tr>
|
||||
|
|
|
@ -1,63 +0,0 @@
|
|||
package org.apache.lucene.codecs.lucene41;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.blocktree.FieldReader;
|
||||
import org.apache.lucene.codecs.blocktree.Stats;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.BasePostingsFormatTestCase;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
public class TestLucene41PostingsFormat extends BasePostingsFormatTestCase {
|
||||
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
return TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat());
|
||||
}
|
||||
|
||||
/** Make sure the final sub-block(s) are not skipped. */
|
||||
public void testFinalBlock() throws Exception {
|
||||
Directory d = newDirectory();
|
||||
IndexWriter w = new IndexWriter(d, new IndexWriterConfig(new MockAnalyzer(random())));
|
||||
for(int i=0;i<25;i++) {
|
||||
Document doc = new Document();
|
||||
doc.add(newStringField("field", Character.toString((char) (97+i)), Field.Store.NO));
|
||||
doc.add(newStringField("field", "z" + Character.toString((char) (97+i)), Field.Store.NO));
|
||||
w.addDocument(doc);
|
||||
}
|
||||
w.forceMerge(1);
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w, true);
|
||||
assertEquals(1, r.leaves().size());
|
||||
FieldReader field = (FieldReader) r.leaves().get(0).reader().fields().terms("field");
|
||||
// We should see exactly two blocks: one root block (prefix empty string) and one block for z* terms (prefix z):
|
||||
Stats stats = field.computeStats();
|
||||
assertEquals(0, stats.floorBlockCount);
|
||||
assertEquals(2, stats.nonFloorBlockCount);
|
||||
r.close();
|
||||
w.close();
|
||||
d.close();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,29 @@
|
|||
package org.apache.lucene.codecs.lucene50;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
public class TestLucene50StoredFieldsFormat extends BaseStoredFieldsFormatTestCase {
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
return TestUtil.getDefaultCodec();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,29 @@
|
|||
package org.apache.lucene.codecs.lucene50;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
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();
|
||||
}
|
||||
}
|
|
@ -107,8 +107,7 @@ public class TestAllFilesHaveCodecHeader extends LuceneTestCase {
|
|||
}
|
||||
String previous = namesToExtensions.put(codecName, extension);
|
||||
if (previous != null && !previous.equals(extension)) {
|
||||
//TODO: not yet
|
||||
// fail("extensions " + previous + " and " + extension + " share same codecName " + codecName);
|
||||
fail("extensions " + previous + " and " + extension + " share same codecName " + codecName);
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
|
|
Loading…
Reference in New Issue