LUCENE-8253: Account for soft-deletes before they are flushed to disk

Inside the IndexWriter buffers are only written to disk if it's needed
or "worth it" which doesn't guarantee soft deletes to be accounted
in time. This is not necessarily a problem since they are eventually
collected and segments that have soft-deletes will me merged eventually
but for tests and on par behavior compared to hard deletes this behavior
is tricky.
This change cuts over to accounting in-place just like hard-deletes. This
results in accurate delete numbers for soft deletes at any give point in time
once the reader is loaded or a pending soft delete occurs.

This change also fixes an issue where all updates to a DV field are allowed
event if the field is unknown. Now this only works if the field is equal
to the soft deletes field. This behavior was never released.
This commit is contained in:
Simon Willnauer 2018-04-16 16:16:43 +02:00
parent 1e759bc505
commit c70cceaee5
9 changed files with 206 additions and 53 deletions

View File

@ -1594,7 +1594,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
if (softDeletes == null || softDeletes.length == 0) {
throw new IllegalArgumentException("at least one soft delete must be present");
}
return updateDocuments(DocumentsWriterDeleteQueue.newNode(buildDocValuesUpdate(term, softDeletes, false)), docs);
return updateDocuments(DocumentsWriterDeleteQueue.newNode(buildDocValuesUpdate(term, softDeletes)), docs);
}
/** Expert: attempts to delete by document ID, as long as
@ -1831,7 +1831,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
if (softDeletes == null || softDeletes.length == 0) {
throw new IllegalArgumentException("at least one soft delete must be present");
}
return updateDocument(DocumentsWriterDeleteQueue.newNode(buildDocValuesUpdate(term, softDeletes, false)), doc);
return updateDocument(DocumentsWriterDeleteQueue.newNode(buildDocValuesUpdate(term, softDeletes)), doc);
}
@ -1940,7 +1940,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
*/
public long updateDocValues(Term term, Field... updates) throws IOException {
ensureOpen();
DocValuesUpdate[] dvUpdates = buildDocValuesUpdate(term, updates, true);
DocValuesUpdate[] dvUpdates = buildDocValuesUpdate(term, updates);
try {
long seqNo = docWriter.updateDocValues(dvUpdates);
if (seqNo < 0) {
@ -1954,7 +1954,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
}
private DocValuesUpdate[] buildDocValuesUpdate(Term term, Field[] updates, boolean enforceFieldExistence) {
private DocValuesUpdate[] buildDocValuesUpdate(Term term, Field[] updates) {
DocValuesUpdate[] dvUpdates = new DocValuesUpdate[updates.length];
for (int i = 0; i < updates.length; i++) {
final Field f = updates[i];
@ -1965,7 +1965,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
if (dvType == DocValuesType.NONE) {
throw new IllegalArgumentException("can only update NUMERIC or BINARY fields! field=" + f.name());
}
if (enforceFieldExistence && !globalFieldNumberMap.contains(f.name(), dvType)) {
if (!globalFieldNumberMap.contains(f.name(), dvType) && f.name().equals(config.softDeletesField) == false) {
throw new IllegalArgumentException("can only update existing docvalues fields! field=" + f.name() + ", type=" + dvType);
}
if (config.getIndexSortFields().contains(f.name())) {
@ -5230,15 +5230,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
*/
public final int numDeletesToMerge(SegmentCommitInfo info) throws IOException {
MergePolicy mergePolicy = config.getMergePolicy();
final ReadersAndUpdates rld = readerPool.get(info, false);
int numDeletesToMerge;
if (rld != null) {
numDeletesToMerge = rld.numDeletesToMerge(mergePolicy);
} else {
numDeletesToMerge = mergePolicy.numDeletesToMerge(info, 0, null);
final ReadersAndUpdates rld = readerPool.get(info, true);
try {
int numDeletesToMerge = rld.numDeletesToMerge(mergePolicy);
assert numDeletesToMerge <= info.info.maxDoc() :
"numDeletesToMerge: " + numDeletesToMerge + " > maxDoc: " + info.info.maxDoc();
return numDeletesToMerge;
} finally {
readerPool.release(rld);
}
assert numDeletesToMerge <= info.info.maxDoc() :
"numDeletesToMerge: " + numDeletesToMerge + " > maxDoc: " + info.info.maxDoc();
return numDeletesToMerge;
}
}

View File

@ -18,7 +18,6 @@
package org.apache.lucene.index;
import java.io.IOException;
import java.util.List;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.LiveDocsFormat;
@ -26,6 +25,7 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.TrackingDirectoryWrapper;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.IOSupplier;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.MutableBits;
@ -135,7 +135,7 @@ class PendingDeletes {
/**
* Called once a new reader is opened for this segment ie. when deletes or updates are applied.
*/
void onNewReader(SegmentReader reader, SegmentCommitInfo info) throws IOException {
void onNewReader(CodecReader reader, SegmentCommitInfo info) throws IOException {
if (liveDocsInitialized == false) {
if (reader.hasDeletions()) {
// we only initialize this once either in the ctor or here
@ -235,10 +235,21 @@ class PendingDeletes {
}
/**
* Called before the given DocValuesFieldUpdates are applied
* Called before the given DocValuesFieldUpdates are written to disk
* @param info the field to apply
* @param fieldUpdates the field updates
*/
void onDocValuesUpdate(FieldInfo info, List<DocValuesFieldUpdates> fieldUpdates) throws IOException {
void onDocValuesUpdate(FieldInfo info) {
}
/**
* Called for every field update for the given field
* @param field the field that's updated
* @param iterator the values to apply
*/
void onDocValuesUpdate(String field, DocValuesFieldUpdates.Iterator iterator) throws IOException {
}
int numDeletesToMerge(MergePolicy policy, IOSupplier<CodecReader> readerIOSupplier) throws IOException {
return policy.numDeletesToMerge(info, numPendingDeletes(), readerIOSupplier);
}
}

View File

@ -17,11 +17,16 @@
package org.apache.lucene.index;
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.DocValuesFieldExistsQuery;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.IOSupplier;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.MutableBits;
final class PendingSoftDeletes extends PendingDeletes {
@ -64,14 +69,12 @@ final class PendingSoftDeletes extends PendingDeletes {
}
@Override
void onNewReader(SegmentReader reader, SegmentCommitInfo info) throws IOException {
void onNewReader(CodecReader reader, SegmentCommitInfo info) throws IOException {
super.onNewReader(reader, info);
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);
if (iterator == null) { // nothing is deleted we don't have a soft deletes field in this segment
this.pendingDeleteCount = 0;
} else {
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());
}
@ -117,15 +120,8 @@ final class PendingSoftDeletes extends PendingDeletes {
}
@Override
void onDocValuesUpdate(FieldInfo info, List<DocValuesFieldUpdates> updatesToApply) throws IOException {
if (field.equals(info.name)) {
assert dvGeneration < info.getDocValuesGen() : "we have seen this generation update already: " + dvGeneration + " vs. " + info.getDocValuesGen();
assert dvGeneration != -2 : "docValues generation is still uninitialized";
DocValuesFieldUpdates.Iterator[] subs = new DocValuesFieldUpdates.Iterator[updatesToApply.size()];
for(int i=0; i<subs.length; i++) {
subs[i] = updatesToApply.get(i).iterator();
}
DocValuesFieldUpdates.Iterator iterator = DocValuesFieldUpdates.mergedIterator(subs);
void onDocValuesUpdate(String field, DocValuesFieldUpdates.Iterator iterator) throws IOException {
if (this.field.equals(field)) {
pendingDeleteCount += applySoftDeletes(new DocIdSetIterator() {
int docID = -1;
@Override
@ -148,6 +144,13 @@ final class PendingSoftDeletes extends PendingDeletes {
throw new UnsupportedOperationException();
}
}, getMutableBits());
}
}
@Override
void onDocValuesUpdate(FieldInfo info) {
if (field.equals(info.name)) {
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();
}
}
@ -162,4 +165,48 @@ final class PendingSoftDeletes extends PendingDeletes {
sb.append(" hardDeletes=").append(hardDeletes);
return sb.toString();
}
@Override
int numDeletesToMerge(MergePolicy policy, IOSupplier<CodecReader> readerIOSupplier) throws IOException {
if (dvGeneration == -2) {
FieldInfos fieldInfos = readFieldInfos();
FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
// we try to only open a reader if it's really necessary ie. indices that are mainly append only might have
// big segments that don't even have any docs in the soft deletes field. In such a case it's simply
// enough to look at the FieldInfo for the field and check if the field has DocValues
if (fieldInfo != null && fieldInfo.getDocValuesType() != DocValuesType.NONE) {
// in order to get accurate numbers we need to have a least one reader see here.
onNewReader(readerIOSupplier.get(), info);
} else {
// we are safe here since we don't have any doc values for the soft-delete field on disk
// no need to open a new reader
dvGeneration = fieldInfo == null ? -1 : fieldInfo.getDocValuesGen();
}
}
return super.numDeletesToMerge(policy, readerIOSupplier);
}
private FieldInfos readFieldInfos() throws IOException {
SegmentInfo segInfo = info.info;
Directory dir = segInfo.dir;
if (info.hasFieldUpdates() == false) {
// updates always outside of CFS
Closeable toClose;
if (segInfo.getUseCompoundFile()) {
toClose = dir = segInfo.getCodec().compoundFormat().getCompoundReader(segInfo.dir, segInfo, IOContext.READONCE);
} else {
toClose = null;
dir = segInfo.dir;
}
try {
return segInfo.getCodec().fieldInfosFormat().read(dir, segInfo, "", IOContext.READONCE);
} finally {
IOUtils.close(toClose);
}
} else {
FieldInfosFormat fisFormat = segInfo.getCodec().fieldInfosFormat();
final String segmentSuffix = Long.toString(info.getFieldInfosGen(), Character.MAX_RADIX);
return fisFormat.read(dir, segInfo, segmentSuffix, IOContext.READONCE);
}
}
}

View File

@ -142,7 +142,7 @@ final class ReadersAndUpdates {
/** Adds a new resolved (meaning it maps docIDs to new values) doc values packet. We buffer these in RAM and write to disk when too much
* RAM is used or when a merge needs to kick off, or a commit/refresh. */
public synchronized void addDVUpdate(DocValuesFieldUpdates update) {
public synchronized void addDVUpdate(DocValuesFieldUpdates update) throws IOException {
if (update.getFinished() == false) {
throw new IllegalArgumentException("call finish first");
}
@ -166,6 +166,7 @@ final class ReadersAndUpdates {
}
fieldUpdates.add(update);
}
pendingDeletes.onDocValuesUpdate(update.field, update.iterator());
}
public synchronized long getNumDVUpdates() {
@ -271,10 +272,9 @@ final class ReadersAndUpdates {
}
return reader;
};
return policy.numDeletesToMerge(info, pendingDeletes.numPendingDeletes(), readerSupplier);
return pendingDeletes.numDeletesToMerge(policy, readerSupplier);
}
public synchronized Bits getLiveDocs() {
return pendingDeletes.getLiveDocs();
}
@ -344,7 +344,7 @@ final class ReadersAndUpdates {
final TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
final SegmentWriteState state = new SegmentWriteState(null, trackingDir, info.info, fieldInfos, null, updatesContext, segmentSuffix);
try (final DocValuesConsumer fieldsConsumer = dvFormat.fieldsConsumer(state)) {
pendingDeletes.onDocValuesUpdate(fieldInfo, updatesToApply);
pendingDeletes.onDocValuesUpdate(fieldInfo);
// write the numeric updates to a new gen'd docvalues file
fieldsConsumer.addNumericField(fieldInfo, new EmptyDocValuesProducer() {
@Override
@ -480,7 +480,7 @@ final class ReadersAndUpdates {
final SegmentWriteState state = new SegmentWriteState(null, trackingDir, info.info, fieldInfos, null, updatesContext, segmentSuffix);
try (final DocValuesConsumer fieldsConsumer = dvFormat.fieldsConsumer(state)) {
// write the binary updates to a new gen'd docvalues file
pendingDeletes.onDocValuesUpdate(fieldInfo, updatesToApply);
pendingDeletes.onDocValuesUpdate(fieldInfo);
fieldsConsumer.addBinaryField(fieldInfo, new EmptyDocValuesProducer() {
@Override
public BinaryDocValues getBinary(FieldInfo fieldInfoIn) throws IOException {

View File

@ -72,7 +72,8 @@ public final class SoftDeletesRetentionMergePolicy extends OneMergeWrappingMerge
@Override
public boolean keepFullyDeletedSegment(CodecReader reader) throws IOException {
Scorer scorer = getScorer(field, retentionQuerySupplier.get(), wrapLiveDocs(reader, null, reader.maxDoc()));
/* we only need a single hit to keep it no need for soft deletes to be checked*/
Scorer scorer = getScorer(retentionQuerySupplier.get(), wrapLiveDocs(reader, null, reader.maxDoc()));
if (scorer != null) {
DocIdSetIterator iterator = scorer.iterator();
boolean atLeastOneHit = iterator.nextDoc() != DocIdSetIterator.NO_MORE_DOCS;
@ -98,7 +99,10 @@ public final class SoftDeletesRetentionMergePolicy extends OneMergeWrappingMerge
return liveDocs.length();
}
}, reader.maxDoc() - reader.numDocs());
Scorer scorer = getScorer(softDeleteField, retentionQuery, wrappedReader);
BooleanQuery.Builder builder = new BooleanQuery.Builder();
builder.add(new DocValuesFieldExistsQuery(softDeleteField), BooleanClause.Occur.FILTER);
builder.add(retentionQuery, BooleanClause.Occur.FILTER);
Scorer scorer = getScorer(builder.build(), wrappedReader);
if (scorer != null) {
FixedBitSet cloneLiveDocs = cloneLiveDocs(liveDocs);
DocIdSetIterator iterator = scorer.iterator();
@ -133,13 +137,10 @@ public final class SoftDeletesRetentionMergePolicy extends OneMergeWrappingMerge
}
}
private static Scorer getScorer(String softDeleteField, Query retentionQuery, CodecReader reader) throws IOException {
BooleanQuery.Builder builder = new BooleanQuery.Builder();
builder.add(new DocValuesFieldExistsQuery(softDeleteField), BooleanClause.Occur.FILTER);
builder.add(retentionQuery, BooleanClause.Occur.FILTER);
private static Scorer getScorer(Query query, CodecReader reader) throws IOException {
IndexSearcher s = new IndexSearcher(reader);
s.setQueryCache(null);
Weight weight = s.createWeight(builder.build(), ScoreMode.COMPLETE_NO_SCORES, 1.0f);
Weight weight = s.createWeight(query, ScoreMode.COMPLETE_NO_SCORES, 1.0f);
return weight.scorer(reader.getContext());
}
@ -172,11 +173,14 @@ public final class SoftDeletesRetentionMergePolicy extends OneMergeWrappingMerge
@Override
public int numDeletesToMerge(SegmentCommitInfo info, int pendingDeleteCount, IOSupplier<CodecReader> readerSupplier) throws IOException {
int numDeletesToMerge = super.numDeletesToMerge(info, pendingDeleteCount, readerSupplier);
final int numDeletesToMerge = super.numDeletesToMerge(info, pendingDeleteCount, readerSupplier);
if (numDeletesToMerge != 0) {
final CodecReader reader = readerSupplier.get();
if (reader.getLiveDocs() != null) {
Scorer scorer = getScorer(field, retentionQuerySupplier.get(), wrapLiveDocs(reader, null, reader.maxDoc()));
BooleanQuery.Builder builder = new BooleanQuery.Builder();
builder.add(new DocValuesFieldExistsQuery(field), BooleanClause.Occur.FILTER);
builder.add(retentionQuerySupplier.get(), BooleanClause.Occur.FILTER);
Scorer scorer = getScorer(builder.build(), wrapLiveDocs(reader, null, reader.maxDoc()));
if (scorer != null) {
DocIdSetIterator iterator = scorer.iterator();
Bits liveDocs = reader.getLiveDocs();

View File

@ -45,7 +45,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
public void testDeleteSoft() throws IOException {
Directory dir = newDirectory();
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); // no soft delete field here
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig().setSoftDeletesField("_soft_deletes"));
Document doc = new Document();
doc.add(new StringField("id", "1", Field.Store.YES));
writer.softUpdateDocument(new Term("id", "1"), doc,
@ -114,7 +114,10 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
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<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 10));
deletes.onDocValuesUpdate(fieldInfo, updates);
for (DocValuesFieldUpdates update : updates) {
deletes.onDocValuesUpdate(update.field, update.iterator());
}
deletes.onDocValuesUpdate(fieldInfo);
assertEquals(4, deletes.numPendingDeletes());
assertTrue(deletes.getLiveDocs().get(0));
assertFalse(deletes.getLiveDocs().get(1));
@ -130,7 +133,10 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
docsDeleted = Arrays.asList(1, 2, DocIdSetIterator.NO_MORE_DOCS);
updates = Arrays.asList(singleUpdate(docsDeleted, 10));
fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 1, Collections.emptyMap(), 0, 0);
deletes.onDocValuesUpdate(fieldInfo, updates);
for (DocValuesFieldUpdates update : updates) {
deletes.onDocValuesUpdate(update.field, update.iterator());
}
deletes.onDocValuesUpdate(fieldInfo);
assertEquals(5, deletes.numPendingDeletes());
assertTrue(deletes.getLiveDocs().get(0));
assertFalse(deletes.getLiveDocs().get(1));
@ -146,7 +152,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
public void testUpdateAppliedOnlyOnce() throws IOException {
Directory dir = newDirectory();
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); // no soft delete field hier
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig().setSoftDeletesField("_soft_deletes"));
Document doc = new Document();
doc.add(new StringField("id", "1", Field.Store.YES));
writer.softUpdateDocument(new Term("id", "1"), doc,
@ -169,7 +175,10 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
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<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 3));
deletes.onDocValuesUpdate(fieldInfo, updates);
for (DocValuesFieldUpdates update : updates) {
deletes.onDocValuesUpdate(update.field, update.iterator());
}
deletes.onDocValuesUpdate(fieldInfo);
assertEquals(1, deletes.numPendingDeletes());
assertTrue(deletes.getLiveDocs().get(0));
assertFalse(deletes.getLiveDocs().get(1));

View File

@ -27,6 +27,7 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
@ -196,4 +197,35 @@ public class TestSoftDeletesDirectoryReaderWrapper extends LuceneTestCase {
assertEquals(1, leafCalled.get());
IOUtils.close(reader, writer, dir);
}
public void testForceMergeDeletes() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig config = newIndexWriterConfig().setSoftDeletesField("soft_delete");
config.setMergePolicy(newMergePolicy(random(), false)); // no mock MP it might not select segments for force merge
if (random().nextBoolean()) {
config.setMergePolicy(new SoftDeletesRetentionMergePolicy("soft_delete",
() -> new MatchNoDocsQuery(), config.getMergePolicy()));
}
IndexWriter writer = new IndexWriter(dir, config);
// The first segment includes d1 and d2
for (int i = 0; i < 2; i++) {
Document d = new Document();
d.add(new StringField("id", Integer.toString(i), Field.Store.YES));
writer.addDocument(d);
}
writer.flush();
// The second segment includes only the tombstone
Document tombstone = new Document();
tombstone.add(new NumericDocValuesField("soft_delete", 1));
writer.softUpdateDocument(new Term("id", "1"), tombstone, new NumericDocValuesField("soft_delete", 1));
// Internally, forceMergeDeletes will call flush to flush pending updates
// Thus, we will have two segments - both having soft-deleted documents.
// We expect any MP to merge these segments into one segment
// when calling forceMergeDeletes.
writer.forceMergeDeletes(true);
assertEquals(1, writer.maxDoc());
assertEquals(1, writer.segmentInfos.asList().size());
writer.close();
dir.close();
}
}

View File

@ -1541,7 +1541,53 @@ public class TestLRUQueryCache extends LuceneTestCase {
reader.close();
w.close();
dir.close();
}
public void testQueryCacheSoftUpdate() throws IOException {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig().setSoftDeletesField("soft_delete");
IndexWriter w = new IndexWriter(dir, iwc);
LRUQueryCache queryCache = new LRUQueryCache(10, 1000 * 1000, ctx -> true);
IndexSearcher.setDefaultQueryCache(queryCache);
IndexSearcher.setDefaultQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
SearcherManager sm = new SearcherManager(w, new SearcherFactory());
Document doc = new Document();
doc.add(new StringField("id", "1", org.apache.lucene.document.Field.Store.YES));
w.addDocument(doc);
doc = new Document();
doc.add(new StringField("id", "2", org.apache.lucene.document.Field.Store.YES));
w.addDocument(doc);
sm.maybeRefreshBlocking();
IndexSearcher searcher = sm.acquire();
Query query = new BooleanQuery.Builder().add(new TermQuery(new Term("id", "1")), BooleanClause.Occur.FILTER).build();
assertEquals(1, searcher.count(query));
assertEquals(1, queryCache.getCacheSize());
assertEquals(0, queryCache.getEvictionCount());
boolean softDelete = true;
if (softDelete) {
Document tombstone = new Document();
tombstone.add(new NumericDocValuesField("soft_delete", 1));
w.softUpdateDocument(new Term("id", "1"), tombstone, new NumericDocValuesField("soft_delete", 1));
w.softUpdateDocument(new Term("id", "2"), tombstone, new NumericDocValuesField("soft_delete", 1));
} else {
w.deleteDocuments(new Term("id", "1"));
w.deleteDocuments(new Term("id", "2"));
}
sm.maybeRefreshBlocking();
// All docs in the first segment are deleted - we should drop it with the default merge policy.
sm.release(searcher);
assertEquals(0, queryCache.getCacheSize());
assertEquals(1, queryCache.getEvictionCount());
sm.close();
w.close();
dir.close();
}
public void testBulkScorerLocking() throws Exception {

View File

@ -1049,7 +1049,11 @@ public abstract class LuceneTestCase extends Assert {
}
public static MergePolicy newMergePolicy(Random r) {
if (rarely(r)) {
return newMergePolicy(r, true);
}
public static MergePolicy newMergePolicy(Random r, boolean includeMockMP) {
if (includeMockMP && rarely(r)) {
return new MockRandomMergePolicy(r);
} else if (r.nextBoolean()) {
return newTieredMergePolicy(r);