LUCENE-7588: DrillSideways can now run its queries concurrently

This commit is contained in:
Mike McCandless 2017-01-03 06:26:49 -05:00
parent 48ca9fc3f4
commit b4a002f7d8
6 changed files with 704 additions and 198 deletions

View File

@ -98,6 +98,11 @@ New features
parsers, by enumerating all paths and creating the corresponding
query/ies as sub-clauses (Matt Weber via Mike McCandless)
* LUCENE-7588: DrillSideways can now run queries concurrently, and
supports an IndexSearcher using an executor service to run each query
concurrently across all segments in the index (Emmanuel Keller via
Mike McCandless)
Bug Fixes
* LUCENE-7547: JapaneseTokenizerFactory was failing to close the

View File

@ -0,0 +1,105 @@
/*
* 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 org.apache.lucene.index.LeafReaderContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
/**
* A {@link CollectorManager} implements which wrap a set of {@link CollectorManager}
* as {@link MultiCollector} acts for {@link Collector}.
*/
public class MultiCollectorManager implements CollectorManager<MultiCollectorManager.Collectors, Object[]> {
final private CollectorManager<Collector, ?>[] collectorManagers;
public MultiCollectorManager(final CollectorManager... collectorManagers) {
this.collectorManagers = collectorManagers;
}
@Override
public Collectors newCollector() throws IOException {
return new Collectors();
}
@Override
public Object[] reduce(Collection<Collectors> reducableCollectors) throws IOException {
final int size = reducableCollectors.size();
final Object[] results = new Object[collectorManagers.length];
for (int i = 0; i < collectorManagers.length; i++) {
final List<Collector> reducableCollector = new ArrayList<>(size);
for (Collectors collectors : reducableCollectors)
reducableCollector.add(collectors.collectors[i]);
results[i] = collectorManagers[i].reduce(reducableCollector);
}
return results;
}
public class Collectors implements Collector {
private final Collector[] collectors;
private Collectors() throws IOException {
collectors = new Collector[collectorManagers.length];
for (int i = 0; i < collectors.length; i++)
collectors[i] = collectorManagers[i].newCollector();
}
@Override
final public LeafCollector getLeafCollector(final LeafReaderContext context) throws IOException {
return new LeafCollectors(context);
}
@Override
final public boolean needsScores() {
for (Collector collector : collectors)
if (collector.needsScores())
return true;
return false;
}
public class LeafCollectors implements LeafCollector {
private final LeafCollector[] leafCollectors;
private LeafCollectors(final LeafReaderContext context) throws IOException {
leafCollectors = new LeafCollector[collectors.length];
for (int i = 0; i < collectors.length; i++)
leafCollectors[i] = collectors[i].getLeafCollector(context);
}
@Override
final public void setScorer(final Scorer scorer) throws IOException {
for (LeafCollector leafCollector : leafCollectors)
if (leafCollector != null)
leafCollector.setScorer(scorer);
}
@Override
final public void collect(final int doc) throws IOException {
for (LeafCollector leafCollector : leafCollectors)
if (leafCollector != null)
leafCollector.collect(doc);
}
}
}
}

View File

