mirror of https://github.com/apache/lucene.git
LUCENE-8267: removed references to memory codecs.
This commit is contained in:
parent
08ee037ff8
commit
85c00e77ef
|
@ -41,6 +41,9 @@ API Changes
|
|||
* LUCENE-8242: Deprecated method IndexSearcher#createNormalizedWeight() has
|
||||
been removed (Alan Woodward)
|
||||
|
||||
* LUCENE-8267: Memory codecs removed from the codebase (MemoryPostings,
|
||||
MemoryDocValues). (Dawid Weiss)
|
||||
|
||||
Changes in Runtime Behavior
|
||||
|
||||
* LUCENE-7837: Indices that were created before the previous major version
|
||||
|
|
|
@ -43,3 +43,7 @@ the same index options for that field.
|
|||
|
||||
Instead use IndexSearcher.createWeight(), rewriting the query first, and using
|
||||
a boost of 1f.
|
||||
|
||||
## Memory codecs removed (LUCENE-8267) ##
|
||||
|
||||
Memory codecs have been removed from the codebase (MemoryPostings, MemoryDocValues).
|
||||
|
|
|
@ -1,539 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.codecs.memory;
|
||||
|
||||
|
||||
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.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.LegacyDocValuesIterables;
|
||||
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.IntsRefBuilder;
|
||||
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.BLOCK_COMPRESSED;
|
||||
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.DELTA_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.FST;
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.GCD_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.NUMBER;
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.SORTED_NUMERIC;
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.SORTED_NUMERIC_SINGLETON;
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.SORTED_SET;
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.SORTED_SET_SINGLETON;
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.TABLE_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.VERSION_CURRENT;
|
||||
|
||||
/**
|
||||
* Writer for {@link MemoryDocValuesFormat}
|
||||
*/
|
||||
class MemoryDocValuesConsumer extends DocValuesConsumer {
|
||||
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.maxDoc();
|
||||
boolean success = false;
|
||||
try {
|
||||
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
|
||||
data = state.directory.createOutput(dataName, state.context);
|
||||
CodecUtil.writeIndexHeader(data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
meta = state.directory.createOutput(metaName, state.context);
|
||||
CodecUtil.writeIndexHeader(meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(this);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
|
||||
addNumericField(field, LegacyDocValuesIterables.numericIterable(field, valuesProducer, maxDoc), 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 blockSum = 0;
|
||||
long gcd = 0;
|
||||
boolean missing = false;
|
||||
// TODO: more efficient?
|
||||
HashSet<Long> uniqueValues = null;
|
||||
long count = 0;
|
||||
|
||||
if (optimizeStorage) {
|
||||
uniqueValues = new HashSet<>();
|
||||
|
||||
long currentBlockMin = Long.MAX_VALUE;
|
||||
long currentBlockMax = Long.MIN_VALUE;
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
currentBlockMin = Math.min(minValue, v);
|
||||
currentBlockMax = Math.max(maxValue, v);
|
||||
|
||||
minValue = Math.min(minValue, v);
|
||||
maxValue = Math.max(maxValue, v);
|
||||
|
||||
if (uniqueValues != null) {
|
||||
if (uniqueValues.add(v)) {
|
||||
if (uniqueValues.size() > 256) {
|
||||
uniqueValues = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
++count;
|
||||
if (count % BLOCK_SIZE == 0) {
|
||||
final long blockDelta = currentBlockMax - currentBlockMin;
|
||||
final int blockDeltaRequired = PackedInts.unsignedBitsRequired(blockDelta);
|
||||
final int blockBPV = PackedInts.fastestFormatAndBits(BLOCK_SIZE, blockDeltaRequired, acceptableOverheadRatio).bitsPerValue;
|
||||
blockSum += blockBPV;
|
||||
currentBlockMax = Long.MIN_VALUE;
|
||||
currentBlockMin = Long.MAX_VALUE;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for (Number nv : values) {
|
||||
long v = nv.longValue();
|
||||
maxValue = Math.max(v, maxValue);
|
||||
minValue = Math.min(v, minValue);
|
||||
count++;
|
||||
}
|
||||
}
|
||||
|
||||
if (missing) {
|
||||
long start = data.getFilePointer();
|
||||
writeMissingBitset(values);
|
||||
meta.writeLong(start);
|
||||
meta.writeLong(data.getFilePointer() - start);
|
||||
} else {
|
||||
meta.writeLong(-1L);
|
||||
}
|
||||
|
||||
final long delta = maxValue - minValue;
|
||||
final int deltaRequired = delta < 0 ? 64 : PackedInts.bitsRequired(delta);
|
||||
final FormatAndBits deltaBPV = PackedInts.fastestFormatAndBits(maxDoc, deltaRequired, acceptableOverheadRatio);
|
||||
|
||||
final FormatAndBits tableBPV;
|
||||
if (count < Integer.MAX_VALUE && uniqueValues != null) {
|
||||
tableBPV = PackedInts.fastestFormatAndBits(maxDoc, PackedInts.bitsRequired(uniqueValues.size()-1), acceptableOverheadRatio);
|
||||
} else {
|
||||
tableBPV = null;
|
||||
}
|
||||
|
||||
final FormatAndBits gcdBPV;
|
||||
if (count < Integer.MAX_VALUE && gcd != 0 && gcd != 1) {
|
||||
final long gcdDelta = (maxValue - minValue) / gcd;
|
||||
final int gcdRequired = gcdDelta < 0 ? 64 : PackedInts.bitsRequired(gcdDelta);
|
||||
gcdBPV = PackedInts.fastestFormatAndBits(maxDoc, gcdRequired, acceptableOverheadRatio);
|
||||
} else {
|
||||
gcdBPV = null;
|
||||
}
|
||||
|
||||
boolean doBlock = false;
|
||||
if (blockSum != 0) {
|
||||
int numBlocks = maxDoc / BLOCK_SIZE;
|
||||
float avgBPV = blockSum / (float)numBlocks;
|
||||
// just a heuristic, with tiny amounts of blocks our estimate is skewed as we ignore the final "incomplete" block.
|
||||
// with at least 4 blocks it's pretty accurate. The difference must also be significant (according to acceptable overhead).
|
||||
if (numBlocks >= 4 && (avgBPV+avgBPV*acceptableOverheadRatio) < deltaBPV.bitsPerValue) {
|
||||
doBlock = true;
|
||||
}
|
||||
}
|
||||
// blockpackedreader allows us to read in huge streams of ints
|
||||
if (count >= Integer.MAX_VALUE) {
|
||||
doBlock = true;
|
||||
}
|
||||
|
||||
if (tableBPV != null && (tableBPV.bitsPerValue+tableBPV.bitsPerValue*acceptableOverheadRatio) < deltaBPV.bitsPerValue) {
|
||||
// small number of unique values
|
||||
meta.writeByte(TABLE_COMPRESSED); // table-compressed
|
||||
Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]);
|
||||
final HashMap<Long,Integer> encode = new HashMap<>();
|
||||
int length = 1 << tableBPV.bitsPerValue;
|
||||
data.writeVInt(length);
|
||||
for (int i = 0; i < decode.length; i++) {
|
||||
data.writeLong(decode[i]);
|
||||
encode.put(decode[i], i);
|
||||
}
|
||||
for (int i = decode.length; i < length; i++) {
|
||||
data.writeLong(0);
|
||||
}
|
||||
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
meta.writeLong(count);
|
||||
data.writeVInt(tableBPV.format.getId());
|
||||
data.writeVInt(tableBPV.bitsPerValue);
|
||||
|
||||
final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, tableBPV.format, (int)count, tableBPV.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
|
||||
for(Number nv : values) {
|
||||
writer.add(encode.get(nv == null ? 0 : nv.longValue()));
|
||||
}
|
||||
writer.finish();
|
||||
} else if (gcdBPV != null && (gcdBPV.bitsPerValue+gcdBPV.bitsPerValue*acceptableOverheadRatio) < deltaBPV.bitsPerValue) {
|
||||
meta.writeByte(GCD_COMPRESSED);
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
meta.writeLong(count);
|
||||
data.writeLong(minValue);
|
||||
data.writeLong(gcd);
|
||||
data.writeVInt(gcdBPV.format.getId());
|
||||
data.writeVInt(gcdBPV.bitsPerValue);
|
||||
|
||||
final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, gcdBPV.format, (int)count, gcdBPV.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
|
||||
for (Number nv : values) {
|
||||
long value = nv == null ? 0 : nv.longValue();
|
||||
writer.add((value - minValue) / gcd);
|
||||
}
|
||||
writer.finish();
|
||||
} else if (doBlock) {
|
||||
meta.writeByte(BLOCK_COMPRESSED); // block delta-compressed
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
meta.writeLong(count);
|
||||
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();
|
||||
} else {
|
||||
meta.writeByte(DELTA_COMPRESSED); // delta-compressed
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
meta.writeLong(count);
|
||||
final long minDelta = deltaBPV.bitsPerValue == 64 ? 0 : minValue;
|
||||
data.writeLong(minDelta);
|
||||
data.writeVInt(deltaBPV.format.getId());
|
||||
data.writeVInt(deltaBPV.bitsPerValue);
|
||||
|
||||
final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, deltaBPV.format, (int)count, deltaBPV.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
|
||||
for (Number nv : values) {
|
||||
long v = nv == null ? 0 : nv.longValue();
|
||||
writer.add(v - minDelta);
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
boolean success = false;
|
||||
try {
|
||||
if (meta != null) {
|
||||
meta.writeVInt(-1); // write EOF marker
|
||||
CodecUtil.writeFooter(meta); // write checksum
|
||||
}
|
||||
if (data != null) {
|
||||
CodecUtil.writeFooter(data);
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(data, meta);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(data, meta);
|
||||
}
|
||||
data = meta = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
|
||||
addBinaryField(field, LegacyDocValuesIterables.binaryIterable(field, valuesProducer, maxDoc));
|
||||
}
|
||||
|
||||
private 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;
|
||||
int upto = 0;
|
||||
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 + " but got length=" + length + " v=" + v + "; upto=" + upto + " values=" + values);
|
||||
}
|
||||
upto++;
|
||||
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, it's 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<>(INPUT_TYPE.BYTE1, outputs);
|
||||
IntsRefBuilder scratch = new IntsRefBuilder();
|
||||
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, DocValuesProducer valuesProducer) throws IOException {
|
||||
addSortedField(field,
|
||||
LegacyDocValuesIterables.valuesIterable(valuesProducer.getSorted(field)),
|
||||
LegacyDocValuesIterables.sortedOrdIterable(valuesProducer, field, maxDoc));
|
||||
}
|
||||
|
||||
private 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);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addSortedNumericField(FieldInfo field, final DocValuesProducer valuesProducer) throws IOException {
|
||||
|
||||
final Iterable<Number> docToValueCount = LegacyDocValuesIterables.sortedNumericToDocCount(valuesProducer, field, maxDoc);
|
||||
final Iterable<Number> values = LegacyDocValuesIterables.sortedNumericToValues(valuesProducer, field);
|
||||
|
||||
meta.writeVInt(field.number);
|
||||
|
||||
if (isSingleValued(docToValueCount)) {
|
||||
meta.writeByte(SORTED_NUMERIC_SINGLETON);
|
||||
addNumericField(field, singletonView(docToValueCount, values, null), true);
|
||||
} else {
|
||||
meta.writeByte(SORTED_NUMERIC);
|
||||
|
||||
// write the addresses:
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
meta.writeVInt(BLOCK_SIZE);
|
||||
meta.writeLong(data.getFilePointer());
|
||||
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
|
||||
long addr = 0;
|
||||
writer.add(addr);
|
||||
for (Number v : docToValueCount) {
|
||||
addr += v.longValue();
|
||||
writer.add(addr);
|
||||
}
|
||||
writer.finish();
|
||||
long valueCount = writer.ord();
|
||||
meta.writeLong(valueCount);
|
||||
|
||||
// write the values
|
||||
addNumericField(field, values, true);
|
||||
}
|
||||
}
|
||||
|
||||
// note: this might not be the most efficient... but it's fairly simple
|
||||
@Override
|
||||
public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
|
||||
Iterable<BytesRef> values = LegacyDocValuesIterables.valuesIterable(valuesProducer.getSortedSet(field));
|
||||
Iterable<Number> docToOrdCount = LegacyDocValuesIterables.sortedSetOrdCountIterable(valuesProducer, field, maxDoc);
|
||||
Iterable<Number> ords = LegacyDocValuesIterables.sortedSetOrdsIterable(valuesProducer, field);
|
||||
meta.writeVInt(field.number);
|
||||
|
||||
if (isSingleValued(docToOrdCount)) {
|
||||
meta.writeByte(SORTED_SET_SINGLETON);
|
||||
addSortedField(field, values, singletonView(docToOrdCount, ords, -1L));
|
||||
} else {
|
||||
meta.writeByte(SORTED_SET);
|
||||
// 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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,72 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.codecs.memory;
|
||||
|
||||
|
||||
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";
|
||||
}
|
|
@ -1,877 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.codecs.memory;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.store.ByteArrayDataInput;
|
||||
import org.apache.lucene.store.ChecksumIndexInput;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.Accountables;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.IntsRefBuilder;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
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<String,NumericEntry> numerics = new HashMap<>();
|
||||
private final Map<String,BinaryEntry> binaries = new HashMap<>();
|
||||
private final Map<String,FSTEntry> fsts = new HashMap<>();
|
||||
private final Map<String,SortedSetEntry> sortedSets = new HashMap<>();
|
||||
private final Map<String,SortedNumericEntry> sortedNumerics = new HashMap<>();
|
||||
private final IndexInput data;
|
||||
|
||||
// ram instances we have already loaded
|
||||
private final Map<String,LegacyNumericDocValues> numericInstances = new HashMap<>();
|
||||
private final Map<String,BytesAndAddresses> pagedBytesInstances = new HashMap<>();
|
||||
private final Map<String,FST<Long>> fstInstances = new HashMap<>();
|
||||
private final Map<String,FixedBitSet> docsWithFieldInstances = new HashMap<>();
|
||||
private final Map<String,MonotonicBlockPackedReader> addresses = new HashMap<>();
|
||||
|
||||
private final Map<String,Accountable> numericInfo = new HashMap<>();
|
||||
|
||||
private final int numEntries;
|
||||
private final int maxDoc;
|
||||
private final AtomicLong ramBytesUsed;
|
||||
private final int version;
|
||||
|
||||
private final boolean merging;
|
||||
|
||||
static final byte NUMBER = 0;
|
||||
static final byte BYTES = 1;
|
||||
static final byte FST = 2;
|
||||
static final byte SORTED_SET = 4;
|
||||
static final byte SORTED_SET_SINGLETON = 5;
|
||||
static final byte SORTED_NUMERIC = 6;
|
||||
static final byte SORTED_NUMERIC_SINGLETON = 7;
|
||||
|
||||
static final int BLOCK_SIZE = 4096;
|
||||
|
||||
static final byte DELTA_COMPRESSED = 0;
|
||||
static final byte TABLE_COMPRESSED = 1;
|
||||
static final byte BLOCK_COMPRESSED = 2;
|
||||
static final byte GCD_COMPRESSED = 3;
|
||||
|
||||
static final int VERSION_START = 4;
|
||||
static final int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
// clone for merge: when merging we don't do any instances.put()s
|
||||
MemoryDocValuesProducer(MemoryDocValuesProducer original) throws IOException {
|
||||
assert Thread.holdsLock(original);
|
||||
numerics.putAll(original.numerics);
|
||||
binaries.putAll(original.binaries);
|
||||
fsts.putAll(original.fsts);
|
||||
sortedSets.putAll(original.sortedSets);
|
||||
sortedNumerics.putAll(original.sortedNumerics);
|
||||
data = original.data.clone();
|
||||
|
||||
numericInstances.putAll(original.numericInstances);
|
||||
pagedBytesInstances.putAll(original.pagedBytesInstances);
|
||||
fstInstances.putAll(original.fstInstances);
|
||||
docsWithFieldInstances.putAll(original.docsWithFieldInstances);
|
||||
addresses.putAll(original.addresses);
|
||||
|
||||
numericInfo.putAll(original.numericInfo);
|
||||
|
||||
numEntries = original.numEntries;
|
||||
maxDoc = original.maxDoc;
|
||||
ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
|
||||
version = original.version;
|
||||
merging = true;
|
||||
}
|
||||
|
||||
MemoryDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
maxDoc = state.segmentInfo.maxDoc();
|
||||
merging = false;
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
// read in the entries from the metadata file.
|
||||
ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
|
||||
boolean success = false;
|
||||
try {
|
||||
version = CodecUtil.checkIndexHeader(in, metaCodec, VERSION_START, VERSION_CURRENT,
|
||||
state.segmentInfo.getId(), state.segmentSuffix);
|
||||
numEntries = readFields(in, state.fieldInfos);
|
||||
CodecUtil.checkFooter(in);
|
||||
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(in);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(in);
|
||||
}
|
||||
}
|
||||
|
||||
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
|
||||
this.data = state.directory.openInput(dataName, state.context);
|
||||
success = false;
|
||||
try {
|
||||
final int version2 = CodecUtil.checkIndexHeader(data, dataCodec, VERSION_START, VERSION_CURRENT,
|
||||
state.segmentInfo.getId(), state.segmentSuffix);
|
||||
if (version != version2) {
|
||||
throw new CorruptIndexException("Format versions mismatch: meta=" + version + ", data=" + version2, data);
|
||||
}
|
||||
|
||||
// NOTE: data file is too costly to verify checksum against all the bytes on open,
|
||||
// but for now we at least verify proper structure of the checksum footer: which looks
|
||||
// for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
|
||||
// such as file truncation.
|
||||
CodecUtil.retrieveChecksum(data);
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(this.data);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private NumericEntry readNumericEntry(IndexInput meta) throws IOException {
|
||||
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 BLOCK_COMPRESSED:
|
||||
case GCD_COMPRESSED:
|
||||
break;
|
||||
default:
|
||||
throw new CorruptIndexException("Unknown format: " + entry.format, meta);
|
||||
}
|
||||
entry.packedIntsVersion = meta.readVInt();
|
||||
entry.count = meta.readLong();
|
||||
return entry;
|
||||
}
|
||||
|
||||
private BinaryEntry readBinaryEntry(IndexInput meta) throws IOException {
|
||||
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();
|
||||
}
|
||||
return entry;
|
||||
}
|
||||
|
||||
private FSTEntry readFSTEntry(IndexInput meta) throws IOException {
|
||||
FSTEntry entry = new FSTEntry();
|
||||
entry.offset = meta.readLong();
|
||||
entry.numOrds = meta.readVLong();
|
||||
return entry;
|
||||
}
|
||||
|
||||
private int readFields(IndexInput meta, FieldInfos infos) throws IOException {
|
||||
int numEntries = 0;
|
||||
int fieldNumber = meta.readVInt();
|
||||
while (fieldNumber != -1) {
|
||||
numEntries++;
|
||||
FieldInfo info = infos.fieldInfo(fieldNumber);
|
||||
if (info == null) {
|
||||
throw new CorruptIndexException("invalid field number: " + fieldNumber, meta);
|
||||
}
|
||||
int fieldType = meta.readByte();
|
||||
if (fieldType == NUMBER) {
|
||||
numerics.put(info.name, readNumericEntry(meta));
|
||||
} else if (fieldType == BYTES) {
|
||||
binaries.put(info.name, readBinaryEntry(meta));
|
||||
} else if (fieldType == FST) {
|
||||
fsts.put(info.name,readFSTEntry(meta));
|
||||
} else if (fieldType == SORTED_SET) {
|
||||
SortedSetEntry entry = new SortedSetEntry();
|
||||
entry.singleton = false;
|
||||
sortedSets.put(info.name, entry);
|
||||
} else if (fieldType == SORTED_SET_SINGLETON) {
|
||||
SortedSetEntry entry = new SortedSetEntry();
|
||||
entry.singleton = true;
|
||||
sortedSets.put(info.name, entry);
|
||||
} else if (fieldType == SORTED_NUMERIC) {
|
||||
SortedNumericEntry entry = new SortedNumericEntry();
|
||||
entry.singleton = false;
|
||||
entry.packedIntsVersion = meta.readVInt();
|
||||
entry.blockSize = meta.readVInt();
|
||||
entry.addressOffset = meta.readLong();
|
||||
entry.valueCount = meta.readLong();
|
||||
sortedNumerics.put(info.name, entry);
|
||||
} else if (fieldType == SORTED_NUMERIC_SINGLETON) {
|
||||
SortedNumericEntry entry = new SortedNumericEntry();
|
||||
entry.singleton = true;
|
||||
sortedNumerics.put(info.name, entry);
|
||||
} else {
|
||||
throw new CorruptIndexException("invalid entry type: " + fieldType + ", fieldName=" + info.name, meta);
|
||||
}
|
||||
fieldNumber = meta.readVInt();
|
||||
}
|
||||
return numEntries;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized NumericDocValues getNumeric(FieldInfo field) throws IOException {
|
||||
LegacyNumericDocValues instance = numericInstances.get(field.name);
|
||||
if (instance == null) {
|
||||
instance = loadNumeric(field);
|
||||
if (!merging) {
|
||||
numericInstances.put(field.name, instance);
|
||||
}
|
||||
}
|
||||
NumericEntry ne = numerics.get(field.name);
|
||||
return new LegacyNumericDocValuesWrapper(getMissingBits(field, ne.missingOffset, ne.missingBytes), instance);
|
||||
}
|
||||
|
||||
private synchronized LegacyNumericDocValues getNumericNonIterator(FieldInfo field) throws IOException {
|
||||
LegacyNumericDocValues instance = numericInstances.get(field.name);
|
||||
if (instance == null) {
|
||||
instance = loadNumeric(field);
|
||||
if (!merging) {
|
||||
numericInstances.put(field.name, instance);
|
||||
}
|
||||
}
|
||||
return instance;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return ramBytesUsed.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("numeric field", numericInfo));
|
||||
resources.addAll(Accountables.namedAccountables("pagedbytes field", pagedBytesInstances));
|
||||
resources.addAll(Accountables.namedAccountables("term dict field", fstInstances));
|
||||
resources.addAll(Accountables.namedAccountables("missing bitset field", docsWithFieldInstances));
|
||||
resources.addAll(Accountables.namedAccountables("addresses field", addresses));
|
||||
return Collections.unmodifiableList(resources);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void checkIntegrity() throws IOException {
|
||||
CodecUtil.checksumEntireFile(data.clone());
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized DocValuesProducer getMergeInstance() throws IOException {
|
||||
return new MemoryDocValuesProducer(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getClass().getSimpleName() + "(entries=" + numEntries + ")";
|
||||
}
|
||||
|
||||
private LegacyNumericDocValues loadNumeric(FieldInfo field) throws IOException {
|
||||
NumericEntry entry = numerics.get(field.name);
|
||||
IndexInput data = this.data.clone();
|
||||
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, got=" + size, 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, (int)entry.count, bitsPerValue);
|
||||
if (!merging) {
|
||||
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed());
|
||||
numericInfo.put(field.name, Accountables.namedAccountable("table compressed", ordsReader));
|
||||
}
|
||||
return new LegacyNumericDocValues() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
return decode[(int)ordsReader.get(docID)];
|
||||
}
|
||||
};
|
||||
case DELTA_COMPRESSED:
|
||||
final long minDelta = data.readLong();
|
||||
final int formatIDDelta = data.readVInt();
|
||||
final int bitsPerValueDelta = data.readVInt();
|
||||
final PackedInts.Reader deltaReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatIDDelta), entry.packedIntsVersion, (int)entry.count, bitsPerValueDelta);
|
||||
if (!merging) {
|
||||
ramBytesUsed.addAndGet(deltaReader.ramBytesUsed());
|
||||
numericInfo.put(field.name, Accountables.namedAccountable("delta compressed", deltaReader));
|
||||
}
|
||||
return new LegacyNumericDocValues() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
return minDelta + deltaReader.get(docID);
|
||||
}
|
||||
};
|
||||
case BLOCK_COMPRESSED:
|
||||
final int blockSize = data.readVInt();
|
||||
final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, entry.count, false);
|
||||
if (!merging) {
|
||||
ramBytesUsed.addAndGet(reader.ramBytesUsed());
|
||||
numericInfo.put(field.name, Accountables.namedAccountable("block compressed", reader));
|
||||
}
|
||||
return new LegacyNumericDocValues() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
return reader.get(docID);
|
||||
}
|
||||
};
|
||||
case GCD_COMPRESSED:
|
||||
final long min = data.readLong();
|
||||
final long mult = data.readLong();
|
||||
final int formatIDGCD = data.readVInt();
|
||||
final int bitsPerValueGCD = data.readVInt();
|
||||
final PackedInts.Reader quotientReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatIDGCD), entry.packedIntsVersion, (int)entry.count, bitsPerValueGCD);
|
||||
if (!merging) {
|
||||
ramBytesUsed.addAndGet(quotientReader.ramBytesUsed());
|
||||
numericInfo.put(field.name, Accountables.namedAccountable("gcd compressed", quotientReader));
|
||||
}
|
||||
return new LegacyNumericDocValues() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
return min + mult * quotientReader.get(docID);
|
||||
}
|
||||
};
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
|
||||
private LegacyBinaryDocValues getLegacyBinary(FieldInfo field) throws IOException {
|
||||
BinaryEntry entry = binaries.get(field.name);
|
||||
|
||||
BytesAndAddresses instance;
|
||||
synchronized (this) {
|
||||
instance = pagedBytesInstances.get(field.name);
|
||||
if (instance == null) {
|
||||
instance = loadBinary(field);
|
||||
if (!merging) {
|
||||
pagedBytesInstances.put(field.name, instance);
|
||||
}
|
||||
}
|
||||
}
|
||||
final PagedBytes.Reader bytesReader = instance.reader;
|
||||
final MonotonicBlockPackedReader addresses = instance.addresses;
|
||||
|
||||
if (addresses == null) {
|
||||
assert entry.minLength == entry.maxLength;
|
||||
final int fixedLength = entry.minLength;
|
||||
return new LegacyBinaryDocValues() {
|
||||
final BytesRef term = new BytesRef();
|
||||
|
||||
@Override
|
||||
public BytesRef get(int docID) {
|
||||
bytesReader.fillSlice(term, fixedLength * (long)docID, fixedLength);
|
||||
return term;
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return new LegacyBinaryDocValues() {
|
||||
final BytesRef term = new BytesRef();
|
||||
|
||||
@Override
|
||||
public BytesRef get(int docID) {
|
||||
long startAddress = docID == 0 ? 0 : addresses.get(docID-1);
|
||||
long endAddress = addresses.get(docID);
|
||||
bytesReader.fillSlice(term, startAddress, (int) (endAddress - startAddress));
|
||||
return term;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized BinaryDocValues getBinary(FieldInfo field) throws IOException {
|
||||
BinaryEntry be = binaries.get(field.name);
|
||||
return new LegacyBinaryDocValuesWrapper(getMissingBits(field, be.missingOffset, be.missingBytes), getLegacyBinary(field));
|
||||
}
|
||||
|
||||
private BytesAndAddresses loadBinary(FieldInfo field) throws IOException {
|
||||
BytesAndAddresses bytesAndAddresses = new BytesAndAddresses();
|
||||
BinaryEntry entry = binaries.get(field.name);
|
||||
IndexInput data = this.data.clone();
|
||||
data.seek(entry.offset);
|
||||
PagedBytes bytes = new PagedBytes(16);
|
||||
bytes.copy(data, entry.numBytes);
|
||||
bytesAndAddresses.reader = bytes.freeze(true);
|
||||
if (!merging) {
|
||||
ramBytesUsed.addAndGet(bytesAndAddresses.reader.ramBytesUsed());
|
||||
}
|
||||
if (entry.minLength != entry.maxLength) {
|
||||
data.seek(data.getFilePointer() + entry.missingBytes);
|
||||
bytesAndAddresses.addresses = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, maxDoc, false);
|
||||
if (!merging) {
|
||||
ramBytesUsed.addAndGet(bytesAndAddresses.addresses.ramBytesUsed());
|
||||
}
|
||||
}
|
||||
return bytesAndAddresses;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedDocValues getSorted(FieldInfo field) throws IOException {
|
||||
return new LegacySortedDocValuesWrapper(getSortedNonIterator(field), maxDoc);
|
||||
}
|
||||
|
||||
private LegacySortedDocValues getSortedNonIterator(FieldInfo field) throws IOException {
|
||||
final FSTEntry entry = fsts.get(field.name);
|
||||
if (entry.numOrds == 0) {
|
||||
return DocValues.emptyLegacySorted();
|
||||
}
|
||||
FST<Long> instance;
|
||||
synchronized(this) {
|
||||
instance = fstInstances.get(field.name);
|
||||
if (instance == null) {
|
||||
IndexInput data = this.data.clone();
|
||||
data.seek(entry.offset);
|
||||
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
|
||||
if (!merging) {
|
||||
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
||||
fstInstances.put(field.name, instance);
|
||||
}
|
||||
}
|
||||
}
|
||||
final LegacyNumericDocValues docToOrd = getNumericNonIterator(field);
|
||||
final FST<Long> fst = instance;
|
||||
|
||||
// per-thread resources
|
||||
final BytesReader in = fst.getBytesReader();
|
||||
final Arc<Long> firstArc = new Arc<>();
|
||||
final Arc<Long> scratchArc = new Arc<>();
|
||||
final IntsRefBuilder scratchInts = new IntsRefBuilder();
|
||||
final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<>(fst);
|
||||
|
||||
return new LegacySortedDocValues() {
|
||||
final BytesRefBuilder term = new BytesRefBuilder();
|
||||
|
||||
@Override
|
||||
public int getOrd(int docID) {
|
||||
return (int) docToOrd.get(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef lookupOrd(int ord) {
|
||||
try {
|
||||
in.setPosition(0);
|
||||
fst.getFirstArc(firstArc);
|
||||
IntsRef output = Util.getByOutput(fst, ord, in, firstArc, scratchArc, scratchInts);
|
||||
return Util.toBytesRef(output, term);
|
||||
} 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 SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
|
||||
SortedNumericEntry entry = sortedNumerics.get(field.name);
|
||||
if (entry.singleton) {
|
||||
LegacyNumericDocValues values = getNumericNonIterator(field);
|
||||
NumericEntry ne = numerics.get(field.name);
|
||||
Bits docsWithField = getMissingBits(field, ne.missingOffset, ne.missingBytes);
|
||||
return DocValues.singleton(new LegacyNumericDocValuesWrapper(docsWithField, values));
|
||||
} else {
|
||||
final LegacyNumericDocValues values = getNumericNonIterator(field);
|
||||
final MonotonicBlockPackedReader addr;
|
||||
synchronized (this) {
|
||||
MonotonicBlockPackedReader res = addresses.get(field.name);
|
||||
if (res == null) {
|
||||
IndexInput data = this.data.clone();
|
||||
data.seek(entry.addressOffset);
|
||||
res = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.valueCount, false);
|
||||
if (!merging) {
|
||||
addresses.put(field.name, res);
|
||||
ramBytesUsed.addAndGet(res.ramBytesUsed());
|
||||
}
|
||||
}
|
||||
addr = res;
|
||||
}
|
||||
return new LegacySortedNumericDocValuesWrapper(new LegacySortedNumericDocValues() {
|
||||
int startOffset;
|
||||
int endOffset;
|
||||
|
||||
@Override
|
||||
public void setDocument(int doc) {
|
||||
startOffset = (int) addr.get(doc);
|
||||
endOffset = (int) addr.get(doc+1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long valueAt(int index) {
|
||||
return values.get(startOffset + index);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int count() {
|
||||
return (endOffset - startOffset);
|
||||
}
|
||||
}, maxDoc);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
|
||||
SortedSetEntry sortedSetEntry = sortedSets.get(field.name);
|
||||
if (sortedSetEntry.singleton) {
|
||||
return DocValues.singleton(getSorted(field));
|
||||
}
|
||||
|
||||
final FSTEntry entry = fsts.get(field.name);
|
||||
if (entry.numOrds == 0) {
|
||||
return DocValues.emptySortedSet(); // empty FST!
|
||||
}
|
||||
FST<Long> instance;
|
||||
synchronized(this) {
|
||||
instance = fstInstances.get(field.name);
|
||||
if (instance == null) {
|
||||
IndexInput data = this.data.clone();
|
||||
data.seek(entry.offset);
|
||||
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
|
||||
if (!merging) {
|
||||
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
||||
fstInstances.put(field.name, instance);
|
||||
}
|
||||
}
|
||||
}
|
||||
final LegacyBinaryDocValues docToOrds = getLegacyBinary(field);
|
||||
final FST<Long> fst = instance;
|
||||
|
||||
// per-thread resources
|
||||
final BytesReader in = fst.getBytesReader();
|
||||
final Arc<Long> firstArc = new Arc<>();
|
||||
final Arc<Long> scratchArc = new Arc<>();
|
||||
final IntsRefBuilder scratchInts = new IntsRefBuilder();
|
||||
final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<>(fst);
|
||||
final ByteArrayDataInput input = new ByteArrayDataInput();
|
||||
return new LegacySortedSetDocValuesWrapper(new LegacySortedSetDocValues() {
|
||||
final BytesRefBuilder term = new BytesRefBuilder();
|
||||
BytesRef ref;
|
||||
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) {
|
||||
ref = docToOrds.get(docID);
|
||||
input.reset(ref.bytes, ref.offset, ref.length);
|
||||
currentOrd = 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef lookupOrd(long ord) {
|
||||
try {
|
||||
in.setPosition(0);
|
||||
fst.getFirstArc(firstArc);
|
||||
IntsRef output = Util.getByOutput(fst, ord, in, firstArc, scratchArc, scratchInts);
|
||||
return Util.toBytesRef(output, term);
|
||||
} 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);
|
||||
}
|
||||
}, maxDoc);
|
||||
}
|
||||
|
||||
private Bits getMissingBits(FieldInfo field, final long offset, final long length) throws IOException {
|
||||
if (offset == -1) {
|
||||
return new Bits.MatchAllBits(maxDoc);
|
||||
} else {
|
||||
FixedBitSet instance;
|
||||
synchronized(this) {
|
||||
instance = docsWithFieldInstances.get(field.name);
|
||||
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);
|
||||
if (!merging) {
|
||||
docsWithFieldInstances.put(field.name, instance);
|
||||
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
||||
}
|
||||
}
|
||||
}
|
||||
return instance;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
data.close();
|
||||
}
|
||||
|
||||
static class NumericEntry {
|
||||
long offset;
|
||||
long count;
|
||||
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;
|
||||
}
|
||||
|
||||
static class SortedSetEntry {
|
||||
boolean singleton;
|
||||
}
|
||||
|
||||
static class SortedNumericEntry {
|
||||
boolean singleton;
|
||||
long addressOffset;
|
||||
int packedIntsVersion;
|
||||
int blockSize;
|
||||
long valueCount;
|
||||
}
|
||||
|
||||
static class BytesAndAddresses implements Accountable {
|
||||
PagedBytes.Reader reader;
|
||||
MonotonicBlockPackedReader addresses;
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long bytesUsed = reader.ramBytesUsed();
|
||||
if (addresses != null) {
|
||||
bytesUsed += addresses.ramBytesUsed();
|
||||
}
|
||||
return bytesUsed;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
if (addresses != null) {
|
||||
resources.add(Accountables.namedAccountable("addresses", addresses));
|
||||
}
|
||||
resources.add(Accountables.namedAccountable("term bytes", reader));
|
||||
return Collections.unmodifiableList(resources);
|
||||
}
|
||||
}
|
||||
|
||||
// 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<>();
|
||||
final Arc<Long> scratchArc = new Arc<>();
|
||||
final IntsRefBuilder scratchInts = new IntsRefBuilder();
|
||||
final BytesRefBuilder scratchBytes = new BytesRefBuilder();
|
||||
|
||||
FSTTermsEnum(FST<Long> fst) {
|
||||
this.fst = fst;
|
||||
in = new BytesRefFSTEnum<>(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 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);
|
||||
// TODO: we could do this lazily, better to try to push into FSTEnum though?
|
||||
in.seekExact(Util.toBytesRef(output, new BytesRefBuilder()));
|
||||
}
|
||||
|
||||
@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 PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImpactsEnum impacts(int flags) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -13,5 +13,4 @@
|
|||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
org.apache.lucene.codecs.memory.MemoryDocValuesFormat
|
||||
org.apache.lucene.codecs.memory.DirectDocValuesFormat
|
||||
|
|
|
@ -18,4 +18,4 @@ org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat
|
|||
org.apache.lucene.codecs.memory.DirectPostingsFormat
|
||||
org.apache.lucene.codecs.memory.FSTOrdPostingsFormat
|
||||
org.apache.lucene.codecs.memory.FSTPostingsFormat
|
||||
org.apache.lucene.codecs.memory.MemoryPostingsFormat
|
||||
|
||||
|
|
|
@ -1,39 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.codecs.memory;
|
||||
|
||||
|
||||
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;
|
||||
}
|
||||
}
|
|
@ -1,35 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.codecs.memory;
|
||||
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.index.BasePostingsFormatTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
/**
|
||||
* Tests MemoryPostingsFormat
|
||||
*/
|
||||
public class TestMemoryPostingsFormat extends BasePostingsFormatTestCase {
|
||||
// TODO: randomize doPack
|
||||
private final Codec codec = TestUtil.alwaysPostingsFormat(new MemoryPostingsFormat());
|
||||
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
return codec;
|
||||
}
|
||||
}
|
|
@ -42,13 +42,10 @@ public class TestExternalCodecs extends LuceneTestCase {
|
|||
|
||||
private final PostingsFormat ramFormat = PostingsFormat.forName("RAMOnly");
|
||||
private final PostingsFormat defaultFormat = TestUtil.getDefaultPostingsFormat();
|
||||
private final PostingsFormat memoryFormat = PostingsFormat.forName("Memory");
|
||||
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
if (field.equals("field2") || field.equals("id")) {
|
||||
return memoryFormat;
|
||||
} else if (field.equals("field1")) {
|
||||
if (field.equals("field2") || field.equals("field1") || field.equals("id")) {
|
||||
return defaultFormat;
|
||||
} else {
|
||||
return ramFormat;
|
||||
|
|
|
@ -89,7 +89,7 @@ public class TestPerFieldDocValuesFormat extends BaseDocValuesFormatTestCase {
|
|||
// we don't use RandomIndexWriter because it might add more docvalues than we expect !!!!1
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
|
||||
final DocValuesFormat fast = TestUtil.getDefaultDocValuesFormat();
|
||||
final DocValuesFormat slow = DocValuesFormat.forName("Memory");
|
||||
final DocValuesFormat slow = DocValuesFormat.forName("Direct");
|
||||
iwc.setCodec(new AssertingCodec() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
|
|
|
@ -33,7 +33,6 @@ import org.apache.lucene.codecs.PostingsFormat;
|
|||
import org.apache.lucene.codecs.asserting.AssertingCodec;
|
||||
import org.apache.lucene.codecs.blockterms.LuceneVarGapFixedInterval;
|
||||
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
|
@ -165,7 +164,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
|
|||
//((LogMergePolicy) iwconf.getMergePolicy()).setMergeFactor(10);
|
||||
iwconf.setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
|
||||
iwconf.setCodec(new MockCodec2()); // uses standard for field content
|
||||
iwconf.setCodec(new MockCodec()); // uses standard for field content
|
||||
writer = newWriter(dir, iwconf);
|
||||
// swap in new codec for currently written segments
|
||||
if (VERBOSE) {
|
||||
|
@ -218,24 +217,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
|
|||
public static class MockCodec extends AssertingCodec {
|
||||
final PostingsFormat luceneDefault = TestUtil.getDefaultPostingsFormat();
|
||||
final PostingsFormat direct = new DirectPostingsFormat();
|
||||
final PostingsFormat memory = new MemoryPostingsFormat();
|
||||
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
if (field.equals("id")) {
|
||||
return direct;
|
||||
} else if (field.equals("content")) {
|
||||
return memory;
|
||||
} else {
|
||||
return luceneDefault;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class MockCodec2 extends AssertingCodec {
|
||||
final PostingsFormat luceneDefault = TestUtil.getDefaultPostingsFormat();
|
||||
final PostingsFormat direct = new DirectPostingsFormat();
|
||||
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
if (field.equals("id")) {
|
||||
|
@ -287,9 +269,9 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
|
|||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
if ("id".equals(field)) {
|
||||
return new MemoryPostingsFormat();
|
||||
return new DirectPostingsFormat();
|
||||
} else if ("date".equals(field)) {
|
||||
return new MemoryPostingsFormat();
|
||||
return new DirectPostingsFormat();
|
||||
} else {
|
||||
return super.getPostingsFormatForField(field);
|
||||
}
|
||||
|
|
|
@ -33,7 +33,7 @@ import org.apache.lucene.util.LuceneTestCase;
|
|||
import org.apache.lucene.util.TimeUnits;
|
||||
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
|
||||
|
||||
@SuppressCodecs({"SimpleText", "Memory", "Direct"})
|
||||
@SuppressCodecs({"SimpleText", "Direct"})
|
||||
@TimeoutSuite(millis = 80 * TimeUnits.HOUR) // effectively no limit
|
||||
// The six hour time was achieved on a Linux 3.13 system with these specs:
|
||||
// 3-core AMD at 2.5Ghz, 12 GB RAM, 5GB test heap, 2 test JVMs, 2TB SATA.
|
||||
|
|
|
@ -35,7 +35,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
|
|||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.lucene.util.TimeUnits;
|
||||
|
||||
@SuppressCodecs({"SimpleText", "Memory", "Direct"})
|
||||
@SuppressCodecs({"SimpleText", "Direct"})
|
||||
@TimeoutSuite(millis = 80 * TimeUnits.HOUR) // effectively no limit
|
||||
@Monster("Takes ~30min")
|
||||
@SuppressSysoutChecks(bugUrl = "Stuff gets printed")
|
||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
|
|||
|
||||
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
|
||||
|
||||
@SuppressCodecs({"SimpleText", "Memory", "Direct"})
|
||||
@SuppressCodecs({"SimpleText", "Direct"})
|
||||
@TimeoutSuite(millis = 8 * TimeUnits.HOUR)
|
||||
// The two hour time was achieved on a Linux 3.13 system with these specs:
|
||||
// 3-core AMD at 2.5Ghz, 12 GB RAM, 5GB test heap, 2 test JVMs, 2TB SATA.
|
||||
|
|
|
@ -35,7 +35,7 @@ import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
|
|||
//
|
||||
// or: python -u /l/util/src/python/repeatLuceneTest.py -heap 6g -once -nolog -tmpDir /b/tmp -logDir /l/logs Test2BPoints.test2D -verbose
|
||||
|
||||
@SuppressCodecs({ "SimpleText", "Memory", "Direct", "Compressing" })
|
||||
@SuppressCodecs({ "SimpleText", "Direct", "Compressing" })
|
||||
@TimeoutSuite(millis = 365 * 24 * TimeUnits.HOUR) // hopefully ~1 year is long enough ;)
|
||||
@Monster("takes at least 4 hours and consumes many GB of temp disk space")
|
||||
public class Test2BPoints extends LuceneTestCase {
|
||||
|
|
|
@ -36,7 +36,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
|||
* Test indexes ~82M docs with 52 positions each, so you get > Integer.MAX_VALUE positions
|
||||
* @lucene.experimental
|
||||
*/
|
||||
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
|
||||
@SuppressCodecs({ "SimpleText", "Direct" })
|
||||
@Monster("uses lots of space and takes a few minutes")
|
||||
public class Test2BPositions extends LuceneTestCase {
|
||||
|
||||
|
|
|
@ -35,7 +35,7 @@ import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
|
|||
* Test indexes ~82M docs with 26 terms each, so you get > Integer.MAX_VALUE terms/docs pairs
|
||||
* @lucene.experimental
|
||||
*/
|
||||
@SuppressCodecs({ "SimpleText", "Memory", "Direct", "Compressing" })
|
||||
@SuppressCodecs({ "SimpleText", "Direct", "Compressing" })
|
||||
@TimeoutSuite(millis = 4 * TimeUnits.HOUR)
|
||||
public class Test2BPostings extends LuceneTestCase {
|
||||
|
||||
|
|
|
@ -42,7 +42,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
|||
* so you get > Integer.MAX_VALUE postings data for the term
|
||||
* @lucene.experimental
|
||||
*/
|
||||
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
|
||||
@SuppressCodecs({ "SimpleText", "Direct" })
|
||||
@Monster("takes ~20GB-30GB of space and 10 minutes")
|
||||
public class Test2BPostingsBytes extends LuceneTestCase {
|
||||
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
|
|||
|
||||
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
|
||||
|
||||
@SuppressCodecs({"SimpleText", "Memory", "Direct"})
|
||||
@SuppressCodecs({"SimpleText", "Direct"})
|
||||
@TimeoutSuite(millis = 80 * TimeUnits.HOUR) // effectively no limit
|
||||
// The six hour time was achieved on a Linux 3.13 system with these specs:
|
||||
// 3-core AMD at 2.5Ghz, 12 GB RAM, 5GB test heap, 2 test JVMs, 2TB SATA.
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
|
|||
|
||||
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
|
||||
|
||||
@SuppressCodecs({"SimpleText", "Memory", "Direct"})
|
||||
@SuppressCodecs({"SimpleText", "Direct"})
|
||||
@TimeoutSuite(millis = 80 * TimeUnits.HOUR) // effectively no limit
|
||||
// The six hour time was achieved on a Linux 3.13 system with these specs:
|
||||
// 3-core AMD at 2.5Ghz, 12 GB RAM, 5GB test heap, 2 test JVMs, 2TB SATA.
|
||||
|
|
|
@ -55,7 +55,7 @@ import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
|
|||
//
|
||||
// ant test -Dtests.monster=true -Dtests.heapsize=8g -Dtests.codec=Lucene62 -Dtestcase=Test2BTerms
|
||||
//
|
||||
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
|
||||
@SuppressCodecs({ "SimpleText", "Direct" })
|
||||
@Monster("very slow, use 5g minimum heap")
|
||||
@TimeoutSuite(millis = 80 * TimeUnits.HOUR) // effectively no limit
|
||||
@SuppressSysoutChecks(bugUrl = "Stuff gets printed")
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.lucene.codecs.Codec;
|
|||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.asserting.AssertingCodec;
|
||||
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
|
@ -1088,14 +1088,11 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
private static final class CustomPerFieldCodec extends AssertingCodec {
|
||||
private final PostingsFormat directFormat = PostingsFormat.forName("Direct");
|
||||
private final PostingsFormat defaultFormat = TestUtil.getDefaultPostingsFormat();
|
||||
private final PostingsFormat memoryFormat = PostingsFormat.forName("Memory");
|
||||
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
if (field.equals("id")) {
|
||||
return directFormat;
|
||||
} else if (field.equals("content")) {
|
||||
return memoryFormat;
|
||||
} else {
|
||||
return defaultFormat;
|
||||
}
|
||||
|
@ -1164,7 +1161,7 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
{
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
conf.setCodec(TestUtil.alwaysPostingsFormat(new MemoryPostingsFormat()));
|
||||
conf.setCodec(TestUtil.alwaysPostingsFormat(new DirectPostingsFormat()));
|
||||
IndexWriter w = new IndexWriter(dir, conf);
|
||||
expectThrows(IllegalArgumentException.class, () -> {
|
||||
w.addIndexes(toAdd);
|
||||
|
|
|
@ -40,7 +40,7 @@ import org.apache.lucene.util.TestUtil;
|
|||
* totalTermFreq of its integer value, and checks that the totalTermFreq is correct.
|
||||
*/
|
||||
// TODO: somehow factor this with BagOfPostings? it's almost the same
|
||||
@SuppressCodecs({"Direct", "Memory"}) // at night this makes like 200k/300k docs and will make Direct's heart beat!
|
||||
@SuppressCodecs({"Direct"}) // at night this makes like 200k/300k docs and will make Direct's heart beat!
|
||||
public class TestBagOfPositions extends LuceneTestCase {
|
||||
public void test() throws Exception {
|
||||
List<String> postingsList = new ArrayList<>();
|
||||
|
|
|
@ -38,7 +38,7 @@ import org.apache.lucene.util.TestUtil;
|
|||
* Simple test that adds numeric terms, where each term has the
|
||||
* docFreq of its integer value, and checks that the docFreq is correct.
|
||||
*/
|
||||
@SuppressCodecs({"Direct", "Memory"}) // at night this makes like 200k/300k docs and will make Direct's heart beat!
|
||||
@SuppressCodecs({"Direct"}) // at night this makes like 200k/300k docs and will make Direct's heart beat!
|
||||
public class TestBagOfPostings extends LuceneTestCase {
|
||||
public void test() throws Exception {
|
||||
List<String> postingsList = new ArrayList<>();
|
||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
|||
* Just like TestDuelingCodecs, only with a lot more documents.
|
||||
*/
|
||||
@Nightly
|
||||
@SuppressCodecs({"Memory", "Direct"}) // it can be too much for these codecs
|
||||
@SuppressCodecs({"Direct"}) // it can be too much for these codecs
|
||||
public class TestDuelingCodecsAtNight extends TestDuelingCodecs {
|
||||
|
||||
// use a big number of documents
|
||||
|
|
|
@ -177,7 +177,6 @@ public class TestIndexWriterCommit extends LuceneTestCase {
|
|||
|
||||
final String idFormat = TestUtil.getPostingsFormat("id");
|
||||
final String contentFormat = TestUtil.getPostingsFormat("content");
|
||||
assumeFalse("This test cannot run with Memory codec", idFormat.equals("Memory") || contentFormat.equals("Memory"));
|
||||
MockDirectoryWrapper dir = newMockDirectory();
|
||||
Analyzer analyzer;
|
||||
if (random().nextBoolean()) {
|
||||
|
|
|
@ -994,7 +994,6 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
|||
// TODO: move this test to its own class and just @SuppressCodecs?
|
||||
// TODO: is it enough to just use newFSDirectory?
|
||||
final String fieldFormat = TestUtil.getPostingsFormat("field");
|
||||
assumeFalse("This test cannot run with Memory codec", fieldFormat.equals("Memory"));
|
||||
assumeFalse("This test cannot run with SimpleText codec", fieldFormat.equals("SimpleText"));
|
||||
assumeFalse("This test cannot run with Direct codec", fieldFormat.equals("Direct"));
|
||||
final Random r = random();
|
||||
|
|
|
@ -39,7 +39,7 @@ import org.apache.lucene.util.TestUtil;
|
|||
import org.apache.lucene.util.TimeUnits;
|
||||
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
|
||||
|
||||
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
|
||||
@SuppressCodecs({ "SimpleText", "Direct" })
|
||||
@TimeoutSuite(millis = 8 * TimeUnits.HOUR)
|
||||
public class TestIndexWriterMaxDocs extends LuceneTestCase {
|
||||
|
||||
|
|
|
@ -162,7 +162,6 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase {
|
|||
|
||||
final String idFormat = TestUtil.getPostingsFormat("id");
|
||||
final String contentFormat = TestUtil.getPostingsFormat("content");
|
||||
assumeFalse("This test cannot run with Memory codec", idFormat.equals("Memory") || contentFormat.equals("Memory"));
|
||||
|
||||
int START_COUNT = 57;
|
||||
int NUM_DIR = TEST_NIGHTLY ? 50 : 5;
|
||||
|
|
|
@ -130,7 +130,6 @@ public class TestLazyProxSkipping extends LuceneTestCase {
|
|||
|
||||
public void testLazySkipping() throws IOException {
|
||||
final String fieldFormat = TestUtil.getPostingsFormat(this.field);
|
||||
assumeFalse("This test cannot run with Memory postings format", fieldFormat.equals("Memory"));
|
||||
assumeFalse("This test cannot run with Direct postings format", fieldFormat.equals("Direct"));
|
||||
assumeFalse("This test cannot run with SimpleText postings format", fieldFormat.equals("SimpleText"));
|
||||
|
||||
|
|
|
@ -35,7 +35,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
|||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
|
||||
@SuppressCodecs({ "SimpleText", "Direct" })
|
||||
public class TestLongPostings extends LuceneTestCase {
|
||||
|
||||
// Produces a realistic unicode random string that
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.junit.Before;
|
|||
// - mix in forceMerge, addIndexes
|
||||
// - randomoly mix in non-congruent docs
|
||||
|
||||
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
|
||||
@SuppressCodecs({ "SimpleText", "Direct" })
|
||||
public class TestNRTThreads extends ThreadedIndexingAndSearchingTestCase {
|
||||
|
||||
private boolean useNonNrtReaders = true;
|
||||
|
|
|
@ -43,7 +43,7 @@ import org.apache.lucene.util.TestUtil;
|
|||
* Test that norms info is preserved during index life - including
|
||||
* separate norms, addDocument, addIndexes, forceMerge.
|
||||
*/
|
||||
@SuppressCodecs({ "Memory", "Direct", "SimpleText" })
|
||||
@SuppressCodecs({ "Direct", "SimpleText" })
|
||||
@Slow
|
||||
public class TestNorms extends LuceneTestCase {
|
||||
static final String BYTE_TEST_FIELD = "normsTestByte";
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.util.Random;
|
|||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
|
||||
import org.apache.lucene.document.*;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
|
@ -42,9 +42,9 @@ public class TestRollingUpdates extends LuceneTestCase {
|
|||
|
||||
final LineFileDocs docs = new LineFileDocs(random);
|
||||
|
||||
//provider.register(new MemoryCodec());
|
||||
if (random().nextBoolean()) {
|
||||
Codec.setDefault(TestUtil.alwaysPostingsFormat(new MemoryPostingsFormat(random().nextBoolean(), random.nextFloat())));
|
||||
Codec.setDefault(TestUtil.alwaysPostingsFormat(
|
||||
new DirectPostingsFormat()));
|
||||
}
|
||||
|
||||
MockAnalyzer analyzer = new MockAnalyzer(random());
|
||||
|
|
|
@ -46,7 +46,7 @@ import org.apache.lucene.util.automaton.Automaton;
|
|||
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||
import org.apache.lucene.util.automaton.RegExp;
|
||||
|
||||
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
|
||||
@SuppressCodecs({ "SimpleText", "Direct" })
|
||||
public class TestTermsEnum extends LuceneTestCase {
|
||||
|
||||
public void test() throws Exception {
|
||||
|
|
|
@ -47,7 +47,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
|||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
|
||||
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
|
||||
@SuppressCodecs({ "SimpleText", "Direct" })
|
||||
public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearchingTestCase {
|
||||
|
||||
// Not guaranteed to reflect deletes:
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
||||
|
||||
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
|
||||
@SuppressCodecs({ "SimpleText", "Direct" })
|
||||
public class TestSearchWithThreads extends LuceneTestCase {
|
||||
int NUM_DOCS;
|
||||
static final int NUM_SEARCH_THREADS = 5;
|
||||
|
|
|
@ -50,7 +50,7 @@ import org.apache.lucene.util.LuceneTestCase;
|
|||
import org.apache.lucene.util.NamedThreadFactory;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
|
||||
@SuppressCodecs({ "SimpleText", "Direct" })
|
||||
public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
|
||||
|
||||
boolean warmCalled;
|
||||
|
|
|
@ -41,7 +41,7 @@ import org.apache.lucene.util.TestUtil;
|
|||
// - test pulling docs in 2nd round trip...
|
||||
// - filter too
|
||||
|
||||
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
|
||||
@SuppressCodecs({ "SimpleText", "Direct" })
|
||||
public class TestShardSearching extends ShardSearchingTestBase {
|
||||
|
||||
private static class PreviousSearchState {
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
|||
|
||||
/** Tests for SortedSetSortField selectors other than MIN,
|
||||
* these require optional codec support (random access to ordinals) */
|
||||
@SuppressCodecs({"Memory", "SimpleText"}) // TODO: fix MemoryDV to support random access to ordinals
|
||||
@SuppressCodecs({"SimpleText"})
|
||||
public class TestSortedSetSelector extends LuceneTestCase {
|
||||
|
||||
public void testMax() throws Exception {
|
||||
|
|
|
@ -81,7 +81,7 @@ import static org.apache.lucene.util.fst.FSTTester.getRandomString;
|
|||
import static org.apache.lucene.util.fst.FSTTester.simpleRandomString;
|
||||
import static org.apache.lucene.util.fst.FSTTester.toIntsRef;
|
||||
|
||||
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
|
||||
@SuppressCodecs({ "SimpleText", "Direct" })
|
||||
@Slow
|
||||
public class TestFSTs extends LuceneTestCase {
|
||||
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
|||
/**
|
||||
* Random testing for {@link InetAddressRange}
|
||||
*/
|
||||
@SuppressCodecs({"Direct", "Memory"})
|
||||
@SuppressCodecs({"Direct"})
|
||||
public class TestInetAddressRangeQueries extends BaseRangeFieldQueryTestCase {
|
||||
private static final String FIELD_NAME = "ipRangeField";
|
||||
|
||||
|
|
|
@ -51,7 +51,7 @@ import org.apache.lucene.util.TestUtil;
|
|||
|
||||
/** Child process with silly naive TCP socket server to handle
|
||||
* between-node commands, launched for each node by TestNRTReplication. */
|
||||
@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
|
||||
@SuppressCodecs({"MockRandom", "Direct", "SimpleText"})
|
||||
@SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")
|
||||
@SuppressForbidden(reason = "We need Unsafe to actually crush :-)")
|
||||
public class SimpleServer extends LuceneTestCase {
|
||||
|
|
|
@ -44,7 +44,7 @@ import org.apache.lucene.util.TestUtil;
|
|||
import com.carrotsearch.randomizedtesting.SeedUtils;
|
||||
|
||||
// MockRandom's .sd file has no index header/footer:
|
||||
@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
|
||||
@SuppressCodecs({"MockRandom", "Direct", "SimpleText"})
|
||||
@SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")
|
||||
public class TestNRTReplication extends LuceneTestCase {
|
||||
|
||||
|
|
|
@ -112,7 +112,7 @@ import com.carrotsearch.randomizedtesting.SeedUtils;
|
|||
*/
|
||||
|
||||
// MockRandom's .sd file has no index header/footer:
|
||||
@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
|
||||
@SuppressCodecs({"MockRandom", "Direct", "SimpleText"})
|
||||
@SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")
|
||||
public class TestStressNRTReplication extends LuceneTestCase {
|
||||
|
||||
|
|
|
@ -549,9 +549,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
|
|||
Directory dir = null;
|
||||
RandomIndexWriter iw = null;
|
||||
final String postingsFormat = TestUtil.getPostingsFormat("dummy");
|
||||
boolean codecOk = iterations * maxWordLength < 100000 ||
|
||||
!(postingsFormat.equals("Memory") ||
|
||||
postingsFormat.equals("SimpleText"));
|
||||
boolean codecOk = iterations * maxWordLength < 100000 || !(postingsFormat.equals("SimpleText"));
|
||||
if (rarely(random) && codecOk) {
|
||||
dir = newFSDirectory(createTempDir("bttc"));
|
||||
iw = new RandomIndexWriter(new Random(seed), dir, a);
|
||||
|
|
|
@ -627,7 +627,6 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
|
|||
@Nightly
|
||||
public void testRandomBig() throws Exception {
|
||||
assumeFalse("Direct codec can OOME on this test", TestUtil.getDocValuesFormat(FIELD_NAME).equals("Direct"));
|
||||
assumeFalse("Memory codec can OOME on this test", TestUtil.getDocValuesFormat(FIELD_NAME).equals("Memory"));
|
||||
doTestRandom(200000);
|
||||
}
|
||||
|
||||
|
|
|
@ -46,8 +46,6 @@ import org.apache.lucene.codecs.memory.DirectDocValuesFormat;
|
|||
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.FSTOrdPostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.FSTPostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.MemoryDocValuesFormat;
|
||||
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
|
||||
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
@ -205,14 +203,11 @@ public class RandomCodec extends AssertingCodec {
|
|||
new LuceneVarGapFixedInterval(TestUtil.nextInt(random, 1, 1000)),
|
||||
new LuceneVarGapDocFreqInterval(TestUtil.nextInt(random, 1, 100), TestUtil.nextInt(random, 1, 1000)),
|
||||
TestUtil.getDefaultPostingsFormat(),
|
||||
new AssertingPostingsFormat(),
|
||||
new MemoryPostingsFormat(true, random.nextFloat()),
|
||||
new MemoryPostingsFormat(false, random.nextFloat()));
|
||||
new AssertingPostingsFormat());
|
||||
|
||||
addDocValues(avoidCodecs,
|
||||
TestUtil.getDefaultDocValuesFormat(),
|
||||
new DirectDocValuesFormat(), // maybe not a great idea...
|
||||
new MemoryDocValuesFormat(),
|
||||
TestUtil.getDefaultDocValuesFormat(),
|
||||
new AssertingDocValuesFormat());
|
||||
|
||||
|
|
|
@ -980,7 +980,7 @@ public final 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") || dvFormat.equals("Memory")) {
|
||||
if (dvFormat.equals("Lucene40") || dvFormat.equals("Lucene42")) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
|
|
|
@ -41,6 +41,10 @@ Upgrade Notes
|
|||
method signatures was required to fix it. Third party components that use this API
|
||||
need to be updated. (Eros Taborelli, Erick Erickson, ab)
|
||||
|
||||
* LUCENE-8267: Memory codecs have been removed from the codebase (MemoryPostings,
|
||||
MemoryDocValues). If you used postingsFormat="Memory" or docValuesFormat="Memory"
|
||||
switch to "Direct" instead. (Dawid Weiss)
|
||||
|
||||
================== 7.4.0 ==================
|
||||
|
||||
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
|
||||
|
|
|
@ -16,11 +16,10 @@
|
|||
limitations under the License.
|
||||
-->
|
||||
<schema name="codec" version="1.2">
|
||||
<fieldType name="string_direct" class="solr.StrField" postingsFormat="Direct"/>
|
||||
<fieldType name="string_direct" class="solr.StrField" postingsFormat="Direct" docValuesFormat="Direct"/>
|
||||
<fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene50"/>
|
||||
|
||||
<fieldType name="string_disk" class="solr.StrField" docValuesFormat="Lucene70"/>
|
||||
<fieldType name="string_memory" class="solr.StrField" docValuesFormat="Memory"/>
|
||||
|
||||
<fieldType name="string" class="solr.StrField"/>
|
||||
|
||||
|
@ -39,7 +38,6 @@
|
|||
<field name="string_standard_f" type="string_standard" indexed="true" stored="true"/>
|
||||
|
||||
<field name="string_disk_f" type="string_disk" indexed="false" stored="false" docValues="true" default=""/>
|
||||
<field name="string_memory_f" type="string_memory" indexed="false" stored="false" docValues="true" default=""/>
|
||||
|
||||
<field name="string_f" type="string" indexed="true" stored="true" docValues="true" required="true"/>
|
||||
<field name="text" type="text_general" indexed="true" stored="true"/>
|
||||
|
@ -48,7 +46,6 @@
|
|||
<dynamicField name="*_standard" type="string_standard" indexed="true" stored="true"/>
|
||||
|
||||
<dynamicField name="*_disk" type="string_disk" indexed="false" stored="false" docValues="true"/>
|
||||
<dynamicField name="*_memory" type="string_memory" indexed="false" stored="false" docValues="true"/>
|
||||
|
||||
<uniqueKey>string_f</uniqueKey>
|
||||
</schema>
|
||||
|
|
|
@ -63,9 +63,8 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
|
|||
SchemaField schemaField = fields.get("string_disk_f");
|
||||
PerFieldDocValuesFormat format = (PerFieldDocValuesFormat) codec.docValuesFormat();
|
||||
assertEquals(TestUtil.getDefaultDocValuesFormat().getName(), format.getDocValuesFormatForField(schemaField.getName()).getName());
|
||||
schemaField = fields.get("string_memory_f");
|
||||
assertEquals("Memory",
|
||||
format.getDocValuesFormatForField(schemaField.getName()).getName());
|
||||
schemaField = fields.get("string_direct_f");
|
||||
assertEquals("Direct", format.getDocValuesFormatForField(schemaField.getName()).getName());
|
||||
schemaField = fields.get("string_f");
|
||||
assertEquals(TestUtil.getDefaultDocValuesFormat().getName(),
|
||||
format.getDocValuesFormatForField(schemaField.getName()).getName());
|
||||
|
@ -87,8 +86,8 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
|
|||
|
||||
assertEquals(TestUtil.getDefaultDocValuesFormat().getName(), format.getDocValuesFormatForField("foo_disk").getName());
|
||||
assertEquals(TestUtil.getDefaultDocValuesFormat().getName(), format.getDocValuesFormatForField("bar_disk").getName());
|
||||
assertEquals("Memory", format.getDocValuesFormatForField("foo_memory").getName());
|
||||
assertEquals("Memory", format.getDocValuesFormatForField("bar_memory").getName());
|
||||
assertEquals("Direct", format.getDocValuesFormatForField("foo_direct").getName());
|
||||
assertEquals("Direct", format.getDocValuesFormatForField("bar_direct").getName());
|
||||
}
|
||||
|
||||
private void reloadCoreAndRecreateIndex() {
|
||||
|
|
|
@ -40,7 +40,7 @@ import org.junit.BeforeClass;
|
|||
* Tests the behavior of <code>field(foo,min|max)</code> on numerious types of multivalued 'foo' fields,
|
||||
* as well as the beahvior of sorting on <code>foo asc|desc</code> to implicitly choose the min|max.
|
||||
*/
|
||||
@SuppressCodecs({"Memory", "SimpleText"}) // see TestSortedSetSelector
|
||||
@SuppressCodecs({"SimpleText"}) // see TestSortedSetSelector
|
||||
public class TestMinMaxOnMultiValuedField extends SolrTestCaseJ4 {
|
||||
|
||||
/** Initializes core and does some sanity checking of schema */
|
||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
|||
*
|
||||
* @see SortByFunctionTest
|
||||
**/
|
||||
@SuppressCodecs({"Memory", "SimpleText"}) // see TestSortedSetSelector
|
||||
@SuppressCodecs({"SimpleText"}) // see TestSortedSetSelector
|
||||
public class TestSortByMinMaxFunction extends SortByFunctionTest {
|
||||
|
||||
@Override
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.lucene.util.LuceneTestCase;
|
|||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.solr.index.SlowCompositeReaderWrapper;
|
||||
|
||||
@LuceneTestCase.SuppressCodecs({"Memory"})
|
||||
public class TestDocTermOrdsUninvertLimit extends LuceneTestCase {
|
||||
|
||||
/* UnInvertedField had a reference block limitation of 2^24. This unit test triggered it.
|
||||
|
|
|
@ -165,38 +165,11 @@ public class TestFieldCacheVsDocValues extends LuceneTestCase {
|
|||
b.length = bytes.length;
|
||||
doc.add(new BinaryDocValuesField("field", b));
|
||||
doc.add(new StringField("id", "" + docID, Field.Store.YES));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
} catch (IllegalArgumentException iae) {
|
||||
if (iae.getMessage().indexOf("is too large") == -1) {
|
||||
throw iae;
|
||||
} else {
|
||||
// OK: some codecs can't handle binary DV > 32K
|
||||
assertFalse(codecAcceptsHugeBinaryValues("field"));
|
||||
w.rollback();
|
||||
d.close();
|
||||
return;
|
||||
}
|
||||
}
|
||||
w.addDocument(doc);
|
||||
}
|
||||
|
||||
|
||||
DirectoryReader r;
|
||||
try {
|
||||
r = DirectoryReader.open(w);
|
||||
} catch (IllegalArgumentException iae) {
|
||||
if (iae.getMessage().indexOf("is too large") == -1) {
|
||||
throw iae;
|
||||
} else {
|
||||
assertFalse(codecAcceptsHugeBinaryValues("field"));
|
||||
|
||||
// OK: some codecs can't handle binary DV > 32K
|
||||
w.rollback();
|
||||
d.close();
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
|
||||
try (LeafReader ar = SlowCompositeReaderWrapper.wrap(r)) {
|
||||
TestUtil.checkReader(ar);
|
||||
|
@ -210,86 +183,11 @@ public class TestFieldCacheVsDocValues extends LuceneTestCase {
|
|||
assertEquals(expected.length, bytes.length);
|
||||
assertEquals(new BytesRef(expected), bytes);
|
||||
}
|
||||
|
||||
assertTrue(codecAcceptsHugeBinaryValues("field"));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static final int LARGE_BINARY_FIELD_LENGTH = (1 << 15) - 2;
|
||||
|
||||
// TODO: get this out of here and into the deprecated codecs (4.0, 4.2)
|
||||
public void testHugeBinaryValueLimit() throws Exception {
|
||||
// We only test DVFormats that have a limit
|
||||
assumeFalse("test requires codec with limits on max binary field length", codecAcceptsHugeBinaryValues("field"));
|
||||
Analyzer analyzer = new MockAnalyzer(random());
|
||||
// FSDirectory because SimpleText will consume gobbs of
|
||||
// space when storing big binary values:
|
||||
Directory d = newFSDirectory(createTempDir("hugeBinaryValues"));
|
||||
boolean doFixed = random().nextBoolean();
|
||||
int numDocs;
|
||||
int fixedLength = 0;
|
||||
if (doFixed) {
|
||||
// Sometimes make all values fixed length since some
|
||||
// codecs have different code paths for this:
|
||||
numDocs = TestUtil.nextInt(random(), 10, 20);
|
||||
fixedLength = LARGE_BINARY_FIELD_LENGTH;
|
||||
} else {
|
||||
numDocs = TestUtil.nextInt(random(), 100, 200);
|
||||
}
|
||||
IndexWriter w = new IndexWriter(d, newIndexWriterConfig(analyzer));
|
||||
List<byte[]> docBytes = new ArrayList<>();
|
||||
long totalBytes = 0;
|
||||
for(int docID=0;docID<numDocs;docID++) {
|
||||
// we don't use RandomIndexWriter because it might add
|
||||
// more docvalues than we expect !!!!
|
||||
|
||||
// Must be > 64KB in size to ensure more than 2 pages in
|
||||
// PagedBytes would be needed:
|
||||
int numBytes;
|
||||
if (doFixed) {
|
||||
numBytes = fixedLength;
|
||||
} else if (docID == 0 || random().nextInt(5) == 3) {
|
||||
numBytes = LARGE_BINARY_FIELD_LENGTH;
|
||||
} else {
|
||||
numBytes = TestUtil.nextInt(random(), 1, LARGE_BINARY_FIELD_LENGTH);
|
||||
}
|
||||
totalBytes += numBytes;
|
||||
if (totalBytes > 5 * 1024*1024) {
|
||||
break;
|
||||
}
|
||||
byte[] bytes = new byte[numBytes];
|
||||
random().nextBytes(bytes);
|
||||
docBytes.add(bytes);
|
||||
Document doc = new Document();
|
||||
BytesRef b = new BytesRef(bytes);
|
||||
b.length = bytes.length;
|
||||
doc.add(new BinaryDocValuesField("field", b));
|
||||
doc.add(new StringField("id", ""+docID, Field.Store.YES));
|
||||
w.addDocument(doc);
|
||||
}
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
w.close();
|
||||
|
||||
LeafReader ar = SlowCompositeReaderWrapper.wrap(r);
|
||||
TestUtil.checkReader(ar);
|
||||
|
||||
BinaryDocValues s = FieldCache.DEFAULT.getTerms(ar, "field");
|
||||
for(int docID=0;docID<docBytes.size();docID++) {
|
||||
assertEquals(docID, s.nextDoc());
|
||||
Document doc = ar.document(docID);
|
||||
BytesRef bytes = s.binaryValue();
|
||||
byte[] expected = docBytes.get(Integer.parseInt(doc.get("id")));
|
||||
assertEquals(expected.length, bytes.length);
|
||||
assertEquals(new BytesRef(expected), bytes);
|
||||
}
|
||||
|
||||
ar.close();
|
||||
d.close();
|
||||
}
|
||||
|
||||
private void doTestSortedVsFieldCache(int minLength, int maxLength) throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -616,9 +514,4 @@ public class TestFieldCacheVsDocValues extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected boolean codecAcceptsHugeBinaryValues(String field) {
|
||||
String name = TestUtil.getDocValuesFormat(field);
|
||||
return !(name.equals("Memory")); // Direct has a different type of limit
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue