You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2018/05/15 20:04:29 UTC

[12/39] lucene-solr:jira/solr-11779: LUCENE-8296: PendingDeletes may no longer write to live docs after they are shared.

LUCENE-8296: PendingDeletes may no longer write to live docs after they are shared.


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

Branch: refs/heads/jira/solr-11779
Commit: 7873cf845e1d513026b6836769b7cbd4d237c2aa
Parents: 8dc6942
Author: Adrien Grand <jp...@gmail.com>
Authored: Wed May 9 15:12:23 2018 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Wed May 9 15:40:14 2018 +0200

----------------------------------------------------------------------
 .../org/apache/lucene/index/PendingDeletes.java | 51 ++++++++------------
 .../apache/lucene/index/PendingSoftDeletes.java |  8 +--
 .../apache/lucene/index/ReadersAndUpdates.java  | 11 ++---
 .../apache/lucene/index/TestPendingDeletes.java | 10 ++--
 .../lucene/index/TestPendingSoftDeletes.java    |  2 -
 5 files changed, 29 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7873cf84/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 354f9d1..4bd90cb 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
@@ -33,15 +33,11 @@ import org.apache.lucene.util.IOUtils;
  */
 class PendingDeletes {
   protected final SegmentCommitInfo info;
-  // True if the current liveDocs is referenced by an
-  // external NRT reader:
-  protected boolean liveDocsShared;
-  // Holds the current shared (readable and writable)
-  // liveDocs.  This is null when there are no deleted
-  // docs, and it's copy-on-write (cloned whenever we need
-  // to change it but it's been shared to an external NRT
-  // reader).
+  // Read-only live docs, null until live docs are initialized or if all docs are alive
   private Bits liveDocs;
+  // Writeable live docs, null if this instance is not ready to accept writes, in which
+  // case getMutableBits needs to be called
+  private FixedBitSet writeableLiveDocs;
   protected int pendingDeleteCount;
   private boolean liveDocsInitialized;
 
@@ -59,7 +55,6 @@ class PendingDeletes {
 
   private PendingDeletes(SegmentCommitInfo info, Bits liveDocs, boolean liveDocsInitialized) {
     this.info = info;
-    liveDocsShared = true;
     this.liveDocs = liveDocs;
     pendingDeleteCount = 0;
     this.liveDocsInitialized = liveDocsInitialized;
@@ -70,24 +65,23 @@ class PendingDeletes {
     // if we pull mutable bits but we haven't been initialized something is completely off.
     // this means we receive deletes without having the bitset that is on-disk ready to be cloned
     assert liveDocsInitialized : "can't delete if liveDocs are not initialized";
-    if (liveDocsShared) {
+    if (writeableLiveDocs == null) {
       // Copy on write: this means we've cloned a
       // SegmentReader sharing the current liveDocs
       // instance; must now make a private clone so we can
       // change it:
-      FixedBitSet mutableBits = new FixedBitSet(info.info.maxDoc());
-      mutableBits.set(0, info.info.maxDoc());
+      writeableLiveDocs = new FixedBitSet(info.info.maxDoc());
+      writeableLiveDocs.set(0, info.info.maxDoc());
       if (liveDocs != null) {
         for (int i = 0; i < liveDocs.length(); ++i) {
           if (liveDocs.get(i) == false) {
-            mutableBits.clear(i);
+            writeableLiveDocs.clear(i);
           }
         }
       }
-      liveDocs = mutableBits;
-      liveDocsShared = false;
+      liveDocs = writeableLiveDocs;
     }
-    return (FixedBitSet) liveDocs;
+    return writeableLiveDocs;
   }
 
 
@@ -100,7 +94,6 @@ class PendingDeletes {
     FixedBitSet mutableBits = getMutableBits();
     assert mutableBits != null;
     assert docID >= 0 && docID < mutableBits.length() : "out of bounds: docid=" + docID + " liveDocsLength=" + mutableBits.length() + " seg=" + info.info.name + " maxDoc=" + info.info.maxDoc();
-    assert !liveDocsShared;
     final boolean didDelete = mutableBits.get(docID);
     if (didDelete) {
       mutableBits.clear(docID);
@@ -110,20 +103,19 @@ class PendingDeletes {
   }
 
   /**
-   * Should be called if the live docs returned from {@link #getLiveDocs()} are shared outside of the
-   * {@link ReadersAndUpdates}
+   * Returns a snapshot of the current live docs.
    */
-  void liveDocsShared() {
-    liveDocsShared = true;
+  Bits getLiveDocs() {
+    // Prevent modifications to the returned live docs
+    writeableLiveDocs = null;
+    return liveDocs;
   }
 
   /**
-   * Returns the current live docs or null if all docs are live. The returned instance might be mutable or is mutated behind the scenes.
-   * If the returned live docs are shared outside of the ReadersAndUpdates {@link #liveDocsShared()} should be called
-   * first.
+   * Returns a snapshot of the hard live docs.
    */
-  Bits getLiveDocs() {
-    return liveDocs;
+  Bits getHardLiveDocs() {
+    return getLiveDocs();
   }
 
   /**
@@ -138,6 +130,7 @@ class PendingDeletes {
    */
   void onNewReader(CodecReader reader, SegmentCommitInfo info) throws IOException {
     if (liveDocsInitialized == false) {
+      assert writeableLiveDocs == null;
       if (reader.hasDeletions()) {
         // we only initialize this once either in the ctor or here
         // if we use the live docs from a reader it has to be in a situation where we don't
@@ -145,7 +138,6 @@ class PendingDeletes {
         assert pendingDeleteCount == 0 : "pendingDeleteCount: " + pendingDeleteCount;
         liveDocs = reader.getLiveDocs();
         assert liveDocs == null || assertCheckLiveDocs(liveDocs, info.info.maxDoc(), info.getDelCount());
-        liveDocsShared = true;
       }
       liveDocsInitialized = true;
     }
@@ -175,7 +167,7 @@ class PendingDeletes {
     StringBuilder sb = new StringBuilder();
     sb.append("PendingDeletes(seg=").append(info);
     sb.append(" numPendingDeletes=").append(pendingDeleteCount);
-    sb.append(" liveDocsShared=").append(liveDocsShared);
+    sb.append(" writeable=").append(writeableLiveDocs != null);
     return sb.toString();
   }
 
@@ -246,7 +238,4 @@ class PendingDeletes {
     return policy.numDeletesToMerge(info, numPendingDeletes(), readerIOSupplier);
   }
 
-  Bits getHardLiveDocs() {
-    return liveDocs;
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7873cf84/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 3809304..637a90c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
@@ -195,13 +195,9 @@ final class PendingSoftDeletes extends PendingDeletes {
     }
   }
 
+  @Override
   Bits getHardLiveDocs() {
-    return hardDeletes.getHardLiveDocs();
+    return hardDeletes.getLiveDocs();
   }
 
-  @Override
-  void liveDocsShared() {
-    super.liveDocsShared();
-    hardDeletes.liveDocsShared();
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7873cf84/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 a02d26a..1a96c13 100644
--- a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
@@ -234,7 +234,6 @@ final class ReadersAndUpdates {
     }
     // force new liveDocs
     Bits liveDocs = pendingDeletes.getLiveDocs();
-    markAsShared();
     if (liveDocs != null) {
       return new SegmentReader(reader.getSegmentInfo(), reader, liveDocs,
           info.info.maxDoc() - info.getDelCount() - pendingDeletes.numPendingDeletes());
@@ -263,6 +262,9 @@ final class ReadersAndUpdates {
     return reader;
   }
 
+  /**
+   * Returns a snapshot of the live docs.
+   */
   public synchronized Bits getLiveDocs() {
     return pendingDeletes.getLiveDocs();
   }
@@ -720,8 +722,6 @@ final class ReadersAndUpdates {
       assert pendingDeletes.getLiveDocs() != null;
       reader = createNewReaderWithLatestLiveDocs(reader);
     }
-
-    markAsShared();
     assert verifyDocCounts();
 
     return new MergeReader(reader, pendingDeletes.getHardLiveDocs());
@@ -754,11 +754,6 @@ final class ReadersAndUpdates {
     return pendingDeletes.isFullyDeleted(this::getLatestReader);
   }
 
-  private final void markAsShared() {
-    assert Thread.holdsLock(this);
-    pendingDeletes.liveDocsShared(); // this is not costly we can just call it even if it's already marked as shared
-  }
-
   boolean keepFullyDeletedSegment(MergePolicy mergePolicy) throws IOException {
     return mergePolicy.keepFullyDeletedSegment(this::getLatestReader);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7873cf84/lucene/core/src/test/org/apache/lucene/index/TestPendingDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPendingDeletes.java b/lucene/core/src/test/org/apache/lucene/index/TestPendingDeletes.java
index 7c6891e..ecc2d4d 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPendingDeletes.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPendingDeletes.java
@@ -52,18 +52,16 @@ public class TestPendingDeletes extends LuceneTestCase {
     assertFalse(liveDocs.get(docToDelete));
     assertFalse(deletes.delete(docToDelete)); // delete again
 
-    // make sure we are live ie. mutable
     assertTrue(liveDocs.get(8));
     assertTrue(deletes.delete(8));
-    assertFalse(liveDocs.get(8));
+    assertTrue(liveDocs.get(8)); // we have a snapshot
     assertEquals(2, deletes.numPendingDeletes());
 
-    deletes.liveDocsShared();
-
-    // make sure we are live ie. mutable
     assertTrue(liveDocs.get(9));
     assertTrue(deletes.delete(9));
     assertTrue(liveDocs.get(9));
+
+    // now make sure new live docs see the deletions
     liveDocs = deletes.getLiveDocs();
     assertFalse(liveDocs.get(9));
     assertFalse(liveDocs.get(8));
@@ -83,7 +81,7 @@ public class TestPendingDeletes extends LuceneTestCase {
     boolean secondDocDeletes = random().nextBoolean();
     deletes.delete(5);
     if (secondDocDeletes) {
-      deletes.liveDocsShared();
+      deletes.getLiveDocs();
       deletes.delete(2);
     }
     assertEquals(-1, commitInfo.getDelGen());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7873cf84/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 aeb5819..903f847 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
@@ -73,7 +73,6 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
     assertNull(pendingSoftDeletes.getHardLiveDocs());
     // pass reader again
     Bits liveDocs = pendingSoftDeletes.getLiveDocs();
-    pendingSoftDeletes.liveDocsShared();
     pendingSoftDeletes.onNewReader(segmentReader, segmentInfo);
     assertEquals(1, pendingSoftDeletes.numPendingDeletes());
     assertSame(liveDocs, pendingSoftDeletes.getLiveDocs());
@@ -189,7 +188,6 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
     assertTrue(deletes.getLiveDocs().get(0));
     assertFalse(deletes.getLiveDocs().get(1));
     assertTrue(deletes.getLiveDocs().get(2));
-    deletes.liveDocsShared();
     Bits liveDocs = deletes.getLiveDocs();
     deletes.onNewReader(segmentReader, segmentInfo);
     // no changes we don't apply updates twice