mikemccand commented on a change in pull request #1274: LUCENE-9164: Prevent IW from closing gracefully if threads are still modifying URL: https://github.com/apache/lucene-solr/pull/1274#discussion_r387130244
########## File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java ########## @@ -2275,57 +2265,73 @@ private void rollbackInternalNoCommit() throws IOException { docWriter.close(); // mark it as closed first to prevent subsequent indexing actions/flushes assert !Thread.holdsLock(this) : "IndexWriter lock should never be hold when aborting"; - docWriter.abort(); // don't sync on IW here - docWriter.flushControl.waitForFlush(); // wait for all concurrently running flushes + docWriter.abort(); // don't sync on IW here - this waits for all concurrently running flushes publishFlushedSegments(true); // empty the flush ticket queue otherwise we might not have cleaned up all resources - synchronized (this) { - - if (pendingCommit != null) { - pendingCommit.rollbackCommit(directory); - try { - deleter.decRef(pendingCommit); - } finally { - pendingCommit = null; - notifyAll(); + // we might rollback due to a tragic event which means we potentially already have + // a lease in this case we we can't acquire all leases. In this case rolling back in best effort in terms + // off letting all threads gracefully finish. In such a situation some threads might run into + // AlreadyClosedExceptions in places they normally wouldn't which doesn't have any impact on + // correctness or consistency. The tragic event is fatal anyway. + final int leases; + if (gracefully) { // in the case + leases = Integer.MAX_VALUE; + modificationLease.acquireUninterruptibly(leases); + } else { + // still try to drain all permits to prevent any new threads modifying the index. + leases = modificationLease.drainPermits(); Review comment: This will block until all threads return their leases? Can you add a comment to that effect? ---------------------------------------------------------------- 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 With regards, Apache Git Services --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org