mirror of https://github.com/apache/lucene.git
LUCENE-8341: Record soft deletes in SegmentCommitInfo
This change add the number of documents that are soft deletes but not hard deleted to the segment commit info. This is the last step towards making soft deletes as powerful as hard deltes since now the number of document can be read from commit points without opening a full blown reader. This also allows merge posliies to make decisions without requiring an NRT reader to get the relevant statistics. This change doesn't enforce any field to be used as soft deletes and the statistic is maintained per segment.
This commit is contained in:
parent
2ef3d07561
commit
fe83838ec3
|
@ -200,4 +200,9 @@ class BinaryDocValuesWriter extends DocValuesWriter {
|
|||
return value.get();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
DocIdSetIterator getDocIdSet() {
|
||||
return docsWithField.iterator();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -259,10 +259,10 @@ final class BufferedUpdatesStream implements Accountable {
|
|||
|
||||
SegmentState(ReadersAndUpdates rld, IOUtils.IOConsumer<ReadersAndUpdates> onClose, SegmentCommitInfo info) throws IOException {
|
||||
this.rld = rld;
|
||||
reader = rld.getReader(IOContext.READ);
|
||||
startDelCount = rld.getDelCount();
|
||||
delGen = info.getBufferedDeletesGen();
|
||||
this.onClose = onClose;
|
||||
reader = rld.getReader(IOContext.READ);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -46,6 +46,7 @@ import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
|
|||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.DocValuesFieldExistsQuery;
|
||||
import org.apache.lucene.search.LeafFieldComparator;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
|
@ -411,7 +412,7 @@ public final class CheckIndex implements Closeable {
|
|||
* that would otherwise be more complicated to debug if they had to close the writer
|
||||
* for each check.
|
||||
*/
|
||||
public CheckIndex(Directory dir, Lock writeLock) throws IOException {
|
||||
public CheckIndex(Directory dir, Lock writeLock) {
|
||||
this.dir = dir;
|
||||
this.writeLock = writeLock;
|
||||
this.infoStream = null;
|
||||
|
@ -781,7 +782,10 @@ public final class CheckIndex implements Closeable {
|
|||
throw new RuntimeException("Points test failed");
|
||||
}
|
||||
}
|
||||
|
||||
final String softDeletesField = reader.getFieldInfos().getSoftDeletesField();
|
||||
if (softDeletesField != null) {
|
||||
checkSoftDeletes(softDeletesField, info, reader, infoStream, failFast);
|
||||
}
|
||||
msg(infoStream, "");
|
||||
|
||||
if (verbose) {
|
||||
|
@ -3049,6 +3053,25 @@ public final class CheckIndex implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
private static void checkSoftDeletes(String softDeletesField, SegmentCommitInfo info, SegmentReader reader, PrintStream infoStream, boolean failFast) throws IOException {
|
||||
if (infoStream != null)
|
||||
infoStream.print(" test: check soft deletes.....");
|
||||
try {
|
||||
int softDeletes = PendingSoftDeletes.countSoftDeletes(DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(softDeletesField, reader), reader.getLiveDocs());
|
||||
if (softDeletes != info.getSoftDelCount()) {
|
||||
throw new RuntimeException("actual soft deletes: " + softDeletes + " but expected: " +info.getSoftDelCount());
|
||||
}
|
||||
} catch (Exception e) {
|
||||
if (failFast) {
|
||||
throw IOUtils.rethrowAlways(e);
|
||||
}
|
||||
msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
|
||||
if (infoStream != null) {
|
||||
e.printStackTrace(infoStream);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static double nsToSec(long ns) {
|
||||
return ns/1000000000.0;
|
||||
}
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.lucene.codecs.NormsProducer;
|
|||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PointsWriter;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
|
@ -841,4 +842,19 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
DocIdSetIterator getHasDocValues(String field) {
|
||||
PerField perField = getPerField(field);
|
||||
if (perField != null) {
|
||||
if (perField.docValuesWriter != null) {
|
||||
if (perField.fieldInfo.getDocValuesType() == DocValuesType.NONE) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return perField.docValuesWriter.getDocIdSet();
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,8 +19,17 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
|
||||
abstract class DocConsumer {
|
||||
abstract void processDocument() throws IOException;
|
||||
abstract Sorter.DocMap flush(final SegmentWriteState state) throws IOException;
|
||||
abstract void abort() throws IOException;
|
||||
|
||||
/**
|
||||
* Returns a {@link DocIdSetIterator} for the given field or null if the field doesn't have
|
||||
* doc values.
|
||||
*/
|
||||
abstract DocIdSetIterator getHasDocValues(String field);
|
||||
|
||||
}
|
||||
|
|
|
@ -20,10 +20,13 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.SortField;
|
||||
|
||||
abstract class DocValuesWriter {
|
||||
abstract void finish(int numDoc);
|
||||
abstract void flush(SegmentWriteState state, Sorter.DocMap sortMap, DocValuesConsumer consumer) throws IOException;
|
||||
abstract Sorter.DocComparator getDocComparator(int numDoc, SortField sortField) throws IOException;
|
||||
abstract DocIdSetIterator getDocIdSet();
|
||||
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicLong;
|
|||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FlushInfo;
|
||||
|
@ -460,14 +461,27 @@ final class DocumentsWriterPerThread {
|
|||
}
|
||||
final Sorter.DocMap sortMap;
|
||||
try {
|
||||
DocIdSetIterator softDeletedDocs;
|
||||
if (indexWriterConfig.getSoftDeletesField() != null) {
|
||||
softDeletedDocs = consumer.getHasDocValues(indexWriterConfig.getSoftDeletesField());
|
||||
} else {
|
||||
softDeletedDocs = null;
|
||||
}
|
||||
sortMap = consumer.flush(flushState);
|
||||
if (softDeletedDocs == null) {
|
||||
flushState.softDelCountOnFlush = 0;
|
||||
} else {
|
||||
flushState.softDelCountOnFlush = PendingSoftDeletes.countSoftDeletes(softDeletedDocs, flushState.liveDocs);
|
||||
assert flushState.segmentInfo.maxDoc() >= flushState.softDelCountOnFlush + flushState.delCountOnFlush;
|
||||
}
|
||||
// We clear this here because we already resolved them (private to this segment) when writing postings:
|
||||
pendingUpdates.clearDeleteTerms();
|
||||
segmentInfo.setFiles(new HashSet<>(directory.getCreatedFiles()));
|
||||
|
||||
final SegmentCommitInfo segmentInfoPerCommit = new SegmentCommitInfo(segmentInfo, 0, -1L, -1L, -1L);
|
||||
final SegmentCommitInfo segmentInfoPerCommit = new SegmentCommitInfo(segmentInfo, 0, flushState.softDelCountOnFlush, -1L, -1L, -1L);
|
||||
if (infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : flushState.delCountOnFlush) + " deleted docs");
|
||||
infoStream.message("DWPT", "new segment has " + flushState.softDelCountOnFlush + " soft-deleted docs");
|
||||
infoStream.message("DWPT", "new segment has " +
|
||||
(flushState.fieldInfos.hasVectors() ? "vectors" : "no vectors") + "; " +
|
||||
(flushState.fieldInfos.hasNorms() ? "norms" : "no norms") + "; " +
|
||||
|
@ -497,8 +511,7 @@ final class DocumentsWriterPerThread {
|
|||
assert segmentInfo != null;
|
||||
|
||||
FlushedSegment fs = new FlushedSegment(infoStream, segmentInfoPerCommit, flushState.fieldInfos,
|
||||
segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush,
|
||||
sortMap);
|
||||
segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush, sortMap);
|
||||
sealFlushedSegment(fs, sortMap, flushNotifications);
|
||||
if (infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", "flush time " + ((System.nanoTime() - t0) / 1000000.0) + " msec");
|
||||
|
|
|
@ -49,6 +49,7 @@ import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
|
|||
import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
|
||||
import org.apache.lucene.index.FieldInfos.FieldNumbers;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.search.DocValuesFieldExistsQuery;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Sort;
|
||||
|
@ -347,6 +348,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
* much like how hotels place an "authorization hold" on your credit
|
||||
* card to make sure they can later charge you when you check out. */
|
||||
final AtomicLong pendingNumDocs = new AtomicLong();
|
||||
final boolean softDeletesEnabled;
|
||||
|
||||
private final DocumentsWriter.FlushNotifications flushNotifications = new DocumentsWriter.FlushNotifications() {
|
||||
@Override
|
||||
|
@ -639,7 +641,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
if (rld != null) {
|
||||
return rld.getDelCount(); // get the full count from here since SCI might change concurrently
|
||||
} else {
|
||||
int delCount = info.getDelCount();
|
||||
final int delCount = info.getDelCount(softDeletesEnabled);
|
||||
assert delCount <= info.info.maxDoc(): "delCount: " + delCount + " maxDoc: " + info.info.maxDoc();
|
||||
return delCount;
|
||||
}
|
||||
|
@ -703,7 +705,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
conf.setIndexWriter(this); // prevent reuse by other instances
|
||||
config = conf;
|
||||
infoStream = config.getInfoStream();
|
||||
|
||||
softDeletesEnabled = config.getSoftDeletesField() != null;
|
||||
// obtain the write.lock. If the user configured a timeout,
|
||||
// we wrap with a sleeper and this might take some time.
|
||||
writeLock = d.obtainLock(WRITE_LOCK_NAME);
|
||||
|
@ -1154,7 +1156,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
if (docWriter.anyDeletions()) {
|
||||
return true;
|
||||
}
|
||||
if (readerPool.anyPendingDeletes()) {
|
||||
if (readerPool.anyDeletions()) {
|
||||
return true;
|
||||
}
|
||||
for (final SegmentCommitInfo info : segmentInfos) {
|
||||
|
@ -2939,11 +2941,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
|
||||
// long so we can detect int overflow:
|
||||
long numDocs = 0;
|
||||
|
||||
Sort indexSort = config.getIndexSort();
|
||||
|
||||
long seqNo;
|
||||
|
||||
try {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "flush at addIndexes(CodecReader...)");
|
||||
|
@ -2951,10 +2949,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
flush(false, true);
|
||||
|
||||
String mergedName = newSegmentName();
|
||||
|
||||
int numSoftDeleted = 0;
|
||||
for (CodecReader leaf : readers) {
|
||||
numDocs += leaf.numDocs();
|
||||
validateMergeReader(leaf);
|
||||
if (softDeletesEnabled) {
|
||||
Bits liveDocs = leaf.getLiveDocs();
|
||||
numSoftDeleted += PendingSoftDeletes.countSoftDeletes(
|
||||
DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(config.getSoftDeletesField(), leaf), liveDocs);
|
||||
}
|
||||
}
|
||||
|
||||
// Best-effort up front check:
|
||||
|
@ -2979,8 +2982,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
}
|
||||
|
||||
merger.merge(); // merge 'em
|
||||
|
||||
SegmentCommitInfo infoPerCommit = new SegmentCommitInfo(info, 0, -1L, -1L, -1L);
|
||||
SegmentCommitInfo infoPerCommit = new SegmentCommitInfo(info, 0, numSoftDeleted, -1L, -1L, -1L);
|
||||
|
||||
info.setFiles(new HashSet<>(trackingDir.getCreatedFiles()));
|
||||
trackingDir.clearCreatedFiles();
|
||||
|
@ -3057,7 +3059,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
SegmentInfo newInfo = new SegmentInfo(directoryOrig, info.info.getVersion(), info.info.getMinVersion(), segName, info.info.maxDoc(),
|
||||
info.info.getUseCompoundFile(), info.info.getCodec(),
|
||||
info.info.getDiagnostics(), info.info.getId(), info.info.getAttributes(), info.info.getIndexSort());
|
||||
SegmentCommitInfo newInfoPerCommit = new SegmentCommitInfo(newInfo, info.getDelCount(), info.getDelGen(),
|
||||
SegmentCommitInfo newInfoPerCommit = new SegmentCommitInfo(newInfo, info.getDelCount(), info.getSoftDelCount(), info.getDelGen(),
|
||||
info.getFieldInfosGen(), info.getDocValuesGen());
|
||||
|
||||
newInfo.setFiles(info.info.files());
|
||||
|
@ -4249,7 +4251,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
details.put("mergeMaxNumSegments", "" + merge.maxNumSegments);
|
||||
details.put("mergeFactor", Integer.toString(merge.segments.size()));
|
||||
setDiagnostics(si, SOURCE_MERGE, details);
|
||||
merge.setMergeInfo(new SegmentCommitInfo(si, 0, -1L, -1L, -1L));
|
||||
merge.setMergeInfo(new SegmentCommitInfo(si, 0, 0, -1L, -1L, -1L));
|
||||
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "merge seg=" + merge.info.info.name + " " + segString(merge.segments));
|
||||
|
@ -4373,16 +4375,25 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
|
||||
// Let the merge wrap readers
|
||||
List<CodecReader> mergeReaders = new ArrayList<>();
|
||||
int numSoftDeleted = 0;
|
||||
for (SegmentReader reader : merge.readers) {
|
||||
CodecReader wrappedReader = merge.wrapForMerge(reader);
|
||||
validateMergeReader(wrappedReader);
|
||||
mergeReaders.add(wrappedReader);
|
||||
if (softDeletesEnabled) {
|
||||
if (reader != wrappedReader) { // if we don't have a wrapped reader we won't preserve any soft-deletes
|
||||
Bits liveDocs = wrappedReader.getLiveDocs();
|
||||
numSoftDeleted += PendingSoftDeletes.countSoftDeletes(
|
||||
DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(config.getSoftDeletesField(), wrappedReader),
|
||||
liveDocs);
|
||||
}
|
||||
}
|
||||
}
|
||||
final SegmentMerger merger = new SegmentMerger(mergeReaders,
|
||||
merge.info.info, infoStream, dirWrapper,
|
||||
globalFieldNumberMap,
|
||||
context);
|
||||
|
||||
merge.info.setSoftDelCount(numSoftDeleted);
|
||||
merge.checkAborted();
|
||||
|
||||
merge.mergeStartNS = System.nanoTime();
|
||||
|
@ -4604,7 +4615,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
*
|
||||
* @lucene.internal */
|
||||
private synchronized String segString(SegmentCommitInfo info) {
|
||||
return info.toString(numDeletedDocs(info) - info.getDelCount());
|
||||
return info.toString(numDeletedDocs(info) - info.getDelCount(softDeletesEnabled));
|
||||
}
|
||||
|
||||
private synchronized void doWait() {
|
||||
|
|
|
@ -82,6 +82,11 @@ class NumericDocValuesWriter extends DocValuesWriter {
|
|||
return Sorter.getDocComparator(maxDoc, sortField, () -> null, () -> docValues);
|
||||
}
|
||||
|
||||
@Override
|
||||
DocIdSetIterator getDocIdSet() {
|
||||
return docsWithField.iterator();
|
||||
}
|
||||
|
||||
static SortingLeafReader.CachedNumericDVs sortDocValues(int maxDoc, Sorter.DocMap sortMap, NumericDocValues oldDocValues) throws IOException {
|
||||
FixedBitSet docsWithField = new FixedBitSet(maxDoc);
|
||||
long[] values = new long[maxDoc];
|
||||
|
|
|
@ -220,7 +220,7 @@ class PendingDeletes {
|
|||
* Returns <code>true</code> iff the segment represented by this {@link PendingDeletes} is fully deleted
|
||||
*/
|
||||
boolean isFullyDeleted(IOSupplier<CodecReader> readerIOSupplier) throws IOException {
|
||||
return info.getDelCount() + numPendingDeletes() == info.info.maxDoc();
|
||||
return getDelCount() == info.info.maxDoc();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -246,7 +246,8 @@ class PendingDeletes {
|
|||
* Returns the number of deleted docs in the segment.
|
||||
*/
|
||||
final int getDelCount() {
|
||||
return info.getDelCount() + numPendingDeletes();
|
||||
int delCount = info.getDelCount() + info.getSoftDelCount() + numPendingDeletes();
|
||||
return delCount;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -270,7 +271,8 @@ class PendingDeletes {
|
|||
count = info.info.maxDoc();
|
||||
}
|
||||
assert numDocs() == count: "info.maxDoc=" + info.info.maxDoc() + " info.getDelCount()=" + info.getDelCount() +
|
||||
" pendingDeletes=" + toString() + " count=" + count;
|
||||
" info.getSoftDelCount()=" + info.getSoftDelCount() +
|
||||
" pendingDeletes=" + toString() + " count=" + count + " numDocs: " + numDocs();
|
||||
assert reader.numDocs() == numDocs() : "reader.numDocs() = " + reader.numDocs() + " numDocs() " + numDocs();
|
||||
assert reader.numDeletedDocs() <= info.info.maxDoc(): "delCount=" + reader.numDeletedDocs() + " info.maxDoc=" +
|
||||
info.info.maxDoc() + " rld.pendingDeleteCount=" + numPendingDeletes() +
|
||||
|
|
|
@ -58,7 +58,7 @@ final class PendingSoftDeletes extends PendingDeletes {
|
|||
} else {
|
||||
// if it was deleted subtract the delCount
|
||||
pendingDeleteCount--;
|
||||
assert pendingDeleteCount >= 0 : " illegal pending delete count: " + pendingDeleteCount;
|
||||
assert assertPendingDeletes();
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
@ -76,11 +76,15 @@ final class PendingSoftDeletes extends PendingDeletes {
|
|||
hardDeletes.onNewReader(reader, info);
|
||||
if (dvGeneration < info.getDocValuesGen()) { // only re-calculate this if we haven't seen this generation
|
||||
final DocIdSetIterator iterator = DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(field, reader);
|
||||
int newDelCount;
|
||||
if (iterator != null) { // nothing is deleted we don't have a soft deletes field in this segment
|
||||
assert info.info.maxDoc() > 0 : "maxDoc is 0";
|
||||
pendingDeleteCount += applySoftDeletes(iterator, getMutableBits());
|
||||
assert pendingDeleteCount >= 0 : " illegal pending delete count: " + pendingDeleteCount;
|
||||
newDelCount = applySoftDeletes(iterator, getMutableBits());
|
||||
assert newDelCount >= 0 : " illegal pending delete count: " + newDelCount;
|
||||
} else {
|
||||
newDelCount = 0;
|
||||
}
|
||||
assert info.getSoftDelCount() == newDelCount : "softDeleteCount doesn't match " + info.getSoftDelCount() + " != " + newDelCount;
|
||||
dvGeneration = info.getDocValuesGen();
|
||||
}
|
||||
assert getDelCount() <= info.info.maxDoc() : getDelCount() + " > " + info.info.maxDoc();
|
||||
|
@ -88,8 +92,15 @@ final class PendingSoftDeletes extends PendingDeletes {
|
|||
|
||||
@Override
|
||||
boolean writeLiveDocs(Directory dir) throws IOException {
|
||||
// we need to set this here to make sure our stats in SCI are up-to-date otherwise we might hit an assertion
|
||||
// when the hard deletes are set since we need to account for docs that used to be only soft-delete but now hard-deleted
|
||||
this.info.setSoftDelCount(this.info.getSoftDelCount() + pendingDeleteCount);
|
||||
super.dropChanges();
|
||||
// delegate the write to the hard deletes - it will only write if somebody used it.
|
||||
return hardDeletes.writeLiveDocs(dir);
|
||||
if (hardDeletes.writeLiveDocs(dir)) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -134,13 +145,21 @@ final class PendingSoftDeletes extends PendingDeletes {
|
|||
void onDocValuesUpdate(FieldInfo info, DocValuesFieldUpdates.Iterator iterator) throws IOException {
|
||||
if (this.field.equals(info.name)) {
|
||||
pendingDeleteCount += applySoftDeletes(iterator, getMutableBits());
|
||||
assert pendingDeleteCount >= 0 : " illegal pending delete count: " + pendingDeleteCount;
|
||||
assert assertPendingDeletes();
|
||||
assert dvGeneration < info.getDocValuesGen() : "we have seen this generation update already: " + dvGeneration + " vs. " + info.getDocValuesGen();
|
||||
assert dvGeneration != -2 : "docValues generation is still uninitialized";
|
||||
dvGeneration = info.getDocValuesGen();
|
||||
this.info.setSoftDelCount(this.info.getSoftDelCount() + pendingDeleteCount);
|
||||
super.dropChanges();
|
||||
}
|
||||
}
|
||||
|
||||
private boolean assertPendingDeletes() {
|
||||
assert pendingDeleteCount + info.getSoftDelCount() >= 0 : " illegal pending delete count: " + pendingDeleteCount + info.getSoftDelCount();
|
||||
assert info.info.maxDoc() >= getDelCount();
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
|
@ -210,4 +229,17 @@ final class PendingSoftDeletes extends PendingDeletes {
|
|||
Bits getHardLiveDocs() {
|
||||
return hardDeletes.getLiveDocs();
|
||||
}
|
||||
|
||||
static int countSoftDeletes(DocIdSetIterator softDeletedDocs, Bits hardDeletes) throws IOException {
|
||||
int count = 0;
|
||||
if (softDeletedDocs != null) {
|
||||
int doc;
|
||||
while ((doc = softDeletedDocs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
if (hardDeletes == null || hardDeletes.get(doc)) {
|
||||
count++;
|
||||
}
|
||||
}
|
||||
}
|
||||
return count;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -130,9 +130,9 @@ final class ReaderPool implements Closeable {
|
|||
/**
|
||||
* Returns <code>true</code> iff any of the buffered readers and updates has at least one pending delete
|
||||
*/
|
||||
synchronized boolean anyPendingDeletes() {
|
||||
synchronized boolean anyDeletions() {
|
||||
for(ReadersAndUpdates rld : readerMap.values()) {
|
||||
if (rld.anyPendingDeletes()) {
|
||||
if (rld.getDelCount() > 0) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -406,10 +406,6 @@ final class ReadersAndUpdates {
|
|||
}
|
||||
}
|
||||
|
||||
synchronized boolean anyPendingDeletes() {
|
||||
return pendingDeletes.numPendingDeletes() != 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* This class merges the current on-disk DV with an incoming update DV instance and merges the two instances
|
||||
* giving the incoming update precedence in terms of values, in other words the values of the update always
|
||||
|
@ -713,8 +709,6 @@ final class ReadersAndUpdates {
|
|||
reader = createNewReaderWithLatestLiveDocs(reader);
|
||||
}
|
||||
assert pendingDeletes.verifyDocCounts(reader);
|
||||
|
||||
|
||||
return new MergeReader(reader, pendingDeletes.getHardLiveDocs());
|
||||
}
|
||||
|
||||
|
|
|
@ -38,6 +38,9 @@ public class SegmentCommitInfo {
|
|||
// How many deleted docs in the segment:
|
||||
private int delCount;
|
||||
|
||||
// How many soft-deleted docs in the segment that are not also hard-deleted:
|
||||
private int softDelCount;
|
||||
|
||||
// Generation number of the live docs file (-1 if there
|
||||
// are no deletes yet):
|
||||
private long delGen;
|
||||
|
@ -88,9 +91,10 @@ public class SegmentCommitInfo {
|
|||
* @param docValuesGen
|
||||
* DocValues generation number (used to name doc-values updates files)
|
||||
*/
|
||||
public SegmentCommitInfo(SegmentInfo info, int delCount, long delGen, long fieldInfosGen, long docValuesGen) {
|
||||
public SegmentCommitInfo(SegmentInfo info, int delCount, int softDelCount, long delGen, long fieldInfosGen, long docValuesGen) {
|
||||
this.info = info;
|
||||
this.delCount = delCount;
|
||||
this.softDelCount = softDelCount;
|
||||
this.delGen = delGen;
|
||||
this.nextWriteDelGen = delGen == -1 ? 1 : delGen + 1;
|
||||
this.fieldInfosGen = fieldInfosGen;
|
||||
|
@ -313,13 +317,29 @@ public class SegmentCommitInfo {
|
|||
return delCount;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of only soft-deleted docs.
|
||||
*/
|
||||
public int getSoftDelCount() {
|
||||
return softDelCount;
|
||||
}
|
||||
|
||||
void setDelCount(int delCount) {
|
||||
if (delCount < 0 || delCount > info.maxDoc()) {
|
||||
throw new IllegalArgumentException("invalid delCount=" + delCount + " (maxDoc=" + info.maxDoc() + ")");
|
||||
}
|
||||
assert softDelCount + delCount <= info.maxDoc();
|
||||
this.delCount = delCount;
|
||||
}
|
||||
|
||||
void setSoftDelCount(int softDelCount) {
|
||||
if (softDelCount < 0 || softDelCount > info.maxDoc()) {
|
||||
throw new IllegalArgumentException("invalid softDelCount=" + softDelCount + " (maxDoc=" + info.maxDoc() + ")");
|
||||
}
|
||||
assert softDelCount + delCount <= info.maxDoc();
|
||||
this.softDelCount = softDelCount;
|
||||
}
|
||||
|
||||
/** Returns a description of this segment. */
|
||||
public String toString(int pendingDelCount) {
|
||||
String s = info.toString(delCount + pendingDelCount);
|
||||
|
@ -332,6 +352,10 @@ public class SegmentCommitInfo {
|
|||
if (docValuesGen != -1) {
|
||||
s += ":dvGen=" + docValuesGen;
|
||||
}
|
||||
if (softDelCount > 0) {
|
||||
s += " :softDel=" + softDelCount;
|
||||
}
|
||||
|
||||
return s;
|
||||
}
|
||||
|
||||
|
@ -342,7 +366,7 @@ public class SegmentCommitInfo {
|
|||
|
||||
@Override
|
||||
public SegmentCommitInfo clone() {
|
||||
SegmentCommitInfo other = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen, docValuesGen);
|
||||
SegmentCommitInfo other = new SegmentCommitInfo(info, delCount, softDelCount, delGen, fieldInfosGen, docValuesGen);
|
||||
// Not clear that we need to carry over nextWriteDelGen
|
||||
// (i.e. do we ever clone after a failed write and
|
||||
// before the next successful write?), but just do it to
|
||||
|
@ -360,4 +384,8 @@ public class SegmentCommitInfo {
|
|||
|
||||
return other;
|
||||
}
|
||||
|
||||
final int getDelCount(boolean includeSoftDeletes) {
|
||||
return includeSoftDeletes ? getDelCount() + getSoftDelCount() : getDelCount();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -122,8 +122,9 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
public static final int VERSION_70 = 7;
|
||||
/** The version that updated segment name counter to be long instead of int. */
|
||||
public static final int VERSION_72 = 8;
|
||||
|
||||
static final int VERSION_CURRENT = VERSION_72;
|
||||
/** The version that recorded softDelCount */
|
||||
public static final int VERSION_74 = 9;
|
||||
static final int VERSION_CURRENT = VERSION_74;
|
||||
|
||||
/** Used to name new segments. */
|
||||
public long counter;
|
||||
|
@ -359,7 +360,14 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
}
|
||||
long fieldInfosGen = input.readLong();
|
||||
long dvGen = input.readLong();
|
||||
SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen, dvGen);
|
||||
int softDelCount = format > VERSION_72 ? input.readInt() : 0;
|
||||
if (softDelCount < 0 || softDelCount > info.maxDoc()) {
|
||||
throw new CorruptIndexException("invalid deletion count: " + softDelCount + " vs maxDoc=" + info.maxDoc(), input);
|
||||
}
|
||||
if (softDelCount + delCount > info.maxDoc()) {
|
||||
throw new CorruptIndexException("invalid deletion count: " + softDelCount + delCount + " vs maxDoc=" + info.maxDoc(), input);
|
||||
}
|
||||
SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, softDelCount, delGen, fieldInfosGen, dvGen);
|
||||
siPerCommit.setFieldInfosFiles(input.readSetOfStrings());
|
||||
final Map<Integer,Set<String>> dvUpdateFiles;
|
||||
final int numDVFields = input.readInt();
|
||||
|
@ -517,6 +525,11 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
out.writeInt(delCount);
|
||||
out.writeLong(siPerCommit.getFieldInfosGen());
|
||||
out.writeLong(siPerCommit.getDocValuesGen());
|
||||
int softDelCount = siPerCommit.getSoftDelCount();
|
||||
if (softDelCount < 0 || softDelCount > si.maxDoc()) {
|
||||
throw new IllegalStateException("cannot write segment: invalid maxDoc segment=" + si.name + " maxDoc=" + si.maxDoc() + " softDelCount=" + softDelCount);
|
||||
}
|
||||
out.writeInt(softDelCount);
|
||||
out.writeSetOfStrings(siPerCommit.getFieldInfosFiles());
|
||||
final Map<Integer,Set<String>> dvUpdatesFiles = siPerCommit.getDocValuesUpdatesFiles();
|
||||
out.writeInt(dvUpdatesFiles.size());
|
||||
|
|
|
@ -71,6 +71,7 @@ final class SegmentMerger {
|
|||
if (minVersion.onOrAfter(leafMinVersion)) {
|
||||
minVersion = leafMinVersion;
|
||||
}
|
||||
|
||||
}
|
||||
assert segmentInfo.minVersion == null : "The min version should be set by SegmentMerger for merged segments";
|
||||
segmentInfo.minVersion = minVersion;
|
||||
|
|
|
@ -47,7 +47,9 @@ public class SegmentWriteState {
|
|||
/** Number of deleted documents set while flushing the
|
||||
* segment. */
|
||||
public int delCountOnFlush;
|
||||
|
||||
/** Number of only soft deleted documents set while flushing the
|
||||
* segment. */
|
||||
public int softDelCountOnFlush;
|
||||
/**
|
||||
* Deletes and updates to apply while we are flushing the segment. A Term is
|
||||
* enrolled in here if it was deleted/updated at one point, and it's mapped to
|
||||
|
|
|
@ -62,6 +62,8 @@ public final class SoftDeletesDirectoryReaderWrapper extends FilterDirectoryRead
|
|||
// we try to reuse the life docs instances here if the reader cache key didn't change
|
||||
if (reader instanceof SoftDeletesFilterLeafReader && reader.getReaderCacheHelper() != null) {
|
||||
readerCache.put(((SoftDeletesFilterLeafReader) reader).reader.getReaderCacheHelper().getKey(), reader);
|
||||
} else if (reader instanceof SoftDeletesFilterCodecReader && reader.getReaderCacheHelper() != null) {
|
||||
readerCache.put(((SoftDeletesFilterCodecReader) reader).reader.getReaderCacheHelper().getKey(), reader);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -112,9 +114,35 @@ public final class SoftDeletesDirectoryReaderWrapper extends FilterDirectoryRead
|
|||
bits = new FixedBitSet(reader.maxDoc());
|
||||
bits.set(0, reader.maxDoc());
|
||||
}
|
||||
int numDeletes = reader.numDeletedDocs() + PendingSoftDeletes.applySoftDeletes(iterator, bits);
|
||||
int numSoftDeletes = PendingSoftDeletes.applySoftDeletes(iterator, bits);
|
||||
int numDeletes = reader.numDeletedDocs() + numSoftDeletes;
|
||||
int numDocs = reader.maxDoc() - numDeletes;
|
||||
return new SoftDeletesFilterLeafReader(reader, bits, numDocs);
|
||||
assert assertDocCounts(numDocs, numSoftDeletes, reader);
|
||||
return reader instanceof CodecReader ? new SoftDeletesFilterCodecReader((CodecReader) reader, bits, numDocs)
|
||||
: new SoftDeletesFilterLeafReader(reader, bits, numDocs);
|
||||
}
|
||||
|
||||
private static boolean assertDocCounts(int expectedNumDocs, int numSoftDeletes, LeafReader reader) {
|
||||
if (reader instanceof SegmentReader) {
|
||||
SegmentReader segmentReader = (SegmentReader) reader;
|
||||
SegmentCommitInfo segmentInfo = segmentReader.getSegmentInfo();
|
||||
if (segmentReader.isNRT == false) {
|
||||
int numDocs = segmentInfo.info.maxDoc() - segmentInfo.getSoftDelCount() - segmentInfo.getDelCount();
|
||||
assert numDocs == expectedNumDocs : "numDocs: " + numDocs + " expected: " + expectedNumDocs
|
||||
+ " maxDoc: " + segmentInfo.info.maxDoc()
|
||||
+ " getDelCount: " + segmentInfo.getDelCount()
|
||||
+ " getSoftDelCount: " + segmentInfo.getSoftDelCount()
|
||||
+ " numSoftDeletes: " + numSoftDeletes
|
||||
+ " reader.numDeletedDocs(): " + reader.numDeletedDocs();
|
||||
}
|
||||
// in the NRT case we don't have accurate numbers for getDelCount and getSoftDelCount since they might not be
|
||||
// flushed to disk when this reader is opened. We don't necessarily flush deleted doc on reopen but
|
||||
// we do for docValues.
|
||||
|
||||
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
static final class SoftDeletesFilterLeafReader extends FilterLeafReader {
|
||||
|
@ -153,6 +181,42 @@ public final class SoftDeletesDirectoryReaderWrapper extends FilterDirectoryRead
|
|||
}
|
||||
}
|
||||
|
||||
final static class SoftDeletesFilterCodecReader extends FilterCodecReader {
|
||||
private final LeafReader reader;
|
||||
private final FixedBitSet bits;
|
||||
private final int numDocs;
|
||||
private final CacheHelper readerCacheHelper;
|
||||
|
||||
private SoftDeletesFilterCodecReader(CodecReader reader, FixedBitSet bits, int numDocs) {
|
||||
super(reader);
|
||||
this.reader = reader;
|
||||
this.bits = bits;
|
||||
this.numDocs = numDocs;
|
||||
this.readerCacheHelper = reader.getReaderCacheHelper() == null ? null :
|
||||
new DelegatingCacheHelper(reader.getReaderCacheHelper());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getLiveDocs() {
|
||||
return bits;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numDocs() {
|
||||
return numDocs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return reader.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return readerCacheHelper;
|
||||
}
|
||||
}
|
||||
|
||||
private static class DelegatingCacheHelper implements CacheHelper {
|
||||
private final CacheHelper delegate;
|
||||
private final CacheKey cacheKey = new CacheKey();
|
||||
|
|
|
@ -175,7 +175,7 @@ public final class SoftDeletesRetentionMergePolicy extends OneMergeWrappingMerge
|
|||
@Override
|
||||
public int numDeletesToMerge(SegmentCommitInfo info, int delCount, IOSupplier<CodecReader> readerSupplier) throws IOException {
|
||||
final int numDeletesToMerge = super.numDeletesToMerge(info, delCount, readerSupplier);
|
||||
if (numDeletesToMerge != 0) {
|
||||
if (numDeletesToMerge != 0 && info.getSoftDelCount() > 0) {
|
||||
final CodecReader reader = readerSupplier.get();
|
||||
if (reader.getLiveDocs() != null) {
|
||||
BooleanQuery.Builder builder = new BooleanQuery.Builder();
|
||||
|
|
|
@ -244,4 +244,9 @@ class SortedDocValuesWriter extends DocValuesWriter {
|
|||
return valueCount;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
DocIdSetIterator getDocIdSet() {
|
||||
return docsWithField.iterator();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -231,4 +231,9 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
|
|||
return docsWithField.cost();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
DocIdSetIterator getDocIdSet() {
|
||||
return docsWithField.iterator();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -315,4 +315,8 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
|
|||
return scratch;
|
||||
}
|
||||
}
|
||||
@Override
|
||||
DocIdSetIterator getDocIdSet() {
|
||||
return docsWithField.iterator();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1410,4 +1410,50 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
dir1.close();
|
||||
dir2.close();
|
||||
}
|
||||
|
||||
public void testAddIndicesWithSoftDeletes() throws IOException {
|
||||
Directory dir1 = newDirectory();
|
||||
IndexWriterConfig iwc1 = newIndexWriterConfig(new MockAnalyzer(random())).setSoftDeletesField("soft_delete");
|
||||
IndexWriter writer = new IndexWriter(dir1, iwc1);
|
||||
for (int i = 0; i < 30; i++) {
|
||||
Document doc = new Document();
|
||||
int docID = random().nextInt(5);
|
||||
doc.add(new StringField("id", "" + docID, Field.Store.YES));
|
||||
writer.softUpdateDocument(new Term("id", "" + docID), doc, new NumericDocValuesField("soft_delete", 1));
|
||||
if (random().nextBoolean()) {
|
||||
writer.flush();
|
||||
}
|
||||
}
|
||||
writer.commit();
|
||||
writer.close();
|
||||
DirectoryReader reader = DirectoryReader.open(dir1);
|
||||
DirectoryReader wrappedReader = new SoftDeletesDirectoryReaderWrapper(reader, "soft_delete");
|
||||
Directory dir2 = newDirectory();
|
||||
int numDocs = reader.numDocs();
|
||||
int maxDoc = reader.maxDoc();
|
||||
assertEquals(numDocs, maxDoc);
|
||||
iwc1 = newIndexWriterConfig(new MockAnalyzer(random())).setSoftDeletesField("soft_delete");
|
||||
writer = new IndexWriter(dir2, iwc1);
|
||||
CodecReader[] readers = new CodecReader[reader.leaves().size()];
|
||||
for (int i = 0; i < readers.length; i++) {
|
||||
readers[i] = (CodecReader)reader.leaves().get(i).reader();
|
||||
}
|
||||
writer.addIndexes(readers);
|
||||
assertEquals(wrappedReader.numDocs(), writer.numDocs());
|
||||
assertEquals(maxDoc, writer.maxDoc());
|
||||
writer.commit();
|
||||
SegmentCommitInfo commitInfo = writer.segmentInfos.asList().get(0);
|
||||
assertEquals(maxDoc-wrappedReader.numDocs(), commitInfo.getSoftDelCount());
|
||||
writer.close();
|
||||
Directory dir3 = newDirectory();
|
||||
iwc1 = newIndexWriterConfig(new MockAnalyzer(random())).setSoftDeletesField("soft_delete");
|
||||
writer = new IndexWriter(dir3, iwc1);
|
||||
for (int i = 0; i < readers.length; i++) {
|
||||
readers[i] = (CodecReader)wrappedReader.leaves().get(i).reader();
|
||||
}
|
||||
writer.addIndexes(readers);
|
||||
assertEquals(wrappedReader.numDocs(), writer.numDocs());
|
||||
assertEquals(wrappedReader.numDocs(), writer.maxDoc());
|
||||
IOUtils.close(reader, writer, dir3, dir2, dir1);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -238,7 +238,7 @@ public class TestDoc extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
return new SegmentCommitInfo(si, 0, -1L, -1L, -1L);
|
||||
return new SegmentCommitInfo(si, 0, 0, -1L, -1L, -1L);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -3137,7 +3137,11 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
searcher = new IndexSearcher(reader);
|
||||
topDocs = searcher.search(new TermQuery(new Term("id", "1")), 10);
|
||||
assertEquals(0, topDocs.totalHits);
|
||||
|
||||
int numSoftDeleted = 0;
|
||||
for (SegmentCommitInfo info : writer.segmentInfos) {
|
||||
numSoftDeleted += info.getSoftDelCount();
|
||||
}
|
||||
assertEquals(writer.maxDoc() - writer.numDocs(), numSoftDeleted);
|
||||
writer.close();
|
||||
reader.close();
|
||||
dir.close();
|
||||
|
@ -3267,6 +3271,20 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
assertEquals(1, reader.docFreq(new Term("id", id)));
|
||||
}
|
||||
}
|
||||
int numSoftDeleted = 0;
|
||||
for (SegmentCommitInfo info : writer.segmentInfos) {
|
||||
numSoftDeleted += info.getSoftDelCount() + info.getDelCount();
|
||||
}
|
||||
assertEquals(writer.maxDoc() - writer.numDocs(), numSoftDeleted);
|
||||
writer.commit();
|
||||
try (DirectoryReader dirReader = DirectoryReader.open(dir)) {
|
||||
int delCount = 0;
|
||||
for (LeafReaderContext ctx : dirReader.leaves()) {
|
||||
SegmentCommitInfo segmentInfo = ((SegmentReader) ctx.reader()).getSegmentInfo();
|
||||
delCount += segmentInfo.getSoftDelCount() + segmentInfo.getDelCount();
|
||||
}
|
||||
assertEquals(numSoftDeleted, delCount);
|
||||
}
|
||||
IOUtils.close(reader, writer, dir);
|
||||
}
|
||||
|
||||
|
|
|
@ -331,7 +331,7 @@ public class TestIndexWriterThreadsToSegments extends LuceneTestCase {
|
|||
byte id[] = readSegmentInfoID(dir, fileName);
|
||||
SegmentInfo si = TestUtil.getDefaultCodec().segmentInfoFormat().read(dir, segName, id, IOContext.DEFAULT);
|
||||
si.setCodec(codec);
|
||||
SegmentCommitInfo sci = new SegmentCommitInfo(si, 0, -1, -1, -1);
|
||||
SegmentCommitInfo sci = new SegmentCommitInfo(si, 0, 0, -1, -1, -1);
|
||||
SegmentReader sr = new SegmentReader(sci, Version.LATEST.major, IOContext.DEFAULT);
|
||||
try {
|
||||
thread0Count += sr.docFreq(new Term("field", "threadID0"));
|
||||
|
|
|
@ -137,7 +137,7 @@ public class TestOneMergeWrappingMergePolicy extends LuceneTestCase {
|
|||
Collections.emptyMap(), // attributes
|
||||
null /* indexSort */);
|
||||
final List<SegmentCommitInfo> segments = new LinkedList<SegmentCommitInfo>();
|
||||
segments.add(new SegmentCommitInfo(si, 0, 0, 0, 0));
|
||||
segments.add(new SegmentCommitInfo(si, 0, 0, 0, 0, 0));
|
||||
ms.add(new MergePolicy.OneMerge(segments));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,7 +40,7 @@ public class TestPendingDeletes extends LuceneTestCase {
|
|||
RAMDirectory dir = new RAMDirectory();
|
||||
SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 10, false, Codec.getDefault(),
|
||||
Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, -1, -1, -1);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, 0, -1, -1, -1);
|
||||
PendingDeletes deletes = newPendingDeletes(commitInfo);
|
||||
assertNull(deletes.getLiveDocs());
|
||||
int docToDelete = TestUtil.nextInt(random(), 0, 7);
|
||||
|
@ -74,7 +74,7 @@ public class TestPendingDeletes extends LuceneTestCase {
|
|||
RAMDirectory dir = new RAMDirectory();
|
||||
SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 6, false, Codec.getDefault(),
|
||||
Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, -1, -1, -1);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, 0, -1, -1, -1);
|
||||
PendingDeletes deletes = newPendingDeletes(commitInfo);
|
||||
assertFalse(deletes.writeLiveDocs(dir));
|
||||
assertEquals(0, dir.listAll().length);
|
||||
|
@ -131,7 +131,7 @@ public class TestPendingDeletes extends LuceneTestCase {
|
|||
RAMDirectory dir = new RAMDirectory();
|
||||
SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 3, false, Codec.getDefault(),
|
||||
Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, -1, -1, -1);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, 0, -1, -1, -1);
|
||||
FieldInfos fieldInfos = new FieldInfos(new FieldInfo[0]);
|
||||
si.getCodec().fieldInfosFormat().write(dir, si, "", fieldInfos, IOContext.DEFAULT);
|
||||
PendingDeletes deletes = newPendingDeletes(commitInfo);
|
||||
|
|
|
@ -44,6 +44,45 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
return new PendingSoftDeletes("_soft_deletes", commitInfo);
|
||||
}
|
||||
|
||||
public void testHardDeleteSoftDeleted() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()
|
||||
.setSoftDeletesField("_soft_deletes")
|
||||
// make sure all docs will end up in the same segment
|
||||
.setMaxBufferedDocs(10)
|
||||
.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();
|
||||
PendingSoftDeletes pendingSoftDeletes = newPendingDeletes(segmentInfo);
|
||||
pendingSoftDeletes.onNewReader(segmentReader, segmentInfo);
|
||||
assertEquals(0, pendingSoftDeletes.numPendingDeletes());
|
||||
assertEquals(1, pendingSoftDeletes.getDelCount());
|
||||
assertTrue(pendingSoftDeletes.getLiveDocs().get(0));
|
||||
assertFalse(pendingSoftDeletes.getLiveDocs().get(1));
|
||||
assertTrue(pendingSoftDeletes.getLiveDocs().get(2));
|
||||
assertNull(pendingSoftDeletes.getHardLiveDocs());
|
||||
assertTrue(pendingSoftDeletes.delete(1));
|
||||
assertEquals(0, pendingSoftDeletes.numPendingDeletes());
|
||||
assertEquals(-1, pendingSoftDeletes.pendingDeleteCount); // transferred the delete
|
||||
assertEquals(1, pendingSoftDeletes.getDelCount());
|
||||
IOUtils.close(reader, writer, dir);
|
||||
}
|
||||
|
||||
public void testDeleteSoft() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()
|
||||
|
@ -70,7 +109,8 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
SegmentCommitInfo segmentInfo = segmentReader.getSegmentInfo();
|
||||
PendingSoftDeletes pendingSoftDeletes = newPendingDeletes(segmentInfo);
|
||||
pendingSoftDeletes.onNewReader(segmentReader, segmentInfo);
|
||||
assertEquals(1, pendingSoftDeletes.numPendingDeletes());
|
||||
assertEquals(0, pendingSoftDeletes.numPendingDeletes());
|
||||
assertEquals(1, pendingSoftDeletes.getDelCount());
|
||||
assertTrue(pendingSoftDeletes.getLiveDocs().get(0));
|
||||
assertFalse(pendingSoftDeletes.getLiveDocs().get(1));
|
||||
assertTrue(pendingSoftDeletes.getLiveDocs().get(2));
|
||||
|
@ -78,7 +118,8 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
// pass reader again
|
||||
Bits liveDocs = pendingSoftDeletes.getLiveDocs();
|
||||
pendingSoftDeletes.onNewReader(segmentReader, segmentInfo);
|
||||
assertEquals(1, pendingSoftDeletes.numPendingDeletes());
|
||||
assertEquals(0, pendingSoftDeletes.numPendingDeletes());
|
||||
assertEquals(1, pendingSoftDeletes.getDelCount());
|
||||
assertSame(liveDocs, pendingSoftDeletes.getLiveDocs());
|
||||
|
||||
// now apply a hard delete
|
||||
|
@ -91,7 +132,8 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
segmentInfo = segmentReader.getSegmentInfo();
|
||||
pendingSoftDeletes = newPendingDeletes(segmentInfo);
|
||||
pendingSoftDeletes.onNewReader(segmentReader, segmentInfo);
|
||||
assertEquals(1, pendingSoftDeletes.numPendingDeletes());
|
||||
assertEquals(0, pendingSoftDeletes.numPendingDeletes());
|
||||
assertEquals(2, pendingSoftDeletes.getDelCount());
|
||||
assertFalse(pendingSoftDeletes.getLiveDocs().get(0));
|
||||
assertFalse(pendingSoftDeletes.getLiveDocs().get(1));
|
||||
assertTrue(pendingSoftDeletes.getLiveDocs().get(2));
|
||||
|
@ -106,7 +148,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
RAMDirectory dir = new RAMDirectory();
|
||||
SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 10, false, Codec.getDefault(),
|
||||
Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, -1, -1, -1);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, 0, -1, -1, -1);
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig());
|
||||
for (int i = 0; i < si.maxDoc(); i++) {
|
||||
writer.addDocument(new Document());
|
||||
|
@ -126,7 +168,8 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
for (DocValuesFieldUpdates update : updates) {
|
||||
deletes.onDocValuesUpdate(fieldInfo, update.iterator());
|
||||
}
|
||||
assertEquals(4, deletes.numPendingDeletes());
|
||||
assertEquals(0, deletes.numPendingDeletes());
|
||||
assertEquals(4, deletes.getDelCount());
|
||||
assertTrue(deletes.getLiveDocs().get(0));
|
||||
assertFalse(deletes.getLiveDocs().get(1));
|
||||
assertTrue(deletes.getLiveDocs().get(2));
|
||||
|
@ -144,7 +187,8 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
for (DocValuesFieldUpdates update : updates) {
|
||||
deletes.onDocValuesUpdate(fieldInfo, update.iterator());
|
||||
}
|
||||
assertEquals(5, deletes.numPendingDeletes());
|
||||
assertEquals(0, deletes.numPendingDeletes());
|
||||
assertEquals(5, deletes.getDelCount());
|
||||
assertTrue(deletes.getLiveDocs().get(0));
|
||||
assertFalse(deletes.getLiveDocs().get(1));
|
||||
assertFalse(deletes.getLiveDocs().get(2));
|
||||
|
@ -188,7 +232,8 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
for (DocValuesFieldUpdates update : updates) {
|
||||
deletes.onDocValuesUpdate(fieldInfo, update.iterator());
|
||||
}
|
||||
assertEquals(1, deletes.numPendingDeletes());
|
||||
assertEquals(0, deletes.numPendingDeletes());
|
||||
assertEquals(1, deletes.getDelCount());
|
||||
assertTrue(deletes.getLiveDocs().get(0));
|
||||
assertFalse(deletes.getLiveDocs().get(1));
|
||||
assertTrue(deletes.getLiveDocs().get(2));
|
||||
|
@ -199,7 +244,8 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
assertTrue(deletes.getLiveDocs().get(0));
|
||||
assertFalse(deletes.getLiveDocs().get(1));
|
||||
assertTrue(deletes.getLiveDocs().get(2));
|
||||
assertEquals(1, deletes.numPendingDeletes());
|
||||
assertEquals(0, deletes.numPendingDeletes());
|
||||
assertEquals(1, deletes.getDelCount());
|
||||
IOUtils.close(reader, writer, dir);
|
||||
}
|
||||
|
||||
|
@ -257,7 +303,8 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
assertTrue(deletes.getLiveDocs().get(0));
|
||||
assertFalse(deletes.getLiveDocs().get(1));
|
||||
assertTrue(deletes.getLiveDocs().get(2));
|
||||
assertEquals(1, deletes.numPendingDeletes());
|
||||
assertEquals(0, deletes.numPendingDeletes());
|
||||
assertEquals(1, deletes.getDelCount());
|
||||
IOUtils.close(reader, writer, dir);
|
||||
|
||||
}
|
||||
|
|
|
@ -60,7 +60,7 @@ public class TestSegmentInfos extends LuceneTestCase {
|
|||
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
|
||||
info.setFiles(Collections.<String>emptySet());
|
||||
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(info, 0, -1, -1, -1);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(info, 0, 0, -1, -1, -1);
|
||||
|
||||
sis.add(commitInfo);
|
||||
sis.commit(dir);
|
||||
|
@ -82,14 +82,14 @@ public class TestSegmentInfos extends LuceneTestCase {
|
|||
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
|
||||
info.setFiles(Collections.<String>emptySet());
|
||||
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(info, 0, -1, -1, -1);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(info, 0, 0, -1, -1, -1);
|
||||
sis.add(commitInfo);
|
||||
|
||||
info = new SegmentInfo(dir, Version.LUCENE_8_0_0, Version.LUCENE_8_0_0, "_1", 1, false, Codec.getDefault(),
|
||||
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
|
||||
info.setFiles(Collections.<String>emptySet());
|
||||
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
|
||||
commitInfo = new SegmentCommitInfo(info, 0, -1, -1, -1);
|
||||
commitInfo = new SegmentCommitInfo(info, 0, 0,-1, -1, -1);
|
||||
sis.add(commitInfo);
|
||||
|
||||
sis.commit(dir);
|
||||
|
|
|
@ -96,7 +96,7 @@ public class TestSegmentMerger extends LuceneTestCase {
|
|||
//Should be able to open a new SegmentReader against the new directory
|
||||
SegmentReader mergedReader = new SegmentReader(new SegmentCommitInfo(
|
||||
mergeState.segmentInfo,
|
||||
0, -1L, -1L, -1L),
|
||||
0, 0, -1L, -1L, -1L),
|
||||
Version.LATEST.major,
|
||||
newIOContext(random()));
|
||||
assertTrue(mergedReader != null);
|
||||
|
|
|
@ -104,7 +104,8 @@ public class TestSoftDeletesDirectoryReaderWrapper extends LuceneTestCase {
|
|||
}
|
||||
|
||||
private boolean isWrapped(LeafReader reader) {
|
||||
return reader instanceof SoftDeletesDirectoryReaderWrapper.SoftDeletesFilterLeafReader;
|
||||
return reader instanceof SoftDeletesDirectoryReaderWrapper.SoftDeletesFilterLeafReader
|
||||
|| reader instanceof SoftDeletesDirectoryReaderWrapper.SoftDeletesFilterCodecReader;
|
||||
}
|
||||
|
||||
public void testMixSoftAndHardDeletes() throws IOException {
|
||||
|
|
|
@ -141,7 +141,7 @@ public class IndexSplitter {
|
|||
// Same info just changing the dir:
|
||||
SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.getMinVersion(), info.name, info.maxDoc(),
|
||||
info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics(), info.getId(), new HashMap<>(), null);
|
||||
destInfos.add(new SegmentCommitInfo(newInfo, infoPerCommit.getDelCount(),
|
||||
destInfos.add(new SegmentCommitInfo(newInfo, infoPerCommit.getDelCount(), infoPerCommit.getSoftDelCount(),
|
||||
infoPerCommit.getDelGen(), infoPerCommit.getFieldInfosGen(),
|
||||
infoPerCommit.getDocValuesGen()));
|
||||
// now copy files over
|
||||
|
|
|
@ -125,10 +125,10 @@ public abstract class BaseLiveDocsFormatTestCase extends LuceneTestCase {
|
|||
final Directory dir = newDirectory();
|
||||
final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "foo", maxDoc, random().nextBoolean(),
|
||||
codec, Collections.emptyMap(), StringHelper.randomId(), Collections.emptyMap(), null);
|
||||
SegmentCommitInfo sci = new SegmentCommitInfo(si, 0, 0, -1, -1);
|
||||
SegmentCommitInfo sci = new SegmentCommitInfo(si, 0, 0, 0, -1, -1);
|
||||
format.writeLiveDocs(bits, dir, sci, maxDoc - numLiveDocs, IOContext.DEFAULT);
|
||||
|
||||
sci = new SegmentCommitInfo(si, maxDoc - numLiveDocs, 1, -1, -1);
|
||||
sci = new SegmentCommitInfo(si, maxDoc - numLiveDocs, 0, 1, -1, -1);
|
||||
final Bits bits2 = format.readLiveDocs(dir, sci, IOContext.READONCE);
|
||||
assertEquals(maxDoc, bits2.length());
|
||||
for (int i = 0; i < maxDoc; ++i) {
|
||||
|
|
|
@ -116,7 +116,7 @@ public abstract class BaseMergePolicyTestCase extends LuceneTestCase {
|
|||
Collections.emptyMap(), // attributes
|
||||
null /* indexSort */);
|
||||
info.setFiles(Collections.emptyList());
|
||||
infos.add(new SegmentCommitInfo(info, random().nextInt(1), -1, -1, -1));
|
||||
infos.add(new SegmentCommitInfo(info, random().nextInt(1), 0, -1, -1, -1));
|
||||
}
|
||||
MergePolicy.MergeSpecification forcedDeletesMerges = mp.findForcedDeletesMerges(infos, context);
|
||||
if (forcedDeletesMerges != null) {
|
||||
|
|
Loading…
Reference in New Issue