mirror of https://github.com/apache/lucene.git
LUCENE-9962: Allow DrillSideways sub-classes to provide their own "drill down" facet counting implementation (or null). (#143)
This commit is contained in:
parent
c4cf7aa3e1
commit
3c7a76a148
|
@ -125,6 +125,9 @@ API Changes
|
|||
|
||||
* LUCENE-9956: Expose the getBaseQuery, getDrillDownQueries APIs from DrillDownQuery (Gautam Worah)
|
||||
|
||||
* LUCENE-9962: DrillSideways allows sub-classes to provide "drill down" FacetsCollectors. They
|
||||
may provide a null collector if they choose to bypass "drill down" facet collection. (Greg Miller)
|
||||
|
||||
Improvements
|
||||
|
||||
* LUCENE-9960: Avoid unnecessary top element replacement for equal elements in PriorityQueue. (Dawid Weiss)
|
||||
|
|
|
@ -124,16 +124,34 @@ public class DrillSideways {
|
|||
this.executor = executor;
|
||||
}
|
||||
|
||||
/**
|
||||
* Subclass can override to customize drill down facets collector. Returning {@code null} is valid
|
||||
* if no drill down facet collection is needed.
|
||||
*/
|
||||
protected FacetsCollector createDrillDownFacetsCollector() {
|
||||
return new FacetsCollector();
|
||||
}
|
||||
|
||||
/**
|
||||
* Subclass can override to customize drill down facets collector. Returning {@code null} is valid
|
||||
* if no drill down facet collection is needed.
|
||||
*/
|
||||
protected FacetsCollectorManager createDrillDownFacetsCollectorManager() {
|
||||
return new FacetsCollectorManager();
|
||||
}
|
||||
|
||||
/** Subclass can override to customize per-dim Facets impl. */
|
||||
protected Facets buildFacetsResult(
|
||||
FacetsCollector drillDowns, FacetsCollector[] drillSideways, String[] drillSidewaysDims)
|
||||
throws IOException {
|
||||
|
||||
Facets drillDownFacets;
|
||||
Facets drillDownFacets = null;
|
||||
Map<String, Facets> drillSidewaysFacets = new HashMap<>();
|
||||
|
||||
if (taxoReader != null) {
|
||||
if (drillDowns != null) {
|
||||
drillDownFacets = new FastTaxonomyFacetCounts(taxoReader, config, drillDowns);
|
||||
}
|
||||
if (drillSideways != null) {
|
||||
for (int i = 0; i < drillSideways.length; i++) {
|
||||
drillSidewaysFacets.put(
|
||||
|
@ -142,7 +160,9 @@ public class DrillSideways {
|
|||
}
|
||||
}
|
||||
} else {
|
||||
if (drillDowns != null) {
|
||||
drillDownFacets = new SortedSetDocValuesFacetCounts(state, drillDowns);
|
||||
}
|
||||
if (drillSideways != null) {
|
||||
for (int i = 0; i < drillSideways.length; i++) {
|
||||
drillSidewaysFacets.put(
|
||||
|
@ -166,12 +186,17 @@ public class DrillSideways {
|
|||
|
||||
Map<String, Integer> drillDownDims = query.getDims();
|
||||
|
||||
FacetsCollector drillDownCollector = new FacetsCollector();
|
||||
FacetsCollector drillDownCollector = createDrillDownFacetsCollector();
|
||||
|
||||
if (drillDownDims.isEmpty()) {
|
||||
// There are no drill-down dims, so there is no
|
||||
// drill-sideways to compute:
|
||||
if (drillDownCollector != null) {
|
||||
// Make sure we still populate a facet collector for the base query if desired:
|
||||
searcher.search(query, MultiCollector.wrap(hitCollector, drillDownCollector));
|
||||
} else {
|
||||
searcher.search(query, hitCollector);
|
||||
}
|
||||
return new DrillSidewaysResult(buildFacetsResult(drillDownCollector, null, null), null);
|
||||
}
|
||||
|
||||
|
@ -396,12 +421,17 @@ public class DrillSideways {
|
|||
final List<CallableCollector> callableCollectors = new ArrayList<>(drillDownDims.size() + 1);
|
||||
|
||||
// Add the main DrillDownQuery
|
||||
callableCollectors.add(
|
||||
new CallableCollector(
|
||||
-1,
|
||||
searcher,
|
||||
query,
|
||||
new MultiCollectorManager(new FacetsCollectorManager(), hitCollectorManager)));
|
||||
FacetsCollectorManager drillDownFacetsCollectorManager =
|
||||
createDrillDownFacetsCollectorManager();
|
||||
CollectorManager<?, ?> mainCollectorManager;
|
||||
if (drillDownFacetsCollectorManager != null) {
|
||||
// Make sure we populate a facet collector corresponding to the base query if desired:
|
||||
mainCollectorManager =
|
||||
new MultiCollectorManager(drillDownFacetsCollectorManager, hitCollectorManager);
|
||||
} else {
|
||||
mainCollectorManager = hitCollectorManager;
|
||||
}
|
||||
callableCollectors.add(new CallableCollector(-1, searcher, query, mainCollectorManager));
|
||||
int i = 0;
|
||||
final Query[] filters = query.getDrillDownQueries();
|
||||
for (String dim : drillDownDims.keySet())
|
||||
|
@ -418,9 +448,15 @@ public class DrillSideways {
|
|||
final List<Future<CallableResult>> futures = executor.invokeAll(callableCollectors);
|
||||
|
||||
// Extract the results
|
||||
if (drillDownFacetsCollectorManager != null) {
|
||||
// If we populated a facets collector for the main query, make sure to unpack it properly
|
||||
final Object[] mainResults = (Object[]) futures.get(0).get().result;
|
||||
mainFacetsCollector = (FacetsCollector) mainResults[0];
|
||||
collectorResult = (R) mainResults[1];
|
||||
} else {
|
||||
mainFacetsCollector = null;
|
||||
collectorResult = (R) futures.get(0).get().result;
|
||||
}
|
||||
for (i = 1; i < futures.size(); i++) {
|
||||
final CallableResult result = futures.get(i).get();
|
||||
facetsCollectors[result.pos] = (FacetsCollector) result.result;
|
||||
|
|
|
@ -25,6 +25,8 @@ import java.util.HashSet;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
|
@ -67,6 +69,7 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InPlaceMergeSorter;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.NamedThreadFactory;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
public class TestDrillSideways extends FacetTestCase {
|
||||
|
@ -1248,6 +1251,115 @@ public class TestDrillSideways extends FacetTestCase {
|
|||
IOUtils.close(taxoWriter, searcher.getIndexReader(), taxoReader, dir, taxoDir);
|
||||
}
|
||||
|
||||
public void testNoDrillDownFacetCollection() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
Directory taxoDir = newDirectory();
|
||||
|
||||
// Writes facet ords to a separate directory from the
|
||||
// main index:
|
||||
DirectoryTaxonomyWriter taxoWriter =
|
||||
new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE);
|
||||
|
||||
FacetsConfig config = new FacetsConfig();
|
||||
config.setHierarchical("Publish Date", true);
|
||||
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
|
||||
|
||||
Document doc = new Document();
|
||||
doc.add(new FacetField("Author", "Bob"));
|
||||
doc.add(new FacetField("Publish Date", "2010", "10", "15"));
|
||||
writer.addDocument(config.build(taxoWriter, doc));
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new FacetField("Author", "Lisa"));
|
||||
doc.add(new FacetField("Publish Date", "2010", "10", "20"));
|
||||
writer.addDocument(config.build(taxoWriter, doc));
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new FacetField("Author", "Lisa"));
|
||||
doc.add(new FacetField("Publish Date", "2012", "1", "1"));
|
||||
writer.addDocument(config.build(taxoWriter, doc));
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new FacetField("Author", "Susan"));
|
||||
doc.add(new FacetField("Publish Date", "2012", "1", "7"));
|
||||
writer.addDocument(config.build(taxoWriter, doc));
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new FacetField("Author", "Frank"));
|
||||
doc.add(new FacetField("Publish Date", "1999", "5", "5"));
|
||||
writer.addDocument(config.build(taxoWriter, doc));
|
||||
|
||||
// NRT open
|
||||
IndexSearcher searcher = getNewSearcher(writer.getReader());
|
||||
|
||||
// NRT open
|
||||
TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
|
||||
|
||||
// Sometimes pass an ExecutorService to test both paths for providing null drill down collectors
|
||||
// The actual configuration of the ExecutorService doesn't matter at all:
|
||||
ExecutorService executorService = null;
|
||||
if (random().nextBoolean()) {
|
||||
executorService = Executors.newSingleThreadExecutor(new NamedThreadFactory("ds_test"));
|
||||
}
|
||||
|
||||
// Don't collect for drill down faceting
|
||||
DrillSideways ds =
|
||||
new DrillSideways(searcher, config, taxoReader, null, executorService) {
|
||||
@Override
|
||||
protected FacetsCollector createDrillDownFacetsCollector() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FacetsCollectorManager createDrillDownFacetsCollectorManager() {
|
||||
return null;
|
||||
}
|
||||
};
|
||||
|
||||
// Another simple case: drill-down on single fields
|
||||
// but OR of two values
|
||||
DrillDownQuery ddq = new DrillDownQuery(config);
|
||||
ddq.add("Author", "Lisa");
|
||||
ddq.add("Author", "Bob");
|
||||
DrillSidewaysResult r = ds.search(null, ddq, 10);
|
||||
Facets facets = r.facets;
|
||||
assertEquals(3, r.hits.totalHits.value);
|
||||
assertEquals(
|
||||
"dim=Author path=[] value=5 childCount=4\n Lisa (2)\n Bob (1)\n Susan (1)\n Frank (1)\n",
|
||||
facets.getTopChildren(10, "Author").toString());
|
||||
// Because we don't collect drill-downs, we shouldn't be able to get counts for Publish Date
|
||||
expectThrows(IllegalArgumentException.class, () -> facets.getTopChildren(10, "Publish Date"));
|
||||
|
||||
assertTrue(facets instanceof MultiFacets);
|
||||
List<FacetResult> allResults = facets.getAllDims(10);
|
||||
// Should only have the one dimension because we didn't collect for drill down
|
||||
assertEquals(1, allResults.size());
|
||||
assertEquals(
|
||||
"dim=Author path=[] value=5 childCount=4\n Lisa (2)\n Bob (1)\n Susan (1)\n Frank (1)\n",
|
||||
allResults.get(0).toString());
|
||||
|
||||
// More interesting case: drill-down on two fields
|
||||
ddq = new DrillDownQuery(config);
|
||||
ddq.add("Author", "Lisa");
|
||||
ddq.add("Publish Date", "2010");
|
||||
r = ds.search(null, ddq, 10);
|
||||
assertEquals(1, r.hits.totalHits.value);
|
||||
// Should be able to count on both fields since they're both drill sideways cases
|
||||
assertEquals(
|
||||
"dim=Publish Date path=[] value=2 childCount=2\n 2010 (1)\n 2012 (1)\n",
|
||||
r.facets.getTopChildren(10, "Publish Date").toString());
|
||||
assertEquals(
|
||||
"dim=Author path=[] value=2 childCount=2\n Bob (1)\n Lisa (1)\n",
|
||||
r.facets.getTopChildren(10, "Author").toString());
|
||||
|
||||
if (executorService != null) {
|
||||
executorService.shutdown();
|
||||
}
|
||||
writer.close();
|
||||
IOUtils.close(searcher.getIndexReader(), taxoReader, taxoWriter, dir, taxoDir);
|
||||
}
|
||||
|
||||
public void testScorer() throws Exception {
|
||||
// LUCENE-6001 some scorers, eg ReqExlScorer, can hit NPE if cost is called after nextDoc
|
||||
Directory dir = newDirectory();
|
||||
|
|
Loading…
Reference in New Issue