LUCENE-10050 Remove DrillSideways#search(DrillDownQuery,Collector) in favor of DrillSideways#search(DrillDownQuery,CollectorManager) (#632)

This commit is contained in:
Gautam Worah 2022-02-04 15:25:52 -08:00 committed by GitHub
parent ff2189c477
commit de4eccbb55
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 133 additions and 271 deletions

View File

@ -75,6 +75,10 @@ API Changes
* LUCENE-10368: IntTaxonomyFacets has been deprecated and is no longer a supported extension point
for user-created faceting implementations. (Greg Miller)
* LUCENE-10050: Deprecate DrillSideways#search(Query, Collector) in favor of
DrillSideways#search(Query, CollectorManager). This reflects the change (LUCENE-10002) being made in
IndexSearcher#search that trends towards using CollectorManagers over Collectors. (Gautam Worah)
New Features
---------------------

View File

@ -31,12 +31,10 @@ import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetField;
import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState;
import org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts;
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.CollectorManager;
import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.MultiCollector;
import org.apache.lucene.search.MultiCollectorManager;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
@ -185,92 +183,6 @@ public class DrillSideways {
}
}
/**
* Search, collecting hits with a {@link Collector}, and computing drill down and sideways counts.
*
* <p>Note that "concurrent" drill sideways will not be invoked here, even if an {@link
* ExecutorService} was supplied to the ctor, since {@code Collector}s are not thread-safe. If
* interested in concurrent drill sideways, please use one of the other static {@code search}
* methods.
*/
public DrillSidewaysResult search(DrillDownQuery query, Collector hitCollector)
throws IOException {
Map<String, Integer> drillDownDims = query.getDims();
if (drillDownDims.isEmpty()) {
// There are no drill-down dims, so there is no
// drill-sideways to compute:
FacetsCollector drillDownCollector = createDrillDownFacetsCollector();
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, drillDownCollector, null, null);
}
Query baseQuery = query.getBaseQuery();
if (baseQuery == null) {
// TODO: we could optimize this pure-browse case by
// making a custom scorer instead:
baseQuery = new MatchAllDocsQuery();
}
Query[] drillDownQueries = query.getDrillDownQueries();
int numDims = drillDownDims.size();
FacetsCollectorManager drillDownCollectorManager = createDrillDownFacetsCollectorManager();
FacetsCollectorManager[] drillSidewaysFacetsCollectorManagers =
new FacetsCollectorManager[numDims];
for (int i = 0; i < numDims; i++) {
drillSidewaysFacetsCollectorManagers[i] = new FacetsCollectorManager();
}
DrillSidewaysQuery dsq =
new DrillSidewaysQuery(
baseQuery,
drillDownCollectorManager,
drillSidewaysFacetsCollectorManagers,
drillDownQueries,
scoreSubDocsAtOnce());
searcher.search(dsq, hitCollector);
FacetsCollector drillDownCollector;
if (drillDownCollectorManager != null) {
drillDownCollector = drillDownCollectorManager.reduce(dsq.managedDrillDownCollectors);
} else {
drillDownCollector = null;
}
FacetsCollector[] drillSidewaysCollectors = new FacetsCollector[numDims];
int numSlices = dsq.managedDrillSidewaysCollectors.size();
for (int dim = 0; dim < numDims; dim++) {
List<FacetsCollector> facetsCollectorsForDim = new ArrayList<>(numSlices);
for (int slice = 0; slice < numSlices; slice++) {
facetsCollectorsForDim.add(dsq.managedDrillSidewaysCollectors.get(slice)[dim]);
}
drillSidewaysCollectors[dim] =
drillSidewaysFacetsCollectorManagers[dim].reduce(facetsCollectorsForDim);
}
String[] drillSidewaysDims = drillDownDims.keySet().toArray(new String[0]);
return new DrillSidewaysResult(
buildFacetsResult(drillDownCollector, drillSidewaysCollectors, drillSidewaysDims),
null,
drillDownCollector,
drillSidewaysCollectors,
drillSidewaysDims);
}
/** Search, sorting by {@link Sort}, and computing drill down and sideways counts. */
public DrillSidewaysResult search(
DrillDownQuery query, Query filter, FieldDoc after, int topN, Sort sort, boolean doDocScores)
@ -285,53 +197,36 @@ public class DrillSideways {
}
final int fTopN = Math.min(topN, limit);
if (executor != null) { // We have an executor, let use the multi-threaded version
final CollectorManager<TopFieldCollector, TopFieldDocs> collectorManager =
new CollectorManager<>() {
final CollectorManager<TopFieldCollector, TopFieldDocs> collectorManager =
new CollectorManager<>() {
@Override
public TopFieldCollector newCollector() {
return TopFieldCollector.create(sort, fTopN, after, Integer.MAX_VALUE);
}
@Override
public TopFieldCollector newCollector() {
return TopFieldCollector.create(sort, fTopN, after, Integer.MAX_VALUE);
}
@Override
public TopFieldDocs reduce(Collection<TopFieldCollector> collectors) {
final TopFieldDocs[] topFieldDocs = new TopFieldDocs[collectors.size()];
int pos = 0;
for (TopFieldCollector collector : collectors)
topFieldDocs[pos++] = collector.topDocs();
return TopDocs.merge(sort, topN, topFieldDocs);
}
};
@Override
public TopFieldDocs reduce(Collection<TopFieldCollector> collectors) {
final TopFieldDocs[] topFieldDocs = new TopFieldDocs[collectors.size()];
int pos = 0;
for (TopFieldCollector collector : collectors)
topFieldDocs[pos++] = collector.topDocs();
return TopDocs.merge(sort, topN, topFieldDocs);
}
};
ConcurrentDrillSidewaysResult<TopFieldDocs> r = searchConcurrently(query, collectorManager);
TopFieldDocs topDocs = r.collectorResult;
if (doDocScores) {
TopFieldCollector.populateScores(topDocs.scoreDocs, searcher, query);
}
return new DrillSidewaysResult(
r.facets,
topDocs,
r.drillDownFacetsCollector,
r.drillSidewaysFacetsCollector,
r.drillSidewaysDims);
final ConcurrentDrillSidewaysResult<TopFieldDocs> r = search(query, collectorManager);
} else {
final TopFieldCollector hitCollector =
TopFieldCollector.create(sort, fTopN, after, Integer.MAX_VALUE);
DrillSidewaysResult r = search(query, hitCollector);
TopFieldDocs topDocs = hitCollector.topDocs();
if (doDocScores) {
TopFieldCollector.populateScores(topDocs.scoreDocs, searcher, query);
}
return new DrillSidewaysResult(
r.facets,
topDocs,
r.drillDownFacetsCollector,
r.drillSidewaysFacetsCollector,
r.drillSidewaysDims);
TopFieldDocs topDocs = r.collectorResult;
if (doDocScores) {
TopFieldCollector.populateScores(topDocs.scoreDocs, searcher, query);
}
return new DrillSidewaysResult(
r.facets,
r.collectorResult,
r.drillDownFacetsCollector,
r.drillSidewaysFacetsCollector,
r.drillSidewaysDims);
} else {
return search(after, query, topN);
}
@ -351,45 +246,32 @@ public class DrillSideways {
}
final int fTopN = Math.min(topN, limit);
if (executor != null) { // We have an executor, let use the multi-threaded version
final CollectorManager<TopScoreDocCollector, TopDocs> collectorManager =
new CollectorManager<>() {
final CollectorManager<TopScoreDocCollector, TopDocs> collectorManager =
new CollectorManager<>() {
@Override
public TopScoreDocCollector newCollector() {
return TopScoreDocCollector.create(fTopN, after, Integer.MAX_VALUE);
}
@Override
public TopScoreDocCollector newCollector() {
return TopScoreDocCollector.create(fTopN, after, Integer.MAX_VALUE);
@Override
public TopDocs reduce(Collection<TopScoreDocCollector> collectors) {
final TopDocs[] topDocs = new TopDocs[collectors.size()];
int pos = 0;
for (TopScoreDocCollector collector : collectors) {
topDocs[pos++] = collector.topDocs();
}
return TopDocs.merge(topN, topDocs);
}
};
@Override
public TopDocs reduce(Collection<TopScoreDocCollector> collectors) {
final TopDocs[] topDocs = new TopDocs[collectors.size()];
int pos = 0;
for (TopScoreDocCollector collector : collectors)
topDocs[pos++] = collector.topDocs();
return TopDocs.merge(topN, topDocs);
}
};
ConcurrentDrillSidewaysResult<TopDocs> r = searchConcurrently(query, collectorManager);
return new DrillSidewaysResult(
r.facets,
r.collectorResult,
r.drillDownFacetsCollector,
r.drillSidewaysFacetsCollector,
r.drillSidewaysDims);
} else {
TopScoreDocCollector hitCollector =
TopScoreDocCollector.create(topN, after, Integer.MAX_VALUE);
DrillSidewaysResult r = search(query, hitCollector);
return new DrillSidewaysResult(
r.facets,
hitCollector.topDocs(),
r.drillDownFacetsCollector,
r.drillSidewaysFacetsCollector,
r.drillSidewaysDims);
}
final ConcurrentDrillSidewaysResult<TopDocs> r = search(query, collectorManager);
return new DrillSidewaysResult(
r.facets,
r.collectorResult,
r.drillDownFacetsCollector,
r.drillSidewaysFacetsCollector,
r.drillSidewaysDims);
}
/**

View File

@ -1,61 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.facet;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.search.Scorable;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.SimpleCollector;
/** Verifies in collect() that all child subScorers are on the collected doc. */
class AssertingSubDocsAtOnceCollector extends SimpleCollector {
// TODO: allow wrapping another Collector
List<Scorable> allScorers;
@Override
public void setScorer(Scorable s) throws IOException {
// Gathers all scorers, including s and "under":
allScorers = new ArrayList<>();
allScorers.add(s);
int upto = 0;
while (upto < allScorers.size()) {
s = allScorers.get(upto++);
for (Scorable.ChildScorable sub : s.getChildren()) {
allScorers.add(sub.child);
}
}
}
@Override
public void collect(int docID) {
for (Scorable s : allScorers) {
if (docID != s.docID()) {
throw new IllegalStateException(
"subScorer=" + s + " has docID=" + s.docID() + " != collected docID=" + docID);
}
}
}
@Override
public ScoreMode scoreMode() {
return ScoreMode.COMPLETE_NO_SCORES;
}
}

