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:
us...@infra.apache.org



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

Reply via email to