@ -16,27 +16,37 @@
*/
package org.apache.lucene.facet;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts;
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.FilterCollector;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.CollectorManager;
import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.FilterCollector;
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;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.TopFieldCollector;
import org.apache.lucene.search.TopScoreDocCollector;
import org.apache.lucene.util.ThreadInterruptedException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
/**
* Computes drill down and sideways counts for the provided
@ -44,11 +54,9 @@ import org.apache.lucene.search.TopScoreDocCollector;
* alternative values/aggregates for the drill-down
* dimensions so that a dimension does not disappear after
* the user drills down into it.
*
* <p> Use one of the static search
* methods to do the search, and then get the hits and facet
* results from the returned {@link DrillSidewaysResult}.
*
* <p><b>NOTE</b>: this allocates one {@link
* FacetsCollector} for each drill-down, plus one. If your
* index has high number of facet labels then this will
@ -58,52 +66,87 @@ import org.apache.lucene.search.TopScoreDocCollector;
*/
public class DrillSideways {
/** {@link IndexSearcher} passed to constructor. */
/**
* {@link IndexSearcher} passed to constructor.
*/
protected final IndexSearcher searcher;
/** {@link TaxonomyReader} passed to constructor. */
/**
* {@link TaxonomyReader} passed to constructor.
*/
protected final TaxonomyReader taxoReader;
/** {@link SortedSetDocValuesReaderState} passed to
* constructor; can be null. */
/**
* {@link SortedSetDocValuesReaderState} passed to
* constructor; can be null.
*/
protected final SortedSetDocValuesReaderState state;
/** {@link FacetsConfig} passed to constructor. */
/**
* {@link FacetsConfig} passed to constructor.
*/
protected final FacetsConfig config;
/** Create a new {@code DrillSideways} instance. */
// These are only used for multi-threaded search
private final ExecutorService executor;
/**
* Create a new {@code DrillSideways} instance.
*/
public DrillSideways(IndexSearcher searcher, FacetsConfig config, TaxonomyReader taxoReader) {
this(searcher, config, taxoReader, null);
}
/** Create a new {@code DrillSideways} instance, assuming the categories were
* indexed with {@link SortedSetDocValuesFacetField}. */
/**
* Create a new {@code DrillSideways} instance, assuming the categories were
* indexed with {@link SortedSetDocValuesFacetField}.
*/
public DrillSideways(IndexSearcher searcher, FacetsConfig config, SortedSetDocValuesReaderState state) {
this(searcher, config, null, state);
}
/** Create a new {@code DrillSideways} instance, where some
/**
* Create a new {@code DrillSideways} instance, where some
* dimensions were indexed with {@link
* SortedSetDocValuesFacetField} and others were indexed
* with {@link FacetField}. */
public DrillSideways(IndexSearcher searcher, FacetsConfig config, TaxonomyReader taxoReader, SortedSetDocValuesReaderState state) {
* with {@link FacetField}.
*/
public DrillSideways(IndexSearcher searcher, FacetsConfig config, TaxonomyReader taxoReader,
SortedSetDocValuesReaderState state) {
this(searcher, config, taxoReader, state, null);
}
/**
* Create a new {@code DrillSideways} instance, where some
* dimensions were indexed with {@link
* SortedSetDocValuesFacetField} and others were indexed
* with {@link FacetField}.
* <p>
* Use this constructor to use the concurrent implementation and/or the CollectorManager
*/
public DrillSideways(IndexSearcher searcher, FacetsConfig config, TaxonomyReader taxoReader,
SortedSetDocValuesReaderState state, ExecutorService executor) {
this.searcher = searcher;
this.config = config;
this.taxoReader = taxoReader;
this.state = state;
this.executor = executor;
}
/** Subclass can override to customize per-dim Facets
* impl. */
protected Facets buildFacetsResult(FacetsCollector drillDowns, FacetsCollector[] drillSideways, String[] drillSidewaysDims) throws IOException {
/**
* Subclass can override to customize per-dim Facets
* impl.
*/
protected Facets buildFacetsResult(FacetsCollector drillDowns, FacetsCollector[] drillSideways,
String[] drillSidewaysDims) throws IOException {
Facets drillDownFacets;
Map<String,Facets> drillSidewaysFacets = new HashMap<>();
Map<String, Facets> drillSidewaysFacets = new HashMap<>();
if (taxoReader != null) {
drillDownFacets = new FastTaxonomyFacetCounts(taxoReader, config, drillDowns);
if (drillSideways != null) {
for(int i=0;i<drillSideways.length;i++) {
for (int i = 0; i < drillSideways.length; i++) {
drillSidewaysFacets.put(drillSidewaysDims[i],
new FastTaxonomyFacetCounts(taxoReader, config, drillSideways[i]));
}
@ -111,9 +154,8 @@ public class DrillSideways {
} else {
drillDownFacets = new SortedSetDocValuesFacetCounts(state, drillDowns);
if (drillSideways != null) {
for(int i=0;i<drillSideways.length;i++) {
drillSidewaysFacets.put(drillSidewaysDims[i],
new SortedSetDocValuesFacetCounts(state, drillSideways[i]));
for (int i = 0; i < drillSideways.length; i++) {
drillSidewaysFacets.put(drillSidewaysDims[i], new SortedSetDocValuesFacetCounts(state, drillSideways[i]));
}
}
}
@ -131,7 +173,7 @@ public class DrillSideways {
*/
public DrillSidewaysResult search(DrillDownQuery query, Collector hitCollector) throws IOException {
Map<String,Integer> drillDownDims = query.getDims();
Map<String, Integer> drillDownDims = query.getDims();
FacetsCollector drillDownCollector = new FacetsCollector();
@ -155,7 +197,9 @@ public class DrillSideways {
drillSidewaysCollectors[i] = new FacetsCollector();
}
DrillSidewaysQuery dsq = new DrillSidewaysQuery(baseQuery, drillDownCollector, drillSidewaysCollectors, drillDownQueries, scoreSubDocsAtOnce());
DrillSidewaysQuery dsq =
new DrillSidewaysQuery(baseQuery, drillDownCollector, drillSidewaysCollectors, drillDownQueries,
scoreSubDocsAtOnce());
if (hitCollector.needsScores() == false) {
// this is a horrible hack in order to make sure IndexSearcher will not
// attempt to cache the DrillSidewaysQuery
@ -168,16 +212,16 @@ public class DrillSideways {
}
searcher.search(dsq, hitCollector);
return new DrillSidewaysResult(buildFacetsResult(drillDownCollector, drillSidewaysCollectors, drillDownDims.keySet().toArray(new String[drillDownDims.size()])), null);
return new DrillSidewaysResult(buildFacetsResult(drillDownCollector, drillSidewaysCollectors,
drillDownDims.keySet().toArray(new String[drillDownDims.size()])), null);
}
/**
* 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,
boolean doMaxScore) throws IOException {
public DrillSidewaysResult search(DrillDownQuery query, Query filter, FieldDoc after, int topN, Sort sort,
boolean doDocScores, boolean doMaxScore) throws IOException {
if (filter != null) {
query = new DrillDownQuery(config, filter, query);
}
@ -186,15 +230,38 @@ public class DrillSideways {
if (limit == 0) {
limit = 1; // the collector does not alow numHits = 0
}
topN = Math.min(topN, limit);
final TopFieldCollector hitCollector = TopFieldCollector.create(sort,
topN,
after,
true,
doDocScores,
doMaxScore);
final int fTopN = Math.min(topN, limit);
if (executor != null) { // We have an executor, let use the multi-threaded version
final CollectorManager<TopFieldCollector, TopDocs> collectorManager =
new CollectorManager<TopFieldCollector, TopDocs>() {
@Override
public TopFieldCollector newCollector() throws IOException {
return TopFieldCollector.create(sort, fTopN, after, true, doDocScores, doMaxScore);
}
@Override
public TopDocs reduce(Collection<TopFieldCollector> collectors) throws IOException {
final TopDocs[] topDocs = new TopDocs[collectors.size()];
int pos = 0;
for (TopFieldCollector collector : collectors)
topDocs[pos++] = collector.topDocs();
return TopDocs.merge(topN, topDocs);
}
};
ConcurrentDrillSidewaysResult<TopDocs> r = search(query, collectorManager);
return new DrillSidewaysResult(r.facets, r.collectorResult);
} else {
final TopFieldCollector hitCollector =
TopFieldCollector.create(sort, fTopN, after, true, doDocScores, doMaxScore);
DrillSidewaysResult r = search(query, hitCollector);
return new DrillSidewaysResult(r.facets, hitCollector.topDocs());
}
} else {
return search(after, query, topN);
}
@ -212,41 +279,184 @@ public class DrillSideways {
* Search, sorting by score, and computing
* drill down and sideways counts.
*/
public DrillSidewaysResult search(ScoreDoc after,
DrillDownQuery query, int topN) throws IOException {
public DrillSidewaysResult search(ScoreDoc after, DrillDownQuery query, int topN) throws IOException {
int limit = searcher.getIndexReader().maxDoc();
if (limit == 0) {
limit = 1; // the collector does not alow numHits = 0
}
topN = Math.min(topN, limit);
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<TopScoreDocCollector, TopDocs>() {
@Override
public TopScoreDocCollector newCollector() throws IOException {
return TopScoreDocCollector.create(fTopN, after);
}
@Override
public TopDocs reduce(Collection<TopScoreDocCollector> collectors) throws IOException {
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 = search(query, collectorManager);
return new DrillSidewaysResult(r.facets, r.collectorResult);
} else {
TopScoreDocCollector hitCollector = TopScoreDocCollector.create(topN, after);
DrillSidewaysResult r = search(query, hitCollector);
return new DrillSidewaysResult(r.facets, hitCollector.topDocs());
}
}
/** Override this and return true if your collector
/**
* Override this and return true if your collector
* (e.g., {@code ToParentBlockJoinCollector}) expects all
* sub-scorers to be positioned on the document being
* collected. This will cause some performance loss;
* default is false. */
* default is false.
*/
protected boolean scoreSubDocsAtOnce() {
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}.
*/
public static class DrillSidewaysResult {
/** Combined drill down and sideways results. */
/**
* Combined drill down and sideways results.
*/
public final Facets facets;
/** Hits. */
/**
* Hits.
*/
public final TopDocs hits;
/** Sole constructor. */
/**
* Sole constructor.
*/
public DrillSidewaysResult(Facets facets, TopDocs hits) {
this.facets = facets;
this.hits = hits;
}
}
private static class CallableCollector implements Callable<CallableResult> {
private final int pos;
private final IndexSearcher searcher;
private final Query query;
private final CollectorManager<?, ?> collectorManager;
private CallableCollector(int pos, IndexSearcher searcher, Query query, CollectorManager<?, ?> collectorManager) {
this.pos = pos;
this.searcher = searcher;
this.query = query;
this.collectorManager = collectorManager;
}
@Override
public CallableResult call() throws Exception {
return new CallableResult(pos, searcher.search(query, collectorManager));
}
}
private static class CallableResult {
private final int pos;
private final Object result;
private CallableResult(int pos, Object result) {
this.pos = pos;
this.result = result;
}
}
private DrillDownQuery getDrillDownQuery(final DrillDownQuery query, Query[] queries,
final String excludedDimension) {
final DrillDownQuery ddl = new DrillDownQuery(config, query.getBaseQuery());
query.getDims().forEach((dim, pos) -> {
if (!dim.equals(excludedDimension))
ddl.add(dim, queries[pos]);
});
return ddl.getDims().size() == queries.length ? null : ddl;
}
/** Runs a search, using a {@link CollectorManager} to gather and merge search results */
public <R> ConcurrentDrillSidewaysResult<R> search(final DrillDownQuery query,
final CollectorManager<?, R> hitCollectorManager) throws IOException {
final Map<String, Integer> drillDownDims = query.getDims();
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)));
int i = 0;
final Query[] filters = query.getDrillDownQueries();
for (String dim : drillDownDims.keySet())
callableCollectors.add(new CallableCollector(i++, searcher, getDrillDownQuery(query, filters, dim),
new FacetsCollectorManager()));
final FacetsCollector mainFacetsCollector;
final FacetsCollector[] facetsCollectors = new FacetsCollector[drillDownDims.size()];
final R collectorResult;
try {
// Run the query pool
final List<Future<CallableResult>> futures = executor.invokeAll(callableCollectors);
// Extract the results
final Object[] mainResults = (Object[]) futures.get(0).get().result;
mainFacetsCollector = (FacetsCollector) mainResults[0];
collectorResult = (R) mainResults[1];
for (i = 1; i < futures.size(); i++) {
final CallableResult result = futures.get(i).get();
facetsCollectors[result.pos] = (FacetsCollector) result.result;
}
// Fill the null results with the mainFacetsCollector
for (i = 0; i < facetsCollectors.length; i++)
if (facetsCollectors[i] == null)
facetsCollectors[i] = mainFacetsCollector;
} catch (InterruptedException e) {
throw new ThreadInterruptedException(e);
} catch (ExecutionException e) {
throw new RuntimeException(e);
}
// build the facets and return the result
return new ConcurrentDrillSidewaysResult<>(buildFacetsResult(mainFacetsCollector, facetsCollectors,
drillDownDims.keySet().toArray(new String[drillDownDims.size()])), null, collectorResult);
}
/**
* Result of a concurrent drill sideways search, including the
* {@link Facets} and {@link TopDocs}.
*/
public static class ConcurrentDrillSidewaysResult<R> extends DrillSidewaysResult {
/** The merged search results */
public final R collectorResult;
/**
* Sole constructor.
*/
ConcurrentDrillSidewaysResult(Facets facets, TopDocs hits, R collectorResult) {
super(facets, hits);
this.collectorResult = collectorResult;
}
}
}

View File

@ -0,0 +1,55 @@
/*
* 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 org.apache.lucene.search.CollectorManager;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
/**
* A {@link CollectorManager} implementation which produce FacetsCollector and product a merged FacetsCollector.
* This is used for concurrent FacetsCollection.
*/
class FacetsCollectorManager implements CollectorManager<FacetsCollector, FacetsCollector> {
public final static FacetsCollector EMPTY = new FacetsCollector();
@Override
public FacetsCollector newCollector() throws IOException {
return new FacetsCollector();
}
@Override
public FacetsCollector reduce(Collection<FacetsCollector> collectors) throws IOException {
if (collectors == null || collectors.size() == 0)
return EMPTY;
if (collectors.size() == 1)
return collectors.iterator().next();
return new ReducedFacetsCollector(collectors);
}
private static class ReducedFacetsCollector extends FacetsCollector {
public ReducedFacetsCollector(final Collection<FacetsCollector> facetsCollectors) {
final List<MatchingDocs> matchingDocs = this.getMatchingDocs();
facetsCollectors.forEach(facetsCollector -> matchingDocs.addAll(facetsCollector.getMatchingDocs()));
}
}
}

View File

@ -16,16 +16,6 @@
*/
package org.apache.lucene.facet;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@ -68,8 +58,61 @@ import org.apache.lucene.util.InPlaceMergeSorter;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.TestUtil;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
public class TestDrillSideways extends FacetTestCase {
protected DrillSideways getNewDrillSideways(IndexSearcher searcher, FacetsConfig config,
SortedSetDocValuesReaderState state) {
return new DrillSideways(searcher, config, state);
}
protected DrillSideways getNewDrillSideways(IndexSearcher searcher, FacetsConfig config, TaxonomyReader taxoReader) {
return new DrillSideways(searcher, config, taxoReader);
}
protected DrillSideways getNewDrillSidewaysScoreSubdocsAtOnce(IndexSearcher searcher, FacetsConfig config,
TaxonomyReader taxoReader) {
return new DrillSideways(searcher, config, taxoReader) {
@Override
protected boolean scoreSubDocsAtOnce() {
return true;
}
};
}
protected DrillSideways getNewDrillSidewaysBuildFacetsResult(IndexSearcher searcher, FacetsConfig config,
TaxonomyReader taxoReader) {
return new DrillSideways(searcher, config, taxoReader) {
@Override
protected Facets buildFacetsResult(FacetsCollector drillDowns, FacetsCollector[] drillSideways,
String[] drillSidewaysDims) throws IOException {
Map<String, Facets> drillSidewaysFacets = new HashMap<>();
Facets drillDownFacets = getTaxonomyFacetCounts(taxoReader, config, drillDowns);
if (drillSideways != null) {
for (int i = 0; i < drillSideways.length; i++) {
drillSidewaysFacets.put(drillSidewaysDims[i], getTaxonomyFacetCounts(taxoReader, config, drillSideways[i]));
}
}
if (drillSidewaysFacets.isEmpty()) {
return drillDownFacets;
} else {
return new MultiFacets(drillSidewaysFacets, drillDownFacets);
}
}
};
}
public void testBasic() throws Exception {
Directory dir = newDirectory();
Directory taxoDir = newDirectory();
@ -116,7 +159,7 @@ public class TestDrillSideways extends FacetTestCase {
// NRT open
TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
DrillSideways ds = new DrillSideways(searcher, config, taxoReader);
DrillSideways ds = getNewDrillSideways(searcher, config, taxoReader);
// case: drill-down on a single field; in this
// case the drill-sideways + drill-down counts ==
@ -127,12 +170,14 @@ public class TestDrillSideways extends FacetTestCase {
assertEquals(2, r.hits.totalHits);
// Publish Date is only drill-down, and Lisa published
// one in 2012 and one in 2010:
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=Publish Date path=[] value=2 childCount=2\n 2010 (1)\n 2012 (1)\n",
r.facets.getTopChildren(10, "Publish Date").toString());
// Author is drill-sideways + drill-down: Lisa
// (drill-down) published twice, and Frank/Susan/Bob
// published once:
assertEquals("dim=Author path=[] value=5 childCount=4\n Lisa (2)\n Bob (1)\n Susan (1)\n Frank (1)\n", r.facets.getTopChildren(10, "Author").toString());
assertEquals("dim=Author path=[] value=5 childCount=4\n Lisa (2)\n Bob (1)\n Susan (1)\n Frank (1)\n",
r.facets.getTopChildren(10, "Author").toString());
// Same simple case, but no baseQuery (pure browse):
// drill-down on a single field; in this case the
@ -145,12 +190,14 @@ public class TestDrillSideways extends FacetTestCase {
assertEquals(2, r.hits.totalHits);
// Publish Date is only drill-down, and Lisa published
// one in 2012 and one in 2010:
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=Publish Date path=[] value=2 childCount=2\n 2010 (1)\n 2012 (1)\n",
r.facets.getTopChildren(10, "Publish Date").toString());
// Author is drill-sideways + drill-down: Lisa
// (drill-down) published twice, and Frank/Susan/Bob
// published once:
assertEquals("dim=Author path=[] value=5 childCount=4\n Lisa (2)\n Bob (1)\n Susan (1)\n Frank (1)\n", r.facets.getTopChildren(10, "Author").toString());
assertEquals("dim=Author path=[] value=5 childCount=4\n Lisa (2)\n Bob (1)\n Susan (1)\n Frank (1)\n",
r.facets.getTopChildren(10, "Author").toString());
// Another simple case: drill-down on single fields
// but OR of two values
@ -161,17 +208,21 @@ public class TestDrillSideways extends FacetTestCase {
assertEquals(3, r.hits.totalHits);
// Publish Date is only drill-down: Lisa and Bob
// (drill-down) published twice in 2010 and once in 2012:
assertEquals("dim=Publish Date path=[] value=3 childCount=2\n 2010 (2)\n 2012 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
assertEquals("dim=Publish Date path=[] value=3 childCount=2\n 2010 (2)\n 2012 (1)\n",
r.facets.getTopChildren(10, "Publish Date").toString());
// Author is drill-sideways + drill-down: Lisa
// (drill-down) published twice, and Frank/Susan/Bob
// published once:
assertEquals("dim=Author path=[] value=5 childCount=4\n Lisa (2)\n Bob (1)\n Susan (1)\n Frank (1)\n", r.facets.getTopChildren(10, "Author").toString());
assertEquals("dim=Author path=[] value=5 childCount=4\n Lisa (2)\n Bob (1)\n Susan (1)\n Frank (1)\n",
r.facets.getTopChildren(10, "Author").toString());
assertTrue(r.facets instanceof MultiFacets);
List<FacetResult> allResults = r.facets.getAllDims(10);
assertEquals(2, 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());
assertEquals("dim=Publish Date path=[] value=3 childCount=2\n 2010 (2)\n 2012 (1)\n", allResults.get(1).toString());
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());
assertEquals("dim=Publish Date path=[] value=3 childCount=2\n 2010 (2)\n 2012 (1)\n",
allResults.get(1).toString());
// More interesting case: drill-down on two fields
ddq = new DrillDownQuery(config);
@ -181,10 +232,12 @@ public class TestDrillSideways extends FacetTestCase {
assertEquals(1, r.hits.totalHits);
// Publish Date is drill-sideways + drill-down: Lisa
// (drill-down) published once in 2010 and once in 2012:
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=Publish Date path=[] value=2 childCount=2\n 2010 (1)\n 2012 (1)\n",
r.facets.getTopChildren(10, "Publish Date").toString());
// Author is drill-sideways + drill-down:
// only Lisa & Bob published (once each) in 2010:
assertEquals("dim=Author path=[] value=2 childCount=2\n Bob (1)\n Lisa (1)\n", r.facets.getTopChildren(10, "Author").toString());
assertEquals("dim=Author path=[] value=2 childCount=2\n Bob (1)\n Lisa (1)\n",
r.facets.getTopChildren(10, "Author").toString());
// Even more interesting case: drill down on two fields,
// but one of them is OR
@ -198,10 +251,12 @@ public class TestDrillSideways extends FacetTestCase {
assertEquals(2, r.hits.totalHits);
// Publish Date is both drill-sideways + drill-down:
// Lisa or Bob published twice in 2010 and once in 2012:
assertEquals("dim=Publish Date path=[] value=3 childCount=2\n 2010 (2)\n 2012 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
assertEquals("dim=Publish Date path=[] value=3 childCount=2\n 2010 (2)\n 2012 (1)\n",
r.facets.getTopChildren(10, "Publish Date").toString());
// Author is drill-sideways + drill-down:
// only Lisa & Bob published (once each) in 2010:
assertEquals("dim=Author path=[] value=2 childCount=2\n Bob (1)\n Lisa (1)\n", r.facets.getTopChildren(10, "Author").toString());
assertEquals("dim=Author path=[] value=2 childCount=2\n Bob (1)\n Lisa (1)\n",
r.facets.getTopChildren(10, "Author").toString());
// Test drilling down on invalid field:
ddq = new DrillDownQuery(config);
@ -219,11 +274,13 @@ public class TestDrillSideways extends FacetTestCase {
assertEquals(2, r.hits.totalHits);
// Publish Date is only drill-down, and Lisa published
// one in 2012 and one in 2010:
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=Publish Date path=[] value=2 childCount=2\n 2010 (1)\n 2012 (1)\n",
r.facets.getTopChildren(10, "Publish Date").toString());
// Author is drill-sideways + drill-down: Lisa
// (drill-down) published twice, and Frank/Susan/Bob
// published once:
assertEquals("dim=Author path=[] value=5 childCount=4\n Lisa (2)\n Bob (1)\n Susan (1)\n Frank (1)\n", r.facets.getTopChildren(10, "Author").toString());
assertEquals("dim=Author path=[] value=5 childCount=4\n Lisa (2)\n Bob (1)\n Susan (1)\n Frank (1)\n",
r.facets.getTopChildren(10, "Author").toString());
// LUCENE-4915: test drilling down on a dimension but
// NOT facet counting it:
@ -234,7 +291,8 @@ public class TestDrillSideways extends FacetTestCase {
assertEquals(2, r.hits.totalHits);
// Publish Date is only drill-down, and Lisa published
// one in 2012 and one in 2010:
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=Publish Date path=[] value=2 childCount=2\n 2010 (1)\n 2012 (1)\n",
r.facets.getTopChildren(10, "Publish Date").toString());
// Test main query gets null scorer:
ddq = new DrillDownQuery(config, new TermQuery(new Term("foobar", "baz")));
@ -288,16 +346,18 @@ public class TestDrillSideways extends FacetTestCase {
DrillDownQuery ddq = new DrillDownQuery(config);
ddq.add("Author", "Lisa");
DrillSidewaysResult r = new DrillSideways(searcher, config, taxoReader).search(null, ddq, 10);
DrillSidewaysResult r = getNewDrillSideways(searcher, config, taxoReader).search(null, ddq, 10);
assertEquals(1, r.hits.totalHits);
// Publish Date is only drill-down, and Lisa published
// one in 2012 and one in 2010:
assertEquals("dim=Publish Date path=[] value=1 childCount=1\n 2010 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
assertEquals("dim=Publish Date path=[] value=1 childCount=1\n 2010 (1)\n",
r.facets.getTopChildren(10, "Publish Date").toString());
// Author is drill-sideways + drill-down: Lisa
// (drill-down) published once, and Bob
// published once:
assertEquals("dim=Author path=[] value=2 childCount=2\n Bob (1)\n Lisa (1)\n", r.facets.getTopChildren(10, "Author").toString());
assertEquals("dim=Author path=[] value=2 childCount=2\n Bob (1)\n Lisa (1)\n",
r.facets.getTopChildren(10, "Author").toString());
writer.close();
IOUtils.close(searcher.getIndexReader(), taxoReader, taxoWriter, dir, taxoDir);
@ -349,11 +409,13 @@ public class TestDrillSideways extends FacetTestCase {
DrillDownQuery ddq = new DrillDownQuery(config);
ddq.add("dim", "a");
DrillSidewaysResult r = new DrillSideways(searcher, config, taxoReader).search(null, ddq, 10);
DrillSidewaysResult r = getNewDrillSideways(searcher, config, taxoReader).search(null, ddq, 10);
assertEquals(3, r.hits.totalHits);
assertEquals("dim=dim path=[] value=6 childCount=4\n a (3)\n b (1)\n c (1)\n d (1)\n", r.facets.getTopChildren(10, "dim").toString());
assertEquals("dim=dim path=[a] value=3 childCount=3\n x (1)\n y (1)\n z (1)\n", r.facets.getTopChildren(10, "dim", "a").toString());
assertEquals("dim=dim path=[] value=6 childCount=4\n a (3)\n b (1)\n c (1)\n d (1)\n",
r.facets.getTopChildren(10, "dim").toString());
assertEquals("dim=dim path=[a] value=3 childCount=3\n x (1)\n y (1)\n z (1)\n",
r.facets.getTopChildren(10, "dim", "a").toString());
writer.close();
IOUtils.close(searcher.getIndexReader(), taxoReader, taxoWriter, dir, taxoDir);
@ -363,7 +425,8 @@ public class TestDrillSideways extends FacetTestCase {
String id;
String contentToken;
public Doc() {}
public Doc() {
}
// -1 if the doc is missing this dim, else the index
// -into the values for this dim:
@ -427,12 +490,14 @@ public class TestDrillSideways extends FacetTestCase {
int numDocs = atLeast(3000);
//int numDocs = 20;
if (VERBOSE) {
System.out.println("numDims=" + numDims + " numDocs=" + numDocs + " aChance=" + aChance + " bChance=" + bChance + " cChance=" + cChance);
System.out.println(
"numDims=" + numDims + " numDocs=" + numDocs + " aChance=" + aChance + " bChance=" + bChance + " cChance="
+ cChance);
}
String[][] dimValues = new String[numDims][];
int valueCount = 2;
for(int dim=0;dim<numDims;dim++) {
for (int dim = 0; dim < numDims; dim++) {
Set<String> values = new HashSet<>();
while (values.size() < valueCount) {
String s = TestUtil.randomRealisticUnicodeString(random());
@ -446,19 +511,19 @@ public class TestDrillSideways extends FacetTestCase {
}
List<Doc> docs = new ArrayList<>();
for(int i=0;i<numDocs;i++) {
for (int i = 0; i < numDocs; i++) {
Doc doc = new Doc();
doc.id = ""+i;
doc.id = "" + i;
doc.contentToken = randomContentToken(false);
doc.dims = new int[numDims];
doc.dims2 = new int[numDims];
for(int dim=0;dim<numDims;dim++) {
for (int dim = 0; dim < numDims; dim++) {
if (random().nextInt(5) == 3) {
// This doc is missing this dim:
doc.dims[dim] = -1;
} else if (dimValues[dim].length <= 4) {
int dimUpto = 0;
doc.dims[dim] = dimValues[dim].length-1;
doc.dims[dim] = dimValues[dim].length - 1;
while (dimUpto < dimValues[dim].length) {
if (random().nextBoolean()) {
doc.dims[dim] = dimUpto;
@ -488,13 +553,13 @@ public class TestDrillSideways extends FacetTestCase {
RandomIndexWriter w = new RandomIndexWriter(random(), d, iwc);
DirectoryTaxonomyWriter tw = new DirectoryTaxonomyWriter(td, IndexWriterConfig.OpenMode.CREATE);
FacetsConfig config = new FacetsConfig();
for(int i=0;i<numDims;i++) {
config.setMultiValued("dim"+i, true);
for (int i = 0; i < numDims; i++) {
config.setMultiValued("dim" + i, true);
}
boolean doUseDV = random().nextBoolean();
for(Doc rawDoc : docs) {
for (Doc rawDoc : docs) {
Document doc = new Document();
doc.add(newStringField("id", rawDoc.id, Field.Store.YES));
doc.add(new SortedDocValuesField("id", new BytesRef(rawDoc.id)));
@ -503,7 +568,7 @@ public class TestDrillSideways extends FacetTestCase {
if (VERBOSE) {
System.out.println(" doc id=" + rawDoc.id + " token=" + rawDoc.contentToken);
}
for(int dim=0;dim<numDims;dim++) {
for (int dim = 0; dim < numDims; dim++) {
int dimValue = rawDoc.dims[dim];
if (dimValue != -1) {
if (doUseDV) {
@ -579,12 +644,13 @@ public class TestDrillSideways extends FacetTestCase {
int numIters = atLeast(10);
for(int iter=0;iter<numIters;iter++) {
for (int iter = 0; iter < numIters; iter++) {
String contentToken = random().nextInt(30) == 17 ? null : randomContentToken(true);
int numDrillDown = TestUtil.nextInt(random(), 1, Math.min(4, numDims));
if (VERBOSE) {
System.out.println("\nTEST: iter=" + iter + " baseQuery=" + contentToken + " numDrillDown=" + numDrillDown + " useSortedSetDV=" + doUseDV);
System.out.println("\nTEST: iter=" + iter + " baseQuery=" + contentToken + " numDrillDown=" + numDrillDown
+ " useSortedSetDV=" + doUseDV);
}
String[][] drillDowns = new String[numDims][];
@ -596,15 +662,15 @@ public class TestDrillSideways extends FacetTestCase {
if (drillDowns[dim] == null) {
if (random().nextBoolean()) {
// Drill down on one value:
drillDowns[dim] = new String[] {dimValues[dim][random().nextInt(dimValues[dim].length)]};
drillDowns[dim] = new String[] { dimValues[dim][random().nextInt(dimValues[dim].length)] };
} else {
int orCount = TestUtil.nextInt(random(), 1, Math.min(5, dimValues[dim].length));
drillDowns[dim] = new String[orCount];
anyMultiValuedDrillDowns |= orCount > 1;
for(int i=0;i<orCount;i++) {
for (int i = 0; i < orCount; i++) {
while (true) {
String value = dimValues[dim][random().nextInt(dimValues[dim].length)];
for(int j=0;j<i;j++) {
for (int j = 0; j < i; j++) {
if (value.equals(drillDowns[dim][j])) {
value = null;
break;
@ -619,7 +685,7 @@ public class TestDrillSideways extends FacetTestCase {
}
if (VERBOSE) {
BytesRef[] values = new BytesRef[drillDowns[dim].length];
for(int i=0;i<values.length;i++) {
for (int i = 0; i < values.length; i++) {
values[i] = new BytesRef(drillDowns[dim][i]);
}
System.out.println(" dim" + dim + "=" + Arrays.toString(values));
@ -637,9 +703,9 @@ public class TestDrillSideways extends FacetTestCase {
DrillDownQuery ddq = new DrillDownQuery(config, baseQuery);
for(int dim=0;dim<numDims;dim++) {
for (int dim = 0; dim < numDims; dim++) {
if (drillDowns[dim] != null) {
for(String value : drillDowns[dim]) {
for (String value : drillDowns[dim]) {
ddq.add("dim" + dim, value);
}
}
@ -699,8 +765,7 @@ public class TestDrillSideways extends FacetTestCase {
// Verify docs are always collected in order. If we
// had an AssertingScorer it could catch it when
// Weight.scoresDocsOutOfOrder lies!:
new DrillSideways(s, config, tr).search(ddq,
new SimpleCollector() {
getNewDrillSideways(s, config, tr).search(ddq, new SimpleCollector() {
int lastDocID;
@Override
@ -728,12 +793,7 @@ public class TestDrillSideways extends FacetTestCase {
// drill-down values, because in that case it's
// easily possible for one of the DD terms to be on
// a future docID:
new DrillSideways(s, config, tr) {
@Override
protected boolean scoreSubDocsAtOnce() {
return true;
}
}.search(ddq, new AssertingSubDocsAtOnceCollector());
getNewDrillSidewaysScoreSubdocsAtOnce(s, config, tr).search(ddq, new AssertingSubDocsAtOnceCollector());
}
TestFacetResult expected = slowDrillSidewaysSearch(s, docs, contentToken, drillDowns, dimValues, filter);
@ -741,36 +801,17 @@ public class TestDrillSideways extends FacetTestCase {
Sort sort = new Sort(new SortField("id", SortField.Type.STRING));
DrillSideways ds;
if (doUseDV) {
ds = new DrillSideways(s, config, sortedSetDVState);
ds = getNewDrillSideways(s, config, sortedSetDVState);
} else {
ds = new DrillSideways(s, config, tr) {
@Override
protected Facets buildFacetsResult(FacetsCollector drillDowns, FacetsCollector[] drillSideways, String[] drillSidewaysDims) throws IOException {
Map<String,Facets> drillSidewaysFacets = new HashMap<>();
Facets drillDownFacets = getTaxonomyFacetCounts(taxoReader, config, drillDowns);
if (drillSideways != null) {
for(int i=0;i<drillSideways.length;i++) {
drillSidewaysFacets.put(drillSidewaysDims[i],
getTaxonomyFacetCounts(taxoReader, config, drillSideways[i]));
}
}
if (drillSidewaysFacets.isEmpty()) {
return drillDownFacets;
} else {
return new MultiFacets(drillSidewaysFacets, drillDownFacets);
}
}
};
ds = getNewDrillSidewaysBuildFacetsResult(s, config, tr);
}
// Retrieve all facets:
DrillSidewaysResult actual = ds.search(ddq, filter, null, numDocs, sort, true, true);
TopDocs hits = s.search(baseQuery, numDocs);
Map<String,Float> scores = new HashMap<>();
for(ScoreDoc sd : hits.scoreDocs) {
Map<String, Float> scores = new HashMap<>();
for (ScoreDoc sd : hits.scoreDocs) {
scores.put(s.doc(sd.doc).get("id"), sd.score);
}
if (VERBOSE) {
@ -781,14 +822,11 @@ public class TestDrillSideways extends FacetTestCase {
// Make sure drill down doesn't change score:
Query q = ddq;
if (filter != null) {
q = new BooleanQuery.Builder()
.add(q, Occur.MUST)
.add(filter, Occur.FILTER)
.build();
q = new BooleanQuery.Builder().add(q, Occur.MUST).add(filter, Occur.FILTER).build();
}
TopDocs ddqHits = s.search(q, numDocs);
assertEquals(expected.hits.size(), ddqHits.totalHits);
for(int i=0;i<expected.hits.size();i++) {
for (int i = 0; i < expected.hits.size(); i++) {
// Score should be IDENTICAL:
assertEquals(scores.get(expected.hits.get(i).id), ddqHits.scoreDocs[i].score, 0.0f);
}
@ -803,7 +841,7 @@ public class TestDrillSideways extends FacetTestCase {
public Counters(String[][] dimValues) {
counts = new int[dimValues.length][];
for(int dim=0;dim<dimValues.length;dim++) {
for (int dim = 0; dim < dimValues.length; dim++) {
counts[dim] = new int[dimValues[dim].length];
}
}
@ -815,7 +853,7 @@ public class TestDrillSideways extends FacetTestCase {
public void inc(int[] dims, int[] dims2, int onlyDim) {
assert dims.length == counts.length;
assert dims2.length == counts.length;
for(int dim=0;dim<dims.length;dim++) {
for (int dim = 0; dim < dims.length; dim++) {
if (onlyDim == -1 || dim == onlyDim) {
if (dims[dim] != -1) {
counts[dim][dims[dim]]++;
@ -832,12 +870,14 @@ public class TestDrillSideways extends FacetTestCase {
List<Doc> hits;
int[][] counts;
int[] uniqueCounts;
public TestFacetResult() {}
public TestFacetResult() {
}
}
private int[] getTopNOrds(final int[] counts, final String[] values, int topN) {
final int[] ids = new int[counts.length];
for(int i=0;i<ids.length;i++) {
for (int i = 0; i < ids.length; i++) {
ids[i] = i;
}
@ -874,7 +914,7 @@ public class TestDrillSideways extends FacetTestCase {
}
int numSet = topN;
for(int i=0;i<topN;i++) {
for (int i = 0; i < topN; i++) {
if (counts[ids[i]] == 0) {
numSet = i;
break;
@ -886,15 +926,14 @@ public class TestDrillSideways extends FacetTestCase {
return topNIDs;
}
private TestFacetResult slowDrillSidewaysSearch(IndexSearcher s, List<Doc> docs,
String contentToken, String[][] drillDowns,
String[][] dimValues, Query onlyEven) throws Exception {
private TestFacetResult slowDrillSidewaysSearch(IndexSearcher s, List<Doc> docs, String contentToken,
String[][] drillDowns, String[][] dimValues, Query onlyEven) throws Exception {
int numDims = dimValues.length;
List<Doc> hits = new ArrayList<>();
Counters drillDownCounts = new Counters(dimValues);
Counters[] drillSidewaysCounts = new Counters[dimValues.length];
for(int dim=0;dim<numDims;dim++) {
for (int dim = 0; dim < numDims; dim++) {
drillSidewaysCounts[dim] = new Counters(dimValues);
}
@ -902,7 +941,8 @@ public class TestDrillSideways extends FacetTestCase {
System.out.println(" compute expected");
}
nextDoc: for(Doc doc : docs) {
nextDoc:
for (Doc doc : docs) {
if (doc.deleted) {
continue;
}
@ -911,12 +951,12 @@ public class TestDrillSideways extends FacetTestCase {
}
if (contentToken == null || doc.contentToken.equals(contentToken)) {
int failDim = -1;
for(int dim=0;dim<numDims;dim++) {
for (int dim = 0; dim < numDims; dim++) {
if (drillDowns[dim] != null) {
String docValue = doc.dims[dim] == -1 ? null : dimValues[dim][doc.dims[dim]];
String docValue2 = doc.dims2[dim] == -1 ? null : dimValues[dim][doc.dims2[dim]];
boolean matches = false;
for(String value : drillDowns[dim]) {
for (String value : drillDowns[dim]) {
if (value.equals(docValue) || value.equals(docValue2)) {
matches = true;
break;
@ -941,7 +981,7 @@ public class TestDrillSideways extends FacetTestCase {
// Hit:
hits.add(doc);
drillDownCounts.inc(doc.dims, doc.dims2);
for(int dim=0;dim<dimValues.length;dim++) {
for (int dim = 0; dim < dimValues.length; dim++) {
drillSidewaysCounts[dim].inc(doc.dims, doc.dims2);
}
} else {
@ -953,8 +993,8 @@ public class TestDrillSideways extends FacetTestCase {
}
}
Map<String,Integer> idToDocID = new HashMap<>();
for(int i=0;i<s.getIndexReader().maxDoc();i++) {
Map<String, Integer> idToDocID = new HashMap<>();
for (int i = 0; i < s.getIndexReader().maxDoc(); i++) {
idToDocID.put(s.doc(i).get("id"), i);
}
@ -982,36 +1022,35 @@ public class TestDrillSideways extends FacetTestCase {
return res;
}
void verifyEquals(String[][] dimValues, IndexSearcher s, TestFacetResult expected,
DrillSidewaysResult actual, Map<String,Float> scores, boolean isSortedSetDV) throws Exception {
void verifyEquals(String[][] dimValues, IndexSearcher s, TestFacetResult expected, DrillSidewaysResult actual,
Map<String, Float> scores, boolean isSortedSetDV) throws Exception {
if (VERBOSE) {
System.out.println(" verify totHits=" + expected.hits.size());
}
assertEquals(expected.hits.size(), actual.hits.totalHits);
assertEquals(expected.hits.size(), actual.hits.scoreDocs.length);
for(int i=0;i<expected.hits.size();i++) {
for (int i = 0; i < expected.hits.size(); i++) {
if (VERBOSE) {
System.out.println(" hit " + i + " expected=" + expected.hits.get(i).id);
}
assertEquals(expected.hits.get(i).id,
s.doc(actual.hits.scoreDocs[i].doc).get("id"));
assertEquals(expected.hits.get(i).id, s.doc(actual.hits.scoreDocs[i].doc).get("id"));
// Score should be IDENTICAL:
assertEquals(scores.get(expected.hits.get(i).id), actual.hits.scoreDocs[i].score, 0.0f);
}
for(int dim=0;dim<expected.counts.length;dim++) {
for (int dim = 0; dim < expected.counts.length; dim++) {
int topN = random().nextBoolean() ? dimValues[dim].length : TestUtil.nextInt(random(), 1, dimValues[dim].length);
FacetResult fr = actual.facets.getTopChildren(topN, "dim"+dim);
FacetResult fr = actual.facets.getTopChildren(topN, "dim" + dim);
if (VERBOSE) {
System.out.println(" dim" + dim + " topN=" + topN + " (vs " + dimValues[dim].length + " unique values)");
System.out.println(" actual");
}
int idx = 0;
Map<String,Integer> actualValues = new HashMap<>();
Map<String, Integer> actualValues = new HashMap<>();
if (fr != null) {
for(LabelAndValue labelValue : fr.labelValues) {
for (LabelAndValue labelValue : fr.labelValues) {
actualValues.put(labelValue.label, labelValue.value.intValue());
if (VERBOSE) {
System.out.println(" " + idx + ": " + new BytesRef(labelValue.label) + ": " + labelValue.value);
@ -1026,10 +1065,11 @@ public class TestDrillSideways extends FacetTestCase {
if (VERBOSE) {
idx = 0;
System.out.println(" expected (sorted)");
for(int i=0;i<topNIDs.length;i++) {
for (int i = 0; i < topNIDs.length; i++) {
int expectedOrd = topNIDs[i];
String value = dimValues[dim][expectedOrd];
System.out.println(" " + idx + ": " + new BytesRef(value) + ": " + expected.counts[dim][expectedOrd]);
System.out.println(
" " + idx + ": " + new BytesRef(value) + ": " + expected.counts[dim][expectedOrd]);
idx++;
}
}
@ -1042,7 +1082,7 @@ public class TestDrillSideways extends FacetTestCase {
} else {
assertEquals(0, topNIDs.length);
}
for(int i=0;i<topNIDs.length;i++) {
for (int i = 0; i < topNIDs.length; i++) {
int expectedOrd = topNIDs[i];
assertEquals(expected.counts[dim][expectedOrd], fr.labelValues[i].value.intValue());
if (isSortedSetDV) {
@ -1056,7 +1096,7 @@ public class TestDrillSideways extends FacetTestCase {
if (VERBOSE) {
idx = 0;
System.out.println(" expected (unsorted)");
for(int i=0;i<dimValues[dim].length;i++) {
for (int i = 0; i < dimValues[dim].length; i++) {
String value = dimValues[dim][i];
if (expected.counts[dim][i] != 0) {
System.out.println(" " + idx + ": " + new BytesRef(value) + ": " + expected.counts[dim][i]);
@ -1066,7 +1106,7 @@ public class TestDrillSideways extends FacetTestCase {
}
int setCount = 0;
for(int i=0;i<dimValues[dim].length;i++) {
for (int i = 0; i < dimValues[dim].length; i++) {
String value = dimValues[dim][i];
if (expected.counts[dim][i] != 0) {
assertTrue(actualValues.containsKey(value));
@ -1092,14 +1132,15 @@ public class TestDrillSideways extends FacetTestCase {
// Count "Author"
FacetsConfig config = new FacetsConfig();
DrillSideways ds = new DrillSideways(searcher, config, taxoReader);
DrillSideways ds = getNewDrillSideways(searcher, config, taxoReader);
DrillDownQuery ddq = new DrillDownQuery(config);
ddq.add("Author", "Lisa");
DrillSidewaysResult r = ds.search(ddq, 10); // this used to fail on IllegalArgEx
assertEquals(0, r.hits.totalHits);
r = ds.search(ddq, null, null, 10, new Sort(new SortField("foo", SortField.Type.INT)), false, false); // this used to fail on IllegalArgEx
r = ds.search(ddq, null, null, 10, new Sort(new SortField("foo", SortField.Type.INT)), false,
false); // this used to fail on IllegalArgEx
assertEquals(0, r.hits.totalHits);
writer.close();
@ -1131,7 +1172,7 @@ public class TestDrillSideways extends FacetTestCase {
// NRT open
TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
DrillSideways ds = new DrillSideways(searcher, config, taxoReader);
DrillSideways ds = getNewDrillSideways(searcher, config, taxoReader);
BooleanQuery.Builder bq = new BooleanQuery.Builder();
bq.add(new TermQuery(new Term("field", "foo")), BooleanClause.Occur.MUST);

View File

@ -0,0 +1,90 @@
/*
* 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.HashMap;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState;
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.util.NamedThreadFactory;
import org.junit.AfterClass;
import org.junit.BeforeClass;
public class TestParallelDrillSideways extends TestDrillSideways {
private static ExecutorService executor;
@BeforeClass
public static void prepareExecutor() {
executor = Executors.newCachedThreadPool(new NamedThreadFactory("TestParallelDrillSideways"));
}
@AfterClass
public static void shutdownExecutor() {
executor.shutdown();
executor = null;
}
protected DrillSideways getNewDrillSideways(IndexSearcher searcher, FacetsConfig config,
SortedSetDocValuesReaderState state) {
return new DrillSideways(searcher, config, null, state, executor);
}
protected DrillSideways getNewDrillSideways(IndexSearcher searcher, FacetsConfig config, TaxonomyReader taxoReader) {
return new DrillSideways(searcher, config, taxoReader, null, executor);
}
protected DrillSideways getNewDrillSidewaysScoreSubdocsAtOnce(IndexSearcher searcher, FacetsConfig config,
TaxonomyReader taxoReader) {
return new DrillSideways(searcher, config, taxoReader, null, executor) {
@Override
protected boolean scoreSubDocsAtOnce() {
return true;
}
};
}
protected DrillSideways getNewDrillSidewaysBuildFacetsResult(IndexSearcher searcher, FacetsConfig config,
TaxonomyReader taxoReader) {
return new DrillSideways(searcher, config, taxoReader, null, executor) {
@Override
protected Facets buildFacetsResult(FacetsCollector drillDowns, FacetsCollector[] drillSideways,
String[] drillSidewaysDims) throws IOException {
Map<String, Facets> drillSidewaysFacets = new HashMap<>();
Facets drillDownFacets = getTaxonomyFacetCounts(taxoReader, config, drillDowns);
if (drillSideways != null) {
for (int i = 0; i < drillSideways.length; i++) {
drillSidewaysFacets.put(drillSidewaysDims[i], getTaxonomyFacetCounts(taxoReader, config, drillSideways[i]));
}
}
if (drillSidewaysFacets.isEmpty()) {
return drillDownFacets;
} else {
return new MultiFacets(drillSidewaysFacets, drillDownFacets);
}
}
};
}
}