Remove CollectorOwner class (#13702)

This commit is contained in:
Egor Potemkin 2024-09-04 01:34:50 +01:00 committed by GitHub
parent 10420c6c47
commit c21bc5405b
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
9 changed files with 141 additions and 314 deletions

View File

@ -286,10 +286,7 @@ API Changes
* GITHUB#13568: Add DoubleValuesSource#toSortableLongDoubleValuesSource and * GITHUB#13568: Add DoubleValuesSource#toSortableLongDoubleValuesSource and
MultiDoubleValuesSource#toSortableMultiLongValuesSource methods. (Shradha Shankar) MultiDoubleValuesSource#toSortableMultiLongValuesSource methods. (Shradha Shankar)
* GITHUB#13568: Add CollectorOwner class that wraps CollectorManager, and handles list of Collectors and results. * GITHUB#13568: Add DrillSideways#search method that supports any CollectorManagers for drill-sideways dimensions
Add IndexSearcher#search method that takes CollectorOwner. (Egor Potemkin)
* GITHUB#13568: Add DrillSideways#search method that supports any collector types for any drill-sideways dimensions
or drill-down. (Egor Potemkin) or drill-down. (Egor Potemkin)
New Features New Features

View File

@ -1,78 +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.search;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
/**
* This class wraps {@link CollectorManager} and owns the collectors the manager creates. It is
* convenient that clients of the class don't have to worry about keeping the list of collectors, as
* well as about making the collector's type (C) compatible when reduce is called. Instances of this
* class cache results of {@link CollectorManager#reduce(Collection)}.
*
* <p>Note that instance of this class ignores any {@link Collector} created by {@link
* CollectorManager#newCollector()} directly, not through {@link #newCollector()}
*
* @lucene.experimental
*/
public final class CollectorOwner<C extends Collector, T> {
private final CollectorManager<C, T> manager;
private T result;
private boolean reduced;
// TODO: For IndexSearcher, the list doesn't have to be synchronized
// because we create new collectors sequentially. Drill sideways creates new collectors in
// DrillSidewaysQuery#Weight#bulkScorer which is already called concurrently.
// I think making the list synchronized here is not a huge concern, at the same time, do we want
// to do something about it?
// e.g. have boolean property in constructor that makes it threads friendly when set?
private final List<C> collectors = Collections.synchronizedList(new ArrayList<>());
public CollectorOwner(CollectorManager<C, T> manager) {
this.manager = manager;
}
/** Return a new {@link Collector}. This must return a different instance on each call. */
public C newCollector() throws IOException {
C collector = manager.newCollector();
collectors.add(collector);
return collector;
}
public C getCollector(int i) {
return collectors.get(i);
}
/**
* Returns result of {@link CollectorManager#reduce(Collection)}. The result is cached.
*
* <p>This method is NOT threadsafe.
*/
public T getResult() throws IOException {
if (reduced == false) {
result = manager.reduce(collectors);
reduced = true;
}
return result;
}
}

View File

@ -671,53 +671,6 @@ public class IndexSearcher {
} }
} }
/**
* Lower-level search API. Search all leaves using the given {@link CollectorOwner}, without
* calling {@link CollectorOwner#getResult()} so that clients can reduce and read results
* themselves.
*
* <p>Note that this method doesn't return anything - users can access results by calling {@link
* CollectorOwner#getResult()}
*
* @lucene.experimental
*/
public <C extends Collector> void search(Query query, CollectorOwner<C, ?> collectorOwner)
throws IOException {
final C firstCollector = collectorOwner.newCollector();
query = rewrite(query, firstCollector.scoreMode().needsScores());
final Weight weight = createWeight(query, firstCollector.scoreMode(), 1);
search(weight, collectorOwner, firstCollector);
}
private <C extends Collector> void search(
Weight weight, CollectorOwner<C, ?> collectorOwner, C firstCollector) throws IOException {
final LeafSlice[] leafSlices = getSlices();
if (leafSlices.length == 0) {
// there are no segments, nothing to offload to the executor
assert leafContexts.isEmpty();
} else {
final ScoreMode scoreMode = firstCollector.scoreMode();
for (int i = 1; i < leafSlices.length; ++i) {
final C collector = collectorOwner.newCollector();
if (scoreMode != collector.scoreMode()) {
throw new IllegalStateException(
"CollectorManager does not always produce collectors with the same score mode");
}
}
final List<Callable<C>> listTasks = new ArrayList<>(leafSlices.length);
for (int i = 0; i < leafSlices.length; ++i) {
final LeafReaderContext[] leaves = leafSlices[i].leaves;
final C collector = collectorOwner.getCollector(i);
listTasks.add(
() -> {
search(Arrays.asList(leaves), weight, collector);
return collector;
});
}
taskExecutor.invokeAll(listTasks);
}
}
/** /**
* Lower-level search API. * Lower-level search API.
* *

View File

@ -21,7 +21,6 @@ import static org.apache.lucene.sandbox.facet.ComparableUtils.byAggregatedValue;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.List; import java.util.List;
import org.apache.lucene.analysis.core.WhitespaceAnalyzer; import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
import org.apache.lucene.document.Document; import org.apache.lucene.document.Document;
@ -58,7 +57,7 @@ import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder;
import org.apache.lucene.sandbox.facet.recorders.LongAggregationsFacetRecorder; import org.apache.lucene.sandbox.facet.recorders.LongAggregationsFacetRecorder;
import org.apache.lucene.sandbox.facet.recorders.MultiFacetsRecorder; import org.apache.lucene.sandbox.facet.recorders.MultiFacetsRecorder;
import org.apache.lucene.sandbox.facet.recorders.Reducer; import org.apache.lucene.sandbox.facet.recorders.Reducer;
import org.apache.lucene.search.CollectorOwner; import org.apache.lucene.search.CollectorManager;
import org.apache.lucene.search.DoubleValuesSource; import org.apache.lucene.search.DoubleValuesSource;
import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.LongValuesSource; import org.apache.lucene.search.LongValuesSource;
@ -564,17 +563,13 @@ public class SandboxFacetsExample {
// FacetFieldCollectorManager anyway, and leaf cutter are not merged or anything like that. // FacetFieldCollectorManager anyway, and leaf cutter are not merged or anything like that.
FacetFieldCollectorManager<CountFacetRecorder> publishDayDimensionCollectorManager = FacetFieldCollectorManager<CountFacetRecorder> publishDayDimensionCollectorManager =
new FacetFieldCollectorManager<>(defaultTaxoCutter, publishDayDimensionRecorder); new FacetFieldCollectorManager<>(defaultTaxoCutter, publishDayDimensionRecorder);
List<CollectorOwner<FacetFieldCollector, CountFacetRecorder>> drillSidewaysOwners = List<CollectorManager<FacetFieldCollector, CountFacetRecorder>> drillSidewaysManagers =
List.of(new CollectorOwner<>(publishDayDimensionCollectorManager)); List.of(publishDayDimensionCollectorManager);
//// (3) search //// (3) search
// Right now we return the same Recorder we created - so we can ignore results // Right now we return the same Recorder we created - so we can ignore results
DrillSideways ds = new DrillSideways(searcher, config, taxoReader); DrillSideways ds = new DrillSideways(searcher, config, taxoReader);
// We must wrap list of drill sideways owner with unmodifiableList to make generics work. ds.search(q, drillDownCollectorManager, drillSidewaysManagers);
ds.search(
q,
new CollectorOwner<>(drillDownCollectorManager),
Collections.unmodifiableList(drillSidewaysOwners));
//// (4) Get top 10 results by count for Author //// (4) Get top 10 results by count for Author
List<FacetResult> facetResults = new ArrayList<>(2); List<FacetResult> facetResults = new ArrayList<>(2);

View File

@ -18,7 +18,6 @@ package org.apache.lucene.facet;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -31,8 +30,8 @@ import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetField;
import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState; import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState;
import org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts; import org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts;
import org.apache.lucene.facet.taxonomy.TaxonomyReader; import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.CollectorManager; import org.apache.lucene.search.CollectorManager;
import org.apache.lucene.search.CollectorOwner;
import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MatchAllDocsQuery;
@ -302,25 +301,13 @@ public class DrillSideways {
} }
} }
private static class CallableCollector implements Callable<Void> { private record CallableCollector<R>(
private final IndexSearcher searcher; IndexSearcher searcher, Query query, CollectorManager<?, R> collectorManager)
private final Query query; implements Callable<R> {
private final CollectorOwner<?, ?> collectorOwner;
private CallableCollector(
IndexSearcher searcher, Query query, CollectorOwner<?, ?> collectorOwner) {
this.searcher = searcher;
this.query = query;
this.collectorOwner = collectorOwner;
}
@Override @Override
public Void call() throws Exception { public R call() throws Exception {
searcher.search(query, collectorOwner); return searcher.search(query, collectorManager);
// Call getResult to trigger reduce, we don't need to return results because users can access
// them directly from collectorOwner
collectorOwner.getResult();
return null;
} }
} }
@ -344,31 +331,30 @@ public class DrillSideways {
// Main query // Main query
FacetsCollectorManager drillDownFacetsCollectorManager = FacetsCollectorManager drillDownFacetsCollectorManager =
createDrillDownFacetsCollectorManager(); createDrillDownFacetsCollectorManager();
final CollectorOwner<?, ?> mainCollectorOwner; final CollectorManager<?, ?> mainCollectorManager;
if (drillDownFacetsCollectorManager != null) { if (drillDownFacetsCollectorManager != null) {
// Make sure we populate a facet collector corresponding to the base query if desired: // Make sure we populate a facet collector corresponding to the base query if desired:
mainCollectorOwner = mainCollectorManager =
new CollectorOwner<>( new MultiCollectorManager(drillDownFacetsCollectorManager, hitCollectorManager);
new MultiCollectorManager(drillDownFacetsCollectorManager, hitCollectorManager));
} else { } else {
mainCollectorOwner = new CollectorOwner<>(hitCollectorManager); mainCollectorManager = hitCollectorManager;
} }
// Drill sideways dimensions // Drill sideways dimensions
final List<CollectorOwner<?, ?>> drillSidewaysCollectorOwners; final List<CollectorManager<FacetsCollector, FacetsCollector>> drillSidewaysCollectorManagers;
if (query.getDims().isEmpty() == false) { if (query.getDims().isEmpty() == false) {
drillSidewaysCollectorOwners = new ArrayList<>(query.getDims().size()); drillSidewaysCollectorManagers = new ArrayList<>(query.getDims().size());
for (int i = 0; i < query.getDims().size(); i++) { for (int i = 0; i < query.getDims().size(); i++) {
drillSidewaysCollectorOwners.add( drillSidewaysCollectorManagers.add(createDrillSidewaysFacetsCollectorManager());
new CollectorOwner<>(createDrillSidewaysFacetsCollectorManager()));
} }
} else { } else {
drillSidewaysCollectorOwners = null; drillSidewaysCollectorManagers = null;
} }
// Execute query // Execute query
final Result<?, FacetsCollector> result;
if (executor != null) { if (executor != null) {
searchConcurrently(query, mainCollectorOwner, drillSidewaysCollectorOwners); result = searchConcurrently(query, mainCollectorManager, drillSidewaysCollectorManagers);
} else { } else {
searchSequentially(query, mainCollectorOwner, drillSidewaysCollectorOwners); result = searchSequentially(query, mainCollectorManager, drillSidewaysCollectorManagers);
} }
// Collect results // Collect results
@ -377,12 +363,12 @@ public class DrillSideways {
if (drillDownFacetsCollectorManager != null) { if (drillDownFacetsCollectorManager != null) {
// drill down collected using MultiCollector // drill down collected using MultiCollector
// Extract the results: // Extract the results:
Object[] drillDownResult = (Object[]) mainCollectorOwner.getResult(); Object[] drillDownResult = (Object[]) result.drillDownResult;
facetsCollectorResult = (FacetsCollector) drillDownResult[0]; facetsCollectorResult = (FacetsCollector) drillDownResult[0];
hitCollectorResult = (R) drillDownResult[1]; hitCollectorResult = (R) drillDownResult[1];
} else { } else {
facetsCollectorResult = null; facetsCollectorResult = null;
hitCollectorResult = (R) mainCollectorOwner.getResult(); hitCollectorResult = (R) result.drillDownResult;
} }
// Getting results for drill sideways dimensions (if any) // Getting results for drill sideways dimensions (if any)
@ -391,12 +377,11 @@ public class DrillSideways {
if (query.getDims().isEmpty() == false) { if (query.getDims().isEmpty() == false) {
drillSidewaysDims = query.getDims().keySet().toArray(new String[0]); drillSidewaysDims = query.getDims().keySet().toArray(new String[0]);
int numDims = query.getDims().size(); int numDims = query.getDims().size();
assert drillSidewaysCollectorOwners != null; assert drillSidewaysCollectorManagers != null;
assert drillSidewaysCollectorOwners.size() == numDims; assert drillSidewaysCollectorManagers.size() == numDims;
drillSidewaysCollectors = new FacetsCollector[numDims]; drillSidewaysCollectors = new FacetsCollector[numDims];
for (int dim = 0; dim < numDims; dim++) { for (int dim = 0; dim < numDims; dim++) {
drillSidewaysCollectors[dim] = drillSidewaysCollectors[dim] = result.drillSidewaysResults.get(dim);
(FacetsCollector) drillSidewaysCollectorOwners.get(dim).getResult();
} }
} else { } else {
drillSidewaysDims = null; drillSidewaysDims = null;
@ -414,52 +399,51 @@ public class DrillSideways {
/** /**
* Search using DrillDownQuery with custom collectors. This method can be used with any {@link * Search using DrillDownQuery with custom collectors. This method can be used with any {@link
* CollectorOwner}s. It doesn't return anything because it is expected that you read results from * CollectorManager}s.
* provided {@link CollectorOwner}s.
* *
* <p>To read the results, run {@link CollectorOwner#getResult()} for drill down and all drill * <p>Note: Use {@link MultiCollectorManager} to collect both hits and facets for the entire query
* sideways dimensions. * and/or for drill-sideways dimensions. You can also use it to wrap different types of {@link
* * CollectorManager} for drill-sideways dimensions.
* <p>Note: use {@link Collections#unmodifiableList(List)} to wrap {@code
* drillSidewaysCollectorOwners} to convince compiler that it is safe to use List here.
*
* <p>Use {@link MultiCollectorManager} wrapped by {@link CollectorOwner} to collect both hits and
* facets for the entire query and/or for drill-sideways dimensions.
*
* <p>TODO: Class CollectorOwner was created so that we can ignore CollectorManager type C,
* because we want each dimensions to be able to use their own types. Alternatively, we can use
* typesafe heterogeneous container and provide CollectorManager type for each dimension to this
* method? I do like CollectorOwner approach as it seems more intuitive?
*/ */
public void search( public <C extends Collector, T, K extends Collector, R> Result<T, R> search(
final DrillDownQuery query, DrillDownQuery query,
CollectorOwner<?, ?> drillDownCollectorOwner, CollectorManager<C, T> drillDownCollectorManager,
List<CollectorOwner<?, ?>> drillSidewaysCollectorOwners) List<CollectorManager<K, R>> drillSidewaysCollectorManagers)
throws IOException { throws IOException {
if (drillDownCollectorOwner == null) { if (drillDownCollectorManager == null) {
throw new IllegalArgumentException( throw new IllegalArgumentException(
"This search method requires client to provide drill down collector manager"); "This search method requires client to provide drill down collector manager");
} }
if (drillSidewaysCollectorOwners == null) { if (drillSidewaysCollectorManagers == null) {
if (query.getDims().isEmpty() == false) { if (query.getDims().isEmpty() == false) {
throw new IllegalArgumentException( throw new IllegalArgumentException(
"The query requires not null drillSidewaysCollectorOwners"); "The query requires not null drillSidewaysCollectorManagers");
} }
} else if (drillSidewaysCollectorOwners.size() != query.getDims().size()) { } else if (drillSidewaysCollectorManagers.size() != query.getDims().size()) {
throw new IllegalArgumentException( throw new IllegalArgumentException(
"drillSidewaysCollectorOwners size must be equal to number of dimensions in the query."); "drillSidewaysCollectorManagers size must be equal to number of dimensions in the query.");
} }
if (executor != null) { if (executor != null) {
searchConcurrently(query, drillDownCollectorOwner, drillSidewaysCollectorOwners); return searchConcurrently(query, drillDownCollectorManager, drillSidewaysCollectorManagers);
} else { } else {
searchSequentially(query, drillDownCollectorOwner, drillSidewaysCollectorOwners); return searchSequentially(query, drillDownCollectorManager, drillSidewaysCollectorManagers);
} }
} }
private void searchSequentially( /**
* {@link #search(DrillDownQuery, CollectorManager, List)} result. It doesn't depend on {@link
* Facets} to allow users to use any type of {@link CollectorManager} for drill-down or
* drill-sideways dimension.
*
* @param drillDownResult result from drill down (main) {@link CollectorManager}
* @param drillSidewaysResults results from drill sideways {@link CollectorManager}s
*/
public record Result<T, R>(T drillDownResult, List<R> drillSidewaysResults) {}
private <C extends Collector, T, K extends Collector, R> Result<T, R> searchSequentially(
final DrillDownQuery query, final DrillDownQuery query,
final CollectorOwner<?, ?> drillDownCollectorOwner, final CollectorManager<C, T> drillDownCollectorManager,
final List<CollectorOwner<?, ?>> drillSidewaysCollectorOwners) final List<CollectorManager<K, R>> drillSidewaysCollectorManagers)
throws IOException { throws IOException {
Map<String, Integer> drillDownDims = query.getDims(); Map<String, Integer> drillDownDims = query.getDims();
@ -467,9 +451,7 @@ public class DrillSideways {
if (drillDownDims.isEmpty()) { if (drillDownDims.isEmpty()) {
// There are no drill-down dims, so there is no // There are no drill-down dims, so there is no
// drill-sideways to compute: // drill-sideways to compute:
searcher.search(query, drillDownCollectorOwner); return new Result<>(searcher.search(query, drillDownCollectorManager), null);
drillDownCollectorOwner.getResult();
return;
} }
Query baseQuery = query.getBaseQuery(); Query baseQuery = query.getBaseQuery();
@ -480,59 +462,60 @@ public class DrillSideways {
} }
Query[] drillDownQueries = query.getDrillDownQueries(); Query[] drillDownQueries = query.getDrillDownQueries();
DrillSidewaysQuery dsq = DrillSidewaysQuery<K, R> dsq =
new DrillSidewaysQuery( new DrillSidewaysQuery<>(
baseQuery, baseQuery, drillSidewaysCollectorManagers, drillDownQueries, scoreSubDocsAtOnce());
// drillDownCollectorOwner,
// Don't pass drill down collector because drill down is collected by IndexSearcher
// itself.
// TODO: deprecate drillDown collection in DrillSidewaysQuery?
null,
drillSidewaysCollectorOwners,
drillDownQueries,
scoreSubDocsAtOnce());
searcher.search(dsq, drillDownCollectorOwner); T collectorResult = searcher.search(dsq, drillDownCollectorManager);
// This method doesn't return results as each dimension might have its own result type. List<R> drillSidewaysResults = new ArrayList<>(drillDownDims.size());
// But we call getResult to trigger results reducing, so that users don't have to worry about assert drillSidewaysCollectorManagers != null
// it. : "Case without drill sideways dimensions is handled above";
drillDownCollectorOwner.getResult(); int numSlices = dsq.managedDrillSidewaysCollectors.size();
if (drillSidewaysCollectorOwners != null) { for (int dim = 0; dim < drillDownDims.size(); dim++) {
for (CollectorOwner<?, ?> sidewaysOwner : drillSidewaysCollectorOwners) { List<K> collectorsForDim = new ArrayList<>(numSlices);
sidewaysOwner.getResult(); for (int slice = 0; slice < numSlices; slice++) {
collectorsForDim.add(dsq.managedDrillSidewaysCollectors.get(slice).get(dim));
} }
drillSidewaysResults.add(
dim, drillSidewaysCollectorManagers.get(dim).reduce(collectorsForDim));
} }
return new Result<>(collectorResult, drillSidewaysResults);
} }
private void searchConcurrently( private <C extends Collector, T, K extends Collector, R> Result<T, R> searchConcurrently(
final DrillDownQuery query, final DrillDownQuery query,
final CollectorOwner<?, ?> drillDownCollectorOwner, final CollectorManager<C, T> drillDownCollectorManager,
final List<CollectorOwner<?, ?>> drillSidewaysCollectorOwners) final List<CollectorManager<K, R>> drillSidewaysCollectorManagers) {
throws IOException {
final Map<String, Integer> drillDownDims = query.getDims(); final Map<String, Integer> drillDownDims = query.getDims();
final List<CallableCollector> callableCollectors = new ArrayList<>(drillDownDims.size() + 1); final CallableCollector<T> drillDownCallableCollector =
new CallableCollector<>(searcher, query, drillDownCollectorManager);
final List<CallableCollector<R>> drillSidewaysCallableCollectors =
new ArrayList<>(drillDownDims.size());
callableCollectors.add(new CallableCollector(searcher, query, drillDownCollectorOwner));
int i = 0; int i = 0;
final Query[] filters = query.getDrillDownQueries(); final Query[] filters = query.getDrillDownQueries();
for (String dim : drillDownDims.keySet()) { for (String dim : drillDownDims.keySet()) {
callableCollectors.add( drillSidewaysCallableCollectors.add(
new CallableCollector( new CallableCollector<>(
searcher, searcher,
getDrillDownQuery(query, filters, dim), getDrillDownQuery(query, filters, dim),
drillSidewaysCollectorOwners.get(i))); drillSidewaysCollectorManagers.get(i)));
i++; i++;
} }
try { try {
// Run the query pool final Future<T> drillDownFuture = executor.submit(drillDownCallableCollector);
final List<Future<Void>> futures = executor.invokeAll(callableCollectors); final List<Future<R>> drillSidewaysFutures =
executor.invokeAll(drillSidewaysCallableCollectors);
// Wait for results. We don't read the results as they are collected by CollectorOwners T collectorResult = drillDownFuture.get();
for (i = 0; i < futures.size(); i++) { List<R> drillSidewaysResults = new ArrayList<>(drillDownDims.size());
futures.get(i).get();
for (i = 0; i < drillSidewaysFutures.size(); i++) {
drillSidewaysResults.add(i, drillSidewaysFutures.get(i).get());
} }
return new Result<>(collectorResult, drillSidewaysResults);
} catch (InterruptedException e) { } catch (InterruptedException e) {
throw new ThreadInterruptedException(e); throw new ThreadInterruptedException(e);
} catch (ExecutionException e) { } catch (ExecutionException e) {

View File

@ -17,19 +17,20 @@
package org.apache.lucene.facet; package org.apache.lucene.facet;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.BulkScorer; import org.apache.lucene.search.BulkScorer;
import org.apache.lucene.search.Collector; import org.apache.lucene.search.Collector;
import org.apache.lucene.search.CollectorOwner; import org.apache.lucene.search.CollectorManager;
import org.apache.lucene.search.ConstantScoreScorer; import org.apache.lucene.search.ConstantScoreScorer;
import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Explanation; import org.apache.lucene.search.Explanation;
import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryVisitor; import org.apache.lucene.search.QueryVisitor;
import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.ScoreMode;
@ -41,12 +42,12 @@ import org.apache.lucene.search.Weight;
// TODO change the way DrillSidewaysScorer is used, this query does not work // TODO change the way DrillSidewaysScorer is used, this query does not work
// with filter caching // with filter caching
class DrillSidewaysQuery extends Query { class DrillSidewaysQuery<K extends Collector, R> extends Query {
final Query baseQuery; final Query baseQuery;
final CollectorOwner<?, ?> drillDownCollectorOwner; final List<CollectorManager<K, R>> drillSidewaysCollectorManagers;
final List<CollectorOwner<?, ?>> drillSidewaysCollectorOwners; final List<List<K>> managedDrillSidewaysCollectors;
final Query[] drillDownQueries; final Query[] drillDownQueries;
@ -58,15 +59,36 @@ class DrillSidewaysQuery extends Query {
*/ */
DrillSidewaysQuery( DrillSidewaysQuery(
Query baseQuery, Query baseQuery,
CollectorOwner<?, ?> drillDownCollectorOwner, List<CollectorManager<K, R>> drillSidewaysCollectorManagers,
List<CollectorOwner<?, ?>> drillSidewaysCollectorOwners, Query[] drillDownQueries,
boolean scoreSubDocsAtOnce) {
// Note that the "managed" collector lists are synchronized here since bulkScorer()
// can be invoked concurrently and needs to remain thread-safe. We're OK with synchronizing
// on the whole list as contention is expected to remain very low:
this(
baseQuery,
drillSidewaysCollectorManagers,
Collections.synchronizedList(new ArrayList<>()),
drillDownQueries,
scoreSubDocsAtOnce);
}
/**
* Needed for {@link Query#rewrite(IndexSearcher)}. Ensures the same "managed" lists get used
* since {@link DrillSideways} accesses references to these through the original {@code
* DrillSidewaysQuery}.
*/
private DrillSidewaysQuery(
Query baseQuery,
List<CollectorManager<K, R>> drillSidewaysCollectorManagers,
List<List<K>> managedDrillSidewaysCollectors,
Query[] drillDownQueries, Query[] drillDownQueries,
boolean scoreSubDocsAtOnce) { boolean scoreSubDocsAtOnce) {
this.baseQuery = Objects.requireNonNull(baseQuery); this.baseQuery = Objects.requireNonNull(baseQuery);
this.drillDownCollectorOwner = drillDownCollectorOwner; this.drillSidewaysCollectorManagers = drillSidewaysCollectorManagers;
this.drillSidewaysCollectorOwners = drillSidewaysCollectorOwners;
this.drillDownQueries = drillDownQueries; this.drillDownQueries = drillDownQueries;
this.scoreSubDocsAtOnce = scoreSubDocsAtOnce; this.scoreSubDocsAtOnce = scoreSubDocsAtOnce;
this.managedDrillSidewaysCollectors = managedDrillSidewaysCollectors;
} }
@Override @Override
@ -87,10 +109,10 @@ class DrillSidewaysQuery extends Query {
if (newQuery == baseQuery) { if (newQuery == baseQuery) {
return super.rewrite(indexSearcher); return super.rewrite(indexSearcher);
} else { } else {
return new DrillSidewaysQuery( return new DrillSidewaysQuery<>(
newQuery, newQuery,
drillDownCollectorOwner, drillSidewaysCollectorManagers,
drillSidewaysCollectorOwners, managedDrillSidewaysCollectors,
drillDownQueries, drillDownQueries,
scoreSubDocsAtOnce); scoreSubDocsAtOnce);
} }
@ -124,14 +146,8 @@ class DrillSidewaysQuery extends Query {
int drillDownCount = drillDowns.length; int drillDownCount = drillDowns.length;
Collector drillDownCollector; List<K> sidewaysCollectors = new ArrayList<>(drillDownCount);
final LeafCollector drillDownLeafCollector; managedDrillSidewaysCollectors.add(sidewaysCollectors);
if (drillDownCollectorOwner != null) {
drillDownCollector = drillDownCollectorOwner.newCollector();
drillDownLeafCollector = drillDownCollector.getLeafCollector(context);
} else {
drillDownLeafCollector = null;
}
DrillSidewaysScorer.DocsAndCost[] dims = DrillSidewaysScorer.DocsAndCost[] dims =
new DrillSidewaysScorer.DocsAndCost[drillDownCount]; new DrillSidewaysScorer.DocsAndCost[drillDownCount];
@ -144,7 +160,8 @@ class DrillSidewaysQuery extends Query {
scorer = new ConstantScoreScorer(0f, scoreMode, DocIdSetIterator.empty()); scorer = new ConstantScoreScorer(0f, scoreMode, DocIdSetIterator.empty());
} }
Collector sidewaysCollector = drillSidewaysCollectorOwners.get(dim).newCollector(); K sidewaysCollector = drillSidewaysCollectorManagers.get(dim).newCollector();
sidewaysCollectors.add(dim, sidewaysCollector);
dims[dim] = dims[dim] =
new DrillSidewaysScorer.DocsAndCost( new DrillSidewaysScorer.DocsAndCost(
@ -155,9 +172,6 @@ class DrillSidewaysQuery extends Query {
// a null scorer in this case, but we need to make sure #finish gets called on all facet // a null scorer in this case, but we need to make sure #finish gets called on all facet
// collectors since IndexSearcher won't handle this for us: // collectors since IndexSearcher won't handle this for us:
if (baseScorerSupplier == null || nullCount > 1) { if (baseScorerSupplier == null || nullCount > 1) {
if (drillDownLeafCollector != null) {
drillDownLeafCollector.finish();
}
for (DrillSidewaysScorer.DocsAndCost dim : dims) { for (DrillSidewaysScorer.DocsAndCost dim : dims) {
dim.sidewaysLeafCollector.finish(); dim.sidewaysLeafCollector.finish();
} }
@ -177,11 +191,7 @@ class DrillSidewaysQuery extends Query {
@Override @Override
public BulkScorer bulkScorer() throws IOException { public BulkScorer bulkScorer() throws IOException {
return new DrillSidewaysScorer( return new DrillSidewaysScorer(
context, context, baseScorerSupplier.get(Long.MAX_VALUE), dims, scoreSubDocsAtOnce);
baseScorerSupplier.get(Long.MAX_VALUE),
drillDownLeafCollector,
dims,
scoreSubDocsAtOnce);
} }
@Override @Override
@ -212,9 +222,8 @@ class DrillSidewaysQuery extends Query {
final int prime = 31; final int prime = 31;
int result = classHash(); int result = classHash();
result = prime * result + Objects.hashCode(baseQuery); result = prime * result + Objects.hashCode(baseQuery);
result = prime * result + Objects.hashCode(drillDownCollectorOwner);
result = prime * result + Arrays.hashCode(drillDownQueries); result = prime * result + Arrays.hashCode(drillDownQueries);
result = prime * result + Objects.hashCode(drillSidewaysCollectorOwners); result = prime * result + Objects.hashCode(drillSidewaysCollectorManagers);
return result; return result;
} }
@ -223,10 +232,9 @@ class DrillSidewaysQuery extends Query {
return sameClassAs(other) && equalsTo(getClass().cast(other)); return sameClassAs(other) && equalsTo(getClass().cast(other));
} }
private boolean equalsTo(DrillSidewaysQuery other) { private boolean equalsTo(DrillSidewaysQuery<?, ?> other) {
return Objects.equals(baseQuery, other.baseQuery) return Objects.equals(baseQuery, other.baseQuery)
&& Objects.equals(drillDownCollectorOwner, other.drillDownCollectorOwner)
&& Arrays.equals(drillDownQueries, other.drillDownQueries) && Arrays.equals(drillDownQueries, other.drillDownQueries)
&& Objects.equals(drillSidewaysCollectorOwners, other.drillSidewaysCollectorOwners); && Objects.equals(drillSidewaysCollectorManagers, other.drillSidewaysCollectorManagers);
} }
} }

View File

@ -45,8 +45,6 @@ class DrillSidewaysScorer extends BulkScorer {
// private static boolean DEBUG = false; // private static boolean DEBUG = false;
private final LeafCollector drillDownLeafCollector;
private final DocsAndCost[] dims; private final DocsAndCost[] dims;
// DrillDown DocsEnums: // DrillDown DocsEnums:
@ -68,7 +66,6 @@ class DrillSidewaysScorer extends BulkScorer {
DrillSidewaysScorer( DrillSidewaysScorer(
LeafReaderContext context, LeafReaderContext context,
Scorer baseScorer, Scorer baseScorer,
LeafCollector drillDownLeafCollector,
DocsAndCost[] dims, DocsAndCost[] dims,
boolean scoreSubDocsAtOnce) { boolean scoreSubDocsAtOnce) {
this.dims = dims; this.dims = dims;
@ -81,7 +78,6 @@ class DrillSidewaysScorer extends BulkScorer {
} else { } else {
this.baseApproximation = baseIterator; this.baseApproximation = baseIterator;
} }
this.drillDownLeafCollector = drillDownLeafCollector;
this.scoreSubDocsAtOnce = scoreSubDocsAtOnce; this.scoreSubDocsAtOnce = scoreSubDocsAtOnce;
} }
@ -709,9 +705,6 @@ class DrillSidewaysScorer extends BulkScorer {
// } // }
collector.collect(collectDocID); collector.collect(collectDocID);
if (drillDownLeafCollector != null) {
drillDownLeafCollector.collect(collectDocID);
}
// TODO: we could "fix" faceting of the sideways counts // TODO: we could "fix" faceting of the sideways counts
// to do this "union" (of the drill down hits) in the // to do this "union" (of the drill down hits) in the
@ -725,9 +718,6 @@ class DrillSidewaysScorer extends BulkScorer {
private void collectHit(LeafCollector collector, DocsAndCost dim) throws IOException { private void collectHit(LeafCollector collector, DocsAndCost dim) throws IOException {
collector.collect(collectDocID); collector.collect(collectDocID);
if (drillDownLeafCollector != null) {
drillDownLeafCollector.collect(collectDocID);
}
// Tally sideways count: // Tally sideways count:
dim.sidewaysLeafCollector.collect(collectDocID); dim.sidewaysLeafCollector.collect(collectDocID);
@ -735,9 +725,6 @@ class DrillSidewaysScorer extends BulkScorer {
private void collectHit(LeafCollector collector, List<DocsAndCost> dims) throws IOException { private void collectHit(LeafCollector collector, List<DocsAndCost> dims) throws IOException {
collector.collect(collectDocID); collector.collect(collectDocID);
if (drillDownLeafCollector != null) {
drillDownLeafCollector.collect(collectDocID);
}
// Tally sideways counts: // Tally sideways counts:
for (DocsAndCost dim : dims) { for (DocsAndCost dim : dims) {
@ -756,9 +743,6 @@ class DrillSidewaysScorer extends BulkScorer {
// Note: We _only_ call #finish on the facets collectors we're managing here, but not the // Note: We _only_ call #finish on the facets collectors we're managing here, but not the
// "main" collector. This is because IndexSearcher handles calling #finish on the main // "main" collector. This is because IndexSearcher handles calling #finish on the main
// collector. // collector.
if (drillDownLeafCollector != null) {
drillDownLeafCollector.finish();
}
for (DocsAndCost dim : dims) { for (DocsAndCost dim : dims) {
dim.sidewaysLeafCollector.finish(); dim.sidewaysLeafCollector.finish();
} }
@ -766,9 +750,6 @@ class DrillSidewaysScorer extends BulkScorer {
private void setScorer(LeafCollector mainCollector, Scorable scorer) throws IOException { private void setScorer(LeafCollector mainCollector, Scorable scorer) throws IOException {
mainCollector.setScorer(scorer); mainCollector.setScorer(scorer);
if (drillDownLeafCollector != null) {
drillDownLeafCollector.setScorer(scorer);
}
for (DocsAndCost dim : dims) { for (DocsAndCost dim : dims) {
dim.sidewaysLeafCollector.setScorer(scorer); dim.sidewaysLeafCollector.setScorer(scorer);
} }

View File

@ -284,7 +284,6 @@ public class TestDrillSideways extends FacetTestCase {
Weight dimWeight = searcher.createWeight(dimQ, ScoreMode.COMPLETE_NO_SCORES, 1f); Weight dimWeight = searcher.createWeight(dimQ, ScoreMode.COMPLETE_NO_SCORES, 1f);
Scorer dimScorer = dimWeight.scorer(ctx); Scorer dimScorer = dimWeight.scorer(ctx);
FacetsCollector baseFC = new FacetsCollector();
FacetsCollector dimFC = new FacetsCollector(); FacetsCollector dimFC = new FacetsCollector();
DrillSidewaysScorer.DocsAndCost docsAndCost = DrillSidewaysScorer.DocsAndCost docsAndCost =
new DrillSidewaysScorer.DocsAndCost(dimScorer, dimFC.getLeafCollector(ctx)); new DrillSidewaysScorer.DocsAndCost(dimScorer, dimFC.getLeafCollector(ctx));
@ -311,7 +310,6 @@ public class TestDrillSideways extends FacetTestCase {
new DrillSidewaysScorer( new DrillSidewaysScorer(
ctx, ctx,
baseScorer, baseScorer,
baseFC.getLeafCollector(ctx),
new DrillSidewaysScorer.DocsAndCost[] {docsAndCost}, new DrillSidewaysScorer.DocsAndCost[] {docsAndCost},
scoreSubDocsAtOnce); scoreSubDocsAtOnce);
expectThrows(CollectionTerminatedException.class, () -> scorer.score(baseCollector, null)); expectThrows(CollectionTerminatedException.class, () -> scorer.score(baseCollector, null));
@ -321,7 +319,6 @@ public class TestDrillSideways extends FacetTestCase {
// both our base and sideways dim facets collectors. What we really want to test here is // both our base and sideways dim facets collectors. What we really want to test here is
// that the matching docs are still correctly present and populated after an early // that the matching docs are still correctly present and populated after an early
// termination occurs (i.e., #finish is properly called in that scenario): // termination occurs (i.e., #finish is properly called in that scenario):
assertEquals(1, baseFC.getMatchingDocs().size());
assertEquals(1, dimFC.getMatchingDocs().size()); assertEquals(1, dimFC.getMatchingDocs().size());
} }
} }

View File

@ -50,7 +50,6 @@ import org.apache.lucene.sandbox.facet.cutters.ranges.LongRangeFacetCutter;
import org.apache.lucene.sandbox.facet.labels.OrdToLabel; import org.apache.lucene.sandbox.facet.labels.OrdToLabel;
import org.apache.lucene.sandbox.facet.labels.RangeOrdToLabel; import org.apache.lucene.sandbox.facet.labels.RangeOrdToLabel;
import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder; import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder;
import org.apache.lucene.search.CollectorOwner;
import org.apache.lucene.search.DoubleValues; import org.apache.lucene.search.DoubleValues;
import org.apache.lucene.search.DoubleValuesSource; import org.apache.lucene.search.DoubleValuesSource;
import org.apache.lucene.search.Explanation; import org.apache.lucene.search.Explanation;
@ -538,7 +537,7 @@ public class TestRangeFacet extends SandboxFacetTestCase {
////// First search, no drill-downs: ////// First search, no drill-downs:
DrillDownQuery ddq = new DrillDownQuery(config); DrillDownQuery ddq = new DrillDownQuery(config);
ds.search(ddq, new CollectorOwner<>(collectorManager), List.of()); ds.search(ddq, collectorManager, List.of());
// assertEquals(100, dsr.hits.totalHits.value); // assertEquals(100, dsr.hits.totalHits.value);
assertEquals( assertEquals(
@ -556,10 +555,7 @@ public class TestRangeFacet extends SandboxFacetTestCase {
dimCollectorManager = new FacetFieldCollectorManager<>(dimCutter, dimCountRecorder); dimCollectorManager = new FacetFieldCollectorManager<>(dimCutter, dimCountRecorder);
ddq = new DrillDownQuery(config); ddq = new DrillDownQuery(config);
ddq.add("dim", "b"); ddq.add("dim", "b");
ds.search( ds.search(ddq, fieldCollectorManager, List.of(dimCollectorManager));
ddq,
new CollectorOwner<>(fieldCollectorManager),
List.of(new CollectorOwner<>(dimCollectorManager)));
// assertEquals(75, dsr.hits.totalHits.value); // assertEquals(75, dsr.hits.totalHits.value);
assertEquals( assertEquals(
@ -577,10 +573,7 @@ public class TestRangeFacet extends SandboxFacetTestCase {
dimCollectorManager = new FacetFieldCollectorManager<>(dimCutter, dimCountRecorder); dimCollectorManager = new FacetFieldCollectorManager<>(dimCutter, dimCountRecorder);
ddq = new DrillDownQuery(config); ddq = new DrillDownQuery(config);
ddq.add("field", LongPoint.newRangeQuery("field", 0L, 10L)); ddq.add("field", LongPoint.newRangeQuery("field", 0L, 10L));
ds.search( ds.search(ddq, dimCollectorManager, List.of(fieldCollectorManager));
ddq,
new CollectorOwner<>(dimCollectorManager),
List.of(new CollectorOwner<>(fieldCollectorManager)));
// assertEquals(11, dsr.hits.totalHits.value); // assertEquals(11, dsr.hits.totalHits.value);
assertEquals( assertEquals(
@ -1629,14 +1622,12 @@ public class TestRangeFacet extends SandboxFacetTestCase {
countRecorder = new CountFacetRecorder(); countRecorder = new CountFacetRecorder();
CollectorOwner<DummyTotalHitCountCollector, Integer> totalHitsCollectorOwner = DrillSideways.Result<Integer, CountFacetRecorder> result =
new CollectorOwner<>(DummyTotalHitCountCollector.createManager()); ds.search(
CollectorOwner<FacetFieldCollector, CountFacetRecorder> drillSidewaysCollectorOwner = ddq,
new CollectorOwner<>( DummyTotalHitCountCollector.createManager(),
new FacetFieldCollectorManager<>(doubleRangeFacetCutter, countRecorder)); List.of(new FacetFieldCollectorManager<>(doubleRangeFacetCutter, countRecorder)));
ds.search(ddq, totalHitsCollectorOwner, List.of(drillSidewaysCollectorOwner)); assertEquals(1, result.drillDownResult().intValue());
assertEquals(1, totalHitsCollectorOwner.getResult().intValue());
drillSidewaysCollectorOwner.getResult();
assertEquals( assertEquals(
"dim=field path=[] value=-2147483648 childCount=6\n < 1 (0)\n < 2 (1)\n < 5 (3)\n < 10 (3)\n < 20 (3)\n < 50 (3)\n", "dim=field path=[] value=-2147483648 childCount=6\n < 1 (0)\n < 2 (1)\n < 5 (3)\n < 10 (3)\n < 20 (3)\n < 50 (3)\n",
getAllSortByOrd(getRangeOrdinals(ranges), countRecorder, "field", ordToLabel).toString()); getAllSortByOrd(getRangeOrdinals(ranges), countRecorder, "field", ordToLabel).toString());