Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions lucene/CHANGES.txt
Original file line number Diff line number Diff line change
Expand Up @@ -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)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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<>();
}

/**
Expand Down Expand Up @@ -182,6 +182,8 @@ public TopFieldDocs reduce(Collection<TopFieldCollector> collectors) throws IOEx
return TopDocs.merge(sort, 0, numHits, topDocs);
}

/** Returns the collectors created by this manager. */
@Deprecated
public List<TopFieldCollector> getCollectors() {
return collectors;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -703,6 +711,45 @@ public void testRandomMinCompetitiveScore() throws Exception {
dir.close();
}

/**
* Test that concurrent newCollector() calls are thread-safe. See <a
* href="https://github.com/apache/lucene/issues/15605">GitHub issue #15605</a>
*/
@SuppressWarnings("deprecation")
Comment thread
prudhvigodithi marked this conversation as resolved.
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<Throwable> error = new AtomicReference<>();
List<Future<?>> 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();
Expand Down
Loading