mirror of https://github.com/apache/lucene.git
LUCENE-8487: add IndexSearcher.getSlices
This commit is contained in:
parent
2b395dabb8
commit
ae3929c3ed
|
@ -140,6 +140,10 @@ API Changes:
|
|||
|
||||
* LUCENE-8397: Add DirectoryTaxonomyWriter.getCache (Mike McCandless)
|
||||
|
||||
* LUCENE-8387: Add experimental IndexSearcher.getSlices API to see which slices
|
||||
IndexSearcher is searching concurrently when it's created with an ExecutorService
|
||||
(Mike McCandless)
|
||||
|
||||
Bug Fixes:
|
||||
|
||||
* LUCENE-8380: UTF8TaxonomyWriterCache inconsistency. (Ruslan Torobaev, Dawid Weiss)
|
||||
|
|
|
@ -90,7 +90,7 @@ public class IndexSearcher {
|
|||
protected final IndexReaderContext readerContext;
|
||||
protected final List<LeafReaderContext> leafContexts;
|
||||
/** used with executor - each slice holds a set of leafs executed within one thread */
|
||||
protected final LeafSlice[] leafSlices;
|
||||
private final LeafSlice[] leafSlices;
|
||||
|
||||
// These are only used for multi-threaded search
|
||||
private final ExecutorService executor;
|
||||
|
@ -352,6 +352,14 @@ public class IndexSearcher {
|
|||
return search(query, collectorManager);
|
||||
}
|
||||
|
||||
/** Returns the leaf slices used for concurrent searching, or null if no {@code ExecutorService} was
|
||||
* passed to the constructor.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
public LeafSlice[] getSlices() {
|
||||
return leafSlices;
|
||||
}
|
||||
|
||||
/** Finds the top <code>n</code>
|
||||
* hits for <code>query</code> where all results are after a previous
|
||||
* result (<code>after</code>).
|
||||
|
@ -709,7 +717,11 @@ public class IndexSearcher {
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public static class LeafSlice {
|
||||
final LeafReaderContext[] leaves;
|
||||
|
||||
/** The leaves that make up this slice.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
public final LeafReaderContext[] leaves;
|
||||
|
||||
public LeafSlice(LeafReaderContext... leaves) {
|
||||
this.leaves = leaves;
|
||||
|
|
|
@ -25,8 +25,8 @@ import java.util.concurrent.ThreadPoolExecutor;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.Field.Store;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.SortedDocValuesField;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
|
@ -219,4 +219,26 @@ public class TestIndexSearcher extends LuceneTestCase {
|
|||
searcher = new IndexSearcher(new MultiReader());
|
||||
assertEquals(dummyPolicy, searcher.getQueryCachingPolicy());
|
||||
}
|
||||
|
||||
public void testGetSlices() throws Exception {
|
||||
assertNull(new IndexSearcher(new MultiReader()).getSlices());
|
||||
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
w.addDocument(new Document());
|
||||
IndexReader r = w.getReader();
|
||||
w.close();
|
||||
|
||||
ExecutorService service = new ThreadPoolExecutor(4, 4, 0L, TimeUnit.MILLISECONDS,
|
||||
new LinkedBlockingQueue<Runnable>(),
|
||||
new NamedThreadFactory("TestIndexSearcher"));
|
||||
IndexSearcher s = new IndexSearcher(r, service);
|
||||
IndexSearcher.LeafSlice[] slices = s.getSlices();
|
||||
assertNotNull(slices);
|
||||
assertEquals(1, slices.length);
|
||||
assertEquals(1, slices[0].leaves.length);
|
||||
assertTrue(slices[0].leaves[0] == r.leaves().get(0));
|
||||
service.shutdown();
|
||||
IOUtils.close(r, dir);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue