You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2020/06/22 17:47:07 UTC

[GitHub] [lucene-solr] s1monw opened a new pull request #1601: LUCENE-8962: Ensure we don't include fully deleted segments in a commit

s1monw opened a new pull request #1601:
URL: https://github.com/apache/lucene-solr/pull/1601


   IW might drop segments that are merged into a fully deleted segment on the floor
   and deletes the newly created files right away. We should not include these segments
   in a commit since we can't guarantee valid ref-counts on these files.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw closed pull request #1601: LUCENE-8962: Ensure we don't include fully deleted segments in a commit

Posted by GitBox <gi...@apache.org>.
s1monw closed pull request #1601:
URL: https://github.com/apache/lucene-solr/pull/1601


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mikemccand commented on a change in pull request #1601: LUCENE-8962: Ensure we don't include fully deleted segments in a commit

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #1601:
URL: https://github.com/apache/lucene-solr/pull/1601#discussion_r444984980



##########
File path: lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java
##########
@@ -396,10 +396,12 @@ protected void merge(MergePolicy.OneMerge merge) throws IOException {
             waitForMerge.await();
             writer.updateDocument(new Term("id", "2"), d2);

Review comment:
       This test fully deleted the 2nd segment (since it only had `d2`, which we are deleting here).  Maybe also add a similar case that leaves one not deleted document in the segment too?  I think we optimize 100% deleted merge source segments too, so this should tickle slightly different path.

##########
File path: lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java
##########
@@ -396,10 +396,12 @@ protected void merge(MergePolicy.OneMerge merge) throws IOException {
             waitForMerge.await();
             writer.updateDocument(new Term("id", "2"), d2);
             writer.flush();
-            waitForUpdate.countDown();
           } catch (Exception e) {
             throw new AssertionError(e);
+          } finally {
+            waitForUpdate.countDown();

Review comment:
       Ahh is this so the test does not hang on an (unexpected) exception in the `try` block?

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3265,24 +3269,40 @@ public void mergeFinished(boolean committed, boolean segmentDropped) throws IOEx
                           }
                           // Construct a OneMerge that applies to toCommit
                           MergePolicy.OneMerge applicableMerge = new MergePolicy.OneMerge(toCommitMergedAwaySegments);
-                          applicableMerge.info = info.clone();
-                          long segmentCounter = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX);
+                          applicableMerge.info = origInfo;
+                          long segmentCounter = Long.parseLong(origInfo.info.name.substring(1), Character.MAX_RADIX);
                           committingSegmentInfos.counter = Math.max(committingSegmentInfos.counter, segmentCounter + 1);
                           committingSegmentInfos.applyMergeChanges(applicableMerge, false);
                         }
                         toWrap.mergeFinished(committed, false);
                         super.mergeFinished(committed, segmentDropped);
                       }
 
