add memorydv with missing support

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5178@1515582 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2013-08-19 20:11:03 +00:00
parent c2f37e0de2
commit 28316a161c
8 changed files with 1152 additions and 2 deletions

View File

@ -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();
}
}
}

View File

@ -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";
}

View File

@ -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();
}
}
}

View File

@ -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>

View File

@ -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

View File

@ -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;
}
}

View File

@ -40,6 +40,7 @@ import org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat;
import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
import org.apache.lucene.codecs.memory.MemoryDocValuesFormat;
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
import org.apache.lucene.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
@ -149,6 +150,7 @@ public class RandomCodec extends Lucene45Codec {
addDocValues(avoidCodecs,
new Lucene45DocValuesFormat(),
new DiskDocValuesFormat(),
new MemoryDocValuesFormat(),
new SimpleTextDocValuesFormat(),
new AssertingDocValuesFormat());

View File

@ -760,7 +760,7 @@ public class _TestUtil {
// TODO: remove this, push this test to Lucene40/Lucene42 codec tests
public static boolean fieldSupportsHugeBinaryDocValues(String field) {
String dvFormat = getDocValuesFormat(field);
if (dvFormat.equals("Lucene40") || dvFormat.equals("Lucene42")) {
if (dvFormat.equals("Lucene40") || dvFormat.equals("Lucene42") || dvFormat.equals("Memory")) {
return false;
}
return true;