LUCENE-9945: Extend DrillSidewaysResult to expose drillDowns and drillSideways (#159)

This commit is contained in:
Sejal Pawar 2021-08-03 01:01:08 +02:00 committed by GitHub
parent 7450a7e64b
commit a76f2f8072
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 177 additions and 23 deletions

View File

@ -245,6 +245,8 @@ Improvements
* LUCENE-10019: Align file starts in CFS files to have proper alignment (8 bytes) * LUCENE-10019: Align file starts in CFS files to have proper alignment (8 bytes)
(Uwe Schinder) (Uwe Schinder)
* LUCENE-9945: Extend DrillSideways to support exposing FacetCollectors directly. (Greg Miller, Sejal Pawar)
Bug fixes Bug fixes
* LUCENE-9686: Fix read past EOF handling in DirectIODirectory. (Zach Chen, * LUCENE-9686: Fix read past EOF handling in DirectIODirectory. (Zach Chen,

View File

@ -210,7 +210,8 @@ public class DrillSideways {
} else { } else {
searcher.search(query, hitCollector); searcher.search(query, hitCollector);
} }
return new DrillSidewaysResult(buildFacetsResult(drillDownCollector, null, null), null); return new DrillSidewaysResult(
buildFacetsResult(drillDownCollector, null, null), null, drillDownCollector, null, null);
} }
Query baseQuery = query.getBaseQuery(); Query baseQuery = query.getBaseQuery();
@ -272,12 +273,14 @@ public class DrillSideways {
drillSidewaysFacetsCollectorManagers[dim].reduce(facetsCollectorsForDim); drillSidewaysFacetsCollectorManagers[dim].reduce(facetsCollectorsForDim);
} }
String[] drillSidewaysDims = drillDownDims.keySet().toArray(new String[0]);
return new DrillSidewaysResult( return new DrillSidewaysResult(
buildFacetsResult( buildFacetsResult(drillDownCollector, drillSidewaysCollectors, drillSidewaysDims),
null,
drillDownCollector, drillDownCollector,
drillSidewaysCollectors, drillSidewaysCollectors,
drillDownDims.keySet().toArray(new String[0])), drillSidewaysDims);
null);
} }
/** Search, sorting by {@link Sort}, and computing drill down and sideways counts. */ /** Search, sorting by {@link Sort}, and computing drill down and sideways counts. */
@ -318,7 +321,12 @@ public class DrillSideways {
if (doDocScores) { if (doDocScores) {
TopFieldCollector.populateScores(topDocs.scoreDocs, searcher, query); TopFieldCollector.populateScores(topDocs.scoreDocs, searcher, query);
} }
return new DrillSidewaysResult(r.facets, topDocs); return new DrillSidewaysResult(
r.facets,
topDocs,
r.drillDownFacetsCollector,
r.drillSidewaysFacetsCollector,
r.drillSidewaysDims);
} else { } else {
@ -329,7 +337,12 @@ public class DrillSideways {
if (doDocScores) { if (doDocScores) {
TopFieldCollector.populateScores(topDocs.scoreDocs, searcher, query); TopFieldCollector.populateScores(topDocs.scoreDocs, searcher, query);
} }
return new DrillSidewaysResult(r.facets, topDocs); return new DrillSidewaysResult(
r.facets,
topDocs,
r.drillDownFacetsCollector,
r.drillSidewaysFacetsCollector,
r.drillSidewaysDims);
} }
} else { } else {
return search(after, query, topN); return search(after, query, topN);
@ -370,14 +383,24 @@ public class DrillSideways {
} }
}; };
ConcurrentDrillSidewaysResult<TopDocs> r = searchConcurrently(query, collectorManager); ConcurrentDrillSidewaysResult<TopDocs> r = searchConcurrently(query, collectorManager);
return new DrillSidewaysResult(r.facets, r.collectorResult); return new DrillSidewaysResult(
r.facets,
r.collectorResult,
r.drillDownFacetsCollector,
r.drillSidewaysFacetsCollector,
r.drillSidewaysDims);
} else { } else {
TopScoreDocCollector hitCollector = TopScoreDocCollector hitCollector =
TopScoreDocCollector.create(topN, after, Integer.MAX_VALUE); TopScoreDocCollector.create(topN, after, Integer.MAX_VALUE);
DrillSidewaysResult r = search(query, hitCollector); DrillSidewaysResult r = search(query, hitCollector);
return new DrillSidewaysResult(r.facets, hitCollector.topDocs()); return new DrillSidewaysResult(
r.facets,
hitCollector.topDocs(),
r.drillDownFacetsCollector,
r.drillSidewaysFacetsCollector,
r.drillSidewaysDims);
} }
} }
@ -390,7 +413,11 @@ public class DrillSideways {
return false; return false;
} }
/** Result of a drill sideways search, including the {@link Facets} and {@link TopDocs}. */ /**
* Result of a drill sideways search, including the {@link Facets} and {@link TopDocs}. The {@link
* FacetsCollector}s for the drill down and drill sideways dimensions are also exposed for
* advanced use-cases that need access to them as an alternative to accessing the {@code Facets}.
*/
public static class DrillSidewaysResult { public static class DrillSidewaysResult {
/** Combined drill down and sideways results. */ /** Combined drill down and sideways results. */
public final Facets facets; public final Facets facets;
@ -398,10 +425,41 @@ public class DrillSideways {
/** Hits. */ /** Hits. */
public final TopDocs hits; public final TopDocs hits;
/**
* FacetsCollector populated based on hits that match the full DrillDownQuery, treating all
* drill down dimensions as required clauses. Useful for advanced use-cases that want to compute
* Facets results separate from the provided Facets in this result.
*/
public final FacetsCollector drillDownFacetsCollector;
/**
* FacetsCollectors populated for each drill sideways dimension. Each collector exposes the hits
* that match on all DrillDownQuery dimensions, but treating their corresponding sideways
* dimension as optional. This array provides a FacetsCollector for each drill down dimension
* present in the original DrillDownQuery, and the associated dimension for each FacetsCollector
* can be determined using the parallel {@link DrillSidewaysResult#drillSidewaysDims} array.
* Useful for advanced use-cases that want to compute Facets results separate from the provided
* Facets in this result.
*/
public final FacetsCollector[] drillSidewaysFacetsCollector;
/**
* Dimensions that correspond to to the {@link DrillSidewaysResult#drillSidewaysFacetsCollector}
*/
public final String[] drillSidewaysDims;
/** Sole constructor. */ /** Sole constructor. */
public DrillSidewaysResult(Facets facets, TopDocs hits) { public DrillSidewaysResult(
Facets facets,
TopDocs hits,
FacetsCollector drillDownFacetsCollector,
FacetsCollector[] drillSidewaysFacetsCollector,
String[] drillSidewaysDims) {
this.facets = facets; this.facets = facets;
this.hits = hits; this.hits = hits;
this.drillDownFacetsCollector = drillDownFacetsCollector;
this.drillSidewaysFacetsCollector = drillSidewaysFacetsCollector;
this.drillSidewaysDims = drillSidewaysDims;
} }
} }
@ -487,7 +545,12 @@ public class DrillSideways {
} }
return new ConcurrentDrillSidewaysResult<>( return new ConcurrentDrillSidewaysResult<>(
buildFacetsResult(mainFacetsCollector, null, null), null, collectorResult); buildFacetsResult(mainFacetsCollector, null, null),
null,
collectorResult,
mainFacetsCollector,
null,
null);
} }
Query baseQuery = query.getBaseQuery(); Query baseQuery = query.getBaseQuery();
@ -539,13 +602,15 @@ public class DrillSideways {
drillSidewaysFacetsCollectorManagers[dim].reduce(facetsCollectorsForDim); drillSidewaysFacetsCollectorManagers[dim].reduce(facetsCollectorsForDim);
} }
String[] drillSidewaysDims = drillDownDims.keySet().toArray(new String[0]);
return new ConcurrentDrillSidewaysResult<>( return new ConcurrentDrillSidewaysResult<>(
buildFacetsResult( buildFacetsResult(drillDownCollector, drillSidewaysCollectors, drillSidewaysDims),
null,
collectorResult,
drillDownCollector, drillDownCollector,
drillSidewaysCollectors, drillSidewaysCollectors,
drillDownDims.keySet().toArray(new String[0])), drillSidewaysDims);
null,
collectorResult);
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@ -607,12 +672,16 @@ public class DrillSideways {
throw new RuntimeException(e); throw new RuntimeException(e);
} }
String[] drillSidewaysDims = drillDownDims.keySet().toArray(new String[0]);
// build the facets and return the result // build the facets and return the result
return new ConcurrentDrillSidewaysResult<>( return new ConcurrentDrillSidewaysResult<>(
buildFacetsResult( buildFacetsResult(mainFacetsCollector, facetsCollectors, drillSidewaysDims),
mainFacetsCollector, facetsCollectors, drillDownDims.keySet().toArray(new String[0])),
null, null,
collectorResult); collectorResult,
mainFacetsCollector,
facetsCollectors,
drillSidewaysDims);
} }
/** /**
@ -624,8 +693,15 @@ public class DrillSideways {
public final R collectorResult; public final R collectorResult;
/** Sole constructor. */ /** Sole constructor. */
ConcurrentDrillSidewaysResult(Facets facets, TopDocs hits, R collectorResult) { ConcurrentDrillSidewaysResult(
super(facets, hits); Facets facets,
TopDocs hits,
R collectorResult,
FacetsCollector drillDownFacetsCollector,
FacetsCollector[] drillSidewaysFacetsCollector,
String[] drillSidewaysDims) {
super(
facets, hits, drillDownFacetsCollector, drillSidewaysFacetsCollector, drillSidewaysDims);
this.collectorResult = collectorResult; this.collectorResult = collectorResult;
} }
} }

