mirror of https://github.com/apache/lucene.git
LUCENE-5178: add missing support for docvalues
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1515977 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
39dbc26716
|
@ -89,6 +89,18 @@ New features
|
|||
http://blog.mikemccandless.com/2013/08/suggeststopfilter-carefully-removes.html
|
||||
for details.
|
||||
|
||||
* LUCENE-5178: Added support for missing values to DocValues fields.
|
||||
AtomicReader.getDocsWithField returns a Bits of documents with a value,
|
||||
and FieldCache.getDocsWithField forwards to that for DocValues fields. Things like
|
||||
SortField.setMissingValue, FunctionValues.exists, and FieldValueFilter now
|
||||
work with DocValues fields. (Robert Muir)
|
||||
|
||||
* LUCENE-5124: Lucene 4.5 has a new Lucene45Codec with Lucene45DocValues,
|
||||
supporting missing values and with most datastructures residing off-heap.
|
||||
Added "Memory" docvalues format that works entirely in heap, and "Disk"
|
||||
loads no datastructures into RAM. Both of these also support missing values.
|
||||
Added DiskNormsFormat (in case you want norms entirely on disk). (Robert Muir)
|
||||
|
||||
Bug Fixes
|
||||
|
||||
* LUCENE-5116: IndexWriter.addIndexes(IndexReader...) should drop empty (or all
|
||||
|
@ -184,6 +196,15 @@ API Changes
|
|||
* LUCENE-5173: Lucene never writes segments with 0 documents anymore.
|
||||
(Shai Erera, Uwe Schindler, Robert Muir)
|
||||
|
||||
* LUCENE-5178: SortedDocValues always returns -1 ord when a document is missing
|
||||
a value for the field. Previously it only did this if the SortedDocValues
|
||||
was produced by uninversion on the FieldCache. (Robert Muir)
|
||||
|
||||
Changes in Runtime Behavior
|
||||
|
||||
* LUCENE-5178: DocValues codec consumer APIs (iterables) return null values
|
||||
when the document has no value for the field. (Robert Muir)
|
||||
|
||||
Optimizations
|
||||
|
||||
* LUCENE-5088: Added TermFilter to filter docs by a specific term.
|
||||
|
|
|
@ -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,726 +17,34 @@ 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 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.codecs.lucene45.Lucene45DocValuesProducer;
|
||||
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 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 IndexInput data;
|
||||
class DiskDocValuesProducer extends Lucene45DocValuesProducer {
|
||||
|
||||
// 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);
|
||||
// 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, 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,
|
||||
DiskDocValuesFormat.VERSION_CURRENT,
|
||||
DiskDocValuesFormat.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 == 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 (resource=" + meta + ")");
|
||||
}
|
||||
if (meta.readByte() != DiskDocValuesFormat.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() != DiskDocValuesFormat.NUMERIC) {
|
||||
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
|
||||
}
|
||||
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 (resource=" + meta + ")");
|
||||
}
|
||||
if (meta.readByte() != DiskDocValuesFormat.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() != DiskDocValuesFormat.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() != DiskDocValuesFormat.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;
|
||||
super(state, dataCodec, dataExtension, metaCodec, metaExtension);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NumericDocValues getNumeric(FieldInfo field) throws IOException {
|
||||
NumericEntry entry = numerics.get(field.number);
|
||||
return getNumeric(entry);
|
||||
protected MonotonicBlockPackedReader getAddressInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
|
||||
data.seek(bytes.addressesOffset);
|
||||
return new MonotonicBlockPackedReader(data.clone(), bytes.packedIntsVersion, bytes.blockSize, bytes.count, true);
|
||||
}
|
||||
|
||||
LongNumericDocValues getNumeric(NumericEntry entry) throws IOException {
|
||||
final IndexInput data = this.data.clone();
|
||||
|
||||
@Override
|
||||
protected MonotonicBlockPackedReader getIntervalInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
|
||||
throw new AssertionError();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MonotonicBlockPackedReader getOrdIndexInstance(IndexInput data, FieldInfo field, NumericEntry entry) throws IOException {
|
||||
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 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();
|
||||
}
|
||||
};
|
||||
}
|
||||
return new MonotonicBlockPackedReader(data.clone(), entry.packedIntsVersion, entry.blockSize, entry.count, true);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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";
|
||||
}
|
|
@ -0,0 +1,403 @@
|
|||
package org.apache.lucene.codecs.memory;
|
||||
|
||||
/*
|
||||
* 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 java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.store.ByteArrayDataOutput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.MathUtil;
|
||||
import org.apache.lucene.util.fst.Builder;
|
||||
import org.apache.lucene.util.fst.FST.INPUT_TYPE;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
import org.apache.lucene.util.fst.PositiveIntOutputs;
|
||||
import org.apache.lucene.util.fst.Util;
|
||||
import org.apache.lucene.util.packed.BlockPackedWriter;
|
||||
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
|
||||
import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.VERSION_CURRENT;
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.BLOCK_SIZE;
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.BYTES;
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.NUMBER;
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.FST;
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.DELTA_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.GCD_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.TABLE_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.UNCOMPRESSED;
|
||||
|
||||
/**
|
||||
* Writer for {@link MemoryDocValuesFormat}
|
||||
*/
|
||||
class MemoryDocValuesConsumer extends DocValuesConsumer {
|
||||
final IndexOutput data, meta;
|
||||
final int maxDoc;
|
||||
final float acceptableOverheadRatio;
|
||||
|
||||
MemoryDocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension, float acceptableOverheadRatio) throws IOException {
|
||||
this.acceptableOverheadRatio = acceptableOverheadRatio;
|
||||
maxDoc = state.segmentInfo.getDocCount();
|
||||
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, VERSION_CURRENT);
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
meta = state.directory.createOutput(metaName, state.context);
|
||||
CodecUtil.writeHeader(meta, metaCodec, VERSION_CURRENT);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(this);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
|
||||
addNumericField(field, values, true);
|
||||
}
|
||||
|
||||
void addNumericField(FieldInfo field, Iterable<Number> values, boolean optimizeStorage) throws IOException {
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(NUMBER);
|
||||
meta.writeLong(data.getFilePointer());
|
||||
long minValue = Long.MAX_VALUE;
|
||||
long maxValue = Long.MIN_VALUE;
|
||||
long gcd = 0;
|
||||
boolean missing = false;
|
||||
// TODO: more efficient?
|
||||
HashSet<Long> uniqueValues = null;
|
||||
if (optimizeStorage) {
|
||||
uniqueValues = new HashSet<>();
|
||||
|
||||
long count = 0;
|
||||
for (Number nv : values) {
|
||||
final long v;
|
||||
if (nv == null) {
|
||||
v = 0;
|
||||
missing = true;
|
||||
} else {
|
||||
v = nv.longValue();
|
||||
}
|
||||
|
||||
if (gcd != 1) {
|
||||
if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
|
||||
// in that case v - minValue might overflow and make the GCD computation return
|
||||
// wrong results. Since these extreme values are unlikely, we just discard
|
||||
// GCD computation for them
|
||||
gcd = 1;
|
||||
} else if (count != 0) { // minValue needs to be set first
|
||||
gcd = MathUtil.gcd(gcd, v - minValue);
|
||||
}
|
||||
}
|
||||
|
||||
minValue = Math.min(minValue, v);
|
||||
maxValue = Math.max(maxValue, v);
|
||||
|
||||
if (uniqueValues != null) {
|
||||
if (uniqueValues.add(v)) {
|
||||
if (uniqueValues.size() > 256) {
|
||||
uniqueValues = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
++count;
|
||||
}
|
||||
assert count == maxDoc;
|
||||
}
|
||||
|
||||
if (missing) {
|
||||
long start = data.getFilePointer();
|
||||
writeMissingBitset(values);
|
||||
meta.writeLong(start);
|
||||
meta.writeLong(data.getFilePointer() - start);
|
||||
} else {
|
||||
meta.writeLong(-1L);
|
||||
}
|
||||
|
||||
if (uniqueValues != null) {
|
||||
// small number of unique values
|
||||
final int bitsPerValue = PackedInts.bitsRequired(uniqueValues.size()-1);
|
||||
FormatAndBits formatAndBits = PackedInts.fastestFormatAndBits(maxDoc, bitsPerValue, acceptableOverheadRatio);
|
||||
if (formatAndBits.bitsPerValue == 8 && minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) {
|
||||
meta.writeByte(UNCOMPRESSED); // uncompressed
|
||||
for (Number nv : values) {
|
||||
data.writeByte(nv == null ? 0 : (byte) nv.longValue());
|
||||
}
|
||||
} else {
|
||||
meta.writeByte(TABLE_COMPRESSED); // table-compressed
|
||||
Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]);
|
||||
final HashMap<Long,Integer> encode = new HashMap<Long,Integer>();
|
||||
data.writeVInt(decode.length);
|
||||
for (int i = 0; i < decode.length; i++) {
|
||||
data.writeLong(decode[i]);
|
||||
encode.put(decode[i], i);
|
||||
}
|
||||
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
data.writeVInt(formatAndBits.format.getId());
|
||||
data.writeVInt(formatAndBits.bitsPerValue);
|
||||
|
||||
final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, formatAndBits.format, maxDoc, formatAndBits.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
|
||||
for(Number nv : values) {
|
||||
writer.add(encode.get(nv == null ? 0 : nv.longValue()));
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
} else if (gcd != 0 && gcd != 1) {
|
||||
meta.writeByte(GCD_COMPRESSED);
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
data.writeLong(minValue);
|
||||
data.writeLong(gcd);
|
||||
data.writeVInt(BLOCK_SIZE);
|
||||
|
||||
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
|
||||
for (Number nv : values) {
|
||||
long value = nv == null ? 0 : nv.longValue();
|
||||
writer.add((value - minValue) / gcd);
|
||||
}
|
||||
writer.finish();
|
||||
} else {
|
||||
meta.writeByte(DELTA_COMPRESSED); // delta-compressed
|
||||
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
data.writeVInt(BLOCK_SIZE);
|
||||
|
||||
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
|
||||
for (Number nv : values) {
|
||||
writer.add(nv == null ? 0 : nv.longValue());
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
boolean success = false;
|
||||
try {
|
||||
if (meta != null) {
|
||||
meta.writeVInt(-1); // write EOF marker
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(data, meta);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(data, meta);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addBinaryField(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
|
||||
// write the byte[] data
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(BYTES);
|
||||
int minLength = Integer.MAX_VALUE;
|
||||
int maxLength = Integer.MIN_VALUE;
|
||||
final long startFP = data.getFilePointer();
|
||||
boolean missing = false;
|
||||
for(BytesRef v : values) {
|
||||
final int length;
|
||||
if (v == null) {
|
||||
length = 0;
|
||||
missing = true;
|
||||
} else {
|
||||
length = v.length;
|
||||
}
|
||||
if (length > MemoryDocValuesFormat.MAX_BINARY_FIELD_LENGTH) {
|
||||
throw new IllegalArgumentException("DocValuesField \"" + field.name + "\" is too large, must be <= " + MemoryDocValuesFormat.MAX_BINARY_FIELD_LENGTH);
|
||||
}
|
||||
minLength = Math.min(minLength, length);
|
||||
maxLength = Math.max(maxLength, length);
|
||||
if (v != null) {
|
||||
data.writeBytes(v.bytes, v.offset, v.length);
|
||||
}
|
||||
}
|
||||
meta.writeLong(startFP);
|
||||
meta.writeLong(data.getFilePointer() - startFP);
|
||||
if (missing) {
|
||||
long start = data.getFilePointer();
|
||||
writeMissingBitset(values);
|
||||
meta.writeLong(start);
|
||||
meta.writeLong(data.getFilePointer() - start);
|
||||
} else {
|
||||
meta.writeLong(-1L);
|
||||
}
|
||||
meta.writeVInt(minLength);
|
||||
meta.writeVInt(maxLength);
|
||||
|
||||
// if minLength == maxLength, its a fixed-length byte[], we are done (the addresses are implicit)
|
||||
// otherwise, we need to record the length fields...
|
||||
if (minLength != maxLength) {
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
meta.writeVInt(BLOCK_SIZE);
|
||||
|
||||
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
|
||||
long addr = 0;
|
||||
for (BytesRef v : values) {
|
||||
if (v != null) {
|
||||
addr += v.length;
|
||||
}
|
||||
writer.add(addr);
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
}
|
||||
|
||||
private void writeFST(FieldInfo field, Iterable<BytesRef> values) throws IOException {
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(FST);
|
||||
meta.writeLong(data.getFilePointer());
|
||||
PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
|
||||
Builder<Long> builder = new Builder<Long>(INPUT_TYPE.BYTE1, outputs);
|
||||
IntsRef scratch = new IntsRef();
|
||||
long ord = 0;
|
||||
for (BytesRef v : values) {
|
||||
builder.add(Util.toIntsRef(v, scratch), ord);
|
||||
ord++;
|
||||
}
|
||||
FST<Long> fst = builder.finish();
|
||||
if (fst != null) {
|
||||
fst.save(data);
|
||||
}
|
||||
meta.writeVLong(ord);
|
||||
}
|
||||
|
||||
// TODO: in some cases representing missing with minValue-1 wouldn't take up additional space and so on,
|
||||
// but this is very simple, and algorithms only check this for values of 0 anyway (doesnt slow down normal decode)
|
||||
void writeMissingBitset(Iterable<?> values) throws IOException {
|
||||
long bits = 0;
|
||||
int count = 0;
|
||||
for (Object v : values) {
|
||||
if (count == 64) {
|
||||
data.writeLong(bits);
|
||||
count = 0;
|
||||
bits = 0;
|
||||
}
|
||||
if (v != null) {
|
||||
bits |= 1L << (count & 0x3f);
|
||||
}
|
||||
count++;
|
||||
}
|
||||
if (count > 0) {
|
||||
data.writeLong(bits);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
|
||||
// write the ordinals as numerics
|
||||
addNumericField(field, docToOrd, false);
|
||||
|
||||
// write the values as FST
|
||||
writeFST(field, values);
|
||||
}
|
||||
|
||||
// note: this might not be the most efficient... but its fairly simple
|
||||
@Override
|
||||
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
|
||||
// write the ordinals as a binary field
|
||||
addBinaryField(field, new Iterable<BytesRef>() {
|
||||
@Override
|
||||
public Iterator<BytesRef> iterator() {
|
||||
return new SortedSetIterator(docToOrdCount.iterator(), ords.iterator());
|
||||
}
|
||||
});
|
||||
|
||||
// write the values as FST
|
||||
writeFST(field, values);
|
||||
}
|
||||
|
||||
// per-document vint-encoded byte[]
|
||||
static class SortedSetIterator implements Iterator<BytesRef> {
|
||||
byte[] buffer = new byte[10];
|
||||
ByteArrayDataOutput out = new ByteArrayDataOutput();
|
||||
BytesRef ref = new BytesRef();
|
||||
|
||||
final Iterator<Number> counts;
|
||||
final Iterator<Number> ords;
|
||||
|
||||
SortedSetIterator(Iterator<Number> counts, Iterator<Number> ords) {
|
||||
this.counts = counts;
|
||||
this.ords = ords;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return counts.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef next() {
|
||||
if (!hasNext()) {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
|
||||
int count = counts.next().intValue();
|
||||
int maxSize = count*9; // worst case
|
||||
if (maxSize > buffer.length) {
|
||||
buffer = ArrayUtil.grow(buffer, maxSize);
|
||||
}
|
||||
|
||||
try {
|
||||
encodeValues(count);
|
||||
} catch (IOException bogus) {
|
||||
throw new RuntimeException(bogus);
|
||||
}
|
||||
|
||||
ref.bytes = buffer;
|
||||
ref.offset = 0;
|
||||
ref.length = out.getPosition();
|
||||
|
||||
return ref;
|
||||
}
|
||||
|
||||
// encodes count values to buffer
|
||||
private void encodeValues(int count) throws IOException {
|
||||
out.reset(buffer);
|
||||
long lastOrd = 0;
|
||||
for (int i = 0; i < count; i++) {
|
||||
long ord = ords.next().longValue();
|
||||
out.writeVLong(ord - lastOrd);
|
||||
lastOrd = ord;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,72 @@
|
|||
package org.apache.lucene.codecs.memory;
|
||||
|
||||
/*
|
||||
* 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.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/** In-memory docvalues format */
|
||||
public class MemoryDocValuesFormat extends DocValuesFormat {
|
||||
|
||||
/** Maximum length for each binary doc values field. */
|
||||
public static final int MAX_BINARY_FIELD_LENGTH = (1 << 15) - 2;
|
||||
|
||||
final float acceptableOverheadRatio;
|
||||
|
||||
/**
|
||||
* Calls {@link #MemoryDocValuesFormat(float)
|
||||
* MemoryDocValuesFormat(PackedInts.DEFAULT)}
|
||||
*/
|
||||
public MemoryDocValuesFormat() {
|
||||
this(PackedInts.DEFAULT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new MemoryDocValuesFormat with the specified
|
||||
* <code>acceptableOverheadRatio</code> for NumericDocValues.
|
||||
* @param acceptableOverheadRatio compression parameter for numerics.
|
||||
* Currently this is only used when the number of unique values is small.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public MemoryDocValuesFormat(float acceptableOverheadRatio) {
|
||||
super("Memory");
|
||||
this.acceptableOverheadRatio = acceptableOverheadRatio;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
return new MemoryDocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
return new MemoryDocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
|
||||
}
|
||||
|
||||
static final String DATA_CODEC = "MemoryDocValuesData";
|
||||
static final String DATA_EXTENSION = "mdvd";
|
||||
static final String METADATA_CODEC = "MemoryDocValuesMetadata";
|
||||
static final String METADATA_EXTENSION = "mdvm";
|
||||
}
|
|
@ -0,0 +1,633 @@
|
|||
package org.apache.lucene.codecs.memory;
|
||||
|
||||
/*
|
||||
* 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 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.store.ByteArrayDataInput;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.fst.BytesRefFSTEnum;
|
||||
import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
import org.apache.lucene.util.fst.FST.Arc;
|
||||
import org.apache.lucene.util.fst.FST.BytesReader;
|
||||
import org.apache.lucene.util.fst.PositiveIntOutputs;
|
||||
import org.apache.lucene.util.fst.Util;
|
||||
import org.apache.lucene.util.packed.BlockPackedReader;
|
||||
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
* Reader for {@link MemoryDocValuesFormat}
|
||||
*/
|
||||
class MemoryDocValuesProducer extends DocValuesProducer {
|
||||
// metadata maps (just file pointers and minimal stuff)
|
||||
private final Map<Integer,NumericEntry> numerics;
|
||||
private final Map<Integer,BinaryEntry> binaries;
|
||||
private final Map<Integer,FSTEntry> fsts;
|
||||
private final IndexInput data;
|
||||
|
||||
// ram instances we have already loaded
|
||||
private final Map<Integer,NumericDocValues> numericInstances =
|
||||
new HashMap<Integer,NumericDocValues>();
|
||||
private final Map<Integer,BinaryDocValues> binaryInstances =
|
||||
new HashMap<Integer,BinaryDocValues>();
|
||||
private final Map<Integer,FST<Long>> fstInstances =
|
||||
new HashMap<Integer,FST<Long>>();
|
||||
private final Map<Integer,Bits> docsWithFieldInstances = new HashMap<Integer,Bits>();
|
||||
|
||||
private final int maxDoc;
|
||||
|
||||
|
||||
static final byte NUMBER = 0;
|
||||
static final byte BYTES = 1;
|
||||
static final byte FST = 2;
|
||||
|
||||
static final int BLOCK_SIZE = 4096;
|
||||
|
||||
static final byte DELTA_COMPRESSED = 0;
|
||||
static final byte TABLE_COMPRESSED = 1;
|
||||
static final byte UNCOMPRESSED = 2;
|
||||
static final byte GCD_COMPRESSED = 3;
|
||||
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_GCD_COMPRESSION = 1;
|
||||
static final int VERSION_CURRENT = VERSION_GCD_COMPRESSION;
|
||||
|
||||
MemoryDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
maxDoc = state.segmentInfo.getDocCount();
|
||||
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);
|
||||
boolean success = false;
|
||||
final int version;
|
||||
try {
|
||||
version = CodecUtil.checkHeader(in, metaCodec,
|
||||
VERSION_START,
|
||||
VERSION_CURRENT);
|
||||
numerics = new HashMap<Integer,NumericEntry>();
|
||||
binaries = new HashMap<Integer,BinaryEntry>();
|
||||
fsts = new HashMap<Integer,FSTEntry>();
|
||||
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,
|
||||
VERSION_START,
|
||||
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) {
|
||||
int fieldType = meta.readByte();
|
||||
if (fieldType == NUMBER) {
|
||||
NumericEntry entry = new NumericEntry();
|
||||
entry.offset = meta.readLong();
|
||||
entry.missingOffset = meta.readLong();
|
||||
if (entry.missingOffset != -1) {
|
||||
entry.missingBytes = meta.readLong();
|
||||
} else {
|
||||
entry.missingBytes = 0;
|
||||
}
|
||||
entry.format = meta.readByte();
|
||||
switch(entry.format) {
|
||||
case DELTA_COMPRESSED:
|
||||
case TABLE_COMPRESSED:
|
||||
case GCD_COMPRESSED:
|
||||
case UNCOMPRESSED:
|
||||
break;
|
||||
default:
|
||||
throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
|
||||
}
|
||||
if (entry.format != UNCOMPRESSED) {
|
||||
entry.packedIntsVersion = meta.readVInt();
|
||||
}
|
||||
numerics.put(fieldNumber, entry);
|
||||
} else if (fieldType == BYTES) {
|
||||
BinaryEntry entry = new BinaryEntry();
|
||||
entry.offset = meta.readLong();
|
||||
entry.numBytes = meta.readLong();
|
||||
entry.missingOffset = meta.readLong();
|
||||
if (entry.missingOffset != -1) {
|
||||
entry.missingBytes = meta.readLong();
|
||||
} else {
|
||||
entry.missingBytes = 0;
|
||||
}
|
||||
entry.minLength = meta.readVInt();
|
||||
entry.maxLength = meta.readVInt();
|
||||
if (entry.minLength != entry.maxLength) {
|
||||
entry.packedIntsVersion = meta.readVInt();
|
||||
entry.blockSize = meta.readVInt();
|
||||
}
|
||||
binaries.put(fieldNumber, entry);
|
||||
} else if (fieldType == FST) {
|
||||
FSTEntry entry = new FSTEntry();
|
||||
entry.offset = meta.readLong();
|
||||
entry.numOrds = meta.readVLong();
|
||||
fsts.put(fieldNumber, entry);
|
||||
} else {
|
||||
throw new CorruptIndexException("invalid entry type: " + fieldType + ", input=" + meta);
|
||||
}
|
||||
fieldNumber = meta.readVInt();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized NumericDocValues getNumeric(FieldInfo field) throws IOException {
|
||||
NumericDocValues instance = numericInstances.get(field.number);
|
||||
if (instance == null) {
|
||||
instance = loadNumeric(field);
|
||||
numericInstances.put(field.number, instance);
|
||||
}
|
||||
return instance;
|
||||
}
|
||||
|
||||
private NumericDocValues loadNumeric(FieldInfo field) throws IOException {
|
||||
NumericEntry entry = numerics.get(field.number);
|
||||
data.seek(entry.offset + entry.missingBytes);
|
||||
switch (entry.format) {
|
||||
case TABLE_COMPRESSED:
|
||||
int size = data.readVInt();
|
||||
if (size > 256) {
|
||||
throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, input=" + data);
|
||||
}
|
||||
final long decode[] = new long[size];
|
||||
for (int i = 0; i < decode.length; i++) {
|
||||
decode[i] = data.readLong();
|
||||
}
|
||||
final int formatID = data.readVInt();
|
||||
final int bitsPerValue = data.readVInt();
|
||||
final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), entry.packedIntsVersion, maxDoc, bitsPerValue);
|
||||
return new NumericDocValues() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
return decode[(int)ordsReader.get(docID)];
|
||||
}
|
||||
};
|
||||
case DELTA_COMPRESSED:
|
||||
final int blockSize = data.readVInt();
|
||||
final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, maxDoc, false);
|
||||
return new NumericDocValues() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
return reader.get(docID);
|
||||
}
|
||||
};
|
||||
case UNCOMPRESSED:
|
||||
final byte bytes[] = new byte[maxDoc];
|
||||
data.readBytes(bytes, 0, bytes.length);
|
||||
return new NumericDocValues() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
return bytes[docID];
|
||||
}
|
||||
};
|
||||
case GCD_COMPRESSED:
|
||||
final long min = data.readLong();
|
||||
final long mult = data.readLong();
|
||||
final int quotientBlockSize = data.readVInt();
|
||||
final BlockPackedReader quotientReader = new BlockPackedReader(data, entry.packedIntsVersion, quotientBlockSize, maxDoc, false);
|
||||
return new NumericDocValues() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
return min + mult * quotientReader.get(docID);
|
||||
}
|
||||
};
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized BinaryDocValues getBinary(FieldInfo field) throws IOException {
|
||||
BinaryDocValues instance = binaryInstances.get(field.number);
|
||||
if (instance == null) {
|
||||
instance = loadBinary(field);
|
||||
binaryInstances.put(field.number, instance);
|
||||
}
|
||||
return instance;
|
||||
}
|
||||
|
||||
private BinaryDocValues loadBinary(FieldInfo field) throws IOException {
|
||||
BinaryEntry entry = binaries.get(field.number);
|
||||
data.seek(entry.offset);
|
||||
PagedBytes bytes = new PagedBytes(16);
|
||||
bytes.copy(data, entry.numBytes);
|
||||
final PagedBytes.Reader bytesReader = bytes.freeze(true);
|
||||
if (entry.minLength == entry.maxLength) {
|
||||
final int fixedLength = entry.minLength;
|
||||
return new BinaryDocValues() {
|
||||
@Override
|
||||
public void get(int docID, BytesRef result) {
|
||||
bytesReader.fillSlice(result, fixedLength * (long)docID, fixedLength);
|
||||
}
|
||||
};
|
||||
} else {
|
||||
data.seek(data.getFilePointer() + entry.missingBytes);
|
||||
final MonotonicBlockPackedReader addresses = new MonotonicBlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, maxDoc, false);
|
||||
return new BinaryDocValues() {
|
||||
@Override
|
||||
public void get(int docID, BytesRef result) {
|
||||
long startAddress = docID == 0 ? 0 : addresses.get(docID-1);
|
||||
long endAddress = addresses.get(docID);
|
||||
bytesReader.fillSlice(result, startAddress, (int) (endAddress - startAddress));
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedDocValues getSorted(FieldInfo field) throws IOException {
|
||||
final FSTEntry entry = fsts.get(field.number);
|
||||
if (entry.numOrds == 0) {
|
||||
return SortedDocValues.EMPTY;
|
||||
}
|
||||
FST<Long> instance;
|
||||
synchronized(this) {
|
||||
instance = fstInstances.get(field.number);
|
||||
if (instance == null) {
|
||||
data.seek(entry.offset);
|
||||
instance = new FST<Long>(data, PositiveIntOutputs.getSingleton());
|
||||
fstInstances.put(field.number, instance);
|
||||
}
|
||||
}
|
||||
final NumericDocValues docToOrd = getNumeric(field);
|
||||
final FST<Long> fst = instance;
|
||||
|
||||
// per-thread resources
|
||||
final BytesReader in = fst.getBytesReader();
|
||||
final Arc<Long> firstArc = new Arc<Long>();
|
||||
final Arc<Long> scratchArc = new Arc<Long>();
|
||||
final IntsRef scratchInts = new IntsRef();
|
||||
final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<Long>(fst);
|
||||
|
||||
return new SortedDocValues() {
|
||||
@Override
|
||||
public int getOrd(int docID) {
|
||||
return (int) docToOrd.get(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void lookupOrd(int ord, BytesRef result) {
|
||||
try {
|
||||
in.setPosition(0);
|
||||
fst.getFirstArc(firstArc);
|
||||
IntsRef output = Util.getByOutput(fst, ord, in, firstArc, scratchArc, scratchInts);
|
||||
result.bytes = new byte[output.length];
|
||||
result.offset = 0;
|
||||
result.length = 0;
|
||||
Util.toBytesRef(output, result);
|
||||
} catch (IOException bogus) {
|
||||
throw new RuntimeException(bogus);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupTerm(BytesRef key) {
|
||||
try {
|
||||
InputOutput<Long> o = fstEnum.seekCeil(key);
|
||||
if (o == null) {
|
||||
return -getValueCount()-1;
|
||||
} else if (o.input.equals(key)) {
|
||||
return o.output.intValue();
|
||||
} else {
|
||||
return (int) -o.output-1;
|
||||
}
|
||||
} catch (IOException bogus) {
|
||||
throw new RuntimeException(bogus);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCount() {
|
||||
return (int)entry.numOrds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum termsEnum() {
|
||||
return new FSTTermsEnum(fst);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
|
||||
final FSTEntry entry = fsts.get(field.number);
|
||||
if (entry.numOrds == 0) {
|
||||
return SortedSetDocValues.EMPTY; // empty FST!
|
||||
}
|
||||
FST<Long> instance;
|
||||
synchronized(this) {
|
||||
instance = fstInstances.get(field.number);
|
||||
if (instance == null) {
|
||||
data.seek(entry.offset);
|
||||
instance = new FST<Long>(data, PositiveIntOutputs.getSingleton());
|
||||
fstInstances.put(field.number, instance);
|
||||
}
|
||||
}
|
||||
final BinaryDocValues docToOrds = getBinary(field);
|
||||
final FST<Long> fst = instance;
|
||||
|
||||
// per-thread resources
|
||||
final BytesReader in = fst.getBytesReader();
|
||||
final Arc<Long> firstArc = new Arc<Long>();
|
||||
final Arc<Long> scratchArc = new Arc<Long>();
|
||||
final IntsRef scratchInts = new IntsRef();
|
||||
final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<Long>(fst);
|
||||
final BytesRef ref = new BytesRef();
|
||||
final ByteArrayDataInput input = new ByteArrayDataInput();
|
||||
return new SortedSetDocValues() {
|
||||
long currentOrd;
|
||||
|
||||
@Override
|
||||
public long nextOrd() {
|
||||
if (input.eof()) {
|
||||
return NO_MORE_ORDS;
|
||||
} else {
|
||||
currentOrd += input.readVLong();
|
||||
return currentOrd;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setDocument(int docID) {
|
||||
docToOrds.get(docID, ref);
|
||||
input.reset(ref.bytes, ref.offset, ref.length);
|
||||
currentOrd = 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void lookupOrd(long ord, BytesRef result) {
|
||||
try {
|
||||
in.setPosition(0);
|
||||
fst.getFirstArc(firstArc);
|
||||
IntsRef output = Util.getByOutput(fst, ord, in, firstArc, scratchArc, scratchInts);
|
||||
result.bytes = new byte[output.length];
|
||||
result.offset = 0;
|
||||
result.length = 0;
|
||||
Util.toBytesRef(output, result);
|
||||
} catch (IOException bogus) {
|
||||
throw new RuntimeException(bogus);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long lookupTerm(BytesRef key) {
|
||||
try {
|
||||
InputOutput<Long> o = fstEnum.seekCeil(key);
|
||||
if (o == null) {
|
||||
return -getValueCount()-1;
|
||||
} else if (o.input.equals(key)) {
|
||||
return o.output.intValue();
|
||||
} else {
|
||||
return -o.output-1;
|
||||
}
|
||||
} catch (IOException bogus) {
|
||||
throw new RuntimeException(bogus);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getValueCount() {
|
||||
return entry.numOrds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum termsEnum() {
|
||||
return new FSTTermsEnum(fst);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private Bits getMissingBits(int fieldNumber, final long offset, final long length) throws IOException {
|
||||
if (offset == -1) {
|
||||
return new Bits.MatchAllBits(maxDoc);
|
||||
} else {
|
||||
Bits instance;
|
||||
synchronized(this) {
|
||||
instance = docsWithFieldInstances.get(fieldNumber);
|
||||
if (instance == null) {
|
||||
IndexInput data = this.data.clone();
|
||||
data.seek(offset);
|
||||
assert length % 8 == 0;
|
||||
long bits[] = new long[(int) length >> 3];
|
||||
for (int i = 0; i < bits.length; i++) {
|
||||
bits[i] = data.readLong();
|
||||
}
|
||||
instance = new FixedBitSet(bits, maxDoc);
|
||||
docsWithFieldInstances.put(fieldNumber, instance);
|
||||
}
|
||||
}
|
||||
return instance;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
switch(field.getDocValuesType()) {
|
||||
case SORTED_SET:
|
||||
return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
|
||||
case SORTED:
|
||||
return new SortedDocsWithField(getSorted(field), maxDoc);
|
||||
case BINARY:
|
||||
BinaryEntry be = binaries.get(field.number);
|
||||
return getMissingBits(field.number, be.missingOffset, be.missingBytes);
|
||||
case NUMERIC:
|
||||
NumericEntry ne = numerics.get(field.number);
|
||||
return getMissingBits(field.number, ne.missingOffset, ne.missingBytes);
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
data.close();
|
||||
}
|
||||
|
||||
static class NumericEntry {
|
||||
long offset;
|
||||
long missingOffset;
|
||||
long missingBytes;
|
||||
byte format;
|
||||
int packedIntsVersion;
|
||||
}
|
||||
|
||||
static class BinaryEntry {
|
||||
long offset;
|
||||
long missingOffset;
|
||||
long missingBytes;
|
||||
long numBytes;
|
||||
int minLength;
|
||||
int maxLength;
|
||||
int packedIntsVersion;
|
||||
int blockSize;
|
||||
}
|
||||
|
||||
static class FSTEntry {
|
||||
long offset;
|
||||
long numOrds;
|
||||
}
|
||||
|
||||
// exposes FSTEnum directly as a TermsEnum: avoids binary-search next()
|
||||
static class FSTTermsEnum extends TermsEnum {
|
||||
final BytesRefFSTEnum<Long> in;
|
||||
|
||||
// this is all for the complicated seek(ord)...
|
||||
// maybe we should add a FSTEnum that supports this operation?
|
||||
final FST<Long> fst;
|
||||
final FST.BytesReader bytesReader;
|
||||
final Arc<Long> firstArc = new Arc<Long>();
|
||||
final Arc<Long> scratchArc = new Arc<Long>();
|
||||
final IntsRef scratchInts = new IntsRef();
|
||||
final BytesRef scratchBytes = new BytesRef();
|
||||
|
||||
FSTTermsEnum(FST<Long> fst) {
|
||||
this.fst = fst;
|
||||
in = new BytesRefFSTEnum<Long>(fst);
|
||||
bytesReader = fst.getBytesReader();
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef next() throws IOException {
|
||||
InputOutput<Long> io = in.next();
|
||||
if (io == null) {
|
||||
return null;
|
||||
} else {
|
||||
return io.input;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public SeekStatus seekCeil(BytesRef text) throws IOException {
|
||||
if (in.seekCeil(text) == null) {
|
||||
return SeekStatus.END;
|
||||
} else if (term().equals(text)) {
|
||||
// TODO: add SeekStatus to FSTEnum like in https://issues.apache.org/jira/browse/LUCENE-3729
|
||||
// to remove this comparision?
|
||||
return SeekStatus.FOUND;
|
||||
} else {
|
||||
return SeekStatus.NOT_FOUND;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean seekExact(BytesRef text) throws IOException {
|
||||
if (in.seekExact(text) == null) {
|
||||
return false;
|
||||
} else {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void seekExact(long ord) throws IOException {
|
||||
// TODO: would be better to make this simpler and faster.
|
||||
// but we dont want to introduce a bug that corrupts our enum state!
|
||||
bytesReader.setPosition(0);
|
||||
fst.getFirstArc(firstArc);
|
||||
IntsRef output = Util.getByOutput(fst, ord, bytesReader, firstArc, scratchArc, scratchInts);
|
||||
scratchBytes.bytes = new byte[output.length];
|
||||
scratchBytes.offset = 0;
|
||||
scratchBytes.length = 0;
|
||||
Util.toBytesRef(output, scratchBytes);
|
||||
// TODO: we could do this lazily, better to try to push into FSTEnum though?
|
||||
in.seekExact(scratchBytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef term() throws IOException {
|
||||
return in.current().input;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ord() throws IOException {
|
||||
return in.current().output;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docFreq() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long totalTermFreq() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -20,6 +20,6 @@
|
|||
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
|
||||
</head>
|
||||
<body>
|
||||
Postings format that is read entirely into memory.
|
||||
Postings and DocValues formats that are read entirely into memory.
|
||||
</body>
|
||||
</html>
|
|
@ -38,12 +38,16 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
* minvalue 0
|
||||
* pattern 000
|
||||
* 005
|
||||
* T
|
||||
* 234
|
||||
* T
|
||||
* 123
|
||||
* T
|
||||
* ...
|
||||
* </pre>
|
||||
* so a document's value (delta encoded from minvalue) can be retrieved by
|
||||
* seeking to startOffset + (1+pattern.length())*docid. The extra 1 is the newline.
|
||||
* seeking to startOffset + (1+pattern.length()+2)*docid. The extra 1 is the newline.
|
||||
* The extra 2 is another newline and 'T' or 'F': true if the value is real, false if missing.
|
||||
*
|
||||
* for bytes this is also a "fixed-width" file, for example:
|
||||
* <pre>
|
||||
|
@ -53,12 +57,15 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
* pattern 0
|
||||
* length 6
|
||||
* foobar[space][space]
|
||||
* T
|
||||
* length 3
|
||||
* baz[space][space][space][space][space]
|
||||
* T
|
||||
* ...
|
||||
* </pre>
|
||||
* so a doc's value can be retrieved by seeking to startOffset + (9+pattern.length+maxlength)*doc
|
||||
* so a doc's value can be retrieved by seeking to startOffset + (9+pattern.length+maxlength+2)*doc
|
||||
* the extra 9 is 2 newlines, plus "length " itself.
|
||||
* the extra 2 is another newline and 'T' or 'F': true if the value is real, false if missing.
|
||||
*
|
||||
* for sorted bytes this is a fixed-width file, for example:
|
||||
* <pre>
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.lucene.index.SortedDocValues;
|
|||
import org.apache.lucene.index.FieldInfo.DocValuesType;
|
||||
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.StringHelper;
|
||||
|
||||
|
@ -100,7 +101,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
assert startsWith(PATTERN);
|
||||
field.pattern = stripPrefix(PATTERN);
|
||||
field.dataStartFilePointer = data.getFilePointer();
|
||||
data.seek(data.getFilePointer() + (1+field.pattern.length()) * maxDoc);
|
||||
data.seek(data.getFilePointer() + (1+field.pattern.length()+2) * maxDoc);
|
||||
} else if (dvType == DocValuesType.BINARY) {
|
||||
readLine();
|
||||
assert startsWith(MAXLENGTH);
|
||||
|
@ -109,7 +110,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
assert startsWith(PATTERN);
|
||||
field.pattern = stripPrefix(PATTERN);
|
||||
field.dataStartFilePointer = data.getFilePointer();
|
||||
data.seek(data.getFilePointer() + (9+field.pattern.length()+field.maxLength) * maxDoc);
|
||||
data.seek(data.getFilePointer() + (9+field.pattern.length()+field.maxLength+2) * maxDoc);
|
||||
} else if (dvType == DocValuesType.SORTED || dvType == DocValuesType.SORTED_SET) {
|
||||
readLine();
|
||||
assert startsWith(NUMVALUES);
|
||||
|
@ -158,7 +159,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
if (docID < 0 || docID >= maxDoc) {
|
||||
throw new IndexOutOfBoundsException("docID must be 0 .. " + (maxDoc-1) + "; got " + docID);
|
||||
}
|
||||
in.seek(field.dataStartFilePointer + (1+field.pattern.length())*docID);
|
||||
in.seek(field.dataStartFilePointer + (1+field.pattern.length()+2)*docID);
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
//System.out.println("parsing delta: " + scratch.utf8ToString());
|
||||
BigDecimal bd;
|
||||
|
@ -169,6 +170,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
e.initCause(pe);
|
||||
throw e;
|
||||
}
|
||||
SimpleTextUtil.readLine(in, scratch); // read the line telling us if its real or not
|
||||
return BigInteger.valueOf(field.minValue).add(bd.toBigIntegerExact()).longValue();
|
||||
} catch (IOException ioe) {
|
||||
throw new RuntimeException(ioe);
|
||||
|
@ -176,6 +178,30 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
private Bits getNumericDocsWithField(FieldInfo fieldInfo) throws IOException {
|
||||
final OneField field = fields.get(fieldInfo.name);
|
||||
final IndexInput in = data.clone();
|
||||
final BytesRef scratch = new BytesRef();
|
||||
return new Bits() {
|
||||
@Override
|
||||
public boolean get(int index) {
|
||||
try {
|
||||
in.seek(field.dataStartFilePointer + (1+field.pattern.length()+2)*index);
|
||||
SimpleTextUtil.readLine(in, scratch); // data
|
||||
SimpleTextUtil.readLine(in, scratch); // 'T' or 'F'
|
||||
return scratch.bytes[scratch.offset] == (byte) 'T';
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return maxDoc;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public BinaryDocValues getBinary(FieldInfo fieldInfo) throws IOException {
|
||||
|
@ -196,7 +222,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
if (docID < 0 || docID >= maxDoc) {
|
||||
throw new IndexOutOfBoundsException("docID must be 0 .. " + (maxDoc-1) + "; got " + docID);
|
||||
}
|
||||
in.seek(field.dataStartFilePointer + (9+field.pattern.length() + field.maxLength)*docID);
|
||||
in.seek(field.dataStartFilePointer + (9+field.pattern.length() + field.maxLength+2)*docID);
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
assert StringHelper.startsWith(scratch, LENGTH);
|
||||
int len;
|
||||
|
@ -217,6 +243,45 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
private Bits getBinaryDocsWithField(FieldInfo fieldInfo) throws IOException {
|
||||
final OneField field = fields.get(fieldInfo.name);
|
||||
final IndexInput in = data.clone();
|
||||
final BytesRef scratch = new BytesRef();
|
||||
final DecimalFormat decoder = new DecimalFormat(field.pattern, new DecimalFormatSymbols(Locale.ROOT));
|
||||
|
||||
return new Bits() {
|
||||
@Override
|
||||
public boolean get(int index) {
|
||||
try {
|
||||
in.seek(field.dataStartFilePointer + (9+field.pattern.length() + field.maxLength+2)*index);
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
assert StringHelper.startsWith(scratch, LENGTH);
|
||||
int len;
|
||||
try {
|
||||
len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, "UTF-8")).intValue();
|
||||
} catch (ParseException pe) {
|
||||
CorruptIndexException e = new CorruptIndexException("failed to parse int length (resource=" + in + ")");
|
||||
e.initCause(pe);
|
||||
throw e;
|
||||
}
|
||||
// skip past bytes
|
||||
byte bytes[] = new byte[len];
|
||||
in.readBytes(bytes, 0, len);
|
||||
SimpleTextUtil.readLine(in, scratch); // newline
|
||||
SimpleTextUtil.readLine(in, scratch); // 'T' or 'F'
|
||||
return scratch.bytes[scratch.offset] == (byte) 'T';
|
||||
} catch (IOException ioe) {
|
||||
throw new RuntimeException(ioe);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return maxDoc;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedDocValues getSorted(FieldInfo fieldInfo) throws IOException {
|
||||
|
@ -241,7 +306,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
in.seek(field.dataStartFilePointer + field.numValues * (9 + field.pattern.length() + field.maxLength) + docID * (1 + field.ordPattern.length()));
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
try {
|
||||
return ordDecoder.parse(scratch.utf8ToString()).intValue();
|
||||
return (int) ordDecoder.parse(scratch.utf8ToString()).longValue()-1;
|
||||
} catch (ParseException pe) {
|
||||
CorruptIndexException e = new CorruptIndexException("failed to parse ord (resource=" + in + ")");
|
||||
e.initCause(pe);
|
||||
|
@ -362,6 +427,22 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
switch (field.getDocValuesType()) {
|
||||
case SORTED_SET:
|
||||
return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
|
||||
case SORTED:
|
||||
return new SortedDocsWithField(getSorted(field), maxDoc);
|
||||
case BINARY:
|
||||
return getBinaryDocsWithField(field);
|
||||
case NUMERIC:
|
||||
return getNumericDocsWithField(field);
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
|
|
@ -78,7 +78,7 @@ class SimpleTextDocValuesWriter extends DocValuesConsumer {
|
|||
long minValue = Long.MAX_VALUE;
|
||||
long maxValue = Long.MIN_VALUE;
|
||||
for(Number n : values) {
|
||||
long v = n.longValue();
|
||||
long v = n == null ? 0 : n.longValue();
|
||||
minValue = Math.min(minValue, v);
|
||||
maxValue = Math.max(maxValue, v);
|
||||
}
|
||||
|
@ -112,13 +112,19 @@ class SimpleTextDocValuesWriter extends DocValuesConsumer {
|
|||
|
||||
// second pass to write the values
|
||||
for(Number n : values) {
|
||||
long value = n.longValue();
|
||||
long value = n == null ? 0 : n.longValue();
|
||||
assert value >= minValue;
|
||||
Number delta = BigInteger.valueOf(value).subtract(BigInteger.valueOf(minValue));
|
||||
String s = encoder.format(delta);
|
||||
assert s.length() == patternString.length();
|
||||
SimpleTextUtil.write(data, s, scratch);
|
||||
SimpleTextUtil.writeNewline(data);
|
||||
if (n == null) {
|
||||
SimpleTextUtil.write(data, "F", scratch);
|
||||
} else {
|
||||
SimpleTextUtil.write(data, "T", scratch);
|
||||
}
|
||||
SimpleTextUtil.writeNewline(data);
|
||||
numDocsWritten++;
|
||||
assert numDocsWritten <= numDocs;
|
||||
}
|
||||
|
@ -132,7 +138,8 @@ class SimpleTextDocValuesWriter extends DocValuesConsumer {
|
|||
assert field.getDocValuesType() == DocValuesType.BINARY;
|
||||
int maxLength = 0;
|
||||
for(BytesRef value : values) {
|
||||
maxLength = Math.max(maxLength, value.length);
|
||||
final int length = value == null ? 0 : value.length;
|
||||
maxLength = Math.max(maxLength, length);
|
||||
}
|
||||
writeFieldEntry(field, FieldInfo.DocValuesType.BINARY);
|
||||
|
||||
|
@ -155,19 +162,28 @@ class SimpleTextDocValuesWriter extends DocValuesConsumer {
|
|||
int numDocsWritten = 0;
|
||||
for(BytesRef value : values) {
|
||||
// write length
|
||||
final int length = value == null ? 0 : value.length;
|
||||
SimpleTextUtil.write(data, LENGTH);
|
||||
SimpleTextUtil.write(data, encoder.format(value.length), scratch);
|
||||
SimpleTextUtil.write(data, encoder.format(length), scratch);
|
||||
SimpleTextUtil.writeNewline(data);
|
||||
|
||||
// write bytes -- don't use SimpleText.write
|
||||
// because it escapes:
|
||||
data.writeBytes(value.bytes, value.offset, value.length);
|
||||
if (value != null) {
|
||||
data.writeBytes(value.bytes, value.offset, value.length);
|
||||
}
|
||||
|
||||
// pad to fit
|
||||
for (int i = value.length; i < maxLength; i++) {
|
||||
for (int i = length; i < maxLength; i++) {
|
||||
data.writeByte((byte)' ');
|
||||
}
|
||||
SimpleTextUtil.writeNewline(data);
|
||||
if (value == null) {
|
||||
SimpleTextUtil.write(data, "F", scratch);
|
||||
} else {
|
||||
SimpleTextUtil.write(data, "T", scratch);
|
||||
}
|
||||
SimpleTextUtil.writeNewline(data);
|
||||
numDocsWritten++;
|
||||
}
|
||||
|
||||
|
@ -209,7 +225,7 @@ class SimpleTextDocValuesWriter extends DocValuesConsumer {
|
|||
SimpleTextUtil.writeNewline(data);
|
||||
final DecimalFormat encoder = new DecimalFormat(sb.toString(), new DecimalFormatSymbols(Locale.ROOT));
|
||||
|
||||
int maxOrdBytes = Integer.toString(valueCount).length();
|
||||
int maxOrdBytes = Long.toString(valueCount+1L).length();
|
||||
sb.setLength(0);
|
||||
for (int i = 0; i < maxOrdBytes; i++) {
|
||||
sb.append('0');
|
||||
|
@ -246,7 +262,7 @@ class SimpleTextDocValuesWriter extends DocValuesConsumer {
|
|||
assert valuesSeen == valueCount;
|
||||
|
||||
for(Number ord : docToOrd) {
|
||||
SimpleTextUtil.write(data, ordEncoder.format(ord.intValue()), scratch);
|
||||
SimpleTextUtil.write(data, ordEncoder.format(ord.longValue()+1), scratch);
|
||||
SimpleTextUtil.writeNewline(data);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -14,4 +14,5 @@
|
|||
# limitations under the License.
|
||||
|
||||
org.apache.lucene.codecs.diskdv.DiskDocValuesFormat
|
||||
org.apache.lucene.codecs.memory.MemoryDocValuesFormat
|
||||
org.apache.lucene.codecs.simpletext.SimpleTextDocValuesFormat
|
||||
|
|
|
@ -0,0 +1,39 @@
|
|||
package org.apache.lucene.codecs.memory;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
|
||||
/**
|
||||
* Tests MemoryDocValuesFormat
|
||||
*/
|
||||
public class TestMemoryDocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
|
||||
private final Codec codec = _TestUtil.alwaysDocValuesFormat(new MemoryDocValuesFormat());
|
||||
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
return codec;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean codecAcceptsHugeBinaryValues(String field) {
|
||||
return false;
|
||||
}
|
||||
}
|
|
@ -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.
|
||||
|
|
|
@ -69,7 +69,8 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
/**
|
||||
* Writes numeric docvalues for a field.
|
||||
* @param field field information
|
||||
* @param values Iterable of numeric values (one for each document).
|
||||
* @param values Iterable of numeric values (one for each document). {@code null} indicates
|
||||
* a missing value.
|
||||
* @throws IOException if an I/O error occurred.
|
||||
*/
|
||||
public abstract void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException;
|
||||
|
@ -77,7 +78,8 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
/**
|
||||
* Writes binary docvalues for a field.
|
||||
* @param field field information
|
||||
* @param values Iterable of binary values (one for each document).
|
||||
* @param values Iterable of binary values (one for each document). {@code null} indicates
|
||||
* a missing value.
|
||||
* @throws IOException if an I/O error occurred.
|
||||
*/
|
||||
public abstract void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException;
|
||||
|
@ -86,7 +88,8 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
* Writes pre-sorted binary docvalues for a field.
|
||||
* @param field field information
|
||||
* @param values Iterable of binary values in sorted order (deduplicated).
|
||||
* @param docToOrd Iterable of ordinals (one for each document).
|
||||
* @param docToOrd Iterable of ordinals (one for each document). {@code -1} indicates
|
||||
* a missing value.
|
||||
* @throws IOException if an I/O error occurred.
|
||||
*/
|
||||
public abstract void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException;
|
||||
|
@ -95,7 +98,8 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
* Writes pre-sorted set docvalues for a field
|
||||
* @param field field information
|
||||
* @param values Iterable of binary values in sorted order (deduplicated).
|
||||
* @param docToOrdCount Iterable of the number of values for each document.
|
||||
* @param docToOrdCount Iterable of the number of values for each document. A zero ordinal
|
||||
* count indicates a missing value.
|
||||
* @param ords Iterable of ordinal occurrences (docToOrdCount*maxDoc total).
|
||||
* @throws IOException if an I/O error occurred.
|
||||
*/
|
||||
|
@ -107,7 +111,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
* The default implementation calls {@link #addNumericField}, passing
|
||||
* an Iterable that merges and filters deleted documents on the fly.
|
||||
*/
|
||||
public void mergeNumericField(FieldInfo fieldInfo, final MergeState mergeState, final List<NumericDocValues> toMerge) throws IOException {
|
||||
public void mergeNumericField(final FieldInfo fieldInfo, final MergeState mergeState, final List<NumericDocValues> toMerge, final List<Bits> docsWithField) throws IOException {
|
||||
|
||||
addNumericField(fieldInfo,
|
||||
new Iterable<Number>() {
|
||||
|
@ -116,10 +120,11 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
return new Iterator<Number>() {
|
||||
int readerUpto = -1;
|
||||
int docIDUpto;
|
||||
long nextValue;
|
||||
Long nextValue;
|
||||
AtomicReader currentReader;
|
||||
NumericDocValues currentValues;
|
||||
Bits currentLiveDocs;
|
||||
Bits currentDocsWithField;
|
||||
boolean nextIsSet;
|
||||
|
||||
@Override
|
||||
|
@ -139,7 +144,6 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
}
|
||||
assert nextIsSet;
|
||||
nextIsSet = false;
|
||||
// TODO: make a mutable number
|
||||
return nextValue;
|
||||
}
|
||||
|
||||
|
@ -155,6 +159,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
currentReader = mergeState.readers.get(readerUpto);
|
||||
currentValues = toMerge.get(readerUpto);
|
||||
currentLiveDocs = currentReader.getLiveDocs();
|
||||
currentDocsWithField = docsWithField.get(readerUpto);
|
||||
}
|
||||
docIDUpto = 0;
|
||||
continue;
|
||||
|
@ -162,7 +167,11 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
|
||||
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
|
||||
nextIsSet = true;
|
||||
nextValue = currentValues.get(docIDUpto);
|
||||
if (currentDocsWithField.get(docIDUpto)) {
|
||||
nextValue = currentValues.get(docIDUpto);
|
||||
} else {
|
||||
nextValue = null;
|
||||
}
|
||||
docIDUpto++;
|
||||
return true;
|
||||
}
|
||||
|
@ -181,7 +190,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
* The default implementation calls {@link #addBinaryField}, passing
|
||||
* an Iterable that merges and filters deleted documents on the fly.
|
||||
*/
|
||||
public void mergeBinaryField(FieldInfo fieldInfo, final MergeState mergeState, final List<BinaryDocValues> toMerge) throws IOException {
|
||||
public void mergeBinaryField(FieldInfo fieldInfo, final MergeState mergeState, final List<BinaryDocValues> toMerge, final List<Bits> docsWithField) throws IOException {
|
||||
|
||||
addBinaryField(fieldInfo,
|
||||
new Iterable<BytesRef>() {
|
||||
|
@ -191,9 +200,11 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
int readerUpto = -1;
|
||||
int docIDUpto;
|
||||
BytesRef nextValue = new BytesRef();
|
||||
BytesRef nextPointer; // points to null if missing, or nextValue
|
||||
AtomicReader currentReader;
|
||||
BinaryDocValues currentValues;
|
||||
Bits currentLiveDocs;
|
||||
Bits currentDocsWithField;
|
||||
boolean nextIsSet;
|
||||
|
||||
@Override
|
||||
|
@ -213,8 +224,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
}
|
||||
assert nextIsSet;
|
||||
nextIsSet = false;
|
||||
// TODO: make a mutable number
|
||||
return nextValue;
|
||||
return nextPointer;
|
||||
}
|
||||
|
||||
private boolean setNext() {
|
||||
|
@ -228,6 +238,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
if (readerUpto < toMerge.size()) {
|
||||
currentReader = mergeState.readers.get(readerUpto);
|
||||
currentValues = toMerge.get(readerUpto);
|
||||
currentDocsWithField = docsWithField.get(readerUpto);
|
||||
currentLiveDocs = currentReader.getLiveDocs();
|
||||
}
|
||||
docIDUpto = 0;
|
||||
|
@ -236,7 +247,12 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
|
||||
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
|
||||
nextIsSet = true;
|
||||
currentValues.get(docIDUpto, nextValue);
|
||||
if (currentDocsWithField.get(docIDUpto)) {
|
||||
currentValues.get(docIDUpto, nextValue);
|
||||
nextPointer = nextValue;
|
||||
} else {
|
||||
nextPointer = null;
|
||||
}
|
||||
docIDUpto++;
|
||||
return true;
|
||||
}
|
||||
|
@ -272,7 +288,10 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
OpenBitSet bitset = new OpenBitSet(dv.getValueCount());
|
||||
for (int i = 0; i < reader.maxDoc(); i++) {
|
||||
if (liveDocs.get(i)) {
|
||||
bitset.set(dv.getOrd(i));
|
||||
int ord = dv.getOrd(i);
|
||||
if (ord >= 0) {
|
||||
bitset.set(ord);
|
||||
}
|
||||
}
|
||||
}
|
||||
liveTerms[sub] = new BitsFilteredTermsEnum(dv.termsEnum(), bitset);
|
||||
|
@ -368,7 +387,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
|
||||
nextIsSet = true;
|
||||
int segOrd = dvs[readerUpto].getOrd(docIDUpto);
|
||||
nextValue = (int) map.getGlobalOrd(readerUpto, segOrd);
|
||||
nextValue = segOrd == -1 ? -1 : (int) map.getGlobalOrd(readerUpto, segOrd);
|
||||
docIDUpto++;
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.lucene.index.FieldInfo;
|
|||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
/** Abstract API that produces numeric, binary and
|
||||
* sorted docvalues.
|
||||
|
@ -56,4 +57,67 @@ public abstract class DocValuesProducer implements Closeable {
|
|||
* The returned instance need not be thread-safe: it will only be
|
||||
* used by a single thread. */
|
||||
public abstract SortedSetDocValues getSortedSet(FieldInfo field) throws IOException;
|
||||
|
||||
/** Returns a {@link Bits} at the size of <code>reader.maxDoc()</code>,
|
||||
* with turned on bits for each docid that does have a value for this field.
|
||||
* The returned instance need not be thread-safe: it will only be
|
||||
* used by a single thread. */
|
||||
public abstract Bits getDocsWithField(FieldInfo field) throws IOException;
|
||||
|
||||
/**
|
||||
* A simple implementation of {@link DocValuesProducer#getDocsWithField} that
|
||||
* returns {@code true} if a document has an ordinal >= 0
|
||||
* <p>
|
||||
* Codecs can choose to use this (or implement it more efficiently another way), but
|
||||
* in most cases a Bits is unnecessary anyway: users can check this as they go.
|
||||
*/
|
||||
public static class SortedDocsWithField implements Bits {
|
||||
final SortedDocValues in;
|
||||
final int maxDoc;
|
||||
|
||||
/** Creates a {@link Bits} returning true if the document has a value */
|
||||
public SortedDocsWithField(SortedDocValues in, int maxDoc) {
|
||||
this.in = in;
|
||||
this.maxDoc = maxDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean get(int index) {
|
||||
return in.getOrd(index) >= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return maxDoc;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A simple implementation of {@link DocValuesProducer#getDocsWithField} that
|
||||
* returns {@code true} if a document has any ordinals.
|
||||
* <p>
|
||||
* Codecs can choose to use this (or implement it more efficiently another way), but
|
||||
* in most cases a Bits is unnecessary anyway: users can check this as they go.
|
||||
*/
|
||||
public static class SortedSetDocsWithField implements Bits {
|
||||
final SortedSetDocValues in;
|
||||
final int maxDoc;
|
||||
|
||||
/** Creates a {@link Bits} returning true if the document has a value */
|
||||
public SortedSetDocsWithField(SortedSetDocValues in, int maxDoc) {
|
||||
this.in = in;
|
||||
this.maxDoc = maxDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean get(int index) {
|
||||
in.setDocument(index);
|
||||
return in.nextOrd() != SortedSetDocValues.NO_MORE_ORDS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return maxDoc;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
||||
/**
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.lucene.index.SortedSetDocValues;
|
|||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
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.PagedBytes;
|
||||
|
@ -620,6 +621,11 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
|||
throw new IllegalStateException("Lucene 4.0 does not support SortedSet: how did you pull this off?");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
return new Bits.MatchAllBits(state.segmentInfo.getDocCount());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
dir.close();
|
||||
|
|
|
@ -17,7 +17,10 @@ package org.apache.lucene.codecs.lucene42;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
|
@ -32,6 +35,7 @@ import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
|
|||
import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
||||
/**
|
||||
* Implements the Lucene 4.2 index format, with configurable per-field postings
|
||||
|
@ -42,10 +46,12 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
|||
*
|
||||
* @see org.apache.lucene.codecs.lucene42 package documentation for file format details.
|
||||
* @lucene.experimental
|
||||
* @deprecated Only for reading old 4.2 segments
|
||||
*/
|
||||
// NOTE: if we make largish changes in a minor release, easier to just make Lucene43Codec or whatever
|
||||
// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
|
||||
// (it writes a minor version, etc).
|
||||
@Deprecated
|
||||
public class Lucene42Codec extends Codec {
|
||||
private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
|
||||
private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
|
||||
|
@ -129,10 +135,15 @@ public class Lucene42Codec extends Codec {
|
|||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
|
||||
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene42");
|
||||
|
||||
private final NormsFormat normsFormat = new Lucene42NormsFormat();
|
||||
private final NormsFormat normsFormat = new Lucene42NormsFormat() {
|
||||
@Override
|
||||
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
|
||||
throw new UnsupportedOperationException("this codec can only be used for reading");
|
||||
}
|
||||
};
|
||||
|
||||
@Override
|
||||
public final NormsFormat normsFormat() {
|
||||
public NormsFormat normsFormat() {
|
||||
return normsFormat;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -123,8 +123,10 @@ import org.apache.lucene.util.packed.BlockPackedWriter;
|
|||
* <ul>
|
||||
* <li> Binary doc values can be at most {@link #MAX_BINARY_FIELD_LENGTH} in length.
|
||||
* </ul>
|
||||
* @deprecated Only for reading old 4.2 segments
|
||||
*/
|
||||
public final class Lucene42DocValuesFormat extends DocValuesFormat {
|
||||
@Deprecated
|
||||
public class Lucene42DocValuesFormat extends DocValuesFormat {
|
||||
|
||||
/** Maximum length for each binary doc values field. */
|
||||
public static final int MAX_BINARY_FIELD_LENGTH = (1 << 15) - 2;
|
||||
|
@ -154,8 +156,7 @@ public final class Lucene42DocValuesFormat extends DocValuesFormat {
|
|||
|
||||
@Override
|
||||
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
// note: we choose DEFAULT here (its reasonably fast, and for small bpv has tiny waste)
|
||||
return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
|
||||
throw new UnsupportedOperationException("this codec can only be used for reading");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -163,8 +164,8 @@ public final class Lucene42DocValuesFormat extends DocValuesFormat {
|
|||
return new Lucene42DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
|
||||
}
|
||||
|
||||
private static final String DATA_CODEC = "Lucene42DocValuesData";
|
||||
private static final String DATA_EXTENSION = "dvd";
|
||||
private static final String METADATA_CODEC = "Lucene42DocValuesMetadata";
|
||||
private static final String METADATA_EXTENSION = "dvm";
|
||||
static final String DATA_CODEC = "Lucene42DocValuesData";
|
||||
static final String DATA_EXTENSION = "dvd";
|
||||
static final String METADATA_CODEC = "Lucene42DocValuesMetadata";
|
||||
static final String METADATA_EXTENSION = "dvm";
|
||||
}
|
||||
|
|
|
@ -17,11 +17,6 @@ package org.apache.lucene.codecs.lucene42;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.DELTA_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.GCD_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.TABLE_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.UNCOMPRESSED;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
|
@ -78,6 +73,22 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
|
|||
new HashMap<Integer,FST<Long>>();
|
||||
|
||||
private final int maxDoc;
|
||||
|
||||
|
||||
static final byte NUMBER = 0;
|
||||
static final byte BYTES = 1;
|
||||
static final byte FST = 2;
|
||||
|
||||
static final int BLOCK_SIZE = 4096;
|
||||
|
||||
static final byte DELTA_COMPRESSED = 0;
|
||||
static final byte TABLE_COMPRESSED = 1;
|
||||
static final byte UNCOMPRESSED = 2;
|
||||
static final byte GCD_COMPRESSED = 3;
|
||||
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_GCD_COMPRESSION = 1;
|
||||
static final int VERSION_CURRENT = VERSION_GCD_COMPRESSION;
|
||||
|
||||
Lucene42DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
maxDoc = state.segmentInfo.getDocCount();
|
||||
|
@ -88,8 +99,8 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
|
|||
final int version;
|
||||
try {
|
||||
version = CodecUtil.checkHeader(in, metaCodec,
|
||||
Lucene42DocValuesConsumer.VERSION_START,
|
||||
Lucene42DocValuesConsumer.VERSION_CURRENT);
|
||||
VERSION_START,
|
||||
VERSION_CURRENT);
|
||||
numerics = new HashMap<Integer,NumericEntry>();
|
||||
binaries = new HashMap<Integer,BinaryEntry>();
|
||||
fsts = new HashMap<Integer,FSTEntry>();
|
||||
|
@ -109,8 +120,8 @@ class Lucene42DocValuesProducer 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,
|
||||
Lucene42DocValuesConsumer.VERSION_START,
|
||||
Lucene42DocValuesConsumer.VERSION_CURRENT);
|
||||
VERSION_START,
|
||||
VERSION_CURRENT);
|
||||
if (version != version2) {
|
||||
throw new CorruptIndexException("Format versions mismatch");
|
||||
}
|
||||
|
@ -127,7 +138,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
|
|||
int fieldNumber = meta.readVInt();
|
||||
while (fieldNumber != -1) {
|
||||
int fieldType = meta.readByte();
|
||||
if (fieldType == Lucene42DocValuesConsumer.NUMBER) {
|
||||
if (fieldType == NUMBER) {
|
||||
NumericEntry entry = new NumericEntry();
|
||||
entry.offset = meta.readLong();
|
||||
entry.format = meta.readByte();
|
||||
|
@ -140,11 +151,11 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
|
|||
default:
|
||||
throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
|
||||
}
|
||||
if (entry.format != Lucene42DocValuesConsumer.UNCOMPRESSED) {
|
||||
if (entry.format != UNCOMPRESSED) {
|
||||
entry.packedIntsVersion = meta.readVInt();
|
||||
}
|
||||
numerics.put(fieldNumber, entry);
|
||||
} else if (fieldType == Lucene42DocValuesConsumer.BYTES) {
|
||||
} else if (fieldType == BYTES) {
|
||||
BinaryEntry entry = new BinaryEntry();
|
||||
entry.offset = meta.readLong();
|
||||
entry.numBytes = meta.readLong();
|
||||
|
@ -155,7 +166,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
|
|||
entry.blockSize = meta.readVInt();
|
||||
}
|
||||
binaries.put(fieldNumber, entry);
|
||||
} else if (fieldType == Lucene42DocValuesConsumer.FST) {
|
||||
} else if (fieldType == FST) {
|
||||
FSTEntry entry = new FSTEntry();
|
||||
entry.offset = meta.readLong();
|
||||
entry.numOrds = meta.readVLong();
|
||||
|
@ -429,6 +440,15 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@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 {
|
||||
|
|
|
@ -0,0 +1,209 @@
|
|||
package org.apache.lucene.codecs.lucene42;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.MathUtil;
|
||||
import org.apache.lucene.util.packed.BlockPackedWriter;
|
||||
import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
* Writer for {@link Lucene42NormsFormat}
|
||||
*/
|
||||
class Lucene42NormsConsumer extends DocValuesConsumer {
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_GCD_COMPRESSION = 1;
|
||||
static final int VERSION_CURRENT = VERSION_GCD_COMPRESSION;
|
||||
|
||||
static final byte NUMBER = 0;
|
||||
|
||||
static final int BLOCK_SIZE = 4096;
|
||||
|
||||
static final byte DELTA_COMPRESSED = 0;
|
||||
static final byte TABLE_COMPRESSED = 1;
|
||||
static final byte UNCOMPRESSED = 2;
|
||||
static final byte GCD_COMPRESSED = 3;
|
||||
|
||||
final IndexOutput data, meta;
|
||||
final int maxDoc;
|
||||
final float acceptableOverheadRatio;
|
||||
|
||||
Lucene42NormsConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension, float acceptableOverheadRatio) throws IOException {
|
||||
this.acceptableOverheadRatio = acceptableOverheadRatio;
|
||||
maxDoc = state.segmentInfo.getDocCount();
|
||||
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, VERSION_CURRENT);
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
meta = state.directory.createOutput(metaName, state.context);
|
||||
CodecUtil.writeHeader(meta, metaCodec, VERSION_CURRENT);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(this);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(NUMBER);
|
||||
meta.writeLong(data.getFilePointer());
|
||||
long minValue = Long.MAX_VALUE;
|
||||
long maxValue = Long.MIN_VALUE;
|
||||
long gcd = 0;
|
||||
// TODO: more efficient?
|
||||
HashSet<Long> uniqueValues = null;
|
||||
if (true) {
|
||||
uniqueValues = new HashSet<>();
|
||||
|
||||
long count = 0;
|
||||
for (Number nv : values) {
|
||||
assert nv != null;
|
||||
final long v = nv.longValue();
|
||||
|
||||
if (gcd != 1) {
|
||||
if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
|
||||
// in that case v - minValue might overflow and make the GCD computation return
|
||||
// wrong results. Since these extreme values are unlikely, we just discard
|
||||
// GCD computation for them
|
||||
gcd = 1;
|
||||
} else if (count != 0) { // minValue needs to be set first
|
||||
gcd = MathUtil.gcd(gcd, v - minValue);
|
||||
}
|
||||
}
|
||||
|
||||
minValue = Math.min(minValue, v);
|
||||
maxValue = Math.max(maxValue, v);
|
||||
|
||||
if (uniqueValues != null) {
|
||||
if (uniqueValues.add(v)) {
|
||||
if (uniqueValues.size() > 256) {
|
||||
uniqueValues = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
++count;
|
||||
}
|
||||
assert count == maxDoc;
|
||||
}
|
||||
|
||||
if (uniqueValues != null) {
|
||||
// small number of unique values
|
||||
final int bitsPerValue = PackedInts.bitsRequired(uniqueValues.size()-1);
|
||||
FormatAndBits formatAndBits = PackedInts.fastestFormatAndBits(maxDoc, bitsPerValue, acceptableOverheadRatio);
|
||||
if (formatAndBits.bitsPerValue == 8 && minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) {
|
||||
meta.writeByte(UNCOMPRESSED); // uncompressed
|
||||
for (Number nv : values) {
|
||||
data.writeByte(nv == null ? 0 : (byte) nv.longValue());
|
||||
}
|
||||
} else {
|
||||
meta.writeByte(TABLE_COMPRESSED); // table-compressed
|
||||
Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]);
|
||||
final HashMap<Long,Integer> encode = new HashMap<Long,Integer>();
|
||||
data.writeVInt(decode.length);
|
||||
for (int i = 0; i < decode.length; i++) {
|
||||
data.writeLong(decode[i]);
|
||||
encode.put(decode[i], i);
|
||||
}
|
||||
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
data.writeVInt(formatAndBits.format.getId());
|
||||
data.writeVInt(formatAndBits.bitsPerValue);
|
||||
|
||||
final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, formatAndBits.format, maxDoc, formatAndBits.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
|
||||
for(Number nv : values) {
|
||||
writer.add(encode.get(nv == null ? 0 : nv.longValue()));
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
} else if (gcd != 0 && gcd != 1) {
|
||||
meta.writeByte(GCD_COMPRESSED);
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
data.writeLong(minValue);
|
||||
data.writeLong(gcd);
|
||||
data.writeVInt(BLOCK_SIZE);
|
||||
|
||||
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
|
||||
for (Number nv : values) {
|
||||
long value = nv == null ? 0 : nv.longValue();
|
||||
writer.add((value - minValue) / gcd);
|
||||
}
|
||||
writer.finish();
|
||||
} else {
|
||||
meta.writeByte(DELTA_COMPRESSED); // delta-compressed
|
||||
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
data.writeVInt(BLOCK_SIZE);
|
||||
|
||||
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
|
||||
for (Number nv : values) {
|
||||
writer.add(nv == null ? 0 : nv.longValue());
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
boolean success = false;
|
||||
try {
|
||||
if (meta != null) {
|
||||
meta.writeVInt(-1); // write EOF marker
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(data, meta);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(data, meta);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addBinaryField(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
|
@ -41,7 +41,7 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
* </ul>
|
||||
* @see Lucene42DocValuesFormat
|
||||
*/
|
||||
public final class Lucene42NormsFormat extends NormsFormat {
|
||||
public class Lucene42NormsFormat extends NormsFormat {
|
||||
final float acceptableOverheadRatio;
|
||||
|
||||
/**
|
||||
|
@ -67,7 +67,7 @@ public final class Lucene42NormsFormat extends NormsFormat {
|
|||
|
||||
@Override
|
||||
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
|
||||
return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
|
||||
return new Lucene42NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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
|
||||
|
@ -17,6 +17,7 @@ package org.apache.lucene.codecs.diskdv;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.Closeable; // javadocs
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
|
@ -36,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 implements Closeable {
|
||||
|
||||
static final int BLOCK_SIZE = 16384;
|
||||
static final int ADDRESS_INTERVAL = 16;
|
||||
|
@ -59,15 +60,16 @@ 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 {
|
||||
/** expert: Creates a new writer */
|
||||
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 {
|
||||
|
@ -87,13 +89,20 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
long minValue = Long.MAX_VALUE;
|
||||
long maxValue = Long.MIN_VALUE;
|
||||
long gcd = 0;
|
||||
boolean missing = false;
|
||||
// TODO: more efficient?
|
||||
HashSet<Long> uniqueValues = null;
|
||||
if (optimizeStorage) {
|
||||
uniqueValues = new HashSet<>();
|
||||
|
||||
for (Number nv : values) {
|
||||
final long v = nv.longValue();
|
||||
final long v;
|
||||
if (nv == null) {
|
||||
v = 0;
|
||||
missing = true;
|
||||
} else {
|
||||
v = nv.longValue();
|
||||
}
|
||||
|
||||
if (gcd != 1) {
|
||||
if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
|
||||
|
@ -138,8 +147,14 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
format = DELTA_COMPRESSED;
|
||||
}
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(DiskDocValuesFormat.NUMERIC);
|
||||
meta.writeByte(Lucene45DocValuesFormat.NUMERIC);
|
||||
meta.writeVInt(format);
|
||||
if (missing) {
|
||||
meta.writeLong(data.getFilePointer());
|
||||
writeMissingBitset(values);
|
||||
} else {
|
||||
meta.writeLong(-1L);
|
||||
}
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
meta.writeLong(data.getFilePointer());
|
||||
meta.writeVLong(count);
|
||||
|
@ -151,14 +166,15 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
meta.writeLong(gcd);
|
||||
final BlockPackedWriter quotientWriter = new BlockPackedWriter(data, BLOCK_SIZE);
|
||||
for (Number nv : values) {
|
||||
quotientWriter.add((nv.longValue() - minValue) / gcd);
|
||||
long value = nv == null ? 0 : nv.longValue();
|
||||
quotientWriter.add((value - minValue) / gcd);
|
||||
}
|
||||
quotientWriter.finish();
|
||||
break;
|
||||
case DELTA_COMPRESSED:
|
||||
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
|
||||
for (Number nv : values) {
|
||||
writer.add(nv.longValue());
|
||||
writer.add(nv == null ? 0 : nv.longValue());
|
||||
}
|
||||
writer.finish();
|
||||
break;
|
||||
|
@ -173,7 +189,7 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
final int bitsRequired = PackedInts.bitsRequired(uniqueValues.size() - 1);
|
||||
final PackedInts.Writer ordsWriter = PackedInts.getWriterNoHeader(data, PackedInts.Format.PACKED, (int) count, bitsRequired, PackedInts.DEFAULT_BUFFER_SIZE);
|
||||
for (Number nv : values) {
|
||||
ordsWriter.add(encode.get(nv.longValue()));
|
||||
ordsWriter.add(encode.get(nv == null ? 0 : nv.longValue()));
|
||||
}
|
||||
ordsWriter.finish();
|
||||
break;
|
||||
|
@ -181,23 +197,60 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: in some cases representing missing with minValue-1 wouldn't take up additional space and so on,
|
||||
// but this is very simple, and algorithms only check this for values of 0 anyway (doesnt slow down normal decode)
|
||||
void writeMissingBitset(Iterable<?> values) throws IOException {
|
||||
byte bits = 0;
|
||||
int count = 0;
|
||||
for (Object v : values) {
|
||||
if (count == 8) {
|
||||
data.writeByte(bits);
|
||||
count = 0;
|
||||
bits = 0;
|
||||
}
|
||||
if (v != null) {
|
||||
bits |= 1 << (count & 7);
|
||||
}
|
||||
count++;
|
||||
}
|
||||
if (count > 0) {
|
||||
data.writeByte(bits);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
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();
|
||||
long count = 0;
|
||||
boolean missing = false;
|
||||
for(BytesRef v : values) {
|
||||
minLength = Math.min(minLength, v.length);
|
||||
maxLength = Math.max(maxLength, v.length);
|
||||
data.writeBytes(v.bytes, v.offset, v.length);
|
||||
final int length;
|
||||
if (v == null) {
|
||||
length = 0;
|
||||
missing = true;
|
||||
} else {
|
||||
length = v.length;
|
||||
}
|
||||
minLength = Math.min(minLength, length);
|
||||
maxLength = Math.max(maxLength, length);
|
||||
if (v != null) {
|
||||
data.writeBytes(v.bytes, v.offset, v.length);
|
||||
}
|
||||
count++;
|
||||
}
|
||||
meta.writeVInt(minLength == maxLength ? BINARY_FIXED_UNCOMPRESSED : BINARY_VARIABLE_UNCOMPRESSED);
|
||||
if (missing) {
|
||||
meta.writeLong(data.getFilePointer());
|
||||
writeMissingBitset(values);
|
||||
} else {
|
||||
meta.writeLong(-1L);
|
||||
}
|
||||
meta.writeVInt(minLength);
|
||||
meta.writeVInt(maxLength);
|
||||
meta.writeVLong(count);
|
||||
|
@ -213,13 +266,16 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
|
||||
long addr = 0;
|
||||
for (BytesRef v : values) {
|
||||
addr += v.length;
|
||||
if (v != null) {
|
||||
addr += v.length;
|
||||
}
|
||||
writer.add(addr);
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
}
|
||||
|
||||
/** expert: writes a value dictionary for a sorted/sortedset field */
|
||||
protected void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
|
||||
// first check if its a "fixed-length" terms dict
|
||||
int minLength = Integer.MAX_VALUE;
|
||||
|
@ -234,8 +290,9 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
} else {
|
||||
// header
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(DiskDocValuesFormat.BINARY);
|
||||
meta.writeByte(Lucene45DocValuesFormat.BINARY);
|
||||
meta.writeVInt(BINARY_PREFIX_COMPRESSED);
|
||||
meta.writeLong(-1L);
|
||||
// now write the bytes: sharing prefixes within a block
|
||||
final long startFP = data.getFilePointer();
|
||||
// currently, we have to store the delta from expected for every 1/nth term
|
||||
|
@ -279,7 +336,7 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
@Override
|
||||
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(DiskDocValuesFormat.SORTED);
|
||||
meta.writeByte(Lucene45DocValuesFormat.SORTED);
|
||||
addTermsDict(field, values);
|
||||
addNumericField(field, docToOrd, false);
|
||||
}
|
||||
|
@ -287,7 +344,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
|
||||
|
@ -296,8 +353,9 @@ 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.writeLong(-1L);
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
meta.writeLong(data.getFilePointer());
|
||||
meta.writeVLong(maxDoc);
|
|
@ -0,0 +1,186 @@
|
|||
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: values are written in chunks of 16, with the first value written
|
||||
* completely and other values sharing prefixes. chunk addresses are written in blocks of 16k,
|
||||
* with the current absolute start for the block, and the average (expected) delta per entry.
|
||||
* For each chunk the deviation from the delta (actual - expected) is written.
|
||||
* </ul>
|
||||
* <p>
|
||||
* {@link DocValuesType#SORTED SORTED}:
|
||||
* <ul>
|
||||
* <li>Sorted: a mapping of ordinals to deduplicated terms is written as Prefix-Compressed Binary,
|
||||
* along with the per-document ordinals written using one of the numeric strategies above.
|
||||
* </ul>
|
||||
* <p>
|
||||
* {@link DocValuesType#SORTED_SET SORTED_SET}:
|
||||
* <ul>
|
||||
* <li>SortedSet: a mapping of ordinals to deduplicated terms is written as Prefix-Compressed Binary,
|
||||
* an ordinal list and per-document index into this list are written using the numeric 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,<Entry><sup>NumFields</sup></p>
|
||||
* <ul>
|
||||
* <li>Entry --> NumericEntry | BinaryEntry | SortedEntry | SortedSetEntry</li>
|
||||
* <li>NumericEntry --> GCDNumericEntry | TableNumericEntry | DeltaNumericEntry</li>
|
||||
* <li>GCDNumericEntry --> NumericHeader,MinValue,GCD</li>
|
||||
* <li>TableNumericEntry --> NumericHeader,TableSize,{@link DataOutput#writeLong Int64}<sup>TableSize</sup></li>
|
||||
* <li>DeltaNumericEntry --> NumericHeader</li>
|
||||
* <li>NumericHeader --> FieldNumber,EntryType,NumericType,MissingOffset,PackedVersion,DataOffset,Count,BlockSize</li>
|
||||
* <li>BinaryEntry --> FixedBinaryEntry | VariableBinaryEntry | PrefixBinaryEntry</li>
|
||||
* <li>FixedBinaryEntry --> BinaryHeader</li>
|
||||
* <li>VariableBinaryEntry --> BinaryHeader,AddressOffset,PackedVersion,BlockSize</li>
|
||||
* <li>PrefixBinaryEntry --> BinaryHeader,AddressInterval,AddressOffset,PackedVersion,BlockSize</li>
|
||||
* <li>BinaryHeader --> FieldNumber,EntryType,BinaryType,MissingOffset,MinLength,MaxLength,DataOffset</li>
|
||||
* <li>SortedEntry --> FieldNumber,EntryType,BinaryEntry,NumericEntry</li>
|
||||
* <li>SortedSetEntry --> EntryType,BinaryEntry,NumericEntry,NumericEntry</li>
|
||||
* <li>FieldNumber,PackedVersion,MinLength,MaxLength,BlockSize,ValueCount --> {@link DataOutput#writeVInt VInt}</li>
|
||||
* <li>EntryType,CompressionType --> {@link DataOutput#writeByte Byte}</li>
|
||||
* <li>Header --> {@link CodecUtil#writeHeader CodecHeader}</li>
|
||||
* <li>MinValue,GCD,MissingOffset,AddressOffset,DataOffset --> {@link DataOutput#writeLong Int64}</li>
|
||||
* <li>TableSize --> {@link DataOutput#writeVInt vInt}</li>
|
||||
* </ul>
|
||||
* <p>Sorted fields have two entries: a BinaryEntry with the value metadata,
|
||||
* and an ordinary NumericEntry for the document-to-ord metadata.</p>
|
||||
* <p>SortedSet fields have three entries: a BinaryEntry with the value metadata,
|
||||
* and two NumericEntries for the document-to-ord-index and ordinal list metadata.</p>
|
||||
* <p>FieldNumber of -1 indicates the end of metadata.</p>
|
||||
* <p>EntryType is a 0 (NumericEntry) or 1 (BinaryEntry)</p>
|
||||
* <p>DataOffset is the pointer to the start of the data in the DocValues data (.dvd)</p>
|
||||
* <p>NumericType 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>BinaryType indicates how Binary values will be stored:
|
||||
* <ul>
|
||||
* <li>0 --> fixed-width. All values have the same length, addressing by multiplication.
|
||||
* <li>1 -->, variable-width. An address for each value is stored.
|
||||
* <li>2 --> prefix-compressed. An address to the start of every interval'th value is stored.
|
||||
* </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.
|
||||
* <p>MissingOffset points to a byte[] containing a bitset of all documents that had a value for the field.
|
||||
* If its -1, then there are no missing values.
|
||||
* <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 --> {@link PackedInts PackedInts}</li>
|
||||
* <li>GCDCompressedNumerics --> {@link BlockPackedWriter BlockPackedInts(blockSize=16k)}</li>
|
||||
* <li>Addresses --> {@link MonotonicBlockPackedWriter MonotonicBlockPackedInts(blockSize=16k)}</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
|
||||
*/
|
||||
public final class Lucene45DocValuesFormat extends DocValuesFormat {
|
||||
|
||||
/** Sole Constructor */
|
||||
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);
|
||||
}
|
||||
|
||||
static final String DATA_CODEC = "Lucene45DocValuesData";
|
||||
static final String DATA_EXTENSION = "dvd";
|
||||
static final String META_CODEC = "Lucene45ValuesMetadata";
|
||||
static final String META_EXTENSION = "dvm";
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_CURRENT = VERSION_START;
|
||||
static final byte NUMERIC = 0;
|
||||
static final byte BINARY = 1;
|
||||
static final byte SORTED = 2;
|
||||
static final byte SORTED_SET = 3;
|
||||
}
|
|
@ -0,0 +1,831 @@
|
|||
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.Closeable; // javadocs
|
||||
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;
|
||||
|
||||
/** reader for {@link Lucene45DocValuesFormat} */
|
||||
public class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
|
||||
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>();
|
||||
|
||||
/** expert: instantiates a new reader */
|
||||
protected 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.missingOffset = meta.readLong();
|
||||
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.missingOffset = meta.readLong();
|
||||
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);
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/** returns an address instance for variable-length binary values.
|
||||
* @lucene.internal */
|
||||
protected MonotonicBlockPackedReader getAddressInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
|
||||
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 addresses;
|
||||
}
|
||||
|
||||
private BinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
|
||||
final IndexInput data = this.data.clone();
|
||||
|
||||
final MonotonicBlockPackedReader addresses = getAddressInstance(data, field, bytes);
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/** returns an address instance for prefix-compressed binary values.
|
||||
* @lucene.internal */
|
||||
protected MonotonicBlockPackedReader getIntervalInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
|
||||
final MonotonicBlockPackedReader addresses;
|
||||
final long interval = bytes.addressInterval;
|
||||
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 addresses;
|
||||
}
|
||||
|
||||
|
||||
private BinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
|
||||
final IndexInput data = this.data.clone();
|
||||
|
||||
final MonotonicBlockPackedReader addresses = getIntervalInstance(data, field, bytes);
|
||||
|
||||
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();
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/** returns an address instance for sortedset ordinal lists
|
||||
* @lucene.internal */
|
||||
protected MonotonicBlockPackedReader getOrdIndexInstance(IndexInput data, FieldInfo field, NumericEntry entry) throws IOException {
|
||||
final MonotonicBlockPackedReader ordIndex;
|
||||
synchronized (ordIndexInstances) {
|
||||
MonotonicBlockPackedReader ordIndexInstance = ordIndexInstances.get(field.number);
|
||||
if (ordIndexInstance == null) {
|
||||
data.seek(entry.offset);
|
||||
ordIndexInstance = new MonotonicBlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, false);
|
||||
ordIndexInstances.put(field.number, ordIndexInstance);
|
||||
}
|
||||
ordIndex = ordIndexInstance;
|
||||
}
|
||||
return ordIndex;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
|
||||
final IndexInput data = this.data.clone();
|
||||
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 = getOrdIndexInstance(data, field, ordIndexes.get(field.number));
|
||||
|
||||
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();
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private Bits getMissingBits(final long offset) throws IOException {
|
||||
if (offset == -1) {
|
||||
return new Bits.MatchAllBits(maxDoc);
|
||||
} else {
|
||||
final IndexInput in = data.clone();
|
||||
return new Bits() {
|
||||
|
||||
@Override
|
||||
public boolean get(int index) {
|
||||
try {
|
||||
in.seek(offset + (index >> 3));
|
||||
return (in.readByte() & (1 << (index & 7))) != 0;
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return maxDoc;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
switch(field.getDocValuesType()) {
|
||||
case SORTED_SET:
|
||||
return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
|
||||
case SORTED:
|
||||
return new SortedDocsWithField(getSorted(field), maxDoc);
|
||||
case BINARY:
|
||||
BinaryEntry be = binaries.get(field.number);
|
||||
return getMissingBits(be.missingOffset);
|
||||
case NUMERIC:
|
||||
NumericEntry ne = numerics.get(field.number);
|
||||
return getMissingBits(ne.missingOffset);
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
data.close();
|
||||
}
|
||||
|
||||
/** metadata entry for a numeric docvalues field */
|
||||
protected static class NumericEntry {
|
||||
private NumericEntry() {}
|
||||
/** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
|
||||
long missingOffset;
|
||||
/** offset to the actual numeric values */
|
||||
public long offset;
|
||||
|
||||
int format;
|
||||
/** packed ints version used to encode these numerics */
|
||||
public int packedIntsVersion;
|
||||
/** count of values written */
|
||||
public long count;
|
||||
/** packed ints blocksize */
|
||||
public int blockSize;
|
||||
|
||||
long minValue;
|
||||
long gcd;
|
||||
long table[];
|
||||
}
|
||||
|
||||
/** metadata entry for a binary docvalues field */
|
||||
protected static class BinaryEntry {
|
||||
private BinaryEntry() {}
|
||||
/** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
|
||||
long missingOffset;
|
||||
/** offset to the actual binary values */
|
||||
long offset;
|
||||
|
||||
int format;
|
||||
/** count of values written */
|
||||
public long count;
|
||||
int minLength;
|
||||
int maxLength;
|
||||
/** offset to the addressing data that maps a value to its slice of the byte[] */
|
||||
public long addressesOffset;
|
||||
/** interval of shared prefix chunks (when using prefix-compressed binary) */
|
||||
public long addressInterval;
|
||||
/** packed ints version used to encode addressing information */
|
||||
public int packedIntsVersion;
|
||||
/** packed ints blocksize */
|
||||
public 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>
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.lucene.index.SegmentReadState;
|
|||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
|
@ -265,6 +266,12 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
|
|||
DocValuesProducer producer = fields.get(field.name);
|
||||
return producer == null ? null : producer.getSortedSet(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
DocValuesProducer producer = fields.get(field.name);
|
||||
return producer == null ? null : producer.getDocsWithField(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
|
|
@ -207,6 +207,12 @@ public abstract class AtomicReader extends IndexReader {
|
|||
* this field. The returned instance should only be
|
||||
* used by a single thread. */
|
||||
public abstract SortedSetDocValues getSortedSetDocValues(String field) throws IOException;
|
||||
|
||||
/** Returns a {@link Bits} at the size of <code>reader.maxDoc()</code>,
|
||||
* with turned on bits for each docid that does have a value for this field,
|
||||
* or null if no DocValues were indexed for this field. The
|
||||
* returned instance should only be used by a single thread */
|
||||
public abstract Bits getDocsWithField(String field) throws IOException;
|
||||
|
||||
/** Returns {@link NumericDocValues} representing norms
|
||||
* for this field, or null if no {@link NumericDocValues}
|
||||
|
|
|
@ -26,6 +26,8 @@ import org.apache.lucene.store.DataInput;
|
|||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.OpenBitSet;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
@ -49,6 +51,7 @@ class BinaryDocValuesWriter extends DocValuesWriter {
|
|||
|
||||
private final Counter iwBytesUsed;
|
||||
private final AppendingDeltaPackedLongBuffer lengths;
|
||||
private final OpenBitSet docsWithField;
|
||||
private final FieldInfo fieldInfo;
|
||||
private int addedValues;
|
||||
private long bytesUsed;
|
||||
|
@ -59,6 +62,9 @@ class BinaryDocValuesWriter extends DocValuesWriter {
|
|||
this.bytesOut = bytes.getDataOutput();
|
||||
this.lengths = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
|
||||
this.iwBytesUsed = iwBytesUsed;
|
||||
this.docsWithField = new OpenBitSet();
|
||||
this.bytesUsed = docsWithFieldBytesUsed();
|
||||
iwBytesUsed.addAndGet(bytesUsed);
|
||||
}
|
||||
|
||||
public void addValue(int docID, BytesRef value) {
|
||||
|
@ -85,11 +91,17 @@ class BinaryDocValuesWriter extends DocValuesWriter {
|
|||
// Should never happen!
|
||||
throw new RuntimeException(ioe);
|
||||
}
|
||||
docsWithField.set(docID);
|
||||
updateBytesUsed();
|
||||
}
|
||||
|
||||
private long docsWithFieldBytesUsed() {
|
||||
// size of the long[] + some overhead
|
||||
return RamUsageEstimator.sizeOf(docsWithField.getBits()) + 64;
|
||||
}
|
||||
|
||||
private void updateBytesUsed() {
|
||||
final long newBytesUsed = lengths.ramBytesUsed() + bytes.ramBytesUsed();
|
||||
final long newBytesUsed = lengths.ramBytesUsed() + bytes.ramBytesUsed() + docsWithFieldBytesUsed();
|
||||
iwBytesUsed.addAndGet(newBytesUsed - bytesUsed);
|
||||
bytesUsed = newBytesUsed;
|
||||
}
|
||||
|
@ -138,6 +150,7 @@ class BinaryDocValuesWriter extends DocValuesWriter {
|
|||
if (!hasNext()) {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
final BytesRef v;
|
||||
if (upto < size) {
|
||||
int length = (int) lengthsIterator.next();
|
||||
value.grow(length);
|
||||
|
@ -148,13 +161,16 @@ class BinaryDocValuesWriter extends DocValuesWriter {
|
|||
// Should never happen!
|
||||
throw new RuntimeException(ioe);
|
||||
}
|
||||
if (docsWithField.get(upto)) {
|
||||
v = value;
|
||||
} else {
|
||||
v = null;
|
||||
}
|
||||
} else {
|
||||
// This is to handle last N documents not having
|
||||
// this DV field in the end of the segment:
|
||||
value.length = 0;
|
||||
v = null;
|
||||
}
|
||||
upto++;
|
||||
return value;
|
||||
return v;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -1280,7 +1280,8 @@ public class CheckIndex {
|
|||
if (reader.getBinaryDocValues(fieldInfo.name) != null ||
|
||||
reader.getNumericDocValues(fieldInfo.name) != null ||
|
||||
reader.getSortedDocValues(fieldInfo.name) != null ||
|
||||
reader.getSortedSetDocValues(fieldInfo.name) != null) {
|
||||
reader.getSortedSetDocValues(fieldInfo.name) != null ||
|
||||
reader.getDocsWithField(fieldInfo.name) != null) {
|
||||
throw new RuntimeException("field: " + fieldInfo.name + " has docvalues but should omit them!");
|
||||
}
|
||||
}
|
||||
|
@ -1301,26 +1302,37 @@ public class CheckIndex {
|
|||
return status;
|
||||
}
|
||||
|
||||
private static void checkBinaryDocValues(String fieldName, AtomicReader reader, BinaryDocValues dv) {
|
||||
private static void checkBinaryDocValues(String fieldName, AtomicReader reader, BinaryDocValues dv, Bits docsWithField) {
|
||||
BytesRef scratch = new BytesRef();
|
||||
for (int i = 0; i < reader.maxDoc(); i++) {
|
||||
dv.get(i, scratch);
|
||||
assert scratch.isValid();
|
||||
if (docsWithField.get(i) == false && scratch.length > 0) {
|
||||
throw new RuntimeException("dv for field: " + fieldName + " is missing but has value=" + scratch + " for doc: " + i);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static void checkSortedDocValues(String fieldName, AtomicReader reader, SortedDocValues dv) {
|
||||
checkBinaryDocValues(fieldName, reader, dv);
|
||||
private static void checkSortedDocValues(String fieldName, AtomicReader reader, SortedDocValues dv, Bits docsWithField) {
|
||||
checkBinaryDocValues(fieldName, reader, dv, docsWithField);
|
||||
final int maxOrd = dv.getValueCount()-1;
|
||||
FixedBitSet seenOrds = new FixedBitSet(dv.getValueCount());
|
||||
int maxOrd2 = -1;
|
||||
for (int i = 0; i < reader.maxDoc(); i++) {
|
||||
int ord = dv.getOrd(i);
|
||||
if (ord < 0 || ord > maxOrd) {
|
||||
if (ord == -1) {
|
||||
if (docsWithField.get(i)) {
|
||||
throw new RuntimeException("dv for field: " + fieldName + " has -1 ord but is not marked missing for doc: " + i);
|
||||
}
|
||||
} else if (ord < -1 || ord > maxOrd) {
|
||||
throw new RuntimeException("ord out of bounds: " + ord);
|
||||
} else {
|
||||
if (!docsWithField.get(i)) {
|
||||
throw new RuntimeException("dv for field: " + fieldName + " is missing but has ord=" + ord + " for doc: " + i);
|
||||
}
|
||||
maxOrd2 = Math.max(maxOrd2, ord);
|
||||
seenOrds.set(ord);
|
||||
}
|
||||
maxOrd2 = Math.max(maxOrd2, ord);
|
||||
seenOrds.set(ord);
|
||||
}
|
||||
if (maxOrd != maxOrd2) {
|
||||
throw new RuntimeException("dv for field: " + fieldName + " reports wrong maxOrd=" + maxOrd + " but this is not the case: " + maxOrd2);
|
||||
|
@ -1342,7 +1354,7 @@ public class CheckIndex {
|
|||
}
|
||||
}
|
||||
|
||||
private static void checkSortedSetDocValues(String fieldName, AtomicReader reader, SortedSetDocValues dv) {
|
||||
private static void checkSortedSetDocValues(String fieldName, AtomicReader reader, SortedSetDocValues dv, Bits docsWithField) {
|
||||
final long maxOrd = dv.getValueCount()-1;
|
||||
OpenBitSet seenOrds = new OpenBitSet(dv.getValueCount());
|
||||
long maxOrd2 = -1;
|
||||
|
@ -1350,16 +1362,28 @@ public class CheckIndex {
|
|||
dv.setDocument(i);
|
||||
long lastOrd = -1;
|
||||
long ord;
|
||||
while ((ord = dv.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
|
||||
if (ord <= lastOrd) {
|
||||
throw new RuntimeException("ords out of order: " + ord + " <= " + lastOrd + " for doc: " + i);
|
||||
if (docsWithField.get(i)) {
|
||||
int ordCount = 0;
|
||||
while ((ord = dv.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
|
||||
ordCount++;
|
||||
if (ord <= lastOrd) {
|
||||
throw new RuntimeException("ords out of order: " + ord + " <= " + lastOrd + " for doc: " + i);
|
||||
}
|
||||
if (ord < 0 || ord > maxOrd) {
|
||||
throw new RuntimeException("ord out of bounds: " + ord);
|
||||
}
|
||||
lastOrd = ord;
|
||||
maxOrd2 = Math.max(maxOrd2, ord);
|
||||
seenOrds.set(ord);
|
||||
}
|
||||
if (ord < 0 || ord > maxOrd) {
|
||||
throw new RuntimeException("ord out of bounds: " + ord);
|
||||
if (ordCount == 0) {
|
||||
throw new RuntimeException("dv for field: " + fieldName + " has no ordinals but is not marked missing for doc: " + i);
|
||||
}
|
||||
} else {
|
||||
long o = dv.nextOrd();
|
||||
if (o != SortedSetDocValues.NO_MORE_ORDS) {
|
||||
throw new RuntimeException("dv for field: " + fieldName + " is marked missing but has ord=" + o + " for doc: " + i);
|
||||
}
|
||||
lastOrd = ord;
|
||||
maxOrd2 = Math.max(maxOrd2, ord);
|
||||
seenOrds.set(ord);
|
||||
}
|
||||
}
|
||||
if (maxOrd != maxOrd2) {
|
||||
|
@ -1383,17 +1407,26 @@ public class CheckIndex {
|
|||
}
|
||||
}
|
||||
|
||||
private static void checkNumericDocValues(String fieldName, AtomicReader reader, NumericDocValues ndv) {
|
||||
private static void checkNumericDocValues(String fieldName, AtomicReader reader, NumericDocValues ndv, Bits docsWithField) {
|
||||
for (int i = 0; i < reader.maxDoc(); i++) {
|
||||
ndv.get(i);
|
||||
long value = ndv.get(i);
|
||||
if (docsWithField.get(i) == false && value != 0) {
|
||||
throw new RuntimeException("dv for field: " + fieldName + " is marked missing but has value=" + value + " for doc: " + i);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static void checkDocValues(FieldInfo fi, AtomicReader reader, PrintStream infoStream, DocValuesStatus status) throws Exception {
|
||||
Bits docsWithField = reader.getDocsWithField(fi.name);
|
||||
if (docsWithField == null) {
|
||||
throw new RuntimeException(fi.name + " docsWithField does not exist");
|
||||
} else if (docsWithField.length() != reader.maxDoc()) {
|
||||
throw new RuntimeException(fi.name + " docsWithField has incorrect length: " + docsWithField.length() + ",expected: " + reader.maxDoc());
|
||||
}
|
||||
switch(fi.getDocValuesType()) {
|
||||
case SORTED:
|
||||
status.totalSortedFields++;
|
||||
checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name));
|
||||
checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name), docsWithField);
|
||||
if (reader.getBinaryDocValues(fi.name) != null ||
|
||||
reader.getNumericDocValues(fi.name) != null ||
|
||||
reader.getSortedSetDocValues(fi.name) != null) {
|
||||
|
@ -1402,7 +1435,7 @@ public class CheckIndex {
|
|||
break;
|
||||
case SORTED_SET:
|
||||
status.totalSortedSetFields++;
|
||||
checkSortedSetDocValues(fi.name, reader, reader.getSortedSetDocValues(fi.name));
|
||||
checkSortedSetDocValues(fi.name, reader, reader.getSortedSetDocValues(fi.name), docsWithField);
|
||||
if (reader.getBinaryDocValues(fi.name) != null ||
|
||||
reader.getNumericDocValues(fi.name) != null ||
|
||||
reader.getSortedDocValues(fi.name) != null) {
|
||||
|
@ -1411,7 +1444,7 @@ public class CheckIndex {
|
|||
break;
|
||||
case BINARY:
|
||||
status.totalBinaryFields++;
|
||||
checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name));
|
||||
checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name), docsWithField);
|
||||
if (reader.getNumericDocValues(fi.name) != null ||
|
||||
reader.getSortedDocValues(fi.name) != null ||
|
||||
reader.getSortedSetDocValues(fi.name) != null) {
|
||||
|
@ -1420,7 +1453,7 @@ public class CheckIndex {
|
|||
break;
|
||||
case NUMERIC:
|
||||
status.totalNumericFields++;
|
||||
checkNumericDocValues(fi.name, reader, reader.getNumericDocValues(fi.name));
|
||||
checkNumericDocValues(fi.name, reader, reader.getNumericDocValues(fi.name), docsWithField);
|
||||
if (reader.getBinaryDocValues(fi.name) != null ||
|
||||
reader.getSortedDocValues(fi.name) != null ||
|
||||
reader.getSortedSetDocValues(fi.name) != null) {
|
||||
|
@ -1435,7 +1468,7 @@ public class CheckIndex {
|
|||
private static void checkNorms(FieldInfo fi, AtomicReader reader, PrintStream infoStream) throws IOException {
|
||||
switch(fi.getNormType()) {
|
||||
case NUMERIC:
|
||||
checkNumericDocValues(fi.name, reader, reader.getNormValues(fi.name));
|
||||
checkNumericDocValues(fi.name, reader, reader.getNormValues(fi.name), new Bits.MatchAllBits(reader.maxDoc()));
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError("wtf: " + fi.getNormType());
|
||||
|
|
|
@ -143,7 +143,7 @@ final class DocValuesProcessor extends StoredFieldsConsumer {
|
|||
DocValuesWriter writer = writers.get(fieldInfo.name);
|
||||
NumericDocValuesWriter numericWriter;
|
||||
if (writer == null) {
|
||||
numericWriter = new NumericDocValuesWriter(fieldInfo, bytesUsed);
|
||||
numericWriter = new NumericDocValuesWriter(fieldInfo, bytesUsed, true);
|
||||
writers.put(fieldInfo.name, numericWriter);
|
||||
} else if (!(writer instanceof NumericDocValuesWriter)) {
|
||||
throw new IllegalArgumentException("Incompatible DocValues type: field \"" + fieldInfo.name + "\" changed from " + getTypeDesc(writer) + " to numeric");
|
||||
|
|
|
@ -414,4 +414,10 @@ public class FilterAtomicReader extends AtomicReader {
|
|||
return in.getNormValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
ensureOpen();
|
||||
return in.getDocsWithField(field);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.List;
|
|||
|
||||
import org.apache.lucene.index.MultiTermsEnum.TermsEnumIndex;
|
||||
import org.apache.lucene.index.MultiTermsEnum.TermsEnumWithSlice;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
|
||||
import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
|
||||
|
@ -135,6 +136,51 @@ public class MultiDocValues {
|
|||
};
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns a Bits for a reader's docsWithField (potentially merging on-the-fly)
|
||||
* <p>
|
||||
* This is a slow way to access this bitset. Instead, access them per-segment
|
||||
* with {@link AtomicReader#getDocsWithField(String)}
|
||||
* </p>
|
||||
* */
|
||||
public static Bits getDocsWithField(final IndexReader r, final String field) throws IOException {
|
||||
final List<AtomicReaderContext> leaves = r.leaves();
|
||||
final int size = leaves.size();
|
||||
if (size == 0) {
|
||||
return null;
|
||||
} else if (size == 1) {
|
||||
return leaves.get(0).reader().getDocsWithField(field);
|
||||
}
|
||||
|
||||
boolean anyReal = false;
|
||||
boolean anyMissing = false;
|
||||
final Bits[] values = new Bits[size];
|
||||
final int[] starts = new int[size+1];
|
||||
for (int i = 0; i < size; i++) {
|
||||
AtomicReaderContext context = leaves.get(i);
|
||||
Bits v = context.reader().getDocsWithField(field);
|
||||
if (v == null) {
|
||||
v = new Bits.MatchNoBits(context.reader().maxDoc());
|
||||
anyMissing = true;
|
||||
} else {
|
||||
anyReal = true;
|
||||
if (v instanceof Bits.MatchAllBits == false) {
|
||||
anyMissing = true;
|
||||
}
|
||||
}
|
||||
values[i] = v;
|
||||
starts[i] = context.docBase;
|
||||
}
|
||||
starts[size] = r.maxDoc();
|
||||
|
||||
if (!anyReal) {
|
||||
return null;
|
||||
} else if (!anyMissing) {
|
||||
return new Bits.MatchAllBits(r.maxDoc());
|
||||
} else {
|
||||
return new MultiBits(values, starts, false);
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns a BinaryDocValues for a reader's docvalues (potentially merging on-the-fly)
|
||||
* <p>
|
||||
|
|
|
@ -44,7 +44,7 @@ final class NormsConsumerPerField extends InvertedDocEndConsumerPerField impleme
|
|||
if (fieldInfo.isIndexed() && !fieldInfo.omitsNorms()) {
|
||||
if (consumer == null) {
|
||||
fieldInfo.setNormValueType(FieldInfo.DocValuesType.NUMERIC);
|
||||
consumer = new NumericDocValuesWriter(fieldInfo, docState.docWriter.bytesUsed);
|
||||
consumer = new NumericDocValuesWriter(fieldInfo, docState.docWriter.bytesUsed, false);
|
||||
}
|
||||
consumer.addValue(docState.docID, similarity.computeNorm(fieldState));
|
||||
}
|
||||
|
|
|
@ -23,6 +23,8 @@ import java.util.NoSuchElementException;
|
|||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.OpenBitSet;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
|
@ -35,14 +37,18 @@ class NumericDocValuesWriter extends DocValuesWriter {
|
|||
private AppendingDeltaPackedLongBuffer pending;
|
||||
private final Counter iwBytesUsed;
|
||||
private long bytesUsed;
|
||||
private final OpenBitSet docsWithField;
|
||||
private final FieldInfo fieldInfo;
|
||||
private final boolean trackDocsWithField;
|
||||
|
||||
public NumericDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
|
||||
public NumericDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed, boolean trackDocsWithField) {
|
||||
pending = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
|
||||
bytesUsed = pending.ramBytesUsed();
|
||||
docsWithField = new OpenBitSet();
|
||||
bytesUsed = pending.ramBytesUsed() + docsWithFieldBytesUsed();
|
||||
this.fieldInfo = fieldInfo;
|
||||
this.iwBytesUsed = iwBytesUsed;
|
||||
iwBytesUsed.addAndGet(bytesUsed);
|
||||
this.trackDocsWithField = trackDocsWithField;
|
||||
}
|
||||
|
||||
public void addValue(int docID, long value) {
|
||||
|
@ -56,12 +62,20 @@ class NumericDocValuesWriter extends DocValuesWriter {
|
|||
}
|
||||
|
||||
pending.add(value);
|
||||
if (trackDocsWithField) {
|
||||
docsWithField.set(docID);
|
||||
}
|
||||
|
||||
updateBytesUsed();
|
||||
}
|
||||
|
||||
private long docsWithFieldBytesUsed() {
|
||||
// size of the long[] + some overhead
|
||||
return RamUsageEstimator.sizeOf(docsWithField.getBits()) + 64;
|
||||
}
|
||||
|
||||
private void updateBytesUsed() {
|
||||
final long newBytesUsed = pending.ramBytesUsed();
|
||||
final long newBytesUsed = pending.ramBytesUsed() + docsWithFieldBytesUsed();
|
||||
iwBytesUsed.addAndGet(newBytesUsed - bytesUsed);
|
||||
bytesUsed = newBytesUsed;
|
||||
}
|
||||
|
@ -109,14 +123,18 @@ class NumericDocValuesWriter extends DocValuesWriter {
|
|||
if (!hasNext()) {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
long value;
|
||||
Long value;
|
||||
if (upto < size) {
|
||||
value = iter.next();
|
||||
long v = iter.next();
|
||||
if (!trackDocsWithField || docsWithField.get(upto)) {
|
||||
value = v;
|
||||
} else {
|
||||
value = null;
|
||||
}
|
||||
} else {
|
||||
value = 0;
|
||||
value = trackDocsWithField ? null : MISSING;
|
||||
}
|
||||
upto++;
|
||||
// TODO: make reusable Number
|
||||
return value;
|
||||
}
|
||||
|
||||
|
|
|
@ -285,6 +285,13 @@ public class ParallelAtomicReader extends AtomicReader {
|
|||
return reader == null ? null : reader.getSortedSetDocValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
ensureOpen();
|
||||
AtomicReader reader = fieldToReader.get(field);
|
||||
return reader == null ? null : reader.getDocsWithField(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NumericDocValues getNormValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.lucene.index.SegmentReader.CoreClosedListener;
|
|||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.CloseableThreadLocal;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
|
@ -87,6 +88,13 @@ final class SegmentCoreReaders {
|
|||
return new HashMap<String,Object>();
|
||||
}
|
||||
};
|
||||
|
||||
final CloseableThreadLocal<Map<String,Bits>> docsWithFieldLocal = new CloseableThreadLocal<Map<String,Bits>>() {
|
||||
@Override
|
||||
protected Map<String,Bits> initialValue() {
|
||||
return new HashMap<String,Bits>();
|
||||
}
|
||||
};
|
||||
|
||||
final CloseableThreadLocal<Map<String,Object>> normsLocal = new CloseableThreadLocal<Map<String,Object>>() {
|
||||
@Override
|
||||
|
@ -274,6 +282,30 @@ final class SegmentCoreReaders {
|
|||
|
||||
return dvs;
|
||||
}
|
||||
|
||||
Bits getDocsWithField(String field) throws IOException {
|
||||
FieldInfo fi = fieldInfos.fieldInfo(field);
|
||||
if (fi == null) {
|
||||
// Field does not exist
|
||||
return null;
|
||||
}
|
||||
if (fi.getDocValuesType() == null) {
|
||||
// Field was not indexed with doc values
|
||||
return null;
|
||||
}
|
||||
|
||||
assert dvProducer != null;
|
||||
|
||||
Map<String,Bits> dvFields = docsWithFieldLocal.get();
|
||||
|
||||
Bits dvs = dvFields.get(field);
|
||||
if (dvs == null) {
|
||||
dvs = dvProducer.getDocsWithField(fi);
|
||||
dvFields.put(field, dvs);
|
||||
}
|
||||
|
||||
return dvs;
|
||||
}
|
||||
|
||||
NumericDocValues getNormValues(String field) throws IOException {
|
||||
FieldInfo fi = fieldInfos.fieldInfo(field);
|
||||
|
@ -300,8 +332,8 @@ final class SegmentCoreReaders {
|
|||
|
||||
void decRef() throws IOException {
|
||||
if (ref.decrementAndGet() == 0) {
|
||||
IOUtils.close(termVectorsLocal, fieldsReaderLocal, docValuesLocal, normsLocal, fields, dvProducer,
|
||||
termVectorsReaderOrig, fieldsReaderOrig, cfsReader, normsProducer);
|
||||
IOUtils.close(termVectorsLocal, fieldsReaderLocal, docValuesLocal, normsLocal, docsWithFieldLocal, fields,
|
||||
dvProducer, termVectorsReaderOrig, fieldsReaderOrig, cfsReader, normsProducer);
|
||||
notifyCoreClosedListeners();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.lucene.codecs.TermVectorsWriter;
|
|||
import org.apache.lucene.index.FieldInfo.DocValuesType;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
|
@ -156,24 +157,32 @@ final class SegmentMerger {
|
|||
if (type != null) {
|
||||
if (type == DocValuesType.NUMERIC) {
|
||||
List<NumericDocValues> toMerge = new ArrayList<NumericDocValues>();
|
||||
List<Bits> docsWithField = new ArrayList<Bits>();
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
NumericDocValues values = reader.getNumericDocValues(field.name);
|
||||
Bits bits = reader.getDocsWithField(field.name);
|
||||
if (values == null) {
|
||||
values = NumericDocValues.EMPTY;
|
||||
bits = new Bits.MatchNoBits(reader.maxDoc());
|
||||
}
|
||||
toMerge.add(values);
|
||||
docsWithField.add(bits);
|
||||
}
|
||||
consumer.mergeNumericField(field, mergeState, toMerge);
|
||||
consumer.mergeNumericField(field, mergeState, toMerge, docsWithField);
|
||||
} else if (type == DocValuesType.BINARY) {
|
||||
List<BinaryDocValues> toMerge = new ArrayList<BinaryDocValues>();
|
||||
List<Bits> docsWithField = new ArrayList<Bits>();
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
BinaryDocValues values = reader.getBinaryDocValues(field.name);
|
||||
Bits bits = reader.getDocsWithField(field.name);
|
||||
if (values == null) {
|
||||
values = BinaryDocValues.EMPTY;
|
||||
bits = new Bits.MatchNoBits(reader.maxDoc());
|
||||
}
|
||||
toMerge.add(values);
|
||||
docsWithField.add(bits);
|
||||
}
|
||||
consumer.mergeBinaryField(field, mergeState, toMerge);
|
||||
consumer.mergeBinaryField(field, mergeState, toMerge, docsWithField);
|
||||
} else if (type == DocValuesType.SORTED) {
|
||||
List<SortedDocValues> toMerge = new ArrayList<SortedDocValues>();
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
|
@ -216,14 +225,16 @@ final class SegmentMerger {
|
|||
for (FieldInfo field : mergeState.fieldInfos) {
|
||||
if (field.hasNorms()) {
|
||||
List<NumericDocValues> toMerge = new ArrayList<NumericDocValues>();
|
||||
List<Bits> docsWithField = new ArrayList<Bits>();
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
NumericDocValues norms = reader.getNormValues(field.name);
|
||||
if (norms == null) {
|
||||
norms = NumericDocValues.EMPTY;
|
||||
}
|
||||
toMerge.add(norms);
|
||||
docsWithField.add(new Bits.MatchAllBits(reader.maxDoc()));
|
||||
}
|
||||
consumer.mergeNumericField(field, mergeState, toMerge);
|
||||
consumer.mergeNumericField(field, mergeState, toMerge, docsWithField);
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
|
|
|
@ -223,6 +223,12 @@ public final class SegmentReader extends AtomicReader {
|
|||
return core.getNumericDocValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
ensureOpen();
|
||||
return core.getDocsWithField(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BinaryDocValues getBinaryDocValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
|
|
|
@ -91,6 +91,12 @@ public final class SlowCompositeReaderWrapper extends AtomicReader {
|
|||
return MultiDocValues.getNumericValues(in, field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
ensureOpen();
|
||||
return MultiDocValues.getDocsWithField(in, field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BinaryDocValues getBinaryDocValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
|
|
|
@ -37,12 +37,13 @@ public abstract class SortedDocValues extends BinaryDocValues {
|
|||
* Returns the ordinal for the specified docID.
|
||||
* @param docID document ID to lookup
|
||||
* @return ordinal for the document: this is dense, starts at 0, then
|
||||
* increments by 1 for the next value in sorted order.
|
||||
* increments by 1 for the next value in sorted order. Note that
|
||||
* missing values are indicated by -1.
|
||||
*/
|
||||
public abstract int getOrd(int docID);
|
||||
|
||||
/** Retrieves the value for the specified ordinal.
|
||||
* @param ord ordinal to lookup
|
||||
* @param ord ordinal to lookup (must be >= 0 and < {@link #getValueCount()})
|
||||
* @param result will be populated with the ordinal's value
|
||||
* @see #getOrd(int)
|
||||
*/
|
||||
|
@ -71,7 +72,7 @@ public abstract class SortedDocValues extends BinaryDocValues {
|
|||
public static final SortedDocValues EMPTY = new SortedDocValues() {
|
||||
@Override
|
||||
public int getOrd(int docID) {
|
||||
return 0;
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -83,7 +84,7 @@ public abstract class SortedDocValues extends BinaryDocValues {
|
|||
|
||||
@Override
|
||||
public int getValueCount() {
|
||||
return 1;
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
|
||||
|
|
|
@ -30,19 +30,19 @@ import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
|
|||
import org.apache.lucene.util.BytesRefHash;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
|
||||
import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/** Buffers up pending byte[] per doc, deref and sorting via
|
||||
* int ord, then flushes when segment flushes. */
|
||||
class SortedDocValuesWriter extends DocValuesWriter {
|
||||
final BytesRefHash hash;
|
||||
private AppendingPackedLongBuffer pending;
|
||||
private AppendingDeltaPackedLongBuffer pending;
|
||||
private final Counter iwBytesUsed;
|
||||
private long bytesUsed; // this currently only tracks differences in 'pending'
|
||||
private final FieldInfo fieldInfo;
|
||||
|
||||
private static final BytesRef EMPTY = new BytesRef(BytesRef.EMPTY_BYTES);
|
||||
private static final int EMPTY_ORD = -1;
|
||||
|
||||
public SortedDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
|
||||
this.fieldInfo = fieldInfo;
|
||||
|
@ -52,7 +52,7 @@ class SortedDocValuesWriter extends DocValuesWriter {
|
|||
new ByteBlockPool.DirectTrackingAllocator(iwBytesUsed)),
|
||||
BytesRefHash.DEFAULT_CAPACITY,
|
||||
new DirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, iwBytesUsed));
|
||||
pending = new AppendingPackedLongBuffer(PackedInts.COMPACT);
|
||||
pending = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
|
||||
bytesUsed = pending.ramBytesUsed();
|
||||
iwBytesUsed.addAndGet(bytesUsed);
|
||||
}
|
||||
|
@ -70,7 +70,7 @@ class SortedDocValuesWriter extends DocValuesWriter {
|
|||
|
||||
// Fill in any holes:
|
||||
while(pending.size() < docID) {
|
||||
addOneValue(EMPTY);
|
||||
pending.add(EMPTY_ORD);
|
||||
}
|
||||
|
||||
addOneValue(value);
|
||||
|
@ -79,8 +79,9 @@ class SortedDocValuesWriter extends DocValuesWriter {
|
|||
@Override
|
||||
public void finish(int maxDoc) {
|
||||
while(pending.size() < maxDoc) {
|
||||
addOneValue(EMPTY);
|
||||
pending.add(EMPTY_ORD);
|
||||
}
|
||||
updateBytesUsed();
|
||||
}
|
||||
|
||||
private void addOneValue(BytesRef value) {
|
||||
|
@ -177,7 +178,7 @@ class SortedDocValuesWriter extends DocValuesWriter {
|
|||
|
||||
// iterates over the ords for each doc we have in ram
|
||||
private class OrdsIterator implements Iterator<Number> {
|
||||
final AppendingPackedLongBuffer.Iterator iter = pending.iterator();
|
||||
final AppendingDeltaPackedLongBuffer.Iterator iter = pending.iterator();
|
||||
final int ordMap[];
|
||||
final int maxDoc;
|
||||
int docUpto;
|
||||
|
@ -200,8 +201,7 @@ class SortedDocValuesWriter extends DocValuesWriter {
|
|||
}
|
||||
int ord = (int) iter.next();
|
||||
docUpto++;
|
||||
// TODO: make reusable Number
|
||||
return ordMap[ord];
|
||||
return ord == -1 ? ord : ordMap[ord];
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -104,26 +104,6 @@ public interface FieldCache {
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
/** Returns MISSING/-1 ordinal for every document */
|
||||
public static final SortedDocValues EMPTY_TERMSINDEX = new SortedDocValues() {
|
||||
@Override
|
||||
public int getOrd(int docID) {
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void lookupOrd(int ord, BytesRef result) {
|
||||
result.bytes = MISSING;
|
||||
result.offset = 0;
|
||||
result.length = 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCount() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* Placeholder indicating creation of this cache is currently in-progress.
|
||||
|
@ -266,13 +246,10 @@ public interface FieldCache {
|
|||
}
|
||||
};
|
||||
|
||||
|
||||
/** Checks the internal cache for an appropriate entry, and if none is found,
|
||||
* reads the terms in <code>field</code> and returns a bit set at the size of
|
||||
* <code>reader.maxDoc()</code>, with turned on bits for each docid that
|
||||
* does have a value for this field. Note that if the field was only indexed
|
||||
* as DocValues then this method will not work (it will return a Bits stating
|
||||
* that no documents contain the field).
|
||||
* does have a value for this field.
|
||||
*/
|
||||
public Bits getDocsWithField(AtomicReader reader, String field) throws IOException;
|
||||
|
||||
|
|
|
@ -501,8 +501,7 @@ class FieldCacheImpl implements FieldCache {
|
|||
// field does not exist or has no value
|
||||
return new Bits.MatchNoBits(reader.maxDoc());
|
||||
} else if (fieldInfo.hasDocValues()) {
|
||||
// doc values are dense
|
||||
return new Bits.MatchAllBits(reader.maxDoc());
|
||||
return reader.getDocsWithField(field);
|
||||
} else if (!fieldInfo.isIndexed()) {
|
||||
return new Bits.MatchNoBits(reader.maxDoc());
|
||||
}
|
||||
|
@ -944,13 +943,13 @@ class FieldCacheImpl implements FieldCache {
|
|||
} else {
|
||||
final FieldInfo info = reader.getFieldInfos().fieldInfo(field);
|
||||
if (info == null) {
|
||||
return EMPTY_TERMSINDEX;
|
||||
return SortedDocValues.EMPTY;
|
||||
} else if (info.hasDocValues()) {
|
||||
// we don't try to build a sorted instance from numeric/binary doc
|
||||
// values because dedup can be very costly
|
||||
throw new IllegalStateException("Type mismatch: " + field + " was indexed as " + info.getDocValuesType());
|
||||
} else if (!info.isIndexed()) {
|
||||
return EMPTY_TERMSINDEX;
|
||||
return SortedDocValues.EMPTY;
|
||||
}
|
||||
return (SortedDocValues) caches.get(SortedDocValues.class).get(reader, new CacheKey(field, acceptableOverheadRatio), false);
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
|
|||
* Tests Lucene42DocValuesFormat
|
||||
*/
|
||||
public class TestLucene42DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
|
||||
private final Codec codec = new Lucene42Codec();
|
||||
private final Codec codec = new Lucene42RWCodec();
|
||||
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -74,7 +74,7 @@ import org.junit.Ignore;
|
|||
// we won't even be running the actual code, only the impostor
|
||||
// @SuppressCodecs("Lucene4x")
|
||||
// Sep codec cannot yet handle the offsets in our 4.x index!
|
||||
@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene40", "Lucene41"})
|
||||
@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene40", "Lucene41", "Lucene42"})
|
||||
public class TestBackwardsCompatibility extends LuceneTestCase {
|
||||
|
||||
// Uncomment these cases & run them on an older Lucene version,
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.lucene.analysis.Analyzer;
|
|||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.document.SortedSetDocValuesField;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -49,7 +50,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()
|
||||
|
||||
|
@ -134,6 +135,11 @@ public class TestDuelingCodecs extends LuceneTestCase {
|
|||
for (String trash : split) {
|
||||
document.add(new SortedSetDocValuesField("sortedset", new BytesRef(trash)));
|
||||
}
|
||||
// add a numeric dv field sometimes
|
||||
document.removeFields("sparsenumeric");
|
||||
if (random.nextInt(4) == 2) {
|
||||
document.add(new NumericDocValuesField("sparsenumeric", random.nextInt()));
|
||||
}
|
||||
writer.addDocument(document);
|
||||
}
|
||||
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.document.NumericDocValuesField;
|
|||
import org.apache.lucene.document.SortedDocValuesField;
|
||||
import org.apache.lucene.document.SortedSetDocValuesField;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
|
@ -321,4 +322,52 @@ public class TestMultiDocValues extends LuceneTestCase {
|
|||
ir2.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testDocsWithField() throws Exception {
|
||||
assumeTrue("codec does not support docsWithField", defaultCodecSupportsDocsWithField());
|
||||
Directory dir = newDirectory();
|
||||
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(random(), TEST_VERSION_CURRENT, null);
|
||||
iwc.setMergePolicy(newLogMergePolicy());
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
|
||||
|
||||
int numDocs = atLeast(500);
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
Document doc = new Document();
|
||||
if (random().nextInt(4) >= 0) {
|
||||
doc.add(new NumericDocValuesField("numbers", random().nextLong()));
|
||||
}
|
||||
doc.add(new NumericDocValuesField("numbersAlways", random().nextLong()));
|
||||
iw.addDocument(doc);
|
||||
if (random().nextInt(17) == 0) {
|
||||
iw.commit();
|
||||
}
|
||||
}
|
||||
DirectoryReader ir = iw.getReader();
|
||||
iw.forceMerge(1);
|
||||
DirectoryReader ir2 = iw.getReader();
|
||||
AtomicReader merged = getOnlySegmentReader(ir2);
|
||||
iw.close();
|
||||
|
||||
Bits multi = MultiDocValues.getDocsWithField(ir, "numbers");
|
||||
Bits single = merged.getDocsWithField("numbers");
|
||||
if (multi == null) {
|
||||
assertNull(single);
|
||||
} else {
|
||||
assertEquals(single.length(), multi.length());
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
assertEquals(single.get(i), multi.get(i));
|
||||
}
|
||||
}
|
||||
|
||||
multi = MultiDocValues.getDocsWithField(ir, "numbersAlways");
|
||||
single = merged.getDocsWithField("numbersAlways");
|
||||
assertEquals(single.length(), multi.length());
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
assertEquals(single.get(i), multi.get(i));
|
||||
}
|
||||
ir.close();
|
||||
ir2.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -480,7 +480,7 @@ public class TestFieldCache extends LuceneTestCase {
|
|||
} catch (IllegalStateException expected) {}
|
||||
|
||||
Bits bits = FieldCache.DEFAULT.getDocsWithField(ar, "binary");
|
||||
assertTrue(bits instanceof Bits.MatchAllBits);
|
||||
assertTrue(bits.get(0));
|
||||
|
||||
// Sorted type: can be retrieved via getTerms(), getTermsIndex(), getDocTermOrds()
|
||||
try {
|
||||
|
@ -510,7 +510,7 @@ public class TestFieldCache extends LuceneTestCase {
|
|||
assertEquals(1, sortedSet.getValueCount());
|
||||
|
||||
bits = FieldCache.DEFAULT.getDocsWithField(ar, "sorted");
|
||||
assertTrue(bits instanceof Bits.MatchAllBits);
|
||||
assertTrue(bits.get(0));
|
||||
|
||||
// Numeric type: can be retrieved via getInts() and so on
|
||||
Ints numeric = FieldCache.DEFAULT.getInts(ar, "numeric", false);
|
||||
|
@ -537,7 +537,7 @@ public class TestFieldCache extends LuceneTestCase {
|
|||
} catch (IllegalStateException expected) {}
|
||||
|
||||
bits = FieldCache.DEFAULT.getDocsWithField(ar, "numeric");
|
||||
assertTrue(bits instanceof Bits.MatchAllBits);
|
||||
assertTrue(bits.get(0));
|
||||
|
||||
// SortedSet type: can be retrieved via getDocTermOrds()
|
||||
if (defaultCodecSupportsSortedSet()) {
|
||||
|
@ -569,7 +569,7 @@ public class TestFieldCache extends LuceneTestCase {
|
|||
assertEquals(2, sortedSet.getValueCount());
|
||||
|
||||
bits = FieldCache.DEFAULT.getDocsWithField(ar, "sortedset");
|
||||
assertTrue(bits instanceof Bits.MatchAllBits);
|
||||
assertTrue(bits.get(0));
|
||||
}
|
||||
|
||||
ir.close();
|
||||
|
|
|
@ -31,10 +31,12 @@ import org.apache.lucene.index.RandomIndexWriter;
|
|||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
||||
|
||||
/** Tests basic sorting on docvalues fields.
|
||||
* These are mostly like TestSort's tests, except each test
|
||||
* indexes the field up-front as docvalues, and checks no fieldcaches were made */
|
||||
@SuppressCodecs({"Lucene40", "Lucene41", "Lucene42"}) // avoid codecs that don't support "missing"
|
||||
public class TestSortDocValues extends LuceneTestCase {
|
||||
|
||||
@Override
|
||||
|
@ -291,6 +293,70 @@ public class TestSortDocValues extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
/** Tests sorting on type int with a missing value */
|
||||
public void testIntMissing() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("value", -1));
|
||||
doc.add(newStringField("value", "-1", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("value", 4));
|
||||
doc.add(newStringField("value", "4", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
IndexReader ir = writer.getReader();
|
||||
writer.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
Sort sort = new Sort(new SortField("value", SortField.Type.INT));
|
||||
|
||||
TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
|
||||
assertEquals(3, td.totalHits);
|
||||
// null is treated as a 0
|
||||
assertEquals("-1", searcher.doc(td.scoreDocs[0].doc).get("value"));
|
||||
assertNull(searcher.doc(td.scoreDocs[1].doc).get("value"));
|
||||
assertEquals("4", searcher.doc(td.scoreDocs[2].doc).get("value"));
|
||||
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
/** Tests sorting on type int, specifying the missing value should be treated as Integer.MAX_VALUE */
|
||||
public void testIntMissingLast() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("value", -1));
|
||||
doc.add(newStringField("value", "-1", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("value", 4));
|
||||
doc.add(newStringField("value", "4", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
IndexReader ir = writer.getReader();
|
||||
writer.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
SortField sortField = new SortField("value", SortField.Type.INT);
|
||||
sortField.setMissingValue(Integer.MAX_VALUE);
|
||||
Sort sort = new Sort(sortField);
|
||||
|
||||
TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
|
||||
assertEquals(3, td.totalHits);
|
||||
// null is treated as a Integer.MAX_VALUE
|
||||
assertEquals("-1", searcher.doc(td.scoreDocs[0].doc).get("value"));
|
||||
assertEquals("4", searcher.doc(td.scoreDocs[1].doc).get("value"));
|
||||
assertNull(searcher.doc(td.scoreDocs[2].doc).get("value"));
|
||||
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
/** Tests sorting on type long */
|
||||
public void testLong() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
|
@ -359,6 +425,70 @@ public class TestSortDocValues extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
/** Tests sorting on type long with a missing value */
|
||||
public void testLongMissing() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("value", -1));
|
||||
doc.add(newStringField("value", "-1", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("value", 4));
|
||||
doc.add(newStringField("value", "4", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
IndexReader ir = writer.getReader();
|
||||
writer.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
Sort sort = new Sort(new SortField("value", SortField.Type.LONG));
|
||||
|
||||
TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
|
||||
assertEquals(3, td.totalHits);
|
||||
// null is treated as 0
|
||||
assertEquals("-1", searcher.doc(td.scoreDocs[0].doc).get("value"));
|
||||
assertNull(searcher.doc(td.scoreDocs[1].doc).get("value"));
|
||||
assertEquals("4", searcher.doc(td.scoreDocs[2].doc).get("value"));
|
||||
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
/** Tests sorting on type long, specifying the missing value should be treated as Long.MAX_VALUE */
|
||||
public void testLongMissingLast() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("value", -1));
|
||||
doc.add(newStringField("value", "-1", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("value", 4));
|
||||
doc.add(newStringField("value", "4", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
IndexReader ir = writer.getReader();
|
||||
writer.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
SortField sortField = new SortField("value", SortField.Type.LONG);
|
||||
sortField.setMissingValue(Long.MAX_VALUE);
|
||||
Sort sort = new Sort(sortField);
|
||||
|
||||
TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
|
||||
assertEquals(3, td.totalHits);
|
||||
// null is treated as Long.MAX_VALUE
|
||||
assertEquals("-1", searcher.doc(td.scoreDocs[0].doc).get("value"));
|
||||
assertEquals("4", searcher.doc(td.scoreDocs[1].doc).get("value"));
|
||||
assertNull(searcher.doc(td.scoreDocs[2].doc).get("value"));
|
||||
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
/** Tests sorting on type float */
|
||||
public void testFloat() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
|
@ -427,6 +557,70 @@ public class TestSortDocValues extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
/** Tests sorting on type float with a missing value */
|
||||
public void testFloatMissing() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new FloatDocValuesField("value", -1.3F));
|
||||
doc.add(newStringField("value", "-1.3", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new FloatDocValuesField("value", 4.2F));
|
||||
doc.add(newStringField("value", "4.2", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
IndexReader ir = writer.getReader();
|
||||
writer.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
Sort sort = new Sort(new SortField("value", SortField.Type.FLOAT));
|
||||
|
||||
TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
|
||||
assertEquals(3, td.totalHits);
|
||||
// null is treated as 0
|
||||
assertEquals("-1.3", searcher.doc(td.scoreDocs[0].doc).get("value"));
|
||||
assertNull(searcher.doc(td.scoreDocs[1].doc).get("value"));
|
||||
assertEquals("4.2", searcher.doc(td.scoreDocs[2].doc).get("value"));
|
||||
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
/** Tests sorting on type float, specifying the missing value should be treated as Float.MAX_VALUE */
|
||||
public void testFloatMissingLast() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new FloatDocValuesField("value", -1.3F));
|
||||
doc.add(newStringField("value", "-1.3", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new FloatDocValuesField("value", 4.2F));
|
||||
doc.add(newStringField("value", "4.2", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
IndexReader ir = writer.getReader();
|
||||
writer.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
SortField sortField = new SortField("value", SortField.Type.FLOAT);
|
||||
sortField.setMissingValue(Float.MAX_VALUE);
|
||||
Sort sort = new Sort(sortField);
|
||||
|
||||
TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
|
||||
assertEquals(3, td.totalHits);
|
||||
// null is treated as Float.MAX_VALUE
|
||||
assertEquals("-1.3", searcher.doc(td.scoreDocs[0].doc).get("value"));
|
||||
assertEquals("4.2", searcher.doc(td.scoreDocs[1].doc).get("value"));
|
||||
assertNull(searcher.doc(td.scoreDocs[2].doc).get("value"));
|
||||
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
/** Tests sorting on type double */
|
||||
public void testDouble() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
|
@ -533,4 +727,78 @@ public class TestSortDocValues extends LuceneTestCase {
|
|||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
/** Tests sorting on type double with a missing value */
|
||||
public void testDoubleMissing() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new DoubleDocValuesField("value", -1.3));
|
||||
doc.add(newStringField("value", "-1.3", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new DoubleDocValuesField("value", 4.2333333333333));
|
||||
doc.add(newStringField("value", "4.2333333333333", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new DoubleDocValuesField("value", 4.2333333333332));
|
||||
doc.add(newStringField("value", "4.2333333333332", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
IndexReader ir = writer.getReader();
|
||||
writer.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
Sort sort = new Sort(new SortField("value", SortField.Type.DOUBLE));
|
||||
|
||||
TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
|
||||
assertEquals(4, td.totalHits);
|
||||
// null treated as a 0
|
||||
assertEquals("-1.3", searcher.doc(td.scoreDocs[0].doc).get("value"));
|
||||
assertNull(searcher.doc(td.scoreDocs[1].doc).get("value"));
|
||||
assertEquals("4.2333333333332", searcher.doc(td.scoreDocs[2].doc).get("value"));
|
||||
assertEquals("4.2333333333333", searcher.doc(td.scoreDocs[3].doc).get("value"));
|
||||
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
/** Tests sorting on type double, specifying the missing value should be treated as Double.MAX_VALUE */
|
||||
public void testDoubleMissingLast() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new DoubleDocValuesField("value", -1.3));
|
||||
doc.add(newStringField("value", "-1.3", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new DoubleDocValuesField("value", 4.2333333333333));
|
||||
doc.add(newStringField("value", "4.2333333333333", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new DoubleDocValuesField("value", 4.2333333333332));
|
||||
doc.add(newStringField("value", "4.2333333333332", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
IndexReader ir = writer.getReader();
|
||||
writer.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
SortField sortField = new SortField("value", SortField.Type.DOUBLE);
|
||||
sortField.setMissingValue(Double.MAX_VALUE);
|
||||
Sort sort = new Sort(sortField);
|
||||
|
||||
TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
|
||||
assertEquals(4, td.totalHits);
|
||||
// null treated as Double.MAX_VALUE
|
||||
assertEquals("-1.3", searcher.doc(td.scoreDocs[0].doc).get("value"));
|
||||
assertEquals("4.2333333333332", searcher.doc(td.scoreDocs[1].doc).get("value"));
|
||||
assertEquals("4.2333333333333", searcher.doc(td.scoreDocs[2].doc).get("value"));
|
||||
assertNull(searcher.doc(td.scoreDocs[3].doc).get("value"));
|
||||
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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"));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -68,7 +68,9 @@ public class Facet42DocValuesConsumer extends DocValuesConsumer {
|
|||
|
||||
long totBytes = 0;
|
||||
for (BytesRef v : values) {
|
||||
totBytes += v.length;
|
||||
if (v != null) {
|
||||
totBytes += v.length;
|
||||
}
|
||||
}
|
||||
|
||||
if (totBytes > Integer.MAX_VALUE) {
|
||||
|
@ -78,7 +80,9 @@ public class Facet42DocValuesConsumer extends DocValuesConsumer {
|
|||
out.writeVInt((int) totBytes);
|
||||
|
||||
for (BytesRef v : values) {
|
||||
out.writeBytes(v.bytes, v.offset, v.length);
|
||||
if (v != null) {
|
||||
out.writeBytes(v.bytes, v.offset, v.length);
|
||||
}
|
||||
}
|
||||
|
||||
PackedInts.Writer w = PackedInts.getWriter(out, maxDoc+1, PackedInts.bitsRequired(totBytes+1), acceptableOverheadRatio);
|
||||
|
@ -86,7 +90,9 @@ public class Facet42DocValuesConsumer extends DocValuesConsumer {
|
|||
int address = 0;
|
||||
for(BytesRef v : values) {
|
||||
w.add(address);
|
||||
address += v.length;
|
||||
if (v != null) {
|
||||
address += v.length;
|
||||
}
|
||||
}
|
||||
w.add(address);
|
||||
w.finish();
|
||||
|
|
|
@ -31,15 +31,18 @@ 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.IOUtils;
|
||||
|
||||
class Facet42DocValuesProducer extends DocValuesProducer {
|
||||
|
||||
private final Map<Integer,Facet42BinaryDocValues> fields = new HashMap<Integer,Facet42BinaryDocValues>();
|
||||
private final int maxDoc;
|
||||
|
||||
Facet42DocValuesProducer(SegmentReadState state) throws IOException {
|
||||
String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Facet42DocValuesFormat.EXTENSION);
|
||||
IndexInput in = state.directory.openInput(fileName, state.context);
|
||||
this.maxDoc = state.segmentInfo.getDocCount();
|
||||
boolean success = false;
|
||||
try {
|
||||
CodecUtil.checkHeader(in, Facet42DocValuesFormat.CODEC,
|
||||
|
@ -80,6 +83,11 @@ class Facet42DocValuesProducer extends DocValuesProducer {
|
|||
throw new UnsupportedOperationException("FacetsDocValues only implements binary");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
return new Bits.MatchAllBits(maxDoc); // TODO: have codec impl this?
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
}
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.facet.codecs.facet42;
|
||||
package org.apache.lucene.facet.codecs.facet45;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -21,12 +21,13 @@ import java.util.HashSet;
|
|||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
|
||||
import org.apache.lucene.facet.codecs.facet42.Facet42DocValuesFormat;
|
||||
import org.apache.lucene.facet.params.CategoryListParams;
|
||||
import org.apache.lucene.facet.params.FacetIndexingParams;
|
||||
|
||||
/**
|
||||
* Same as {@link Lucene42Codec} except it uses {@link Facet42DocValuesFormat}
|
||||
* Same as {@link Lucene45Codec} except it uses {@link Facet42DocValuesFormat}
|
||||
* for facet fields (faster-but-more-RAM-consuming doc values).
|
||||
*
|
||||
* <p>
|
||||
|
@ -42,15 +43,14 @@ import org.apache.lucene.facet.params.FacetIndexingParams;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class Facet42Codec extends Lucene42Codec {
|
||||
public class Facet45Codec extends Lucene45Codec {
|
||||
|
||||
private final Set<String> facetFields;
|
||||
private final DocValuesFormat facetsDVFormat = DocValuesFormat.forName("Facet42");
|
||||
private final DocValuesFormat lucene42DVFormat = DocValuesFormat.forName("Lucene42");
|
||||
|
||||
// must have that for SPI purposes
|
||||
/** Default constructor, uses {@link FacetIndexingParams#DEFAULT}. */
|
||||
public Facet42Codec() {
|
||||
public Facet45Codec() {
|
||||
this(FacetIndexingParams.DEFAULT);
|
||||
}
|
||||
|
||||
|
@ -59,7 +59,7 @@ public class Facet42Codec extends Lucene42Codec {
|
|||
* {@link DocValuesFormat} for the fields that are returned by
|
||||
* {@link FacetIndexingParams#getAllCategoryListParams()}.
|
||||
*/
|
||||
public Facet42Codec(FacetIndexingParams fip) {
|
||||
public Facet45Codec(FacetIndexingParams fip) {
|
||||
if (fip.getPartitionSize() != Integer.MAX_VALUE) {
|
||||
throw new IllegalArgumentException("this Codec does not support partitions");
|
||||
}
|
||||
|
@ -74,8 +74,7 @@ public class Facet42Codec extends Lucene42Codec {
|
|||
if (facetFields.contains(field)) {
|
||||
return facetsDVFormat;
|
||||
} else {
|
||||
return lucene42DVFormat;
|
||||
return super.getDocValuesFormatForField(field);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
<?xml version="1.0" ?>
|
||||
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
|
||||
<!--
|
||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
contributor license agreements. See the NOTICE file distributed with
|
||||
|
@ -15,19 +15,8 @@
|
|||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
-->
|
||||
|
||||
<schema name="bad-schema-docValues-not-required-no-default" version="1.0">
|
||||
<types>
|
||||
<fieldType name="string" class="solr.StrField" />
|
||||
</types>
|
||||
|
||||
|
||||
<fields>
|
||||
<!-- docValues must be required or have a default value -->
|
||||
<field name="id" type="string" docValues="true" multiValued="false"/>
|
||||
</fields>
|
||||
|
||||
<defaultSearchField>id</defaultSearchField>
|
||||
<uniqueKey>id</uniqueKey>
|
||||
|
||||
</schema>
|
||||
<html>
|
||||
<body>
|
||||
Codec + DocValuesFormat that are optimized for facets.
|
||||
</body>
|
||||
</html>
|
|
@ -40,6 +40,7 @@ import org.apache.lucene.index.Terms;
|
|||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
|
@ -222,6 +223,18 @@ public class FacetsPayloadMigrationReader extends FilterAtomicReader {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
Term term = fieldTerms.get(field);
|
||||
if (term == null) {
|
||||
return super.getDocsWithField(field);
|
||||
} else {
|
||||
// we shouldn't return null, even if the term does not exist or has no
|
||||
// payloads, since we already marked the field as having DocValues.
|
||||
return new Bits.MatchAllBits(maxDoc());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldInfos getFieldInfos() {
|
||||
FieldInfos innerInfos = super.getFieldInfos();
|
||||
|
|
|
@ -13,4 +13,4 @@
|
|||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
org.apache.lucene.facet.codecs.facet42.Facet42Codec
|
||||
org.apache.lucene.facet.codecs.facet45.Facet45Codec
|
||||
|
|
|
@ -3,7 +3,7 @@ package org.apache.lucene.facet;
|
|||
import java.util.Random;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.facet.codecs.facet42.Facet42Codec;
|
||||
import org.apache.lucene.facet.codecs.facet45.Facet45Codec;
|
||||
import org.apache.lucene.facet.encoding.DGapIntEncoder;
|
||||
import org.apache.lucene.facet.encoding.DGapVInt8IntEncoder;
|
||||
import org.apache.lucene.facet.encoding.EightFlagsIntEncoder;
|
||||
|
@ -53,7 +53,7 @@ public abstract class FacetTestCase extends LuceneTestCase {
|
|||
public static void beforeClassFacetTestCase() throws Exception {
|
||||
if (random().nextDouble() < 0.3) {
|
||||
savedDefault = Codec.getDefault(); // save to restore later
|
||||
Codec.setDefault(new Facet42Codec());
|
||||
Codec.setDefault(new Facet45Codec());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.lucene.document.Document;
|
|||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.facet.FacetTestCase;
|
||||
import org.apache.lucene.facet.FacetTestUtils;
|
||||
import org.apache.lucene.facet.codecs.facet42.Facet42Codec;
|
||||
import org.apache.lucene.facet.codecs.facet45.Facet45Codec;
|
||||
import org.apache.lucene.facet.index.FacetFields;
|
||||
import org.apache.lucene.facet.params.CategoryListParams;
|
||||
import org.apache.lucene.facet.params.FacetIndexingParams;
|
||||
|
@ -260,7 +260,7 @@ public class TestDemoFacets extends FacetTestCase {
|
|||
Directory dir = newDirectory();
|
||||
Directory taxoDir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||
iwc.setCodec(new Facet42Codec());
|
||||
iwc.setCodec(new Facet45Codec());
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
|
||||
DirectoryTaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE);
|
||||
|
||||
|
|
|
@ -422,6 +422,11 @@ public class WeightedSpanTermExtractor {
|
|||
public NumericDocValues getNormValues(String field) throws IOException {
|
||||
return super.getNormValues(FIELD_NAME);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
return super.getDocsWithField(FIELD_NAME);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -756,6 +756,11 @@ public class MemoryIndex {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
private class MemoryFields extends Fields {
|
||||
@Override
|
||||
public Iterator<String> iterator() {
|
||||
|
|
|
@ -221,6 +221,27 @@ public class SortingAtomicReader extends FilterAtomicReader {
|
|||
}
|
||||
}
|
||||
|
||||
private static class SortingBits implements Bits {
|
||||
|
||||
private final Bits in;
|
||||
private final Sorter.DocMap docMap;
|
||||
|
||||
public SortingBits(final Bits in, Sorter.DocMap docMap) {
|
||||
this.in = in;
|
||||
this.docMap = docMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean get(int index) {
|
||||
return in.get(docMap.newToOld(index));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return in.length();
|
||||
}
|
||||
}
|
||||
|
||||
private static class SortingSortedDocValues extends SortedDocValues {
|
||||
|
||||
private final SortedDocValues in;
|
||||
|
@ -743,20 +764,9 @@ public class SortingAtomicReader extends FilterAtomicReader {
|
|||
final Bits inLiveDocs = in.getLiveDocs();
|
||||
if (inLiveDocs == null) {
|
||||
return null;
|
||||
} else {
|
||||
return new SortingBits(inLiveDocs, docMap);
|
||||
}
|
||||
return new Bits() {
|
||||
|
||||
@Override
|
||||
public boolean get(int index) {
|
||||
return inLiveDocs.get(docMap.newToOld(index));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return inLiveDocs.length();
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -796,6 +806,16 @@ public class SortingAtomicReader extends FilterAtomicReader {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
Bits bits = in.getDocsWithField(field);
|
||||
if (bits == null || bits instanceof Bits.MatchAllBits || bits instanceof Bits.MatchNoBits) {
|
||||
return bits;
|
||||
} else {
|
||||
return new SortingBits(bits, docMap);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields getTermVectors(final int docID) throws IOException {
|
||||
return in.getTermVectors(docMap.newToOld(docID));
|
||||
|
|
|
@ -45,12 +45,13 @@ public class BytesRefFieldSource extends FieldCacheSource {
|
|||
// To be sorted or not to be sorted, that is the question
|
||||
// TODO: do it cleaner?
|
||||
if (fieldInfo != null && fieldInfo.getDocValuesType() == DocValuesType.BINARY) {
|
||||
final Bits docsWithField = FieldCache.DEFAULT.getDocsWithField(readerContext.reader(), field);
|
||||
final BinaryDocValues binaryValues = FieldCache.DEFAULT.getTerms(readerContext.reader(), field);
|
||||
return new FunctionValues() {
|
||||
|
||||
@Override
|
||||
public boolean exists(int doc) {
|
||||
return true; // doc values are dense
|
||||
return docsWithField.get(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -0,0 +1,122 @@
|
|||
package org.apache.lucene.codecs;
|
||||
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* a utility class to write missing values for SORTED as if they were the empty string
|
||||
* (to simulate pre-Lucene4.5 dv behavior for testing old codecs)
|
||||
*/
|
||||
public class MissingOrdRemapper {
|
||||
|
||||
/** insert an empty byte[] to the front of this iterable */
|
||||
public static Iterable<BytesRef> insertEmptyValue(final Iterable<BytesRef> iterable) {
|
||||
return new Iterable<BytesRef>() {
|
||||
@Override
|
||||
public Iterator<BytesRef> iterator() {
|
||||
return new Iterator<BytesRef>() {
|
||||
boolean seenEmpty = false;
|
||||
Iterator<BytesRef> in = iterable.iterator();
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return !seenEmpty || in.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef next() {
|
||||
if (!seenEmpty) {
|
||||
seenEmpty = true;
|
||||
return new BytesRef();
|
||||
} else {
|
||||
return in.next();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/** remaps ord -1 to ord 0 on this iterable. */
|
||||
public static Iterable<Number> mapMissingToOrd0(final Iterable<Number> iterable) {
|
||||
return new Iterable<Number>() {
|
||||
@Override
|
||||
public Iterator<Number> iterator() {
|
||||
return new Iterator<Number>() {
|
||||
Iterator<Number> in = iterable.iterator();
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return in.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Number next() {
|
||||
Number n = in.next();
|
||||
if (n.longValue() == -1) {
|
||||
return 0;
|
||||
} else {
|
||||
return n;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/** remaps every ord+1 on this iterable */
|
||||
public static Iterable<Number> mapAllOrds(final Iterable<Number> iterable) {
|
||||
return new Iterable<Number>() {
|
||||
@Override
|
||||
public Iterator<Number> iterator() {
|
||||
return new Iterator<Number>() {
|
||||
Iterator<Number> in = iterable.iterator();
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return in.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Number next() {
|
||||
Number n = in.next();
|
||||
return n.longValue()+1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
|
@ -33,15 +33,16 @@ import org.apache.lucene.index.SegmentReadState;
|
|||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
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");
|
||||
|
@ -75,11 +76,10 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
|
||||
int count = 0;
|
||||
for (Number v : values) {
|
||||
assert v != null;
|
||||
count++;
|
||||
}
|
||||
assert count == maxDoc;
|
||||
checkIterator(values.iterator(), maxDoc);
|
||||
checkIterator(values.iterator(), maxDoc, true);
|
||||
in.addNumericField(field, values);
|
||||
}
|
||||
|
||||
|
@ -87,12 +87,11 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
|
||||
int count = 0;
|
||||
for (BytesRef b : values) {
|
||||
assert b != null;
|
||||
assert b.isValid();
|
||||
assert b == null || b.isValid();
|
||||
count++;
|
||||
}
|
||||
assert count == maxDoc;
|
||||
checkIterator(values.iterator(), maxDoc);
|
||||
checkIterator(values.iterator(), maxDoc, true);
|
||||
in.addBinaryField(field, values);
|
||||
}
|
||||
|
||||
|
@ -117,15 +116,17 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
for (Number v : docToOrd) {
|
||||
assert v != null;
|
||||
int ord = v.intValue();
|
||||
assert ord >= 0 && ord < valueCount;
|
||||
seenOrds.set(ord);
|
||||
assert ord >= -1 && ord < valueCount;
|
||||
if (ord >= 0) {
|
||||
seenOrds.set(ord);
|
||||
}
|
||||
count++;
|
||||
}
|
||||
|
||||
assert count == maxDoc;
|
||||
assert seenOrds.cardinality() == valueCount;
|
||||
checkIterator(values.iterator(), valueCount);
|
||||
checkIterator(docToOrd.iterator(), maxDoc);
|
||||
checkIterator(values.iterator(), valueCount, false);
|
||||
checkIterator(docToOrd.iterator(), maxDoc, false);
|
||||
in.addSortedField(field, values, docToOrd);
|
||||
}
|
||||
|
||||
|
@ -169,33 +170,11 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
|
||||
assert docCount == maxDoc;
|
||||
assert seenOrds.cardinality() == valueCount;
|
||||
checkIterator(values.iterator(), valueCount);
|
||||
checkIterator(docToOrdCount.iterator(), maxDoc);
|
||||
checkIterator(ords.iterator(), ordCount);
|
||||
checkIterator(values.iterator(), valueCount, false);
|
||||
checkIterator(docToOrdCount.iterator(), maxDoc, false);
|
||||
checkIterator(ords.iterator(), ordCount, false);
|
||||
in.addSortedSetField(field, values, docToOrdCount, ords);
|
||||
}
|
||||
|
||||
private <T> void checkIterator(Iterator<T> iterator, long expectedSize) {
|
||||
for (long i = 0; i < expectedSize; i++) {
|
||||
boolean hasNext = iterator.hasNext();
|
||||
assert hasNext;
|
||||
T v = iterator.next();
|
||||
assert v != null;
|
||||
try {
|
||||
iterator.remove();
|
||||
throw new AssertionError("broken iterator (supports remove): " + iterator);
|
||||
} catch (UnsupportedOperationException expected) {
|
||||
// ok
|
||||
}
|
||||
}
|
||||
assert !iterator.hasNext();
|
||||
try {
|
||||
iterator.next();
|
||||
throw new AssertionError("broken iterator (allows next() when hasNext==false) " + iterator);
|
||||
} catch (NoSuchElementException expected) {
|
||||
// ok
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
@ -203,6 +182,70 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
}
|
||||
}
|
||||
|
||||
static class AssertingNormsConsumer extends DocValuesConsumer {
|
||||
private final DocValuesConsumer in;
|
||||
private final int maxDoc;
|
||||
|
||||
AssertingNormsConsumer(DocValuesConsumer in, int maxDoc) {
|
||||
this.in = in;
|
||||
this.maxDoc = maxDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
|
||||
int count = 0;
|
||||
for (Number v : values) {
|
||||
assert v != null;
|
||||
count++;
|
||||
}
|
||||
assert count == maxDoc;
|
||||
checkIterator(values.iterator(), maxDoc, false);
|
||||
in.addNumericField(field, values);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
in.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
}
|
||||
|
||||
private static <T> void checkIterator(Iterator<T> iterator, long expectedSize, boolean allowNull) {
|
||||
for (long i = 0; i < expectedSize; i++) {
|
||||
boolean hasNext = iterator.hasNext();
|
||||
assert hasNext;
|
||||
T v = iterator.next();
|
||||
assert allowNull || v != null;
|
||||
try {
|
||||
iterator.remove();
|
||||
throw new AssertionError("broken iterator (supports remove): " + iterator);
|
||||
} catch (UnsupportedOperationException expected) {
|
||||
// ok
|
||||
}
|
||||
}
|
||||
assert !iterator.hasNext();
|
||||
try {
|
||||
iterator.next();
|
||||
throw new AssertionError("broken iterator (allows next() when hasNext==false) " + iterator);
|
||||
} catch (NoSuchElementException expected) {
|
||||
// ok
|
||||
}
|
||||
}
|
||||
|
||||
static class AssertingDocValuesProducer extends DocValuesProducer {
|
||||
private final DocValuesProducer in;
|
||||
private final int maxDoc;
|
||||
|
@ -244,6 +287,15 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
assert values != null;
|
||||
return new AssertingAtomicReader.AssertingSortedSetDocValues(values, maxDoc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
assert field.getDocValuesType() != null;
|
||||
Bits bits = in.getDocsWithField(field);
|
||||
assert bits != null;
|
||||
assert bits.length() == maxDoc;
|
||||
return new AssertingAtomicReader.AssertingBits(bits);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
|
|
@ -22,7 +22,7 @@ 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.asserting.AssertingDocValuesFormat.AssertingDocValuesConsumer;
|
||||
import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat.AssertingNormsConsumer;
|
||||
import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat.AssertingDocValuesProducer;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
|
@ -38,7 +38,7 @@ public class AssertingNormsFormat extends NormsFormat {
|
|||
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
|
||||
DocValuesConsumer consumer = in.normsConsumer(state);
|
||||
assert consumer != null;
|
||||
return new AssertingDocValuesConsumer(consumer, state.segmentInfo.getDocCount());
|
||||
return new AssertingNormsConsumer(consumer, state.segmentInfo.getDocCount());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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,431 +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.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.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;
|
||||
|
||||
CheapBastardDocValuesProducer(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);
|
||||
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 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);
|
||||
}
|
||||
|
|
|
@ -1,8 +1,6 @@
|
|||
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.util.packed.PackedInts;
|
||||
|
||||
|
@ -42,9 +40,4 @@ public class FastCompressingCodec extends CompressingCodec {
|
|||
public NormsFormat normsFormat() {
|
||||
return new Lucene42NormsFormat(PackedInts.FAST);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesFormat docValuesFormat() {
|
||||
return new Lucene42DocValuesFormat(PackedInts.FAST);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,8 +1,6 @@
|
|||
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.util.packed.PackedInts;
|
||||
|
||||
|
@ -42,9 +40,4 @@ public class FastDecompressionCompressingCodec extends CompressingCodec {
|
|||
public NormsFormat normsFormat() {
|
||||
return new Lucene42NormsFormat(PackedInts.DEFAULT);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesFormat docValuesFormat() {
|
||||
return new Lucene42DocValuesFormat(PackedInts.DEFAULT);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,7 +24,9 @@ import java.util.TreeSet;
|
|||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.MissingOrdRemapper;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosReader.LegacyDocValuesType;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
@ -54,7 +56,7 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
long minValue = Long.MAX_VALUE;
|
||||
long maxValue = Long.MIN_VALUE;
|
||||
for (Number n : values) {
|
||||
long v = n.longValue();
|
||||
long v = n == null ? 0 : n.longValue();
|
||||
minValue = Math.min(minValue, v);
|
||||
maxValue = Math.max(maxValue, v);
|
||||
}
|
||||
|
@ -92,7 +94,7 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
Lucene40DocValuesFormat.INTS_VERSION_CURRENT);
|
||||
output.writeInt(1); // size
|
||||
for (Number n : values) {
|
||||
output.writeByte(n.byteValue());
|
||||
output.writeByte(n == null ? 0 : n.byteValue());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -103,7 +105,7 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
Lucene40DocValuesFormat.INTS_VERSION_CURRENT);
|
||||
output.writeInt(2); // size
|
||||
for (Number n : values) {
|
||||
output.writeShort(n.shortValue());
|
||||
output.writeShort(n == null ? 0 : n.shortValue());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -114,7 +116,7 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
Lucene40DocValuesFormat.INTS_VERSION_CURRENT);
|
||||
output.writeInt(4); // size
|
||||
for (Number n : values) {
|
||||
output.writeInt(n.intValue());
|
||||
output.writeInt(n == null ? 0 : n.intValue());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -131,7 +133,7 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
// writes longs
|
||||
output.writeByte(Lucene40DocValuesFormat.VAR_INTS_FIXED_64);
|
||||
for (Number n : values) {
|
||||
output.writeLong(n.longValue());
|
||||
output.writeLong(n == null ? 0 : n.longValue());
|
||||
}
|
||||
} else {
|
||||
// writes packed ints
|
||||
|
@ -143,7 +145,8 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
PackedInts.bitsRequired(delta),
|
||||
PackedInts.DEFAULT);
|
||||
for (Number n : values) {
|
||||
writer.add(n.longValue() - minValue);
|
||||
long v = n == null ? 0 : n.longValue();
|
||||
writer.add(v - minValue);
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
|
@ -156,6 +159,9 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
int minLength = Integer.MAX_VALUE;
|
||||
int maxLength = Integer.MIN_VALUE;
|
||||
for (BytesRef b : values) {
|
||||
if (b == null) {
|
||||
b = new BytesRef(); // 4.0 doesnt distinguish
|
||||
}
|
||||
if (b.length > Lucene40DocValuesFormat.MAX_BINARY_FIELD_LENGTH) {
|
||||
throw new IllegalArgumentException("DocValuesField \"" + field.name + "\" is too large, must be <= " + Lucene40DocValuesFormat.MAX_BINARY_FIELD_LENGTH);
|
||||
}
|
||||
|
@ -246,7 +252,9 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
|
||||
output.writeInt(length);
|
||||
for (BytesRef v : values) {
|
||||
output.writeBytes(v.bytes, v.offset, v.length);
|
||||
if (v != null) {
|
||||
output.writeBytes(v.bytes, v.offset, v.length);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -267,7 +275,9 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
final long startPos = data.getFilePointer();
|
||||
|
||||
for (BytesRef v : values) {
|
||||
data.writeBytes(v.bytes, v.offset, v.length);
|
||||
if (v != null) {
|
||||
data.writeBytes(v.bytes, v.offset, v.length);
|
||||
}
|
||||
}
|
||||
|
||||
/* addresses */
|
||||
|
@ -282,7 +292,9 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
long currentPosition = 0;
|
||||
for (BytesRef v : values) {
|
||||
w.add(currentPosition);
|
||||
currentPosition += v.length;
|
||||
if (v != null) {
|
||||
currentPosition += v.length;
|
||||
}
|
||||
}
|
||||
// write sentinel
|
||||
assert currentPosition == maxAddress;
|
||||
|
@ -304,7 +316,7 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
// deduplicate
|
||||
TreeSet<BytesRef> dictionary = new TreeSet<BytesRef>();
|
||||
for (BytesRef v : values) {
|
||||
dictionary.add(BytesRef.deepCopyOf(v));
|
||||
dictionary.add(v == null ? new BytesRef() : BytesRef.deepCopyOf(v));
|
||||
}
|
||||
|
||||
/* values */
|
||||
|
@ -321,6 +333,9 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
final PackedInts.Writer w = PackedInts.getWriter(index, maxDoc, PackedInts.bitsRequired(valueCount-1), PackedInts.DEFAULT);
|
||||
|
||||
for (BytesRef v : values) {
|
||||
if (v == null) {
|
||||
v = new BytesRef();
|
||||
}
|
||||
int ord = dictionary.headSet(v).size();
|
||||
w.add(ord);
|
||||
}
|
||||
|
@ -341,7 +356,7 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
// deduplicate
|
||||
TreeSet<BytesRef> dictionary = new TreeSet<BytesRef>();
|
||||
for (BytesRef v : values) {
|
||||
dictionary.add(BytesRef.deepCopyOf(v));
|
||||
dictionary.add(v == null ? new BytesRef() : BytesRef.deepCopyOf(v));
|
||||
}
|
||||
|
||||
/* values */
|
||||
|
@ -362,7 +377,7 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
final PackedInts.Writer w = PackedInts.getWriter(index, maxDoc, PackedInts.bitsRequired(currentAddress), PackedInts.DEFAULT);
|
||||
|
||||
for (BytesRef v : values) {
|
||||
w.add(valueToAddress.get(v));
|
||||
w.add(valueToAddress.get(v == null ? new BytesRef() : v));
|
||||
}
|
||||
w.finish();
|
||||
}
|
||||
|
@ -388,6 +403,15 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
maxLength = Math.max(maxLength, b.length);
|
||||
}
|
||||
|
||||
// but dont use fixed if there are missing values (we are simulating how lucene40 wrote dv...)
|
||||
boolean anyMissing = false;
|
||||
for (Number n : docToOrd) {
|
||||
if (n.longValue() == -1) {
|
||||
anyMissing = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
boolean success = false;
|
||||
IndexOutput data = null;
|
||||
IndexOutput index = null;
|
||||
|
@ -397,12 +421,22 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
try {
|
||||
data = dir.createOutput(dataName, state.context);
|
||||
index = dir.createOutput(indexName, state.context);
|
||||
if (minLength == maxLength) {
|
||||
if (minLength == maxLength && !anyMissing) {
|
||||
// fixed byte[]
|
||||
addFixedSortedBytesField(field, data, index, values, docToOrd, minLength);
|
||||
} else {
|
||||
// var byte[]
|
||||
addVarSortedBytesField(field, data, index, values, docToOrd);
|
||||
// three cases for simulating the old writer:
|
||||
// 1. no missing
|
||||
// 2. missing (and empty string in use): remap ord=-1 -> ord=0
|
||||
// 3. missing (and empty string not in use): remap all ords +1, insert empty string into values
|
||||
if (!anyMissing) {
|
||||
addVarSortedBytesField(field, data, index, values, docToOrd);
|
||||
} else if (minLength == 0) {
|
||||
addVarSortedBytesField(field, data, index, values, MissingOrdRemapper.mapMissingToOrd0(docToOrd));
|
||||
} else {
|
||||
addVarSortedBytesField(field, data, index, MissingOrdRemapper.insertEmptyValue(values), MissingOrdRemapper.mapAllOrds(docToOrd));
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.NoSuchElementException;
|
|||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.MissingOrdRemapper;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
@ -45,25 +46,20 @@ import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
|
|||
import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.VERSION_CURRENT;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.BLOCK_SIZE;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.BYTES;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.NUMBER;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.FST;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.DELTA_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.GCD_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.TABLE_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.UNCOMPRESSED;
|
||||
|
||||
/**
|
||||
* Writer for {@link Lucene42DocValuesFormat}
|
||||
*/
|
||||
class Lucene42DocValuesConsumer extends DocValuesConsumer {
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_GCD_COMPRESSION = 1;
|
||||
static final int VERSION_CURRENT = VERSION_GCD_COMPRESSION;
|
||||
|
||||
static final byte NUMBER = 0;
|
||||
static final byte BYTES = 1;
|
||||
static final byte FST = 2;
|
||||
|
||||
static final int BLOCK_SIZE = 4096;
|
||||
|
||||
static final byte DELTA_COMPRESSED = 0;
|
||||
static final byte TABLE_COMPRESSED = 1;
|
||||
static final byte UNCOMPRESSED = 2;
|
||||
static final byte GCD_COMPRESSED = 3;
|
||||
|
||||
final IndexOutput data, meta;
|
||||
final int maxDoc;
|
||||
final float acceptableOverheadRatio;
|
||||
|
@ -106,7 +102,8 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
|
|||
|
||||
long count = 0;
|
||||
for (Number nv : values) {
|
||||
final long v = nv.longValue();
|
||||
// TODO: support this as MemoryDVFormat (and be smart about missing maybe)
|
||||
final long v = nv == null ? 0 : nv.longValue();
|
||||
|
||||
if (gcd != 1) {
|
||||
if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
|
||||
|
@ -142,7 +139,7 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
|
|||
if (formatAndBits.bitsPerValue == 8 && minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) {
|
||||
meta.writeByte(UNCOMPRESSED); // uncompressed
|
||||
for (Number nv : values) {
|
||||
data.writeByte((byte) nv.longValue());
|
||||
data.writeByte(nv == null ? 0 : (byte) nv.longValue());
|
||||
}
|
||||
} else {
|
||||
meta.writeByte(TABLE_COMPRESSED); // table-compressed
|
||||
|
@ -160,7 +157,7 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
|
|||
|
||||
final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, formatAndBits.format, maxDoc, formatAndBits.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
|
||||
for(Number nv : values) {
|
||||
writer.add(encode.get(nv.longValue()));
|
||||
writer.add(encode.get(nv == null ? 0 : nv.longValue()));
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
|
@ -173,7 +170,8 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
|
|||
|
||||
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
|
||||
for (Number nv : values) {
|
||||
writer.add((nv.longValue() - minValue) / gcd);
|
||||
long value = nv == null ? 0 : nv.longValue();
|
||||
writer.add((value - minValue) / gcd);
|
||||
}
|
||||
writer.finish();
|
||||
} else {
|
||||
|
@ -184,7 +182,7 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
|
|||
|
||||
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
|
||||
for (Number nv : values) {
|
||||
writer.add(nv.longValue());
|
||||
writer.add(nv == null ? 0 : nv.longValue());
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
|
@ -216,12 +214,15 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
|
|||
int maxLength = Integer.MIN_VALUE;
|
||||
final long startFP = data.getFilePointer();
|
||||
for(BytesRef v : values) {
|
||||
if (v.length > Lucene42DocValuesFormat.MAX_BINARY_FIELD_LENGTH) {
|
||||
final int length = v == null ? 0 : v.length;
|
||||
if (length > Lucene42DocValuesFormat.MAX_BINARY_FIELD_LENGTH) {
|
||||
throw new IllegalArgumentException("DocValuesField \"" + field.name + "\" is too large, must be <= " + Lucene42DocValuesFormat.MAX_BINARY_FIELD_LENGTH);
|
||||
}
|
||||
minLength = Math.min(minLength, v.length);
|
||||
maxLength = Math.max(maxLength, v.length);
|
||||
data.writeBytes(v.bytes, v.offset, v.length);
|
||||
minLength = Math.min(minLength, length);
|
||||
maxLength = Math.max(maxLength, length);
|
||||
if (v != null) {
|
||||
data.writeBytes(v.bytes, v.offset, v.length);
|
||||
}
|
||||
}
|
||||
meta.writeLong(startFP);
|
||||
meta.writeLong(data.getFilePointer() - startFP);
|
||||
|
@ -237,7 +238,9 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
|
|||
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
|
||||
long addr = 0;
|
||||
for (BytesRef v : values) {
|
||||
addr += v.length;
|
||||
if (v != null) {
|
||||
addr += v.length;
|
||||
}
|
||||
writer.add(addr);
|
||||
}
|
||||
writer.finish();
|
||||
|
@ -265,6 +268,33 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
|
|||
|
||||
@Override
|
||||
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
|
||||
// three cases for simulating the old writer:
|
||||
// 1. no missing
|
||||
// 2. missing (and empty string in use): remap ord=-1 -> ord=0
|
||||
// 3. missing (and empty string not in use): remap all ords +1, insert empty string into values
|
||||
boolean anyMissing = false;
|
||||
for (Number n : docToOrd) {
|
||||
if (n.longValue() == -1) {
|
||||
anyMissing = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
boolean hasEmptyString = false;
|
||||
for (BytesRef b : values) {
|
||||
hasEmptyString = b.length == 0;
|
||||
break;
|
||||
}
|
||||
|
||||
if (!anyMissing) {
|
||||
// nothing to do
|
||||
} else if (hasEmptyString) {
|
||||
docToOrd = MissingOrdRemapper.mapMissingToOrd0(docToOrd);
|
||||
} else {
|
||||
docToOrd = MissingOrdRemapper.mapAllOrds(docToOrd);
|
||||
values = MissingOrdRemapper.insertEmptyValue(values);
|
||||
}
|
||||
|
||||
// write the ordinals as numerics
|
||||
addNumericField(field, docToOrd, false);
|
||||
|
|
@ -0,0 +1,39 @@
|
|||
package org.apache.lucene.codecs.lucene42;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
|
||||
/**
|
||||
* Read-write version of {@link Lucene42Codec} for testing.
|
||||
*/
|
||||
public class Lucene42RWCodec extends Lucene42Codec {
|
||||
private static final DocValuesFormat dv = new Lucene42RWDocValuesFormat();
|
||||
private static final NormsFormat norms = new Lucene42NormsFormat();
|
||||
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return dv;
|
||||
}
|
||||
|
||||
@Override
|
||||
public NormsFormat normsFormat() {
|
||||
return norms;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
package org.apache.lucene.codecs.lucene42;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
||||
/**
|
||||
* Read-write version of {@link Lucene42DocValuesFormat} for testing.
|
||||
*/
|
||||
public class Lucene42RWDocValuesFormat extends Lucene42DocValuesFormat {
|
||||
|
||||
@Override
|
||||
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
// note: we choose DEFAULT here (its reasonably fast, and for small bpv has tiny waste)
|
||||
return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,25 @@
|
|||
<!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>
|
||||
Support for testing {@link org.apache.lucene.codecs.lucene42.Lucene42Codec}.
|
||||
</body>
|
||||
</html>
|
|
@ -0,0 +1,25 @@
|
|||
<!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>
|
||||
Codecs for testing (simulate old disk formats, wacky theoretical use cases, etc)
|
||||
</body>
|
||||
</html>
|
|
@ -438,14 +438,14 @@ public class AssertingAtomicReader extends FilterAtomicReader {
|
|||
this.in = in;
|
||||
this.maxDoc = maxDoc;
|
||||
this.valueCount = in.getValueCount();
|
||||
assert valueCount >= 1 && valueCount <= maxDoc;
|
||||
assert valueCount >= 0 && valueCount <= maxDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getOrd(int docID) {
|
||||
assert docID >= 0 && docID < maxDoc;
|
||||
int ord = in.getOrd(docID);
|
||||
assert ord >= 0 && ord < valueCount;
|
||||
assert ord >= -1 && ord < valueCount;
|
||||
return ord;
|
||||
}
|
||||
|
||||
|
@ -607,6 +607,54 @@ public class AssertingAtomicReader extends FilterAtomicReader {
|
|||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/** Wraps a Bits but with additional asserts */
|
||||
public static class AssertingBits implements Bits {
|
||||
final Bits in;
|
||||
|
||||
public AssertingBits(Bits in) {
|
||||
this.in = in;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean get(int index) {
|
||||
assert index >= 0 && index < length();
|
||||
return in.get(index);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return in.length();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getLiveDocs() {
|
||||
Bits liveDocs = super.getLiveDocs();
|
||||
if (liveDocs != null) {
|
||||
assert maxDoc() == liveDocs.length();
|
||||
liveDocs = new AssertingBits(liveDocs);
|
||||
} else {
|
||||
assert maxDoc() == numDocs();
|
||||
assert !hasDeletions();
|
||||
}
|
||||
return liveDocs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
Bits docsWithField = super.getDocsWithField(field);
|
||||
FieldInfo fi = getFieldInfos().fieldInfo(field);
|
||||
if (docsWithField != null) {
|
||||
assert fi != null;
|
||||
assert fi.hasDocValues();
|
||||
assert maxDoc() == docsWithField.length();
|
||||
docsWithField = new AssertingBits(docsWithField);
|
||||
} else {
|
||||
assert fi == null || fi.hasDocValues() == false;
|
||||
}
|
||||
return docsWithField;
|
||||
}
|
||||
|
||||
// this is the same hack as FCInvisible
|
||||
@Override
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue