LUCENE-6030: Add norms patched compression for a small number of common values

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1635854 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Ryan Ernst 2014-10-31 20:16:30 +00:00
parent df21abecc4
commit ae4db8f48d
6 changed files with 316 additions and 231 deletions

View File

@ -250,6 +250,9 @@ Optimizations
* LUCENE-6022: DocValuesDocIdSet checks live docs before doc values. * LUCENE-6022: DocValuesDocIdSet checks live docs before doc values.
(Adrien Grand) (Adrien Grand)
* LUCENE-6030: Add norms patched compression for a small number of common values
(Ryan Ernst)
Build Build
* LUCENE-5909: Smoke tester now has better command line parsing and * LUCENE-5909: Smoke tester now has better command line parsing and

View File

@ -19,9 +19,7 @@ package org.apache.lucene.codecs.lucene50;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.Map;
import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.NormsConsumer; import org.apache.lucene.codecs.NormsConsumer;
@ -31,6 +29,7 @@ import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.FilterIterator; import org.apache.lucene.util.FilterIterator;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.InPlaceMergeSorter;
import org.apache.lucene.util.packed.BlockPackedWriter; import org.apache.lucene.util.packed.BlockPackedWriter;
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter; import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
import org.apache.lucene.util.packed.PackedInts; import org.apache.lucene.util.packed.PackedInts;
@ -47,7 +46,8 @@ class Lucene50NormsConsumer extends NormsConsumer {
static final byte CONST_COMPRESSED = 2; static final byte CONST_COMPRESSED = 2;
static final byte UNCOMPRESSED = 3; static final byte UNCOMPRESSED = 3;
static final byte INDIRECT = 4; static final byte INDIRECT = 4;
static final byte PATCHED = 5; static final byte PATCHED_BITSET = 5;
static final byte PATCHED_TABLE = 6;
static final int BLOCK_SIZE = 1 << 14; static final int BLOCK_SIZE = 1 << 14;
// threshold for indirect encoding, computed as 1 - 1/log2(maxint) // threshold for indirect encoding, computed as 1 - 1/log2(maxint)
@ -89,11 +89,7 @@ class Lucene50NormsConsumer extends NormsConsumer {
private void writeNormsField(FieldInfo field, Iterable<Number> values, int level) throws IOException { private void writeNormsField(FieldInfo field, Iterable<Number> values, int level) throws IOException {
assert level <= 1; // we only "recurse" once in the indirect case assert level <= 1; // we only "recurse" once in the indirect case
meta.writeVInt(field.number); meta.writeVInt(field.number);
long minValue = Long.MAX_VALUE;
long maxValue = Long.MIN_VALUE;
// TODO: more efficient?
NormMap uniqueValues = new NormMap(); NormMap uniqueValues = new NormMap();
int count = 0; int count = 0;
for (Number nv : values) { for (Number nv : values) {
@ -102,42 +98,73 @@ class Lucene50NormsConsumer extends NormsConsumer {
} }
final long v = nv.longValue(); final long v = nv.longValue();
minValue = Math.min(minValue, v);
maxValue = Math.max(maxValue, v);
if (uniqueValues != null) { if (uniqueValues != null) {
if (uniqueValues.add(v)) { if (v >= Byte.MIN_VALUE && v <= Byte.MAX_VALUE) {
if (uniqueValues.add((byte) v)) {
if (uniqueValues.size > 256) { if (uniqueValues.size > 256) {
uniqueValues = null; uniqueValues = null;
} }
} }
} else {
// anything outside an 8 bit float comes from a custom scorer, which is an extreme edge case
uniqueValues = null;
}
} }
count++; count++;
} }
if (uniqueValues != null && uniqueValues.size == 1) {
// 0 bpv
addConstant(minValue);
} 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);
if (compression.bitsPerValue == 8 && minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) { if (uniqueValues == null) {
addDeltaCompressed(values, count);
} else if (uniqueValues.size == 1) {
// 0 bpv
addConstant(uniqueValues.values[0]);
} else {
// small number of unique values: this is the typical case
uniqueValues.optimizeOrdinals();
int numCommonValues = -1;
int commonValuesCount = 0;
if (level == 0 && count > 256) {
float threshold_count = count * INDIRECT_THRESHOLD;
if (uniqueValues.freqs[0] > threshold_count) {
numCommonValues = 1;
} else if ((commonValuesCount = sum(uniqueValues.freqs, 0, 3)) > threshold_count && uniqueValues.size > 4) {
numCommonValues = 3;
} else if ((commonValuesCount = sum(uniqueValues.freqs, 0, 15)) > threshold_count && uniqueValues.size > 16) {
numCommonValues = 15;
}
}
if (numCommonValues == -1) {
// no pattern in values, just find the most efficient way to pack the values
FormatAndBits compression = fastestFormatAndBits(uniqueValues.size - 1);
if (compression.bitsPerValue == 8) {
addUncompressed(values, count); addUncompressed(values, count);
} else { } else {
addTableCompressed(values, compression, count, uniqueValues); addTableCompressed(values, compression, count, uniqueValues);
} }
} else if (numCommonValues == 1) {
byte commonValue = uniqueValues.values[0];
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, 0);
} else { } else {
addDeltaCompressed(values, count); // otherwise, write a sparse bitset, where 1 indicates 'uncommon value'.
addPatchedBitset(field, values, count, uniqueValues);
} }
} else {
addPatchedTable(field, values, numCommonValues, commonValuesCount, count, uniqueValues);
}
}
}
private int sum(int[] freqs, int start, int end) {
int accum = 0;
for (int i = start; i < end; ++i) {
accum += freqs[i];
}
return accum;
} }
private FormatAndBits fastestFormatAndBits(int max) { private FormatAndBits fastestFormatAndBits(int max) {
@ -152,7 +179,7 @@ class Lucene50NormsConsumer extends NormsConsumer {
return new FormatAndBits(format, bitsPerValue); return new FormatAndBits(format, bitsPerValue);
} }
private void addConstant(long constant) throws IOException { private void addConstant(byte constant) throws IOException {
meta.writeVInt(0); meta.writeVInt(0);
meta.writeByte(CONST_COMPRESSED); meta.writeByte(CONST_COMPRESSED);
meta.writeLong(constant); meta.writeLong(constant);
@ -163,7 +190,7 @@ class Lucene50NormsConsumer extends NormsConsumer {
meta.writeByte(UNCOMPRESSED); // uncompressed byte[] meta.writeByte(UNCOMPRESSED); // uncompressed byte[]
meta.writeLong(data.getFilePointer()); meta.writeLong(data.getFilePointer());
for (Number nv : values) { for (Number nv : values) {
data.writeByte((byte) nv.longValue()); data.writeByte(nv.byteValue());
} }
} }
@ -171,25 +198,28 @@ class Lucene50NormsConsumer extends NormsConsumer {
meta.writeVInt(count); meta.writeVInt(count);
meta.writeByte(TABLE_COMPRESSED); // table-compressed meta.writeByte(TABLE_COMPRESSED); // table-compressed
meta.writeLong(data.getFilePointer()); meta.writeLong(data.getFilePointer());
writeTable(values, compression, count, uniqueValues, uniqueValues.size);
}
private void writeTable(Iterable<Number> values, FormatAndBits compression, int count, NormMap uniqueValues, int numOrds) throws IOException {
data.writeVInt(PackedInts.VERSION_CURRENT); data.writeVInt(PackedInts.VERSION_CURRENT);
long[] decode = uniqueValues.getDecodeTable();
// upgrade to power of two sized array
int size = 1 << compression.bitsPerValue;
data.writeVInt(size);
for (int i = 0; i < decode.length; i++) {
data.writeLong(decode[i]);
}
for (int i = decode.length; i < size; i++) {
data.writeLong(0);
}
data.writeVInt(compression.format.getId()); data.writeVInt(compression.format.getId());
data.writeVInt(compression.bitsPerValue); data.writeVInt(compression.bitsPerValue);
data.writeVInt(numOrds);
for (int i = 0; i < numOrds; i++) {
data.writeByte(uniqueValues.values[i]);
}
final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, compression.format, count, compression.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE); final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, compression.format, count, compression.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
for(Number nv : values) { for(Number nv : values) {
writer.add(uniqueValues.getOrd(nv.longValue())); int ord = uniqueValues.ord(nv.byteValue());
if (ord < numOrds) {
writer.add(ord);
} else {
writer.add(numOrds); // collapses all ords >= numOrds into a single value
}
} }
writer.finish(); writer.finish();
} }
@ -211,17 +241,15 @@ class Lucene50NormsConsumer extends NormsConsumer {
// encodes only uncommon values in a sparse bitset // encodes only uncommon values in a sparse bitset
// access is constant time, and the common case is predictable // 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) // 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 { private void addPatchedBitset(FieldInfo field, final Iterable<Number> values, int count, NormMap uniqueValues) throws IOException {
final long decodeTable[] = uniqueValues.getDecodeTable(); int commonCount = uniqueValues.freqs[0];
int commonCount = uniqueValues.maxFreq();
final long commonValue = decodeTable[uniqueValues.maxOrd()];
meta.writeVInt(count - commonCount); meta.writeVInt(count - commonCount);
meta.writeByte(PATCHED); meta.writeByte(PATCHED_BITSET);
meta.writeLong(data.getFilePointer()); meta.writeLong(data.getFilePointer());
// write docs with value // write docs with value
writeDocsWithValue(values, commonValue); writeDocsWithValue(values, uniqueValues, 0);
// write exceptions: only two cases make sense // write exceptions: only two cases make sense
// bpv = 1 (folded into sparse bitset already) // bpv = 1 (folded into sparse bitset already)
@ -229,27 +257,41 @@ class Lucene50NormsConsumer extends NormsConsumer {
meta.writeVInt(field.number); meta.writeVInt(field.number);
if (uniqueValues.size == 2) { if (uniqueValues.size == 2) {
// special case: implicit in bitset // special case: implicit in bitset
int otherOrd = uniqueValues.maxOrd() == 0 ? 1 : 0; addConstant(uniqueValues.values[1]);
addConstant(decodeTable[otherOrd]);
} else { } else {
// exception table // exception table
addIndirect(field, values, count, uniqueValues); addIndirect(field, values, count, uniqueValues, 0);
} }
} }
// encodes common values in a table, and the rest of the values as exceptions using INDIRECT.
// the exceptions should not be accessed very often, since the values are uncommon
private void addPatchedTable(FieldInfo field, final Iterable<Number> values, final int numCommonValues, int commonValuesCount, int count, final NormMap uniqueValues) throws IOException {
meta.writeVInt(count);
meta.writeByte(PATCHED_TABLE);
meta.writeLong(data.getFilePointer());
assert numCommonValues == 3 || numCommonValues == 15;
FormatAndBits compression = fastestFormatAndBits(numCommonValues);
writeTable(values, compression, count, uniqueValues, numCommonValues);
meta.writeVInt(field.number);
addIndirect(field, values, count - commonValuesCount, uniqueValues, numCommonValues);
}
// encodes values as sparse array: keys[] and values[] // encodes values as sparse array: keys[] and values[]
// access is log(N) where N = keys.length (slow!) // 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) // 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 { private void addIndirect(FieldInfo field, final Iterable<Number> values, int count, final NormMap uniqueValues, final int minOrd) throws IOException {
int commonCount = uniqueValues.maxFreq(); int commonCount = uniqueValues.freqs[minOrd];
final long commonValue = uniqueValues.getDecodeTable()[uniqueValues.maxOrd()];
meta.writeVInt(count - commonCount); meta.writeVInt(count - commonCount);
meta.writeByte(INDIRECT); meta.writeByte(INDIRECT);
meta.writeLong(data.getFilePointer()); meta.writeLong(data.getFilePointer());
// write docs with value // write docs with value
writeDocsWithValue(values, commonValue); writeDocsWithValue(values, uniqueValues, minOrd);
// write actual values // write actual values
writeNormsField(field, new Iterable<Number>() { writeNormsField(field, new Iterable<Number>() {
@ -258,15 +300,15 @@ class Lucene50NormsConsumer extends NormsConsumer {
return new FilterIterator<Number, Number>(values.iterator()) { return new FilterIterator<Number, Number>(values.iterator()) {
@Override @Override
protected boolean predicateFunction(Number value) { protected boolean predicateFunction(Number value) {
return value.longValue() != commonValue; return uniqueValues.ord(value.byteValue()) > minOrd;
} }
}; };
} }
}, 1); }, 1);
} }
private void writeDocsWithValue(final Iterable<Number> values, long commonValue) throws IOException { private void writeDocsWithValue(final Iterable<Number> values, NormMap uniqueValues, int minOrd) throws IOException {
data.writeLong(commonValue); data.writeLong(uniqueValues.values[minOrd]);
data.writeVInt(PackedInts.VERSION_CURRENT); data.writeVInt(PackedInts.VERSION_CURRENT);
data.writeVInt(BLOCK_SIZE); data.writeVInt(BLOCK_SIZE);
@ -274,8 +316,8 @@ class Lucene50NormsConsumer extends NormsConsumer {
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE); final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
int doc = 0; int doc = 0;
for (Number n : values) { for (Number n : values) {
long v = n.longValue(); int ord = uniqueValues.ord(n.byteValue());
if (v != commonValue) { if (ord > minOrd) {
writer.add(doc); writer.add(doc);
} }
doc++; doc++;
@ -308,36 +350,25 @@ class Lucene50NormsConsumer extends NormsConsumer {
// specialized deduplication of long->ord for norms: 99.99999% of the time this will be a single-byte range. // specialized deduplication of long->ord for norms: 99.99999% of the time this will be a single-byte range.
static class NormMap { static class NormMap {
// we use short: at most we will add 257 values to this map before its rejected as too big above. // 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]; private final short[] ords = new short[256];
final int[] freqs = new int[257]; final int[] freqs = new int[257];
final Map<Long,Short> other = new HashMap<Long,Short>(); final byte[] values = new byte[257];
int size; int size;
{ {
Arrays.fill(singleByteRange, (short)-1); Arrays.fill(ords, (short)-1);
} }
/** adds an item to the mapping. returns true if actually added */ // adds an item to the mapping. returns true if actually added
public boolean add(long l) { public boolean add(byte l) {
assert size <= 256; // once we add > 256 values, we nullify the map in addNumericField and don't use this strategy assert size <= 256; // once we add > 256 values, we nullify the map in addNumericField and don't use this strategy
if (l >= Byte.MIN_VALUE && l <= Byte.MAX_VALUE) { int index = (int)l + 128;
int index = (int) (l + 128); short previous = ords[index];
short previous = singleByteRange[index];
if (previous < 0) { if (previous < 0) {
short slot = (short)size; short slot = (short)size;
singleByteRange[index] = slot; ords[index] = slot;
freqs[slot]++; freqs[slot]++;
size++; values[slot] = l;
return true;
} else {
freqs[previous]++;
return false;
}
} else {
Short previous = other.get(l);
if (previous == null) {
freqs[size]++;
other.put(l, (short)size);
size++; size++;
return true; return true;
} else { } else {
@ -345,62 +376,33 @@ class Lucene50NormsConsumer extends NormsConsumer {
return false; return false;
} }
} }
public int ord(byte value) {
return ords[(int)value + 128];
} }
/** gets the ordinal for a previously added item */ // reassign ordinals so higher frequencies have lower ordinals
public int getOrd(long l) { public void optimizeOrdinals() {
if (l >= Byte.MIN_VALUE && l <= Byte.MAX_VALUE) { new InPlaceMergeSorter() {
int index = (int) (l + 128); @Override
return singleByteRange[index]; protected int compare(int i, int j) {
} else { return freqs[j] - freqs[i]; // sort descending
// NPE if something is screwed up }
return other.get(l); @Override
protected void swap(int i, int j) {
// swap ordinal i with ordinal j
ords[(int)values[i] + 128] = (short)j;
ords[(int)values[j] + 128] = (short)i;
int tmpFreq = freqs[i];
byte tmpValue = values[i];
freqs[i] = freqs[j];
values[i] = values[j];
freqs[j] = tmpFreq;
values[j] = tmpValue;
}
}.sort(0, size);
} }
} }
/** retrieves the ordinal table for previously added items */
public long[] getDecodeTable() {
long decode[] = new long[size];
for (int i = 0; i < singleByteRange.length; i++) {
short s = singleByteRange[i];
if (s >= 0) {
decode[s] = i - 128;
}
}
for (Map.Entry<Long,Short> entry : other.entrySet()) {
decode[entry.getValue()] = entry.getKey();
}
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;
}
}
} }

View File

@ -51,9 +51,12 @@ import org.apache.lucene.util.packed.PackedInts;
* <li>Indirect: when norms are extremely sparse, missing values are omitted. * <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 * Access to an individual value is slower, but missing norm values are never accessed
* by search code. * by search code.
* <li>Patched: when a single norm value dominates, a sparse bitset encodes docs with exceptions, * <li>Patched bitset: when a single norm value dominates, a sparse bitset encodes docs
* so that access to the common value is still very fast. outliers fall thru to an exception * with exceptions, so that access to the common value is still very fast. outliers
* handling mechanism (Indirect or Constant). * fall through to an exception handling mechanism (Indirect or Constant).
* <li>Patched table: when a small number of norm values dominate, a table is used for the
* common values to allow fast access. less common values fall through to an exception
* handling mechanism (Indirect).
* </ul> * </ul>
* <p> * <p>
* Files: * Files:
@ -87,7 +90,9 @@ import org.apache.lucene.util.packed.PackedInts;
* <li>3 --&gt; uncompressed: Values written as a simple byte[]. * <li>3 --&gt; uncompressed: Values written as a simple byte[].
* <li>4 --&gt; indirect. Only documents with a value are written with monotonic compression. a nested * <li>4 --&gt; indirect. Only documents with a value are written with monotonic compression. a nested
* entry for the same field will follow for the exception handler. * entry for the same field will follow for the exception handler.
* <li>5 --&gt; patched. Encoded the same as indirect. * <li>5 --&gt; patched bitset. Encoded the same as indirect.
* <li>6 --&gt; patched table. Documents with very common values are written with a lookup table.
* Other values are written using a nested indirect.
* </ul> * </ul>
* <li><a name="nvd" id="nvd"></a> * <li><a name="nvd" id="nvd"></a>
* <p>The Norms data or .nvd file.</p> * <p>The Norms data or .nvd file.</p>

View File

@ -49,7 +49,8 @@ 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.TABLE_COMPRESSED;
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.UNCOMPRESSED; 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.INDIRECT;
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.PATCHED; import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.PATCHED_BITSET;
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.PATCHED_TABLE;
/** /**
* Reader for {@link Lucene50NormsFormat} * Reader for {@link Lucene50NormsFormat}
@ -151,7 +152,8 @@ class Lucene50NormsProducer extends NormsProducer {
case TABLE_COMPRESSED: case TABLE_COMPRESSED:
case DELTA_COMPRESSED: case DELTA_COMPRESSED:
break; break;
case PATCHED: case PATCHED_BITSET:
case PATCHED_TABLE:
case INDIRECT: case INDIRECT:
if (meta.readVInt() != info.number) { if (meta.readVInt() != info.number) {
throw new CorruptIndexException("indirect norms entry for field: " + info.name + " is corrupt", meta); throw new CorruptIndexException("indirect norms entry for field: " + info.name + " is corrupt", meta);
@ -237,16 +239,22 @@ class Lucene50NormsProducer extends NormsProducer {
case TABLE_COMPRESSED: { case TABLE_COMPRESSED: {
data.seek(entry.offset); data.seek(entry.offset);
int packedIntsVersion = data.readVInt(); int packedIntsVersion = data.readVInt();
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 formatID = data.readVInt();
final int bitsPerValue = data.readVInt(); final int bitsPerValue = data.readVInt();
if (bitsPerValue != 1 && bitsPerValue != 2 && bitsPerValue != 4) {
throw new CorruptIndexException("TABLE_COMPRESSED only supports bpv=1, bpv=2 and bpv=4, got=" + bitsPerValue, data);
}
int size = 1 << bitsPerValue;
final byte decode[] = new byte[size];
final int ordsSize = data.readVInt();
for (int i = 0; i < ordsSize; ++i) {
decode[i] = data.readByte();
}
for (int i = ordsSize; i < size; ++i) {
decode[i] = 0;
}
final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), packedIntsVersion, entry.count, bitsPerValue); final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), packedIntsVersion, entry.count, bitsPerValue);
instance.info = Accountables.namedAccountable("table compressed", ordsReader); instance.info = Accountables.namedAccountable("table compressed", ordsReader);
instance.ramBytesUsed = RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed(); instance.ramBytesUsed = RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed();
@ -291,7 +299,7 @@ class Lucene50NormsProducer extends NormsProducer {
}; };
break; break;
} }
case PATCHED: { case PATCHED_BITSET: {
data.seek(entry.offset); data.seek(entry.offset);
final long common = data.readLong(); final long common = data.readLong();
int packedIntsVersion = data.readVInt(); int packedIntsVersion = data.readVInt();
@ -304,7 +312,7 @@ class Lucene50NormsProducer extends NormsProducer {
} }
LoadedNorms nestedInstance = loadNorms(entry.nested); LoadedNorms nestedInstance = loadNorms(entry.nested);
instance.ramBytesUsed = set.ramBytesUsed() + nestedInstance.ramBytesUsed; instance.ramBytesUsed = set.ramBytesUsed() + nestedInstance.ramBytesUsed;
instance.info = Accountables.namedAccountable("patched -> " + nestedInstance.info, instance.ramBytesUsed); instance.info = Accountables.namedAccountable("patched bitset -> " + nestedInstance.info, instance.ramBytesUsed);
final NumericDocValues values = nestedInstance.norms; final NumericDocValues values = nestedInstance.norms;
instance.norms = new NumericDocValues() { instance.norms = new NumericDocValues() {
@Override @Override
@ -318,6 +326,42 @@ class Lucene50NormsProducer extends NormsProducer {
}; };
break; break;
} }
case PATCHED_TABLE: {
data.seek(entry.offset);
int packedIntsVersion = data.readVInt();
final int formatID = data.readVInt();
final int bitsPerValue = data.readVInt();
if (bitsPerValue != 2 && bitsPerValue != 4) {
throw new CorruptIndexException("PATCHED_TABLE only supports bpv=2 and bpv=4, got=" + bitsPerValue, data);
}
final int size = 1 << bitsPerValue;
final int ordsSize = data.readVInt();
final byte decode[] = new byte[ordsSize];
assert ordsSize + 1 == size;
for (int i = 0; i < ordsSize; ++i) {
decode[i] = data.readByte();
}
final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), packedIntsVersion, entry.count, bitsPerValue);
final LoadedNorms nestedInstance = loadNorms(entry.nested);
instance.ramBytesUsed = RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed() + nestedInstance.ramBytesUsed;
instance.info = Accountables.namedAccountable("patched table -> " + nestedInstance.info, instance.ramBytesUsed);
final NumericDocValues values = nestedInstance.norms;
instance.norms = new NumericDocValues() {
@Override
public long get(int docID) {
int ord = (int)ordsReader.get(docID);
try {
// doing a try/catch here eliminates a seemingly unavoidable branch in hotspot...
return decode[ord];
} catch (IndexOutOfBoundsException e) {
return values.get(docID);
}
}
};
break;
}
default: default:
throw new AssertionError(); throw new AssertionError();
} }

View File

@ -43,85 +43,88 @@ public class TestLucene50NormsFormat extends BaseNormsFormatTestCase {
public void testNormMapSimple() { public void testNormMapSimple() {
NormMap map = new NormMap(); NormMap map = new NormMap();
map.add(10); map.add((byte)4);
map.add(5); map.add((byte) 10);
map.add(4); map.add((byte) 5);
map.add(10); map.add((byte)10);
assertEquals(3, map.size); assertEquals(3, map.size);
// first come, first serve ord assignment // first come, first serve ord assignment
assertEquals(0, map.ord((byte) 4));
assertEquals(1, map.ord((byte) 10));
assertEquals(2, map.ord((byte) 5));
// encode assertEquals(4, map.values[0]);
assertEquals(0, map.getOrd(10)); assertEquals(10, map.values[1]);
assertEquals(1, map.getOrd(5)); assertEquals(5, map.values[2]);
assertEquals(2, map.getOrd(4));
// decode assertEquals(1, map.freqs[0]);
long decode[] = map.getDecodeTable(); assertEquals(2, map.freqs[1]);
assertEquals(10, decode[0]); assertEquals(1, map.freqs[2]);
assertEquals(5, decode[1]);
assertEquals(4, decode[2]);
// freqs // optimizing reorders the ordinals
int freqs[] = map.getFreqs(); map.optimizeOrdinals();
assertEquals(2, freqs[0]); assertEquals(0, map.ord((byte)10));
assertEquals(1, freqs[1]); assertEquals(1, map.ord((byte)4));
assertEquals(1, freqs[2]); assertEquals(2, map.ord((byte)5));
assertEquals(2, map.maxFreq()); assertEquals(10, map.values[0]);
assertEquals(4, map.values[1]);
assertEquals(5, map.values[2]);
assertEquals(2, map.freqs[0]);
assertEquals(1, map.freqs[1]);
assertEquals(1, map.freqs[2]);
} }
public void testNormMapRandom() { public void testNormMapRandom() {
Map<Long,Integer> freqs = new HashMap<>();
Map<Long,Integer> ords = new HashMap<>();
Set<Long> uniqueValuesSet = new HashSet<>(); Set<Byte> uniqueValuesSet = new HashSet<>();
int numUniqValues = TestUtil.nextInt(random(), 1, 256); int numUniqValues = TestUtil.nextInt(random(), 1, 256);
for (int i = 0; i < numUniqValues; i++) { for (int i = 0; i < numUniqValues; i++) {
if (random().nextBoolean()) { uniqueValuesSet.add(Byte.valueOf((byte)TestUtil.nextInt(random(), Byte.MIN_VALUE, Byte.MAX_VALUE)));
uniqueValuesSet.add(TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE));
} else {
uniqueValuesSet.add(TestUtil.nextLong(random(), Byte.MIN_VALUE, Byte.MAX_VALUE));
}
} }
Byte uniqueValues[] = uniqueValuesSet.toArray(new Byte[uniqueValuesSet.size()]);
Long uniqueValues[] = uniqueValuesSet.toArray(new Long[uniqueValuesSet.size()]); Map<Byte,Integer> freqs = new HashMap<>();
NormMap map = new NormMap(); NormMap map = new NormMap();
int numdocs = TestUtil.nextInt(random(), 1, 100000); int numdocs = TestUtil.nextInt(random(), 1, 100000);
for (int i = 0; i < numdocs; i++) { for (int i = 0; i < numdocs; i++) {
long value = uniqueValues[random().nextInt(uniqueValues.length)]; byte value = uniqueValues[random().nextInt(uniqueValues.length)];
// now add to both expected and actual // now add to both expected and actual
map.add(value); map.add(value);
if (freqs.containsKey(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); freqs.put(value, freqs.get(value) + 1);
} else {
freqs.put(value, 1);
} }
} }
// value -> ord assertEquals(freqs.size(), map.size);
assertEquals(ords.size(), map.size); for (Map.Entry<Byte,Integer> kv : freqs.entrySet()) {
for (Map.Entry<Long,Integer> kv : ords.entrySet()) { byte value = kv.getKey();
assertEquals(kv.getValue().intValue(), map.getOrd(kv.getKey())); int freq = kv.getValue();
int ord = map.ord(value);
assertEquals(freq, map.freqs[ord]);
assertEquals(value, map.values[ord]);
} }
// ord -> value // optimizing should reorder ordinals from greatest to least frequency
Map<Long,Integer> reversed = new HashMap<>(); map.optimizeOrdinals();
long table[] = map.getDecodeTable(); // recheck consistency
for (int i = 0; i < map.size; i++) { assertEquals(freqs.size(), map.size);
reversed.put(table[i], i); for (Map.Entry<Byte,Integer> kv : freqs.entrySet()) {
byte value = kv.getKey();
int freq = kv.getValue();
int ord = map.ord(value);
assertEquals(freq, map.freqs[ord]);
assertEquals(value, map.values[ord]);
} }
assertEquals(ords, reversed); // also check descending freq
int prevFreq = map.freqs[0];
// freqs for (int i = 1; i < map.size; ++i) {
int freqTable[] = map.getFreqs(); assertTrue(prevFreq >= map.freqs[i]);
for (int i = 0; i < map.size; i++) { prevFreq = map.freqs[i];
assertEquals(freqs.get(table[i]).longValue(), freqTable[i]);
} }
} }
} }

View File

@ -22,6 +22,7 @@ import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Random; import java.util.Random;
import com.carrotsearch.randomizedtesting.annotations.Seed;
import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockTokenizer; import org.apache.lucene.analysis.MockTokenizer;
@ -43,6 +44,7 @@ import org.apache.lucene.util.TestUtil;
* test passes, then all Lucene/Solr tests should also pass. Ie, * test passes, then all Lucene/Solr tests should also pass. Ie,
* if there is some bug in a given NormsFormat that this * if there is some bug in a given NormsFormat that this
* test fails to catch then this test needs to be improved! */ * test fails to catch then this test needs to be improved! */
@Seed(value = "AD2222476BCB8800")
public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCase { public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCase {
public void testByteRange() throws Exception { public void testByteRange() throws Exception {
@ -182,6 +184,32 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
} }
} }
public void testNCommon() throws Exception {
final int iterations = atLeast(1);
final Random r = random();
for (int i = 0; i < iterations; ++i) {
// 16 is 4 bpv, the max before we jump to 8bpv
for (int n = 2; n < 16; ++n) {
final int N = n;
final long[] commonValues = new long[N];
for (int j = 0; j < N; ++j) {
commonValues[j] = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
}
final int numOtherValues = TestUtil.nextInt(r, 2, 256 - N);
final long[] otherValues = new long[numOtherValues];
for (int j = 0; j < numOtherValues; ++j) {
otherValues[j] = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
}
doTestNormsVersusStoredFields(new LongProducer() {
@Override
long next() {
return r.nextInt(100) == 0 ? otherValues[r.nextInt(numOtherValues - 1)] : commonValues[r.nextInt(N - 1)];
}
});
}
}
}
private void doTestNormsVersusStoredFields(LongProducer longs) throws Exception { private void doTestNormsVersusStoredFields(LongProducer longs) throws Exception {
int numDocs = atLeast(500); int numDocs = atLeast(500);
long norms[] = new long[numDocs]; long norms[] = new long[numDocs];
@ -226,7 +254,7 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
NumericDocValues docValues = r.getNormValues("stored"); NumericDocValues docValues = r.getNormValues("stored");
for (int i = 0; i < r.maxDoc(); i++) { for (int i = 0; i < r.maxDoc(); i++) {
long storedValue = Long.parseLong(r.document(i).get("stored")); long storedValue = Long.parseLong(r.document(i).get("stored"));
assertEquals(storedValue, docValues.get(i)); assertEquals("doc " + i, storedValue, docValues.get(i));
} }
} }
ir.close(); ir.close();