LUCENE-8298: Allow DocValues updates to reset a value

Today once a document has a value in a certain DV field this values
can only be changed but not removed. While resetting / removing a value
from a field is certainly a corner case it can be used to undelete a
soft-deleted document unless it's merged away.
This allows to rollback changes without rolling back to another commitpoint
or trashing all uncommitted changes. In certain cenarios it can be used to
"repair" history of documents in distributed systems.
This commit is contained in:
Simon Willnauer 2018-05-07 11:52:51 +02:00
parent 7873cf845e
commit a3c86373e4
18 changed files with 522 additions and 92 deletions

View File

@ -167,6 +167,11 @@ New Features
fields for individual documents. This allows for instance to soft-delete fields for individual documents. This allows for instance to soft-delete
individual documents. (Simon Willnauer) individual documents. (Simon Willnauer)
* LUCENE-8298: Allow DocValues updates to reset a value. Passing a DV field with a null
value to IW#updateDocValues or IW#tryUpdateDocValues will now remove the value from the
provided document. This allows to undelete a soft-deleted document unless it's been claimed
by a merge. (Simon Willnauer)
Bug Fixes Bug Fixes
* LUCENE-8266: Detect bogus tiles when creating a standard polygon and * LUCENE-8266: Detect bogus tiles when creating a standard polygon and

View File

@ -156,5 +156,4 @@ public class SimpleTextLiveDocsFormat extends LiveDocsFormat {
return size; return size;
} }
} }
} }

View File

@ -59,8 +59,18 @@ public class NumericDocValuesField extends Field {
* @throws IllegalArgumentException if the field name is null * @throws IllegalArgumentException if the field name is null
*/ */
public NumericDocValuesField(String name, long value) { public NumericDocValuesField(String name, long value) {
this(name, Long.valueOf(value));
}
/**
* Creates a new DocValues field with the specified 64-bit long value
* @param name field name
* @param value 64-bit long value or <code>null</code> if the existing fields value should be removed on update
* @throws IllegalArgumentException if the field name is null
*/
public NumericDocValuesField(String name, Long value) {
super(name, TYPE); super(name, TYPE);
fieldsData = Long.valueOf(value); fieldsData = value;
} }
/** /**

View File

@ -542,6 +542,9 @@ final class DefaultIndexingChain extends DocConsumer {
if (fp.docValuesWriter == null) { if (fp.docValuesWriter == null) {
fp.docValuesWriter = new NumericDocValuesWriter(fp.fieldInfo, bytesUsed); fp.docValuesWriter = new NumericDocValuesWriter(fp.fieldInfo, bytesUsed);
} }
if (field.numericValue() == null) {
throw new IllegalArgumentException("field=\"" + fp.fieldInfo.name + "\": null value not allowed");
}
((NumericDocValuesWriter) fp.docValuesWriter).addValue(docID, field.numericValue().longValue()); ((NumericDocValuesWriter) fp.docValuesWriter).addValue(docID, field.numericValue().longValue());
break; break;

View File

@ -35,6 +35,9 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
abstract class DocValuesFieldUpdates implements Accountable { abstract class DocValuesFieldUpdates implements Accountable {
protected static final int PAGE_SIZE = 1024; protected static final int PAGE_SIZE = 1024;
private static final long HAS_VALUE_MASK = 1;
private static final long HAS_NO_VALUE_MASK = 0;
private static final int SHIFT = 1; // we use the first bit of each value to mark if the doc has a value or not
/** /**
* An iterator over documents and their updated values. Only documents with * An iterator over documents and their updated values. Only documents with
@ -73,6 +76,11 @@ abstract class DocValuesFieldUpdates implements Accountable {
/** Returns delGen for this packet. */ /** Returns delGen for this packet. */
abstract long delGen(); abstract long delGen();
/**
* Returns true if this doc has a value
*/
abstract boolean hasValue();
/** /**
* Wraps the given iterator as a BinaryDocValues instance. * Wraps the given iterator as a BinaryDocValues instance.
*/ */
@ -217,6 +225,11 @@ abstract class DocValuesFieldUpdates implements Accountable {
public long delGen() { public long delGen() {
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }
@Override
boolean hasValue() {
return queue.top().hasValue();
}
}; };
} }
@ -237,7 +250,7 @@ abstract class DocValuesFieldUpdates implements Accountable {
throw new NullPointerException("DocValuesType must not be null"); throw new NullPointerException("DocValuesType must not be null");
} }
this.type = type; this.type = type;
bitsPerValue = PackedInts.bitsRequired(maxDoc - 1); bitsPerValue = PackedInts.bitsRequired(maxDoc - 1) + SHIFT;
docs = new PagedMutable(1, PAGE_SIZE, bitsPerValue, PackedInts.COMPACT); docs = new PagedMutable(1, PAGE_SIZE, bitsPerValue, PackedInts.COMPACT);
} }
@ -299,7 +312,18 @@ abstract class DocValuesFieldUpdates implements Accountable {
return size; return size;
} }
/**
* Adds an update that resets the documents value.
* @param doc the doc to update
*/
final synchronized void reset(int doc) {
addInternal(doc, HAS_NO_VALUE_MASK);
}
final synchronized int add(int doc) { final synchronized int add(int doc) {
return addInternal(doc, HAS_VALUE_MASK);
}
private synchronized int addInternal(int doc, long hasValueMask) {
if (finished) { if (finished) {
throw new IllegalStateException("already finished"); throw new IllegalStateException("already finished");
} }
@ -313,8 +337,7 @@ abstract class DocValuesFieldUpdates implements Accountable {
if (docs.size() == size) { if (docs.size() == size) {
grow(size+1); grow(size+1);
} }
docs.set(size, (((long)doc) << SHIFT) | hasValueMask);
docs.set(size, doc);
++size; ++size;
return size-1; return size-1;
} }
@ -354,6 +377,7 @@ abstract class DocValuesFieldUpdates implements Accountable {
private long idx = 0; // long so we don't overflow if size == Integer.MAX_VALUE private long idx = 0; // long so we don't overflow if size == Integer.MAX_VALUE
private int doc = -1; private int doc = -1;
private final long delGen; private final long delGen;
private boolean hasValue;
AbstractIterator(int size, PagedMutable docs, long delGen) { AbstractIterator(int size, PagedMutable docs, long delGen) {
this.size = size; this.size = size;
@ -366,13 +390,17 @@ abstract class DocValuesFieldUpdates implements Accountable {
if (idx >= size) { if (idx >= size) {
return doc = DocIdSetIterator.NO_MORE_DOCS; return doc = DocIdSetIterator.NO_MORE_DOCS;
} }
doc = (int) docs.get(idx); long longDoc = docs.get(idx);
++idx; ++idx;
while (idx < size && docs.get(idx) == doc) { while (idx < size && docs.get(idx) == longDoc) {
// scan forward to last update to this doc // scan forward to last update to this doc
++idx; ++idx;
} }
hasValue = (longDoc & HAS_VALUE_MASK) > 0;
if (hasValue) {
set(idx - 1); set(idx - 1);
}
doc = (int)(longDoc >> SHIFT);
return doc; return doc;
} }
@ -391,5 +419,10 @@ abstract class DocValuesFieldUpdates implements Accountable {
final long delGen() { final long delGen() {
return delGen; return delGen;
} }
@Override
final boolean hasValue() {
return hasValue;
}
} }
} }

