mirror of https://github.com/apache/lucene.git
LUCENE-5969: use sparsebitset to expand sparse encoding to cover more absurd cases
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5969@1632631 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
1d3e228646
commit
922d1edbff
|
@ -47,6 +47,7 @@ class Lucene50NormsConsumer extends NormsConsumer {
|
|||
static final byte CONST_COMPRESSED = 2;
|
||||
static final byte UNCOMPRESSED = 3;
|
||||
static final byte INDIRECT = 4;
|
||||
static final byte PATCHED = 5;
|
||||
static final int BLOCK_SIZE = 1 << 14;
|
||||
|
||||
// threshold for indirect encoding, computed as 1 - 1/log2(maxint)
|
||||
|
@ -82,6 +83,11 @@ class Lucene50NormsConsumer extends NormsConsumer {
|
|||
|
||||
@Override
|
||||
public void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {
|
||||
writeNormsField(field, values, 0);
|
||||
}
|
||||
|
||||
private void writeNormsField(FieldInfo field, Iterable<Number> values, int level) throws IOException {
|
||||
assert level <= 1; // we only "recurse" once in the indirect case
|
||||
meta.writeVInt(field.number);
|
||||
long minValue = Long.MAX_VALUE;
|
||||
long maxValue = Long.MIN_VALUE;
|
||||
|
@ -89,16 +95,12 @@ class Lucene50NormsConsumer extends NormsConsumer {
|
|||
NormMap uniqueValues = new NormMap();
|
||||
|
||||
int count = 0;
|
||||
int missingCount = 0;
|
||||
|
||||
for (Number nv : values) {
|
||||
if (nv == null) {
|
||||
throw new IllegalStateException("illegal norms data for field " + field.name + ", got null for value: " + count);
|
||||
}
|
||||
final long v = nv.longValue();
|
||||
if (v == 0) {
|
||||
missingCount++;
|
||||
}
|
||||
|
||||
minValue = Math.min(minValue, v);
|
||||
maxValue = Math.max(maxValue, v);
|
||||
|
@ -115,9 +117,15 @@ class Lucene50NormsConsumer extends NormsConsumer {
|
|||
if (uniqueValues != null && uniqueValues.size == 1) {
|
||||
// 0 bpv
|
||||
addConstant(minValue);
|
||||
} else if (count > 256 && missingCount > count * INDIRECT_THRESHOLD) {
|
||||
// sparse encoding
|
||||
addIndirect(field, values, count, missingCount);
|
||||
} else if (level == 0 && count > 256 && uniqueValues != null && uniqueValues.maxFreq() > count * INDIRECT_THRESHOLD) {
|
||||
long commonValue = uniqueValues.getDecodeTable()[uniqueValues.maxOrd()];
|
||||
if (commonValue == 0) {
|
||||
// if the common value is missing, don't waste RAM on a bitset, since we won't be searching those docs
|
||||
addIndirect(field, values, count, uniqueValues);
|
||||
} else {
|
||||
// otherwise, write a sparse bitset, where 1 indicates 'uncommon value'.
|
||||
addPatched(field, values, count, uniqueValues);
|
||||
}
|
||||
} else if (uniqueValues != null) {
|
||||
// small number of unique values: this is the typical case:
|
||||
FormatAndBits compression = fastestFormatAndBits(uniqueValues.size-1);
|
||||
|
@ -200,10 +208,65 @@ class Lucene50NormsConsumer extends NormsConsumer {
|
|||
writer.finish();
|
||||
}
|
||||
|
||||
private void addIndirect(FieldInfo field, final Iterable<Number> values, int count, int missingCount) throws IOException {
|
||||
meta.writeVInt(count - missingCount);
|
||||
// encodes only uncommon values in a sparse bitset
|
||||
// access is constant time, and the common case is predictable
|
||||
// exceptions nest either to CONST (if there are only 2 values), or INDIRECT (if there are > 2 values)
|
||||
private void addPatched(FieldInfo field, final Iterable<Number> values, int count, NormMap uniqueValues) throws IOException {
|
||||
final long decodeTable[] = uniqueValues.getDecodeTable();
|
||||
int commonCount = uniqueValues.maxFreq();
|
||||
final long commonValue = decodeTable[uniqueValues.maxOrd()];
|
||||
|
||||
meta.writeVInt(count - commonCount);
|
||||
meta.writeByte(PATCHED);
|
||||
meta.writeLong(data.getFilePointer());
|
||||
|
||||
// write docs with value
|
||||
writeDocsWithValue(values, commonValue);
|
||||
|
||||
// write exceptions: only two cases make sense
|
||||
// bpv = 1 (folded into sparse bitset already)
|
||||
// bpv > 1 (add indirect exception table)
|
||||
meta.writeVInt(field.number);
|
||||
if (uniqueValues.size == 2) {
|
||||
// special case: implicit in bitset
|
||||
int otherOrd = uniqueValues.maxOrd() == 0 ? 1 : 0;
|
||||
addConstant(decodeTable[otherOrd]);
|
||||
} else {
|
||||
// exception table
|
||||
addIndirect(field, values, count, uniqueValues);
|
||||
}
|
||||
}
|
||||
|
||||
// encodes values as sparse array: keys[] and values[]
|
||||
// access is log(N) where N = keys.length (slow!)
|
||||
// so this is only appropriate as an exception table for patched, or when common value is 0 (wont be accessed by searching)
|
||||
private void addIndirect(FieldInfo field, final Iterable<Number> values, int count, NormMap uniqueValues) throws IOException {
|
||||
int commonCount = uniqueValues.maxFreq();
|
||||
final long commonValue = uniqueValues.getDecodeTable()[uniqueValues.maxOrd()];
|
||||
|
||||
meta.writeVInt(count - commonCount);
|
||||
meta.writeByte(INDIRECT);
|
||||
meta.writeLong(data.getFilePointer());
|
||||
|
||||
// write docs with value
|
||||
writeDocsWithValue(values, commonValue);
|
||||
|
||||
// write actual values
|
||||
writeNormsField(field, new Iterable<Number>() {
|
||||
@Override
|
||||
public Iterator<Number> iterator() {
|
||||
return new FilterIterator<Number,Number>(values.iterator()) {
|
||||
@Override
|
||||
protected boolean predicateFunction(Number value) {
|
||||
return value.longValue() != commonValue;
|
||||
}
|
||||
};
|
||||
}
|
||||
}, 1);
|
||||
}
|
||||
|
||||
private void writeDocsWithValue(final Iterable<Number> values, long commonValue) throws IOException {
|
||||
data.writeLong(commonValue);
|
||||
data.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
data.writeVInt(BLOCK_SIZE);
|
||||
|
||||
|
@ -212,25 +275,12 @@ class Lucene50NormsConsumer extends NormsConsumer {
|
|||
int doc = 0;
|
||||
for (Number n : values) {
|
||||
long v = n.longValue();
|
||||
if (v != 0) {
|
||||
if (v != commonValue) {
|
||||
writer.add(doc);
|
||||
}
|
||||
doc++;
|
||||
}
|
||||
writer.finish();
|
||||
|
||||
// write actual values
|
||||
addNormsField(field, new Iterable<Number>() {
|
||||
@Override
|
||||
public Iterator<Number> iterator() {
|
||||
return new FilterIterator<Number,Number>(values.iterator()) {
|
||||
@Override
|
||||
protected boolean predicateFunction(Number value) {
|
||||
return value.longValue() != 0;
|
||||
}
|
||||
};
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -259,6 +309,7 @@ class Lucene50NormsConsumer extends NormsConsumer {
|
|||
static class NormMap {
|
||||
// we use short: at most we will add 257 values to this map before its rejected as too big above.
|
||||
final short[] singleByteRange = new short[256];
|
||||
final int[] freqs = new int[257];
|
||||
final Map<Long,Short> other = new HashMap<Long,Short>();
|
||||
int size;
|
||||
|
||||
|
@ -273,18 +324,24 @@ class Lucene50NormsConsumer extends NormsConsumer {
|
|||
int index = (int) (l + 128);
|
||||
short previous = singleByteRange[index];
|
||||
if (previous < 0) {
|
||||
singleByteRange[index] = (short) size;
|
||||
short slot = (short) size;
|
||||
singleByteRange[index] = slot;
|
||||
freqs[slot]++;
|
||||
size++;
|
||||
return true;
|
||||
} else {
|
||||
freqs[previous]++;
|
||||
return false;
|
||||
}
|
||||
} else {
|
||||
if (!other.containsKey(l)) {
|
||||
Short previous = other.get(l);
|
||||
if (previous == null) {
|
||||
freqs[size]++;
|
||||
other.put(l, (short)size);
|
||||
size++;
|
||||
return true;
|
||||
} else {
|
||||
freqs[previous]++;
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
@ -315,5 +372,35 @@ class Lucene50NormsConsumer extends NormsConsumer {
|
|||
}
|
||||
return decode;
|
||||
}
|
||||
|
||||
// TODO: if we need more complicated frequency-driven optos, maybe add 'finish' to this api
|
||||
// and sort all ords by frequency. we could then lower BPV and waste a value to represent 'patched',
|
||||
|
||||
/** retrieves frequency table for items (indexed by ordinal) */
|
||||
public int[] getFreqs() {
|
||||
return freqs;
|
||||
}
|
||||
|
||||
/** sugar: returns max value over getFreqs() */
|
||||
public int maxFreq() {
|
||||
int max = 0;
|
||||
for (int i = 0; i < size; i++) {
|
||||
max = Math.max(max, freqs[i]);
|
||||
}
|
||||
return max;
|
||||
}
|
||||
|
||||
/** sugar: returns ordinal with maxFreq() */
|
||||
public int maxOrd() {
|
||||
long max = 0;
|
||||
int maxOrd = 0;
|
||||
for (int i = 0; i < size; i++) {
|
||||
if (freqs[i] > max) {
|
||||
max = freqs[i];
|
||||
maxOrd = i;
|
||||
}
|
||||
}
|
||||
return maxOrd;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -50,6 +50,9 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
* <li>Indirect: when norms are extremely sparse, missing values are omitted.
|
||||
* Access to an individual value is slower, but missing norm values are never accessed
|
||||
* by search code.
|
||||
* <li>Patched: when a single norm value dominates, a sparse bitset encodes docs with exceptions,
|
||||
* so that access to the common value is still very fast. outliers fall thru to an exception
|
||||
* handling mechanism (Indirect or Constant).
|
||||
* </ul>
|
||||
* <p>
|
||||
* Files:
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.lucene.util.Accountable;
|
|||
import org.apache.lucene.util.Accountables;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.SparseFixedBitSet;
|
||||
import org.apache.lucene.util.packed.BlockPackedReader;
|
||||
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
@ -48,6 +49,7 @@ import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.DELTA_COMP
|
|||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.TABLE_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.UNCOMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.INDIRECT;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.PATCHED;
|
||||
|
||||
/**
|
||||
* Reader for {@link Lucene50NormsFormat}
|
||||
|
@ -63,6 +65,7 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
|
||||
private final AtomicLong ramBytesUsed;
|
||||
private final AtomicInteger activeCount = new AtomicInteger();
|
||||
private final int maxDoc;
|
||||
|
||||
private final boolean merging;
|
||||
|
||||
|
@ -75,11 +78,13 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
instancesInfo.putAll(original.instancesInfo);
|
||||
ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
|
||||
activeCount.set(original.activeCount.get());
|
||||
maxDoc = original.maxDoc;
|
||||
merging = true;
|
||||
}
|
||||
|
||||
Lucene50NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
merging = false;
|
||||
maxDoc = state.segmentInfo.getDocCount();
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
|
||||
int version = -1;
|
||||
|
@ -146,6 +151,7 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
case TABLE_COMPRESSED:
|
||||
case DELTA_COMPRESSED:
|
||||
break;
|
||||
case PATCHED:
|
||||
case INDIRECT:
|
||||
if (meta.readVInt() != info.number) {
|
||||
throw new CorruptIndexException("indirect norms entry for field: " + info.name + " is corrupt", meta);
|
||||
|
@ -254,6 +260,7 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
}
|
||||
case INDIRECT: {
|
||||
data.seek(entry.offset);
|
||||
final long common = data.readLong();
|
||||
int packedIntsVersion = data.readVInt();
|
||||
int blockSize = data.readVInt();
|
||||
final MonotonicBlockPackedReader live = MonotonicBlockPackedReader.of(data, packedIntsVersion, blockSize, entry.count, false);
|
||||
|
@ -279,7 +286,34 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
return values.get(mid);
|
||||
}
|
||||
}
|
||||
return 0;
|
||||
return common;
|
||||
}
|
||||
};
|
||||
break;
|
||||
}
|
||||
case PATCHED: {
|
||||
data.seek(entry.offset);
|
||||
final long common = data.readLong();
|
||||
int packedIntsVersion = data.readVInt();
|
||||
int blockSize = data.readVInt();
|
||||
MonotonicBlockPackedReader live = MonotonicBlockPackedReader.of(data, packedIntsVersion, blockSize, entry.count, true);
|
||||
final SparseFixedBitSet set = new SparseFixedBitSet(maxDoc);
|
||||
for (int i = 0; i < live.size(); i++) {
|
||||
int doc = (int) live.get(i);
|
||||
set.set(doc);
|
||||
}
|
||||
LoadedNorms nestedInstance = loadNorms(entry.nested);
|
||||
instance.ramBytesUsed = set.ramBytesUsed() + nestedInstance.ramBytesUsed;
|
||||
instance.info = Accountables.namedAccountable("patched -> " + nestedInstance.info, instance.ramBytesUsed);
|
||||
final NumericDocValues values = nestedInstance.norms;
|
||||
instance.norms = new NumericDocValues() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
if (set.get(docID)) {
|
||||
return values.get(docID);
|
||||
} else {
|
||||
return common;
|
||||
}
|
||||
}
|
||||
};
|
||||
break;
|
||||
|
|
|
@ -17,7 +17,13 @@ package org.apache.lucene.codecs.lucene50;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.NormMap;
|
||||
import org.apache.lucene.index.BaseNormsFormatTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
|
@ -30,5 +36,92 @@ public class TestLucene50NormsFormat extends BaseNormsFormatTestCase {
|
|||
@Override
|
||||
protected Codec getCodec() {
|
||||
return codec;
|
||||
}
|
||||
}
|
||||
|
||||
// NormMap is rather complicated, doing domain encoding / tracking frequencies etc.
|
||||
// test it directly some here...
|
||||
|
||||
public void testNormMapSimple() {
|
||||
NormMap map = new NormMap();
|
||||
map.add(10);
|
||||
map.add(5);
|
||||
map.add(4);
|
||||
map.add(10);
|
||||
assertEquals(3, map.size);
|
||||
|
||||
// first come, first serve ord assignment
|
||||
|
||||
// encode
|
||||
assertEquals(0, map.getOrd(10));
|
||||
assertEquals(1, map.getOrd(5));
|
||||
assertEquals(2, map.getOrd(4));
|
||||
|
||||
// decode
|
||||
long decode[] = map.getDecodeTable();
|
||||
assertEquals(10, decode[0]);
|
||||
assertEquals(5, decode[1]);
|
||||
assertEquals(4, decode[2]);
|
||||
|
||||
// freqs
|
||||
int freqs[] = map.getFreqs();
|
||||
assertEquals(2, freqs[0]);
|
||||
assertEquals(1, freqs[1]);
|
||||
assertEquals(1, freqs[2]);
|
||||
|
||||
assertEquals(2, map.maxFreq());
|
||||
}
|
||||
|
||||
public void testNormMapRandom() {
|
||||
Map<Long,Integer> freqs = new HashMap<>();
|
||||
Map<Long,Integer> ords = new HashMap<>();
|
||||
|
||||
Set<Long> uniqueValuesSet = new HashSet<>();
|
||||
int numUniqValues = TestUtil.nextInt(random(), 1, 256);
|
||||
for (int i = 0; i < numUniqValues; i++) {
|
||||
if (random().nextBoolean()) {
|
||||
uniqueValuesSet.add(TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE));
|
||||
} else {
|
||||
uniqueValuesSet.add(TestUtil.nextLong(random(), Byte.MIN_VALUE, Byte.MAX_VALUE));
|
||||
}
|
||||
}
|
||||
|
||||
Long uniqueValues[] = uniqueValuesSet.toArray(new Long[uniqueValuesSet.size()]);
|
||||
|
||||
NormMap map = new NormMap();
|
||||
int numdocs = TestUtil.nextInt(random(), 1, 100000);
|
||||
for (int i = 0; i < numdocs; i++) {
|
||||
long value = uniqueValues[random().nextInt(uniqueValues.length)];
|
||||
// now add to both expected and actual
|
||||
map.add(value);
|
||||
|
||||
Integer ord = ords.get(value);
|
||||
if (ord == null) {
|
||||
ord = ords.size();
|
||||
ords.put(value, ord);
|
||||
freqs.put(value, 1);
|
||||
} else {
|
||||
freqs.put(value, freqs.get(value)+1);
|
||||
}
|
||||
}
|
||||
|
||||
// value -> ord
|
||||
assertEquals(ords.size(), map.size);
|
||||
for (Map.Entry<Long,Integer> kv : ords.entrySet()) {
|
||||
assertEquals(kv.getValue().intValue(), map.getOrd(kv.getKey()));
|
||||
}
|
||||
|
||||
// ord -> value
|
||||
Map<Long,Integer> reversed = new HashMap<>();
|
||||
long table[] = map.getDecodeTable();
|
||||
for (int i = 0; i < map.size; i++) {
|
||||
reversed.put(table[i], i);
|
||||
}
|
||||
assertEquals(ords, reversed);
|
||||
|
||||
// freqs
|
||||
int freqTable[] = map.getFreqs();
|
||||
for (int i = 0; i < map.size; i++) {
|
||||
assertEquals(freqs.get(table[i]).longValue(), freqTable[i]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -153,6 +153,35 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
|
|||
}
|
||||
}
|
||||
|
||||
public void testOutliers() throws Exception {
|
||||
int iterations = atLeast(1);
|
||||
final Random r = random();
|
||||
for (int i = 0; i < iterations; i++) {
|
||||
final long commonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
|
||||
doTestNormsVersusStoredFields(new LongProducer() {
|
||||
@Override
|
||||
long next() {
|
||||
return r.nextInt(100) == 0 ? TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE) : commonValue;
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
public void testOutliers2() throws Exception {
|
||||
int iterations = atLeast(1);
|
||||
final Random r = random();
|
||||
for (int i = 0; i < iterations; i++) {
|
||||
final long commonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
|
||||
final long uncommonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
|
||||
doTestNormsVersusStoredFields(new LongProducer() {
|
||||
@Override
|
||||
long next() {
|
||||
return r.nextInt(100) == 0 ? uncommonValue : commonValue;
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
private void doTestNormsVersusStoredFields(LongProducer longs) throws Exception {
|
||||
int numDocs = atLeast(500);
|
||||
long norms[] = new long[numDocs];
|
||||
|
|
Loading…
Reference in New Issue