LUCENE-7589: Prevent outliers from raising the bpv for everyone.

This commit is contained in:
Adrien Grand 2016-12-15 16:33:36 +01:00
parent ea1569e291
commit 3b182aa2fb
7 changed files with 479 additions and 82 deletions

View File

@ -48,6 +48,10 @@ Optimizations
* LUCENE-7519: Add optimized APIs to compute browse-only top level
facets (Mike McCandless)
* LUCENE-7589: Numeric doc values now have the ability to encode blocks of
values using different numbers of bits per value if this proves to save
storage. (Adrien Grand)
Other
* LUCENE-7328: Remove LegacyNumericEncoding from GeoPointField. (Nick Knize)

View File

@ -18,6 +18,8 @@ package org.apache.lucene.codecs.lucene70;
import static org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
import static org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat.NUMERIC_BLOCK_SHIFT;
import static org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE;
import java.io.Closeable; // javadocs
import java.io.IOException;
@ -42,6 +44,7 @@ import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.SortedSetSelector;
import org.apache.lucene.store.GrowableByteArrayDataOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.BytesRef;
@ -112,12 +115,46 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
});
}
private static class MinMaxTracker {
long min, max, numValues, spaceInBits;
MinMaxTracker() {
reset();
spaceInBits = 0;
}
private void reset() {
min = Long.MAX_VALUE;
max = Long.MIN_VALUE;
numValues = 0;
}
/** Accumulate a new value. */
void update(long v) {
min = Math.min(min, v);
max = Math.max(max, v);
++numValues;
}
/** Update the required space. */
void finish() {
if (max > min) {
spaceInBits += DirectWriter.unsignedBitsRequired(max - min) * numValues;
}
}
/** Update space usage and get ready for accumulating values for the next block. */
void nextBlock() {
finish();
reset();
}
}
private long[] writeValues(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
SortedNumericDocValues values = valuesProducer.getSortedNumeric(field);
int numDocsWithValue = 0;
long numValues = 0;
long min = Long.MAX_VALUE;
long max = Long.MIN_VALUE;
MinMaxTracker minMax = new MinMaxTracker();
MinMaxTracker blockMinMax = new MinMaxTracker();
long gcd = 0;
Set<Long> uniqueValues = new HashSet<>();
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
@ -130,26 +167,35 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
// wrong results. Since these extreme values are unlikely, we just discard
// GCD computation for them
gcd = 1;
} else if (numValues != 0) { // minValue needs to be set first
gcd = MathUtil.gcd(gcd, v - min);
} else if (minMax.numValues != 0) { // minValue needs to be set first
gcd = MathUtil.gcd(gcd, v - minMax.min);
}
}
min = Math.min(min, v);
max = Math.max(max, v);
minMax.update(v);
blockMinMax.update(v);
if (blockMinMax.numValues == NUMERIC_BLOCK_SIZE) {
blockMinMax.nextBlock();
}
if (uniqueValues != null
&& uniqueValues.add(v)
&& uniqueValues.size() > 256) {
uniqueValues = null;
}
numValues++;
}
numDocsWithValue++;
}
minMax.finish();
blockMinMax.finish();
final long numValues = minMax.numValues;
long min = minMax.min;
final long max = minMax.max;
assert blockMinMax.spaceInBits <= minMax.spaceInBits;
if (numDocsWithValue == 0) {
meta.writeLong(-2);
meta.writeLong(0L);
@ -166,6 +212,7 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
meta.writeLong(numValues);
final int numBitsPerValue;
boolean doBlocks = false;
Map<Long, Integer> encode = null;
if (min >= max) {
numBitsPerValue = 0;
@ -189,12 +236,19 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
gcd = 1;
} else {
uniqueValues = null;
numBitsPerValue = DirectWriter.unsignedBitsRequired((max - min) / gcd);
if (gcd == 1 && min > 0
&& DirectWriter.unsignedBitsRequired(max) == DirectWriter.unsignedBitsRequired(max - min)) {
min = 0;
// we do blocks if that appears to save 10+% storage
doBlocks = minMax.spaceInBits > 0 && (double) blockMinMax.spaceInBits / minMax.spaceInBits <= 0.9;
if (doBlocks) {
numBitsPerValue = 0xFF;
meta.writeInt(-2 - NUMERIC_BLOCK_SHIFT);
} else {
numBitsPerValue = DirectWriter.unsignedBitsRequired((max - min) / gcd);
if (gcd == 1 && min > 0
&& DirectWriter.unsignedBitsRequired(max) == DirectWriter.unsignedBitsRequired(max - min)) {
min = 0;
}
meta.writeInt(-1);
}
meta.writeInt(-1);
}
}
@ -203,26 +257,79 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
meta.writeLong(gcd);
long startOffset = data.getFilePointer();
meta.writeLong(startOffset);
if (numBitsPerValue != 0) {
values = valuesProducer.getSortedNumeric(field);
DirectWriter writer = DirectWriter.getInstance(data, numValues, numBitsPerValue);
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
for (int i = 0, count = values.docValueCount(); i < count; ++i) {
long v = values.nextValue();
if (encode == null) {
writer.add((v - min) / gcd);
} else {
writer.add(encode.get(v));
}
}
}
writer.finish();
if (doBlocks) {
writeValuesMultipleBlocks(valuesProducer.getSortedNumeric(field), gcd);
} else if (numBitsPerValue != 0) {
writeValuesSingleBlock(valuesProducer.getSortedNumeric(field), numValues, numBitsPerValue, min, gcd, encode);
}
meta.writeLong(data.getFilePointer() - startOffset);
return new long[] {numDocsWithValue, numValues};
}
private void writeValuesSingleBlock(SortedNumericDocValues values, long numValues, int numBitsPerValue,
long min, long gcd, Map<Long, Integer> encode) throws IOException {
DirectWriter writer = DirectWriter.getInstance(data, numValues, numBitsPerValue);
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
for (int i = 0, count = values.docValueCount(); i < count; ++i) {
long v = values.nextValue();
if (encode == null) {
writer.add((v - min) / gcd);
} else {
writer.add(encode.get(v));
}
}
}
writer.finish();
}
private void writeValuesMultipleBlocks(SortedNumericDocValues values, long gcd) throws IOException {
final long[] buffer = new long[NUMERIC_BLOCK_SIZE];
final GrowableByteArrayDataOutput encodeBuffer = new GrowableByteArrayDataOutput(NUMERIC_BLOCK_SIZE);
int upTo = 0;
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
for (int i = 0, count = values.docValueCount(); i < count; ++i) {
buffer[upTo++] = values.nextValue();
if (upTo == NUMERIC_BLOCK_SIZE) {
writeBlock(buffer, NUMERIC_BLOCK_SIZE, gcd, encodeBuffer);
upTo = 0;
}
}
}
if (upTo > 0) {
writeBlock(buffer, upTo, gcd, encodeBuffer);
}
}
private void writeBlock(long[] values, int length, long gcd, GrowableByteArrayDataOutput buffer) throws IOException {
assert length > 0;
long min = values[0];
long max = values[0];
for (int i = 1; i < length; ++i) {
final long v = values[i];
assert Math.floorMod(values[i] - min, gcd) == 0;
min = Math.min(min, v);
max = Math.max(max, v);
}
if (min == max) {
data.writeByte((byte) 0);
data.writeLong(min);
} else {
final int bitsPerValue = DirectWriter.unsignedBitsRequired(max - min);
buffer.reset();
assert buffer.getPosition() == 0;
final DirectWriter w = DirectWriter.getInstance(buffer, length, bitsPerValue);
for (int i = 0; i < length; ++i) {
w.add((values[i] - min) / gcd);
}
w.finish();
data.writeByte((byte) bitsPerValue);
data.writeLong(min);
data.writeInt(buffer.getPosition());
data.writeBytes(buffer.getBytes(), buffer.getPosition());
}
}
@Override
public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
meta.writeInt(field.number);