View File

@ -1798,4 +1798,80 @@ public class TestDrillSideways extends FacetTestCase {
writer.close(); writer.close();
IOUtils.close(searcher.getIndexReader(), taxoReader, taxoWriter, dir, taxoDir); IOUtils.close(searcher.getIndexReader(), taxoReader, taxoWriter, dir, taxoDir);
} }
public void testExtendedDrillSidewaysResult() throws Exception {
// LUCENE-9945: Extend DrillSideways to support exposing FacetCollectors directly
Directory dir = newDirectory();
Directory taxoDir = newDirectory();
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
DirectoryTaxonomyWriter taxoWriter =
new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE);
FacetsConfig config = new FacetsConfig();
config.setHierarchical("dim", true);
Document doc = new Document();
doc.add(new FacetField("dim", "a"));
writer.addDocument(config.build(taxoWriter, doc));
Document doc2 = new Document();
doc.add(new FacetField("dim", "x"));
writer.addDocument(config.build(taxoWriter, doc2));
// open NRT
IndexSearcher searcher = getNewSearcher(writer.getReader());
TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
DrillDownQuery ddq = new DrillDownQuery(config);
ddq.add("dim", "x");
DrillSideways ds = getNewDrillSidewaysBuildFacetsResult(searcher, config, taxoReader);
SimpleCollectorManager manager =
new SimpleCollectorManager(
10, (a, b) -> Float.compare(b.docAndScore.score, a.docAndScore.score));
SimpleCollector collector = manager.newCollector();
// Sometimes pass in a Collector and sometimes CollectorManager
// so that we can test both DrillSidewaysResult and ConcurrentDrillSidewaysResult
DrillSidewaysResult r;
if (random().nextBoolean()) {
r = ds.search(ddq, collector);
} else {
r = ds.search(ddq, manager);
}
// compute Facets using exposed FacetCollectors from DrillSidewaysResult
Map<String, Facets> drillSidewaysFacets = new HashMap<>();
Facets drillDownFacets = getTaxonomyFacetCounts(taxoReader, config, r.drillDownFacetsCollector);
if (r.drillSidewaysFacetsCollector != null) {
for (int i = 0; i < r.drillSidewaysFacetsCollector.length; i++) {
drillSidewaysFacets.put(
r.drillSidewaysDims[i],
getTaxonomyFacetCounts(taxoReader, config, r.drillSidewaysFacetsCollector[i]));
}
}
Facets facets;
if (drillSidewaysFacets.isEmpty()) {
facets = drillDownFacets;
} else {
facets = new MultiFacets(drillSidewaysFacets, drillDownFacets);
}
// Facets computed using FacetsCollecter exposed in DrillSidewaysResult
// should match the Facets computed by {@link DrillSideways#buildFacetsResult}
FacetResult facetResultActual = facets.getTopChildren(2, "dim");
FacetResult facetResultExpected = r.facets.getTopChildren(2, "dim");
assertEquals(facetResultExpected.dim, facetResultActual.dim);
assertEquals(facetResultExpected.path.length, facetResultActual.path.length);
assertEquals(facetResultExpected.value, facetResultActual.value);
assertEquals(facetResultExpected.childCount, facetResultActual.childCount);
writer.close();
IOUtils.close(searcher.getIndexReader(), taxoReader, taxoWriter, dir, taxoDir);
}
} }