diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 9db4d195d4ba..434ea60d8ac3 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -130,6 +130,8 @@ Optimizations Bug Fixes --------------------- +* GITHUB#15605: Make TopFieldCollectorManager thread-safe by using CopyOnWriteArrayList and deprecated getCollectors(). (Prudhvi Godithi) + * GITHUB#14049: Randomize KNN codec params in RandomCodec. Fixes scalar quantization div-by-zero when all values are identical. (Mike Sokolov) diff --git a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java index c62e9dc9f255..56367811255d 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java @@ -17,9 +17,9 @@ package org.apache.lucene.search; import java.io.IOException; -import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; /** * Create a TopFieldCollectorManager which uses a shared hit counter to maintain number of hits and @@ -111,7 +111,7 @@ public TopFieldCollectorManager(Sort sort, int numHits, FieldDoc after, int tota this.after = after; this.totalHitsThreshold = totalHitsThreshold; this.minScoreAcc = totalHitsThreshold != Integer.MAX_VALUE ? new MaxScoreAccumulator() : null; - this.collectors = new ArrayList<>(); + this.collectors = new CopyOnWriteArrayList<>(); } /** @@ -182,6 +182,8 @@ public TopFieldDocs reduce(Collection collectors) throws IOEx return TopDocs.merge(sort, 0, numHits, topDocs); } + /** Returns the collectors created by this manager. */ + @Deprecated public List getCollectors() { return collectors; } diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java index bcbc8cac50d2..0ad511a9501c 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java @@ -22,9 +22,16 @@ import static org.hamcrest.Matchers.sameInstance; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Comparator; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicReference; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.Field.Store; @@ -45,6 +52,7 @@ import org.apache.lucene.tests.search.CheckHits; import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.lucene.tests.util.TestUtil; +import org.apache.lucene.util.NamedThreadFactory; public class TestTopFieldCollector extends LuceneTestCase { private IndexSearcher is; @@ -703,6 +711,45 @@ public void testRandomMinCompetitiveScore() throws Exception { dir.close(); } + /** + * Test that concurrent newCollector() calls are thread-safe. See GitHub issue #15605 + */ + @SuppressWarnings("deprecation") + public void testConcurrentNewCollector() throws Exception { + Sort sort = new Sort(SortField.FIELD_SCORE); + TopFieldCollectorManager manager = new TopFieldCollectorManager(sort, 10, null, 100); + int numThreads = 8; + int callsPerThread = 1000; + int expectedTotal = numThreads * callsPerThread; + ExecutorService executor = + Executors.newFixedThreadPool( + numThreads, new NamedThreadFactory("testConcurrentNewCollector")); + CountDownLatch startLatch = new CountDownLatch(1); + AtomicReference error = new AtomicReference<>(); + List> futures = new ArrayList<>(); + for (int i = 0; i < numThreads; i++) { + futures.add( + executor.submit( + () -> { + try { + startLatch.await(); + for (int j = 0; j < callsPerThread; j++) { + manager.newCollector(); + } + } catch (Throwable t) { + error.compareAndSet(null, t); + } + })); + } + startLatch.countDown(); + for (Future f : futures) { + f.get(); + } + executor.shutdown(); + assertEquals(expectedTotal, manager.getCollectors().size()); + } + public void testRelationVsTopDocsCount() throws Exception { Sort sort = new Sort(SortField.FIELD_SCORE, SortField.FIELD_DOC); try (Directory dir = newDirectory();