mirror of https://github.com/apache/lucene.git
bump 4.5 codec
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5178@1514897 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
7a4f7c669b
commit
2c6bf04190
|
@ -22,8 +22,11 @@ import java.io.IOException;
|
|||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* DocValues format that keeps most things on disk.
|
||||
|
@ -40,7 +43,12 @@ public final class DiskDocValuesFormat extends DocValuesFormat {
|
|||
|
||||
@Override
|
||||
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
return new DiskDocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
|
||||
return new Lucene45DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION) {
|
||||
@Override
|
||||
protected void addTermsDict(FieldInfo field, Iterable<BytesRef> values) throws IOException {
|
||||
addBinaryField(field, values);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -52,11 +60,4 @@ public final class DiskDocValuesFormat extends DocValuesFormat {
|
|||
public static final String DATA_EXTENSION = "dvdd";
|
||||
public static final String META_CODEC = "DiskDocValuesMetadata";
|
||||
public static final String META_EXTENSION = "dvdm";
|
||||
public static final int VERSION_START = 0;
|
||||
public static final int VERSION_COMPRESSED_TERMS = 1;
|
||||
public static final int VERSION_CURRENT = VERSION_COMPRESSED_TERMS;
|
||||
public static final byte NUMERIC = 0;
|
||||
public static final byte BINARY = 1;
|
||||
public static final byte SORTED = 2;
|
||||
public static final byte SORTED_SET = 3;
|
||||
}
|
||||
|
|
|
@ -17,35 +17,26 @@ package org.apache.lucene.codecs.diskdv;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.DELTA_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.GCD_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.TABLE_COMPRESSED;
|
||||
|
||||
import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.BINARY_FIXED_UNCOMPRESSED;
|
||||
import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED;
|
||||
import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.BINARY_PREFIX_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.DELTA_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.GCD_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.TABLE_COMPRESSED;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.DocValuesProducer.SortedSetDocsWithField;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.TermsEnum.SeekStatus;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -56,32 +47,28 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
|
||||
class DiskDocValuesProducer extends DocValuesProducer {
|
||||
private final Map<Integer,NumericEntry> numerics;
|
||||
private final Map<Integer,BinaryEntry> binaries;
|
||||
private final Map<Integer,NumericEntry> ords;
|
||||
private final Map<Integer,NumericEntry> ordIndexes;
|
||||
private final Map<Integer,BinaryEntry> binaries;
|
||||
private final IndexInput data;
|
||||
private final int maxDoc;
|
||||
|
||||
// memory-resident structures
|
||||
private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
|
||||
private final Map<Integer,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
|
||||
|
||||
DiskDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
this.maxDoc = state.segmentInfo.getDocCount();
|
||||
// read in the entries from the metadata file.
|
||||
IndexInput in = state.directory.openInput(metaName, state.context);
|
||||
this.maxDoc = state.segmentInfo.getDocCount();
|
||||
boolean success = false;
|
||||
final int version;
|
||||
try {
|
||||
version = CodecUtil.checkHeader(in, metaCodec,
|
||||
DiskDocValuesFormat.VERSION_CURRENT,
|
||||
DiskDocValuesFormat.VERSION_CURRENT);
|
||||
Lucene45DocValuesFormat.VERSION_CURRENT,
|
||||
Lucene45DocValuesFormat.VERSION_CURRENT);
|
||||
numerics = new HashMap<Integer,NumericEntry>();
|
||||
ords = new HashMap<Integer,NumericEntry>();
|
||||
ordIndexes = new HashMap<Integer,NumericEntry>();
|
||||
binaries = new HashMap<Integer,BinaryEntry>();
|
||||
readFields(in, state.fieldInfos);
|
||||
readFields(in);
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
|
@ -97,10 +84,10 @@ class DiskDocValuesProducer extends DocValuesProducer {
|
|||
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
|
||||
data = state.directory.openInput(dataName, state.context);
|
||||
final int version2 = CodecUtil.checkHeader(data, dataCodec,
|
||||
DiskDocValuesFormat.VERSION_CURRENT,
|
||||
DiskDocValuesFormat.VERSION_CURRENT);
|
||||
Lucene45DocValuesFormat.VERSION_CURRENT,
|
||||
Lucene45DocValuesFormat.VERSION_CURRENT);
|
||||
if (version != version2) {
|
||||
throw new CorruptIndexException("Format versions mismatch");
|
||||
throw new CorruptIndexException("Versions mismatch");
|
||||
}
|
||||
|
||||
success = true;
|
||||
|
@ -109,61 +96,62 @@ class DiskDocValuesProducer extends DocValuesProducer {
|
|||
IOUtils.closeWhileHandlingException(this.data);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
|
||||
private void readFields(IndexInput meta) throws IOException {
|
||||
int fieldNumber = meta.readVInt();
|
||||
while (fieldNumber != -1) {
|
||||
byte type = meta.readByte();
|
||||
if (type == DiskDocValuesFormat.NUMERIC) {
|
||||
if (type == Lucene45DocValuesFormat.NUMERIC) {
|
||||
numerics.put(fieldNumber, readNumericEntry(meta));
|
||||
} else if (type == DiskDocValuesFormat.BINARY) {
|
||||
} else if (type == Lucene45DocValuesFormat.BINARY) {
|
||||
BinaryEntry b = readBinaryEntry(meta);
|
||||
binaries.put(fieldNumber, b);
|
||||
} else if (type == DiskDocValuesFormat.SORTED) {
|
||||
} else if (type == Lucene45DocValuesFormat.SORTED) {
|
||||
// sorted = binary + numeric
|
||||
if (meta.readVInt() != fieldNumber) {
|
||||
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
if (meta.readByte() != DiskDocValuesFormat.BINARY) {
|
||||
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
if (meta.readByte() != Lucene45DocValuesFormat.BINARY) {
|
||||
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
BinaryEntry b = readBinaryEntry(meta);
|
||||
binaries.put(fieldNumber, b);
|
||||
|
||||
if (meta.readVInt() != fieldNumber) {
|
||||
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
|
||||
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
|
||||
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
NumericEntry n = readNumericEntry(meta);
|
||||
ords.put(fieldNumber, n);
|
||||
} else if (type == DiskDocValuesFormat.SORTED_SET) {
|
||||
} else if (type == Lucene45DocValuesFormat.SORTED_SET) {
|
||||
// sortedset = binary + numeric + ordIndex
|
||||
if (meta.readVInt() != fieldNumber) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
if (meta.readByte() != DiskDocValuesFormat.BINARY) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
if (meta.readByte() != Lucene45DocValuesFormat.BINARY) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
BinaryEntry b = readBinaryEntry(meta);
|
||||
binaries.put(fieldNumber, b);
|
||||
|
||||
if (meta.readVInt() != fieldNumber) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
NumericEntry n1 = readNumericEntry(meta);
|
||||
ords.put(fieldNumber, n1);
|
||||
|
||||
if (meta.readVInt() != fieldNumber) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
NumericEntry n2 = readNumericEntry(meta);
|
||||
ordIndexes.put(fieldNumber, n2);
|
||||
|
@ -209,27 +197,18 @@ class DiskDocValuesProducer extends DocValuesProducer {
|
|||
|
||||
static BinaryEntry readBinaryEntry(IndexInput meta) throws IOException {
|
||||
BinaryEntry entry = new BinaryEntry();
|
||||
entry.format = meta.readVInt();
|
||||
int format = meta.readVInt();
|
||||
if (format != Lucene45DocValuesConsumer.BINARY_FIXED_UNCOMPRESSED && format != Lucene45DocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED) {
|
||||
throw new CorruptIndexException("Unexpected format for binary entry: " + format + ", input=" + meta);
|
||||
}
|
||||
entry.minLength = meta.readVInt();
|
||||
entry.maxLength = meta.readVInt();
|
||||
entry.count = meta.readVLong();
|
||||
entry.offset = meta.readLong();
|
||||
switch(entry.format) {
|
||||
case BINARY_FIXED_UNCOMPRESSED:
|
||||
break;
|
||||
case BINARY_PREFIX_COMPRESSED:
|
||||
entry.addressInterval = meta.readVInt();
|
||||
if (entry.minLength != entry.maxLength) {
|
||||
entry.addressesOffset = meta.readLong();
|
||||
entry.packedIntsVersion = meta.readVInt();
|
||||
entry.blockSize = meta.readVInt();
|
||||
break;
|
||||
case BINARY_VARIABLE_UNCOMPRESSED:
|
||||
entry.addressesOffset = meta.readLong();
|
||||
entry.packedIntsVersion = meta.readVInt();
|
||||
entry.blockSize = meta.readVInt();
|
||||
break;
|
||||
default:
|
||||
throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
|
||||
}
|
||||
return entry;
|
||||
}
|
||||
|
@ -237,10 +216,10 @@ class DiskDocValuesProducer extends DocValuesProducer {
|
|||
@Override
|
||||
public NumericDocValues getNumeric(FieldInfo field) throws IOException {
|
||||
NumericEntry entry = numerics.get(field.number);
|
||||
return getNumeric(entry);
|
||||
return getNumeric(field, entry);
|
||||
}
|
||||
|
||||
LongNumericDocValues getNumeric(NumericEntry entry) throws IOException {
|
||||
private LongNumericDocValues getNumeric(FieldInfo field, final NumericEntry entry) throws IOException {
|
||||
final IndexInput data = this.data.clone();
|
||||
data.seek(entry.offset);
|
||||
|
||||
|
@ -264,12 +243,12 @@ class DiskDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
};
|
||||
case TABLE_COMPRESSED:
|
||||
final long table[] = entry.table;
|
||||
final long[] table = entry.table;
|
||||
final int bitsRequired = PackedInts.bitsRequired(table.length - 1);
|
||||
final PackedInts.Reader ords = PackedInts.getDirectReaderNoHeader(data, PackedInts.Format.PACKED, entry.packedIntsVersion, (int) entry.count, bitsRequired);
|
||||
return new LongNumericDocValues() {
|
||||
@Override
|
||||
public long get(long id) {
|
||||
long get(long id) {
|
||||
return table[(int) ords.get((int) id)];
|
||||
}
|
||||
};
|
||||
|
@ -281,15 +260,10 @@ class DiskDocValuesProducer extends DocValuesProducer {
|
|||
@Override
|
||||
public BinaryDocValues getBinary(FieldInfo field) throws IOException {
|
||||
BinaryEntry bytes = binaries.get(field.number);
|
||||
switch(bytes.format) {
|
||||
case BINARY_FIXED_UNCOMPRESSED:
|
||||
if (bytes.minLength == bytes.maxLength) {
|
||||
return getFixedBinary(field, bytes);
|
||||
case BINARY_VARIABLE_UNCOMPRESSED:
|
||||
} else {
|
||||
return getVariableBinary(field, bytes);
|
||||
case BINARY_PREFIX_COMPRESSED:
|
||||
return getCompressedBinary(field, bytes);
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -318,22 +292,13 @@ class DiskDocValuesProducer extends DocValuesProducer {
|
|||
|
||||
private BinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
|
||||
final IndexInput data = this.data.clone();
|
||||
|
||||
final MonotonicBlockPackedReader addresses;
|
||||
synchronized (addressInstances) {
|
||||
MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
|
||||
if (addrInstance == null) {
|
||||
data.seek(bytes.addressesOffset);
|
||||
addrInstance = new MonotonicBlockPackedReader(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count, false);
|
||||
addressInstances.put(field.number, addrInstance);
|
||||
}
|
||||
addresses = addrInstance;
|
||||
}
|
||||
|
||||
final MonotonicBlockPackedReader addresses = new MonotonicBlockPackedReader(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count, true);
|
||||
return new LongBinaryDocValues() {
|
||||
@Override
|
||||
public void get(long id, BytesRef result) {
|
||||
long startAddress = bytes.offset + (id == 0 ? 0 : addresses.get(id-1));
|
||||
long startAddress = bytes.offset + (id == 0 ? 0 : + addresses.get(id-1));
|
||||
long endAddress = bytes.offset + addresses.get(id);
|
||||
int length = (int) (endAddress - startAddress);
|
||||
try {
|
||||
|
@ -352,39 +317,11 @@ class DiskDocValuesProducer extends DocValuesProducer {
|
|||
};
|
||||
}
|
||||
|
||||
private BinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
|
||||
final IndexInput data = this.data.clone();
|
||||
final long interval = bytes.addressInterval;
|
||||
|
||||
final MonotonicBlockPackedReader addresses;
|
||||
synchronized (addressInstances) {
|
||||
MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
|
||||
if (addrInstance == null) {
|
||||
data.seek(bytes.addressesOffset);
|
||||
final long size;
|
||||
if (bytes.count % interval == 0) {
|
||||
size = bytes.count / interval;
|
||||
} else {
|
||||
size = 1L + bytes.count / interval;
|
||||
}
|
||||
addrInstance = new MonotonicBlockPackedReader(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
|
||||
addressInstances.put(field.number, addrInstance);
|
||||
}
|
||||
addresses = addrInstance;
|
||||
}
|
||||
|
||||
return new CompressedBinaryDocValues(bytes, addresses, data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedDocValues getSorted(FieldInfo field) throws IOException {
|
||||
final int valueCount = (int) binaries.get(field.number).count;
|
||||
final BinaryDocValues binary = getBinary(field);
|
||||
NumericEntry entry = ords.get(field.number);
|
||||
IndexInput data = this.data.clone();
|
||||
data.seek(entry.offset);
|
||||
final BlockPackedReader ordinals = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
|
||||
|
||||
final NumericDocValues ordinals = getNumeric(field, ords.get(field.number));
|
||||
return new SortedDocValues() {
|
||||
|
||||
@Override
|
||||
|
@ -401,46 +338,18 @@ class DiskDocValuesProducer extends DocValuesProducer {
|
|||
public int getValueCount() {
|
||||
return valueCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupTerm(BytesRef key) {
|
||||
if (binary instanceof CompressedBinaryDocValues) {
|
||||
return (int) ((CompressedBinaryDocValues)binary).lookupTerm(key);
|
||||
} else {
|
||||
return super.lookupTerm(key);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum termsEnum() {
|
||||
if (binary instanceof CompressedBinaryDocValues) {
|
||||
return ((CompressedBinaryDocValues)binary).getTermsEnum();
|
||||
} else {
|
||||
return super.termsEnum();
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
|
||||
final long valueCount = binaries.get(field.number).count;
|
||||
// we keep the byte[]s and list of ords on disk, these could be large
|
||||
final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
|
||||
final LongNumericDocValues ordinals = getNumeric(ords.get(field.number));
|
||||
// but the addresses to the ord stream are in RAM
|
||||
final MonotonicBlockPackedReader ordIndex;
|
||||
synchronized (ordIndexInstances) {
|
||||
MonotonicBlockPackedReader ordIndexInstance = ordIndexInstances.get(field.number);
|
||||
if (ordIndexInstance == null) {
|
||||
final LongNumericDocValues ordinals = getNumeric(field, ords.get(field.number));
|
||||
NumericEntry entry = ordIndexes.get(field.number);
|
||||
IndexInput data = this.data.clone();
|
||||
data.seek(entry.offset);
|
||||
ordIndexInstance = new MonotonicBlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, false);
|
||||
ordIndexInstances.put(field.number, ordIndexInstance);
|
||||
}
|
||||
ordIndex = ordIndexInstance;
|
||||
}
|
||||
final MonotonicBlockPackedReader ordIndex = new MonotonicBlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
|
||||
|
||||
return new SortedSetDocValues() {
|
||||
long offset;
|
||||
|
@ -472,31 +381,11 @@ class DiskDocValuesProducer extends DocValuesProducer {
|
|||
public long getValueCount() {
|
||||
return valueCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long lookupTerm(BytesRef key) {
|
||||
if (binary instanceof CompressedBinaryDocValues) {
|
||||
return ((CompressedBinaryDocValues)binary).lookupTerm(key);
|
||||
} else {
|
||||
return super.lookupTerm(key);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum termsEnum() {
|
||||
if (binary instanceof CompressedBinaryDocValues) {
|
||||
return ((CompressedBinaryDocValues)binary).getTermsEnum();
|
||||
} else {
|
||||
return super.termsEnum();
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
// nocommit: only use this if the field's entry has missing values (write that),
|
||||
// otherwise return MatchAllBits
|
||||
if (field.getDocValuesType() == FieldInfo.DocValuesType.SORTED_SET) {
|
||||
return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
|
||||
} else {
|
||||
|
@ -525,12 +414,10 @@ class DiskDocValuesProducer extends DocValuesProducer {
|
|||
static class BinaryEntry {
|
||||
long offset;
|
||||
|
||||
int format;
|
||||
long count;
|
||||
int minLength;
|
||||
int maxLength;
|
||||
long addressesOffset;
|
||||
long addressInterval;
|
||||
int packedIntsVersion;
|
||||
int blockSize;
|
||||
}
|
||||
|
@ -553,204 +440,4 @@ class DiskDocValuesProducer extends DocValuesProducer {
|
|||
|
||||
abstract void get(long id, BytesRef Result);
|
||||
}
|
||||
|
||||
// in the compressed case, we add a few additional operations for
|
||||
// more efficient reverse lookup and enumeration
|
||||
static class CompressedBinaryDocValues extends LongBinaryDocValues {
|
||||
final BinaryEntry bytes;
|
||||
final long interval;
|
||||
final long numValues;
|
||||
final long numIndexValues;
|
||||
final MonotonicBlockPackedReader addresses;
|
||||
final IndexInput data;
|
||||
final TermsEnum termsEnum;
|
||||
|
||||
public CompressedBinaryDocValues(BinaryEntry bytes, MonotonicBlockPackedReader addresses, IndexInput data) throws IOException {
|
||||
this.bytes = bytes;
|
||||
this.interval = bytes.addressInterval;
|
||||
this.addresses = addresses;
|
||||
this.data = data;
|
||||
this.numValues = bytes.count;
|
||||
this.numIndexValues = addresses.size();
|
||||
this.termsEnum = getTermsEnum(data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void get(long id, BytesRef result) {
|
||||
try {
|
||||
termsEnum.seekExact(id);
|
||||
BytesRef term = termsEnum.term();
|
||||
result.bytes = term.bytes;
|
||||
result.offset = term.offset;
|
||||
result.length = term.length;
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
long lookupTerm(BytesRef key) {
|
||||
try {
|
||||
SeekStatus status = termsEnum.seekCeil(key);
|
||||
if (status == SeekStatus.END) {
|
||||
return -numValues-1;
|
||||
} else if (status == SeekStatus.FOUND) {
|
||||
return termsEnum.ord();
|
||||
} else {
|
||||
return -termsEnum.ord()-1;
|
||||
}
|
||||
} catch (IOException bogus) {
|
||||
throw new RuntimeException(bogus);
|
||||
}
|
||||
}
|
||||
|
||||
TermsEnum getTermsEnum() {
|
||||
try {
|
||||
return getTermsEnum(data.clone());
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private TermsEnum getTermsEnum(final IndexInput input) throws IOException {
|
||||
input.seek(bytes.offset);
|
||||
|
||||
return new TermsEnum() {
|
||||
private long currentOrd = -1;
|
||||
// TODO: maxLength is negative when all terms are merged away...
|
||||
private final BytesRef termBuffer = new BytesRef(bytes.maxLength < 0 ? 0 : bytes.maxLength);
|
||||
private final BytesRef term = new BytesRef(); // TODO: paranoia?
|
||||
|
||||
@Override
|
||||
public BytesRef next() throws IOException {
|
||||
if (doNext() == null) {
|
||||
return null;
|
||||
} else {
|
||||
setTerm();
|
||||
return term;
|
||||
}
|
||||
}
|
||||
|
||||
private BytesRef doNext() throws IOException {
|
||||
if (++currentOrd >= numValues) {
|
||||
return null;
|
||||
} else {
|
||||
int start = input.readVInt();
|
||||
int suffix = input.readVInt();
|
||||
input.readBytes(termBuffer.bytes, start, suffix);
|
||||
termBuffer.length = start + suffix;
|
||||
return termBuffer;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public SeekStatus seekCeil(BytesRef text) throws IOException {
|
||||
// binary-search just the index values to find the block,
|
||||
// then scan within the block
|
||||
long low = 0;
|
||||
long high = numIndexValues-1;
|
||||
|
||||
while (low <= high) {
|
||||
long mid = (low + high) >>> 1;
|
||||
doSeek(mid * interval);
|
||||
int cmp = termBuffer.compareTo(text);
|
||||
|
||||
if (cmp < 0) {
|
||||
low = mid + 1;
|
||||
} else if (cmp > 0) {
|
||||
high = mid - 1;
|
||||
} else {
|
||||
// we got lucky, found an indexed term
|
||||
setTerm();
|
||||
return SeekStatus.FOUND;
|
||||
}
|
||||
}
|
||||
|
||||
if (numIndexValues == 0) {
|
||||
return SeekStatus.END;
|
||||
}
|
||||
|
||||
// block before insertion point
|
||||
long block = low-1;
|
||||
doSeek(block < 0 ? -1 : block * interval);
|
||||
|
||||
while (doNext() != null) {
|
||||
int cmp = termBuffer.compareTo(text);
|
||||
if (cmp == 0) {
|
||||
setTerm();
|
||||
return SeekStatus.FOUND;
|
||||
} else if (cmp > 0) {
|
||||
setTerm();
|
||||
return SeekStatus.NOT_FOUND;
|
||||
}
|
||||
}
|
||||
|
||||
return SeekStatus.END;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void seekExact(long ord) throws IOException {
|
||||
doSeek(ord);
|
||||
setTerm();
|
||||
}
|
||||
|
||||
private void doSeek(long ord) throws IOException {
|
||||
long block = ord / interval;
|
||||
|
||||
if (ord >= currentOrd && block == currentOrd / interval) {
|
||||
// seek within current block
|
||||
} else {
|
||||
// position before start of block
|
||||
currentOrd = ord - ord % interval - 1;
|
||||
input.seek(bytes.offset + addresses.get(block));
|
||||
}
|
||||
|
||||
while (currentOrd < ord) {
|
||||
doNext();
|
||||
}
|
||||
}
|
||||
|
||||
private void setTerm() {
|
||||
// TODO: is there a cleaner way
|
||||
term.bytes = new byte[termBuffer.length];
|
||||
term.offset = 0;
|
||||
term.copyBytes(termBuffer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef term() throws IOException {
|
||||
return term;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ord() throws IOException {
|
||||
return currentOrd;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docFreq() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long totalTermFreq() throws IOException {
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.codecs.cheapbastard;
|
||||
package org.apache.lucene.codecs.diskdv;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -22,25 +22,25 @@ import java.io.IOException;
|
|||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
||||
/** Norms format that keeps all norms on disk */
|
||||
public final class CheapBastardNormsFormat extends NormsFormat {
|
||||
public final class DiskNormsFormat extends NormsFormat {
|
||||
|
||||
@Override
|
||||
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
|
||||
return new DiskDocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
|
||||
return new Lucene45DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesProducer normsProducer(SegmentReadState state) throws IOException {
|
||||
return new CheapBastardDocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
|
||||
return new DiskDocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
|
||||
}
|
||||
|
||||
static final String DATA_CODEC = "CheapBastardNormsData";
|
||||
static final String DATA_EXTENSION = "cbnd";
|
||||
static final String META_CODEC = "CheapBastardNormsMetadata";
|
||||
static final String META_EXTENSION = "cbnm";
|
||||
static final String DATA_CODEC = "DiskNormsData";
|
||||
static final String DATA_EXTENSION = "dnvd";
|
||||
static final String META_CODEC = "DiskNormsMetadata";
|
||||
static final String META_EXTENSION = "dnvm";
|
||||
}
|
|
@ -119,7 +119,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
|
|||
loader.reload(classloader);
|
||||
}
|
||||
|
||||
private static Codec defaultCodec = Codec.forName("Lucene42");
|
||||
private static Codec defaultCodec = Codec.forName("Lucene45");
|
||||
|
||||
/** expert: returns the default codec used for newly created
|
||||
* {@link IndexWriterConfig}s.
|
||||
|
|
|
@ -21,13 +21,13 @@ package org.apache.lucene.codecs;
|
|||
* A codec that forwards all its method calls to another codec.
|
||||
* <p>
|
||||
* Extend this class when you need to reuse the functionality of an existing
|
||||
* codec. For example, if you want to build a codec that redefines Lucene42's
|
||||
* codec. For example, if you want to build a codec that redefines Lucene45's
|
||||
* {@link LiveDocsFormat}:
|
||||
* <pre class="prettyprint">
|
||||
* public final class CustomCodec extends FilterCodec {
|
||||
*
|
||||
* public CustomCodec() {
|
||||
* super("CustomCodec", new Lucene42Codec());
|
||||
* super("CustomCodec", new Lucene45Codec());
|
||||
* }
|
||||
*
|
||||
* public LiveDocsFormat liveDocsFormat() {
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.apache.lucene.codecs.DocValuesFormat;
|
|||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
|
||||
/**
|
||||
|
|
|
@ -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.lucene40.Lucene40TermVectorsFormat Term Vectors}.
|
||||
{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat 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
|
||||
|
@ -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.lucene40.Lucene40TermVectorsFormat Term Vector Index}</td>
|
||||
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Index}</td>
|
||||
<td>.tvx</td>
|
||||
<td>Stores offset into the document data file</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Documents}</td>
|
||||
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat 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.lucene40.Lucene40TermVectorsFormat Term Vector Fields}</td>
|
||||
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Fields}</td>
|
||||
<td>.tvf</td>
|
||||
<td>The field level info about term vectors</td>
|
||||
</tr>
|
||||
|
|
|
@ -0,0 +1,141 @@
|
|||
package org.apache.lucene.codecs.lucene45;
|
||||
|
||||
/*
|
||||
* 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.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
|
||||
/**
|
||||
* Implements the Lucene 4.5 index format, with configurable per-field postings
|
||||
* and docvalues formats.
|
||||
* <p>
|
||||
* If you want to reuse functionality of this codec in another codec, extend
|
||||
* {@link FilterCodec}.
|
||||
*
|
||||
* @see org.apache.lucene.codecs.lucene45 package documentation for file format details.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
// NOTE: if we make largish changes in a minor release, easier to just make Lucene46Codec or whatever
|
||||
// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
|
||||
// (it writes a minor version, etc).
|
||||
public class Lucene45Codec extends Codec {
|
||||
private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
|
||||
private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
|
||||
private final FieldInfosFormat fieldInfosFormat = new Lucene42FieldInfosFormat();
|
||||
private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
|
||||
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
|
||||
|
||||
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return Lucene45Codec.this.getPostingsFormatForField(field);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return Lucene45Codec.this.getDocValuesFormatForField(field);
|
||||
}
|
||||
};
|
||||
|
||||
/** Sole constructor. */
|
||||
public Lucene45Codec() {
|
||||
super("Lucene45");
|
||||
}
|
||||
|
||||
@Override
|
||||
public final StoredFieldsFormat storedFieldsFormat() {
|
||||
return fieldsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final TermVectorsFormat termVectorsFormat() {
|
||||
return vectorsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final PostingsFormat postingsFormat() {
|
||||
return postingsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final FieldInfosFormat fieldInfosFormat() {
|
||||
return fieldInfosFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final SegmentInfoFormat segmentInfoFormat() {
|
||||
return infosFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final LiveDocsFormat liveDocsFormat() {
|
||||
return liveDocsFormat;
|
||||
}
|
||||
|
||||
/** Returns the postings format that should be used for writing
|
||||
* new segments of <code>field</code>.
|
||||
*
|
||||
* The default implementation always returns "Lucene41"
|
||||
*/
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return defaultFormat;
|
||||
}
|
||||
|
||||
/** Returns the docvalues format that should be used for writing
|
||||
* new segments of <code>field</code>.
|
||||
*
|
||||
* The default implementation always returns "Lucene45"
|
||||
*/
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return defaultDVFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final DocValuesFormat docValuesFormat() {
|
||||
return docValuesFormat;
|
||||
}
|
||||
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
|
||||
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene45");
|
||||
|
||||
private final NormsFormat normsFormat = new Lucene42NormsFormat();
|
||||
|
||||
@Override
|
||||
public final NormsFormat normsFormat() {
|
||||
return normsFormat;
|
||||
}
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.codecs.diskdv;
|
||||
package org.apache.lucene.codecs.lucene45;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -37,8 +37,8 @@ import org.apache.lucene.util.packed.BlockPackedWriter;
|
|||
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/** writer for {@link DiskDocValuesFormat} */
|
||||
public class DiskDocValuesConsumer extends DocValuesConsumer {
|
||||
/** writer for {@link Lucene45DocValuesFormat} */
|
||||
public class Lucene45DocValuesConsumer extends DocValuesConsumer {
|
||||
|
||||
static final int BLOCK_SIZE = 16384;
|
||||
static final int ADDRESS_INTERVAL = 16;
|
||||
|
@ -60,15 +60,15 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
final IndexOutput data, meta;
|
||||
final int maxDoc;
|
||||
|
||||
public DiskDocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
public Lucene45DocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
boolean success = false;
|
||||
try {
|
||||
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
|
||||
data = state.directory.createOutput(dataName, state.context);
|
||||
CodecUtil.writeHeader(data, dataCodec, DiskDocValuesFormat.VERSION_CURRENT);
|
||||
CodecUtil.writeHeader(data, dataCodec, Lucene45DocValuesFormat.VERSION_CURRENT);
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
meta = state.directory.createOutput(metaName, state.context);
|
||||
CodecUtil.writeHeader(meta, metaCodec, DiskDocValuesFormat.VERSION_CURRENT);
|
||||
CodecUtil.writeHeader(meta, metaCodec, Lucene45DocValuesFormat.VERSION_CURRENT);
|
||||
maxDoc = state.segmentInfo.getDocCount();
|
||||
success = true;
|
||||
} finally {
|
||||
|
@ -140,7 +140,7 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
format = DELTA_COMPRESSED;
|
||||
}
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(DiskDocValuesFormat.NUMERIC);
|
||||
meta.writeByte(Lucene45DocValuesFormat.NUMERIC);
|
||||
meta.writeVInt(format);
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
meta.writeLong(data.getFilePointer());
|
||||
|
@ -189,7 +189,7 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
|
||||
// write the byte[] data
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(DiskDocValuesFormat.BINARY);
|
||||
meta.writeByte(Lucene45DocValuesFormat.BINARY);
|
||||
int minLength = Integer.MAX_VALUE;
|
||||
int maxLength = Integer.MIN_VALUE;
|
||||
final long startFP = data.getFilePointer();
|
||||
|
@ -242,7 +242,7 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
} else {
|
||||
// header
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(DiskDocValuesFormat.BINARY);
|
||||
meta.writeByte(Lucene45DocValuesFormat.BINARY);
|
||||
meta.writeVInt(BINARY_PREFIX_COMPRESSED);
|
||||
// now write the bytes: sharing prefixes within a block
|
||||
final long startFP = data.getFilePointer();
|
||||
|
@ -315,7 +315,7 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
values = MissingOrdRemapper.insertEmptyValue(values);
|
||||
}
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(DiskDocValuesFormat.SORTED);
|
||||
meta.writeByte(Lucene45DocValuesFormat.SORTED);
|
||||
addTermsDict(field, values);
|
||||
addNumericField(field, docToOrd, false);
|
||||
}
|
||||
|
@ -323,7 +323,7 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
@Override
|
||||
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(DiskDocValuesFormat.SORTED_SET);
|
||||
meta.writeByte(Lucene45DocValuesFormat.SORTED_SET);
|
||||
// write the ord -> byte[] as a binary field
|
||||
addTermsDict(field, values);
|
||||
// write the stream of ords as a numeric field
|
||||
|
@ -332,7 +332,7 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
|
||||
// write the doc -> ord count as a absolute index to the stream
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(DiskDocValuesFormat.NUMERIC);
|
||||
meta.writeByte(Lucene45DocValuesFormat.NUMERIC);
|
||||
meta.writeVInt(DELTA_COMPRESSED);
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
meta.writeLong(data.getFilePointer());
|
|
@ -0,0 +1,167 @@
|
|||
package org.apache.lucene.codecs.lucene45;
|
||||
|
||||
/*
|
||||
* 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.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.FieldInfo.DocValuesType;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.util.SmallFloat;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
import org.apache.lucene.util.packed.BlockPackedWriter;
|
||||
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
* Lucene 4.5 DocValues format.
|
||||
* <p>
|
||||
* Encodes the four per-document value types (Numeric,Binary,Sorted,SortedSet) with these strategies:
|
||||
* <p>
|
||||
* {@link DocValuesType#NUMERIC NUMERIC}:
|
||||
* <ul>
|
||||
* <li>Delta-compressed: per-document integers written in blocks of 16k. For each block
|
||||
* the minimum value in that block is encoded, and each entry is a delta from that
|
||||
* minimum value. Each block of deltas is compressed with bitpacking. For more
|
||||
* information, see {@link BlockPackedWriter}.
|
||||
* <li>Table-compressed: when the number of unique values is very small (< 256), and
|
||||
* when there are unused "gaps" in the range of values used (such as {@link SmallFloat}),
|
||||
* a lookup table is written instead. Each per-document entry is instead the ordinal
|
||||
* to this table, and those ordinals are compressed with bitpacking ({@link PackedInts}).
|
||||
* <li>GCD-compressed: when all numbers share a common divisor, such as dates, the greatest
|
||||
* common denominator (GCD) is computed, and quotients are stored using Delta-compressed Numerics.
|
||||
* </ul>
|
||||
* <p>
|
||||
* {@link DocValuesType#BINARY BINARY}:
|
||||
* <ul>
|
||||
* <li>Fixed-width Binary: one large concatenated byte[] is written, along with the fixed length.
|
||||
* Each document's value can be addressed directly with multiplication ({@code docID * length}).
|
||||
* <li>Variable-width Binary: one large concatenated byte[] is written, along with end addresses
|
||||
* for each document. The addresses are written in blocks of 16k, with the current absolute
|
||||
* start for the block, and the average (expected) delta per entry. For each document the
|
||||
* deviation from the delta (actual - expected) is written.
|
||||
* <li>Prefix-compressed Binary: nocommit
|
||||
* </ul>
|
||||
* <p>
|
||||
* {@link DocValuesType#SORTED SORTED}:
|
||||
* <ul>
|
||||
* <li>Sorted: an FST mapping deduplicated terms to ordinals is written, along with the per-document
|
||||
* ordinals written using one of the numeric strategies above.
|
||||
* </ul>
|
||||
* <p>
|
||||
* {@link DocValuesType#SORTED_SET SORTED_SET}:
|
||||
* <ul>
|
||||
* <li>SortedSet: an FST mapping deduplicated terms to ordinals is written, along with the per-document
|
||||
* ordinal list written using one of the binary strategies above.
|
||||
* </ul>
|
||||
* <p>
|
||||
* Files:
|
||||
* <ol>
|
||||
* <li><tt>.dvd</tt>: DocValues data</li>
|
||||
* <li><tt>.dvm</tt>: DocValues metadata</li>
|
||||
* </ol>
|
||||
* <ol>
|
||||
* <li><a name="dvm" id="dvm"></a>
|
||||
* <p>The DocValues metadata or .dvm file.</p>
|
||||
* <p>For DocValues field, this stores metadata, such as the offset into the
|
||||
* DocValues data (.dvd)</p>
|
||||
* <p>DocValues metadata (.dvm) --> Header,<FieldNumber,EntryType,Entry><sup>NumFields</sup></p>
|
||||
* <ul>
|
||||
* <li>Entry --> NumericEntry | BinaryEntry | SortedEntry</li>
|
||||
* <li>NumericEntry --> DataOffset,NumericCompressionType,PackedVersion</li>
|
||||
* <li>BinaryEntry --> DataOffset,DataLength,MinLength,MaxLength,PackedVersion?,BlockSize?</li>
|
||||
* <li>SortedEntry --> DataOffset,ValueCount</li>
|
||||
* <li>FieldNumber,PackedVersion,MinLength,MaxLength,BlockSize,ValueCount --> {@link DataOutput#writeVInt VInt}</li>
|
||||
* <li>DataOffset,DataLength --> {@link DataOutput#writeLong Int64}</li>
|
||||
* <li>EntryType,CompressionType --> {@link DataOutput#writeByte Byte}</li>
|
||||
* <li>Header --> {@link CodecUtil#writeHeader CodecHeader}</li>
|
||||
* </ul>
|
||||
* <p>Sorted fields have two entries: a SortedEntry with the FST metadata,
|
||||
* and an ordinary NumericEntry for the document-to-ord metadata.</p>
|
||||
* <p>SortedSet fields have two entries: a SortedEntry with the FST metadata,
|
||||
* and an ordinary BinaryEntry for the document-to-ord-list metadata.</p>
|
||||
* <p>FieldNumber of -1 indicates the end of metadata.</p>
|
||||
* <p>EntryType is a 0 (NumericEntry), 1 (BinaryEntry, or 2 (SortedEntry)</p>
|
||||
* <p>DataOffset is the pointer to the start of the data in the DocValues data (.dvd)</p>
|
||||
* <p>NumericCompressionType indicates how Numeric values will be compressed:
|
||||
* <ul>
|
||||
* <li>0 --> delta-compressed. For each block of 16k integers, every integer is delta-encoded
|
||||
* from the minimum value within the block.
|
||||
* <li>1 -->, gcd-compressed. When all integers share a common divisor, only quotients are stored
|
||||
* using blocks of delta-encoded ints.
|
||||
* <li>2 --> table-compressed. When the number of unique numeric values is small and it would save space,
|
||||
* a lookup table of unique values is written, followed by the ordinal for each document.
|
||||
* </ul>
|
||||
* <p>MinLength and MaxLength represent the min and max byte[] value lengths for Binary values.
|
||||
* If they are equal, then all values are of a fixed size, and can be addressed as DataOffset + (docID * length).
|
||||
* Otherwise, the binary values are of variable size, and packed integer metadata (PackedVersion,BlockSize)
|
||||
* is written for the addresses.
|
||||
* <li><a name="dvd" id="dvd"></a>
|
||||
* <p>The DocValues data or .dvd file.</p>
|
||||
* <p>For DocValues field, this stores the actual per-document data (the heavy-lifting)</p>
|
||||
* <p>DocValues data (.dvd) --> Header,<NumericData | BinaryData | SortedData><sup>NumFields</sup></p>
|
||||
* <ul>
|
||||
* <li>NumericData --> DeltaCompressedNumerics | TableCompressedNumerics | GCDCompressedNumerics</li>
|
||||
* <li>BinaryData --> {@link DataOutput#writeByte Byte}<sup>DataLength</sup>,Addresses</li>
|
||||
* <li>SortedData --> {@link FST FST<Int64>}</li>
|
||||
* <li>DeltaCompressedNumerics --> {@link BlockPackedWriter BlockPackedInts(blockSize=16k)}</li>
|
||||
* <li>TableCompressedNumerics --> TableSize,{@link DataOutput#writeLong Int64}<sup>TableSize</sup>,{@link PackedInts PackedInts}</li>
|
||||
* <li>GCDCompressedNumerics --> MinValue,GCD,{@link BlockPackedWriter BlockPackedInts(blockSize=16k)}</li>
|
||||
* <li>Addresses --> {@link MonotonicBlockPackedWriter MonotonicBlockPackedInts(blockSize=16k)}</li>
|
||||
* <li>TableSize --> {@link DataOutput#writeVInt vInt}</li>
|
||||
* <li>MinValue --> {@link DataOutput#writeLong Int64}</li>
|
||||
* <li>GCD --> {@link DataOutput#writeLong Int64}</li>
|
||||
* </ul>
|
||||
* <p>SortedSet entries store the list of ordinals in their BinaryData as a
|
||||
* sequences of increasing {@link DataOutput#writeVLong vLong}s, delta-encoded.</p>
|
||||
* </ol>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
// nocommit: docs are incomplete
|
||||
public final class Lucene45DocValuesFormat extends DocValuesFormat {
|
||||
|
||||
public Lucene45DocValuesFormat() {
|
||||
super("Lucene45");
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
return new Lucene45DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
return new Lucene45DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
|
||||
}
|
||||
|
||||
public static final String DATA_CODEC = "Lucene45DocValuesData";
|
||||
public static final String DATA_EXTENSION = "dvd";
|
||||
public static final String META_CODEC = "Lucene45ValuesMetadata";
|
||||
public static final String META_EXTENSION = "dvm";
|
||||
public static final int VERSION_START = 0;
|
||||
public static final int VERSION_CURRENT = VERSION_START;
|
||||
public static final byte NUMERIC = 0;
|
||||
public static final byte BINARY = 1;
|
||||
public static final byte SORTED = 2;
|
||||
public static final byte SORTED_SET = 3;
|
||||
}
|
|
@ -0,0 +1,755 @@
|
|||
package org.apache.lucene.codecs.lucene45;
|
||||
|
||||
/*
|
||||
* 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.lucene45.Lucene45DocValuesConsumer.DELTA_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.GCD_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.TABLE_COMPRESSED;
|
||||
|
||||
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.BINARY_FIXED_UNCOMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.BINARY_PREFIX_COMPRESSED;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.TermsEnum.SeekStatus;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.packed.BlockPackedReader;
|
||||
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
class Lucene45DocValuesProducer extends DocValuesProducer {
|
||||
private final Map<Integer,NumericEntry> numerics;
|
||||
private final Map<Integer,BinaryEntry> binaries;
|
||||
private final Map<Integer,NumericEntry> ords;
|
||||
private final Map<Integer,NumericEntry> ordIndexes;
|
||||
private final IndexInput data;
|
||||
private final int maxDoc;
|
||||
|
||||
// memory-resident structures
|
||||
private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
|
||||
private final Map<Integer,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
|
||||
|
||||
Lucene45DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
// read in the entries from the metadata file.
|
||||
IndexInput in = state.directory.openInput(metaName, state.context);
|
||||
this.maxDoc = state.segmentInfo.getDocCount();
|
||||
boolean success = false;
|
||||
final int version;
|
||||
try {
|
||||
version = CodecUtil.checkHeader(in, metaCodec,
|
||||
Lucene45DocValuesFormat.VERSION_CURRENT,
|
||||
Lucene45DocValuesFormat.VERSION_CURRENT);
|
||||
numerics = new HashMap<Integer,NumericEntry>();
|
||||
ords = new HashMap<Integer,NumericEntry>();
|
||||
ordIndexes = new HashMap<Integer,NumericEntry>();
|
||||
binaries = new HashMap<Integer,BinaryEntry>();
|
||||
readFields(in, state.fieldInfos);
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(in);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(in);
|
||||
}
|
||||
}
|
||||
|
||||
success = false;
|
||||
try {
|
||||
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
|
||||
data = state.directory.openInput(dataName, state.context);
|
||||
final int version2 = CodecUtil.checkHeader(data, dataCodec,
|
||||
Lucene45DocValuesFormat.VERSION_CURRENT,
|
||||
Lucene45DocValuesFormat.VERSION_CURRENT);
|
||||
if (version != version2) {
|
||||
throw new CorruptIndexException("Format versions mismatch");
|
||||
}
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(this.data);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
|
||||
int fieldNumber = meta.readVInt();
|
||||
while (fieldNumber != -1) {
|
||||
byte type = meta.readByte();
|
||||
if (type == Lucene45DocValuesFormat.NUMERIC) {
|
||||
numerics.put(fieldNumber, readNumericEntry(meta));
|
||||
} else if (type == Lucene45DocValuesFormat.BINARY) {
|
||||
BinaryEntry b = readBinaryEntry(meta);
|
||||
binaries.put(fieldNumber, b);
|
||||
} else if (type == Lucene45DocValuesFormat.SORTED) {
|
||||
// sorted = binary + numeric
|
||||
if (meta.readVInt() != fieldNumber) {
|
||||
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
}
|
||||
if (meta.readByte() != Lucene45DocValuesFormat.BINARY) {
|
||||
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
}
|
||||
BinaryEntry b = readBinaryEntry(meta);
|
||||
binaries.put(fieldNumber, b);
|
||||
|
||||
if (meta.readVInt() != fieldNumber) {
|
||||
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
}
|
||||
if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
|
||||
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
}
|
||||
NumericEntry n = readNumericEntry(meta);
|
||||
ords.put(fieldNumber, n);
|
||||
} else if (type == Lucene45DocValuesFormat.SORTED_SET) {
|
||||
// sortedset = binary + numeric + ordIndex
|
||||
if (meta.readVInt() != fieldNumber) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
}
|
||||
if (meta.readByte() != Lucene45DocValuesFormat.BINARY) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
}
|
||||
BinaryEntry b = readBinaryEntry(meta);
|
||||
binaries.put(fieldNumber, b);
|
||||
|
||||
if (meta.readVInt() != fieldNumber) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
}
|
||||
if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
}
|
||||
NumericEntry n1 = readNumericEntry(meta);
|
||||
ords.put(fieldNumber, n1);
|
||||
|
||||
if (meta.readVInt() != fieldNumber) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
}
|
||||
if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
}
|
||||
NumericEntry n2 = readNumericEntry(meta);
|
||||
ordIndexes.put(fieldNumber, n2);
|
||||
} else {
|
||||
throw new CorruptIndexException("invalid type: " + type + ", resource=" + meta);
|
||||
}
|
||||
fieldNumber = meta.readVInt();
|
||||
}
|
||||
}
|
||||
|
||||
static NumericEntry readNumericEntry(IndexInput meta) throws IOException {
|
||||
NumericEntry entry = new NumericEntry();
|
||||
entry.format = meta.readVInt();
|
||||
entry.packedIntsVersion = meta.readVInt();
|
||||
entry.offset = meta.readLong();
|
||||
entry.count = meta.readVLong();
|
||||
entry.blockSize = meta.readVInt();
|
||||
switch(entry.format) {
|
||||
case GCD_COMPRESSED:
|
||||
entry.minValue = meta.readLong();
|
||||
entry.gcd = meta.readLong();
|
||||
break;
|
||||
case TABLE_COMPRESSED:
|
||||
if (entry.count > Integer.MAX_VALUE) {
|
||||
throw new CorruptIndexException("Cannot use TABLE_COMPRESSED with more than MAX_VALUE values, input=" + meta);
|
||||
}
|
||||
final int uniqueValues = meta.readVInt();
|
||||
if (uniqueValues > 256) {
|
||||
throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, input=" + meta);
|
||||
}
|
||||
entry.table = new long[uniqueValues];
|
||||
for (int i = 0; i < uniqueValues; ++i) {
|
||||
entry.table[i] = meta.readLong();
|
||||
}
|
||||
break;
|
||||
case DELTA_COMPRESSED:
|
||||
break;
|
||||
default:
|
||||
throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
|
||||
}
|
||||
return entry;
|
||||
}
|
||||
|
||||
static BinaryEntry readBinaryEntry(IndexInput meta) throws IOException {
|
||||
BinaryEntry entry = new BinaryEntry();
|
||||
entry.format = meta.readVInt();
|
||||
entry.minLength = meta.readVInt();
|
||||
entry.maxLength = meta.readVInt();
|
||||
entry.count = meta.readVLong();
|
||||
entry.offset = meta.readLong();
|
||||
switch(entry.format) {
|
||||
case BINARY_FIXED_UNCOMPRESSED:
|
||||
break;
|
||||
case BINARY_PREFIX_COMPRESSED:
|
||||
entry.addressInterval = meta.readVInt();
|
||||
entry.addressesOffset = meta.readLong();
|
||||
entry.packedIntsVersion = meta.readVInt();
|
||||
entry.blockSize = meta.readVInt();
|
||||
break;
|
||||
case BINARY_VARIABLE_UNCOMPRESSED:
|
||||
entry.addressesOffset = meta.readLong();
|
||||
entry.packedIntsVersion = meta.readVInt();
|
||||
entry.blockSize = meta.readVInt();
|
||||
break;
|
||||
default:
|
||||
throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
|
||||
}
|
||||
return entry;
|
||||
}
|
||||
|
||||
@Override
|
||||
public NumericDocValues getNumeric(FieldInfo field) throws IOException {
|
||||
NumericEntry entry = numerics.get(field.number);
|
||||
return getNumeric(entry);
|
||||
}
|
||||
|
||||
LongNumericDocValues getNumeric(NumericEntry entry) throws IOException {
|
||||
final IndexInput data = this.data.clone();
|
||||
data.seek(entry.offset);
|
||||
|
||||
switch (entry.format) {
|
||||
case DELTA_COMPRESSED:
|
||||
final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
|
||||
return new LongNumericDocValues() {
|
||||
@Override
|
||||
public long get(long id) {
|
||||
return reader.get(id);
|
||||
}
|
||||
};
|
||||
case GCD_COMPRESSED:
|
||||
final long min = entry.minValue;
|
||||
final long mult = entry.gcd;
|
||||
final BlockPackedReader quotientReader = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
|
||||
return new LongNumericDocValues() {
|
||||
@Override
|
||||
public long get(long id) {
|
||||
return min + mult * quotientReader.get(id);
|
||||
}
|
||||
};
|
||||
case TABLE_COMPRESSED:
|
||||
final long table[] = entry.table;
|
||||
final int bitsRequired = PackedInts.bitsRequired(table.length - 1);
|
||||
final PackedInts.Reader ords = PackedInts.getDirectReaderNoHeader(data, PackedInts.Format.PACKED, entry.packedIntsVersion, (int) entry.count, bitsRequired);
|
||||
return new LongNumericDocValues() {
|
||||
@Override
|
||||
public long get(long id) {
|
||||
return table[(int) ords.get((int) id)];
|
||||
}
|
||||
};
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public BinaryDocValues getBinary(FieldInfo field) throws IOException {
|
||||
BinaryEntry bytes = binaries.get(field.number);
|
||||
switch(bytes.format) {
|
||||
case BINARY_FIXED_UNCOMPRESSED:
|
||||
return getFixedBinary(field, bytes);
|
||||
case BINARY_VARIABLE_UNCOMPRESSED:
|
||||
return getVariableBinary(field, bytes);
|
||||
case BINARY_PREFIX_COMPRESSED:
|
||||
return getCompressedBinary(field, bytes);
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
|
||||
private BinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) {
|
||||
final IndexInput data = this.data.clone();
|
||||
|
||||
return new LongBinaryDocValues() {
|
||||
@Override
|
||||
public void get(long id, BytesRef result) {
|
||||
long address = bytes.offset + id * bytes.maxLength;
|
||||
try {
|
||||
data.seek(address);
|
||||
// NOTE: we could have one buffer, but various consumers (e.g. FieldComparatorSource)
|
||||
// assume "they" own the bytes after calling this!
|
||||
final byte[] buffer = new byte[bytes.maxLength];
|
||||
data.readBytes(buffer, 0, buffer.length);
|
||||
result.bytes = buffer;
|
||||
result.offset = 0;
|
||||
result.length = buffer.length;
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private BinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
|
||||
final IndexInput data = this.data.clone();
|
||||
|
||||
final MonotonicBlockPackedReader addresses;
|
||||
synchronized (addressInstances) {
|
||||
MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
|
||||
if (addrInstance == null) {
|
||||
data.seek(bytes.addressesOffset);
|
||||
addrInstance = new MonotonicBlockPackedReader(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count, false);
|
||||
addressInstances.put(field.number, addrInstance);
|
||||
}
|
||||
addresses = addrInstance;
|
||||
}
|
||||
|
||||
return new LongBinaryDocValues() {
|
||||
@Override
|
||||
public void get(long id, BytesRef result) {
|
||||
long startAddress = bytes.offset + (id == 0 ? 0 : addresses.get(id-1));
|
||||
long endAddress = bytes.offset + addresses.get(id);
|
||||
int length = (int) (endAddress - startAddress);
|
||||
try {
|
||||
data.seek(startAddress);
|
||||
// NOTE: we could have one buffer, but various consumers (e.g. FieldComparatorSource)
|
||||
// assume "they" own the bytes after calling this!
|
||||
final byte[] buffer = new byte[length];
|
||||
data.readBytes(buffer, 0, buffer.length);
|
||||
result.bytes = buffer;
|
||||
result.offset = 0;
|
||||
result.length = length;
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private BinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
|
||||
final IndexInput data = this.data.clone();
|
||||
final long interval = bytes.addressInterval;
|
||||
|
||||
final MonotonicBlockPackedReader addresses;
|
||||
synchronized (addressInstances) {
|
||||
MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
|
||||
if (addrInstance == null) {
|
||||
data.seek(bytes.addressesOffset);
|
||||
final long size;
|
||||
if (bytes.count % interval == 0) {
|
||||
size = bytes.count / interval;
|
||||
} else {
|
||||
size = 1L + bytes.count / interval;
|
||||
}
|
||||
addrInstance = new MonotonicBlockPackedReader(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
|
||||
addressInstances.put(field.number, addrInstance);
|
||||
}
|
||||
addresses = addrInstance;
|
||||
}
|
||||
|
||||
return new CompressedBinaryDocValues(bytes, addresses, data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedDocValues getSorted(FieldInfo field) throws IOException {
|
||||
final int valueCount = (int) binaries.get(field.number).count;
|
||||
final BinaryDocValues binary = getBinary(field);
|
||||
NumericEntry entry = ords.get(field.number);
|
||||
IndexInput data = this.data.clone();
|
||||
data.seek(entry.offset);
|
||||
final BlockPackedReader ordinals = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
|
||||
|
||||
return new SortedDocValues() {
|
||||
|
||||
@Override
|
||||
public int getOrd(int docID) {
|
||||
return (int) ordinals.get(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void lookupOrd(int ord, BytesRef result) {
|
||||
binary.get(ord, result);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCount() {
|
||||
return valueCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupTerm(BytesRef key) {
|
||||
if (binary instanceof CompressedBinaryDocValues) {
|
||||
return (int) ((CompressedBinaryDocValues)binary).lookupTerm(key);
|
||||
} else {
|
||||
return super.lookupTerm(key);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum termsEnum() {
|
||||
if (binary instanceof CompressedBinaryDocValues) {
|
||||
return ((CompressedBinaryDocValues)binary).getTermsEnum();
|
||||
} else {
|
||||
return super.termsEnum();
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
|
||||
final long valueCount = binaries.get(field.number).count;
|
||||
// we keep the byte[]s and list of ords on disk, these could be large
|
||||
final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
|
||||
final LongNumericDocValues ordinals = getNumeric(ords.get(field.number));
|
||||
// but the addresses to the ord stream are in RAM
|
||||
final MonotonicBlockPackedReader ordIndex;
|
||||
synchronized (ordIndexInstances) {
|
||||
MonotonicBlockPackedReader ordIndexInstance = ordIndexInstances.get(field.number);
|
||||
if (ordIndexInstance == null) {
|
||||
NumericEntry entry = ordIndexes.get(field.number);
|
||||
IndexInput data = this.data.clone();
|
||||
data.seek(entry.offset);
|
||||
ordIndexInstance = new MonotonicBlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, false);
|
||||
ordIndexInstances.put(field.number, ordIndexInstance);
|
||||
}
|
||||
ordIndex = ordIndexInstance;
|
||||
}
|
||||
|
||||
return new SortedSetDocValues() {
|
||||
long offset;
|
||||
long endOffset;
|
||||
|
||||
@Override
|
||||
public long nextOrd() {
|
||||
if (offset == endOffset) {
|
||||
return NO_MORE_ORDS;
|
||||
} else {
|
||||
long ord = ordinals.get(offset);
|
||||
offset++;
|
||||
return ord;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setDocument(int docID) {
|
||||
offset = (docID == 0 ? 0 : ordIndex.get(docID-1));
|
||||
endOffset = ordIndex.get(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void lookupOrd(long ord, BytesRef result) {
|
||||
binary.get(ord, result);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getValueCount() {
|
||||
return valueCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long lookupTerm(BytesRef key) {
|
||||
if (binary instanceof CompressedBinaryDocValues) {
|
||||
return ((CompressedBinaryDocValues)binary).lookupTerm(key);
|
||||
} else {
|
||||
return super.lookupTerm(key);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum termsEnum() {
|
||||
if (binary instanceof CompressedBinaryDocValues) {
|
||||
return ((CompressedBinaryDocValues)binary).getTermsEnum();
|
||||
} else {
|
||||
return super.termsEnum();
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
// nocommit: only use this if the field's entry has missing values (write that),
|
||||
// otherwise return MatchAllBits
|
||||
if (field.getDocValuesType() == FieldInfo.DocValuesType.SORTED_SET) {
|
||||
return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
|
||||
} else {
|
||||
return new Bits.MatchAllBits(maxDoc);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
data.close();
|
||||
}
|
||||
|
||||
static class NumericEntry {
|
||||
long offset;
|
||||
|
||||
int format;
|
||||
int packedIntsVersion;
|
||||
long count;
|
||||
int blockSize;
|
||||
|
||||
long minValue;
|
||||
long gcd;
|
||||
long table[];
|
||||
}
|
||||
|
||||
static class BinaryEntry {
|
||||
long offset;
|
||||
|
||||
int format;
|
||||
long count;
|
||||
int minLength;
|
||||
int maxLength;
|
||||
long addressesOffset;
|
||||
long addressInterval;
|
||||
int packedIntsVersion;
|
||||
int blockSize;
|
||||
}
|
||||
|
||||
// internally we compose complex dv (sorted/sortedset) from other ones
|
||||
static abstract class LongNumericDocValues extends NumericDocValues {
|
||||
@Override
|
||||
public final long get(int docID) {
|
||||
return get((long) docID);
|
||||
}
|
||||
|
||||
abstract long get(long id);
|
||||
}
|
||||
|
||||
static abstract class LongBinaryDocValues extends BinaryDocValues {
|
||||
@Override
|
||||
public final void get(int docID, BytesRef result) {
|
||||
get((long)docID, result);
|
||||
}
|
||||
|
||||
abstract void get(long id, BytesRef Result);
|
||||
}
|
||||
|
||||
// in the compressed case, we add a few additional operations for
|
||||
// more efficient reverse lookup and enumeration
|
||||
static class CompressedBinaryDocValues extends LongBinaryDocValues {
|
||||
final BinaryEntry bytes;
|
||||
final long interval;
|
||||
final long numValues;
|
||||
final long numIndexValues;
|
||||
final MonotonicBlockPackedReader addresses;
|
||||
final IndexInput data;
|
||||
final TermsEnum termsEnum;
|
||||
|
||||
public CompressedBinaryDocValues(BinaryEntry bytes, MonotonicBlockPackedReader addresses, IndexInput data) throws IOException {
|
||||
this.bytes = bytes;
|
||||
this.interval = bytes.addressInterval;
|
||||
this.addresses = addresses;
|
||||
this.data = data;
|
||||
this.numValues = bytes.count;
|
||||
this.numIndexValues = addresses.size();
|
||||
this.termsEnum = getTermsEnum(data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void get(long id, BytesRef result) {
|
||||
try {
|
||||
termsEnum.seekExact(id);
|
||||
BytesRef term = termsEnum.term();
|
||||
result.bytes = term.bytes;
|
||||
result.offset = term.offset;
|
||||
result.length = term.length;
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
long lookupTerm(BytesRef key) {
|
||||
try {
|
||||
SeekStatus status = termsEnum.seekCeil(key);
|
||||
if (status == SeekStatus.END) {
|
||||
return -numValues-1;
|
||||
} else if (status == SeekStatus.FOUND) {
|
||||
return termsEnum.ord();
|
||||
} else {
|
||||
return -termsEnum.ord()-1;
|
||||
}
|
||||
} catch (IOException bogus) {
|
||||
throw new RuntimeException(bogus);
|
||||
}
|
||||
}
|
||||
|
||||
TermsEnum getTermsEnum() {
|
||||
try {
|
||||
return getTermsEnum(data.clone());
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private TermsEnum getTermsEnum(final IndexInput input) throws IOException {
|
||||
input.seek(bytes.offset);
|
||||
|
||||
return new TermsEnum() {
|
||||
private long currentOrd = -1;
|
||||
// TODO: maxLength is negative when all terms are merged away...
|
||||
private final BytesRef termBuffer = new BytesRef(bytes.maxLength < 0 ? 0 : bytes.maxLength);
|
||||
private final BytesRef term = new BytesRef(); // TODO: paranoia?
|
||||
|
||||
@Override
|
||||
public BytesRef next() throws IOException {
|
||||
if (doNext() == null) {
|
||||
return null;
|
||||
} else {
|
||||
setTerm();
|
||||
return term;
|
||||
}
|
||||
}
|
||||
|
||||
private BytesRef doNext() throws IOException {
|
||||
if (++currentOrd >= numValues) {
|
||||
return null;
|
||||
} else {
|
||||
int start = input.readVInt();
|
||||
int suffix = input.readVInt();
|
||||
input.readBytes(termBuffer.bytes, start, suffix);
|
||||
termBuffer.length = start + suffix;
|
||||
return termBuffer;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public SeekStatus seekCeil(BytesRef text) throws IOException {
|
||||
// binary-search just the index values to find the block,
|
||||
// then scan within the block
|
||||
long low = 0;
|
||||
long high = numIndexValues-1;
|
||||
|
||||
while (low <= high) {
|
||||
long mid = (low + high) >>> 1;
|
||||
doSeek(mid * interval);
|
||||
int cmp = termBuffer.compareTo(text);
|
||||
|
||||
if (cmp < 0) {
|
||||
low = mid + 1;
|
||||
} else if (cmp > 0) {
|
||||
high = mid - 1;
|
||||
} else {
|
||||
// we got lucky, found an indexed term
|
||||
setTerm();
|
||||
return SeekStatus.FOUND;
|
||||
}
|
||||
}
|
||||
|
||||
if (numIndexValues == 0) {
|
||||
return SeekStatus.END;
|
||||
}
|
||||
|
||||
// block before insertion point
|
||||
long block = low-1;
|
||||
doSeek(block < 0 ? -1 : block * interval);
|
||||
|
||||
while (doNext() != null) {
|
||||
int cmp = termBuffer.compareTo(text);
|
||||
if (cmp == 0) {
|
||||
setTerm();
|
||||
return SeekStatus.FOUND;
|
||||
} else if (cmp > 0) {
|
||||
setTerm();
|
||||
return SeekStatus.NOT_FOUND;
|
||||
}
|
||||
}
|
||||
|
||||
return SeekStatus.END;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void seekExact(long ord) throws IOException {
|
||||
doSeek(ord);
|
||||
setTerm();
|
||||
}
|
||||
|
||||
private void doSeek(long ord) throws IOException {
|
||||
long block = ord / interval;
|
||||
|
||||
if (ord >= currentOrd && block == currentOrd / interval) {
|
||||
// seek within current block
|
||||
} else {
|
||||
// position before start of block
|
||||
currentOrd = ord - ord % interval - 1;
|
||||
input.seek(bytes.offset + addresses.get(block));
|
||||
}
|
||||
|
||||
while (currentOrd < ord) {
|
||||
doNext();
|
||||
}
|
||||
}
|
||||
|
||||
private void setTerm() {
|
||||
// TODO: is there a cleaner way
|
||||
term.bytes = new byte[termBuffer.length];
|
||||
term.offset = 0;
|
||||
term.copyBytes(termBuffer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef term() throws IOException {
|
||||
return term;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ord() throws IOException {
|
||||
return currentOrd;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docFreq() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long totalTermFreq() throws IOException {
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,396 @@
|
|||
<!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.5 file format.
|
||||
|
||||
<h1>Apache Lucene - Index File Formats</h1>
|
||||
<div>
|
||||
<ul>
|
||||
<li><a href="#Introduction">Introduction</a></li>
|
||||
<li><a href="#Definitions">Definitions</a>
|
||||
<ul>
|
||||
<li><a href="#Inverted_Indexing">Inverted Indexing</a></li>
|
||||
<li><a href="#Types_of_Fields">Types of Fields</a></li>
|
||||
<li><a href="#Segments">Segments</a></li>
|
||||
<li><a href="#Document_Numbers">Document Numbers</a></li>
|
||||
</ul>
|
||||
</li>
|
||||
<li><a href="#Overview">Index Structure Overview</a></li>
|
||||
<li><a href="#File_Naming">File Naming</a></li>
|
||||
<li><a href="#file-names">Summary of File Extensions</a></li>
|
||||
<ul>
|
||||
<li><a href="#Lock_File">Lock File</a></li>
|
||||
<li><a href="#History">History</a></li>
|
||||
<li><a href="#Limitations">Limitations</a></li>
|
||||
</ul>
|
||||
</ul>
|
||||
</div>
|
||||
<a name="Introduction"></a>
|
||||
<h2>Introduction</h2>
|
||||
<div>
|
||||
<p>This document defines the index file formats used in this version of Lucene.
|
||||
If you are using a different version of Lucene, please consult the copy of
|
||||
<code>docs/</code> that was distributed with
|
||||
the version you are using.</p>
|
||||
<p>Apache Lucene is written in Java, but several efforts are underway to write
|
||||
<a href="http://wiki.apache.org/lucene-java/LuceneImplementations">versions of
|
||||
Lucene in other programming languages</a>. If these versions are to remain
|
||||
compatible with Apache Lucene, then a language-independent definition of the
|
||||
Lucene index format is required. This document thus attempts to provide a
|
||||
complete and independent definition of the Apache Lucene file formats.</p>
|
||||
<p>As Lucene evolves, this document should evolve. Versions of Lucene in
|
||||
different programming languages should endeavor to agree on file formats, and
|
||||
generate new versions of this document.</p>
|
||||
</div>
|
||||
<a name="Definitions" id="Definitions"></a>
|
||||
<h2>Definitions</h2>
|
||||
<div>
|
||||
<p>The fundamental concepts in Lucene are index, document, field and term.</p>
|
||||
<p>An index contains a sequence of documents.</p>
|
||||
<ul>
|
||||
<li>A document is a sequence of fields.</li>
|
||||
<li>A field is a named sequence of terms.</li>
|
||||
<li>A term is a sequence of bytes.</li>
|
||||
</ul>
|
||||
<p>The same sequence of bytes in two different fields is considered a different
|
||||
term. Thus terms are represented as a pair: the string naming the field, and the
|
||||
bytes within the field.</p>
|
||||
<a name="Inverted_Indexing"></a>
|
||||
<h3>Inverted Indexing</h3>
|
||||
<p>The index stores statistics about terms in order to make term-based search
|
||||
more efficient. Lucene's index falls into the family of indexes known as an
|
||||
<i>inverted index.</i> This is because it can list, for a term, the documents
|
||||
that contain it. This is the inverse of the natural relationship, in which
|
||||
documents list terms.</p>
|
||||
<a name="Types_of_Fields"></a>
|
||||
<h3>Types of Fields</h3>
|
||||
<p>In Lucene, fields may be <i>stored</i>, in which case their text is stored
|
||||
in the index literally, in a non-inverted manner. Fields that are inverted are
|
||||
called <i>indexed</i>. A field may be both stored and indexed.</p>
|
||||
<p>The text of a field may be <i>tokenized</i> into terms to be indexed, or the
|
||||
text of a field may be used literally as a term to be indexed. Most fields are
|
||||
tokenized, but sometimes it is useful for certain identifier fields to be
|
||||
indexed literally.</p>
|
||||
<p>See the {@link org.apache.lucene.document.Field Field}
|
||||
java docs for more information on Fields.</p>
|
||||
<a name="Segments" id="Segments"></a>
|
||||
<h3>Segments</h3>
|
||||
<p>Lucene indexes may be composed of multiple sub-indexes, or <i>segments</i>.
|
||||
Each segment is a fully independent index, which could be searched separately.
|
||||
Indexes evolve by:</p>
|
||||
<ol>
|
||||
<li>Creating new segments for newly added documents.</li>
|
||||
<li>Merging existing segments.</li>
|
||||
</ol>
|
||||
<p>Searches may involve multiple segments and/or multiple indexes, each index
|
||||
potentially composed of a set of segments.</p>
|
||||
<a name="Document_Numbers"></a>
|
||||
<h3>Document Numbers</h3>
|
||||
<p>Internally, Lucene refers to documents by an integer <i>document number</i>.
|
||||
The first document added to an index is numbered zero, and each subsequent
|
||||
document added gets a number one greater than the previous.</p>
|
||||
<p>Note that a document's number may change, so caution should be taken when
|
||||
storing these numbers outside of Lucene. In particular, numbers may change in
|
||||
the following situations:</p>
|
||||
<ul>
|
||||
<li>
|
||||
<p>The numbers stored in each segment are unique only within the segment, and
|
||||
must be converted before they can be used in a larger context. The standard
|
||||
technique is to allocate each segment a range of values, based on the range of
|
||||
numbers used in that segment. To convert a document number from a segment to an
|
||||
external value, the segment's <i>base</i> document number is added. To convert
|
||||
an external value back to a segment-specific value, the segment is identified
|
||||
by the range that the external value is in, and the segment's base value is
|
||||
subtracted. For example two five document segments might be combined, so that
|
||||
the first segment has a base value of zero, and the second of five. Document
|
||||
three from the second segment would have an external value of eight.</p>
|
||||
</li>
|
||||
<li>
|
||||
<p>When documents are deleted, gaps are created in the numbering. These are
|
||||
eventually removed as the index evolves through merging. Deleted documents are
|
||||
dropped when segments are merged. A freshly-merged segment thus has no gaps in
|
||||
its numbering.</p>
|
||||
</li>
|
||||
</ul>
|
||||
</div>
|
||||
<a name="Overview" id="Overview"></a>
|
||||
<h2>Index Structure Overview</h2>
|
||||
<div>
|
||||
<p>Each segment index maintains the following:</p>
|
||||
<ul>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat Segment info}.
|
||||
This contains metadata about a segment, such as the number of documents,
|
||||
what files it uses,
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene42.Lucene42FieldInfosFormat Field names}.
|
||||
This contains the set of field names used in the index.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat Stored Field values}.
|
||||
This contains, for each document, a list of attribute-value pairs, where the attributes
|
||||
are field names. These are used to store auxiliary information about the document, such as
|
||||
its title, url, or an identifier to access a database. The set of stored fields are what is
|
||||
returned for each hit when searching. This is keyed by document number.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term dictionary}.
|
||||
A dictionary containing all of the terms used in all of the
|
||||
indexed fields of all of the documents. The dictionary also contains the number
|
||||
of documents which contain the term, and pointers to the term's frequency and
|
||||
proximity data.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Frequency data}.
|
||||
For each term in the dictionary, the numbers of all the
|
||||
documents that contain that term, and the frequency of the term in that
|
||||
document, unless frequencies are omitted (IndexOptions.DOCS_ONLY)
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Proximity data}.
|
||||
For each term in the dictionary, the positions that the
|
||||
term occurs in each document. Note that this will not exist if all fields in
|
||||
all documents omit position data.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene42.Lucene42NormsFormat Normalization factors}.
|
||||
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}.
|
||||
For each field in each document, the term vector (sometimes
|
||||
called document vector) may be stored. A term vector consists of term text and
|
||||
term frequency. To add Term Vectors to your index see the
|
||||
{@link org.apache.lucene.document.Field Field} constructors
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat Per-document values}.
|
||||
Like stored values, these are also keyed by document
|
||||
number, but are generally intended to be loaded into main memory for fast
|
||||
access. Whereas stored values are generally intended for summary results from
|
||||
searches, per-document values are useful for things like scoring factors.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat Deleted documents}.
|
||||
An optional file indicating which documents are deleted.
|
||||
</li>
|
||||
</ul>
|
||||
<p>Details on each of these are provided in their linked pages.</p>
|
||||
</div>
|
||||
<a name="File_Naming"></a>
|
||||
<h2>File Naming</h2>
|
||||
<div>
|
||||
<p>All files belonging to a segment have the same name with varying extensions.
|
||||
The extensions correspond to the different file formats described below. When
|
||||
using the Compound File format (default in 1.4 and greater) these files (except
|
||||
for the Segment info file, the Lock file, and Deleted documents file) are collapsed
|
||||
into a single .cfs file (see below for details)</p>
|
||||
<p>Typically, all segments in an index are stored in a single directory,
|
||||
although this is not required.</p>
|
||||
<p>As of version 2.1 (lock-less commits), file names are never re-used (there
|
||||
is one exception, "segments.gen", see below). That is, when any file is saved
|
||||
to the Directory it is given a never before used filename. This is achieved
|
||||
using a simple generations approach. For example, the first segments file is
|
||||
segments_1, then segments_2, etc. The generation is a sequential long integer
|
||||
represented in alpha-numeric (base 36) form.</p>
|
||||
</div>
|
||||
<a name="file-names" id="file-names"></a>
|
||||
<h2>Summary of File Extensions</h2>
|
||||
<div>
|
||||
<p>The following table summarizes the names and extensions of the files in
|
||||
Lucene:</p>
|
||||
<table cellspacing="1" cellpadding="4">
|
||||
<tr>
|
||||
<th>Name</th>
|
||||
<th>Extension</th>
|
||||
<th>Brief Description</th>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
|
||||
<td>segments.gen, segments_N</td>
|
||||
<td>Stores information about a commit point</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td><a href="#Lock_File">Lock File</a></td>
|
||||
<td>write.lock</td>
|
||||
<td>The Write lock prevents multiple IndexWriters from writing to the same
|
||||
file.</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat Segment Info}</td>
|
||||
<td>.si</td>
|
||||
<td>Stores metadata about a segment</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.store.CompoundFileDirectory Compound File}</td>
|
||||
<td>.cfs, .cfe</td>
|
||||
<td>An optional "virtual" file consisting of all the other index files for
|
||||
systems that frequently run out of file handles.</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42FieldInfosFormat Fields}</td>
|
||||
<td>.fnm</td>
|
||||
<td>Stores information about the fields</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat 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>.fdt</td>
|
||||
<td>The stored fields for documents</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Dictionary}</td>
|
||||
<td>.tim</td>
|
||||
<td>The term dictionary, stores term info</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Index}</td>
|
||||
<td>.tip</td>
|
||||
<td>The index into the Term Dictionary</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Frequencies}</td>
|
||||
<td>.doc</td>
|
||||
<td>Contains the list of docs which contain each term along with frequency</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Positions}</td>
|
||||
<td>.pos</td>
|
||||
<td>Stores position information about where a term occurs in the index</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Payloads}</td>
|
||||
<td>.pay</td>
|
||||
<td>Stores additional per-position metadata information such as character offsets and user payloads</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42NormsFormat Norms}</td>
|
||||
<td>.nvd, .nvm</td>
|
||||
<td>Encodes length and boost factors for docs and fields</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat Per-Document Values}</td>
|
||||
<td>.dvd, .dvm</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>.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>.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>.tvf</td>
|
||||
<td>The field level info about term vectors</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat Deleted Documents}</td>
|
||||
<td>.del</td>
|
||||
<td>Info about what files are deleted</td>
|
||||
</tr>
|
||||
</table>
|
||||
</div>
|
||||
<a name="Lock_File" id="Lock_File"></a>
|
||||
<h2>Lock File</h2>
|
||||
The write lock, which is stored in the index directory by default, is named
|
||||
"write.lock". If the lock directory is different from the index directory then
|
||||
the write lock will be named "XXXX-write.lock" where XXXX is a unique prefix
|
||||
derived from the full path to the index directory. When this file is present, a
|
||||
writer is currently modifying the index (adding or removing documents). This
|
||||
lock file ensures that only one writer is modifying the index at a time.</p>
|
||||
<a name="History"></a>
|
||||
<h2>History</h2>
|
||||
<p>Compatibility notes are provided in this document, describing how file
|
||||
formats have changed from prior versions:</p>
|
||||
<ul>
|
||||
<li>In version 2.1, the file format was changed to allow lock-less commits (ie,
|
||||
no more commit lock). The change is fully backwards compatible: you can open a
|
||||
pre-2.1 index for searching or adding/deleting of docs. When the new segments
|
||||
file is saved (committed), it will be written in the new file format (meaning
|
||||
no specific "upgrade" process is needed). But note that once a commit has
|
||||
occurred, pre-2.1 Lucene will not be able to read the index.</li>
|
||||
<li>In version 2.3, the file format was changed to allow segments to share a
|
||||
single set of doc store (vectors & stored fields) files. This allows for
|
||||
faster indexing in certain cases. The change is fully backwards compatible (in
|
||||
the same way as the lock-less commits change in 2.1).</li>
|
||||
<li>In version 2.4, Strings are now written as true UTF-8 byte sequence, not
|
||||
Java's modified UTF-8. See <a href="http://issues.apache.org/jira/browse/LUCENE-510">
|
||||
LUCENE-510</a> for details.</li>
|
||||
<li>In version 2.9, an optional opaque Map<String,String> CommitUserData
|
||||
may be passed to IndexWriter's commit methods (and later retrieved), which is
|
||||
recorded in the segments_N file. See <a href="http://issues.apache.org/jira/browse/LUCENE-1382">
|
||||
LUCENE-1382</a> for details. Also,
|
||||
diagnostics were added to each segment written recording details about why it
|
||||
was written (due to flush, merge; which OS/JRE was used; etc.). See issue
|
||||
<a href="http://issues.apache.org/jira/browse/LUCENE-1654">LUCENE-1654</a> for details.</li>
|
||||
<li>In version 3.0, compressed fields are no longer written to the index (they
|
||||
can still be read, but on merge the new segment will write them, uncompressed).
|
||||
See issue <a href="http://issues.apache.org/jira/browse/LUCENE-1960">LUCENE-1960</a>
|
||||
for details.</li>
|
||||
<li>In version 3.1, segments records the code version that created them. See
|
||||
<a href="http://issues.apache.org/jira/browse/LUCENE-2720">LUCENE-2720</a> for details.
|
||||
Additionally segments track explicitly whether or not they have term vectors.
|
||||
See <a href="http://issues.apache.org/jira/browse/LUCENE-2811">LUCENE-2811</a>
|
||||
for details.</li>
|
||||
<li>In version 3.2, numeric fields are written as natively to stored fields
|
||||
file, previously they were stored in text format only.</li>
|
||||
<li>In version 3.4, fields can omit position data while still indexing term
|
||||
frequencies.</li>
|
||||
<li>In version 4.0, the format of the inverted index became extensible via
|
||||
the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
|
||||
({@code DocValues}) was introduced. Normalization factors need no longer be a
|
||||
single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}.
|
||||
Terms need not be unicode strings, they can be any byte sequence. Term offsets
|
||||
can optionally be indexed into the postings lists. Payloads can be stored in the
|
||||
term vectors.</li>
|
||||
<li>In version 4.1, the format of the postings list changed to use either
|
||||
of FOR compression or variable-byte encoding, depending upon the frequency
|
||||
of the term. Terms appearing only once were changed to inline directly into
|
||||
the term dictionary. Stored fields are compressed by default. </li>
|
||||
<li>In version 4.2, term vectors are compressed by default. DocValues has
|
||||
a new multi-valued type (SortedSet), that can be used for faceting/grouping/joining
|
||||
on multi-valued fields.</li>
|
||||
<li>In version 4.5, DocValues were extended to explicitly represent missing values.</li>
|
||||
</ul>
|
||||
<a name="Limitations" id="Limitations"></a>
|
||||
<h2>Limitations</h2>
|
||||
<div>
|
||||
<p>Lucene uses a Java <code>int</code> to refer to
|
||||
document numbers, and the index file format uses an <code>Int32</code>
|
||||
on-disk to store document numbers. This is a limitation
|
||||
of both the index file format and the current implementation. Eventually these
|
||||
should be replaced with either <code>UInt64</code> values, or
|
||||
better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.</p>
|
||||
</div>
|
||||
</body>
|
||||
</html>
|
|
@ -61,9 +61,13 @@ name of your codec.
|
|||
If you just want to customise the {@link org.apache.lucene.codecs.PostingsFormat}, or use different postings
|
||||
formats for different fields, then you can register your custom postings format in the same way (in
|
||||
META-INF/services/org.apache.lucene.codecs.PostingsFormat), and then extend the default
|
||||
{@link org.apache.lucene.codecs.lucene42.Lucene42Codec} and override
|
||||
{@link org.apache.lucene.codecs.lucene42.Lucene42Codec#getPostingsFormatForField(String)} to return your custom
|
||||
{@link org.apache.lucene.codecs.lucene45.Lucene45Codec} and override
|
||||
{@link org.apache.lucene.codecs.lucene45.Lucene45Codec#getPostingsFormatForField(String)} to return your custom
|
||||
postings format.
|
||||
</p>
|
||||
<p>
|
||||
Similarly, if you just want to customise the {@link org.apache.lucene.codecs.DocValuesFormat} per-field, have
|
||||
a look at {@link org.apache.lucene.codecs.lucene45.Lucene45Codec#getDocValuesFormatForField(String)}.
|
||||
</p>
|
||||
</body>
|
||||
</html>
|
||||
|
|
|
@ -1410,7 +1410,7 @@ public class CheckIndex {
|
|||
private static void checkNumericDocValues(String fieldName, AtomicReader reader, NumericDocValues ndv, Bits docsWithField) {
|
||||
for (int i = 0; i < reader.maxDoc(); i++) {
|
||||
long value = ndv.get(i);
|
||||
if (docsWithField.get(i) == false && value > 0) {
|
||||
if (docsWithField.get(i) == false && value != 0) {
|
||||
throw new RuntimeException("dv for field: " + fieldName + " is marked missing but has value=" + value + " for doc: " + i);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,3 +16,4 @@
|
|||
org.apache.lucene.codecs.lucene40.Lucene40Codec
|
||||
org.apache.lucene.codecs.lucene41.Lucene41Codec
|
||||
org.apache.lucene.codecs.lucene42.Lucene42Codec
|
||||
org.apache.lucene.codecs.lucene45.Lucene45Codec
|
|
@ -14,3 +14,4 @@
|
|||
# limitations under the License.
|
||||
|
||||
org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat
|
||||
org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat
|
|
@ -17,21 +17,27 @@ package org.apache.lucene;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.analysis.*;
|
||||
import org.apache.lucene.codecs.*;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
|
||||
import org.apache.lucene.document.*;
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.search.*;
|
||||
import org.apache.lucene.store.*;
|
||||
import org.apache.lucene.util.*;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.store.BaseDirectoryWrapper;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
|
||||
/* Intentionally outside of oal.index to verify fully
|
||||
external codecs work fine */
|
||||
|
||||
public class TestExternalCodecs extends LuceneTestCase {
|
||||
|
||||
private static final class CustomPerFieldCodec extends Lucene42Codec {
|
||||
private static final class CustomPerFieldCodec extends Lucene45Codec {
|
||||
|
||||
private final PostingsFormat ramFormat = PostingsFormat.forName("RAMOnly");
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.codecs.diskdv;
|
||||
package org.apache.lucene.codecs.lucene45;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -18,15 +18,14 @@ package org.apache.lucene.codecs.diskdv;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.cheapbastard.CheapBastardDocValuesFormat;
|
||||
import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
|
||||
/**
|
||||
* Tests CheapBastardDocValuesFormat
|
||||
* Tests Lucene45DocValuesFormat
|
||||
*/
|
||||
public class TestCheapBastardDocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
|
||||
private final Codec codec = _TestUtil.alwaysDocValuesFormat(new CheapBastardDocValuesFormat());
|
||||
public class TestLucene45DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
|
||||
private final Codec codec = _TestUtil.alwaysDocValuesFormat(new Lucene45DocValuesFormat());
|
||||
|
||||
@Override
|
||||
protected Codec getCodec() {
|
|
@ -25,7 +25,7 @@ import org.apache.lucene.analysis.Analyzer;
|
|||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
|
||||
import org.apache.lucene.document.BinaryDocValuesField;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
|
@ -79,9 +79,9 @@ public class TestPerFieldDocValuesFormat extends BaseDocValuesFormatTestCase {
|
|||
Directory directory = newDirectory();
|
||||
// we don't use RandomIndexWriter because it might add more docvalues than we expect !!!!1
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
|
||||
final DocValuesFormat fast = DocValuesFormat.forName("Lucene42");
|
||||
final DocValuesFormat fast = DocValuesFormat.forName("Lucene45");
|
||||
final DocValuesFormat slow = DocValuesFormat.forName("SimpleText");
|
||||
iwc.setCodec(new Lucene42Codec() {
|
||||
iwc.setCodec(new Lucene45Codec() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
if ("dv1".equals(field)) {
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.io.IOException;
|
|||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
|
||||
import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
|
||||
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
|
||||
|
@ -200,7 +200,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
|
|||
|
||||
}
|
||||
|
||||
public static class MockCodec extends Lucene42Codec {
|
||||
public static class MockCodec extends Lucene45Codec {
|
||||
final PostingsFormat lucene40 = new Lucene41PostingsFormat();
|
||||
final PostingsFormat simpleText = new SimpleTextPostingsFormat();
|
||||
final PostingsFormat mockSep = new MockSepPostingsFormat();
|
||||
|
@ -217,7 +217,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public static class MockCodec2 extends Lucene42Codec {
|
||||
public static class MockCodec2 extends Lucene45Codec {
|
||||
final PostingsFormat lucene40 = new Lucene41PostingsFormat();
|
||||
final PostingsFormat simpleText = new SimpleTextPostingsFormat();
|
||||
|
||||
|
@ -268,7 +268,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testSameCodecDifferentInstance() throws Exception {
|
||||
Codec codec = new Lucene42Codec() {
|
||||
Codec codec = new Lucene45Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
if ("id".equals(field)) {
|
||||
|
@ -284,7 +284,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testSameCodecDifferentParams() throws Exception {
|
||||
Codec codec = new Lucene42Codec() {
|
||||
Codec codec = new Lucene45Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
if ("id".equals(field)) {
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
|
|||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
|
||||
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
|
@ -1060,7 +1060,7 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
aux2.close();
|
||||
}
|
||||
|
||||
private static final class CustomPerFieldCodec extends Lucene42Codec {
|
||||
private static final class CustomPerFieldCodec extends Lucene45Codec {
|
||||
private final PostingsFormat simpleTextFormat = PostingsFormat.forName("SimpleText");
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
|
||||
private final PostingsFormat mockSepFormat = PostingsFormat.forName("MockSep");
|
||||
|
@ -1111,7 +1111,7 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
|
||||
private static final class UnRegisteredCodec extends FilterCodec {
|
||||
public UnRegisteredCodec() {
|
||||
super("NotRegistered", new Lucene42Codec());
|
||||
super("NotRegistered", new Lucene45Codec());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
|
@ -41,7 +41,7 @@ public class TestAllFilesHaveCodecHeader extends LuceneTestCase {
|
|||
public void test() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||
conf.setCodec(new Lucene42Codec());
|
||||
conf.setCodec(new Lucene45Codec());
|
||||
// riw should sometimes create docvalues fields, etc
|
||||
RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
|
||||
Document doc = new Document();
|
||||
|
|
|
@ -49,7 +49,7 @@ public class TestDuelingCodecs extends LuceneTestCase {
|
|||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
|
||||
// for now its SimpleText vs Lucene42(random postings format)
|
||||
// for now its SimpleText vs Lucene45(random postings format)
|
||||
// as this gives the best overall coverage. when we have more
|
||||
// codecs we should probably pick 2 from Codec.availableCodecs()
|
||||
|
||||
|
|
|
@ -25,8 +25,8 @@ import org.apache.lucene.codecs.Codec;
|
|||
// enough to test the basics via Codec
|
||||
public class TestNamedSPILoader extends LuceneTestCase {
|
||||
public void testLookup() {
|
||||
Codec codec = Codec.forName("Lucene42");
|
||||
assertEquals("Lucene42", codec.getName());
|
||||
Codec codec = Codec.forName("Lucene45");
|
||||
assertEquals("Lucene45", codec.getName());
|
||||
}
|
||||
|
||||
// we want an exception if its not found.
|
||||
|
@ -39,6 +39,6 @@ public class TestNamedSPILoader extends LuceneTestCase {
|
|||
|
||||
public void testAvailableServices() {
|
||||
Set<String> codecs = Codec.availableCodecs();
|
||||
assertTrue(codecs.contains("Lucene42"));
|
||||
assertTrue(codecs.contains("Lucene45"));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.lucene.facet.params.FacetIndexingParams;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
// nocommit
|
||||
public class Facet42Codec extends Lucene42Codec {
|
||||
|
||||
private final Set<String> facetFields;
|
||||
|
|
|
@ -68,8 +68,10 @@ public class Facet42DocValuesConsumer extends DocValuesConsumer {
|
|||
|
||||
long totBytes = 0;
|
||||
for (BytesRef v : values) {
|
||||
if (v != null) {
|
||||
totBytes += v.length;
|
||||
}
|
||||
}
|
||||
|
||||
if (totBytes > Integer.MAX_VALUE) {
|
||||
throw new IllegalStateException("too many facets in one segment: Facet42DocValues cannot handle more than 2 GB facet data per segment");
|
||||
|
@ -78,16 +80,20 @@ public class Facet42DocValuesConsumer extends DocValuesConsumer {
|
|||
out.writeVInt((int) totBytes);
|
||||
|
||||
for (BytesRef v : values) {
|
||||
if (v != null) {
|
||||
out.writeBytes(v.bytes, v.offset, v.length);
|
||||
}
|
||||
}
|
||||
|
||||
PackedInts.Writer w = PackedInts.getWriter(out, maxDoc+1, PackedInts.bitsRequired(totBytes+1), acceptableOverheadRatio);
|
||||
|
||||
int address = 0;
|
||||
for(BytesRef v : values) {
|
||||
w.add(address);
|
||||
if (v != null) {
|
||||
address += v.length;
|
||||
}
|
||||
}
|
||||
w.add(address);
|
||||
w.finish();
|
||||
}
|
||||
|
|
|
@ -68,7 +68,7 @@ public class DoubleFieldSource extends FieldCacheSource {
|
|||
|
||||
@Override
|
||||
public boolean exists(int doc) {
|
||||
return valid.get(doc);
|
||||
return arr.get(doc) != 0 || valid.get(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -142,7 +142,7 @@ public class DoubleFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public void fillValue(int doc) {
|
||||
mval.value = arr.get(doc);
|
||||
mval.exists = valid.get(doc);
|
||||
mval.exists = mval.value != 0 || valid.get(doc);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -72,7 +72,7 @@ public class FloatFieldSource extends FieldCacheSource {
|
|||
|
||||
@Override
|
||||
public boolean exists(int doc) {
|
||||
return valid.get(doc);
|
||||
return arr.get(doc) != 0 || valid.get(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -88,7 +88,7 @@ public class FloatFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public void fillValue(int doc) {
|
||||
mval.value = arr.get(doc);
|
||||
mval.exists = valid.get(doc);
|
||||
mval.exists = mval.value != 0 || valid.get(doc);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -95,7 +95,7 @@ public class IntFieldSource extends FieldCacheSource {
|
|||
|
||||
@Override
|
||||
public boolean exists(int doc) {
|
||||
return valid.get(doc);
|
||||
return arr.get(doc) != 0 || valid.get(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -150,7 +150,7 @@ public class IntFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public void fillValue(int doc) {
|
||||
mval.value = arr.get(doc);
|
||||
mval.exists = valid.get(doc);
|
||||
mval.exists = mval.value != 0 || valid.get(doc);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -81,7 +81,7 @@ public class LongFieldSource extends FieldCacheSource {
|
|||
|
||||
@Override
|
||||
public boolean exists(int doc) {
|
||||
return valid.get(doc);
|
||||
return arr.get(doc) != 0 || valid.get(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -141,7 +141,7 @@ public class LongFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public void fillValue(int doc) {
|
||||
mval.value = arr.get(doc);
|
||||
mval.exists = valid.get(doc);
|
||||
mval.exists = mval.value != 0 || valid.get(doc);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -75,7 +75,7 @@
|
|||
<li><a href="SYSTEM_REQUIREMENTS.html">System Requirements</a>: Minimum and supported Java versions.</li>
|
||||
<li><a href="MIGRATE.html">Migration Guide</a>: What changed in Lucene 4; how to migrate code from Lucene 3.x.</li>
|
||||
<li><a href="JRE_VERSION_MIGRATION.html">JRE Version Migration</a>: Information about upgrading between major JRE versions.</li>
|
||||
<li><a href="core/org/apache/lucene/codecs/lucene42/package-summary.html#package_description">File Formats</a>: Guide to the supported index format used by Lucene. This can be customized by using <a href="core/org/apache/lucene/codecs/package-summary.html#package_description">an alternate codec</a>.</li>
|
||||
<li><a href="core/org/apache/lucene/codecs/lucene45/package-summary.html#package_description">File Formats</a>: Guide to the supported index format used by Lucene. This can be customized by using <a href="core/org/apache/lucene/codecs/package-summary.html#package_description">an alternate codec</a>.</li>
|
||||
<li><a href="core/org/apache/lucene/search/package-summary.html#package_description">Search and Scoring in Lucene</a>: Introduction to how Lucene scores documents.</li>
|
||||
<li><a href="core/org/apache/lucene/search/similarities/TFIDFSimilarity.html">Classic Scoring Formula</a>: Formula of Lucene's classic <a href="http://en.wikipedia.org/wiki/Vector_Space_Model">Vector Space</a> implementation. (look <a href="core/org/apache/lucene/search/similarities/package-summary.html#package_description">here</a> for other models)</li>
|
||||
<li><a href="queryparser/org/apache/lucene/queryparser/classic/package-summary.html#package_description">Classic QueryParser Syntax</a>: Overview of the Classic QueryParser's syntax and features.</li>
|
||||
|
|
|
@ -78,10 +78,12 @@ public class BBoxSimilarityValueSource extends ValueSource {
|
|||
|
||||
@Override
|
||||
public float floatVal(int doc) {
|
||||
double minXVal = minX.get(doc);
|
||||
double maxXVal = maxX.get(doc);
|
||||
// make sure it has minX and area
|
||||
if (validMinX.get(doc) && validMaxX.get(doc)) {
|
||||
if ((minXVal != 0 || validMinX.get(doc)) && (maxXVal != 0 || validMaxX.get(doc))) {
|
||||
rect.reset(
|
||||
minX.get(doc), maxX.get(doc),
|
||||
minXVal, maxXVal,
|
||||
minY.get(doc), maxY.get(doc));
|
||||
return (float) similarity.score(rect, null);
|
||||
} else {
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.apache.lucene.analysis.TokenStream;
|
|||
import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
|
||||
import org.apache.lucene.document.BinaryDocValuesField;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
|
@ -161,7 +161,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
|||
* codec to use. */
|
||||
protected IndexWriterConfig getIndexWriterConfig(Version matchVersion, Analyzer indexAnalyzer) {
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(matchVersion, indexAnalyzer);
|
||||
iwc.setCodec(new Lucene42Codec());
|
||||
iwc.setCodec(new Lucene45Codec());
|
||||
iwc.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
|
||||
return iwc;
|
||||
}
|
||||
|
|
|
@ -23,10 +23,10 @@ import org.apache.lucene.codecs.NormsFormat;
|
|||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
|
||||
|
||||
/**
|
||||
* Acts like {@link Lucene42Codec} but with additional asserts.
|
||||
* Acts like {@link Lucene45Codec} but with additional asserts.
|
||||
*/
|
||||
public final class AssertingCodec extends FilterCodec {
|
||||
|
||||
|
@ -37,7 +37,7 @@ public final class AssertingCodec extends FilterCodec {
|
|||
private final NormsFormat norms = new AssertingNormsFormat();
|
||||
|
||||
public AssertingCodec() {
|
||||
super("Asserting", new Lucene42Codec());
|
||||
super("Asserting", new Lucene45Codec());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -24,7 +24,7 @@ import java.util.NoSuchElementException;
|
|||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat;
|
||||
import org.apache.lucene.index.AssertingAtomicReader;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
|
@ -39,10 +39,10 @@ import org.apache.lucene.util.FixedBitSet;
|
|||
import org.apache.lucene.util.OpenBitSet;
|
||||
|
||||
/**
|
||||
* Just like {@link Lucene42DocValuesFormat} but with additional asserts.
|
||||
* Just like {@link Lucene45DocValuesFormat} but with additional asserts.
|
||||
*/
|
||||
public class AssertingDocValuesFormat extends DocValuesFormat {
|
||||
private final DocValuesFormat in = new Lucene42DocValuesFormat();
|
||||
private final DocValuesFormat in = new Lucene45DocValuesFormat();
|
||||
|
||||
public AssertingDocValuesFormat() {
|
||||
super("Asserting");
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
|
|||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
||||
// nocommit
|
||||
/**
|
||||
* Just like {@link Lucene42NormsFormat} but with additional asserts.
|
||||
*/
|
||||
|
|
|
@ -23,10 +23,12 @@ import org.apache.lucene.codecs.NormsFormat;
|
|||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
|
||||
import org.apache.lucene.codecs.diskdv.DiskNormsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
|
||||
|
||||
/** Codec that tries to use as little ram as possible because he spent all his money on beer */
|
||||
// TODO: better name :)
|
||||
|
@ -39,11 +41,11 @@ public class CheapBastardCodec extends FilterCodec {
|
|||
private final StoredFieldsFormat storedFields = new Lucene40StoredFieldsFormat();
|
||||
private final TermVectorsFormat termVectors = new Lucene40TermVectorsFormat();
|
||||
// these go to disk for all docvalues/norms datastructures
|
||||
private final DocValuesFormat docValues = new CheapBastardDocValuesFormat();
|
||||
private final NormsFormat norms = new CheapBastardNormsFormat();
|
||||
private final DocValuesFormat docValues = new DiskDocValuesFormat();
|
||||
private final NormsFormat norms = new DiskNormsFormat();
|
||||
|
||||
public CheapBastardCodec() {
|
||||
super("CheapBastard", new Lucene42Codec());
|
||||
super("CheapBastard", new Lucene45Codec());
|
||||
}
|
||||
|
||||
public PostingsFormat postingsFormat() {
|
||||
|
|
|
@ -1,74 +0,0 @@
|
|||
package org.apache.lucene.codecs.cheapbastard;
|
||||
|
||||
/*
|
||||
* 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.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer;
|
||||
import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* DocValues format that keeps everything on disk.
|
||||
* <p>
|
||||
* Internally there are only 2 field types:
|
||||
* <ul>
|
||||
* <li>BINARY: a big byte[].
|
||||
* <li>NUMERIC: packed ints
|
||||
* </ul>
|
||||
* SORTED is encoded as BINARY + NUMERIC
|
||||
* <p>
|
||||
* NOTE: Don't use this format in production (its not very efficient).
|
||||
* Most likely you would want some parts in RAM, other parts on disk.
|
||||
* <p>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class CheapBastardDocValuesFormat extends DocValuesFormat {
|
||||
|
||||
public CheapBastardDocValuesFormat() {
|
||||
super("CheapBastard");
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
return new DiskDocValuesConsumer(state, DiskDocValuesFormat.DATA_CODEC,
|
||||
DiskDocValuesFormat.DATA_EXTENSION,
|
||||
DiskDocValuesFormat.META_CODEC,
|
||||
DiskDocValuesFormat.META_EXTENSION) {
|
||||
// don't ever write an index, we dont want to use RAM :)
|
||||
@Override
|
||||
protected void addTermsDict(FieldInfo field, Iterable<BytesRef> values) throws IOException {
|
||||
addBinaryField(field, values);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
return new CheapBastardDocValuesProducer(state, DiskDocValuesFormat.DATA_CODEC,
|
||||
DiskDocValuesFormat.DATA_EXTENSION,
|
||||
DiskDocValuesFormat.META_CODEC,
|
||||
DiskDocValuesFormat.META_EXTENSION);
|
||||
}
|
||||
}
|
|
@ -1,444 +0,0 @@
|
|||
package org.apache.lucene.codecs.cheapbastard;
|
||||
|
||||
/*
|
||||
* 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.diskdv.DiskDocValuesConsumer.DELTA_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.GCD_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.TABLE_COMPRESSED;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.DocValuesProducer.SortedSetDocsWithField;
|
||||
import org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer;
|
||||
import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.packed.BlockPackedReader;
|
||||
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
class CheapBastardDocValuesProducer extends DocValuesProducer {
|
||||
private final Map<Integer,NumericEntry> numerics;
|
||||
private final Map<Integer,NumericEntry> ords;
|
||||
private final Map<Integer,NumericEntry> ordIndexes;
|
||||
private final Map<Integer,BinaryEntry> binaries;
|
||||
private final IndexInput data;
|
||||
private final int maxDoc;
|
||||
|
||||
CheapBastardDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
this.maxDoc = state.segmentInfo.getDocCount();
|
||||
// read in the entries from the metadata file.
|
||||
IndexInput in = state.directory.openInput(metaName, state.context);
|
||||
boolean success = false;
|
||||
final int version;
|
||||
try {
|
||||
version = CodecUtil.checkHeader(in, metaCodec,
|
||||
DiskDocValuesFormat.VERSION_CURRENT,
|
||||
DiskDocValuesFormat.VERSION_CURRENT);
|
||||
numerics = new HashMap<Integer,NumericEntry>();
|
||||
ords = new HashMap<Integer,NumericEntry>();
|
||||
ordIndexes = new HashMap<Integer,NumericEntry>();
|
||||
binaries = new HashMap<Integer,BinaryEntry>();
|
||||
readFields(in);
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(in);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(in);
|
||||
}
|
||||
}
|
||||
|
||||
success = false;
|
||||
try {
|
||||
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
|
||||
data = state.directory.openInput(dataName, state.context);
|
||||
final int version2 = CodecUtil.checkHeader(data, dataCodec,
|
||||
DiskDocValuesFormat.VERSION_CURRENT,
|
||||
DiskDocValuesFormat.VERSION_CURRENT);
|
||||
if (version != version2) {
|
||||
throw new CorruptIndexException("Versions mismatch");
|
||||
}
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(this.data);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private void readFields(IndexInput meta) throws IOException {
|
||||
int fieldNumber = meta.readVInt();
|
||||
while (fieldNumber != -1) {
|
||||
byte type = meta.readByte();
|
||||
if (type == DiskDocValuesFormat.NUMERIC) {
|
||||
numerics.put(fieldNumber, readNumericEntry(meta));
|
||||
} else if (type == DiskDocValuesFormat.BINARY) {
|
||||
BinaryEntry b = readBinaryEntry(meta);
|
||||
binaries.put(fieldNumber, b);
|
||||
} else if (type == DiskDocValuesFormat.SORTED) {
|
||||
// sorted = binary + numeric
|
||||
if (meta.readVInt() != fieldNumber) {
|
||||
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
if (meta.readByte() != DiskDocValuesFormat.BINARY) {
|
||||
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
BinaryEntry b = readBinaryEntry(meta);
|
||||
binaries.put(fieldNumber, b);
|
||||
|
||||
if (meta.readVInt() != fieldNumber) {
|
||||
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
|
||||
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
NumericEntry n = readNumericEntry(meta);
|
||||
ords.put(fieldNumber, n);
|
||||
} else if (type == DiskDocValuesFormat.SORTED_SET) {
|
||||
// sortedset = binary + numeric + ordIndex
|
||||
if (meta.readVInt() != fieldNumber) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
if (meta.readByte() != DiskDocValuesFormat.BINARY) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
BinaryEntry b = readBinaryEntry(meta);
|
||||
binaries.put(fieldNumber, b);
|
||||
|
||||
if (meta.readVInt() != fieldNumber) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
NumericEntry n1 = readNumericEntry(meta);
|
||||
ords.put(fieldNumber, n1);
|
||||
|
||||
if (meta.readVInt() != fieldNumber) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
|
||||
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
|
||||
}
|
||||
NumericEntry n2 = readNumericEntry(meta);
|
||||
ordIndexes.put(fieldNumber, n2);
|
||||
} else {
|
||||
throw new CorruptIndexException("invalid type: " + type + ", resource=" + meta);
|
||||
}
|
||||
fieldNumber = meta.readVInt();
|
||||
}
|
||||
}
|
||||
|
||||
static NumericEntry readNumericEntry(IndexInput meta) throws IOException {
|
||||
NumericEntry entry = new NumericEntry();
|
||||
entry.format = meta.readVInt();
|
||||
entry.packedIntsVersion = meta.readVInt();
|
||||
entry.offset = meta.readLong();
|
||||
entry.count = meta.readVLong();
|
||||
entry.blockSize = meta.readVInt();
|
||||
switch(entry.format) {
|
||||
case GCD_COMPRESSED:
|
||||
entry.minValue = meta.readLong();
|
||||
entry.gcd = meta.readLong();
|
||||
break;
|
||||
case TABLE_COMPRESSED:
|
||||
if (entry.count > Integer.MAX_VALUE) {
|
||||
throw new CorruptIndexException("Cannot use TABLE_COMPRESSED with more than MAX_VALUE values, input=" + meta);
|
||||
}
|
||||
final int uniqueValues = meta.readVInt();
|
||||
if (uniqueValues > 256) {
|
||||
throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, input=" + meta);
|
||||
}
|
||||
entry.table = new long[uniqueValues];
|
||||
for (int i = 0; i < uniqueValues; ++i) {
|
||||
entry.table[i] = meta.readLong();
|
||||
}
|
||||
break;
|
||||
case DELTA_COMPRESSED:
|
||||
break;
|
||||
default:
|
||||
throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
|
||||
}
|
||||
return entry;
|
||||
}
|
||||
|
||||
static BinaryEntry readBinaryEntry(IndexInput meta) throws IOException {
|
||||
BinaryEntry entry = new BinaryEntry();
|
||||
int format = meta.readVInt();
|
||||
if (format != DiskDocValuesConsumer.BINARY_FIXED_UNCOMPRESSED && format != DiskDocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED) {
|
||||
throw new CorruptIndexException("Unexpected format for binary entry: " + format + ", input=" + meta);
|
||||
}
|
||||
entry.minLength = meta.readVInt();
|
||||
entry.maxLength = meta.readVInt();
|
||||
entry.count = meta.readVLong();
|
||||
entry.offset = meta.readLong();
|
||||
if (entry.minLength != entry.maxLength) {
|
||||
entry.addressesOffset = meta.readLong();
|
||||
entry.packedIntsVersion = meta.readVInt();
|
||||
entry.blockSize = meta.readVInt();
|
||||
}
|
||||
return entry;
|
||||
}
|
||||
|
||||
@Override
|
||||
public NumericDocValues getNumeric(FieldInfo field) throws IOException {
|
||||
NumericEntry entry = numerics.get(field.number);
|
||||
return getNumeric(field, entry);
|
||||
}
|
||||
|
||||
private LongNumericDocValues getNumeric(FieldInfo field, final NumericEntry entry) throws IOException {
|
||||
final IndexInput data = this.data.clone();
|
||||
data.seek(entry.offset);
|
||||
|
||||
switch (entry.format) {
|
||||
case DELTA_COMPRESSED:
|
||||
final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
|
||||
return new LongNumericDocValues() {
|
||||
@Override
|
||||
public long get(long id) {
|
||||
return reader.get(id);
|
||||
}
|
||||
};
|
||||
case GCD_COMPRESSED:
|
||||
final long min = entry.minValue;
|
||||
final long mult = entry.gcd;
|
||||
final BlockPackedReader quotientReader = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
|
||||
return new LongNumericDocValues() {
|
||||
@Override
|
||||
public long get(long id) {
|
||||
return min + mult * quotientReader.get(id);
|
||||
}
|
||||
};
|
||||
case TABLE_COMPRESSED:
|
||||
final long[] table = entry.table;
|
||||
final int bitsRequired = PackedInts.bitsRequired(table.length - 1);
|
||||
final PackedInts.Reader ords = PackedInts.getDirectReaderNoHeader(data, PackedInts.Format.PACKED, entry.packedIntsVersion, (int) entry.count, bitsRequired);
|
||||
return new LongNumericDocValues() {
|
||||
@Override
|
||||
long get(long id) {
|
||||
return table[(int) ords.get((int) id)];
|
||||
}
|
||||
};
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public BinaryDocValues getBinary(FieldInfo field) throws IOException {
|
||||
BinaryEntry bytes = binaries.get(field.number);
|
||||
if (bytes.minLength == bytes.maxLength) {
|
||||
return getFixedBinary(field, bytes);
|
||||
} else {
|
||||
return getVariableBinary(field, bytes);
|
||||
}
|
||||
}
|
||||
|
||||
private BinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) {
|
||||
final IndexInput data = this.data.clone();
|
||||
|
||||
return new LongBinaryDocValues() {
|
||||
@Override
|
||||
public void get(long id, BytesRef result) {
|
||||
long address = bytes.offset + id * bytes.maxLength;
|
||||
try {
|
||||
data.seek(address);
|
||||
// NOTE: we could have one buffer, but various consumers (e.g. FieldComparatorSource)
|
||||
// assume "they" own the bytes after calling this!
|
||||
final byte[] buffer = new byte[bytes.maxLength];
|
||||
data.readBytes(buffer, 0, buffer.length);
|
||||
result.bytes = buffer;
|
||||
result.offset = 0;
|
||||
result.length = buffer.length;
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private BinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
|
||||
final IndexInput data = this.data.clone();
|
||||
data.seek(bytes.addressesOffset);
|
||||
|
||||
final MonotonicBlockPackedReader addresses = new MonotonicBlockPackedReader(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count, true);
|
||||
return new LongBinaryDocValues() {
|
||||
@Override
|
||||
public void get(long id, BytesRef result) {
|
||||
long startAddress = bytes.offset + (id == 0 ? 0 : + addresses.get(id-1));
|
||||
long endAddress = bytes.offset + addresses.get(id);
|
||||
int length = (int) (endAddress - startAddress);
|
||||
try {
|
||||
data.seek(startAddress);
|
||||
// NOTE: we could have one buffer, but various consumers (e.g. FieldComparatorSource)
|
||||
// assume "they" own the bytes after calling this!
|
||||
final byte[] buffer = new byte[length];
|
||||
data.readBytes(buffer, 0, buffer.length);
|
||||
result.bytes = buffer;
|
||||
result.offset = 0;
|
||||
result.length = length;
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedDocValues getSorted(FieldInfo field) throws IOException {
|
||||
final int valueCount = (int) binaries.get(field.number).count;
|
||||
final BinaryDocValues binary = getBinary(field);
|
||||
final NumericDocValues ordinals = getNumeric(field, ords.get(field.number));
|
||||
return new SortedDocValues() {
|
||||
|
||||
@Override
|
||||
public int getOrd(int docID) {
|
||||
return (int) ordinals.get(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void lookupOrd(int ord, BytesRef result) {
|
||||
binary.get(ord, result);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCount() {
|
||||
return valueCount;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
|
||||
final long valueCount = binaries.get(field.number).count;
|
||||
final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
|
||||
final LongNumericDocValues ordinals = getNumeric(field, ords.get(field.number));
|
||||
NumericEntry entry = ordIndexes.get(field.number);
|
||||
IndexInput data = this.data.clone();
|
||||
data.seek(entry.offset);
|
||||
final MonotonicBlockPackedReader ordIndex = new MonotonicBlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
|
||||
|
||||
return new SortedSetDocValues() {
|
||||
long offset;
|
||||
long endOffset;
|
||||
|
||||
@Override
|
||||
public long nextOrd() {
|
||||
if (offset == endOffset) {
|
||||
return NO_MORE_ORDS;
|
||||
} else {
|
||||
long ord = ordinals.get(offset);
|
||||
offset++;
|
||||
return ord;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setDocument(int docID) {
|
||||
offset = (docID == 0 ? 0 : ordIndex.get(docID-1));
|
||||
endOffset = ordIndex.get(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void lookupOrd(long ord, BytesRef result) {
|
||||
binary.get(ord, result);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getValueCount() {
|
||||
return valueCount;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
if (field.getDocValuesType() == FieldInfo.DocValuesType.SORTED_SET) {
|
||||
return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
|
||||
} else {
|
||||
return new Bits.MatchAllBits(maxDoc);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
data.close();
|
||||
}
|
||||
|
||||
static class NumericEntry {
|
||||
long offset;
|
||||
|
||||
int format;
|
||||
int packedIntsVersion;
|
||||
long count;
|
||||
int blockSize;
|
||||
|
||||
long minValue;
|
||||
long gcd;
|
||||
long table[];
|
||||
}
|
||||
|
||||
static class BinaryEntry {
|
||||
long offset;
|
||||
|
||||
long count;
|
||||
int minLength;
|
||||
int maxLength;
|
||||
long addressesOffset;
|
||||
int packedIntsVersion;
|
||||
int blockSize;
|
||||
}
|
||||
|
||||
// internally we compose complex dv (sorted/sortedset) from other ones
|
||||
static abstract class LongNumericDocValues extends NumericDocValues {
|
||||
@Override
|
||||
public final long get(int docID) {
|
||||
return get((long) docID);
|
||||
}
|
||||
|
||||
abstract long get(long id);
|
||||
}
|
||||
|
||||
static abstract class LongBinaryDocValues extends BinaryDocValues {
|
||||
@Override
|
||||
public final void get(int docID, BytesRef result) {
|
||||
get((long)docID, result);
|
||||
}
|
||||
|
||||
abstract void get(long id, BytesRef Result);
|
||||
}
|
||||
}
|
|
@ -23,13 +23,13 @@ import org.apache.lucene.codecs.FilterCodec;
|
|||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.compressing.dummy.DummyCompressingCodec;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomInts;
|
||||
|
||||
/**
|
||||
* A codec that uses {@link CompressingStoredFieldsFormat} for its stored
|
||||
* fields and delegates to {@link Lucene42Codec} for everything else.
|
||||
* fields and delegates to {@link Lucene45Codec} for everything else.
|
||||
*/
|
||||
public abstract class CompressingCodec extends FilterCodec {
|
||||
|
||||
|
@ -73,7 +73,7 @@ public abstract class CompressingCodec extends FilterCodec {
|
|||
* Creates a compressing codec with a given segment suffix
|
||||
*/
|
||||
public CompressingCodec(String name, String segmentSuffix, CompressionMode compressionMode, int chunkSize) {
|
||||
super(name, new Lucene42Codec());
|
||||
super(name, new Lucene45Codec());
|
||||
this.storedFieldsFormat = new CompressingStoredFieldsFormat(name, segmentSuffix, compressionMode, chunkSize);
|
||||
this.termVectorsFormat = new CompressingTermVectorsFormat(name, segmentSuffix, compressionMode, chunkSize);
|
||||
}
|
||||
|
|
|
@ -2,8 +2,8 @@ package org.apache.lucene.codecs.compressing;
|
|||
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat; // nocommit
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat; // nocommit
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/*
|
||||
|
|
|
@ -6,6 +6,7 @@ import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
|
|||
import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
// nocommit
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
|
|
|
@ -21,6 +21,7 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
// nocommit
|
||||
/** CompressionCodec that uses {@link CompressionMode#HIGH_COMPRESSION} */
|
||||
public class HighCompressionCompressingCodec extends CompressingCodec {
|
||||
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
|
|||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.compressing.CompressingCodec;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
|
||||
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.DoubleField;
|
||||
|
@ -502,7 +502,7 @@ public abstract class BaseStoredFieldsFormatTestCase extends LuceneTestCase {
|
|||
// get another codec, other than the default: so we are merging segments across different codecs
|
||||
final Codec otherCodec;
|
||||
if ("SimpleText".equals(Codec.getDefault().getName())) {
|
||||
otherCodec = new Lucene42Codec();
|
||||
otherCodec = new Lucene45Codec();
|
||||
} else {
|
||||
otherCodec = new SimpleTextCodec();
|
||||
}
|
||||
|
|
|
@ -35,10 +35,9 @@ import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
|
|||
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
|
||||
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapDocFreqInterval;
|
||||
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat;
|
||||
import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
|
||||
import org.apache.lucene.codecs.cheapbastard.CheapBastardDocValuesFormat;
|
||||
import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
|
||||
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
|
||||
|
@ -62,7 +61,7 @@ import org.apache.lucene.util._TestUtil;
|
|||
* documents in different orders and the test will still be deterministic
|
||||
* and reproducable.
|
||||
*/
|
||||
public class RandomCodec extends Lucene42Codec {
|
||||
public class RandomCodec extends Lucene45Codec {
|
||||
/** Shuffled list of postings formats to use for new mappings */
|
||||
private List<PostingsFormat> formats = new ArrayList<PostingsFormat>();
|
||||
|
||||
|
@ -148,11 +147,10 @@ public class RandomCodec extends Lucene42Codec {
|
|||
new MemoryPostingsFormat(false, random.nextFloat()));
|
||||
|
||||
addDocValues(avoidCodecs,
|
||||
new Lucene42DocValuesFormat(),
|
||||
new Lucene45DocValuesFormat(),
|
||||
new DiskDocValuesFormat(),
|
||||
new SimpleTextDocValuesFormat(),
|
||||
new AssertingDocValuesFormat(),
|
||||
new CheapBastardDocValuesFormat());
|
||||
new AssertingDocValuesFormat());
|
||||
|
||||
Collections.shuffle(formats, random);
|
||||
Collections.shuffle(dvFormats, random);
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.lucene.codecs.lucene40.Lucene40RWCodec;
|
|||
import org.apache.lucene.codecs.lucene40.Lucene40RWPostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41RWCodec;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
|
||||
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
|
||||
import org.apache.lucene.index.RandomCodec;
|
||||
import org.apache.lucene.search.RandomSimilarityProvider;
|
||||
|
@ -146,6 +147,7 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
|
|||
|
||||
savedCodec = Codec.getDefault();
|
||||
int randomVal = random.nextInt(10);
|
||||
// nocommit: 4.2 impersonator
|
||||
if ("Lucene40".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
|
||||
"random".equals(TEST_POSTINGSFORMAT) &&
|
||||
"random".equals(TEST_DOCVALUESFORMAT) &&
|
||||
|
@ -182,7 +184,7 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
|
|||
dvFormat = DocValuesFormat.forName(TEST_DOCVALUESFORMAT);
|
||||
}
|
||||
|
||||
codec = new Lucene42Codec() {
|
||||
codec = new Lucene45Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return format;
|
||||
|
|
|
@ -44,7 +44,7 @@ import java.util.zip.ZipFile;
|
|||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
import org.apache.lucene.document.BinaryDocValuesField;
|
||||
|
@ -703,7 +703,7 @@ public class _TestUtil {
|
|||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("forcing postings format to:" + format);
|
||||
}
|
||||
return new Lucene42Codec() {
|
||||
return new Lucene45Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return format;
|
||||
|
@ -721,7 +721,7 @@ public class _TestUtil {
|
|||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("forcing docvalues format to:" + format);
|
||||
}
|
||||
return new Lucene42Codec() {
|
||||
return new Lucene45Codec() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return format;
|
||||
|
@ -757,9 +757,12 @@ public class _TestUtil {
|
|||
}
|
||||
}
|
||||
|
||||
// nocommit: remove this, push this test to Lucene40/Lucene42 codec tests
|
||||
public static boolean fieldSupportsHugeBinaryDocValues(String field) {
|
||||
String dvFormat = getDocValuesFormat(field);
|
||||
return dvFormat.equals("CheapBastard") ||
|
||||
System.out.println(dvFormat);
|
||||
return dvFormat.equals("Lucene45") ||
|
||||
dvFormat.equals("Asserting") ||
|
||||
dvFormat.equals("Disk") ||
|
||||
dvFormat.equals("SimpleText");
|
||||
}
|
||||
|
|
|
@ -14,4 +14,3 @@
|
|||
# limitations under the License.
|
||||
|
||||
org.apache.lucene.codecs.asserting.AssertingDocValuesFormat
|
||||
org.apache.lucene.codecs.cheapbastard.CheapBastardDocValuesFormat
|
|
@ -3,7 +3,7 @@ package org.apache.solr.core;
|
|||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.schema.SchemaField;
|
||||
import org.apache.solr.util.plugin.SolrCoreAware;
|
||||
|
@ -51,7 +51,7 @@ public class SchemaCodecFactory extends CodecFactory implements SolrCoreAware {
|
|||
@Override
|
||||
public void init(NamedList args) {
|
||||
super.init(args);
|
||||
codec = new Lucene42Codec() {
|
||||
codec = new Lucene45Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
final SchemaField fieldOrNull = core.getLatestSchema().getFieldOrNull(field);
|
||||
|
|
|
@ -190,8 +190,9 @@ final class NumericFacets {
|
|||
}
|
||||
docsWithField = FieldCache.DEFAULT.getDocsWithField(ctx.reader(), fieldName);
|
||||
}
|
||||
if (docsWithField.get(doc - ctx.docBase)) {
|
||||
hashTable.add(doc, longs.get(doc - ctx.docBase), 1);
|
||||
long v = longs.get(doc - ctx.docBase);
|
||||
if (v != 0 || docsWithField.get(doc - ctx.docBase)) {
|
||||
hashTable.add(doc, v, 1);
|
||||
} else {
|
||||
++missingCount;
|
||||
}
|
||||
|
|
|
@ -22,7 +22,7 @@
|
|||
<fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene41"/>
|
||||
|
||||
<fieldType name="string_disk" class="solr.StrField" docValuesFormat="Disk" />
|
||||
<fieldType name="string_memory" class="solr.StrField" docValuesFormat="Lucene42" />
|
||||
<fieldType name="string_memory" class="solr.StrField" docValuesFormat="Lucene45" />
|
||||
|
||||
<fieldType name="string" class="solr.StrField" />
|
||||
|
||||
|
|
|
@ -55,10 +55,10 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
|
|||
PerFieldDocValuesFormat format = (PerFieldDocValuesFormat) codec.docValuesFormat();
|
||||
assertEquals("Disk", format.getDocValuesFormatForField(schemaField.getName()).getName());
|
||||
schemaField = fields.get("string_memory_f");
|
||||
assertEquals("Lucene42",
|
||||
assertEquals("Lucene45",
|
||||
format.getDocValuesFormatForField(schemaField.getName()).getName());
|
||||
schemaField = fields.get("string_f");
|
||||
assertEquals("Lucene42",
|
||||
assertEquals("Lucene45",
|
||||
format.getDocValuesFormatForField(schemaField.getName()).getName());
|
||||
}
|
||||
|
||||
|
@ -80,8 +80,8 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
|
|||
|
||||
assertEquals("Disk", format.getDocValuesFormatForField("foo_disk").getName());
|
||||
assertEquals("Disk", format.getDocValuesFormatForField("bar_disk").getName());
|
||||
assertEquals("Lucene42", format.getDocValuesFormatForField("foo_memory").getName());
|
||||
assertEquals("Lucene42", format.getDocValuesFormatForField("bar_memory").getName());
|
||||
assertEquals("Lucene45", format.getDocValuesFormatForField("foo_memory").getName());
|
||||
assertEquals("Lucene45", format.getDocValuesFormatForField("bar_memory").getName());
|
||||
}
|
||||
|
||||
public void testUnknownField() {
|
||||
|
|
Loading…
Reference in New Issue