View File

@ -0,0 +1,76 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.facet;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import org.apache.lucene.search.CollectorManager;
import org.apache.lucene.search.Scorable;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.SimpleCollector;
class AssertingSubDocsAtOnceCollectorManager implements CollectorManager<SimpleCollector, Object> {
@Override
public AssertingSubDocsAtOnceCollector newCollector() {
return new AssertingSubDocsAtOnceCollector();
}
@Override
public Object reduce(Collection<SimpleCollector> collectors) {
return null;
}
/** Verifies in collect() that all child subScorers are on the collected doc. */
static class AssertingSubDocsAtOnceCollector extends SimpleCollector {
// TODO: allow wrapping another Collector
List<Scorable> allScorers;
@Override
public void setScorer(Scorable s) throws IOException {
// Gathers all scorers, including s and "under":
allScorers = new ArrayList<>();
allScorers.add(s);
int upto = 0;
while (upto < allScorers.size()) {
s = allScorers.get(upto++);
for (Scorable.ChildScorable sub : s.getChildren()) {
allScorers.add(sub.child);
}
}
}
@Override
public void collect(int docID) {
for (Scorable s : allScorers) {
if (docID != s.docID()) {
throw new IllegalStateException(
"subScorer=" + s + " has docID=" + s.docID() + " != collected docID=" + docID);
}
}
}
@Override
public ScoreMode scoreMode() {
return ScoreMode.COMPLETE_NO_SCORES;
}
}
}

