mikemccand commented on a change in pull request #1443: URL: https://github.com/apache/lucene-solr/pull/1443#discussion_r412877253
########## File path: lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java ########## @@ -657,4 +659,88 @@ protected void doStall() { assertFalse(failed.get()); } + + /* + * This test tries to produce 2 merges running concurrently with 2 segments per merge. While these + * merges run we kick off a forceMerge that puts a pending merge in the queue but waits for things to happen. + * While we do this we reduce maxMergeCount to 1. If concurrency in CMS is not right the forceMerge will wait forever + * since non of the currently running merges picks up the pending merge. This test fails every + */ + public void testChangeMaxMergeCountyWhileForceMerge() throws IOException, InterruptedException { + int numIters = TEST_NIGHTLY ? 100 : 10; + for (int iters = 0; iters < numIters; iters++) { + LogDocMergePolicy mp = new LogDocMergePolicy(); + mp.setMergeFactor(2); + CountDownLatch forceMergeWaits = new CountDownLatch(1); + CountDownLatch mergeThreadsStartAfterWait = new CountDownLatch(1); + CountDownLatch mergeThreadsArrived = new CountDownLatch(2); + InfoStream stream = new InfoStream() { + @Override + public void message(String component, String message) { Review comment: Sneaky (using `InfoStream` to tickle things)! ########## File path: lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java ########## @@ -633,6 +634,27 @@ protected synchronized MergeThread getMergeThread(IndexWriter writer, OneMerge m return thread; } + synchronized void runOnMergeFinished(IndexWriter writer) { + // the merge call as well as the merge thread handling in the finally + // block must be synched on CMS otherwise stalling decisions might cause Review comment: s/`synched`/`sync'd`? ---------------------------------------------------------------- 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