You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2018/04/16 14:35:31 UTC

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

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x 476376f7d -> aeac55a60


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.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/aeac55a6
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/aeac55a6
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/aeac55a6

Branch: refs/heads/branch_7x
Commit: aeac55a602980c92ffee25602c6450e40eab6e6f
Parents: 476376f
Author: Simon Willnauer <si...@apache.org>
Authored: Mon Apr 16 16:16:43 2018 +0200
Committer: Simon Willnauer <si...@apache.org>
Committed: Mon Apr 16 16:35:24 2018 +0200

----------------------------------------------------------------------
 .../org/apache/lucene/index/IndexWriter.java    | 28 ++++----
 .../org/apache/lucene/index/PendingDeletes.java | 21 ++++--
 .../apache/lucene/index/PendingSoftDeletes.java | 75 ++++++++++++++++----
 .../apache/lucene/index/ReadersAndUpdates.java  | 10 +--
 .../index/SoftDeletesRetentionMergePolicy.java  | 22 +++---
 .../lucene/index/TestPendingSoftDeletes.java    | 19 +++--
 .../TestSoftDeletesDirectoryReaderWrapper.java  | 32 +++++++++
 .../apache/lucene/search/TestLRUQueryCache.java | 46 ++++++++++++
 .../org/apache/lucene/util/LuceneTestCase.java  |  6 +-
 9 files changed, 206 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aeac55a6/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index f82865051..d60b311 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -1596,7 +1596,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
@@ -1833,7 +1833,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);
   }
 
 
@@ -1942,7 +1942,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) {
@@ -1956,7 +1956,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];
@@ -1967,7 +1967,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())) {
@@ -5232,15 +5232,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;
+
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aeac55a6/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java b/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
index 52d06e8..c0aed38 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
@@ -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);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aeac55a6/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java b/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
index 4c3db48..3dca782 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
@@ -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);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aeac55a6/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
index b236bea..2543721 100644
--- a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
@@ -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 {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aeac55a6/lucene/core/src/java/org/apache/lucene/index/SoftDeletesRetentionMergePolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SoftDeletesRetentionMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/SoftDeletesRetentionMergePolicy.java
index 60451ff..25df3c6 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SoftDeletesRetentionMergePolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SoftDeletesRetentionMergePolicy.java
@@ -71,7 +71,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;
@@ -97,7 +98,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();
@@ -132,13 +136,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(), false, 1.0f);
+    Weight weight = s.createWeight(query, false, 1.0f);
     return weight.scorer(reader.getContext());
   }
 
@@ -171,11 +172,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();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aeac55a6/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java b/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
index 255ff9e..eac4388 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
@@ -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));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aeac55a6/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesDirectoryReaderWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesDirectoryReaderWrapper.java b/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesDirectoryReaderWrapper.java
index dea7bc9..30a11b6 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesDirectoryReaderWrapper.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesDirectoryReaderWrapper.java
@@ -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();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aeac55a6/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
index 874d525..c3d6ac3 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
@@ -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 {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aeac55a6/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
index c8b070c..8734f19 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
@@ -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);