+                      @Override
+                      void onMergeCommit() {
+                        origInfo = this.info.clone();

Review comment:
       Whoa!  This is the functional part of the fix?  Because we snapshot the `SegmentCommitInfo` *before* carrying over deletes that happened while the merge was running?  Very cool.
   
   But, how do we ensure that the prior live docs (without concurrent deletions carried over) are written for this merged segment, and matching `origInfo`'s `delGen`?  Normally `commitMergedDeletesAndUpdates` would do that, except it is carrying over the concurrent deletions, which we need to avoid.  Oh, wait, sorry: such deletions are of course merged away!  So, we want 100% live docs in the newly merged segment.  Tricky.
   
   We probably also need a test case confirm that (concurrent) DV updates are also transactional?  I think this same approach should work for DV updates too?  We will keep only the DV updates that got merged away, and any new ones that we carry over will not be visible in the merged segment because you are cloning the `SegmentCommitInfo` before `commitMerge` runs?

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3907,6 +3927,7 @@ private static void carryOverHardDeletes(ReadersAndUpdates mergedReadersAndUpdat
   @SuppressWarnings("try")
   private synchronized boolean commitMerge(MergePolicy.OneMerge merge, MergeState mergeState) throws IOException {

Review comment:
       We should really rename this method to not use the word `commit` since that already means something (different!) in `IndexWriter`.  Maybe `finishMerge`, or `applyMergedSegments` or something.  Naming is the hardest part!  We don't have to do that in this PR...

##########
File path: lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
##########
@@ -403,6 +409,22 @@ boolean isDone() {
     Optional<Boolean> hasCompletedSuccessfully() {
       return Optional.ofNullable(mergeCompleted.getNow(null));
     }
+
+    void onMergeCommit() {
+    }
+
+    void setMergeReaders(IOContext mergeContext, ReaderPool readerPool) throws IOException {
+      for (final SegmentCommitInfo info : segments) {
+        // Hold onto the "live" reader; we will use this to
+        // commit merged deletes
+        final ReadersAndUpdates rld = readerPool.get(info, true);

Review comment:
       This used to be `getPooledInstance`, but I think this is functionally identical.

##########
File path: lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
##########
@@ -403,6 +409,22 @@ boolean isDone() {
     Optional<Boolean> hasCompletedSuccessfully() {
       return Optional.ofNullable(mergeCompleted.getNow(null));
     }
+
+    void onMergeCommit() {
+    }
+
+    void setMergeReaders(IOContext mergeContext, ReaderPool readerPool) throws IOException {
+      for (final SegmentCommitInfo info : segments) {

Review comment:
       Maybe `assert` that `readers` is empty at the start here?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1601: LUCENE-8962: Ensure we don't include fully deleted segments in a commit

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1601:
URL: https://github.com/apache/lucene-solr/pull/1601#discussion_r443727012



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3234,15 +3234,23 @@ private long prepareCommitInternal() throws IOException {
               // corresponding add from an updateDocument) can
               // sneak into the commit point:
               toCommit = segmentInfos.clone();
-
+              pendingCommitChangeCount = changeCount.get();

Review comment:
       I moved these above the updatePendingMerges. not sure it's a problem but looks better from a readers perspective to incRef before we do anything to it potentially.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1601: LUCENE-8962: Ensure we don't include fully deleted segments in a commit

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1601:
URL: https://github.com/apache/lucene-solr/pull/1601#discussion_r445021609



##########
File path: lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java
##########
@@ -396,10 +396,12 @@ protected void merge(MergePolicy.OneMerge merge) throws IOException {
             waitForMerge.await();
             writer.updateDocument(new Term("id", "2"), d2);
             writer.flush();
-            waitForUpdate.countDown();
           } catch (Exception e) {
             throw new AssertionError(e);
+          } finally {
+            waitForUpdate.countDown();

Review comment:
       yes




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1601: LUCENE-8962: Ensure we don't include fully deleted segments in a commit

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1601:
URL: https://github.com/apache/lucene-solr/pull/1601#discussion_r443727356



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -4341,27 +4342,33 @@ private synchronized void mergeFinish(MergePolicy.OneMerge merge) {
   }
 
   @SuppressWarnings("try")
-  private synchronized void closeMergeReaders(MergePolicy.OneMerge merge, boolean suppressExceptions) throws IOException {
+  private synchronized void closeMergeReaders(MergePolicy.OneMerge merge, boolean suppressExceptions, boolean droppedSegment) throws IOException {
     final boolean drop = suppressExceptions == false;
-    try (Closeable finalizer = () -> merge.mergeFinished(suppressExceptions == false)) {
-      IOUtils.applyToAll(merge.readers, sr -> {
-        final ReadersAndUpdates rld = getPooledInstance(sr.getOriginalSegmentInfo(), false);
-        // We still hold a ref so it should not have been removed:
-        assert rld != null;
-        if (drop) {
-          rld.dropChanges();
-        } else {
-          rld.dropMergingUpdates();
-        }
-        rld.release(sr);
-        release(rld);
-        if (drop) {
-          readerPool.drop(rld.info);
-        }
-      });
+    try {
+      // first call mergeFinished before we potentially drop the reader and the last reference.
+      merge.mergeFinished(suppressExceptions == false, droppedSegment);

Review comment:
       this adds clear semantics when we do what rather than relying on implicit ordering of things in a finalizer.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mikemccand commented on a change in pull request #1601: LUCENE-8962: Ensure we don't include fully deleted segments in a commit

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #1601:
URL: https://github.com/apache/lucene-solr/pull/1601#discussion_r443772827



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -4341,27 +4342,33 @@ private synchronized void mergeFinish(MergePolicy.OneMerge merge) {
   }
 
   @SuppressWarnings("try")
-  private synchronized void closeMergeReaders(MergePolicy.OneMerge merge, boolean suppressExceptions) throws IOException {
+  private synchronized void closeMergeReaders(MergePolicy.OneMerge merge, boolean suppressExceptions, boolean droppedSegment) throws IOException {
     final boolean drop = suppressExceptions == false;
-    try (Closeable finalizer = () -> merge.mergeFinished(suppressExceptions == false)) {
-      IOUtils.applyToAll(merge.readers, sr -> {
-        final ReadersAndUpdates rld = getPooledInstance(sr.getOriginalSegmentInfo(), false);
-        // We still hold a ref so it should not have been removed:
-        assert rld != null;
-        if (drop) {
-          rld.dropChanges();
-        } else {
-          rld.dropMergingUpdates();
-        }
-        rld.release(sr);
-        release(rld);
-        if (drop) {
-          readerPool.drop(rld.info);
-        }
-      });
+    try {
+      // first call mergeFinished before we potentially drop the reader and the last reference.
+      merge.mergeFinished(suppressExceptions == false, droppedSegment);

Review comment:
       And this also reversed the order of these operations, right?  So now we first call `mergeFinished`, and then all the rld release/drop logic.

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3234,15 +3234,23 @@ private long prepareCommitInternal() throws IOException {
               // corresponding add from an updateDocument) can
               // sneak into the commit point:
               toCommit = segmentInfos.clone();
-
+              pendingCommitChangeCount = changeCount.get();

Review comment:
       I agree this is cleaner.  I don't think it should be a functional change since we are sync'd on IW's monitor lock here, no other threads will be able to make conflicting changes.

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3234,15 +3234,23 @@ private long prepareCommitInternal() throws IOException {
               // corresponding add from an updateDocument) can
               // sneak into the commit point:
               toCommit = segmentInfos.clone();
-
+              pendingCommitChangeCount = changeCount.get();
+              // This protects the segmentInfos we are now going
+              // to commit.  This is important in case, eg, while
+              // we are trying to sync all referenced files, a
+              // merge completes which would otherwise have
+              // removed the files we are now syncing.
+              deleter.incRef(toCommit.files(false));
               if (anyChanges && maxCommitMergeWaitSeconds > 0) {
                 SegmentInfos committingSegmentInfos = toCommit;
                 onCommitMerges = updatePendingMerges(new OneMergeWrappingMergePolicy(config.getMergePolicy(), toWrap ->
                     new MergePolicy.OneMerge(toWrap.segments) {
                       @Override
-                      public void mergeFinished(boolean committed) throws IOException {
+                      public void mergeFinished(boolean committed, boolean segmentDropped) throws IOException {
                         assert Thread.holdsLock(IndexWriter.this);
-                        if (committed && includeInCommit.get()) {
+                        if (segmentDropped == false

Review comment:
       This is the functional part of the fix, right?  If we will drop the segment (because it is 100% deleted), now we will not attempt to apply it to the commit?
   
   But, does this now mean that the attempted merged (for commit) segments, that are 100% deleted, will remain in the commit?  Shouldn't we also drop them from the commit point?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on pull request #1601: LUCENE-8962: Ensure we don't include fully deleted segments in a commit

Posted by GitBox <gi...@apache.org>.
s1monw commented on pull request #1601:
URL: https://github.com/apache/lucene-solr/pull/1601#issuecomment-648938745


   I pushed all the commits and fixes for your comments to this branch https://github.com/apache/lucene-solr/tree/jira/lucene-8962


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on pull request #1601: LUCENE-8962: Ensure we don't include fully deleted segments in a commit

Posted by GitBox <gi...@apache.org>.
s1monw commented on pull request #1601:
URL: https://github.com/apache/lucene-solr/pull/1601#issuecomment-647679593


   @mikemccand can you take a look


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org