View File

@ -234,22 +234,6 @@ public class TestDrillSideways extends FacetTestCase {
"dim=Size path=[] value=2 childCount=2\n Small (1)\n Medium (1)\n",
concurrentResult.facets.getTopChildren(10, "Size").toString());
// Now do the same thing but use a Collector directly:
SimpleCollector collector = new SimpleCollector(ScoreMode.COMPLETE_NO_SCORES);
// Make sure our Collector _does not_ need scores to ensure IndexSearcher tries to cache:
assertFalse(collector.scoreMode().needsScores());
// If we incorrectly cache here, the "sideways" FacetsCollectors will get populated with counts
// for the deleted
// docs. Make sure they don't:
DrillSidewaysResult result = ds.search(ddq, collector);
assertEquals(2, collector.hits.size());
assertEquals(
"dim=Color path=[] value=4 childCount=3\n Blue (2)\n Red (1)\n Green (1)\n",
result.facets.getTopChildren(10, "Color").toString());
assertEquals(
"dim=Size path=[] value=2 childCount=2\n Small (1)\n Medium (1)\n",
result.facets.getTopChildren(10, "Size").toString());
writer.close();
IOUtils.close(searcher.getIndexReader(), taxoReader, taxoWriter, dir, taxoDir);
}
@ -1230,25 +1214,10 @@ public class TestDrillSideways extends FacetTestCase {
getNewDrillSideways(s, config, tr)
.search(
ddq,
new org.apache.lucene.search.SimpleCollector() {
int lastDocID;
@Override
public void collect(int doc) {
assert doc > lastDocID;
lastDocID = doc;
}
@Override
protected void doSetNextReader(LeafReaderContext context) throws IOException {
lastDocID = -1;
}
@Override
public ScoreMode scoreMode() {
return ScoreMode.COMPLETE_NO_SCORES;
}
});
new SimpleCollectorManager(
numDocs,
Comparator.comparing(cr -> cr.docAndScore.doc),
ScoreMode.COMPLETE_NO_SCORES));
// Also separately verify that DS respects the
// scoreSubDocsAtOnce method, to ensure that all
@ -1259,7 +1228,7 @@ public class TestDrillSideways extends FacetTestCase {
// easily possible for one of the DD terms to be on
// a future docID:
getNewDrillSidewaysScoreSubdocsAtOnce(s, config, tr)
.search(ddq, new AssertingSubDocsAtOnceCollector());
.search(ddq, new AssertingSubDocsAtOnceCollectorManager());
}
Sort sort = new Sort(new SortField("id", SortField.Type.STRING));
@ -1970,16 +1939,8 @@ public class TestDrillSideways extends FacetTestCase {
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);
}
DrillSidewaysResult r = ds.search(ddq, manager);
// compute Facets using exposed FacetCollectors from DrillSidewaysResult
Map<String, Facets> drillSidewaysFacets = new HashMap<>();