mirror of https://github.com/apache/lucene.git
LUCENE-5178: add 'missing' support to docvalues (simpletext only)
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5178@1514642 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
1c982a5f56
commit
3be8ed1d10
|
@ -23,6 +23,7 @@ import java.util.HashSet;
|
|||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.MissingOrdRemapper;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
@ -92,8 +93,9 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
if (optimizeStorage) {
|
||||
uniqueValues = new HashSet<>();
|
||||
|
||||
// nocommit: impl null values (ideally smartly)
|
||||
for (Number nv : values) {
|
||||
final long v = nv.longValue();
|
||||
final long v = nv == null ? 0 : nv.longValue();
|
||||
|
||||
if (gcd != 1) {
|
||||
if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
|
||||
|
@ -151,14 +153,15 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
meta.writeLong(gcd);
|
||||
final BlockPackedWriter quotientWriter = new BlockPackedWriter(data, BLOCK_SIZE);
|
||||
for (Number nv : values) {
|
||||
quotientWriter.add((nv.longValue() - minValue) / gcd);
|
||||
long value = nv == null ? 0 : nv.longValue();
|
||||
quotientWriter.add((value - minValue) / gcd);
|
||||
}
|
||||
quotientWriter.finish();
|
||||
break;
|
||||
case DELTA_COMPRESSED:
|
||||
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
|
||||
for (Number nv : values) {
|
||||
writer.add(nv.longValue());
|
||||
writer.add(nv == null ? 0 : nv.longValue());
|
||||
}
|
||||
writer.finish();
|
||||
break;
|
||||
|
@ -173,7 +176,7 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
final int bitsRequired = PackedInts.bitsRequired(uniqueValues.size() - 1);
|
||||
final PackedInts.Writer ordsWriter = PackedInts.getWriterNoHeader(data, PackedInts.Format.PACKED, (int) count, bitsRequired, PackedInts.DEFAULT_BUFFER_SIZE);
|
||||
for (Number nv : values) {
|
||||
ordsWriter.add(encode.get(nv.longValue()));
|
||||
ordsWriter.add(encode.get(nv == null ? 0 : nv.longValue()));
|
||||
}
|
||||
ordsWriter.finish();
|
||||
break;
|
||||
|
@ -192,9 +195,12 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
final long startFP = data.getFilePointer();
|
||||
long count = 0;
|
||||
for(BytesRef v : values) {
|
||||
minLength = Math.min(minLength, v.length);
|
||||
maxLength = Math.max(maxLength, v.length);
|
||||
data.writeBytes(v.bytes, v.offset, v.length);
|
||||
final int length = v == null ? 0 : v.length;
|
||||
minLength = Math.min(minLength, length);
|
||||
maxLength = Math.max(maxLength, length);
|
||||
if (v != null) {
|
||||
data.writeBytes(v.bytes, v.offset, v.length);
|
||||
}
|
||||
count++;
|
||||
}
|
||||
meta.writeVInt(minLength == maxLength ? BINARY_FIXED_UNCOMPRESSED : BINARY_VARIABLE_UNCOMPRESSED);
|
||||
|
@ -213,7 +219,9 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
|
||||
long addr = 0;
|
||||
for (BytesRef v : values) {
|
||||
addr += v.length;
|
||||
if (v != null) {
|
||||
addr += v.length;
|
||||
}
|
||||
writer.add(addr);
|
||||
}
|
||||
writer.finish();
|
||||
|
@ -278,6 +286,34 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
|
|||
|
||||
@Override
|
||||
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
|
||||
// nocommit: remove this hack and support missing!
|
||||
|
||||
// three cases for simulating the old writer:
|
||||
// 1. no missing
|
||||
// 2. missing (and empty string in use): remap ord=-1 -> ord=0
|
||||
// 3. missing (and empty string not in use): remap all ords +1, insert empty string into values
|
||||
boolean anyMissing = false;
|
||||
for (Number n : docToOrd) {
|
||||
if (n.longValue() == -1) {
|
||||
anyMissing = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
boolean hasEmptyString = false;
|
||||
for (BytesRef b : values) {
|
||||
hasEmptyString = b.length == 0;
|
||||
break;
|
||||
}
|
||||
|
||||
if (!anyMissing) {
|
||||
// nothing to do
|
||||
} else if (hasEmptyString) {
|
||||
docToOrd = MissingOrdRemapper.mapMissingToOrd0(docToOrd);
|
||||
} else {
|
||||
docToOrd = MissingOrdRemapper.mapAllOrds(docToOrd);
|
||||
values = MissingOrdRemapper.insertEmptyValue(values);
|
||||
}
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(DiskDocValuesFormat.SORTED);
|
||||
addTermsDict(field, values);
|
||||
|
|
|
@ -32,6 +32,7 @@ import java.util.Map;
|
|||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.DocValuesProducer.SortedSetDocsWithField;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
|
@ -59,6 +60,7 @@ class DiskDocValuesProducer extends DocValuesProducer {
|
|||
private final Map<Integer,NumericEntry> ords;
|
||||
private final Map<Integer,NumericEntry> ordIndexes;
|
||||
private final IndexInput data;
|
||||
private final int maxDoc;
|
||||
|
||||
// memory-resident structures
|
||||
private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
|
||||
|
@ -68,6 +70,7 @@ class DiskDocValuesProducer extends DocValuesProducer {
|
|||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
// read in the entries from the metadata file.
|
||||
IndexInput in = state.directory.openInput(metaName, state.context);
|
||||
this.maxDoc = state.segmentInfo.getDocCount();
|
||||
boolean success = false;
|
||||
final int version;
|
||||
try {
|
||||
|
@ -490,6 +493,17 @@ class DiskDocValuesProducer extends DocValuesProducer {
|
|||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
// nocommit: only use this if the field's entry has missing values (write that),
|
||||
// otherwise return MatchAllBits
|
||||
if (field.getDocValuesType() == FieldInfo.DocValuesType.SORTED_SET) {
|
||||
return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
|
||||
} else {
|
||||
return new Bits.MatchAllBits(maxDoc);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
data.close();
|
||||
|
|
|
@ -38,12 +38,16 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
* minvalue 0
|
||||
* pattern 000
|
||||
* 005
|
||||
* T
|
||||
* 234
|
||||
* T
|
||||
* 123
|
||||
* T
|
||||
* ...
|
||||
* </pre>
|
||||
* so a document's value (delta encoded from minvalue) can be retrieved by
|
||||
* seeking to startOffset + (1+pattern.length())*docid. The extra 1 is the newline.
|
||||
* seeking to startOffset + (1+pattern.length()+2)*docid. The extra 1 is the newline.
|
||||
* The extra 2 is another newline and 'T' or 'F': true if the value is real, false if missing.
|
||||
*
|
||||
* for bytes this is also a "fixed-width" file, for example:
|
||||
* <pre>
|
||||
|
@ -53,12 +57,15 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
* pattern 0
|
||||
* length 6
|
||||
* foobar[space][space]
|
||||
* T
|
||||
* length 3
|
||||
* baz[space][space][space][space][space]
|
||||
* T
|
||||
* ...
|
||||
* </pre>
|
||||
* so a doc's value can be retrieved by seeking to startOffset + (9+pattern.length+maxlength)*doc
|
||||
* so a doc's value can be retrieved by seeking to startOffset + (9+pattern.length+maxlength+2)*doc
|
||||
* the extra 9 is 2 newlines, plus "length " itself.
|
||||
* the extra 2 is another newline and 'T' or 'F': true if the value is real, false if missing.
|
||||
*
|
||||
* for sorted bytes this is a fixed-width file, for example:
|
||||
* <pre>
|
||||
|
|
|
@ -28,6 +28,7 @@ import java.util.Locale;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.DocValuesProducer.SortedSetDocsWithField;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
|
@ -38,6 +39,7 @@ import org.apache.lucene.index.SortedDocValues;
|
|||
import org.apache.lucene.index.FieldInfo.DocValuesType;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
|
@ -100,7 +102,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
assert startsWith(PATTERN);
|
||||
field.pattern = stripPrefix(PATTERN);
|
||||
field.dataStartFilePointer = data.getFilePointer();
|
||||
data.seek(data.getFilePointer() + (1+field.pattern.length()) * maxDoc);
|
||||
data.seek(data.getFilePointer() + (1+field.pattern.length()+2) * maxDoc);
|
||||
} else if (dvType == DocValuesType.BINARY) {
|
||||
readLine();
|
||||
assert startsWith(MAXLENGTH);
|
||||
|
@ -109,7 +111,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
assert startsWith(PATTERN);
|
||||
field.pattern = stripPrefix(PATTERN);
|
||||
field.dataStartFilePointer = data.getFilePointer();
|
||||
data.seek(data.getFilePointer() + (9+field.pattern.length()+field.maxLength) * maxDoc);
|
||||
data.seek(data.getFilePointer() + (9+field.pattern.length()+field.maxLength+2) * maxDoc);
|
||||
} else if (dvType == DocValuesType.SORTED || dvType == DocValuesType.SORTED_SET) {
|
||||
readLine();
|
||||
assert startsWith(NUMVALUES);
|
||||
|
@ -158,7 +160,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
if (docID < 0 || docID >= maxDoc) {
|
||||
throw new IndexOutOfBoundsException("docID must be 0 .. " + (maxDoc-1) + "; got " + docID);
|
||||
}
|
||||
in.seek(field.dataStartFilePointer + (1+field.pattern.length())*docID);
|
||||
in.seek(field.dataStartFilePointer + (1+field.pattern.length()+2)*docID);
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
//System.out.println("parsing delta: " + scratch.utf8ToString());
|
||||
BigDecimal bd;
|
||||
|
@ -169,6 +171,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
e.initCause(pe);
|
||||
throw e;
|
||||
}
|
||||
SimpleTextUtil.readLine(in, scratch); // read the line telling us if its real or not
|
||||
return BigInteger.valueOf(field.minValue).add(bd.toBigIntegerExact()).longValue();
|
||||
} catch (IOException ioe) {
|
||||
throw new RuntimeException(ioe);
|
||||
|
@ -176,6 +179,30 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
private Bits getNumericDocsWithField(FieldInfo fieldInfo) throws IOException {
|
||||
final OneField field = fields.get(fieldInfo.name);
|
||||
final IndexInput in = data.clone();
|
||||
final BytesRef scratch = new BytesRef();
|
||||
return new Bits() {
|
||||
@Override
|
||||
public boolean get(int index) {
|
||||
try {
|
||||
in.seek(field.dataStartFilePointer + (1+field.pattern.length()+2)*index);
|
||||
SimpleTextUtil.readLine(in, scratch); // data
|
||||
SimpleTextUtil.readLine(in, scratch); // 'T' or 'F'
|
||||
return scratch.bytes[scratch.offset] == (byte) 'T';
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return maxDoc;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public BinaryDocValues getBinary(FieldInfo fieldInfo) throws IOException {
|
||||
|
@ -196,7 +223,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
if (docID < 0 || docID >= maxDoc) {
|
||||
throw new IndexOutOfBoundsException("docID must be 0 .. " + (maxDoc-1) + "; got " + docID);
|
||||
}
|
||||
in.seek(field.dataStartFilePointer + (9+field.pattern.length() + field.maxLength)*docID);
|
||||
in.seek(field.dataStartFilePointer + (9+field.pattern.length() + field.maxLength+2)*docID);
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
assert StringHelper.startsWith(scratch, LENGTH);
|
||||
int len;
|
||||
|
@ -217,6 +244,45 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
private Bits getBinaryDocsWithField(FieldInfo fieldInfo) throws IOException {
|
||||
final OneField field = fields.get(fieldInfo.name);
|
||||
final IndexInput in = data.clone();
|
||||
final BytesRef scratch = new BytesRef();
|
||||
final DecimalFormat decoder = new DecimalFormat(field.pattern, new DecimalFormatSymbols(Locale.ROOT));
|
||||
|
||||
return new Bits() {
|
||||
@Override
|
||||
public boolean get(int index) {
|
||||
try {
|
||||
in.seek(field.dataStartFilePointer + (9+field.pattern.length() + field.maxLength+2)*index);
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
assert StringHelper.startsWith(scratch, LENGTH);
|
||||
int len;
|
||||
try {
|
||||
len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, "UTF-8")).intValue();
|
||||
} catch (ParseException pe) {
|
||||
CorruptIndexException e = new CorruptIndexException("failed to parse int length (resource=" + in + ")");
|
||||
e.initCause(pe);
|
||||
throw e;
|
||||
}
|
||||
// skip past bytes
|
||||
byte bytes[] = new byte[len];
|
||||
in.readBytes(bytes, 0, len);
|
||||
SimpleTextUtil.readLine(in, scratch); // newline
|
||||
SimpleTextUtil.readLine(in, scratch); // 'T' or 'F'
|
||||
return scratch.bytes[scratch.offset] == (byte) 'T';
|
||||
} catch (IOException ioe) {
|
||||
throw new RuntimeException(ioe);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return maxDoc;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedDocValues getSorted(FieldInfo fieldInfo) throws IOException {
|
||||
|
@ -241,7 +307,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
in.seek(field.dataStartFilePointer + field.numValues * (9 + field.pattern.length() + field.maxLength) + docID * (1 + field.ordPattern.length()));
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
try {
|
||||
return ordDecoder.parse(scratch.utf8ToString()).intValue();
|
||||
return (int) ordDecoder.parse(scratch.utf8ToString()).longValue()-1;
|
||||
} catch (ParseException pe) {
|
||||
CorruptIndexException e = new CorruptIndexException("failed to parse ord (resource=" + in + ")");
|
||||
e.initCause(pe);
|
||||
|
@ -255,8 +321,12 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
@Override
|
||||
public void lookupOrd(int ord, BytesRef result) {
|
||||
try {
|
||||
if (ord < 0 || ord >= field.numValues) {
|
||||
throw new IndexOutOfBoundsException("ord must be 0 .. " + (field.numValues-1) + "; got " + ord);
|
||||
if (ord == -1) {
|
||||
result.length = 0;
|
||||
return;
|
||||
}
|
||||
if (ord < -1 || ord >= field.numValues) {
|
||||
throw new IndexOutOfBoundsException("ord must be -1 .. " + (field.numValues-1) + "; got " + ord);
|
||||
}
|
||||
in.seek(field.dataStartFilePointer + ord * (9 + field.pattern.length() + field.maxLength));
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
|
@ -362,6 +432,21 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
if (field.getDocValuesType() == FieldInfo.DocValuesType.SORTED_SET) {
|
||||
return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
|
||||
} else if (field.getDocValuesType() == FieldInfo.DocValuesType.SORTED) {
|
||||
return new SortedDocsWithField(getSorted(field), maxDoc);
|
||||
} else if (field.getDocValuesType() == FieldInfo.DocValuesType.BINARY) {
|
||||
return getBinaryDocsWithField(field);
|
||||
} else if (field.getDocValuesType() == FieldInfo.DocValuesType.NUMERIC) {
|
||||
return getNumericDocsWithField(field);
|
||||
} else {
|
||||
return new Bits.MatchAllBits(maxDoc);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
|
|
@ -78,7 +78,7 @@ class SimpleTextDocValuesWriter extends DocValuesConsumer {
|
|||
long minValue = Long.MAX_VALUE;
|
||||
long maxValue = Long.MIN_VALUE;
|
||||
for(Number n : values) {
|
||||
long v = n.longValue();
|
||||
long v = n == null ? 0 : n.longValue();
|
||||
minValue = Math.min(minValue, v);
|
||||
maxValue = Math.max(maxValue, v);
|
||||
}
|
||||
|
@ -112,13 +112,19 @@ class SimpleTextDocValuesWriter extends DocValuesConsumer {
|
|||
|
||||
// second pass to write the values
|
||||
for(Number n : values) {
|
||||
long value = n.longValue();
|
||||
long value = n == null ? 0 : n.longValue();
|
||||
assert value >= minValue;
|
||||
Number delta = BigInteger.valueOf(value).subtract(BigInteger.valueOf(minValue));
|
||||
String s = encoder.format(delta);
|
||||
assert s.length() == patternString.length();
|
||||
SimpleTextUtil.write(data, s, scratch);
|
||||
SimpleTextUtil.writeNewline(data);
|
||||
if (n == null) {
|
||||
SimpleTextUtil.write(data, "F", scratch);
|
||||
} else {
|
||||
SimpleTextUtil.write(data, "T", scratch);
|
||||
}
|
||||
SimpleTextUtil.writeNewline(data);
|
||||
numDocsWritten++;
|
||||
assert numDocsWritten <= numDocs;
|
||||
}
|
||||
|
@ -132,7 +138,8 @@ class SimpleTextDocValuesWriter extends DocValuesConsumer {
|
|||
assert field.getDocValuesType() == DocValuesType.BINARY;
|
||||
int maxLength = 0;
|
||||
for(BytesRef value : values) {
|
||||
maxLength = Math.max(maxLength, value.length);
|
||||
final int length = value == null ? 0 : value.length;
|
||||
maxLength = Math.max(maxLength, length);
|
||||
}
|
||||
writeFieldEntry(field, FieldInfo.DocValuesType.BINARY);
|
||||
|
||||
|
@ -155,19 +162,28 @@ class SimpleTextDocValuesWriter extends DocValuesConsumer {
|
|||
int numDocsWritten = 0;
|
||||
for(BytesRef value : values) {
|
||||
// write length
|
||||
final int length = value == null ? 0 : value.length;
|
||||
SimpleTextUtil.write(data, LENGTH);
|
||||
SimpleTextUtil.write(data, encoder.format(value.length), scratch);
|
||||
SimpleTextUtil.write(data, encoder.format(length), scratch);
|
||||
SimpleTextUtil.writeNewline(data);
|
||||
|
||||
// write bytes -- don't use SimpleText.write
|
||||
// because it escapes:
|
||||
data.writeBytes(value.bytes, value.offset, value.length);
|
||||
if (value != null) {
|
||||
data.writeBytes(value.bytes, value.offset, value.length);
|
||||
}
|
||||
|
||||
// pad to fit
|
||||
for (int i = value.length; i < maxLength; i++) {
|
||||
for (int i = length; i < maxLength; i++) {
|
||||
data.writeByte((byte)' ');
|
||||
}
|
||||
SimpleTextUtil.writeNewline(data);
|
||||
if (value == null) {
|
||||
SimpleTextUtil.write(data, "F", scratch);
|
||||
} else {
|
||||
SimpleTextUtil.write(data, "T", scratch);
|
||||
}
|
||||
SimpleTextUtil.writeNewline(data);
|
||||
numDocsWritten++;
|
||||
}
|
||||
|
||||
|
@ -209,7 +225,7 @@ class SimpleTextDocValuesWriter extends DocValuesConsumer {
|
|||
SimpleTextUtil.writeNewline(data);
|
||||
final DecimalFormat encoder = new DecimalFormat(sb.toString(), new DecimalFormatSymbols(Locale.ROOT));
|
||||
|
||||
int maxOrdBytes = Integer.toString(valueCount).length();
|
||||
int maxOrdBytes = Long.toString(valueCount+1L).length();
|
||||
sb.setLength(0);
|
||||
for (int i = 0; i < maxOrdBytes; i++) {
|
||||
sb.append('0');
|
||||
|
@ -246,7 +262,7 @@ class SimpleTextDocValuesWriter extends DocValuesConsumer {
|
|||
assert valuesSeen == valueCount;
|
||||
|
||||
for(Number ord : docToOrd) {
|
||||
SimpleTextUtil.write(data, ordEncoder.format(ord.intValue()), scratch);
|
||||
SimpleTextUtil.write(data, ordEncoder.format(ord.longValue()+1), scratch);
|
||||
SimpleTextUtil.writeNewline(data);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -69,7 +69,8 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
/**
|
||||
* Writes numeric docvalues for a field.
|
||||
* @param field field information
|
||||
* @param values Iterable of numeric values (one for each document).
|
||||
* @param values Iterable of numeric values (one for each document). {@code null} indicates
|
||||
* a missing value.
|
||||
* @throws IOException if an I/O error occurred.
|
||||
*/
|
||||
public abstract void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException;
|
||||
|
@ -77,7 +78,8 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
/**
|
||||
* Writes binary docvalues for a field.
|
||||
* @param field field information
|
||||
* @param values Iterable of binary values (one for each document).
|
||||
* @param values Iterable of binary values (one for each document). {@code null} indicates
|
||||
* a missing value.
|
||||
* @throws IOException if an I/O error occurred.
|
||||
*/
|
||||
public abstract void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException;
|
||||
|
@ -86,7 +88,8 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
* Writes pre-sorted binary docvalues for a field.
|
||||
* @param field field information
|
||||
* @param values Iterable of binary values in sorted order (deduplicated).
|
||||
* @param docToOrd Iterable of ordinals (one for each document).
|
||||
* @param docToOrd Iterable of ordinals (one for each document). {@code -1} indicates
|
||||
* a missing value.
|
||||
* @throws IOException if an I/O error occurred.
|
||||
*/
|
||||
public abstract void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException;
|
||||
|
@ -95,7 +98,8 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
* Writes pre-sorted set docvalues for a field
|
||||
* @param field field information
|
||||
* @param values Iterable of binary values in sorted order (deduplicated).
|
||||
* @param docToOrdCount Iterable of the number of values for each document.
|
||||
* @param docToOrdCount Iterable of the number of values for each document. A zero ordinal
|
||||
* count indicates a missing value.
|
||||
* @param ords Iterable of ordinal occurrences (docToOrdCount*maxDoc total).
|
||||
* @throws IOException if an I/O error occurred.
|
||||
*/
|
||||
|
@ -107,7 +111,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
* The default implementation calls {@link #addNumericField}, passing
|
||||
* an Iterable that merges and filters deleted documents on the fly.
|
||||
*/
|
||||
public void mergeNumericField(FieldInfo fieldInfo, final MergeState mergeState, final List<NumericDocValues> toMerge) throws IOException {
|
||||
public void mergeNumericField(final FieldInfo fieldInfo, final MergeState mergeState, final List<NumericDocValues> toMerge, final List<Bits> docsWithField) throws IOException {
|
||||
|
||||
addNumericField(fieldInfo,
|
||||
new Iterable<Number>() {
|
||||
|
@ -116,10 +120,11 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
return new Iterator<Number>() {
|
||||
int readerUpto = -1;
|
||||
int docIDUpto;
|
||||
long nextValue;
|
||||
Long nextValue;
|
||||
AtomicReader currentReader;
|
||||
NumericDocValues currentValues;
|
||||
Bits currentLiveDocs;
|
||||
Bits currentDocsWithField;
|
||||
boolean nextIsSet;
|
||||
|
||||
@Override
|
||||
|
@ -139,7 +144,6 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
}
|
||||
assert nextIsSet;
|
||||
nextIsSet = false;
|
||||
// TODO: make a mutable number
|
||||
return nextValue;
|
||||
}
|
||||
|
||||
|
@ -155,6 +159,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
currentReader = mergeState.readers.get(readerUpto);
|
||||
currentValues = toMerge.get(readerUpto);
|
||||
currentLiveDocs = currentReader.getLiveDocs();
|
||||
currentDocsWithField = docsWithField.get(readerUpto);
|
||||
}
|
||||
docIDUpto = 0;
|
||||
continue;
|
||||
|
@ -162,7 +167,11 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
|
||||
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
|
||||
nextIsSet = true;
|
||||
nextValue = currentValues.get(docIDUpto);
|
||||
if (currentDocsWithField.get(docIDUpto)) {
|
||||
nextValue = currentValues.get(docIDUpto);
|
||||
} else {
|
||||
nextValue = null;
|
||||
}
|
||||
docIDUpto++;
|
||||
return true;
|
||||
}
|
||||
|
@ -181,7 +190,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
* The default implementation calls {@link #addBinaryField}, passing
|
||||
* an Iterable that merges and filters deleted documents on the fly.
|
||||
*/
|
||||
public void mergeBinaryField(FieldInfo fieldInfo, final MergeState mergeState, final List<BinaryDocValues> toMerge) throws IOException {
|
||||
public void mergeBinaryField(FieldInfo fieldInfo, final MergeState mergeState, final List<BinaryDocValues> toMerge, final List<Bits> docsWithField) throws IOException {
|
||||
|
||||
addBinaryField(fieldInfo,
|
||||
new Iterable<BytesRef>() {
|
||||
|
@ -191,9 +200,11 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
int readerUpto = -1;
|
||||
int docIDUpto;
|
||||
BytesRef nextValue = new BytesRef();
|
||||
BytesRef nextPointer; // points to null if missing, or nextValue
|
||||
AtomicReader currentReader;
|
||||
BinaryDocValues currentValues;
|
||||
Bits currentLiveDocs;
|
||||
Bits currentDocsWithField;
|
||||
boolean nextIsSet;
|
||||
|
||||
@Override
|
||||
|
@ -213,8 +224,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
}
|
||||
assert nextIsSet;
|
||||
nextIsSet = false;
|
||||
// TODO: make a mutable number
|
||||
return nextValue;
|
||||
return nextPointer;
|
||||
}
|
||||
|
||||
private boolean setNext() {
|
||||
|
@ -228,6 +238,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
if (readerUpto < toMerge.size()) {
|
||||
currentReader = mergeState.readers.get(readerUpto);
|
||||
currentValues = toMerge.get(readerUpto);
|
||||
currentDocsWithField = docsWithField.get(readerUpto);
|
||||
currentLiveDocs = currentReader.getLiveDocs();
|
||||
}
|
||||
docIDUpto = 0;
|
||||
|
@ -236,7 +247,12 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
|
||||
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
|
||||
nextIsSet = true;
|
||||
currentValues.get(docIDUpto, nextValue);
|
||||
if (currentDocsWithField.get(docIDUpto)) {
|
||||
currentValues.get(docIDUpto, nextValue);
|
||||
nextPointer = nextValue;
|
||||
} else {
|
||||
nextPointer = null;
|
||||
}
|
||||
docIDUpto++;
|
||||
return true;
|
||||
}
|
||||
|
@ -272,7 +288,10 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
OpenBitSet bitset = new OpenBitSet(dv.getValueCount());
|
||||
for (int i = 0; i < reader.maxDoc(); i++) {
|
||||
if (liveDocs.get(i)) {
|
||||
bitset.set(dv.getOrd(i));
|
||||
int ord = dv.getOrd(i);
|
||||
if (ord >= 0) {
|
||||
bitset.set(ord);
|
||||
}
|
||||
}
|
||||
}
|
||||
liveTerms[sub] = new BitsFilteredTermsEnum(dv.termsEnum(), bitset);
|
||||
|
@ -368,7 +387,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
|
||||
nextIsSet = true;
|
||||
int segOrd = dvs[readerUpto].getOrd(docIDUpto);
|
||||
nextValue = (int) map.getGlobalOrd(readerUpto, segOrd);
|
||||
nextValue = segOrd == -1 ? -1 : (int) map.getGlobalOrd(readerUpto, segOrd);
|
||||
docIDUpto++;
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.lucene.index.FieldInfo;
|
|||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
/** Abstract API that produces numeric, binary and
|
||||
* sorted docvalues.
|
||||
|
@ -56,4 +57,65 @@ public abstract class DocValuesProducer implements Closeable {
|
|||
* The returned instance need not be thread-safe: it will only be
|
||||
* used by a single thread. */
|
||||
public abstract SortedSetDocValues getSortedSet(FieldInfo field) throws IOException;
|
||||
|
||||
/** Returns a {@link Bits} at the size of <code>reader.maxDoc()</code>,
|
||||
* with turned on bits for each docid that does have a value for this field.
|
||||
* The returned instance need not be thread-safe: it will only be
|
||||
* used by a single thread. */
|
||||
public abstract Bits getDocsWithField(FieldInfo field) throws IOException;
|
||||
|
||||
/**
|
||||
* A simple implementation of {@link DocValuesProducer#getDocsWithField} that
|
||||
* returns {@code true} if a document has an ordinal >= 0
|
||||
* <p>
|
||||
* Codecs can choose to use this (or implement it more efficiently another way), but
|
||||
* in most cases a Bits is unnecessary anyway: users can check this as they go.
|
||||
*/
|
||||
public static class SortedDocsWithField implements Bits {
|
||||
final SortedDocValues in;
|
||||
final int maxDoc;
|
||||
|
||||
public SortedDocsWithField(SortedDocValues in, int maxDoc) {
|
||||
this.in = in;
|
||||
this.maxDoc = maxDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean get(int index) {
|
||||
return in.getOrd(index) >= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return maxDoc;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A simple implementation of {@link DocValuesProducer#getDocsWithField} that
|
||||
* returns {@code true} if a document has any ordinals.
|
||||
* <p>
|
||||
* Codecs can choose to use this (or implement it more efficiently another way), but
|
||||
* in most cases a Bits is unnecessary anyway: users can check this as they go.
|
||||
*/
|
||||
public static class SortedSetDocsWithField implements Bits {
|
||||
final SortedSetDocValues in;
|
||||
final int maxDoc;
|
||||
|
||||
public SortedSetDocsWithField(SortedSetDocValues in, int maxDoc) {
|
||||
this.in = in;
|
||||
this.maxDoc = maxDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean get(int index) {
|
||||
in.setDocument(index);
|
||||
return in.nextOrd() != SortedSetDocValues.NO_MORE_ORDS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return maxDoc;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,124 @@
|
|||
package org.apache.lucene.codecs;
|
||||
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* a utility class to write missing values for SORTED_SET as if they were the empty string
|
||||
* (to simulate pre-Lucene4.5 dv behavior for testing old codecs)
|
||||
*/
|
||||
// nocommit: move this to test-framework with all the impersonators of
|
||||
// these old codecs once new memory/disk codecs are written that support missing
|
||||
public class MissingOrdRemapper {
|
||||
|
||||
/** insert an empty byte[] to the front of this iterable */
|
||||
public static Iterable<BytesRef> insertEmptyValue(final Iterable<BytesRef> iterable) {
|
||||
return new Iterable<BytesRef>() {
|
||||
@Override
|
||||
public Iterator<BytesRef> iterator() {
|
||||
return new Iterator<BytesRef>() {
|
||||
boolean seenEmpty = false;
|
||||
Iterator<BytesRef> in = iterable.iterator();
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return !seenEmpty || in.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef next() {
|
||||
if (!seenEmpty) {
|
||||
seenEmpty = true;
|
||||
return new BytesRef();
|
||||
} else {
|
||||
return in.next();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/** remaps ord -1 to ord 0 on this iterable. */
|
||||
public static Iterable<Number> mapMissingToOrd0(final Iterable<Number> iterable) {
|
||||
return new Iterable<Number>() {
|
||||
@Override
|
||||
public Iterator<Number> iterator() {
|
||||
return new Iterator<Number>() {
|
||||
Iterator<Number> in = iterable.iterator();
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return in.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Number next() {
|
||||
Number n = in.next();
|
||||
if (n.longValue() == -1) {
|
||||
return 0;
|
||||
} else {
|
||||
return n;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/** remaps every ord+1 on this iterable */
|
||||
public static Iterable<Number> mapAllOrds(final Iterable<Number> iterable) {
|
||||
return new Iterable<Number>() {
|
||||
@Override
|
||||
public Iterator<Number> iterator() {
|
||||
return new Iterator<Number>() {
|
||||
Iterator<Number> in = iterable.iterator();
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return in.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Number next() {
|
||||
Number n = in.next();
|
||||
return n.longValue()+1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -35,6 +35,7 @@ import org.apache.lucene.index.SortedSetDocValues;
|
|||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
|
@ -620,6 +621,11 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
|||
throw new IllegalStateException("Lucene 4.0 does not support SortedSet: how did you pull this off?");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
return new Bits.MatchAllBits(state.segmentInfo.getDocCount());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
dir.close();
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.NoSuchElementException;
|
|||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.MissingOrdRemapper;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
@ -106,7 +107,8 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
|
|||
|
||||
long count = 0;
|
||||
for (Number nv : values) {
|
||||
final long v = nv.longValue();
|
||||
// TODO: support this as MemoryDVFormat (and be smart about missing maybe)
|
||||
final long v = nv == null ? 0 : nv.longValue();
|
||||
|
||||
if (gcd != 1) {
|
||||
if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
|
||||
|
@ -142,7 +144,7 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
|
|||
if (formatAndBits.bitsPerValue == 8 && minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) {
|
||||
meta.writeByte(UNCOMPRESSED); // uncompressed
|
||||
for (Number nv : values) {
|
||||
data.writeByte((byte) nv.longValue());
|
||||
data.writeByte(nv == null ? 0 : (byte) nv.longValue());
|
||||
}
|
||||
} else {
|
||||
meta.writeByte(TABLE_COMPRESSED); // table-compressed
|
||||
|
@ -160,7 +162,7 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
|
|||
|
||||
final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, formatAndBits.format, maxDoc, formatAndBits.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
|
||||
for(Number nv : values) {
|
||||
writer.add(encode.get(nv.longValue()));
|
||||
writer.add(encode.get(nv == null ? 0 : nv.longValue()));
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
|
@ -173,7 +175,8 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
|
|||
|
||||
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
|
||||
for (Number nv : values) {
|
||||
writer.add((nv.longValue() - minValue) / gcd);
|
||||
long value = nv == null ? 0 : nv.longValue();
|
||||
writer.add((value - minValue) / gcd);
|
||||
}
|
||||
writer.finish();
|
||||
} else {
|
||||
|
@ -184,7 +187,7 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
|
|||
|
||||
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
|
||||
for (Number nv : values) {
|
||||
writer.add(nv.longValue());
|
||||
writer.add(nv == null ? 0 : nv.longValue());
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
|
@ -216,9 +219,12 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
|
|||
int maxLength = Integer.MIN_VALUE;
|
||||
final long startFP = data.getFilePointer();
|
||||
for(BytesRef v : values) {
|
||||
minLength = Math.min(minLength, v.length);
|
||||
maxLength = Math.max(maxLength, v.length);
|
||||
data.writeBytes(v.bytes, v.offset, v.length);
|
||||
final int length = v == null ? 0 : v.length;
|
||||
minLength = Math.min(minLength, length);
|
||||
maxLength = Math.max(maxLength, length);
|
||||
if (v != null) {
|
||||
data.writeBytes(v.bytes, v.offset, v.length);
|
||||
}
|
||||
}
|
||||
meta.writeLong(startFP);
|
||||
meta.writeLong(data.getFilePointer() - startFP);
|
||||
|
@ -234,7 +240,9 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
|
|||
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
|
||||
long addr = 0;
|
||||
for (BytesRef v : values) {
|
||||
addr += v.length;
|
||||
if (v != null) {
|
||||
addr += v.length;
|
||||
}
|
||||
writer.add(addr);
|
||||
}
|
||||
writer.finish();
|
||||
|
@ -262,6 +270,33 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
|
|||
|
||||
@Override
|
||||
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
|
||||
// three cases for simulating the old writer:
|
||||
// 1. no missing
|
||||
// 2. missing (and empty string in use): remap ord=-1 -> ord=0
|
||||
// 3. missing (and empty string not in use): remap all ords +1, insert empty string into values
|
||||
boolean anyMissing = false;
|
||||
for (Number n : docToOrd) {
|
||||
if (n.longValue() == -1) {
|
||||
anyMissing = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
boolean hasEmptyString = false;
|
||||
for (BytesRef b : values) {
|
||||
hasEmptyString = b.length == 0;
|
||||
break;
|
||||
}
|
||||
|
||||
if (!anyMissing) {
|
||||
// nothing to do
|
||||
} else if (hasEmptyString) {
|
||||
docToOrd = MissingOrdRemapper.mapMissingToOrd0(docToOrd);
|
||||
} else {
|
||||
docToOrd = MissingOrdRemapper.mapAllOrds(docToOrd);
|
||||
values = MissingOrdRemapper.insertEmptyValue(values);
|
||||
}
|
||||
|
||||
// write the ordinals as numerics
|
||||
addNumericField(field, docToOrd, false);
|
||||
|
||||
|
|
|
@ -429,6 +429,15 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
if (field.getDocValuesType() == FieldInfo.DocValuesType.SORTED_SET) {
|
||||
return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
|
||||
} else {
|
||||
return new Bits.MatchAllBits(maxDoc);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.lucene.index.SegmentReadState;
|
|||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
|
@ -265,6 +266,12 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
|
|||
DocValuesProducer producer = fields.get(field.name);
|
||||
return producer == null ? null : producer.getSortedSet(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
DocValuesProducer producer = fields.get(field.name);
|
||||
return producer == null ? null : producer.getDocsWithField(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
|
|
@ -207,6 +207,12 @@ public abstract class AtomicReader extends IndexReader {
|
|||
* this field. The returned instance should only be
|
||||
* used by a single thread. */
|
||||
public abstract SortedSetDocValues getSortedSetDocValues(String field) throws IOException;
|
||||
|
||||
/** Returns a {@link Bits} at the size of <code>reader.maxDoc()</code>,
|
||||
* with turned on bits for each docid that does have a value for this field,
|
||||
* or null if no DocValues were indexed for this field. The
|
||||
* returned instance should only be used by a single thread */
|
||||
public abstract Bits getDocsWithField(String field) throws IOException;
|
||||
|
||||
/** Returns {@link NumericDocValues} representing norms
|
||||
* for this field, or null if no {@link NumericDocValues}
|
||||
|
|
|
@ -26,6 +26,8 @@ import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
|
|||
import org.apache.lucene.util.ByteBlockPool;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.OpenBitSet;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
|
@ -38,6 +40,9 @@ class BinaryDocValuesWriter extends DocValuesWriter {
|
|||
|
||||
private final ByteBlockPool pool;
|
||||
private final AppendingDeltaPackedLongBuffer lengths;
|
||||
private final OpenBitSet docsWithField;
|
||||
private final Counter iwBytesUsed;
|
||||
private long bytesUsed;
|
||||
private final FieldInfo fieldInfo;
|
||||
private int addedValues = 0;
|
||||
|
||||
|
@ -45,6 +50,10 @@ class BinaryDocValuesWriter extends DocValuesWriter {
|
|||
this.fieldInfo = fieldInfo;
|
||||
this.pool = new ByteBlockPool(new DirectTrackingAllocator(iwBytesUsed));
|
||||
this.lengths = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
|
||||
this.iwBytesUsed = iwBytesUsed;
|
||||
this.docsWithField = new OpenBitSet();
|
||||
this.bytesUsed = docsWithFieldBytesUsed();
|
||||
iwBytesUsed.addAndGet(bytesUsed);
|
||||
}
|
||||
|
||||
public void addValue(int docID, BytesRef value) {
|
||||
|
@ -66,6 +75,19 @@ class BinaryDocValuesWriter extends DocValuesWriter {
|
|||
addedValues++;
|
||||
lengths.add(value.length);
|
||||
pool.append(value);
|
||||
docsWithField.set(docID);
|
||||
updateBytesUsed();
|
||||
}
|
||||
|
||||
private long docsWithFieldBytesUsed() {
|
||||
// nocommit: this is not correct
|
||||
return docsWithField.getBits().length*RamUsageEstimator.NUM_BYTES_LONG;
|
||||
}
|
||||
|
||||
private void updateBytesUsed() {
|
||||
final long newBytesUsed = docsWithFieldBytesUsed();
|
||||
iwBytesUsed.addAndGet(newBytesUsed - bytesUsed);
|
||||
bytesUsed = newBytesUsed;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -111,19 +133,23 @@ class BinaryDocValuesWriter extends DocValuesWriter {
|
|||
if (!hasNext()) {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
final BytesRef v;
|
||||
if (upto < size) {
|
||||
int length = (int) lengthsIterator.next();
|
||||
value.grow(length);
|
||||
value.length = length;
|
||||
pool.readBytes(byteOffset, value.bytes, value.offset, value.length);
|
||||
byteOffset += length;
|
||||
if (docsWithField.get(upto)) {
|
||||
v = value;
|
||||
} else {
|
||||
v = null;
|
||||
}
|
||||
} else {
|
||||
// This is to handle last N documents not having
|
||||
// this DV field in the end of the segment:
|
||||
value.length = 0;
|
||||
v = null;
|
||||
}
|
||||
upto++;
|
||||
return value;
|
||||
return v;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -1280,7 +1280,8 @@ public class CheckIndex {
|
|||
if (reader.getBinaryDocValues(fieldInfo.name) != null ||
|
||||
reader.getNumericDocValues(fieldInfo.name) != null ||
|
||||
reader.getSortedDocValues(fieldInfo.name) != null ||
|
||||
reader.getSortedSetDocValues(fieldInfo.name) != null) {
|
||||
reader.getSortedSetDocValues(fieldInfo.name) != null ||
|
||||
reader.getDocsWithField(fieldInfo.name) != null) {
|
||||
throw new RuntimeException("field: " + fieldInfo.name + " has docvalues but should omit them!");
|
||||
}
|
||||
}
|
||||
|
@ -1301,26 +1302,37 @@ public class CheckIndex {
|
|||
return status;
|
||||
}
|
||||
|
||||
private static void checkBinaryDocValues(String fieldName, AtomicReader reader, BinaryDocValues dv) {
|
||||
private static void checkBinaryDocValues(String fieldName, AtomicReader reader, BinaryDocValues dv, Bits docsWithField) {
|
||||
BytesRef scratch = new BytesRef();
|
||||
for (int i = 0; i < reader.maxDoc(); i++) {
|
||||
dv.get(i, scratch);
|
||||
assert scratch.isValid();
|
||||
if (docsWithField.get(i) == false && scratch.length > 0) {
|
||||
throw new RuntimeException("dv for field: " + fieldName + " is missing but has value=" + scratch + " for doc: " + i);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static void checkSortedDocValues(String fieldName, AtomicReader reader, SortedDocValues dv) {
|
||||
checkBinaryDocValues(fieldName, reader, dv);
|
||||
private static void checkSortedDocValues(String fieldName, AtomicReader reader, SortedDocValues dv, Bits docsWithField) {
|
||||
checkBinaryDocValues(fieldName, reader, dv, docsWithField);
|
||||
final int maxOrd = dv.getValueCount()-1;
|
||||
FixedBitSet seenOrds = new FixedBitSet(dv.getValueCount());
|
||||
int maxOrd2 = -1;
|
||||
for (int i = 0; i < reader.maxDoc(); i++) {
|
||||
int ord = dv.getOrd(i);
|
||||
if (ord < 0 || ord > maxOrd) {
|
||||
if (ord == -1) {
|
||||
if (docsWithField.get(i)) {
|
||||
throw new RuntimeException("dv for field: " + fieldName + " has -1 ord but is not marked missing for doc: " + i);
|
||||
}
|
||||
} else if (ord < -1 || ord > maxOrd) {
|
||||
throw new RuntimeException("ord out of bounds: " + ord);
|
||||
} else {
|
||||
if (!docsWithField.get(i)) {
|
||||
throw new RuntimeException("dv for field: " + fieldName + " is missing but has ord=" + ord + " for doc: " + i);
|
||||
}
|
||||
maxOrd2 = Math.max(maxOrd2, ord);
|
||||
seenOrds.set(ord);
|
||||
}
|
||||
maxOrd2 = Math.max(maxOrd2, ord);
|
||||
seenOrds.set(ord);
|
||||
}
|
||||
if (maxOrd != maxOrd2) {
|
||||
throw new RuntimeException("dv for field: " + fieldName + " reports wrong maxOrd=" + maxOrd + " but this is not the case: " + maxOrd2);
|
||||
|
@ -1342,7 +1354,7 @@ public class CheckIndex {
|
|||
}
|
||||
}
|
||||
|
||||
private static void checkSortedSetDocValues(String fieldName, AtomicReader reader, SortedSetDocValues dv) {
|
||||
private static void checkSortedSetDocValues(String fieldName, AtomicReader reader, SortedSetDocValues dv, Bits docsWithField) {
|
||||
final long maxOrd = dv.getValueCount()-1;
|
||||
OpenBitSet seenOrds = new OpenBitSet(dv.getValueCount());
|
||||
long maxOrd2 = -1;
|
||||
|
@ -1350,16 +1362,28 @@ public class CheckIndex {
|
|||
dv.setDocument(i);
|
||||
long lastOrd = -1;
|
||||
long ord;
|
||||
while ((ord = dv.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
|
||||
if (ord <= lastOrd) {
|
||||
throw new RuntimeException("ords out of order: " + ord + " <= " + lastOrd + " for doc: " + i);
|
||||
if (docsWithField.get(i)) {
|
||||
int ordCount = 0;
|
||||
while ((ord = dv.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
|
||||
ordCount++;
|
||||
if (ord <= lastOrd) {
|
||||
throw new RuntimeException("ords out of order: " + ord + " <= " + lastOrd + " for doc: " + i);
|
||||
}
|
||||
if (ord < 0 || ord > maxOrd) {
|
||||
throw new RuntimeException("ord out of bounds: " + ord);
|
||||
}
|
||||
lastOrd = ord;
|
||||
maxOrd2 = Math.max(maxOrd2, ord);
|
||||
seenOrds.set(ord);
|
||||
}
|
||||
if (ord < 0 || ord > maxOrd) {
|
||||
throw new RuntimeException("ord out of bounds: " + ord);
|
||||
if (ordCount == 0) {
|
||||
throw new RuntimeException("dv for field: " + fieldName + " has no ordinals but is not marked missing for doc: " + i);
|
||||
}
|
||||
} else {
|
||||
long o = dv.nextOrd();
|
||||
if (o != SortedSetDocValues.NO_MORE_ORDS) {
|
||||
throw new RuntimeException("dv for field: " + fieldName + " is marked missing but has ord=" + o + " for doc: " + i);
|
||||
}
|
||||
lastOrd = ord;
|
||||
maxOrd2 = Math.max(maxOrd2, ord);
|
||||
seenOrds.set(ord);
|
||||
}
|
||||
}
|
||||
if (maxOrd != maxOrd2) {
|
||||
|
@ -1383,17 +1407,26 @@ public class CheckIndex {
|
|||
}
|
||||
}
|
||||
|
||||
private static void checkNumericDocValues(String fieldName, AtomicReader reader, NumericDocValues ndv) {
|
||||
private static void checkNumericDocValues(String fieldName, AtomicReader reader, NumericDocValues ndv, Bits docsWithField) {
|
||||
for (int i = 0; i < reader.maxDoc(); i++) {
|
||||
ndv.get(i);
|
||||
long value = ndv.get(i);
|
||||
if (docsWithField.get(i) == false && value > 0) {
|
||||
throw new RuntimeException("dv for field: " + fieldName + " is marked missing but has value=" + value + " for doc: " + i);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static void checkDocValues(FieldInfo fi, AtomicReader reader, PrintStream infoStream, DocValuesStatus status) throws Exception {
|
||||
Bits docsWithField = reader.getDocsWithField(fi.name);
|
||||
if (docsWithField == null) {
|
||||
throw new RuntimeException(fi.name + " docsWithField does not exist");
|
||||
} else if (docsWithField.length() != reader.maxDoc()) {
|
||||
throw new RuntimeException(fi.name + " docsWithField has incorrect length: " + docsWithField.length() + ",expected: " + reader.maxDoc());
|
||||
}
|
||||
switch(fi.getDocValuesType()) {
|
||||
case SORTED:
|
||||
status.totalSortedFields++;
|
||||
checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name));
|
||||
checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name), docsWithField);
|
||||
if (reader.getBinaryDocValues(fi.name) != null ||
|
||||
reader.getNumericDocValues(fi.name) != null ||
|
||||
reader.getSortedSetDocValues(fi.name) != null) {
|
||||
|
@ -1402,7 +1435,7 @@ public class CheckIndex {
|
|||
break;
|
||||
case SORTED_SET:
|
||||
status.totalSortedSetFields++;
|
||||
checkSortedSetDocValues(fi.name, reader, reader.getSortedSetDocValues(fi.name));
|
||||
checkSortedSetDocValues(fi.name, reader, reader.getSortedSetDocValues(fi.name), docsWithField);
|
||||
if (reader.getBinaryDocValues(fi.name) != null ||
|
||||
reader.getNumericDocValues(fi.name) != null ||
|
||||
reader.getSortedDocValues(fi.name) != null) {
|
||||
|
@ -1411,7 +1444,7 @@ public class CheckIndex {
|
|||
break;
|
||||
case BINARY:
|
||||
status.totalBinaryFields++;
|
||||
checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name));
|
||||
checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name), docsWithField);
|
||||
if (reader.getNumericDocValues(fi.name) != null ||
|
||||
reader.getSortedDocValues(fi.name) != null ||
|
||||
reader.getSortedSetDocValues(fi.name) != null) {
|
||||
|
@ -1420,7 +1453,7 @@ public class CheckIndex {
|
|||
break;
|
||||
case NUMERIC:
|
||||
status.totalNumericFields++;
|
||||
checkNumericDocValues(fi.name, reader, reader.getNumericDocValues(fi.name));
|
||||
checkNumericDocValues(fi.name, reader, reader.getNumericDocValues(fi.name), docsWithField);
|
||||
if (reader.getBinaryDocValues(fi.name) != null ||
|
||||
reader.getSortedDocValues(fi.name) != null ||
|
||||
reader.getSortedSetDocValues(fi.name) != null) {
|
||||
|
@ -1435,7 +1468,7 @@ public class CheckIndex {
|
|||
private static void checkNorms(FieldInfo fi, AtomicReader reader, PrintStream infoStream) throws IOException {
|
||||
switch(fi.getNormType()) {
|
||||
case NUMERIC:
|
||||
checkNumericDocValues(fi.name, reader, reader.getNormValues(fi.name));
|
||||
checkNumericDocValues(fi.name, reader, reader.getNormValues(fi.name), new Bits.MatchAllBits(reader.maxDoc()));
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError("wtf: " + fi.getNormType());
|
||||
|
|
|
@ -143,7 +143,7 @@ final class DocValuesProcessor extends StoredFieldsConsumer {
|
|||
DocValuesWriter writer = writers.get(fieldInfo.name);
|
||||
NumericDocValuesWriter numericWriter;
|
||||
if (writer == null) {
|
||||
numericWriter = new NumericDocValuesWriter(fieldInfo, bytesUsed);
|
||||
numericWriter = new NumericDocValuesWriter(fieldInfo, bytesUsed, true);
|
||||
writers.put(fieldInfo.name, numericWriter);
|
||||
} else if (!(writer instanceof NumericDocValuesWriter)) {
|
||||
throw new IllegalArgumentException("Incompatible DocValues type: field \"" + fieldInfo.name + "\" changed from " + getTypeDesc(writer) + " to numeric");
|
||||
|
|
|
@ -414,4 +414,10 @@ public class FilterAtomicReader extends AtomicReader {
|
|||
return in.getNormValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
ensureOpen();
|
||||
return in.getDocsWithField(field);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.List;
|
|||
|
||||
import org.apache.lucene.index.MultiTermsEnum.TermsEnumIndex;
|
||||
import org.apache.lucene.index.MultiTermsEnum.TermsEnumWithSlice;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
|
||||
import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
|
||||
|
@ -135,6 +136,51 @@ public class MultiDocValues {
|
|||
};
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns a Bits for a reader's docsWithField (potentially merging on-the-fly)
|
||||
* <p>
|
||||
* This is a slow way to access this bitset. Instead, access them per-segment
|
||||
* with {@link AtomicReader#getDocsWithField(String)}
|
||||
* </p>
|
||||
* */
|
||||
public static Bits getDocsWithField(final IndexReader r, final String field) throws IOException {
|
||||
final List<AtomicReaderContext> leaves = r.leaves();
|
||||
final int size = leaves.size();
|
||||
if (size == 0) {
|
||||
return null;
|
||||
} else if (size == 1) {
|
||||
return leaves.get(0).reader().getDocsWithField(field);
|
||||
}
|
||||
|
||||
boolean anyReal = false;
|
||||
boolean anyMissing = false;
|
||||
final Bits[] values = new Bits[size];
|
||||
final int[] starts = new int[size+1];
|
||||
for (int i = 0; i < size; i++) {
|
||||
AtomicReaderContext context = leaves.get(i);
|
||||
Bits v = context.reader().getDocsWithField(field);
|
||||
if (v == null) {
|
||||
v = new Bits.MatchNoBits(context.reader().maxDoc());
|
||||
anyMissing = true;
|
||||
} else {
|
||||
anyReal = true;
|
||||
if (v instanceof Bits.MatchAllBits == false) {
|
||||
anyMissing = true;
|
||||
}
|
||||
}
|
||||
values[i] = v;
|
||||
starts[i] = context.docBase;
|
||||
}
|
||||
starts[size] = r.maxDoc();
|
||||
|
||||
if (!anyReal) {
|
||||
return null;
|
||||
} else if (!anyMissing) {
|
||||
return new Bits.MatchAllBits(r.maxDoc());
|
||||
} else {
|
||||
return new MultiBits(values, starts, false);
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns a BinaryDocValues for a reader's docvalues (potentially merging on-the-fly)
|
||||
* <p>
|
||||
|
|
|
@ -44,7 +44,7 @@ final class NormsConsumerPerField extends InvertedDocEndConsumerPerField impleme
|
|||
if (fieldInfo.isIndexed() && !fieldInfo.omitsNorms()) {
|
||||
if (consumer == null) {
|
||||
fieldInfo.setNormValueType(FieldInfo.DocValuesType.NUMERIC);
|
||||
consumer = new NumericDocValuesWriter(fieldInfo, docState.docWriter.bytesUsed);
|
||||
consumer = new NumericDocValuesWriter(fieldInfo, docState.docWriter.bytesUsed, false);
|
||||
}
|
||||
consumer.addValue(docState.docID, similarity.computeNorm(fieldState));
|
||||
}
|
||||
|
|
|
@ -23,6 +23,8 @@ import java.util.NoSuchElementException;
|
|||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.OpenBitSet;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
|
@ -35,14 +37,18 @@ class NumericDocValuesWriter extends DocValuesWriter {
|
|||
private AppendingDeltaPackedLongBuffer pending;
|
||||
private final Counter iwBytesUsed;
|
||||
private long bytesUsed;
|
||||
private final OpenBitSet docsWithField;
|
||||
private final FieldInfo fieldInfo;
|
||||
private final boolean trackDocsWithField;
|
||||
|
||||
public NumericDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
|
||||
public NumericDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed, boolean trackDocsWithField) {
|
||||
pending = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
|
||||
bytesUsed = pending.ramBytesUsed();
|
||||
docsWithField = new OpenBitSet();
|
||||
bytesUsed = pending.ramBytesUsed() + docsWithFieldBytesUsed();
|
||||
this.fieldInfo = fieldInfo;
|
||||
this.iwBytesUsed = iwBytesUsed;
|
||||
iwBytesUsed.addAndGet(bytesUsed);
|
||||
this.trackDocsWithField = trackDocsWithField;
|
||||
}
|
||||
|
||||
public void addValue(int docID, long value) {
|
||||
|
@ -56,12 +62,20 @@ class NumericDocValuesWriter extends DocValuesWriter {
|
|||
}
|
||||
|
||||
pending.add(value);
|
||||
if (trackDocsWithField) {
|
||||
docsWithField.set(docID);
|
||||
}
|
||||
|
||||
updateBytesUsed();
|
||||
}
|
||||
|
||||
private long docsWithFieldBytesUsed() {
|
||||
// nocommit: this is not correct
|
||||
return docsWithField.getBits().length*RamUsageEstimator.NUM_BYTES_LONG;
|
||||
}
|
||||
|
||||
private void updateBytesUsed() {
|
||||
final long newBytesUsed = pending.ramBytesUsed();
|
||||
final long newBytesUsed = pending.ramBytesUsed() + docsWithFieldBytesUsed();
|
||||
iwBytesUsed.addAndGet(newBytesUsed - bytesUsed);
|
||||
bytesUsed = newBytesUsed;
|
||||
}
|
||||
|
@ -109,14 +123,18 @@ class NumericDocValuesWriter extends DocValuesWriter {
|
|||
if (!hasNext()) {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
long value;
|
||||
Long value;
|
||||
if (upto < size) {
|
||||
value = iter.next();
|
||||
long v = iter.next();
|
||||
if (!trackDocsWithField || docsWithField.get(upto)) {
|
||||
value = v;
|
||||
} else {
|
||||
value = null;
|
||||
}
|
||||
} else {
|
||||
value = 0;
|
||||
value = trackDocsWithField ? null : MISSING;
|
||||
}
|
||||
upto++;
|
||||
// TODO: make reusable Number
|
||||
return value;
|
||||
}
|
||||
|
||||
|
|
|
@ -285,6 +285,13 @@ public class ParallelAtomicReader extends AtomicReader {
|
|||
return reader == null ? null : reader.getSortedSetDocValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
ensureOpen();
|
||||
AtomicReader reader = fieldToReader.get(field);
|
||||
return reader == null ? null : reader.getDocsWithField(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NumericDocValues getNormValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.lucene.index.SegmentReader.CoreClosedListener;
|
|||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.CloseableThreadLocal;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
|
@ -87,6 +88,13 @@ final class SegmentCoreReaders {
|
|||
return new HashMap<String,Object>();
|
||||
}
|
||||
};
|
||||
|
||||
final CloseableThreadLocal<Map<String,Bits>> docsWithFieldLocal = new CloseableThreadLocal<Map<String,Bits>>() {
|
||||
@Override
|
||||
protected Map<String,Bits> initialValue() {
|
||||
return new HashMap<String,Bits>();
|
||||
}
|
||||
};
|
||||
|
||||
final CloseableThreadLocal<Map<String,Object>> normsLocal = new CloseableThreadLocal<Map<String,Object>>() {
|
||||
@Override
|
||||
|
@ -274,6 +282,30 @@ final class SegmentCoreReaders {
|
|||
|
||||
return dvs;
|
||||
}
|
||||
|
||||
Bits getDocsWithField(String field) throws IOException {
|
||||
FieldInfo fi = fieldInfos.fieldInfo(field);
|
||||
if (fi == null) {
|
||||
// Field does not exist
|
||||
return null;
|
||||
}
|
||||
if (fi.getDocValuesType() == null) {
|
||||
// Field was not indexed with doc values
|
||||
return null;
|
||||
}
|
||||
|
||||
assert dvProducer != null;
|
||||
|
||||
Map<String,Bits> dvFields = docsWithFieldLocal.get();
|
||||
|
||||
Bits dvs = dvFields.get(field);
|
||||
if (dvs == null) {
|
||||
dvs = dvProducer.getDocsWithField(fi);
|
||||
dvFields.put(field, dvs);
|
||||
}
|
||||
|
||||
return dvs;
|
||||
}
|
||||
|
||||
NumericDocValues getNormValues(String field) throws IOException {
|
||||
FieldInfo fi = fieldInfos.fieldInfo(field);
|
||||
|
@ -300,8 +332,8 @@ final class SegmentCoreReaders {
|
|||
|
||||
void decRef() throws IOException {
|
||||
if (ref.decrementAndGet() == 0) {
|
||||
IOUtils.close(termVectorsLocal, fieldsReaderLocal, docValuesLocal, normsLocal, fields, dvProducer,
|
||||
termVectorsReaderOrig, fieldsReaderOrig, cfsReader, normsProducer);
|
||||
IOUtils.close(termVectorsLocal, fieldsReaderLocal, docValuesLocal, normsLocal, docsWithFieldLocal, fields,
|
||||
dvProducer, termVectorsReaderOrig, fieldsReaderOrig, cfsReader, normsProducer);
|
||||
notifyCoreClosedListeners();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.lucene.codecs.TermVectorsWriter;
|
|||
import org.apache.lucene.index.FieldInfo.DocValuesType;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
|
@ -156,24 +157,32 @@ final class SegmentMerger {
|
|||
if (type != null) {
|
||||
if (type == DocValuesType.NUMERIC) {
|
||||
List<NumericDocValues> toMerge = new ArrayList<NumericDocValues>();
|
||||
List<Bits> docsWithField = new ArrayList<Bits>();
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
NumericDocValues values = reader.getNumericDocValues(field.name);
|
||||
Bits bits = reader.getDocsWithField(field.name);
|
||||
if (values == null) {
|
||||
values = NumericDocValues.EMPTY;
|
||||
bits = new Bits.MatchNoBits(reader.maxDoc());
|
||||
}
|
||||
toMerge.add(values);
|
||||
docsWithField.add(bits);
|
||||
}
|
||||
consumer.mergeNumericField(field, mergeState, toMerge);
|
||||
consumer.mergeNumericField(field, mergeState, toMerge, docsWithField);
|
||||
} else if (type == DocValuesType.BINARY) {
|
||||
List<BinaryDocValues> toMerge = new ArrayList<BinaryDocValues>();
|
||||
List<Bits> docsWithField = new ArrayList<Bits>();
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
BinaryDocValues values = reader.getBinaryDocValues(field.name);
|
||||
Bits bits = reader.getDocsWithField(field.name);
|
||||
if (values == null) {
|
||||
values = BinaryDocValues.EMPTY;
|
||||
bits = new Bits.MatchNoBits(reader.maxDoc());
|
||||
}
|
||||
toMerge.add(values);
|
||||
docsWithField.add(bits);
|
||||
}
|
||||
consumer.mergeBinaryField(field, mergeState, toMerge);
|
||||
consumer.mergeBinaryField(field, mergeState, toMerge, docsWithField);
|
||||
} else if (type == DocValuesType.SORTED) {
|
||||
List<SortedDocValues> toMerge = new ArrayList<SortedDocValues>();
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
|
@ -216,14 +225,16 @@ final class SegmentMerger {
|
|||
for (FieldInfo field : mergeState.fieldInfos) {
|
||||
if (field.hasNorms()) {
|
||||
List<NumericDocValues> toMerge = new ArrayList<NumericDocValues>();
|
||||
List<Bits> docsWithField = new ArrayList<Bits>();
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
NumericDocValues norms = reader.getNormValues(field.name);
|
||||
if (norms == null) {
|
||||
norms = NumericDocValues.EMPTY;
|
||||
}
|
||||
toMerge.add(norms);
|
||||
docsWithField.add(new Bits.MatchAllBits(reader.maxDoc()));
|
||||
}
|
||||
consumer.mergeNumericField(field, mergeState, toMerge);
|
||||
consumer.mergeNumericField(field, mergeState, toMerge, docsWithField);
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
|
|
|
@ -223,6 +223,12 @@ public final class SegmentReader extends AtomicReader {
|
|||
return core.getNumericDocValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
ensureOpen();
|
||||
return core.getDocsWithField(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BinaryDocValues getBinaryDocValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
|
|
|
@ -91,6 +91,12 @@ public final class SlowCompositeReaderWrapper extends AtomicReader {
|
|||
return MultiDocValues.getNumericValues(in, field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
ensureOpen();
|
||||
return MultiDocValues.getDocsWithField(in, field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BinaryDocValues getBinaryDocValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
|
|
|
@ -37,7 +37,8 @@ public abstract class SortedDocValues extends BinaryDocValues {
|
|||
* Returns the ordinal for the specified docID.
|
||||
* @param docID document ID to lookup
|
||||
* @return ordinal for the document: this is dense, starts at 0, then
|
||||
* increments by 1 for the next value in sorted order.
|
||||
* increments by 1 for the next value in sorted order. Note that
|
||||
* missing values are indicated by -1.
|
||||
*/
|
||||
public abstract int getOrd(int docID);
|
||||
|
||||
|
@ -71,7 +72,7 @@ public abstract class SortedDocValues extends BinaryDocValues {
|
|||
public static final SortedDocValues EMPTY = new SortedDocValues() {
|
||||
@Override
|
||||
public int getOrd(int docID) {
|
||||
return 0;
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -83,7 +84,7 @@ public abstract class SortedDocValues extends BinaryDocValues {
|
|||
|
||||
@Override
|
||||
public int getValueCount() {
|
||||
return 1;
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
|
||||
|
|
|
@ -30,19 +30,19 @@ import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
|
|||
import org.apache.lucene.util.BytesRefHash;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
|
||||
import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/** Buffers up pending byte[] per doc, deref and sorting via
|
||||
* int ord, then flushes when segment flushes. */
|
||||
class SortedDocValuesWriter extends DocValuesWriter {
|
||||
final BytesRefHash hash;
|
||||
private AppendingPackedLongBuffer pending;
|
||||
private AppendingDeltaPackedLongBuffer pending;
|
||||
private final Counter iwBytesUsed;
|
||||
private long bytesUsed; // this currently only tracks differences in 'pending'
|
||||
private final FieldInfo fieldInfo;
|
||||
|
||||
private static final BytesRef EMPTY = new BytesRef(BytesRef.EMPTY_BYTES);
|
||||
private static final int EMPTY_ORD = -1;
|
||||
|
||||
public SortedDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
|
||||
this.fieldInfo = fieldInfo;
|
||||
|
@ -52,7 +52,7 @@ class SortedDocValuesWriter extends DocValuesWriter {
|
|||
new ByteBlockPool.DirectTrackingAllocator(iwBytesUsed)),
|
||||
BytesRefHash.DEFAULT_CAPACITY,
|
||||
new DirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, iwBytesUsed));
|
||||
pending = new AppendingPackedLongBuffer(PackedInts.COMPACT);
|
||||
pending = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
|
||||
bytesUsed = pending.ramBytesUsed();
|
||||
iwBytesUsed.addAndGet(bytesUsed);
|
||||
}
|
||||
|
@ -70,7 +70,7 @@ class SortedDocValuesWriter extends DocValuesWriter {
|
|||
|
||||
// Fill in any holes:
|
||||
while(pending.size() < docID) {
|
||||
addOneValue(EMPTY);
|
||||
pending.add(EMPTY_ORD);
|
||||
}
|
||||
|
||||
addOneValue(value);
|
||||
|
@ -79,8 +79,9 @@ class SortedDocValuesWriter extends DocValuesWriter {
|
|||
@Override
|
||||
public void finish(int maxDoc) {
|
||||
while(pending.size() < maxDoc) {
|
||||
addOneValue(EMPTY);
|
||||
pending.add(EMPTY_ORD);
|
||||
}
|
||||
updateBytesUsed();
|
||||
}
|
||||
|
||||
private void addOneValue(BytesRef value) {
|
||||
|
@ -177,7 +178,7 @@ class SortedDocValuesWriter extends DocValuesWriter {
|
|||
|
||||
// iterates over the ords for each doc we have in ram
|
||||
private class OrdsIterator implements Iterator<Number> {
|
||||
final AppendingPackedLongBuffer.Iterator iter = pending.iterator();
|
||||
final AppendingDeltaPackedLongBuffer.Iterator iter = pending.iterator();
|
||||
final int ordMap[];
|
||||
final int maxDoc;
|
||||
int docUpto;
|
||||
|
@ -200,8 +201,7 @@ class SortedDocValuesWriter extends DocValuesWriter {
|
|||
}
|
||||
int ord = (int) iter.next();
|
||||
docUpto++;
|
||||
// TODO: make reusable Number
|
||||
return ordMap[ord];
|
||||
return ord == -1 ? ord : ordMap[ord];
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -104,26 +104,6 @@ public interface FieldCache {
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
/** Returns MISSING/-1 ordinal for every document */
|
||||
public static final SortedDocValues EMPTY_TERMSINDEX = new SortedDocValues() {
|
||||
@Override
|
||||
public int getOrd(int docID) {
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void lookupOrd(int ord, BytesRef result) {
|
||||
result.bytes = MISSING;
|
||||
result.offset = 0;
|
||||
result.length = 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCount() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* Placeholder indicating creation of this cache is currently in-progress.
|
||||
|
@ -266,13 +246,10 @@ public interface FieldCache {
|
|||
}
|
||||
};
|
||||
|
||||
|
||||
/** Checks the internal cache for an appropriate entry, and if none is found,
|
||||
* reads the terms in <code>field</code> and returns a bit set at the size of
|
||||
* <code>reader.maxDoc()</code>, with turned on bits for each docid that
|
||||
* does have a value for this field. Note that if the field was only indexed
|
||||
* as DocValues then this method will not work (it will return a Bits stating
|
||||
* that no documents contain the field).
|
||||
* does have a value for this field.
|
||||
*/
|
||||
public Bits getDocsWithField(AtomicReader reader, String field) throws IOException;
|
||||
|
||||
|
|
|
@ -501,8 +501,7 @@ class FieldCacheImpl implements FieldCache {
|
|||
// field does not exist or has no value
|
||||
return new Bits.MatchNoBits(reader.maxDoc());
|
||||
} else if (fieldInfo.hasDocValues()) {
|
||||
// doc values are dense
|
||||
return new Bits.MatchAllBits(reader.maxDoc());
|
||||
return reader.getDocsWithField(field);
|
||||
} else if (!fieldInfo.isIndexed()) {
|
||||
return new Bits.MatchNoBits(reader.maxDoc());
|
||||
}
|
||||
|
@ -944,13 +943,13 @@ class FieldCacheImpl implements FieldCache {
|
|||
} else {
|
||||
final FieldInfo info = reader.getFieldInfos().fieldInfo(field);
|
||||
if (info == null) {
|
||||
return EMPTY_TERMSINDEX;
|
||||
return SortedDocValues.EMPTY;
|
||||
} else if (info.hasDocValues()) {
|
||||
// we don't try to build a sorted instance from numeric/binary doc
|
||||
// values because dedup can be very costly
|
||||
throw new IllegalStateException("Type mismatch: " + field + " was indexed as " + info.getDocValuesType());
|
||||
} else if (!info.isIndexed()) {
|
||||
return EMPTY_TERMSINDEX;
|
||||
return SortedDocValues.EMPTY;
|
||||
}
|
||||
return (SortedDocValues) caches.get(SortedDocValues.class).get(reader, new CacheKey(field, acceptableOverheadRatio), false);
|
||||
}
|
||||
|
|
|
@ -569,7 +569,7 @@ public class TestFieldCache extends LuceneTestCase {
|
|||
assertEquals(2, sortedSet.getValueCount());
|
||||
|
||||
bits = FieldCache.DEFAULT.getDocsWithField(ar, "sortedset");
|
||||
assertTrue(bits instanceof Bits.MatchAllBits);
|
||||
assertTrue(bits.get(0));
|
||||
}
|
||||
|
||||
ir.close();
|
||||
|
|
|
@ -31,15 +31,18 @@ import org.apache.lucene.index.SegmentReadState;
|
|||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
class Facet42DocValuesProducer extends DocValuesProducer {
|
||||
|
||||
private final Map<Integer,Facet42BinaryDocValues> fields = new HashMap<Integer,Facet42BinaryDocValues>();
|
||||
private final int maxDoc;
|
||||
|
||||
Facet42DocValuesProducer(SegmentReadState state) throws IOException {
|
||||
String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Facet42DocValuesFormat.EXTENSION);
|
||||
IndexInput in = state.directory.openInput(fileName, state.context);
|
||||
this.maxDoc = state.segmentInfo.getDocCount();
|
||||
boolean success = false;
|
||||
try {
|
||||
CodecUtil.checkHeader(in, Facet42DocValuesFormat.CODEC,
|
||||
|
@ -80,6 +83,11 @@ class Facet42DocValuesProducer extends DocValuesProducer {
|
|||
throw new UnsupportedOperationException("FacetsDocValues only implements binary");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
return new Bits.MatchAllBits(maxDoc); // TODO: have codec impl this?
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
}
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.lucene.index.Terms;
|
|||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
|
@ -222,6 +223,18 @@ public class FacetsPayloadMigrationReader extends FilterAtomicReader {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
Term term = fieldTerms.get(field);
|
||||
if (term == null) {
|
||||
return super.getDocsWithField(field);
|
||||
} else {
|
||||
// we shouldn't return null, even if the term does not exist or has no
|
||||
// payloads, since we already marked the field as having DocValues.
|
||||
return new Bits.MatchAllBits(maxDoc());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldInfos getFieldInfos() {
|
||||
FieldInfos innerInfos = super.getFieldInfos();
|
||||
|
|
|
@ -422,6 +422,11 @@ public class WeightedSpanTermExtractor {
|
|||
public NumericDocValues getNormValues(String field) throws IOException {
|
||||
return super.getNormValues(FIELD_NAME);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
return super.getDocsWithField(FIELD_NAME);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -756,6 +756,11 @@ public class MemoryIndex {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
private class MemoryFields extends Fields {
|
||||
@Override
|
||||
public Iterator<String> iterator() {
|
||||
|
|
|
@ -221,6 +221,27 @@ public class SortingAtomicReader extends FilterAtomicReader {
|
|||
}
|
||||
}
|
||||
|
||||
private static class SortingBits implements Bits {
|
||||
|
||||
private final Bits in;
|
||||
private final Sorter.DocMap docMap;
|
||||
|
||||
public SortingBits(final Bits in, Sorter.DocMap docMap) {
|
||||
this.in = in;
|
||||
this.docMap = docMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean get(int index) {
|
||||
return in.get(docMap.newToOld(index));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return in.length();
|
||||
}
|
||||
}
|
||||
|
||||
private static class SortingSortedDocValues extends SortedDocValues {
|
||||
|
||||
private final SortedDocValues in;
|
||||
|
@ -743,20 +764,9 @@ public class SortingAtomicReader extends FilterAtomicReader {
|
|||
final Bits inLiveDocs = in.getLiveDocs();
|
||||
if (inLiveDocs == null) {
|
||||
return null;
|
||||
} else {
|
||||
return new SortingBits(inLiveDocs, docMap);
|
||||
}
|
||||
return new Bits() {
|
||||
|
||||
@Override
|
||||
public boolean get(int index) {
|
||||
return inLiveDocs.get(docMap.newToOld(index));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return inLiveDocs.length();
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -796,6 +806,16 @@ public class SortingAtomicReader extends FilterAtomicReader {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
Bits bits = in.getDocsWithField(field);
|
||||
if (bits == null || bits instanceof Bits.MatchAllBits || bits instanceof Bits.MatchNoBits) {
|
||||
return bits;
|
||||
} else {
|
||||
return new SortingBits(bits, docMap);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields getTermVectors(final int docID) throws IOException {
|
||||
return in.getTermVectors(docMap.newToOld(docID));
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.lucene.index.SegmentReadState;
|
|||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.OpenBitSet;
|
||||
|
@ -75,11 +76,10 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
|
||||
int count = 0;
|
||||
for (Number v : values) {
|
||||
assert v != null;
|
||||
count++;
|
||||
}
|
||||
assert count == maxDoc;
|
||||
checkIterator(values.iterator(), maxDoc);
|
||||
checkIterator(values.iterator(), maxDoc, true);
|
||||
in.addNumericField(field, values);
|
||||
}
|
||||
|
||||
|
@ -87,12 +87,11 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
|
||||
int count = 0;
|
||||
for (BytesRef b : values) {
|
||||
assert b != null;
|
||||
assert b.isValid();
|
||||
assert b == null || b.isValid();
|
||||
count++;
|
||||
}
|
||||
assert count == maxDoc;
|
||||
checkIterator(values.iterator(), maxDoc);
|
||||
checkIterator(values.iterator(), maxDoc, true);
|
||||
in.addBinaryField(field, values);
|
||||
}
|
||||
|
||||
|
@ -117,15 +116,17 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
for (Number v : docToOrd) {
|
||||
assert v != null;
|
||||
int ord = v.intValue();
|
||||
assert ord >= 0 && ord < valueCount;
|
||||
seenOrds.set(ord);
|
||||
assert ord >= -1 && ord < valueCount;
|
||||
if (ord >= 0) {
|
||||
seenOrds.set(ord);
|
||||
}
|
||||
count++;
|
||||
}
|
||||
|
||||
assert count == maxDoc;
|
||||
assert seenOrds.cardinality() == valueCount;
|
||||
checkIterator(values.iterator(), valueCount);
|
||||
checkIterator(docToOrd.iterator(), maxDoc);
|
||||
checkIterator(values.iterator(), valueCount, false);
|
||||
checkIterator(docToOrd.iterator(), maxDoc, false);
|
||||
in.addSortedField(field, values, docToOrd);
|
||||
}
|
||||
|
||||
|
@ -169,18 +170,18 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
|
||||
assert docCount == maxDoc;
|
||||
assert seenOrds.cardinality() == valueCount;
|
||||
checkIterator(values.iterator(), valueCount);
|
||||
checkIterator(docToOrdCount.iterator(), maxDoc);
|
||||
checkIterator(ords.iterator(), ordCount);
|
||||
checkIterator(values.iterator(), valueCount, false);
|
||||
checkIterator(docToOrdCount.iterator(), maxDoc, false);
|
||||
checkIterator(ords.iterator(), ordCount, false);
|
||||
in.addSortedSetField(field, values, docToOrdCount, ords);
|
||||
}
|
||||
|
||||
private <T> void checkIterator(Iterator<T> iterator, long expectedSize) {
|
||||
private <T> void checkIterator(Iterator<T> iterator, long expectedSize, boolean allowNull) {
|
||||
for (long i = 0; i < expectedSize; i++) {
|
||||
boolean hasNext = iterator.hasNext();
|
||||
assert hasNext;
|
||||
T v = iterator.next();
|
||||
assert v != null;
|
||||
assert allowNull || v != null;
|
||||
try {
|
||||
iterator.remove();
|
||||
throw new AssertionError("broken iterator (supports remove): " + iterator);
|
||||
|
@ -244,6 +245,15 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
assert values != null;
|
||||
return new AssertingAtomicReader.AssertingSortedSetDocValues(values, maxDoc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
assert field.getDocValuesType() != null;
|
||||
Bits bits = in.getDocsWithField(field);
|
||||
assert bits != null;
|
||||
assert bits.length() == maxDoc;
|
||||
return bits; // TODO: add AssertingBits w/ bounds check
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
|
|
@ -27,6 +27,7 @@ import java.util.Map;
|
|||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.DocValuesProducer.SortedSetDocsWithField;
|
||||
import org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer;
|
||||
import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
|
@ -38,6 +39,7 @@ import org.apache.lucene.index.SegmentReadState;
|
|||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.packed.BlockPackedReader;
|
||||
|
@ -50,9 +52,11 @@ class CheapBastardDocValuesProducer extends DocValuesProducer {
|
|||
private final Map<Integer,NumericEntry> ordIndexes;
|
||||
private final Map<Integer,BinaryEntry> binaries;
|
||||
private final IndexInput data;
|
||||
private final int maxDoc;
|
||||
|
||||
CheapBastardDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
this.maxDoc = state.segmentInfo.getDocCount();
|
||||
// read in the entries from the metadata file.
|
||||
IndexInput in = state.directory.openInput(metaName, state.context);
|
||||
boolean success = false;
|
||||
|
@ -380,6 +384,15 @@ class CheapBastardDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(FieldInfo field) throws IOException {
|
||||
if (field.getDocValuesType() == FieldInfo.DocValuesType.SORTED_SET) {
|
||||
return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
|
||||
} else {
|
||||
return new Bits.MatchAllBits(maxDoc);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
|
|
@ -24,7 +24,9 @@ import java.util.TreeSet;
|
|||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.MissingOrdRemapper;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosReader.LegacyDocValuesType;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
@ -54,7 +56,7 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
long minValue = Long.MAX_VALUE;
|
||||
long maxValue = Long.MIN_VALUE;
|
||||
for (Number n : values) {
|
||||
long v = n.longValue();
|
||||
long v = n == null ? 0 : n.longValue();
|
||||
minValue = Math.min(minValue, v);
|
||||
maxValue = Math.max(maxValue, v);
|
||||
}
|
||||
|
@ -92,7 +94,7 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
Lucene40DocValuesFormat.INTS_VERSION_CURRENT);
|
||||
output.writeInt(1); // size
|
||||
for (Number n : values) {
|
||||
output.writeByte(n.byteValue());
|
||||
output.writeByte(n == null ? 0 : n.byteValue());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -103,7 +105,7 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
Lucene40DocValuesFormat.INTS_VERSION_CURRENT);
|
||||
output.writeInt(2); // size
|
||||
for (Number n : values) {
|
||||
output.writeShort(n.shortValue());
|
||||
output.writeShort(n == null ? 0 : n.shortValue());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -114,7 +116,7 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
Lucene40DocValuesFormat.INTS_VERSION_CURRENT);
|
||||
output.writeInt(4); // size
|
||||
for (Number n : values) {
|
||||
output.writeInt(n.intValue());
|
||||
output.writeInt(n == null ? 0 : n.intValue());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -131,7 +133,7 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
// writes longs
|
||||
output.writeByte(Lucene40DocValuesFormat.VAR_INTS_FIXED_64);
|
||||
for (Number n : values) {
|
||||
output.writeLong(n.longValue());
|
||||
output.writeLong(n == null ? 0 : n.longValue());
|
||||
}
|
||||
} else {
|
||||
// writes packed ints
|
||||
|
@ -143,7 +145,8 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
PackedInts.bitsRequired(delta),
|
||||
PackedInts.DEFAULT);
|
||||
for (Number n : values) {
|
||||
writer.add(n.longValue() - minValue);
|
||||
long v = n == null ? 0 : n.longValue();
|
||||
writer.add(v - minValue);
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
|
@ -156,6 +159,9 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
int minLength = Integer.MAX_VALUE;
|
||||
int maxLength = Integer.MIN_VALUE;
|
||||
for (BytesRef b : values) {
|
||||
if (b == null) {
|
||||
b = new BytesRef(); // 4.0 doesnt distinguish
|
||||
}
|
||||
minLength = Math.min(minLength, b.length);
|
||||
maxLength = Math.max(maxLength, b.length);
|
||||
if (uniqueValues != null) {
|
||||
|
@ -243,7 +249,9 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
|
||||
output.writeInt(length);
|
||||
for (BytesRef v : values) {
|
||||
output.writeBytes(v.bytes, v.offset, v.length);
|
||||
if (v != null) {
|
||||
output.writeBytes(v.bytes, v.offset, v.length);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -264,7 +272,9 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
final long startPos = data.getFilePointer();
|
||||
|
||||
for (BytesRef v : values) {
|
||||
data.writeBytes(v.bytes, v.offset, v.length);
|
||||
if (v != null) {
|
||||
data.writeBytes(v.bytes, v.offset, v.length);
|
||||
}
|
||||
}
|
||||
|
||||
/* addresses */
|
||||
|
@ -279,7 +289,9 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
long currentPosition = 0;
|
||||
for (BytesRef v : values) {
|
||||
w.add(currentPosition);
|
||||
currentPosition += v.length;
|
||||
if (v != null) {
|
||||
currentPosition += v.length;
|
||||
}
|
||||
}
|
||||
// write sentinel
|
||||
assert currentPosition == maxAddress;
|
||||
|
@ -301,7 +313,7 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
// deduplicate
|
||||
TreeSet<BytesRef> dictionary = new TreeSet<BytesRef>();
|
||||
for (BytesRef v : values) {
|
||||
dictionary.add(BytesRef.deepCopyOf(v));
|
||||
dictionary.add(v == null ? new BytesRef() : BytesRef.deepCopyOf(v));
|
||||
}
|
||||
|
||||
/* values */
|
||||
|
@ -318,6 +330,9 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
final PackedInts.Writer w = PackedInts.getWriter(index, maxDoc, PackedInts.bitsRequired(valueCount-1), PackedInts.DEFAULT);
|
||||
|
||||
for (BytesRef v : values) {
|
||||
if (v == null) {
|
||||
v = new BytesRef();
|
||||
}
|
||||
int ord = dictionary.headSet(v).size();
|
||||
w.add(ord);
|
||||
}
|
||||
|
@ -338,7 +353,7 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
// deduplicate
|
||||
TreeSet<BytesRef> dictionary = new TreeSet<BytesRef>();
|
||||
for (BytesRef v : values) {
|
||||
dictionary.add(BytesRef.deepCopyOf(v));
|
||||
dictionary.add(v == null ? new BytesRef() : BytesRef.deepCopyOf(v));
|
||||
}
|
||||
|
||||
/* values */
|
||||
|
@ -359,7 +374,7 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
final PackedInts.Writer w = PackedInts.getWriter(index, maxDoc, PackedInts.bitsRequired(currentAddress), PackedInts.DEFAULT);
|
||||
|
||||
for (BytesRef v : values) {
|
||||
w.add(valueToAddress.get(v));
|
||||
w.add(valueToAddress.get(v == null ? new BytesRef() : v));
|
||||
}
|
||||
w.finish();
|
||||
}
|
||||
|
@ -385,6 +400,15 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
maxLength = Math.max(maxLength, b.length);
|
||||
}
|
||||
|
||||
// but dont use fixed if there are missing values (we are simulating how lucene40 wrote dv...)
|
||||
boolean anyMissing = false;
|
||||
for (Number n : docToOrd) {
|
||||
if (n.longValue() == -1) {
|
||||
anyMissing = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
boolean success = false;
|
||||
IndexOutput data = null;
|
||||
IndexOutput index = null;
|
||||
|
@ -394,12 +418,22 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
|
|||
try {
|
||||
data = dir.createOutput(dataName, state.context);
|
||||
index = dir.createOutput(indexName, state.context);
|
||||
if (minLength == maxLength) {
|
||||
if (minLength == maxLength && !anyMissing) {
|
||||
// fixed byte[]
|
||||
addFixedSortedBytesField(field, data, index, values, docToOrd, minLength);
|
||||
} else {
|
||||
// var byte[]
|
||||
addVarSortedBytesField(field, data, index, values, docToOrd);
|
||||
// three cases for simulating the old writer:
|
||||
// 1. no missing
|
||||
// 2. missing (and empty string in use): remap ord=-1 -> ord=0
|
||||
// 3. missing (and empty string not in use): remap all ords +1, insert empty string into values
|
||||
if (!anyMissing) {
|
||||
addVarSortedBytesField(field, data, index, values, docToOrd);
|
||||
} else if (minLength == 0) {
|
||||
addVarSortedBytesField(field, data, index, values, MissingOrdRemapper.mapMissingToOrd0(docToOrd));
|
||||
} else {
|
||||
addVarSortedBytesField(field, data, index, MissingOrdRemapper.insertEmptyValue(values), MissingOrdRemapper.mapAllOrds(docToOrd));
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
|
|
|
@ -52,6 +52,7 @@ import org.apache.lucene.search.ScoreDoc;
|
|||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefHash;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
@ -1073,8 +1074,10 @@ public abstract class BaseDocValuesFormatTestCase extends LuceneTestCase {
|
|||
doc.add(newTextField("id", "noValue", Field.Store.YES));
|
||||
w.addDocument(doc);
|
||||
}
|
||||
BytesRef bytesRef = new BytesRef();
|
||||
hash.add(bytesRef); // add empty value for the gaps
|
||||
if (!codecSupportsDocsWithField("field")) {
|
||||
BytesRef bytesRef = new BytesRef();
|
||||
hash.add(bytesRef); // add empty value for the gaps
|
||||
}
|
||||
if (rarely()) {
|
||||
w.commit();
|
||||
}
|
||||
|
@ -2197,5 +2200,205 @@ public abstract class BaseDocValuesFormatTestCase extends LuceneTestCase {
|
|||
doTestNumericsVsStoredFields(longs);
|
||||
}
|
||||
}
|
||||
|
||||
public void testTwoNumbersOneMissing() throws IOException {
|
||||
assumeTrue("Codec does not support getDocsWithField", codecSupportsDocsWithField("dv1"));
|
||||
Directory directory = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
|
||||
conf.setMergePolicy(newLogMergePolicy());
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", "0", Field.Store.YES));
|
||||
doc.add(new NumericDocValuesField("dv1", 0));
|
||||
iw.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new StringField("id", "1", Field.Store.YES));
|
||||
iw.addDocument(doc);
|
||||
iw.forceMerge(1);
|
||||
iw.close();
|
||||
|
||||
IndexReader ir = DirectoryReader.open(directory);
|
||||
assertEquals(1, ir.leaves().size());
|
||||
AtomicReader ar = ir.leaves().get(0).reader();
|
||||
NumericDocValues dv = ar.getNumericDocValues("dv1");
|
||||
assertEquals(0, dv.get(0));
|
||||
assertEquals(0, dv.get(1));
|
||||
Bits docsWithField = ar.getDocsWithField("dv1");
|
||||
assertTrue(docsWithField.get(0));
|
||||
assertFalse(docsWithField.get(1));
|
||||
ir.close();
|
||||
directory.close();
|
||||
}
|
||||
|
||||
public void testTwoNumbersOneMissingWithMerging() throws IOException {
|
||||
assumeTrue("Codec does not support getDocsWithField", codecSupportsDocsWithField("dv1"));
|
||||
Directory directory = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
|
||||
conf.setMergePolicy(newLogMergePolicy());
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", "0", Field.Store.YES));
|
||||
doc.add(new NumericDocValuesField("dv1", 0));
|
||||
iw.addDocument(doc);
|
||||
iw.commit();
|
||||
doc = new Document();
|
||||
doc.add(new StringField("id", "1", Field.Store.YES));
|
||||
iw.addDocument(doc);
|
||||
iw.forceMerge(1);
|
||||
iw.close();
|
||||
|
||||
IndexReader ir = DirectoryReader.open(directory);
|
||||
assertEquals(1, ir.leaves().size());
|
||||
AtomicReader ar = ir.leaves().get(0).reader();
|
||||
NumericDocValues dv = ar.getNumericDocValues("dv1");
|
||||
assertEquals(0, dv.get(0));
|
||||
assertEquals(0, dv.get(1));
|
||||
Bits docsWithField = ar.getDocsWithField("dv1");
|
||||
assertTrue(docsWithField.get(0));
|
||||
assertFalse(docsWithField.get(1));
|
||||
ir.close();
|
||||
directory.close();
|
||||
}
|
||||
|
||||
public void testThreeNumbersOneMissingWithMerging() throws IOException {
|
||||
assumeTrue("Codec does not support getDocsWithField", codecSupportsDocsWithField("dv1"));
|
||||
Directory directory = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
|
||||
conf.setMergePolicy(newLogMergePolicy());
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", "0", Field.Store.YES));
|
||||
doc.add(new NumericDocValuesField("dv1", 0));
|
||||
iw.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new StringField("id", "1", Field.Store.YES));
|
||||
iw.addDocument(doc);
|
||||
iw.commit();
|
||||
doc = new Document();
|
||||
doc.add(new StringField("id", "2", Field.Store.YES));
|
||||
doc.add(new NumericDocValuesField("dv1", 5));
|
||||
iw.addDocument(doc);
|
||||
iw.forceMerge(1);
|
||||
iw.close();
|
||||
|
||||
IndexReader ir = DirectoryReader.open(directory);
|
||||
assertEquals(1, ir.leaves().size());
|
||||
AtomicReader ar = ir.leaves().get(0).reader();
|
||||
NumericDocValues dv = ar.getNumericDocValues("dv1");
|
||||
assertEquals(0, dv.get(0));
|
||||
assertEquals(0, dv.get(1));
|
||||
assertEquals(5, dv.get(2));
|
||||
Bits docsWithField = ar.getDocsWithField("dv1");
|
||||
assertTrue(docsWithField.get(0));
|
||||
assertFalse(docsWithField.get(1));
|
||||
assertTrue(docsWithField.get(2));
|
||||
ir.close();
|
||||
directory.close();
|
||||
}
|
||||
|
||||
public void testTwoBytesOneMissing() throws IOException {
|
||||
assumeTrue("Codec does not support getDocsWithField", codecSupportsDocsWithField("dv1"));
|
||||
Directory directory = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
|
||||
conf.setMergePolicy(newLogMergePolicy());
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", "0", Field.Store.YES));
|
||||
doc.add(new BinaryDocValuesField("dv1", new BytesRef()));
|
||||
iw.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new StringField("id", "1", Field.Store.YES));
|
||||
iw.addDocument(doc);
|
||||
iw.forceMerge(1);
|
||||
iw.close();
|
||||
|
||||
IndexReader ir = DirectoryReader.open(directory);
|
||||
assertEquals(1, ir.leaves().size());
|
||||
AtomicReader ar = ir.leaves().get(0).reader();
|
||||
BinaryDocValues dv = ar.getBinaryDocValues("dv1");
|
||||
BytesRef ref = new BytesRef();
|
||||
dv.get(0, ref);
|
||||
assertEquals(new BytesRef(), ref);
|
||||
dv.get(1, ref);
|
||||
assertEquals(new BytesRef(), ref);
|
||||
Bits docsWithField = ar.getDocsWithField("dv1");
|
||||
assertTrue(docsWithField.get(0));
|
||||
assertFalse(docsWithField.get(1));
|
||||
ir.close();
|
||||
directory.close();
|
||||
}
|
||||
|
||||
public void testTwoBytesOneMissingWithMerging() throws IOException {
|
||||
assumeTrue("Codec does not support getDocsWithField", codecSupportsDocsWithField("dv1"));
|
||||
Directory directory = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
|
||||
conf.setMergePolicy(newLogMergePolicy());
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", "0", Field.Store.YES));
|
||||
doc.add(new BinaryDocValuesField("dv1", new BytesRef()));
|
||||
iw.addDocument(doc);
|
||||
iw.commit();
|
||||
doc = new Document();
|
||||
doc.add(new StringField("id", "1", Field.Store.YES));
|
||||
iw.addDocument(doc);
|
||||
iw.forceMerge(1);
|
||||
iw.close();
|
||||
|
||||
IndexReader ir = DirectoryReader.open(directory);
|
||||
assertEquals(1, ir.leaves().size());
|
||||
AtomicReader ar = ir.leaves().get(0).reader();
|
||||
BinaryDocValues dv = ar.getBinaryDocValues("dv1");
|
||||
BytesRef ref = new BytesRef();
|
||||
dv.get(0, ref);
|
||||
assertEquals(new BytesRef(), ref);
|
||||
dv.get(1, ref);
|
||||
assertEquals(new BytesRef(), ref);
|
||||
Bits docsWithField = ar.getDocsWithField("dv1");
|
||||
assertTrue(docsWithField.get(0));
|
||||
assertFalse(docsWithField.get(1));
|
||||
ir.close();
|
||||
directory.close();
|
||||
}
|
||||
|
||||
public void testThreeBytesOneMissingWithMerging() throws IOException {
|
||||
assumeTrue("Codec does not support getDocsWithField", codecSupportsDocsWithField("dv1"));
|
||||
Directory directory = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
|
||||
conf.setMergePolicy(newLogMergePolicy());
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", "0", Field.Store.YES));
|
||||
doc.add(new BinaryDocValuesField("dv1", new BytesRef()));
|
||||
iw.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new StringField("id", "1", Field.Store.YES));
|
||||
iw.addDocument(doc);
|
||||
iw.commit();
|
||||
doc = new Document();
|
||||
doc.add(new StringField("id", "2", Field.Store.YES));
|
||||
doc.add(new BinaryDocValuesField("dv1", new BytesRef("boo")));
|
||||
iw.addDocument(doc);
|
||||
iw.forceMerge(1);
|
||||
iw.close();
|
||||
|
||||
IndexReader ir = DirectoryReader.open(directory);
|
||||
assertEquals(1, ir.leaves().size());
|
||||
AtomicReader ar = ir.leaves().get(0).reader();
|
||||
BinaryDocValues dv = ar.getBinaryDocValues("dv1");
|
||||
BytesRef ref = new BytesRef();
|
||||
dv.get(0, ref);
|
||||
assertEquals(new BytesRef(), ref);
|
||||
dv.get(1, ref);
|
||||
assertEquals(new BytesRef(), ref);
|
||||
dv.get(2, ref);
|
||||
assertEquals(new BytesRef("boo"), ref);
|
||||
Bits docsWithField = ar.getDocsWithField("dv1");
|
||||
assertTrue(docsWithField.get(0));
|
||||
assertFalse(docsWithField.get(1));
|
||||
assertTrue(docsWithField.get(2));
|
||||
ir.close();
|
||||
directory.close();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.ArrayList;
|
|||
import java.util.Iterator;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.FilterIterator;
|
||||
|
||||
/**
|
||||
|
@ -136,6 +137,11 @@ public final class FieldFilterAtomicReader extends FilterAtomicReader {
|
|||
return hasField(field) ? super.getNormValues(field) : null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
return hasField(field) ? super.getDocsWithField(field) : null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("FieldFilterAtomicReader(reader=");
|
||||
|
|
|
@ -1368,6 +1368,13 @@ public abstract class LuceneTestCase extends Assert {
|
|||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/** Returns true if the codec for the field "supports" docsWithField
|
||||
* (other codecs return MatchAllBits, because you couldnt write missing values before) */
|
||||
public static boolean codecSupportsDocsWithField(String field) {
|
||||
// currently only one codec!
|
||||
return _TestUtil.getDocValuesFormat(Codec.getDefault(), field).equals("SimpleText");
|
||||
}
|
||||
|
||||
public void assertReaderEquals(String info, IndexReader leftReader, IndexReader rightReader) throws IOException {
|
||||
assertReaderStatisticsEquals(info, leftReader, rightReader);
|
||||
|
|
|
@ -45,6 +45,7 @@ import org.apache.lucene.codecs.Codec;
|
|||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
import org.apache.lucene.document.BinaryDocValuesField;
|
||||
import org.apache.lucene.document.Document;
|
||||
|
@ -742,6 +743,15 @@ public class _TestUtil {
|
|||
return p.getName();
|
||||
}
|
||||
}
|
||||
|
||||
public static String getDocValuesFormat(Codec codec, String field) {
|
||||
DocValuesFormat d = codec.docValuesFormat();
|
||||
if (d instanceof PerFieldDocValuesFormat) {
|
||||
return ((PerFieldDocValuesFormat)d).getDocValuesFormatForField(field).getName();
|
||||
} else {
|
||||
return d.getName();
|
||||
}
|
||||
}
|
||||
|
||||
public static boolean anyFilesExceptWriteLock(Directory dir) throws IOException {
|
||||
String[] files = dir.listAll();
|
||||
|
|
|
@ -221,6 +221,7 @@ public class DocValuesFacets {
|
|||
if (schemaField.multiValued()) {
|
||||
missingCount = SimpleFacets.getFieldMissingCount(searcher,docs,schemaField.getName());
|
||||
} else {
|
||||
// nocommit: support missing count (ord = -1) for single-valued here.
|
||||
missingCount = 0; // single-valued dv is implicitly 0
|
||||
}
|
||||
}
|
||||
|
|
|
@ -403,6 +403,11 @@ public class TestDocSet extends LuceneTestCase {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public NumericDocValues getNormValues(String field) {
|
||||
return null;
|
||||
|
|
Loading…
Reference in New Issue