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: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org