View File

@ -146,10 +146,11 @@ public final class Lucene70DocValuesFormat extends DocValuesFormat {
static final byte SORTED_SET = 3;
static final byte SORTED_NUMERIC = 4;
// addressing uses 16k blocks
static final int MONOTONIC_BLOCK_SIZE = 16384;
static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16;
static final int NUMERIC_BLOCK_SHIFT = 14;
static final int NUMERIC_BLOCK_SIZE = 1 << NUMERIC_BLOCK_SHIFT;
static final int TERMS_DICT_BLOCK_SHIFT = 4;
static final int TERMS_DICT_BLOCK_SIZE = 1 << TERMS_DICT_BLOCK_SHIFT;
static final int TERMS_DICT_BLOCK_MASK = TERMS_DICT_BLOCK_SIZE - 1;

View File

@ -144,7 +144,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
entry.docsWithFieldLength = meta.readLong();
entry.numValues = meta.readLong();
int tableSize = meta.readInt();
if (tableSize < -1 || tableSize > 256) {
if (tableSize > 256) {
throw new CorruptIndexException("invalid table size: " + tableSize, meta);
}
if (tableSize >= 0) {
@ -154,6 +154,11 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
entry.table[i] = meta.readLong();
}
}
if (tableSize < -1) {
entry.blockShift = -2 - tableSize;
} else {
entry.blockShift = -1;
}
entry.bitsPerValue = meta.readByte();
entry.minValue = meta.readLong();
entry.gcd = meta.readLong();
@ -260,6 +265,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
private static class NumericEntry {
long[] table;
int blockShift;
byte bitsPerValue;
long docsWithFieldOffset;
long docsWithFieldLength;
@ -429,24 +435,62 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
};
} else {
final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
if (entry.table != null) {
final long[] table = entry.table;
if (entry.blockShift >= 0) {
// dense but split into blocks of different bits per value
final int shift = entry.blockShift;
final long mul = entry.gcd;
final int mask = (1 << shift) - 1;
return new DenseNumericDocValues(maxDoc) {
int block = -1;
long delta;
long offset;
long blockEndOffset;
LongValues values;
@Override
public long longValue() throws IOException {
return table[(int) values.get(doc)];
final int block = doc >>> shift;
if (this.block != block) {
int bitsPerValue;
do {
offset = blockEndOffset;
bitsPerValue = slice.readByte(offset++);
delta = slice.readLong(offset);
offset += Long.BYTES;
if (bitsPerValue == 0) {
blockEndOffset = offset;
} else {
final int length = slice.readInt(offset);
offset += Integer.BYTES;
blockEndOffset = offset + length;
}
this.block ++;
} while (this.block != block);
values = bitsPerValue == 0 ? LongValues.ZEROES : DirectReader.getInstance(slice, bitsPerValue, offset);
}
return mul * values.get(doc & mask) + delta;
}
};
} else {
final long mul = entry.gcd;
final long delta = entry.minValue;
return new DenseNumericDocValues(maxDoc) {
@Override
public long longValue() throws IOException {
return mul * values.get(doc) + delta;
}
};
final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
if (entry.table != null) {
final long[] table = entry.table;
return new DenseNumericDocValues(maxDoc) {
@Override
public long longValue() throws IOException {
return table[(int) values.get(doc)];
}
};
} else {
final long mul = entry.gcd;
final long delta = entry.minValue;
return new DenseNumericDocValues(maxDoc) {
@Override
public long longValue() throws IOException {
return mul * values.get(doc) + delta;
}
};
}
}
}
} else {
@ -461,24 +505,63 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
};
} else {
final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
if (entry.table != null) {
final long[] table = entry.table;
if (entry.blockShift >= 0) {
// sparse and split into blocks of different bits per value
final int shift = entry.blockShift;
final long mul = entry.gcd;
final int mask = (1 << shift) - 1;
return new SparseNumericDocValues(disi) {
int block = -1;
long delta;
long offset;
long blockEndOffset;
LongValues values;
@Override
public long longValue() throws IOException {
return table[(int) values.get(disi.index())];
final int index = disi.index();
final int block = index >>> shift;
if (this.block != block) {
int bitsPerValue;
do {
offset = blockEndOffset;
bitsPerValue = slice.readByte(offset++);
delta = slice.readLong(offset);
offset += Long.BYTES;
if (bitsPerValue == 0) {
blockEndOffset = offset;
} else {
final int length = slice.readInt(offset);
offset += Integer.BYTES;
blockEndOffset = offset + length;
}
this.block ++;
} while (this.block != block);
values = bitsPerValue == 0 ? LongValues.ZEROES : DirectReader.getInstance(slice, bitsPerValue, offset);
}
return mul * values.get(index & mask) + delta;
}
};
} else {
final long mul = entry.gcd;
final long delta = entry.minValue;
return new SparseNumericDocValues(disi) {
@Override
public long longValue() throws IOException {
return mul * values.get(disi.index()) + delta;
}
};
final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
if (entry.table != null) {
final long[] table = entry.table;
return new SparseNumericDocValues(disi) {
@Override
public long longValue() throws IOException {
return table[(int) values.get(disi.index())];
}
};
} else {
final long mul = entry.gcd;
final long delta = entry.minValue;
return new SparseNumericDocValues(disi) {
@Override
public long longValue() throws IOException {
return mul * values.get(disi.index()) + delta;
}
};
}
}
}
}
@ -494,34 +577,75 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
};
} else {
final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
if (entry.table != null) {
final long[] table = entry.table;
if (entry.blockShift >= 0) {
final int shift = entry.blockShift;
final long mul = entry.gcd;
final long mask = (1L << shift) - 1;
return new LongValues() {
@Override
long block = -1;
long delta;
long offset;
long blockEndOffset;
LongValues values;
public long get(long index) {
return table[(int) values.get(index)];
}
};
} else if (entry.gcd != 1) {
final long gcd = entry.gcd;
final long minValue = entry.minValue;
return new LongValues() {
@Override
public long get(long index) {
return values.get(index) * gcd + minValue;
}
};
} else if (entry.minValue != 0) {
final long minValue = entry.minValue;
return new LongValues() {
@Override
public long get(long index) {
return values.get(index) + minValue;
final long block = index >>> shift;
if (this.block != block) {
assert block > this.block : "Reading backwards is illegal: " + this.block + " < " + block;
int bitsPerValue;
do {
offset = blockEndOffset;
try {
bitsPerValue = slice.readByte(offset++);
delta = slice.readLong(offset);
offset += Long.BYTES;
if (bitsPerValue == 0) {
blockEndOffset = offset;
} else {
final int length = slice.readInt(offset);
offset += Integer.BYTES;
blockEndOffset = offset + length;
}
} catch (IOException e) {
throw new RuntimeException(e);
}
this.block ++;
} while (this.block != block);
values = bitsPerValue == 0 ? LongValues.ZEROES : DirectReader.getInstance(slice, bitsPerValue, offset);
}
return mul * values.get(index & mask) + delta;
}
};
} else {
return values;
final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
if (entry.table != null) {
final long[] table = entry.table;
return new LongValues() {
@Override
public long get(long index) {
return table[(int) values.get(index)];
}
};
} else if (entry.gcd != 1) {
final long gcd = entry.gcd;
final long minValue = entry.minValue;
return new LongValues() {
@Override
public long get(long index) {
return values.get(index) * gcd + minValue;
}
};
} else if (entry.minValue != 0) {
final long minValue = entry.minValue;
return new LongValues() {
@Override
public long get(long index) {
return values.get(index) + minValue;
}
};
} else {
return values;
}
}
}
}

