msfroh commented on a change in pull request #1155: LUCENE-8962: Add ability to selectively merge on commit URL: https://github.com/apache/lucene-solr/pull/1155#discussion_r368182168
########## File path: lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java ########## @@ -277,6 +285,92 @@ public void testSetters() { assertSetters(new LogDocMergePolicy()); } + public void testMergeOnCommit() throws IOException, InterruptedException { + Directory dir = newDirectory(); + IndexWriter firstWriter = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())) + .setMergePolicy(NoMergePolicy.INSTANCE)); + for (int i = 0; i < 5; i++) { + TestIndexWriter.addDoc(firstWriter); + firstWriter.flush(); + } + DirectoryReader firstReader = DirectoryReader.open(firstWriter); + assertEquals(5, firstReader.leaves().size()); + firstReader.close(); + firstWriter.close(); + + MergePolicy mergeOnCommitPolicy = new LogDocMergePolicy() { + @Override + public MergeSpecification findCommitMerges(SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException { + // Optimize down to a single segment on commit + MergeSpecification mergeSpecification = new MergeSpecification(); + List<SegmentCommitInfo> nonMergingSegments = new ArrayList<>(); + for (SegmentCommitInfo sci : segmentInfos) { + if (mergeContext.getMergingSegments().contains(sci) == false) { + nonMergingSegments.add(sci); + } + } + mergeSpecification.add(new OneMerge(nonMergingSegments)); + return mergeSpecification; + } + }; + + IndexWriter writerWithMergePolicy = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())) + .setMergePolicy(mergeOnCommitPolicy)); + + writerWithMergePolicy.commit(); + + DirectoryReader unmergedReader = DirectoryReader.open(writerWithMergePolicy); + assertEquals(5, unmergedReader.leaves().size()); // Don't merge unless there's a change + unmergedReader.close(); + + TestIndexWriter.addDoc(writerWithMergePolicy); + writerWithMergePolicy.commit(); + + DirectoryReader mergedReader = DirectoryReader.open(writerWithMergePolicy); + assertEquals(1, mergedReader.leaves().size()); // Now we merge on commit + mergedReader.close(); + + LineFileDocs lineFileDocs = new LineFileDocs(random()); + int docCount = atLeast(1000); + AtomicInteger indexedDocs = new AtomicInteger(0); + int numIndexingThreads = atLeast(2); + CountDownLatch startingGun = new CountDownLatch(1); + Collection<Thread> indexingThreads = new ArrayList<>(); + for (int i = 0; i < numIndexingThreads; i++) { + Thread t = new Thread(() -> { + try { + while (indexedDocs.getAndIncrement() < docCount) { + writerWithMergePolicy.addDocument(lineFileDocs.nextDoc()); + if (rarely()) { + writerWithMergePolicy.commit(); + } + } + } catch (IOException e) { + e.printStackTrace(); + fail(); + } + }); + t.start(); + indexingThreads.add(t); + } + startingGun.countDown(); + for (Thread t : indexingThreads) { + t.join(); + } + writerWithMergePolicy.commit(); + assertEquals(1, writerWithMergePolicy.listOfSegmentCommitInfos().size()); Review comment: I just found that this assertion sometimes fails. If there are some pending/running merges left over from the indexing threads, the segments associated with those merges will be excluded from merging on commit. I'll update this test to wait for pending merges to finish before committing. ---------------------------------------------------------------- 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