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/05/31 07:24:47 UTC

lucene-solr:branch_7x: LUCENE-8338: Ensure number returned for PendingDeletes are well defined

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x 528b96540 -> a462c147a


LUCENE-8338: Ensure number returned for PendingDeletes are well defined

Today a call to PendingDeletes#numPendingDeletes might return 0
if the deletes are written to disk. This doesn't mean these values are committed
or refreshed in the latest reader. Some places in IW use these numbers to make
decisions if there has been deletes added since last time checked (BufferedUpdateStream)
which can cause wrong (while not fatal) decision ie. to kick of new merges.

Now this API is made protected and not visible outside of PendingDeletes to prevent
any kind of confusion. The APIs now allow to get absolute numbers of getDelCount and numDocs
which have the same name and semantics as their relatives on IndexReader/Writer
and SegmentCommitInfo.


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

Branch: refs/heads/branch_7x
Commit: a462c147a0607a984ce157d9b7fae0c20c8a9ad7
Parents: 528b965
Author: Simon Willnauer <si...@apache.org>
Authored: Tue May 29 16:08:12 2018 +0200
Committer: Simon Willnauer <si...@apache.org>
Committed: Thu May 31 09:21:38 2018 +0200

----------------------------------------------------------------------
 .../lucene/index/BufferedUpdatesStream.java     |  2 +-
 .../apache/lucene/index/FilterMergePolicy.java  |  4 +-
 .../lucene/index/FrozenBufferedUpdates.java     |  4 +-
 .../org/apache/lucene/index/IndexWriter.java    | 16 +++----
 .../org/apache/lucene/index/MergePolicy.java    |  6 +--
 .../org/apache/lucene/index/NoMergePolicy.java  |  4 +-
 .../org/apache/lucene/index/PendingDeletes.java | 46 +++++++++++++++++++-
 .../apache/lucene/index/PendingSoftDeletes.java |  9 ++--
 .../org/apache/lucene/index/ReaderPool.java     |  3 +-
 .../apache/lucene/index/ReadersAndUpdates.java  | 45 +++++--------------
 .../index/SoftDeletesRetentionMergePolicy.java  |  4 +-
 .../org/apache/lucene/index/TestReaderPool.java |  5 +--
 .../org/apache/lucene/index/TestTryDelete.java  |  2 +-
 13 files changed, 83 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java b/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java