View File

@ -46,6 +46,7 @@ abstract class DocValuesUpdate {
// used in BufferedDeletes to apply this update only to a slice of docs. It's initialized to BufferedUpdates.MAX_INT // used in BufferedDeletes to apply this update only to a slice of docs. It's initialized to BufferedUpdates.MAX_INT
// since it's safe and most often used this way we safe object creations. // since it's safe and most often used this way we safe object creations.
final int docIDUpto; final int docIDUpto;
final boolean hasValue;
/** /**
* Constructor. * Constructor.
@ -53,12 +54,13 @@ abstract class DocValuesUpdate {
* @param term the {@link Term} which determines the documents that will be updated * @param term the {@link Term} which determines the documents that will be updated
* @param field the {@link NumericDocValuesField} to update * @param field the {@link NumericDocValuesField} to update
*/ */
protected DocValuesUpdate(DocValuesType type, Term term, String field, int docIDUpto) { protected DocValuesUpdate(DocValuesType type, Term term, String field, int docIDUpto, boolean hasValue) {
assert docIDUpto >= 0 : docIDUpto + "must be >= 0"; assert docIDUpto >= 0 : docIDUpto + "must be >= 0";
this.type = type; this.type = type;
this.term = term; this.term = term;
this.field = field; this.field = field;
this.docIDUpto = docIDUpto; this.docIDUpto = docIDUpto;
this.hasValue = hasValue;
} }
abstract long valueSizeInBytes(); abstract long valueSizeInBytes();
@ -69,6 +71,7 @@ abstract class DocValuesUpdate {
sizeInBytes += term.bytes.bytes.length; sizeInBytes += term.bytes.bytes.length;
sizeInBytes += field.length() * Character.BYTES; sizeInBytes += field.length() * Character.BYTES;
sizeInBytes += valueSizeInBytes(); sizeInBytes += valueSizeInBytes();
sizeInBytes += 1; // hasValue
return sizeInBytes; return sizeInBytes;
} }
@ -76,6 +79,10 @@ abstract class DocValuesUpdate {
abstract void writeTo(DataOutput output) throws IOException; abstract void writeTo(DataOutput output) throws IOException;
boolean hasValue() {
return hasValue;
}
@Override @Override
public String toString() { public String toString() {
return "term=" + term + ",field=" + field + ",value=" + valueToString() + ",docIDUpto=" + docIDUpto; return "term=" + term + ",field=" + field + ",value=" + valueToString() + ",docIDUpto=" + docIDUpto;
@ -83,7 +90,7 @@ abstract class DocValuesUpdate {
/** An in-place update to a binary DocValues field */ /** An in-place update to a binary DocValues field */
static final class BinaryDocValuesUpdate extends DocValuesUpdate { static final class BinaryDocValuesUpdate extends DocValuesUpdate {
final BytesRef value; private final BytesRef value;
/* Size of BytesRef: 2*INT + ARRAY_HEADER + PTR */ /* Size of BytesRef: 2*INT + ARRAY_HEADER + PTR */
private static final long RAW_VALUE_SIZE_IN_BYTES = NUM_BYTES_ARRAY_HEADER + 2*Integer.BYTES + NUM_BYTES_OBJECT_REF; private static final long RAW_VALUE_SIZE_IN_BYTES = NUM_BYTES_ARRAY_HEADER + 2*Integer.BYTES + NUM_BYTES_OBJECT_REF;
@ -93,7 +100,7 @@ abstract class DocValuesUpdate {
} }
private BinaryDocValuesUpdate(Term term, String field, BytesRef value, int docIDUpTo) { private BinaryDocValuesUpdate(Term term, String field, BytesRef value, int docIDUpTo) {
super(DocValuesType.BINARY, term, field, docIDUpTo); super(DocValuesType.BINARY, term, field, docIDUpTo, value != null);
this.value = value; this.value = value;
} }
@ -106,7 +113,7 @@ abstract class DocValuesUpdate {
@Override @Override
long valueSizeInBytes() { long valueSizeInBytes() {
return RAW_VALUE_SIZE_IN_BYTES + value.bytes.length; return RAW_VALUE_SIZE_IN_BYTES + (value == null ? 0 : value.bytes.length);
} }
@Override @Override
@ -114,8 +121,14 @@ abstract class DocValuesUpdate {
return value.toString(); return value.toString();
} }
BytesRef getValue() {
assert hasValue : "getValue should only be called if this update has a value";
return value;
}
@Override @Override
void writeTo(DataOutput out) throws IOException { void writeTo(DataOutput out) throws IOException {
assert hasValue;
out.writeVInt(value.length); out.writeVInt(value.length);
out.writeBytes(value.bytes, value.offset, value.length); out.writeBytes(value.bytes, value.offset, value.length);
} }
@ -132,14 +145,19 @@ abstract class DocValuesUpdate {
/** An in-place update to a numeric DocValues field */ /** An in-place update to a numeric DocValues field */
static final class NumericDocValuesUpdate extends DocValuesUpdate { static final class NumericDocValuesUpdate extends DocValuesUpdate {
final long value; private final long value;
NumericDocValuesUpdate(Term term, String field, long value) { NumericDocValuesUpdate(Term term, String field, long value) {
this(term, field, value, BufferedUpdates.MAX_INT); this(term, field, value, BufferedUpdates.MAX_INT, true);
} }
private NumericDocValuesUpdate(Term term, String field, long value, int docIDUpTo) { NumericDocValuesUpdate(Term term, String field, Long value) {
super(DocValuesType.NUMERIC, term, field, docIDUpTo); this(term, field, value != null ? value.longValue() : -1, BufferedUpdates.MAX_INT, value != null);
}
private NumericDocValuesUpdate(Term term, String field, long value, int docIDUpTo, boolean hasValue) {
super(DocValuesType.NUMERIC, term, field, docIDUpTo, hasValue);
this.value = value; this.value = value;
} }
@ -147,7 +165,7 @@ abstract class DocValuesUpdate {
if (docIDUpto == this.docIDUpto) { if (docIDUpto == this.docIDUpto) {
return this; return this;
} }
return new NumericDocValuesUpdate(term, field, value, docIDUpto); return new NumericDocValuesUpdate(term, field, value, docIDUpto, hasValue);
} }
@Override @Override
@ -157,16 +175,22 @@ abstract class DocValuesUpdate {
@Override @Override
protected String valueToString() { protected String valueToString() {
return Long.toString(value); return hasValue ? Long.toString(value) : "null";
} }
@Override @Override
void writeTo(DataOutput out) throws IOException { void writeTo(DataOutput out) throws IOException {
assert hasValue;
out.writeZLong(value); out.writeZLong(value);
} }
static long readFrom(DataInput in) throws IOException { static long readFrom(DataInput in) throws IOException {
return in.readZLong(); return in.readZLong();
} }
long getValue() {
assert hasValue : "getValue should only be called if this update has a value";
return value;
}
} }
} }

View File

@ -157,7 +157,7 @@ final class FrozenBufferedUpdates {
for (LinkedHashMap<Term, T> updates : dvUpdates.values()) { for (LinkedHashMap<Term, T> updates : dvUpdates.values()) {
updateSizeConsumer.accept(updates.size()); updateSizeConsumer.accept(updates.size());
for (T update : updates.values()) { for (T update : updates.values()) {
int code = update.term.bytes().length << 2; int code = update.term.bytes().length << 3;
String termField = update.term.field(); String termField = update.term.field();
if (termField.equals(lastTermField) == false) { if (termField.equals(lastTermField) == false) {
@ -167,6 +167,9 @@ final class FrozenBufferedUpdates {
if (updateField.equals(lastUpdateField) == false) { if (updateField.equals(lastUpdateField) == false) {
code |= 2; code |= 2;
} }
if (update.hasValue()) {
code |= 4;
}
out.writeVInt(code); out.writeVInt(code);
out.writeVInt(update.docIDUpto); out.writeVInt(update.docIDUpto);
if (termField.equals(lastTermField) == false) { if (termField.equals(lastTermField) == false) {
@ -178,9 +181,11 @@ final class FrozenBufferedUpdates {
lastUpdateField = updateField; lastUpdateField = updateField;
} }
out.writeBytes(update.term.bytes().bytes, update.term.bytes().offset, update.term.bytes().length); out.writeBytes(update.term.bytes().bytes, update.term.bytes().offset, update.term.bytes().length);
if (update.hasValue()) {
update.writeTo(out); update.writeTo(out);
} }
} }
}
byte[] bytes = new byte[(int) out.getFilePointer()]; byte[] bytes = new byte[(int) out.getFilePointer()];
out.writeTo(bytes, 0); out.writeTo(bytes, 0);
return bytes; return bytes;
@ -538,7 +543,7 @@ final class FrozenBufferedUpdates {
while (in.getPosition() != updates.length) { while (in.getPosition() != updates.length) {
int code = in.readVInt(); int code = in.readVInt();
int docIDUpto = in.readVInt(); int docIDUpto = in.readVInt();
term.length = code >> 2; term.length = code >> 3;
if ((code & 1) != 0) { if ((code & 1) != 0) {
termField = in.readString(); termField = in.readString();
@ -546,6 +551,7 @@ final class FrozenBufferedUpdates {
if ((code & 2) != 0) { if ((code & 2) != 0) {
updateField = in.readString(); updateField = in.readString();
} }
boolean hasValue = (code & 4) != 0;
if (term.bytes.length < term.length) { if (term.bytes.length < term.length) {
term.bytes = ArrayUtil.grow(term.bytes, term.length); term.bytes = ArrayUtil.grow(term.bytes, term.length);
@ -584,7 +590,10 @@ final class FrozenBufferedUpdates {
final BytesRef binaryValue; final BytesRef binaryValue;
final long longValue; final long longValue;
if (isNumeric) { if (hasValue == false) {
longValue = -1;
binaryValue = null;
} else if (isNumeric) {
longValue = NumericDocValuesUpdate.readFrom(in); longValue = NumericDocValuesUpdate.readFrom(in);
binaryValue = null; binaryValue = null;
} else { } else {
@ -611,7 +620,9 @@ final class FrozenBufferedUpdates {
} }
final IntConsumer docIdConsumer; final IntConsumer docIdConsumer;
final DocValuesFieldUpdates update = dvUpdates; final DocValuesFieldUpdates update = dvUpdates;
if (isNumeric) { if (hasValue == false) {
docIdConsumer = doc -> update.reset(doc);
} else if (isNumeric) {
docIdConsumer = doc -> update.add(doc, longValue); docIdConsumer = doc -> update.add(doc, longValue);
} else { } else {
docIdConsumer = doc -> update.add(doc, binaryValue); docIdConsumer = doc -> update.add(doc, binaryValue);

View File

@ -1370,6 +1370,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* the update succeeds and this method returns a valid (&gt; 0) sequence * the update succeeds and this method returns a valid (&gt; 0) sequence
* number; else, it returns -1 and the caller must then * number; else, it returns -1 and the caller must then
* either retry the update and resolve the document again. * either retry the update and resolve the document again.
* If a doc values fields data is <code>null</code> the existing
* value is removed from all documents matching the term. This can be used
* to un-delete a soft-deleted document since this method will apply the
* field update even if the document is marked as deleted.
* *
* <b>NOTE</b>: this method can only updates documents * <b>NOTE</b>: this method can only updates documents
* visible to the currently open NRT reader. If you need * visible to the currently open NRT reader. If you need
@ -1393,16 +1397,20 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
throw new AssertionError("type: " + update.type + " is not supported"); throw new AssertionError("type: " + update.type + " is not supported");
} }
}); });
if (update.hasValue()) {
switch (update.type) { switch (update.type) {
case NUMERIC: case NUMERIC:
docValuesFieldUpdates.add(leafDocId, ((NumericDocValuesUpdate) update).value); docValuesFieldUpdates.add(leafDocId, ((NumericDocValuesUpdate) update).getValue());
break; break;
case BINARY: case BINARY:
docValuesFieldUpdates.add(leafDocId, ((BinaryDocValuesUpdate) update).value); docValuesFieldUpdates.add(leafDocId, ((BinaryDocValuesUpdate) update).getValue());
break; break;
default: default:
throw new AssertionError("type: " + update.type + " is not supported"); throw new AssertionError("type: " + update.type + " is not supported");
} }
} else {
docValuesFieldUpdates.reset(leafDocId);
}
} }
for (DocValuesFieldUpdates updates : fieldUpdatesMap.values()) { for (DocValuesFieldUpdates updates : fieldUpdatesMap.values()) {
updates.finish(); updates.finish();
@ -1726,7 +1734,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* Updates documents' DocValues fields to the given values. Each field update * Updates documents' DocValues fields to the given values. Each field update
* is applied to the set of documents that are associated with the * is applied to the set of documents that are associated with the
* {@link Term} to the same value. All updates are atomically applied and * {@link Term} to the same value. All updates are atomically applied and
* flushed together. * flushed together. If a doc values fields data is <code>null</code> the existing
* value is removed from all documents matching the term.
*
* *
* @param updates * @param updates
* the updates to apply * the updates to apply
@ -1772,9 +1782,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
if (config.getIndexSortFields().contains(f.name())) { if (config.getIndexSortFields().contains(f.name())) {
throw new IllegalArgumentException("cannot update docvalues field involved in the index sort, field=" + f.name() + ", sort=" + config.getIndexSort()); throw new IllegalArgumentException("cannot update docvalues field involved in the index sort, field=" + f.name() + ", sort=" + config.getIndexSort());
} }
switch (dvType) { switch (dvType) {
case NUMERIC: case NUMERIC:
dvUpdates[i] = new NumericDocValuesUpdate(term, f.name(), (Long) f.numericValue()); Long value = (Long)f.numericValue();
dvUpdates[i] = new NumericDocValuesUpdate(term, f.name(), value);
break; break;
case BINARY: case BINARY:
dvUpdates[i] = new BinaryDocValuesUpdate(term, f.name(), f.binaryValue()); dvUpdates[i] = new BinaryDocValuesUpdate(term, f.name(), f.binaryValue());
@ -3740,8 +3752,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
while ((doc = it.nextDoc()) != NO_MORE_DOCS) { while ((doc = it.nextDoc()) != NO_MORE_DOCS) {
int mappedDoc = segDocMap.get(segLeafDocMap.get(doc)); int mappedDoc = segDocMap.get(segLeafDocMap.get(doc));
if (mappedDoc != -1) { if (mappedDoc != -1) {
if (it.hasValue()) {
// not deleted // not deleted
mappedUpdates.add(mappedDoc, it); mappedUpdates.add(mappedDoc, it);
} else {
mappedUpdates.reset(mappedDoc);
}
anyDVUpdates = true; anyDVUpdates = true;
} }
} }

View File

@ -61,7 +61,7 @@ class PendingDeletes {
} }
protected FixedBitSet getMutableBits() throws IOException { protected FixedBitSet getMutableBits() {
// if we pull mutable bits but we haven't been initialized something is completely off. // if we pull mutable bits but we haven't been initialized something is completely off.
// this means we receive deletes without having the bitset that is on-disk ready to be cloned // this means we receive deletes without having the bitset that is on-disk ready to be cloned
assert liveDocsInitialized : "can't delete if liveDocs are not initialized"; assert liveDocsInitialized : "can't delete if liveDocs are not initialized";
@ -70,14 +70,11 @@ class PendingDeletes {
// SegmentReader sharing the current liveDocs // SegmentReader sharing the current liveDocs
// instance; must now make a private clone so we can // instance; must now make a private clone so we can
// change it: // change it:
if (liveDocs != null) {
writeableLiveDocs = FixedBitSet.copyOf(liveDocs);
} else {
writeableLiveDocs = new FixedBitSet(info.info.maxDoc()); writeableLiveDocs = new FixedBitSet(info.info.maxDoc());
writeableLiveDocs.set(0, info.info.maxDoc()); writeableLiveDocs.set(0, info.info.maxDoc());
if (liveDocs != null) {
for (int i = 0; i < liveDocs.length(); ++i) {
if (liveDocs.get(i) == false) {
writeableLiveDocs.clear(i);
}
}
} }
liveDocs = writeableLiveDocs; liveDocs = writeableLiveDocs;
} }

View File

@ -109,13 +109,22 @@ final class PendingSoftDeletes extends PendingDeletes {
assert iterator != null; assert iterator != null;
int newDeletes = 0; int newDeletes = 0;
int docID; int docID;
DocValuesFieldUpdates.Iterator hasValue = iterator instanceof DocValuesFieldUpdates.Iterator
? (DocValuesFieldUpdates.Iterator) iterator : null;
while ((docID = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) { while ((docID = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
if (hasValue == null || hasValue.hasValue()) {
if (bits.get(docID)) { // doc is live - clear it if (bits.get(docID)) { // doc is live - clear it
bits.clear(docID); bits.clear(docID);
newDeletes++; newDeletes++;
// now that we know we deleted it and we fully control the hard deletes we can do correct accounting // now that we know we deleted it and we fully control the hard deletes we can do correct accounting
// below. // below.
} }
} else {
if (bits.get(docID) == false) {
bits.set(docID);
newDeletes--;
}
}
} }
return newDeletes; return newDeletes;
} }

View File

@ -350,9 +350,10 @@ final class ReadersAndUpdates {
fieldsConsumer.addBinaryField(fieldInfo, new EmptyDocValuesProducer() { fieldsConsumer.addBinaryField(fieldInfo, new EmptyDocValuesProducer() {
@Override @Override
public BinaryDocValues getBinary(FieldInfo fieldInfoIn) throws IOException { public BinaryDocValues getBinary(FieldInfo fieldInfoIn) throws IOException {
DocValuesFieldUpdates.Iterator iterator = updateSupplier.apply(fieldInfo);
final MergedDocValues<BinaryDocValues> mergedDocValues = new MergedDocValues<>( final MergedDocValues<BinaryDocValues> mergedDocValues = new MergedDocValues<>(
reader.getBinaryDocValues(field), reader.getBinaryDocValues(field),
DocValuesFieldUpdates.Iterator.asBinaryDocValues(updateSupplier.apply(fieldInfo))); DocValuesFieldUpdates.Iterator.asBinaryDocValues(iterator), iterator);
// Merge sort of the original doc values with updated doc values: // Merge sort of the original doc values with updated doc values:
return new BinaryDocValues() { return new BinaryDocValues() {
@Override @Override
@ -387,9 +388,10 @@ final class ReadersAndUpdates {
fieldsConsumer.addNumericField(fieldInfo, new EmptyDocValuesProducer() { fieldsConsumer.addNumericField(fieldInfo, new EmptyDocValuesProducer() {
@Override @Override
public NumericDocValues getNumeric(FieldInfo fieldInfoIn) throws IOException { public NumericDocValues getNumeric(FieldInfo fieldInfoIn) throws IOException {
DocValuesFieldUpdates.Iterator iterator = updateSupplier.apply(fieldInfo);
final MergedDocValues<NumericDocValues> mergedDocValues = new MergedDocValues<>( final MergedDocValues<NumericDocValues> mergedDocValues = new MergedDocValues<>(
reader.getNumericDocValues(field), reader.getNumericDocValues(field),
DocValuesFieldUpdates.Iterator.asNumericDocValues(updateSupplier.apply(fieldInfo))); DocValuesFieldUpdates.Iterator.asNumericDocValues(iterator), iterator);
// Merge sort of the original doc values with updated doc values: // Merge sort of the original doc values with updated doc values:
return new NumericDocValues() { return new NumericDocValues() {
@Override @Override
@ -433,6 +435,7 @@ final class ReadersAndUpdates {
* wins over the on-disk version. * wins over the on-disk version.
*/ */
static final class MergedDocValues<DocValuesInstance extends DocValuesIterator> extends DocValuesIterator { static final class MergedDocValues<DocValuesInstance extends DocValuesIterator> extends DocValuesIterator {
private final DocValuesFieldUpdates.Iterator updateIterator;
// merged docID // merged docID
private int docIDOut = -1; private int docIDOut = -1;
// docID from our original doc values // docID from our original doc values
@ -444,9 +447,10 @@ final class ReadersAndUpdates {
private final DocValuesInstance updateDocValues; private final DocValuesInstance updateDocValues;
DocValuesInstance currentValuesSupplier; DocValuesInstance currentValuesSupplier;
protected MergedDocValues(DocValuesInstance onDiskDocValues, DocValuesInstance updateDocValues) { protected MergedDocValues(DocValuesInstance onDiskDocValues, DocValuesInstance updateDocValues, DocValuesFieldUpdates.Iterator updateIterator) {
this.onDiskDocValues = onDiskDocValues; this.onDiskDocValues = onDiskDocValues;
this.updateDocValues = updateDocValues; this.updateDocValues = updateDocValues;
this.updateIterator = updateIterator;
} }
@Override @Override
@ -471,6 +475,8 @@ final class ReadersAndUpdates {
@Override @Override
public int nextDoc() throws IOException { public int nextDoc() throws IOException {
boolean hasValue = false;
do {
if (docIDOnDisk == docIDOut) { if (docIDOnDisk == docIDOut) {
if (onDiskDocValues == null) { if (onDiskDocValues == null) {
docIDOnDisk = NO_MORE_DOCS; docIDOnDisk = NO_MORE_DOCS;
@ -485,12 +491,17 @@ final class ReadersAndUpdates {
// no update to this doc - we use the on-disk values // no update to this doc - we use the on-disk values
docIDOut = docIDOnDisk; docIDOut = docIDOnDisk;
currentValuesSupplier = onDiskDocValues; currentValuesSupplier = onDiskDocValues;
hasValue = true;
} else { } else {
docIDOut = updateDocID; docIDOut = updateDocID;
if (docIDOut != NO_MORE_DOCS) { if (docIDOut != NO_MORE_DOCS) {
currentValuesSupplier = updateDocValues; currentValuesSupplier = updateDocValues;
hasValue = updateIterator.hasValue();
} else {
hasValue = true;
} }
} }
} while (hasValue == false);
return docIDOut; return docIDOut;
} }
}; };

View File

@ -510,4 +510,23 @@ public final class FixedBitSet extends BitSet implements Bits, Accountable {
// empty sets from returning 0, which is too common. // empty sets from returning 0, which is too common.
return (int) ((h>>32) ^ h) + 0x98761234; return (int) ((h>>32) ^ h) + 0x98761234;
} }
/**
* Make a copy of the given bits.
*/
public static FixedBitSet copyOf(Bits bits) {
if (bits instanceof FixedBitSet) {
return ((FixedBitSet)bits).clone();
} else {
int length = bits.length();
FixedBitSet bitSet = new FixedBitSet(length);
bitSet.set(0, length);
for (int i = 0; i < length; ++i) {
if (bits.get(i) == false) {
bitSet.clear(i);
}
}
return bitSet;
}
}
} }

View File

@ -17,6 +17,8 @@
package org.apache.lucene.index; package org.apache.lucene.index;
import java.io.IOException;
import org.apache.lucene.document.BinaryDocValuesField; import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.Document; import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field; import org.apache.lucene.document.Field;
@ -27,6 +29,7 @@ import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.document.StringField; import org.apache.lucene.document.StringField;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase;
/** Tests helper methods in DocValues */ /** Tests helper methods in DocValues */
@ -252,4 +255,18 @@ public class TestDocValues extends LuceneTestCase {
iw.close(); iw.close();
dir.close(); dir.close();
} }
public void testAddNullNumericDocValues() throws IOException {
Directory dir = newDirectory();
IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null));
Document doc = new Document();
if (random().nextBoolean()) {
doc.add(new NumericDocValuesField("foo", null));
} else {
doc.add(new BinaryDocValuesField("foo", null));
}
IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> iw.addDocument(doc));
assertEquals("field=\"foo\": null value not allowed", iae.getMessage());
IOUtils.close(iw, dir);
}
} }

View File

@ -36,6 +36,9 @@ import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.StringField; import org.apache.lucene.document.StringField;
import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.DocValuesFieldExistsQuery;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
@ -456,11 +459,12 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
IndexWriterConfig conf = newIndexWriterConfig(); IndexWriterConfig conf = newIndexWriterConfig();
IndexWriter writer = new IndexWriter(dir, conf); IndexWriter writer = new IndexWriter(dir, conf);
ReentrantLock[] locks = new ReentrantLock[25 + random().nextInt(50)]; ReentrantLock[] locks = new ReentrantLock[25 + random().nextInt(50)];
int[] values = new int[locks.length]; Long[] values = new Long[locks.length];
for (int i = 0; i < locks.length; i++) { for (int i = 0; i < locks.length; i++) {
locks[i] = new ReentrantLock(); locks[i] = new ReentrantLock();
Document doc = new Document(); Document doc = new Document();
values[i] = random().nextInt(); values[i] = random().nextLong();
doc.add(new StringField("id", Integer.toString(i), Store.NO)); doc.add(new StringField("id", Integer.toString(i), Store.NO));
doc.add(new NumericDocValuesField("value", values[i])); doc.add(new NumericDocValuesField("value", values[i]));
writer.addDocument(doc); writer.addDocument(doc);
@ -476,7 +480,7 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
int docId = random().nextInt(locks.length); int docId = random().nextInt(locks.length);
locks[docId].lock(); locks[docId].lock();
try { try {
int value = random().nextInt(); Long value = rarely() ? null : random().nextLong(); // sometimes reset it
if (random().nextBoolean()) { if (random().nextBoolean()) {
writer.updateDocValues(new Term("id", docId + ""), new NumericDocValuesField("value", value)); writer.updateDocValues(new Term("id", docId + ""), new NumericDocValuesField("value", value));
} else { } else {
@ -488,7 +492,6 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
} finally { } finally {
locks[docId].unlock(); locks[docId].unlock();
} }
if (rarely()) { if (rarely()) {
writer.flush(); writer.flush();
} }
@ -508,7 +511,7 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
for (int i = 0; i < locks.length; i++) { for (int i = 0; i < locks.length; i++) {
locks[i].lock(); locks[i].lock();
try { try {
int value = values[i]; Long value = values[i];
TopDocs topDocs = new IndexSearcher(reader).search(new TermQuery(new Term("id", "" + i)), 10); TopDocs topDocs = new IndexSearcher(reader).search(new TermQuery(new Term("id", "" + i)), 10);
assertEquals(topDocs.totalHits, 1); assertEquals(topDocs.totalHits, 1);
int docID = topDocs.scoreDocs[0].doc; int docID = topDocs.scoreDocs[0].doc;
@ -517,12 +520,15 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
LeafReader leafReader = leaves.get(subIndex).reader(); LeafReader leafReader = leaves.get(subIndex).reader();
docID -= leaves.get(subIndex).docBase; docID -= leaves.get(subIndex).docBase;
NumericDocValues numericDocValues = leafReader.getNumericDocValues("value"); NumericDocValues numericDocValues = leafReader.getNumericDocValues("value");
assertEquals(docID, numericDocValues.advance(docID)); if (value == null) {
assertEquals(numericDocValues.longValue(), value); assertFalse("docID: " + docID, numericDocValues.advanceExact(docID));
} else {
assertTrue("docID: " + docID, numericDocValues.advanceExact(docID));
assertEquals(numericDocValues.longValue(), value.longValue());
}
} finally { } finally {
locks[i].unlock(); locks[i].unlock();
} }
} }
} }
@ -540,5 +546,93 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
} }
} while (seqId == -1); } while (seqId == -1);
} }
public void testResetValue() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
IndexWriter writer = new IndexWriter(dir, conf);
Document doc = new Document();
doc.add(new StringField("id", "0", Store.NO));
doc.add(new NumericDocValuesField("val", 5));
doc.add(new BinaryDocValuesField("val-bin", new BytesRef(new byte[] {(byte)5})));
writer.addDocument(doc);
if (random().nextBoolean()) {
writer.commit();
}
try(DirectoryReader reader = writer.getReader()) {
assertEquals(1, reader.leaves().size());
LeafReader r = reader.leaves().get(0).reader();
NumericDocValues ndv = r.getNumericDocValues("val");
assertEquals(0, ndv.nextDoc());
assertEquals(5, ndv.longValue());
assertEquals(DocIdSetIterator.NO_MORE_DOCS, ndv.nextDoc());
BinaryDocValues bdv = r.getBinaryDocValues("val-bin");
assertEquals(0, bdv.nextDoc());
assertEquals(new BytesRef(new byte[]{(byte) 5}), bdv.binaryValue());
assertEquals(DocIdSetIterator.NO_MORE_DOCS, bdv.nextDoc());
}
writer.updateDocValues(new Term("id", "0"), new BinaryDocValuesField("val-bin", null));
try(DirectoryReader reader = writer.getReader()) {
assertEquals(1, reader.leaves().size());
LeafReader r = reader.leaves().get(0).reader();
NumericDocValues ndv = r.getNumericDocValues("val");
assertEquals(0, ndv.nextDoc());
assertEquals(5, ndv.longValue());
assertEquals(DocIdSetIterator.NO_MORE_DOCS, ndv.nextDoc());
BinaryDocValues bdv = r.getBinaryDocValues("val-bin");
assertEquals(DocIdSetIterator.NO_MORE_DOCS, bdv.nextDoc());
}
IOUtils.close(writer, dir);
}
public void testResetValueMultipleDocs() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
IndexWriter writer = new IndexWriter(dir, conf);
int numDocs = 10 + random().nextInt(50);
int currentSeqId = 0;
int[] seqId = new int[5];
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
int id = random().nextInt(5);
seqId[id] = currentSeqId;
doc.add(new StringField("id", "" + id, Store.YES));
doc.add(new NumericDocValuesField("seqID", currentSeqId++));
doc.add(new NumericDocValuesField("is_live", 1));
if (i > 0) {
writer.updateDocValues(new Term("id", "" + id), new NumericDocValuesField("is_live", null));
}
writer.addDocument(doc);
if (random().nextBoolean()) {
writer.flush();
}
}
if (random().nextBoolean()) {
writer.commit();
}
try(DirectoryReader reader = writer.getReader()) {
IndexSearcher searcher = new IndexSearcher(reader);
TopDocs is_live = searcher.search(new DocValuesFieldExistsQuery("is_live"), 5);
assertEquals(5, is_live.totalHits);
for (ScoreDoc doc : is_live.scoreDocs) {
int id = Integer.parseInt(reader.document(doc.doc).get("id"));
int i = ReaderUtil.subIndex(doc.doc, reader.leaves());
assertTrue(i >= 0);
LeafReaderContext leafReaderContext = reader.leaves().get(i);
NumericDocValues seqID = leafReaderContext.reader().getNumericDocValues("seqID");
assertNotNull(seqID);
assertTrue(seqID.advanceExact(doc.doc - leafReaderContext.docBase));
assertEquals(seqId[id], seqID.longValue());
}
}
IOUtils.close(writer, dir);
}
} }

View File

@ -118,7 +118,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
writer.close(); writer.close();
FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 0, Collections.emptyMap(), 0, 0); FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 0, Collections.emptyMap(), 0, 0);
List<Integer> docsDeleted = Arrays.asList(1, 3, 7, 8, DocIdSetIterator.NO_MORE_DOCS); List<Integer> docsDeleted = Arrays.asList(1, 3, 7, 8, DocIdSetIterator.NO_MORE_DOCS);
List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 10)); List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 10, true));
for (DocValuesFieldUpdates update : updates) { for (DocValuesFieldUpdates update : updates) {
deletes.onDocValuesUpdate(fieldInfo, update.iterator()); deletes.onDocValuesUpdate(fieldInfo, update.iterator());
} }
@ -135,7 +135,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
assertTrue(deletes.getLiveDocs().get(9)); assertTrue(deletes.getLiveDocs().get(9));
docsDeleted = Arrays.asList(1, 2, DocIdSetIterator.NO_MORE_DOCS); docsDeleted = Arrays.asList(1, 2, DocIdSetIterator.NO_MORE_DOCS);
updates = Arrays.asList(singleUpdate(docsDeleted, 10)); updates = Arrays.asList(singleUpdate(docsDeleted, 10, true));
fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 1, Collections.emptyMap(), 0, 0); fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 1, Collections.emptyMap(), 0, 0);
for (DocValuesFieldUpdates update : updates) { for (DocValuesFieldUpdates update : updates) {
deletes.onDocValuesUpdate(fieldInfo, update.iterator()); deletes.onDocValuesUpdate(fieldInfo, update.iterator());
@ -180,7 +180,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
deletes.onNewReader(segmentReader, segmentInfo); deletes.onNewReader(segmentReader, segmentInfo);
FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0); FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0);
List<Integer> docsDeleted = Arrays.asList(1, DocIdSetIterator.NO_MORE_DOCS); List<Integer> docsDeleted = Arrays.asList(1, DocIdSetIterator.NO_MORE_DOCS);
List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 3)); List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 3, true));
for (DocValuesFieldUpdates update : updates) { for (DocValuesFieldUpdates update : updates) {
deletes.onDocValuesUpdate(fieldInfo, update.iterator()); deletes.onDocValuesUpdate(fieldInfo, update.iterator());
} }
@ -199,7 +199,66 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
IOUtils.close(reader, writer, dir); IOUtils.close(reader, writer, dir);
} }
private DocValuesFieldUpdates singleUpdate(List<Integer> docsDeleted, int maxDoc) { public void testResetOnUpdate() throws IOException {
Directory dir = newDirectory();
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()
.setSoftDeletesField("_soft_deletes")
.setMaxBufferedDocs(3) // make sure we write one segment
.setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH));
Document doc = new Document();
doc.add(new StringField("id", "1", Field.Store.YES));
writer.softUpdateDocument(new Term("id", "1"), doc,
new NumericDocValuesField("_soft_deletes", 1));
doc = new Document();
doc.add(new StringField("id", "2", Field.Store.YES));
writer.softUpdateDocument(new Term("id", "2"), doc,
new NumericDocValuesField("_soft_deletes", 1));
doc = new Document();
doc.add(new StringField("id", "2", Field.Store.YES));
writer.softUpdateDocument(new Term("id", "2"), doc,
new NumericDocValuesField("_soft_deletes", 1));
writer.commit();
DirectoryReader reader = writer.getReader();
assertEquals(1, reader.leaves().size());
SegmentReader segmentReader = (SegmentReader) reader.leaves().get(0).reader();
SegmentCommitInfo segmentInfo = segmentReader.getSegmentInfo();
PendingDeletes deletes = newPendingDeletes(segmentInfo);
deletes.onNewReader(segmentReader, segmentInfo);
FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0);
List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(Arrays.asList(0, 1, DocIdSetIterator.NO_MORE_DOCS), 3, false));
for (DocValuesFieldUpdates update : updates) {
deletes.onDocValuesUpdate(fieldInfo, update.iterator());
}
assertEquals(0, deletes.numPendingDeletes());
assertTrue(deletes.getLiveDocs().get(0));
assertTrue(deletes.getLiveDocs().get(1));
assertTrue(deletes.getLiveDocs().get(2));
Bits liveDocs = deletes.getLiveDocs();
deletes.onNewReader(segmentReader, segmentInfo);
// no changes we keep this update
assertSame(liveDocs, deletes.getLiveDocs());
assertTrue(deletes.getLiveDocs().get(0));
assertTrue(deletes.getLiveDocs().get(1));
assertTrue(deletes.getLiveDocs().get(2));
assertEquals(0, deletes.numPendingDeletes());
segmentInfo.advanceDocValuesGen();
fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0);
updates = Arrays.asList(singleUpdate(Arrays.asList(1, DocIdSetIterator.NO_MORE_DOCS), 3, true));
for (DocValuesFieldUpdates update : updates) {
deletes.onDocValuesUpdate(fieldInfo, update.iterator());
}
// no changes we keep this update
assertNotSame(liveDocs, deletes.getLiveDocs());
assertTrue(deletes.getLiveDocs().get(0));
assertFalse(deletes.getLiveDocs().get(1));
assertTrue(deletes.getLiveDocs().get(2));
assertEquals(1, deletes.numPendingDeletes());
IOUtils.close(reader, writer, dir);
}
private DocValuesFieldUpdates singleUpdate(List<Integer> docsChanged, int maxDoc, boolean hasValue) {
return new DocValuesFieldUpdates(maxDoc, 0, "_soft_deletes", DocValuesType.NUMERIC) { return new DocValuesFieldUpdates(maxDoc, 0, "_soft_deletes", DocValuesType.NUMERIC) {
@Override @Override
public void add(int doc, long value) { public void add(int doc, long value) {
@ -219,7 +278,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
@Override @Override
public Iterator iterator() { public Iterator iterator() {
return new Iterator() { return new Iterator() {
java.util.Iterator<Integer> iter = docsDeleted.iterator(); java.util.Iterator<Integer> iter = docsChanged.iterator();
int doc = -1; int doc = -1;
@Override @Override
@ -246,6 +305,11 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
long delGen() { long delGen() {
return 0; return 0;
} }
@Override
boolean hasValue() {
return hasValue;
}
}; };
} }
}; };

View File

@ -538,4 +538,93 @@ public class TestSoftDeletesRetentionMergePolicy extends LuceneTestCase {
assertFalse(delete.get()); assertFalse(delete.get());
IOUtils.close(reader, writer, dir); IOUtils.close(reader, writer, dir);
} }
public void testUndeleteDocument() throws IOException {
Directory dir = newDirectory();
String softDelete = "soft_delete";
IndexWriterConfig config = newIndexWriterConfig()
.setSoftDeletesField(softDelete)
.setMergePolicy(new SoftDeletesRetentionMergePolicy("soft_delete",
MatchAllDocsQuery::new, new LogDocMergePolicy()));
config.setReaderPooling(true);
config.setMergePolicy(new LogDocMergePolicy());
IndexWriter writer = new IndexWriter(dir, config);
Document d = new Document();
d.add(new StringField("id", "0", Field.Store.YES));
d.add(new StringField("seq_id", "0", Field.Store.YES));
writer.addDocument(d);
d = new Document();
d.add(new StringField("id", "1", Field.Store.YES));
writer.addDocument(d);
writer.updateDocValues(new Term("id", "0"), new NumericDocValuesField("soft_delete", 1));
try (IndexReader reader = writer.getReader()) {
assertEquals(2, reader.maxDoc());
assertEquals(1, reader.numDocs());
}
doUpdate(new Term("id", "0"), writer, new NumericDocValuesField("soft_delete", null));
try (IndexReader reader = writer.getReader()) {
assertEquals(2, reader.maxDoc());
assertEquals(2, reader.numDocs());
}
IOUtils.close(writer, dir);
}
static void doUpdate(Term doc, IndexWriter writer, Field... fields) throws IOException {
long seqId = -1;
do { // retry if we just committing a merge
try (DirectoryReader reader = writer.getReader()) {
TopDocs topDocs = new IndexSearcher(new NoDeletesWrapper(reader)).search(new TermQuery(doc), 10);
assertEquals(1, topDocs.totalHits);
int theDoc = topDocs.scoreDocs[0].doc;
seqId = writer.tryUpdateDocValue(reader, theDoc, fields);
}
} while (seqId == -1);
}
private static final class NoDeletesSubReaderWrapper extends FilterDirectoryReader.SubReaderWrapper {
@Override
public LeafReader wrap(LeafReader reader) {
return new FilterLeafReader(reader) {
@Override
public int numDocs() {
return maxDoc();
}
@Override
public Bits getLiveDocs() {
return null;
}
@Override
public CacheHelper getCoreCacheHelper() {
return null;
}
@Override
public CacheHelper getReaderCacheHelper() {
return null;
}
};
}
}
private static final class NoDeletesWrapper extends FilterDirectoryReader {
NoDeletesWrapper(DirectoryReader in) throws IOException {
super(in, new NoDeletesSubReaderWrapper());
}
@Override
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
return new NoDeletesWrapper(in);
}
@Override
public CacheHelper getReaderCacheHelper() {
return null;
}
}
} }

View File

@ -493,4 +493,34 @@ public class TestFixedBitSet extends BaseBitSetTestCase<FixedBitSet> {
assertEquals(bitSet1.cardinality(), andNotCount); assertEquals(bitSet1.cardinality(), andNotCount);
} }
public void testCopyOf() {
Random random = random();
int numBits = TestUtil.nextInt(random, 1000, 2000);
int count = TestUtil.nextInt(random, 0, numBits - 1);
int[] bits = makeIntArray(random, count, 0, numBits - 1);
FixedBitSet fixedBitSet = makeFixedBitSet(bits, numBits);
FixedBitSet mutableCopy = FixedBitSet.copyOf(fixedBitSet);
assertNotSame(mutableCopy, fixedBitSet);
assertEquals(mutableCopy, fixedBitSet);
FixedBitSet mutableCopy1 = FixedBitSet.copyOf(new Bits() {
@Override
public boolean get(int index) {
return fixedBitSet.get(index);
}
@Override
public int length() {
return fixedBitSet.length();
}
});
assertNotSame(mutableCopy, mutableCopy1);
assertNotSame(fixedBitSet, mutableCopy1);
assertEquals(mutableCopy1, mutableCopy);
assertEquals(mutableCopy1, fixedBitSet);
}
} }

View File

@ -92,5 +92,4 @@ public class AssertingLiveDocsFormat extends LiveDocsFormat {
return "Asserting(" + in + ")"; return "Asserting(" + in + ")";
} }
} }
} }