View File

@ -30,6 +30,15 @@ public abstract class LongValues {
};
public static final LongValues ZEROES = new LongValues() {
@Override
public long get(long index) {
return 0;
}
};
/** Get value at <code>index</code>. */
public abstract long get(long index);

View File

@ -21,7 +21,7 @@ import java.io.EOFException;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.DataOutput;
/**
* Class for writing packed integers to be directly read from Directory.
@ -44,7 +44,7 @@ import org.apache.lucene.store.IndexOutput;
public final class DirectWriter {
final int bitsPerValue;
final long numValues;
final IndexOutput output;
final DataOutput output;
long count;
boolean finished;
@ -56,7 +56,7 @@ public final class DirectWriter {
final BulkOperation encoder;
final int iterations;
DirectWriter(IndexOutput output, long numValues, int bitsPerValue) {
DirectWriter(DataOutput output, long numValues, int bitsPerValue) {
this.output = output;
this.numValues = numValues;
this.bitsPerValue = bitsPerValue;
@ -103,7 +103,7 @@ public final class DirectWriter {
}
/** Returns an instance suitable for encoding {@code numValues} using {@code bitsPerValue} */
public static DirectWriter getInstance(IndexOutput output, long numValues, int bitsPerValue) {
public static DirectWriter getInstance(DataOutput output, long numValues, int bitsPerValue) {
if (Arrays.binarySearch(SUPPORTED_BITS_PER_VALUE, bitsPerValue) < 0) {
throw new IllegalArgumentException("Unsupported bitsPerValue " + bitsPerValue + ". Did you use bitsRequired?");
}

View File

@ -25,6 +25,7 @@ import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.TreeSet;
import java.util.function.LongSupplier;
import java.util.function.Supplier;
import org.apache.lucene.analysis.MockAnalyzer;
@ -61,6 +62,7 @@ import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.RAMFile;
@ -534,4 +536,154 @@ public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatT
dir.close();
}
}
@Slow
public void testSortedNumericBlocksOfVariousBitsPerValue() throws Exception {
doTestSortedNumericBlocksOfVariousBitsPerValue(() -> TestUtil.nextInt(random(), 1, 3));
}
@Slow
public void testSparseSortedNumericBlocksOfVariousBitsPerValue() throws Exception {
doTestSortedNumericBlocksOfVariousBitsPerValue(() -> TestUtil.nextInt(random(), 0, 2));
}
@Slow
public void testNumericBlocksOfVariousBitsPerValue() throws Exception {
doTestSparseNumericBlocksOfVariousBitsPerValue(1);
}
@Slow
public void testSparseNumericBlocksOfVariousBitsPerValue() throws Exception {
doTestSparseNumericBlocksOfVariousBitsPerValue(random().nextDouble());
}
private static LongSupplier blocksOfVariousBPV() {
final long mul = TestUtil.nextInt(random(), 1, 100);
final long min = random().nextInt();
return new LongSupplier() {
int i = Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE;
int maxDelta;
@Override
public long getAsLong() {
if (i == Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE) {
maxDelta = 1 << random().nextInt(5);
i = 0;
}
i++;
return min + mul * random().nextInt(maxDelta);
}
};
}
private void doTestSortedNumericBlocksOfVariousBitsPerValue(LongSupplier counts) throws Exception {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
conf.setMaxBufferedDocs(atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE));
conf.setRAMBufferSizeMB(-1);
conf.setMergePolicy(newLogMergePolicy(random().nextBoolean()));
IndexWriter writer = new IndexWriter(dir, conf);
final int numDocs = atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE*3);
final LongSupplier values = blocksOfVariousBPV();
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
int valueCount = (int) counts.getAsLong();
long valueArray[] = new long[valueCount];
for (int j = 0; j < valueCount; j++) {
long value = values.getAsLong();
valueArray[j] = value;
doc.add(new SortedNumericDocValuesField("dv", value));
}
Arrays.sort(valueArray);
for (int j = 0; j < valueCount; j++) {
doc.add(new StoredField("stored", Long.toString(valueArray[j])));
}
writer.addDocument(doc);
if (random().nextInt(31) == 0) {
writer.commit();
}
}
writer.forceMerge(1);
writer.close();
// compare
DirectoryReader ir = DirectoryReader.open(dir);
TestUtil.checkReader(ir);
for (LeafReaderContext context : ir.leaves()) {
LeafReader r = context.reader();
SortedNumericDocValues docValues = DocValues.getSortedNumeric(r, "dv");
for (int i = 0; i < r.maxDoc(); i++) {
if (i > docValues.docID()) {
docValues.nextDoc();
}
String expected[] = r.document(i).getValues("stored");
if (i < docValues.docID()) {
assertEquals(0, expected.length);
} else {
String actual[] = new String[docValues.docValueCount()];
for (int j = 0; j < actual.length; j++) {
actual[j] = Long.toString(docValues.nextValue());
}
assertArrayEquals(expected, actual);
}
}
}
ir.close();
dir.close();
}
private void doTestSparseNumericBlocksOfVariousBitsPerValue(double density) throws Exception {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
conf.setMaxBufferedDocs(atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE));
conf.setRAMBufferSizeMB(-1);
conf.setMergePolicy(newLogMergePolicy(random().nextBoolean()));
IndexWriter writer = new IndexWriter(dir, conf);
Document doc = new Document();
Field storedField = newStringField("stored", "", Field.Store.YES);
Field dvField = new NumericDocValuesField("dv", 0);
doc.add(storedField);
doc.add(dvField);
final int numDocs = atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE*3);
final LongSupplier longs = blocksOfVariousBPV();
for (int i = 0; i < numDocs; i++) {
if (random().nextDouble() > density) {
writer.addDocument(new Document());
continue;
}
long value = longs.getAsLong();
storedField.setStringValue(Long.toString(value));
dvField.setLongValue(value);
writer.addDocument(doc);
}
writer.forceMerge(1);
writer.close();
// compare
DirectoryReader ir = DirectoryReader.open(dir);
TestUtil.checkReader(ir);
for (LeafReaderContext context : ir.leaves()) {
LeafReader r = context.reader();
NumericDocValues docValues = DocValues.getNumeric(r, "dv");
docValues.nextDoc();
for (int i = 0; i < r.maxDoc(); i++) {
String storedValue = r.document(i).get("stored");
if (storedValue == null) {
assertTrue(docValues.docID() > i);
} else {
assertEquals(i, docValues.docID());
assertEquals(Long.parseLong(storedValue), docValues.longValue());
docValues.nextDoc();
}
}
assertEquals(DocIdSetIterator.NO_MORE_DOCS, docValues.docID());
}
ir.close();
dir.close();
}
}