index c93e4b6..dcc8bbf 100644
--- a/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java
+++ b/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java
@@ -259,7 +259,7 @@ final class BufferedUpdatesStream implements Accountable {
 
     SegmentState(ReadersAndUpdates rld, IOUtils.IOConsumer<ReadersAndUpdates> onClose, SegmentCommitInfo info) throws IOException {
       this.rld = rld;
-      startDelCount = rld.getPendingDeleteCount();
+      startDelCount = rld.getDelCount();
       delGen = info.getBufferedDeletesGen();
       this.onClose = onClose;
       reader = rld.getReader(IOContext.READ);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java
index cbc8b18..eb634b4 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java
@@ -99,8 +99,8 @@ public class FilterMergePolicy extends MergePolicy {
   }
 
   @Override
-  public int numDeletesToMerge(SegmentCommitInfo info, int pendingDeleteCount,
+  public int numDeletesToMerge(SegmentCommitInfo info, int delCount,
                                IOSupplier<CodecReader> readerSupplier) throws IOException {
-    return in.numDeletesToMerge(info, pendingDeleteCount, readerSupplier);
+    return in.numDeletesToMerge(info, delCount, readerSupplier);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java b/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
index 5ea9392..560d86d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
@@ -389,8 +389,8 @@ final class FrozenBufferedUpdates {
     final List<BufferedUpdatesStream.SegmentState> segmentStates = Arrays.asList(segStates);
     for (BufferedUpdatesStream.SegmentState segState : segmentStates) {
       if (success) {
-        totDelCount += segState.rld.getPendingDeleteCount() - segState.startDelCount;
-        int fullDelCount = segState.rld.info.getDelCount() + segState.rld.getPendingDeleteCount();
+        totDelCount += segState.rld.getDelCount() - segState.startDelCount;
+        int fullDelCount = segState.rld.getDelCount();
         assert fullDelCount <= segState.rld.info.info.maxDoc() : fullDelCount + " > " + segState.rld.info.info.maxDoc();
         if (segState.rld.isFullyDeleted() && writer.getConfig().getMergePolicy().keepFullyDeletedSegment(() -> segState.reader) == false) {
           if (allDeleted == null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/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 5c53cb7..ba30df0 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -635,14 +635,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
   public int numDeletedDocs(SegmentCommitInfo info) {
     ensureOpen(false);
     validate(info);
-    int delCount = info.getDelCount();
-
     final ReadersAndUpdates rld = getPooledInstance(info, false);
     if (rld != null) {
-      delCount += rld.getPendingDeleteCount();
+      return rld.getDelCount(); // get the full count from here since SCI might change concurrently
+    } else {
+      int delCount = info.getDelCount();
+      assert delCount <= info.info.maxDoc(): "delCount: " + delCount + " maxDoc: " + info.info.maxDoc();
+      return delCount;
     }
-    assert delCount <= info.info.maxDoc(): "delCount: " + delCount + " maxDoc: " + info.info.maxDoc();
-    return delCount;
   }
 
   /**
@@ -3697,7 +3697,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
 
     // Lazy init (only when we find a delete or update to carry over):
     final ReadersAndUpdates mergedDeletesAndUpdates = getPooledInstance(merge.info, true);
-    
+    int numDeletesBefore = mergedDeletesAndUpdates.getDelCount();
     // field -> delGen -> dv field updates
     Map<String,Map<Long,DocValuesFieldUpdates>> mappedDVUpdates = new HashMap<>();
 
@@ -3788,7 +3788,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
       if (mergedDeletesAndUpdates == null) {
         infoStream.message("IW", "no new deletes or field updates since merge started");
       } else {
-        String msg = mergedDeletesAndUpdates.getPendingDeleteCount() + " new deletes";
+        String msg = mergedDeletesAndUpdates.getDelCount() - numDeletesBefore + " new deletes";
         if (anyDVUpdates) {
           msg += " and " + mergedDeletesAndUpdates.getNumDVUpdates() + " new field updates";
           msg += " (" + mergedDeletesAndUpdates.ramBytesUsed.get() + ") bytes";
@@ -4363,7 +4363,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
 
         ReadersAndUpdates.MergeReader mr = rld.getReaderForMerge(context);
         SegmentReader reader = mr.reader;
-        int delCount = reader.numDeletedDocs();
 
         if (infoStream.isEnabled("IW")) {
           infoStream.message("IW", "seg=" + segString(info) + " reader=" + reader);
@@ -4371,7 +4370,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
 
         merge.hardLiveDocs.add(mr.hardLiveDocs);
         merge.readers.add(reader);
-        assert delCount <= info.info.maxDoc(): "delCount=" + delCount + " info.maxDoc=" + info.info.maxDoc() + " rld.pendingDeleteCount=" + rld.getPendingDeleteCount() + " info.getDelCount()=" + info.getDelCount();
         segUpto++;
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
index 47a09e8..d552d74 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
@@ -638,12 +638,12 @@ public abstract class MergePolicy {
    * @see IndexWriter#softUpdateDocument(Term, Iterable, Field...)
    * @see IndexWriterConfig#setSoftDeletesField(String)
    * @param info the segment info that identifies the segment
-   * @param pendingDeleteCount the number of pending deletes for this segment
+   * @param delCount the number deleted documents for this segment
    * @param readerSupplier a supplier that allows to obtain a {@link CodecReader} for this segment
    */
-  public int numDeletesToMerge(SegmentCommitInfo info, int pendingDeleteCount,
+  public int numDeletesToMerge(SegmentCommitInfo info, int delCount,
                                IOSupplier<CodecReader> readerSupplier) throws IOException {
-    return info.getDelCount() + pendingDeleteCount;
+    return delCount;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java
index f3449e2..1480ce4 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java
@@ -81,8 +81,8 @@ public final class NoMergePolicy extends MergePolicy {
   }
 
   @Override
-  public int numDeletesToMerge(SegmentCommitInfo info, int pendingDeleteCount, IOSupplier<CodecReader> readerSupplier) throws IOException {
-    return  super.numDeletesToMerge(info, pendingDeleteCount, readerSupplier);
+  public int numDeletesToMerge(SegmentCommitInfo info, int delCount, IOSupplier<CodecReader> readerSupplier) throws IOException {
+    return  super.numDeletesToMerge(info, delCount, readerSupplier);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/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 506d397..f19b053 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
@@ -118,7 +118,7 @@ class PendingDeletes {
   /**
    * Returns the number of pending deletes that are not written to disk.
    */
-  int numPendingDeletes() {
+  protected int numPendingDeletes() {
     return pendingDeleteCount;
   }
 
@@ -232,7 +232,49 @@ class PendingDeletes {
   }
 
   int numDeletesToMerge(MergePolicy policy, IOSupplier<CodecReader> readerIOSupplier) throws IOException {
-    return policy.numDeletesToMerge(info, numPendingDeletes(), readerIOSupplier);
+    return policy.numDeletesToMerge(info, getDelCount(), readerIOSupplier);
   }
 
+  /**
+   * Returns true if the given reader needs to be refreshed in order to see the latest deletes
+   */
+  final boolean needsRefresh(CodecReader reader) {
+    return reader.getLiveDocs() != getLiveDocs() || reader.numDeletedDocs() != getDelCount();
+  }
+
+  /**
+   * Returns the number of deleted docs in the segment.
+   */
+  final int getDelCount() {
+    return info.getDelCount() + numPendingDeletes();
+  }
+
+  /**
+   * Returns the number of live documents in this segment
+   */
+  final int numDocs() {
+    return info.info.maxDoc() - getDelCount();
+  }
+
+  // Call only from assert!
+  boolean verifyDocCounts(CodecReader reader) {
+    int count = 0;
+    Bits liveDocs = getLiveDocs();
+    if (liveDocs != null) {
+      for(int docID = 0; docID < info.info.maxDoc(); docID++) {
+        if (liveDocs.get(docID)) {
+          count++;
+        }
+      }
+    } else {
+      count = info.info.maxDoc();
+    }
+    assert numDocs() == count: "info.maxDoc=" + info.info.maxDoc() + " info.getDelCount()=" + info.getDelCount() +
+        " pendingDeletes=" + toString() + " count=" + count;
+    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() +
+        " info.getDelCount()=" + info.getDelCount();
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/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 fe012f6..1c32e4f 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
@@ -58,6 +58,7 @@ final class PendingSoftDeletes extends PendingDeletes {
       } else {
         // if it was deleted subtract the delCount
         pendingDeleteCount--;
+        assert pendingDeleteCount >= 0 : " illegal pending delete count: " + pendingDeleteCount;
       }
       return true;
     }
@@ -65,7 +66,7 @@ final class PendingSoftDeletes extends PendingDeletes {
   }
 
   @Override
-  int numPendingDeletes() {
+  protected int numPendingDeletes() {
     return super.numPendingDeletes() + hardDeletes.numPendingDeletes();
   }
 
@@ -78,11 +79,11 @@ final class PendingSoftDeletes extends PendingDeletes {
       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;
       }
       dvGeneration = info.getDocValuesGen();
     }
-    assert numPendingDeletes() + info.getDelCount() <= info.info.maxDoc() :
-        numPendingDeletes() + " + " + info.getDelCount() + " > " + info.info.maxDoc();
+    assert getDelCount() <= info.info.maxDoc() : getDelCount() + " > " + info.info.maxDoc();
   }
 
   @Override
@@ -133,6 +134,7 @@ 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 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();
@@ -208,5 +210,4 @@ final class PendingSoftDeletes extends PendingDeletes {
   Bits getHardLiveDocs() {
     return hardDeletes.getLiveDocs();
   }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/java/org/apache/lucene/index/ReaderPool.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/ReaderPool.java b/lucene/core/src/java/org/apache/lucene/index/ReaderPool.java
index 861cfaf..45f58a6 100644
--- a/lucene/core/src/java/org/apache/lucene/index/ReaderPool.java
+++ b/lucene/core/src/java/org/apache/lucene/index/ReaderPool.java
@@ -132,7 +132,7 @@ final class ReaderPool implements Closeable {
    */
   synchronized boolean anyPendingDeletes() {
     for(ReadersAndUpdates rld : readerMap.values()) {
-      if (rld.getPendingDeleteCount() != 0) {
+      if (rld.anyPendingDeletes()) {
         return true;
       }
     }
@@ -321,7 +321,6 @@ final class ReaderPool implements Closeable {
 
   /**
    * Returns <code>true</code> iff there are any buffered doc values updates. Otherwise <code>false</code>.
-   * @see #anyPendingDeletes()
    */
   synchronized boolean anyDocValuesChanges() {
     for (ReadersAndUpdates rld : readerMap.values()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/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 5558595..710b748 100644
--- a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
@@ -100,8 +100,6 @@ final class ReadersAndUpdates {
    * <p>NOTE: steals incoming ref from reader. */
   ReadersAndUpdates(int indexCreatedVersionMajor, SegmentReader reader, PendingDeletes pendingDeletes) throws IOException {
     this(indexCreatedVersionMajor, reader.getOriginalSegmentInfo(), pendingDeletes);
-    assert pendingDeletes.numPendingDeletes() >= 0
-        : "got " + pendingDeletes.numPendingDeletes() + " reader.numDeletedDocs()=" + reader.numDeletedDocs() + " info.getDelCount()=" + info.getDelCount() + " maxDoc=" + reader.maxDoc() + " numDocs=" + reader.numDocs();
     this.reader = reader;
     pendingDeletes.onNewReader(reader, info);
   }
@@ -122,10 +120,9 @@ final class ReadersAndUpdates {
     return rc;
   }
 
-  public synchronized int getPendingDeleteCount() {
-    return pendingDeletes.numPendingDeletes();
+  public synchronized int getDelCount() {
+    return pendingDeletes.getDelCount();
   }
-
   private synchronized boolean assertNoDupGen(List<DocValuesFieldUpdates> fieldUpdates, DocValuesFieldUpdates update) {
     for (int i=0;i<fieldUpdates.size();i++) {
       DocValuesFieldUpdates oldUpdate = fieldUpdates.get(i);
@@ -167,24 +164,6 @@ final class ReadersAndUpdates {
     return count;
   }
   
-  // Call only from assert!
-  public synchronized boolean verifyDocCounts() {
-    int count;
-    Bits liveDocs = pendingDeletes.getLiveDocs();
-    if (liveDocs != null) {
-      count = 0;
-      for(int docID=0;docID<info.info.maxDoc();docID++) {
-        if (liveDocs.get(docID)) {
-          count++;
-        }
-      }
-    } else {
-      count = info.info.maxDoc();
-    }
-
-    assert info.info.maxDoc() - info.getDelCount() - pendingDeletes.numPendingDeletes() == count: "info.maxDoc=" + info.info.maxDoc() + " info.getDelCount()=" + info.getDelCount() + " pendingDeletes=" + pendingDeletes.numPendingDeletes() + " count=" + count;
-    return true;
-  }
 
   /** Returns a {@link SegmentReader}. */
   public synchronized SegmentReader getReader(IOContext context) throws IOException {
@@ -235,8 +214,7 @@ final class ReadersAndUpdates {
     // force new liveDocs
     Bits liveDocs = pendingDeletes.getLiveDocs();
     if (liveDocs != null) {
-      return new SegmentReader(info, reader, liveDocs,
-          info.info.maxDoc() - info.getDelCount() - pendingDeletes.numPendingDeletes());
+      return new SegmentReader(info, reader, liveDocs, pendingDeletes.numDocs());
     } else {
       // liveDocs == null and reader != null. That can only be if there are no deletes
       assert reader.getLiveDocs() == null;
@@ -254,8 +232,7 @@ final class ReadersAndUpdates {
       // get a reader and dec the ref right away we just make sure we have a reader
       getReader(IOContext.READ).decRef();
     }
-    if (reader.getLiveDocs() != pendingDeletes.getLiveDocs()
-        || reader.numDeletedDocs() != info.getDelCount() - pendingDeletes.numPendingDeletes()) {
+    if (pendingDeletes.needsRefresh(reader)) {
       // we have a reader but its live-docs are out of sync. let's create a temporary one that we never share
       swapNewReaderWithLatestLiveDocs();
     }
@@ -429,6 +406,10 @@ 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
@@ -668,8 +649,7 @@ final class ReadersAndUpdates {
   private SegmentReader createNewReaderWithLatestLiveDocs(SegmentReader reader) throws IOException {
     assert reader != null;
     assert Thread.holdsLock(this) : Thread.currentThread().getName();
-    SegmentReader newReader = new SegmentReader(info, reader, pendingDeletes.getLiveDocs(),
-        info.info.maxDoc() - info.getDelCount() - pendingDeletes.numPendingDeletes());
+    SegmentReader newReader = new SegmentReader(info, reader, pendingDeletes.getLiveDocs(), pendingDeletes.numDocs());
     boolean success2 = false;
     try {
       pendingDeletes.onNewReader(newReader, info);
@@ -727,14 +707,13 @@ final class ReadersAndUpdates {
     }
     
     SegmentReader reader = getReader(context);
-    int delCount = pendingDeletes.numPendingDeletes() + info.getDelCount();
-    if (delCount != reader.numDeletedDocs()) {
+    if (pendingDeletes.needsRefresh(reader)) {
       // beware of zombies:
-      assert delCount > reader.numDeletedDocs(): "delCount=" + delCount + " reader.numDeletedDocs()=" + reader.numDeletedDocs();
       assert pendingDeletes.getLiveDocs() != null;
       reader = createNewReaderWithLatestLiveDocs(reader);
     }
-    assert verifyDocCounts();
+    assert pendingDeletes.verifyDocCounts(reader);
+
 
     return new MergeReader(reader, pendingDeletes.getHardLiveDocs());
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/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 68f51bb..3e5b56e 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SoftDeletesRetentionMergePolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SoftDeletesRetentionMergePolicy.java
@@ -172,8 +172,8 @@ public final class SoftDeletesRetentionMergePolicy extends OneMergeWrappingMerge
   }
 
   @Override
-  public int numDeletesToMerge(SegmentCommitInfo info, int pendingDeleteCount, IOSupplier<CodecReader> readerSupplier) throws IOException {
-    final int numDeletesToMerge = super.numDeletesToMerge(info, pendingDeleteCount, readerSupplier);
+  public int numDeletesToMerge(SegmentCommitInfo info, int delCount, IOSupplier<CodecReader> readerSupplier) throws IOException {
+    final int numDeletesToMerge = super.numDeletesToMerge(info, delCount, readerSupplier);
     if (numDeletesToMerge != 0) {
       final CodecReader reader = readerSupplier.get();
       if (reader.getLiveDocs() != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/test/org/apache/lucene/index/TestReaderPool.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestReaderPool.java b/lucene/core/src/test/org/apache/lucene/index/TestReaderPool.java
index 19a6af2..36070b5 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestReaderPool.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestReaderPool.java
@@ -178,12 +178,9 @@ public class TestReaderPool extends LuceneTestCase {
       boolean expectUpdate = false;
       int doc = -1;
       if (postings != null && postings.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
-        readersAndUpdates.delete(doc = postings.docID());
+        assertTrue(readersAndUpdates.delete(doc = postings.docID()));
         expectUpdate = true;
         assertEquals(DocIdSetIterator.NO_MORE_DOCS, postings.nextDoc());
-        assertTrue(pool.anyPendingDeletes());
-      } else {
-        assertFalse(pool.anyPendingDeletes());
       }
       assertFalse(pool.anyDocValuesChanges()); // deletes are not accounted here
       readOnlyClone.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a462c147/lucene/core/src/test/org/apache/lucene/index/TestTryDelete.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTryDelete.java b/lucene/core/src/test/org/apache/lucene/index/TestTryDelete.java
index 6ce519d..561006f 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestTryDelete.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestTryDelete.java
@@ -104,7 +104,7 @@ public class TestTryDelete extends LuceneTestCase
     }
 
     assertTrue(writer.hasDeletions());
-    
+
     mgr.maybeRefresh();
 
     searcher = mgr.acquire();