mirror of https://github.com/apache/lucene.git
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:
parent
c2f37e0de2
commit
28316a161c
|
@ -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>
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
Loading…
Reference in New Issue