LUCENE-4600: add CountingFacetsCollector

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1436435 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Shai Erera 2013-01-21 16:04:56 +00:00
parent a78e69c665
commit cefb5f8523
30 changed files with 1120 additions and 226 deletions

View File

@ -32,6 +32,12 @@ Changes in backwards compatibility policy
* LUCENE-4697: FacetResultNode is now a concrete class with public members
(instead of getter methods). (Shai Erera)
* LUCENE-4600: FacetsCollector is now an abstract class with two
implementations: StandardFacetsCollector (the old version of
FacetsCollector) and CountingFacetsCollector. FacetsCollector.create()
returns the most optimized collector for the given parameters.
(Shai Erera, Michael McCandless)
Optimizations
* LUCENE-4687: BloomFilterPostingsFormat now lazily initializes delegate

View File

@ -109,8 +109,7 @@ public class MultiCLSearcher {
// behavior - in those
// situations other, more low-level interfaces are available, as
// demonstrated in other search examples.
FacetsCollector facetsCollector = new FacetsCollector(
facetSearchParams, indexReader, taxo);
FacetsCollector facetsCollector = FacetsCollector.create(facetSearchParams, indexReader, taxo);
// perform documents search and facets accumulation
searcher.search(q, MultiCollector.wrap(topDocsCollector, facetsCollector));

View File

@ -104,9 +104,9 @@ public class SimpleSearcher {
}
// Faceted search parameters indicate which facets are we interested in
FacetSearchParams facetSearchParams = new FacetSearchParams(Arrays.asList(facetRequests), indexingParams);
FacetSearchParams facetSearchParams = new FacetSearchParams(indexingParams, facetRequests);
FacetsCollector facetsCollector = new FacetsCollector(facetSearchParams, indexReader, taxoReader);
FacetsCollector facetsCollector = FacetsCollector.create(facetSearchParams, indexReader, taxoReader);
// perform documents search and facets accumulation
searcher.search(q, MultiCollector.wrap(topDocsCollector, facetsCollector));

View File

@ -48,7 +48,7 @@ public class FacetIndexingParams {
/**
* A {@link FacetIndexingParams} which fixes {@link OrdinalPolicy} to
* {@link OrdinalPolicy#NO_PARENTS}. This is a singleton equivalent to new
* {@link OrdinalPolicy#ALL_PARENTS}. This is a singleton equivalent to new
* {@link #FacetIndexingParams()}.
*/
public static final FacetIndexingParams ALL_PARENTS = new FacetIndexingParams();

View File

@ -0,0 +1,346 @@
package org.apache.lucene.facet.search;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map.Entry;
import org.apache.lucene.facet.index.categorypolicy.OrdinalPolicy;
import org.apache.lucene.facet.index.params.CategoryListParams;
import org.apache.lucene.facet.index.params.FacetIndexingParams;
import org.apache.lucene.facet.search.params.CountFacetRequest;
import org.apache.lucene.facet.search.params.FacetRequest;
import org.apache.lucene.facet.search.params.FacetRequest.SortBy;
import org.apache.lucene.facet.search.params.FacetRequest.SortOrder;
import org.apache.lucene.facet.search.params.FacetSearchParams;
import org.apache.lucene.facet.search.results.FacetResult;
import org.apache.lucene.facet.search.results.FacetResultNode;
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.facet.taxonomy.directory.ParallelTaxonomyArrays;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.PriorityQueue;
import org.apache.lucene.util.encoding.DGapVInt8IntDecoder;
/*
* 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.
*/
/**
* A {@link Collector} which counts facets associated with matching documents.
* This {@link Collector} can be used only in the following conditions:
* <ul>
* <li>All {@link FacetRequest requests} must be {@link CountFacetRequest}, with
* their {@link FacetRequest#getDepth() depth} equals to 1, and
* {@link FacetRequest#getNumLabel()} must be &ge; than
* {@link FacetRequest#getNumResults()}. Also, their sorting options must be
* {@link SortOrder#DESCENDING} and {@link SortBy#VALUE} (although ties are
* broken by ordinals).
* <li>Partitions should be disabled (
* {@link FacetIndexingParams#getPartitionSize()} should return
* Integer.MAX_VALUE).
* <li>There can be only one {@link CategoryListParams} in the
* {@link FacetIndexingParams}, with {@link DGapVInt8IntDecoder}.
* </ul>
*
* <p>
* <b>NOTE:</b> this colletro uses {@link DocValues#getSource()} by default,
* which pre-loads the values into memory. If your application cannot afford the
* RAM, you should use
* {@link #CountingFacetsCollector(FacetSearchParams, TaxonomyReader, FacetArrays, boolean)}
* and specify to use a direct source (corresponds to
* {@link DocValues#getDirectSource()}).
*
* <p>
* <b>NOTE:</b> this collector supports category lists that were indexed with
* {@link OrdinalPolicy#NO_PARENTS}, by counting up the parents too, after
* resolving the leafs counts. Note though that it is your responsibility to
* guarantee that indeed a document wasn't indexed with two categories that
* share a common parent, or otherwise the parent's count will be wrong.
*
* @lucene.experimental
*/
public class CountingFacetsCollector extends FacetsCollector {
private final FacetSearchParams fsp;
private final TaxonomyReader taxoReader;
private final BytesRef buf = new BytesRef(32);
private final FacetArrays facetArrays;
private final int[] counts;
private final String facetsField;
private final boolean useDirectSource;
private final HashMap<Source,FixedBitSet> matchingDocs = new HashMap<Source,FixedBitSet>();
private DocValues facetsValues;
private FixedBitSet bits;
public CountingFacetsCollector(FacetSearchParams fsp, TaxonomyReader taxoReader) {
this(fsp, taxoReader, new FacetArrays(taxoReader.getSize()), false);
}
public CountingFacetsCollector(FacetSearchParams fsp, TaxonomyReader taxoReader, FacetArrays facetArrays,
boolean useDirectSource) {
assert facetArrays.arrayLength >= taxoReader.getSize() : "too small facet array";
assert assertParams(fsp) == null : assertParams(fsp);
this.fsp = fsp;
this.taxoReader = taxoReader;
this.facetArrays = facetArrays;
this.counts = facetArrays.getIntArray();
this.facetsField = fsp.indexingParams.getCategoryListParams(null).field;
this.useDirectSource = useDirectSource;
}
/**
* Asserts that this {@link FacetsCollector} can handle the given
* {@link FacetSearchParams}. Returns {@code null} if true, otherwise an error
* message.
*/
static String assertParams(FacetSearchParams fsp) {
// verify that all facet requests are CountFacetRequest
for (FacetRequest fr : fsp.facetRequests) {
if (!(fr instanceof CountFacetRequest)) {
return "all FacetRequests must be CountFacetRequest";
}
if (fr.getDepth() != 1) {
return "all requests must be of depth 1";
}
if (fr.getNumLabel() < fr.getNumResults()) {
return "this Collector always labels all requested results";
}
if (fr.getSortOrder() != SortOrder.DESCENDING) {
return "this Collector always sorts results in descending order";
}
if (fr.getSortBy() != SortBy.VALUE) {
return "this Collector always sorts by results' values";
}
}
// verify that there's only one CategoryListParams
List<CategoryListParams> clps = fsp.indexingParams.getAllCategoryListParams();
if (clps.size() != 1) {
return "this Collector supports only one CategoryListParams";
}
// verify DGapVInt decoder
CategoryListParams clp = clps.get(0);
if (clp.createEncoder().createMatchingDecoder().getClass() != DGapVInt8IntDecoder.class) {
return "this Collector supports only DGap + VInt encoding";
}
// verify that partitions are disabled
if (fsp.indexingParams.getPartitionSize() != Integer.MAX_VALUE) {
return "this Collector does not support partitions";
}
return null;
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
facetsValues = context.reader().docValues(facetsField);
if (facetsValues != null) {
Source facetSource = useDirectSource ? facetsValues.getDirectSource() : facetsValues.getSource();
bits = new FixedBitSet(context.reader().maxDoc());
matchingDocs.put(facetSource, bits);
}
}
@Override
public void collect(int doc) throws IOException {
if (facetsValues == null) {
return;
}
bits.set(doc);
}
private void countFacets() {
for (Entry<Source,FixedBitSet> entry : matchingDocs.entrySet()) {
Source facetsSource = entry.getKey();
FixedBitSet bits = entry.getValue();
int doc = 0;
int length = bits.length();
while (doc < length && (doc = bits.nextSetBit(doc)) != -1) {
facetsSource .getBytes(doc, buf);
if (buf.length > 0) {
// this document has facets
int upto = buf.offset + buf.length;
int ord = 0;
int offset = buf.offset;
int prev = 0;
while (offset < upto) {
byte b = buf.bytes[offset++];
if (b >= 0) {
prev = ord = ((ord << 7) | b) + prev;
counts[ord]++;
ord = 0;
} else {
ord = (ord << 7) | (b & 0x7F);
}
}
}
++doc;
}
}
}
private void countParents(int[] parents) {
// counts[0] is the count of ROOT, which we don't care about and counts[1]
// can only update counts[0], so we don't bother to visit it too. also,
// since parents always have lower ordinals than their children, we traverse
// the array backwards. this also allows us to update just the immediate
// parent's count (actually, otherwise it would be a mistake).
for (int i = counts.length - 1; i > 1; i--) {
int count = counts[i];
if (count > 0) {
int parent = parents[i];
if (parent != 0) {
counts[parent] += count;
}
}
}
}
@Override
public synchronized List<FacetResult> getFacetResults() throws IOException {
try {
// first, count matching documents' facets
countFacets();
ParallelTaxonomyArrays arrays = taxoReader.getParallelTaxonomyArrays();
if (fsp.indexingParams.getOrdinalPolicy() == OrdinalPolicy.NO_PARENTS) {
// need to count parents
countParents(arrays.parents());
}
// compute top-K
final int[] children = arrays.children();
final int[] siblings = arrays.siblings();
List<FacetResult> res = new ArrayList<FacetResult>();
for (FacetRequest fr : fsp.facetRequests) {
int rootOrd = taxoReader.getOrdinal(fr.categoryPath);
if (rootOrd == TaxonomyReader.INVALID_ORDINAL) { // category does not exist
continue;
}
FacetResultNode root = new FacetResultNode();
root.ordinal = rootOrd;
root.label = fr.categoryPath;
root.value = counts[rootOrd];
if (fr.getNumResults() > taxoReader.getSize()) {
// specialize this case, user is interested in all available results
ArrayList<FacetResultNode> nodes = new ArrayList<FacetResultNode>();
int child = children[rootOrd];
while (child != TaxonomyReader.INVALID_ORDINAL) {
int count = counts[child];
if (count > 0) {
FacetResultNode node = new FacetResultNode();
node.label = taxoReader.getPath(child);
node.value = count;
nodes.add(node);
}
child = siblings[child];
}
root.residue = 0;
root.subResults = nodes;
res.add(new FacetResult(fr, root, nodes.size()));
continue;
}
// since we use sentinel objects, we cannot reuse PQ. but that's ok because it's not big
FacetResultNodeQueue pq = new FacetResultNodeQueue(fr.getNumResults(), true);
FacetResultNode top = pq.top();
int child = children[rootOrd];
int numResults = 0; // count the number of results
int residue = 0;
while (child != TaxonomyReader.INVALID_ORDINAL) {
int count = counts[child];
if (count > top.value) {
residue += top.value;
top.value = count;
top.ordinal = child;
top = pq.updateTop();
++numResults;
} else {
residue += count;
}
child = siblings[child];
}
// pop() the least (sentinel) elements
int pqsize = pq.size();
int size = numResults < pqsize ? numResults : pqsize;
for (int i = pqsize - size; i > 0; i--) { pq.pop(); }
// create the FacetResultNodes.
FacetResultNode[] subResults = new FacetResultNode[size];
for (int i = size - 1; i >= 0; i--) {
FacetResultNode node = pq.pop();
node.label = taxoReader.getPath(node.ordinal);
subResults[i] = node;
}
root.residue = residue;
root.subResults = Arrays.asList(subResults);
res.add(new FacetResult(fr, root, size));
}
return res;
} finally {
facetArrays.free();
}
}
@Override
public boolean acceptsDocsOutOfOrder() {
// the actual work is done post-collection, so we always support out-of-order.
return true;
}
@Override
public void setScorer(Scorer scorer) throws IOException {
}
// TODO: review ResultSortUtils queues and check if we can reuse any of them here
// and then alleviate the SortOrder/SortBy constraint
private static class FacetResultNodeQueue extends PriorityQueue<FacetResultNode> {
public FacetResultNodeQueue(int maxSize, boolean prepopulate) {
super(maxSize, prepopulate);
}
@Override
protected FacetResultNode getSentinelObject() {
return new FacetResultNode();
}
@Override
protected boolean lessThan(FacetResultNode a, FacetResultNode b) {
if (a.value < b.value) return true;
if (a.value > b.value) return false;
// both have the same value, break tie by ordinal
return a.ordinal < b.ordinal;
}
}
}

View File

@ -47,7 +47,7 @@ public final class DrillDown {
* @see #term(FacetIndexingParams, CategoryPath)
*/
public static final Term term(FacetSearchParams sParams, CategoryPath path) {
return term(sParams.getFacetIndexingParams(), path);
return term(sParams.indexingParams, path);
}
/** Return a drill-down {@link Term} for a category. */
@ -103,7 +103,7 @@ public final class DrillDown {
* @see #query(FacetIndexingParams, Query, CategoryPath...)
*/
public static final Query query(FacetSearchParams sParams, Query baseQuery, CategoryPath... paths) {
return query(sParams.getFacetIndexingParams(), baseQuery, paths);
return query(sParams.indexingParams, baseQuery, paths);
}
}

View File

@ -138,7 +138,7 @@ public abstract class FacetsAccumulator {
/** check if all requests are complementable */
protected boolean mayComplement() {
for (FacetRequest freq:searchParams.getFacetRequests()) {
for (FacetRequest freq:searchParams.facetRequests) {
if (!freq.supportsComplements()) {
return false;
}

View File

@ -3,15 +3,13 @@ package org.apache.lucene.facet.search;
import java.io.IOException;
import java.util.List;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.facet.search.params.FacetRequest;
import org.apache.lucene.facet.search.params.FacetSearchParams;
import org.apache.lucene.facet.search.results.FacetResult;
import org.apache.lucene.facet.taxonomy.CategoryPath;
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.Collector;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -31,109 +29,35 @@ import org.apache.lucene.facet.taxonomy.TaxonomyReader;
*/
/**
* Collector for facet accumulation. *
* A {@link Collector} which executes faceted search and computes the weight of
* requested facets. To get the facet results you should call
* {@link #getFacetResults()}.
* {@link #create(FacetSearchParams, IndexReader, TaxonomyReader)} returns the
* most optimized {@link FacetsCollector} for the given parameters.
*
* @lucene.experimental
*/
public class FacetsCollector extends Collector {
protected final FacetsAccumulator facetsAccumulator;
private ScoredDocIdCollector scoreDocIdCollector;
private List<FacetResult> results;
private Object resultsGuard;
public abstract class FacetsCollector extends Collector {
/**
* Create a collector for accumulating facets while collecting documents
* during search.
*
* @param facetSearchParams
* faceted search parameters defining which facets are required and
* how.
* @param indexReader
* searched index.
* @param taxonomyReader
* taxonomy containing the facets.
* Returns the most optimized {@link FacetsCollector} for the given search
* parameters. The returned {@link FacetsCollector} is guaranteed to satisfy
* the requested parameters.
*/
public FacetsCollector(FacetSearchParams facetSearchParams,
IndexReader indexReader, TaxonomyReader taxonomyReader) {
facetsAccumulator = initFacetsAccumulator(facetSearchParams, indexReader, taxonomyReader);
scoreDocIdCollector = initScoredDocCollector(facetSearchParams, indexReader, taxonomyReader);
resultsGuard = new Object();
}
/**
* Create a {@link ScoredDocIdCollector} to be used as the first phase of
* the facet collection. If all facetRequests are do not require the
* document score, a ScoredDocIdCollector which does not store the document
* scores would be returned. Otherwise a SDIC which does store the documents
* will be returned, having an initial allocated space for 1000 such
* documents' scores.
*/
protected ScoredDocIdCollector initScoredDocCollector(
FacetSearchParams facetSearchParams, IndexReader indexReader,
TaxonomyReader taxonomyReader) {
boolean scoresNeeded = false;
for (FacetRequest frq : facetSearchParams.getFacetRequests()) {
if (frq.requireDocumentScore()) {
scoresNeeded = true;
break;
}
public static FacetsCollector create(FacetSearchParams fsp, IndexReader indexReader, TaxonomyReader taxoReader) {
if (CountingFacetsCollector.assertParams(fsp) == null) {
return new CountingFacetsCollector(fsp, taxoReader);
}
return ScoredDocIdCollector.create(indexReader.maxDoc(), scoresNeeded);
return new StandardFacetsCollector(fsp, indexReader, taxoReader);
}
/**
* Create the {@link FacetsAccumulator} to be used. Default is
* {@link StandardFacetsAccumulator}. Called once at the constructor of the collector.
*
* @param facetSearchParams
* The search params.
* @param indexReader
* A reader to the index to search in.
* @param taxonomyReader
* A reader to the active taxonomy.
* @return The {@link FacetsAccumulator} to use.
* Returns a {@link FacetResult} per {@link FacetRequest} set in
* {@link FacetSearchParams}. Note that if one of the {@link FacetRequest
* requests} is for a {@link CategoryPath} that does not exist in the taxonomy,
* no matching {@link FacetResult} will be returned.
*/
protected FacetsAccumulator initFacetsAccumulator(FacetSearchParams facetSearchParams,
IndexReader indexReader,
TaxonomyReader taxonomyReader) {
return new StandardFacetsAccumulator(facetSearchParams, indexReader, taxonomyReader);
}
/**
* Return accumulated facets results (according to faceted search parameters)
* for collected documents.
* @throws IOException on error
*/
public List<FacetResult> getFacetResults() throws IOException {
synchronized (resultsGuard) { // over protection
if (results == null) {
// lazy creation but just once
results = facetsAccumulator.accumulate(scoreDocIdCollector.getScoredDocIDs());
scoreDocIdCollector = null;
}
return results;
}
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
@Override
public void collect(int doc) throws IOException {
scoreDocIdCollector.collect(doc);
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
scoreDocIdCollector.setNextReader(context);
}
@Override
public void setScorer(Scorer scorer) throws IOException {
scoreDocIdCollector.setScorer(scorer);
}
public abstract List<FacetResult> getFacetResults() throws IOException;
}

View File

@ -87,7 +87,7 @@ public class StandardFacetsAccumulator extends FacetsAccumulator {
this.facetArrays = facetArrays;
// can only be computed later when docids size is known
isUsingComplements = false;
partitionSize = PartitionsUtils.partitionSize(searchParams.getFacetIndexingParams(), taxonomyReader);
partitionSize = PartitionsUtils.partitionSize(searchParams.indexingParams, taxonomyReader);
maxPartitions = (int) Math.ceil(this.taxonomyReader.getSize() / (double) partitionSize);
accumulateGuard = new Object();
}
@ -95,7 +95,7 @@ public class StandardFacetsAccumulator extends FacetsAccumulator {
public StandardFacetsAccumulator(FacetSearchParams searchParams,
IndexReader indexReader, TaxonomyReader taxonomyReader) {
this(searchParams, indexReader, taxonomyReader, new FacetArrays(
PartitionsUtils.partitionSize(searchParams.getFacetIndexingParams(), taxonomyReader)));
PartitionsUtils.partitionSize(searchParams.indexingParams, taxonomyReader)));
}
@Override
@ -112,7 +112,7 @@ public class StandardFacetsAccumulator extends FacetsAccumulator {
if (isUsingComplements) {
try {
totalFacetCounts = TotalFacetCountsCache.getSingleton().getTotalCounts(indexReader, taxonomyReader,
searchParams.getFacetIndexingParams());
searchParams.indexingParams);
if (totalFacetCounts != null) {
docids = ScoredDocIdsUtils.getComplementSet(docids, indexReader);
} else {
@ -159,7 +159,7 @@ public class StandardFacetsAccumulator extends FacetsAccumulator {
// In this implementation merges happen after each partition,
// but other impl could merge only at the end.
final HashSet<FacetRequest> handledRequests = new HashSet<FacetRequest>();
for (FacetRequest fr : searchParams.getFacetRequests()) {
for (FacetRequest fr : searchParams.facetRequests) {
// Handle and merge only facet requests which were not already handled.
if (handledRequests.add(fr)) {
FacetResultsHandler frHndlr = fr.createFacetResultsHandler(taxonomyReader);
@ -178,7 +178,7 @@ public class StandardFacetsAccumulator extends FacetsAccumulator {
// gather results from all requests into a list for returning them
List<FacetResult> res = new ArrayList<FacetResult>();
for (FacetRequest fr : searchParams.getFacetRequests()) {
for (FacetRequest fr : searchParams.facetRequests) {
FacetResultsHandler frHndlr = fr.createFacetResultsHandler(taxonomyReader);
IntermediateFacetResult tmpResult = fr2tmpRes.get(fr);
if (tmpResult == null) {
@ -321,8 +321,8 @@ public class StandardFacetsAccumulator extends FacetsAccumulator {
HashMap<CategoryListIterator, Aggregator> categoryLists = new HashMap<CategoryListIterator, Aggregator>();
FacetIndexingParams indexingParams = searchParams.getFacetIndexingParams();
for (FacetRequest facetRequest : searchParams.getFacetRequests()) {
FacetIndexingParams indexingParams = searchParams.indexingParams;
for (FacetRequest facetRequest : searchParams.facetRequests) {
Aggregator categoryAggregator = facetRequest.createAggregator(isUsingComplements, facetArrays, taxonomyReader);
CategoryListIterator cli = indexingParams.getCategoryListParams(facetRequest.categoryPath).createCategoryListIterator(partition);

View File

@ -0,0 +1,139 @@
package org.apache.lucene.facet.search;
import java.io.IOException;
import java.util.List;
import org.apache.lucene.facet.index.categorypolicy.OrdinalPolicy;
import org.apache.lucene.facet.index.params.CategoryListParams;
import org.apache.lucene.facet.search.params.FacetRequest;
import org.apache.lucene.facet.search.params.FacetSearchParams;
import org.apache.lucene.facet.search.results.FacetResult;
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.Scorer;
/*
* 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.
*/
/**
* A {@link FacetsCollector} which allows initilizing e.g.
* {@link FacetsAccumulator}. Supports facet partitions, generic
* {@link FacetRequest facet requests}, {@link CategoryListParams} etc.
*
* <p>
* <b>NOTE:</b> this collector, with the default {@link FacetsAccumulator} does
* not support category lists which were indexed with
* {@link OrdinalPolicy#NO_PARENTS}.
*
* @lucene.experimental
*/
public class StandardFacetsCollector extends FacetsCollector {
protected final FacetsAccumulator facetsAccumulator;
private ScoredDocIdCollector scoreDocIdCollector;
private List<FacetResult> results;
private Object resultsGuard;
/**
* Create a collector for accumulating facets while collecting documents
* during search.
*
* @param facetSearchParams
* faceted search parameters defining which facets are required and
* how.
* @param indexReader
* searched index.
* @param taxonomyReader
* taxonomy containing the facets.
*/
public StandardFacetsCollector(FacetSearchParams facetSearchParams, IndexReader indexReader, TaxonomyReader taxonomyReader) {
facetsAccumulator = initFacetsAccumulator(facetSearchParams, indexReader, taxonomyReader);
scoreDocIdCollector = initScoredDocCollector(facetSearchParams, indexReader, taxonomyReader);
resultsGuard = new Object();
}
/**
* Create a {@link ScoredDocIdCollector} to be used as the first phase of
* the facet collection. If all facetRequests are do not require the
* document score, a ScoredDocIdCollector which does not store the document
* scores would be returned. Otherwise a SDIC which does store the documents
* will be returned, having an initial allocated space for 1000 such
* documents' scores.
*/
protected ScoredDocIdCollector initScoredDocCollector(FacetSearchParams facetSearchParams, IndexReader indexReader,
TaxonomyReader taxonomyReader) {
boolean scoresNeeded = false;
for (FacetRequest frq : facetSearchParams.facetRequests) {
if (frq.requireDocumentScore()) {
scoresNeeded = true;
break;
}
}
return ScoredDocIdCollector.create(indexReader.maxDoc(), scoresNeeded);
}
/**
* Create the {@link FacetsAccumulator} to be used. Default is
* {@link StandardFacetsAccumulator}. Called once at the constructor of the collector.
*
* @param facetSearchParams
* The search params.
* @param indexReader
* A reader to the index to search in.
* @param taxonomyReader
* A reader to the active taxonomy.
* @return The {@link FacetsAccumulator} to use.
*/
protected FacetsAccumulator initFacetsAccumulator(FacetSearchParams facetSearchParams, IndexReader indexReader,
TaxonomyReader taxonomyReader) {
return new StandardFacetsAccumulator(facetSearchParams, indexReader, taxonomyReader);
}
@Override
public List<FacetResult> getFacetResults() throws IOException {
synchronized (resultsGuard) { // over protection
if (results == null) {
// lazy creation but just once
results = facetsAccumulator.accumulate(scoreDocIdCollector.getScoredDocIDs());
scoreDocIdCollector = null;
}
return results;
}
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
@Override
public void collect(int doc) throws IOException {
scoreDocIdCollector.collect(doc);
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
scoreDocIdCollector.setNextReader(context);
}
@Override
public void setScorer(Scorer scorer) throws IOException {
scoreDocIdCollector.setScorer(scorer);
}
}

View File

@ -8,9 +8,7 @@ import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.facet.index.params.CategoryListParams;
@ -150,14 +148,13 @@ public class TotalFacetCounts {
}
// needed because FacetSearchParams do not allow empty FacetRequests
private static final List<FacetRequest> DUMMY_REQ = Arrays.asList(
new FacetRequest[] { new CountFacetRequest(CategoryPath.EMPTY, 1) });
private static final FacetRequest DUMMY_REQ = new CountFacetRequest(CategoryPath.EMPTY, 1);
static TotalFacetCounts compute(final IndexReader indexReader, final TaxonomyReader taxonomy,
final FacetIndexingParams facetIndexingParams) throws IOException {
int partitionSize = PartitionsUtils.partitionSize(facetIndexingParams, taxonomy);
final int[][] counts = new int[(int) Math.ceil(taxonomy.getSize() /(float) partitionSize)][partitionSize];
FacetSearchParams newSearchParams = new FacetSearchParams(DUMMY_REQ, facetIndexingParams);
FacetSearchParams newSearchParams = new FacetSearchParams(facetIndexingParams, DUMMY_REQ);
//createAllListsSearchParams(facetIndexingParams, this.totalCounts);
FacetsAccumulator fe = new StandardFacetsAccumulator(newSearchParams, indexReader, taxonomy) {
@Override

View File

@ -35,8 +35,8 @@ import org.apache.lucene.facet.index.params.FacetIndexingParams;
*/
public class FacetSearchParams {
protected final FacetIndexingParams indexingParams;
protected final List<FacetRequest> facetRequests;
public final FacetIndexingParams indexingParams;
public final List<FacetRequest> facetRequests;
/**
* Initializes with the given {@link FacetRequest requests} and default
@ -57,6 +57,14 @@ public class FacetSearchParams {
public FacetSearchParams(List<FacetRequest> facetRequests) {
this(facetRequests, FacetIndexingParams.ALL_PARENTS);
}
/**
* Initilizes with the given {@link FacetRequest requests} and
* {@link FacetIndexingParams}.
*/
public FacetSearchParams(FacetIndexingParams indexingParams, FacetRequest... facetRequests) {
this(Arrays.asList(facetRequests), indexingParams);
}
/**
* Initilizes with the given {@link FacetRequest requests} and
@ -66,24 +74,8 @@ public class FacetSearchParams {
if (facetRequests == null || facetRequests.size() == 0) {
throw new IllegalArgumentException("at least one FacetRequest must be defined");
}
this.indexingParams = indexingParams;
this.facetRequests = facetRequests;
}
/**
* Returns the {@link FacetIndexingParams} that were passed to the
* constructor.
*/
public FacetIndexingParams getFacetIndexingParams() {
return indexingParams;
}
/**
* Returns the list of {@link FacetRequest facet requests} that were passed to
* the constructor.
*/
public List<FacetRequest> getFacetRequests() {
return facetRequests;
this.indexingParams = indexingParams;
}
@Override
@ -92,10 +84,10 @@ public class FacetSearchParams {
final char NEWLINE = '\n';
StringBuilder sb = new StringBuilder("IndexingParams: ");
sb.append(NEWLINE).append(TAB).append(getFacetIndexingParams());
sb.append(NEWLINE).append(TAB).append(indexingParams);
sb.append(NEWLINE).append("FacetRequests:");
for (FacetRequest facetRequest : getFacetRequests()) {
for (FacetRequest facetRequest : facetRequests) {
sb.append(NEWLINE).append(TAB).append(facetRequest);
}

View File

@ -210,11 +210,11 @@ public abstract class Sampler {
double overSampleFactor = getSamplingParams().getOversampleFactor();
if (overSampleFactor > 1) { // any factoring to do?
List<FacetRequest> facetRequests = new ArrayList<FacetRequest>();
for (FacetRequest frq : original.getFacetRequests()) {
for (FacetRequest frq : original.facetRequests) {
int overSampledNumResults = (int) Math.ceil(frq.getNumResults() * overSampleFactor);
facetRequests.add(new OverSampledFacetRequest(frq, overSampledNumResults));
}
res = new FacetSearchParams(facetRequests, original.getFacetIndexingParams());
res = new FacetSearchParams(facetRequests, original.indexingParams);
}
return res;
}

View File

@ -196,7 +196,7 @@ public abstract class FacetTestBase extends LuceneTestCase {
* test with different faceted search params.
*/
protected FacetSearchParams getFacetSearchParams(FacetIndexingParams iParams, FacetRequest... facetRequests) {
return new FacetSearchParams(Arrays.asList(facetRequests), iParams);
return new FacetSearchParams(iParams, facetRequests);
}
/**
@ -315,7 +315,7 @@ public abstract class FacetTestBase extends LuceneTestCase {
assertCountsAndCardinality(facetCountsTruth, subRes, reqNumResults); // recurse into child results
}
}
/** Validate results equality */
protected static void assertSameResults(List<FacetResult> expected, List<FacetResult> actual) {
assertEquals("wrong number of facet results", expected.size(), actual.size());

View File

@ -83,8 +83,7 @@ public class FacetTestUtils {
return dirs;
}
public static IndexTaxonomyReaderPair[] createIndexTaxonomyReaderPair(
Directory[][] dirs) throws IOException {
public static IndexTaxonomyReaderPair[] createIndexTaxonomyReaderPair(Directory[][] dirs) throws IOException {
IndexTaxonomyReaderPair[] pairs = new IndexTaxonomyReaderPair[dirs.length];
for (int i = 0; i < dirs.length; i++) {
IndexTaxonomyReaderPair pair = new IndexTaxonomyReaderPair();
@ -96,8 +95,7 @@ public class FacetTestUtils {
return pairs;
}
public static IndexTaxonomyWriterPair[] createIndexTaxonomyWriterPair(
Directory[][] dirs) throws IOException {
public static IndexTaxonomyWriterPair[] createIndexTaxonomyWriterPair(Directory[][] dirs) throws IOException {
IndexTaxonomyWriterPair[] pairs = new IndexTaxonomyWriterPair[dirs.length];
for (int i = 0; i < dirs.length; i++) {
IndexTaxonomyWriterPair pair = new IndexTaxonomyWriterPair();
@ -111,9 +109,8 @@ public class FacetTestUtils {
return pairs;
}
public static Collector[] search(IndexSearcher searcher,
TaxonomyReader taxonomyReader, FacetIndexingParams iParams, int k,
String... facetNames) throws IOException {
public static Collector[] search(IndexSearcher searcher, TaxonomyReader taxonomyReader, FacetIndexingParams iParams,
int k, String... facetNames) throws IOException {
Collector[] collectors = new Collector[2];
@ -125,10 +122,8 @@ public class FacetTestUtils {
}
FacetSearchParams facetSearchParams = new FacetSearchParams(fRequests, iParams);
TopScoreDocCollector topDocsCollector = TopScoreDocCollector.create(
searcher.getIndexReader().maxDoc(), true);
FacetsCollector facetsCollector = new FacetsCollector(
facetSearchParams, searcher.getIndexReader(), taxonomyReader);
TopScoreDocCollector topDocsCollector = TopScoreDocCollector.create(searcher.getIndexReader().maxDoc(), true);
FacetsCollector facetsCollector = FacetsCollector.create(facetSearchParams, searcher.getIndexReader(), taxonomyReader);
Collector mColl = MultiCollector.wrap(topDocsCollector, facetsCollector);
collectors[0] = topDocsCollector;

View File

@ -71,7 +71,7 @@ public class OrdinalMappingReaderTest extends LuceneTestCase {
DirectoryTaxonomyReader taxReader = new DirectoryTaxonomyReader(taxDir);
IndexSearcher searcher = newSearcher(reader1);
FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(new CategoryPath("tag"), NUM_DOCS));
FacetsCollector collector = new FacetsCollector(fsp, reader1, taxReader);
FacetsCollector collector = FacetsCollector.create(fsp, reader1, taxReader);
searcher.search(new MatchAllDocsQuery(), collector);
FacetResult result = collector.getFacetResults().get(0);
FacetResultNode node = result.getFacetResultNode();

View File

@ -266,7 +266,7 @@ public class TestFacetsPayloadMigrationReader extends LuceneTestCase {
requests.add(new CountFacetRequest(new CategoryPath(dim), 5));
}
FacetSearchParams fsp = new FacetSearchParams(requests, fip);
FacetsCollector fc = new FacetsCollector(fsp, indexReader, taxoReader);
FacetsCollector fc = FacetsCollector.create(fsp, indexReader, taxoReader);
MatchAllDocsQuery base = new MatchAllDocsQuery();
searcher.search(base, fc);
List<FacetResult> facetResults = fc.getFacetResults();
@ -283,12 +283,10 @@ public class TestFacetsPayloadMigrationReader extends LuceneTestCase {
// verify drill-down
for (String dim : expectedCounts.keySet()) {
CategoryPath drillDownCP = new CategoryPath(dim);
ArrayList<FacetRequest> request = new ArrayList<FacetRequest>(1);
request.add(new CountFacetRequest(drillDownCP, 10));
FacetSearchParams fsp = new FacetSearchParams(request, fip);
FacetSearchParams fsp = new FacetSearchParams(fip, new CountFacetRequest(drillDownCP, 10));
Query drillDown = DrillDown.query(fsp, new MatchAllDocsQuery(), drillDownCP);
TotalHitCountCollector total = new TotalHitCountCollector();
FacetsCollector fc = new FacetsCollector(fsp, indexReader, taxoReader);
FacetsCollector fc = FacetsCollector.create(fsp, indexReader, taxoReader);
searcher.search(drillDown, MultiCollector.wrap(fc, total));
assertTrue("no results for drill-down query " + drillDown, total.getTotalHits() > 0);
List<FacetResult> facetResults = fc.getFacetResults();

View File

@ -0,0 +1,515 @@
package org.apache.lucene.facet.search;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.StringField;
import org.apache.lucene.facet.index.FacetFields;
import org.apache.lucene.facet.index.categorypolicy.OrdinalPolicy;
import org.apache.lucene.facet.index.params.CategoryListParams;
import org.apache.lucene.facet.index.params.FacetIndexingParams;
import org.apache.lucene.facet.search.params.CountFacetRequest;
import org.apache.lucene.facet.search.params.FacetRequest;
import org.apache.lucene.facet.search.params.FacetRequest.SortBy;
import org.apache.lucene.facet.search.params.FacetRequest.SortOrder;
import org.apache.lucene.facet.search.params.FacetSearchParams;
import org.apache.lucene.facet.search.params.ScoreFacetRequest;
import org.apache.lucene.facet.search.results.FacetResult;
import org.apache.lucene.facet.search.results.FacetResultNode;
import org.apache.lucene.facet.taxonomy.CategoryPath;
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.NoMergePolicy;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.collections.ObjectToIntMap;
import org.apache.lucene.util.encoding.IntEncoder;
import org.apache.lucene.util.encoding.VInt8IntEncoder;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
/*
* 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.
*/
public class CountingFacetsCollectorTest extends LuceneTestCase {
private static final Term A = new Term("f", "a");
private static final CategoryPath CP_A = new CategoryPath("A"), CP_B = new CategoryPath("B");
private static final int NUM_CHILDREN_CP_A = 5, NUM_CHILDREN_CP_B = 3;
private static final CategoryPath[] CATEGORIES_A, CATEGORIES_B;
static {
CATEGORIES_A = new CategoryPath[NUM_CHILDREN_CP_A];
for (int i = 0; i < NUM_CHILDREN_CP_A; i++) {
CATEGORIES_A[i] = new CategoryPath(CP_A.components[0], Integer.toString(i));
}
CATEGORIES_B = new CategoryPath[NUM_CHILDREN_CP_B];
for (int i = 0; i < NUM_CHILDREN_CP_B; i++) {
CATEGORIES_B[i] = new CategoryPath(CP_B.components[0], Integer.toString(i));
}
}
protected static Directory indexDir, taxoDir;
protected static ObjectToIntMap<CategoryPath> allExpectedCounts, termExpectedCounts;
protected static int numChildrenIndexedA, numChildrenIndexedB;
@AfterClass
public static void afterClassCountingFacetsCollectorTest() throws Exception {
IOUtils.close(indexDir, taxoDir);
}
private static List<CategoryPath> randomCategories(Random random) {
// add random categories from the two dimensions, ensuring that the same
// category is not added twice.
int numFacetsA = random.nextInt(3) + 1; // 1-3
int numFacetsB = random.nextInt(2) + 1; // 1-2
ArrayList<CategoryPath> categories_a = new ArrayList<CategoryPath>();
categories_a.addAll(Arrays.asList(CATEGORIES_A));
ArrayList<CategoryPath> categories_b = new ArrayList<CategoryPath>();
categories_b.addAll(Arrays.asList(CATEGORIES_B));
Collections.shuffle(categories_a, random);
Collections.shuffle(categories_b, random);
ArrayList<CategoryPath> categories = new ArrayList<CategoryPath>();
categories.addAll(categories_a.subList(0, numFacetsA));
categories.addAll(categories_b.subList(0, numFacetsB));
return categories;
}
private static void addField(Document doc) {
doc.add(new StringField(A.field(), A.text(), Store.NO));
}
private static void addFacets(Document doc, FacetFields facetFields, boolean updateTermExpectedCounts)
throws IOException {
List<CategoryPath> docCategories = randomCategories(random());
for (CategoryPath cp : docCategories) {
allExpectedCounts.put(cp, allExpectedCounts.get(cp) + 1);
if (updateTermExpectedCounts) {
termExpectedCounts.put(cp, termExpectedCounts.get(cp) + 1);
}
}
// add 1 to each dimension
allExpectedCounts.put(CP_A, allExpectedCounts.get(CP_A) + 1);
allExpectedCounts.put(CP_B, allExpectedCounts.get(CP_B) + 1);
if (updateTermExpectedCounts) {
termExpectedCounts.put(CP_A, termExpectedCounts.get(CP_A) + 1);
termExpectedCounts.put(CP_B, termExpectedCounts.get(CP_B) + 1);
}
facetFields.addFields(doc, docCategories);
}
private static void indexDocsNoFacets(IndexWriter indexWriter) throws IOException {
int numDocs = atLeast(2);
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
addField(doc);
indexWriter.addDocument(doc);
}
indexWriter.commit(); // flush a segment
}
private static void indexDocsWithFacetsNoTerms(IndexWriter indexWriter, TaxonomyWriter taxoWriter,
ObjectToIntMap<CategoryPath> expectedCounts) throws IOException {
Random random = random();
int numDocs = atLeast(random, 2);
FacetFields facetFields = new FacetFields(taxoWriter);
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
addFacets(doc, facetFields, false);
indexWriter.addDocument(doc);
}
indexWriter.commit(); // flush a segment
}
private static void indexDocsWithFacetsAndTerms(IndexWriter indexWriter, TaxonomyWriter taxoWriter,
ObjectToIntMap<CategoryPath> expectedCounts) throws IOException {
Random random = random();
int numDocs = atLeast(random, 2);
FacetFields facetFields = new FacetFields(taxoWriter);
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
addFacets(doc, facetFields, true);
addField(doc);
indexWriter.addDocument(doc);
}
indexWriter.commit(); // flush a segment
}
private static void indexDocsWithFacetsAndSomeTerms(IndexWriter indexWriter, TaxonomyWriter taxoWriter,
ObjectToIntMap<CategoryPath> expectedCounts) throws IOException {
Random random = random();
int numDocs = atLeast(random, 2);
FacetFields facetFields = new FacetFields(taxoWriter);
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
boolean hasContent = random.nextBoolean();
if (hasContent) {
addField(doc);
}
addFacets(doc, facetFields, hasContent);
indexWriter.addDocument(doc);
}
indexWriter.commit(); // flush a segment
}
// initialize expectedCounts w/ 0 for all categories
private static ObjectToIntMap<CategoryPath> newCounts() {
ObjectToIntMap<CategoryPath> counts = new ObjectToIntMap<CategoryPath>();
counts.put(CP_A, 0);
counts.put(CP_B, 0);
for (CategoryPath cp : CATEGORIES_A) {
counts.put(cp, 0);
}
for (CategoryPath cp : CATEGORIES_B) {
counts.put(cp, 0);
}
return counts;
}
@BeforeClass
public static void beforeClassCountingFacetsCollectorTest() throws Exception {
indexDir = newDirectory();
taxoDir = newDirectory();
// create an index which has:
// 1. Segment with no categories, but matching results
// 2. Segment w/ categories, but no results
// 3. Segment w/ categories and results
// 4. Segment w/ categories, but only some results
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
conf.setMergePolicy(NoMergePolicy.COMPOUND_FILES); // prevent merges, so we can control the index segments
IndexWriter indexWriter = new IndexWriter(indexDir, conf);
TaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
allExpectedCounts = newCounts();
termExpectedCounts = newCounts();
// segment w/ no categories
indexDocsNoFacets(indexWriter);
// segment w/ categories, no content
indexDocsWithFacetsNoTerms(indexWriter, taxoWriter, allExpectedCounts);
// segment w/ categories and content
indexDocsWithFacetsAndTerms(indexWriter, taxoWriter, allExpectedCounts);
// segment w/ categories and some content
indexDocsWithFacetsAndSomeTerms(indexWriter, taxoWriter, allExpectedCounts);
// set num children indexed from each dimension
for (CategoryPath cp : CATEGORIES_A) {
if (termExpectedCounts.get(cp) > 0) {
++numChildrenIndexedA;
}
}
for (CategoryPath cp : CATEGORIES_B) {
if (termExpectedCounts.get(cp) > 0) {
++numChildrenIndexedB;
}
}
IOUtils.close(indexWriter, taxoWriter);
}
@Test
public void testInvalidValidParams() throws Exception {
final CategoryPath dummyCP = new CategoryPath("a");
final FacetRequest dummyFR = new CountFacetRequest(dummyCP, 10);
// only CountFacetRequests are allowed
assertNotNull("only CountFacetRequests should be allowed",
CountingFacetsCollector.assertParams(new FacetSearchParams(new ScoreFacetRequest(dummyCP, 10))));
// only depth=1
FacetRequest cfr = new CountFacetRequest(dummyCP, 10);
cfr.setDepth(2);
assertNotNull("only depth 1 should be allowed", CountingFacetsCollector.assertParams(new FacetSearchParams(cfr)));
// only SortOrder.DESCENDING
cfr = new CountFacetRequest(dummyCP, 10);
cfr.setSortOrder(SortOrder.ASCENDING);
assertNotNull("only SortOrder.DESCENDING should be allowed", CountingFacetsCollector.assertParams(new FacetSearchParams(cfr)));
// only SortBy.VALUE
cfr = new CountFacetRequest(dummyCP, 10);
cfr.setSortBy(SortBy.ORDINAL);
assertNotNull("only SortBy.VALUE should be allowed", CountingFacetsCollector.assertParams(new FacetSearchParams(cfr)));
// no numToLabel
cfr = new CountFacetRequest(dummyCP, 10);
cfr.setNumLabel(2);
assertNotNull("numToLabel should not be allowed", CountingFacetsCollector.assertParams(new FacetSearchParams(cfr)));
FacetIndexingParams fip = new FacetIndexingParams(new CategoryListParams("moo")) {
@Override
public List<CategoryListParams> getAllCategoryListParams() {
return Arrays.asList(new CategoryListParams[] { clParams, clParams });
}
};
assertNotNull("only one CLP should be allowed", CountingFacetsCollector.assertParams(new FacetSearchParams(fip, dummyFR)));
fip = new FacetIndexingParams(new CategoryListParams("moo")) {
final CategoryListParams clp = new CategoryListParams() {
@Override
public IntEncoder createEncoder() {
return new VInt8IntEncoder();
}
};
@Override
public List<CategoryListParams> getAllCategoryListParams() {
return Collections.singletonList(clp);
}
@Override
public CategoryListParams getCategoryListParams(CategoryPath category) {
return clp;
}
};
assertNotNull("only DGapVIntEncoder should be allowed", CountingFacetsCollector.assertParams(new FacetSearchParams(fip, dummyFR)));
fip = new FacetIndexingParams(new CategoryListParams("moo")) {
@Override
public int getPartitionSize() {
return 2;
}
};
assertNotNull("partitions should be allowed", CountingFacetsCollector.assertParams(new FacetSearchParams(fip, dummyFR)));
}
@Test
public void testDifferentNumResults() throws Exception {
// test the collector w/ FacetRequests and different numResults
DirectoryReader indexReader = DirectoryReader.open(indexDir);
TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
IndexSearcher searcher = new IndexSearcher(indexReader);
FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(CP_A, NUM_CHILDREN_CP_A),
new CountFacetRequest(CP_B, NUM_CHILDREN_CP_B));
FacetsCollector fc = new CountingFacetsCollector(fsp , taxoReader);
TermQuery q = new TermQuery(A);
searcher.search(q, fc);
List<FacetResult> facetResults = fc.getFacetResults();
assertEquals("invalid number of facet results", 2, facetResults.size());
for (FacetResult res : facetResults) {
FacetResultNode root = res.getFacetResultNode();
assertEquals("wrong count for " + root.label, termExpectedCounts.get(root.label), (int) root.value);
assertEquals("invalid residue", 0, (int) root.residue);
for (FacetResultNode child : root.subResults) {
assertEquals("wrong count for " + child.label, termExpectedCounts.get(child.label), (int) child.value);
}
}
IOUtils.close(indexReader, taxoReader);
}
@Test
public void testResidue() throws Exception {
// test the collector's handling of residue
DirectoryReader indexReader = DirectoryReader.open(indexDir);
TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
IndexSearcher searcher = new IndexSearcher(indexReader);
// asking for top 1 is the only way to guarantee there will be a residue
// provided that enough children were indexed (see below)
FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(CP_A, 1), new CountFacetRequest(CP_B, 1));
FacetsCollector fc = new CountingFacetsCollector(fsp , taxoReader);
TermQuery q = new TermQuery(A);
searcher.search(q, fc);
List<FacetResult> facetResults = fc.getFacetResults();
assertEquals("invalid number of facet results", 2, facetResults.size());
for (FacetResult res : facetResults) {
FacetResultNode root = res.getFacetResultNode();
assertEquals("wrong count for " + root.label, termExpectedCounts.get(root.label), (int) root.value);
// make sure randomness didn't pick only one child of root (otherwise there's no residue)
int numChildrenIndexed = res.getFacetRequest().categoryPath == CP_A ? numChildrenIndexedA : numChildrenIndexedB;
if (numChildrenIndexed > 1) {
assertTrue("expected residue", root.residue > 0);
}
for (FacetResultNode child : root.subResults) {
assertEquals("wrong count for " + child.label, termExpectedCounts.get(child.label), (int) child.value);
}
}
IOUtils.close(indexReader, taxoReader);
}
@Test
public void testAllCounts() throws Exception {
DirectoryReader indexReader = DirectoryReader.open(indexDir);
TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
IndexSearcher searcher = new IndexSearcher(indexReader);
FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(CP_A, NUM_CHILDREN_CP_A),
new CountFacetRequest(CP_B, NUM_CHILDREN_CP_B));
FacetsCollector fc = new CountingFacetsCollector(fsp , taxoReader);
searcher.search(new MatchAllDocsQuery(), fc);
List<FacetResult> facetResults = fc.getFacetResults();
assertEquals("invalid number of facet results", 2, facetResults.size());
for (FacetResult res : facetResults) {
FacetResultNode root = res.getFacetResultNode();
assertEquals("wrong count for " + root.label, allExpectedCounts.get(root.label), (int) root.value);
assertEquals("invalid residue", 0, (int) root.residue);
for (FacetResultNode child : root.subResults) {
assertEquals("wrong count for " + child.label, allExpectedCounts.get(child.label), (int) child.value);
}
}
IOUtils.close(indexReader, taxoReader);
}
@Test
public void testBigNumResults() throws Exception {
DirectoryReader indexReader = DirectoryReader.open(indexDir);
TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
IndexSearcher searcher = new IndexSearcher(indexReader);
FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(CP_A, Integer.MAX_VALUE),
new CountFacetRequest(CP_B, Integer.MAX_VALUE));
FacetsCollector fc = new CountingFacetsCollector(fsp , taxoReader);
searcher.search(new MatchAllDocsQuery(), fc);
List<FacetResult> facetResults = fc.getFacetResults();
assertEquals("invalid number of facet results", 2, facetResults.size());
for (FacetResult res : facetResults) {
FacetResultNode root = res.getFacetResultNode();
assertEquals("wrong count for " + root.label, allExpectedCounts.get(root.label), (int) root.value);
assertEquals("invalid residue", 0, (int) root.residue);
for (FacetResultNode child : root.subResults) {
assertEquals("wrong count for " + child.label, allExpectedCounts.get(child.label), (int) child.value);
}
}
IOUtils.close(indexReader, taxoReader);
}
@Test
public void testDirectSource() throws Exception {
DirectoryReader indexReader = DirectoryReader.open(indexDir);
TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
IndexSearcher searcher = new IndexSearcher(indexReader);
FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(CP_A, NUM_CHILDREN_CP_A),
new CountFacetRequest(CP_B, NUM_CHILDREN_CP_B));
FacetsCollector fc = new CountingFacetsCollector(fsp , taxoReader, new FacetArrays(taxoReader.getSize()), true);
searcher.search(new MatchAllDocsQuery(), fc);
List<FacetResult> facetResults = fc.getFacetResults();
assertEquals("invalid number of facet results", 2, facetResults.size());
for (FacetResult res : facetResults) {
FacetResultNode root = res.getFacetResultNode();
assertEquals("wrong count for " + root.label, allExpectedCounts.get(root.label), (int) root.value);
assertEquals("invalid residue", 0, (int) root.residue);
for (FacetResultNode child : root.subResults) {
assertEquals("wrong count for " + child.label, allExpectedCounts.get(child.label), (int) child.value);
}
}
IOUtils.close(indexReader, taxoReader);
}
@Test
public void testNoParents() throws Exception {
// TODO: when OrdinalPolicy is on CLP, index the NO_PARENTS categories into
// their own dimension, and avoid this index creation
Directory indexDir = newDirectory();
Directory taxoDir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
conf.setMaxBufferedDocs(2);
conf.setMergePolicy(NoMergePolicy.COMPOUND_FILES);
IndexWriter indexWriter = new IndexWriter(indexDir, conf);
TaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
FacetIndexingParams fip = new FacetIndexingParams() {
@Override
public OrdinalPolicy getOrdinalPolicy() {
return OrdinalPolicy.NO_PARENTS;
}
};
FacetFields facetFields = new FacetFields(taxoWriter, fip);
ObjectToIntMap<CategoryPath> expCounts = newCounts();
// index few docs with categories, not sharing parents.
int numDocs = atLeast(10);
final CategoryPath cpc = new CategoryPath("L1", "L2", "L3");
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
ArrayList<CategoryPath> categories = new ArrayList<CategoryPath>();
CategoryPath cpa = CATEGORIES_A[random().nextInt(NUM_CHILDREN_CP_A)];
CategoryPath cpb = CATEGORIES_B[random().nextInt(NUM_CHILDREN_CP_B)];
categories.add(cpa);
categories.add(cpb);
categories.add(cpc);
expCounts.put(cpa, expCounts.get(cpa) + 1);
expCounts.put(cpb, expCounts.get(cpb) + 1);
facetFields.addFields(doc, categories);
indexWriter.addDocument(doc);
}
expCounts.put(CP_A, numDocs);
expCounts.put(CP_B, numDocs);
for (int i = 0; i < cpc.length; i++) {
expCounts.put(cpc.subpath(i+1), numDocs);
}
IOUtils.close(indexWriter, taxoWriter);
DirectoryReader indexReader = DirectoryReader.open(indexDir);
TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
IndexSearcher searcher = new IndexSearcher(indexReader);
FacetSearchParams fsp = new FacetSearchParams(fip, new CountFacetRequest(CP_A, NUM_CHILDREN_CP_A),
new CountFacetRequest(CP_B, NUM_CHILDREN_CP_B), new CountFacetRequest(cpc.subpath(1), 10));
FacetsCollector fc = new CountingFacetsCollector(fsp , taxoReader);
searcher.search(new MatchAllDocsQuery(), fc);
List<FacetResult> facetResults = fc.getFacetResults();
assertEquals("invalid number of facet results", 3, facetResults.size());
for (FacetResult res : facetResults) {
FacetResultNode root = res.getFacetResultNode();
assertEquals("wrong count for " + root.label, expCounts.get(root.label), (int) root.value);
assertEquals("invalid residue", 0, (int) root.residue);
for (FacetResultNode child : root.subResults) {
assertEquals("wrong count for " + child.label, expCounts.get(child.label), (int) child.value);
}
}
IOUtils.close(indexReader, taxoReader);
IOUtils.close(indexDir, taxoDir);
}
}

View File

@ -29,11 +29,10 @@ import org.apache.lucene.facet.taxonomy.TaxonomyReader;
public class SamplingWrapperTest extends BaseSampleTestTopK {
@Override
protected FacetsAccumulator getSamplingAccumulator(Sampler sampler,
TaxonomyReader taxoReader, IndexReader indexReader,
FacetSearchParams searchParams) {
FacetsAccumulator fExtrctr = new StandardFacetsAccumulator(searchParams,
indexReader, taxoReader);
return new SamplingWrapper(fExtrctr, sampler);
protected FacetsAccumulator getSamplingAccumulator(Sampler sampler, TaxonomyReader taxoReader,
IndexReader indexReader, FacetSearchParams searchParams) {
FacetsAccumulator fa = new StandardFacetsAccumulator(searchParams, indexReader, taxoReader);
return new SamplingWrapper(fa, sampler);
}
}

View File

@ -22,11 +22,11 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.document.Document;
import org.apache.lucene.facet.FacetTestUtils;
import org.apache.lucene.facet.index.FacetFields;
import org.apache.lucene.facet.search.params.CountFacetRequest;
import org.apache.lucene.facet.search.params.FacetSearchParams;
import org.apache.lucene.facet.search.results.FacetResult;
import org.apache.lucene.facet.search.results.FacetResultNode;
import org.apache.lucene.facet.taxonomy.CategoryPath;
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
@ -89,7 +89,7 @@ public class TestDemoFacets extends LuceneTestCase {
new CountFacetRequest(new CategoryPath("Author"), 10));
// Aggregatses the facet counts:
FacetsCollector c = new FacetsCollector(fsp, searcher.getIndexReader(), taxoReader);
FacetsCollector c = FacetsCollector.create(fsp, searcher.getIndexReader(), taxoReader);
// MatchAllDocsQuery is for "browsing" (counts facets
// for all non-deleted docs in the index); normally
@ -101,20 +101,20 @@ public class TestDemoFacets extends LuceneTestCase {
List<FacetResult> results = c.getFacetResults();
assertEquals(2, results.size());
assertEquals("Publish Date (5)\n 2012 (2)\n 2010 (2)\n 1999 (1)\n",
toSimpleString(results.get(0)));
FacetTestUtils.toSimpleString(results.get(0)));
assertEquals("Author (5)\n Lisa (2)\n Frank (1)\n Susan (1)\n Bob (1)\n",
toSimpleString(results.get(1)));
FacetTestUtils.toSimpleString(results.get(1)));
// Now user drills down on Publish Date/2010:
fsp = new FacetSearchParams(new CountFacetRequest(new CategoryPath("Author"), 10));
Query q2 = DrillDown.query(fsp, new MatchAllDocsQuery(), new CategoryPath("Publish Date/2010", '/'));
c = new FacetsCollector(fsp, searcher.getIndexReader(), taxoReader);
c = FacetsCollector.create(fsp, searcher.getIndexReader(), taxoReader);
searcher.search(q2, c);
results = c.getFacetResults();
assertEquals(1, results.size());
assertEquals("Author (2)\n Lisa (1)\n Bob (1)\n",
toSimpleString(results.get(0)));
FacetTestUtils.toSimpleString(results.get(0)));
taxoReader.close();
searcher.getIndexReader().close();
@ -122,16 +122,4 @@ public class TestDemoFacets extends LuceneTestCase {
taxoDir.close();
}
private String toSimpleString(FacetResult fr) {
StringBuilder sb = new StringBuilder();
toSimpleString(0, sb, fr.getFacetResultNode(), "");
return sb.toString();
}
private void toSimpleString(int depth, StringBuilder sb, FacetResultNode node, String indent) {
sb.append(indent + node.label.components[depth] + " (" + (int) node.value + ")\n");
for(FacetResultNode childNode : node.subResults) {
toSimpleString(depth+1, sb, childNode, indent + " ");
}
}
}

View File

@ -71,7 +71,7 @@ public class TestFacetsCollector extends LuceneTestCase {
DirectoryReader r = DirectoryReader.open(indexDir);
DirectoryTaxonomyReader taxo = new DirectoryTaxonomyReader(taxoDir);
FacetsCollector fc = new FacetsCollector(sParams, r, taxo);
FacetsCollector fc = FacetsCollector.create(sParams, r, taxo);
TopScoreDocCollector topDocs = TopScoreDocCollector.create(10, false);
new IndexSearcher(r).search(new MatchAllDocsQuery(), MultiCollector.wrap(fc, topDocs));

View File

@ -334,7 +334,7 @@ public class TestMultipleCategoryLists extends LuceneTestCase {
FacetSearchParams facetSearchParams = new FacetSearchParams(facetRequests, iParams);
// perform documents search and facets accumulation
FacetsCollector facetsCollector = new FacetsCollector(facetSearchParams, ir, tr);
FacetsCollector facetsCollector = FacetsCollector.create(facetSearchParams, ir, tr);
searcher.search(q, MultiCollector.wrap(topDocsCollector, facetsCollector));
return facetsCollector;
}

View File

@ -44,7 +44,7 @@ public class TestSameRequestAccumulation extends FacetTestBase {
final CountFacetRequest facetRequest = new CountFacetRequest(new CategoryPath("root"), 10);
FacetSearchParams fsp = new FacetSearchParams(facetRequest);
FacetsCollector fc = new FacetsCollector(fsp, indexReader, taxoReader);
FacetsCollector fc = FacetsCollector.create(fsp, indexReader, taxoReader);
searcher.search(new MatchAllDocsQuery(), fc);
final String expected = fc.getFacetResults().get(0).toString();
@ -53,9 +53,9 @@ public class TestSameRequestAccumulation extends FacetTestBase {
fsp = new FacetSearchParams(facetRequest, facetRequest, new CountFacetRequest(new CategoryPath("root"), 10));
// make sure the search params holds 3 requests now
assertEquals(3, fsp.getFacetRequests().size());
assertEquals(3, fsp.facetRequests.size());
fc = new FacetsCollector(fsp, indexReader, taxoReader);
fc = FacetsCollector.create(fsp, indexReader, taxoReader);
searcher.search(new MatchAllDocsQuery(), fc);
List<FacetResult> actual = fc.getFacetResults();

View File

@ -1,7 +1,6 @@
package org.apache.lucene.facet.search;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
@ -102,16 +101,14 @@ public class TestStandardFacetsAccumulator extends LuceneTestCase {
// search for "f:a", only segments 1 and 3 should match results
Query q = new TermQuery(new Term("f", "a"));
ArrayList<FacetRequest> requests = new ArrayList<FacetRequest>(1);
CountFacetRequest countNoComplements = new CountFacetRequest(new CategoryPath("A"), 10) {
FacetRequest countNoComplements = new CountFacetRequest(new CategoryPath("A"), 10) {
@Override
public boolean supportsComplements() {
return false; // disable complements
}
};
requests.add(countNoComplements);
FacetSearchParams fsp = new FacetSearchParams(requests, fip);
FacetsCollector fc = new FacetsCollector(fsp , indexReader, taxoReader);
FacetSearchParams fsp = new FacetSearchParams(fip, countNoComplements);
FacetsCollector fc = new StandardFacetsCollector(fsp , indexReader, taxoReader);
indexSearcher.search(q, fc);
List<FacetResult> results = fc.getFacetResults();
assertEquals("received too many facet results", 1, results.size());

View File

@ -165,7 +165,7 @@ public class TestTopKInEachNodeResultHandler extends LuceneTestCase {
facetRequests.add(cfrb20);
FacetSearchParams facetSearchParams = new FacetSearchParams(facetRequests, iParams);
FacetArrays facetArrays = new FacetArrays(PartitionsUtils.partitionSize(facetSearchParams.getFacetIndexingParams(), tr));
FacetArrays facetArrays = new FacetArrays(PartitionsUtils.partitionSize(facetSearchParams.indexingParams, tr));
FacetsAccumulator fctExtrctr = new StandardFacetsAccumulator(facetSearchParams, is.getIndexReader(), tr, facetArrays);
fctExtrctr.setComplementThreshold(FacetsAccumulator.DISABLE_COMPLEMENT);
long start = System.currentTimeMillis();

View File

@ -89,7 +89,7 @@ public class TestTopKResultsHandler extends BaseTestTopK {
// do different facet counts and compare to control
FacetSearchParams sParams = getFacetSearchParams(facetRequests, getFacetIndexingParams(partitionSize));
FacetsCollector fc = new FacetsCollector(sParams, indexReader, taxoReader) {
FacetsCollector fc = new StandardFacetsCollector(sParams, indexReader, taxoReader) {
@Override
protected FacetsAccumulator initFacetsAccumulator(FacetSearchParams facetSearchParams, IndexReader indexReader, TaxonomyReader taxonomyReader) {
FacetsAccumulator fa = new StandardFacetsAccumulator(facetSearchParams, indexReader, taxonomyReader);
@ -153,10 +153,10 @@ public class TestTopKResultsHandler extends BaseTestTopK {
// do different facet counts and compare to control
CategoryPath path = new CategoryPath("a", "b");
FacetSearchParams sParams = getFacetSearchParams(
getFacetIndexingParams(partitionSize), new CountFacetRequest(path, Integer.MAX_VALUE));
FacetSearchParams sParams = getFacetSearchParams(getFacetIndexingParams(partitionSize),
new CountFacetRequest(path, Integer.MAX_VALUE));
FacetsCollector fc = new FacetsCollector(sParams, indexReader, taxoReader) {
FacetsCollector fc = new StandardFacetsCollector(sParams, indexReader, taxoReader) {
@Override
protected FacetsAccumulator initFacetsAccumulator(FacetSearchParams facetSearchParams, IndexReader indexReader, TaxonomyReader taxonomyReader) {
FacetsAccumulator fa = new StandardFacetsAccumulator(facetSearchParams, indexReader, taxonomyReader);
@ -176,7 +176,7 @@ public class TestTopKResultsHandler extends BaseTestTopK {
FacetSearchParams sParams2 = getFacetSearchParams(
getFacetIndexingParams(partitionSize), new CountFacetRequest(path, Integer.MAX_VALUE));
FacetsCollector fc2 = new FacetsCollector(sParams2, indexReader, taxoReader) {
FacetsCollector fc2 = new StandardFacetsCollector(sParams2, indexReader, taxoReader) {
@Override
protected FacetsAccumulator initFacetsAccumulator(FacetSearchParams facetSearchParams, IndexReader indexReader, TaxonomyReader taxonomyReader) {
FacetsAccumulator fa = new StandardFacetsAccumulator(facetSearchParams, indexReader, taxonomyReader);
@ -214,7 +214,7 @@ public class TestTopKResultsHandler extends BaseTestTopK {
getFacetIndexingParams(partitionSize),
new CountFacetRequest(path, 10));
FacetsCollector fc = new FacetsCollector(sParams, indexReader, taxoReader);
FacetsCollector fc = FacetsCollector.create(sParams, indexReader, taxoReader);
searcher.search(new MatchAllDocsQuery(), fc);

View File

@ -4,15 +4,14 @@ import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.junit.Test;
import org.apache.lucene.facet.search.params.FacetSearchParams;
import org.apache.lucene.facet.search.results.FacetResult;
import org.apache.lucene.facet.search.results.FacetResultNode;
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.junit.Test;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -37,7 +36,7 @@ public class TestTopKResultsHandlerRandom extends BaseTestTopK {
throws IOException {
Query q = new MatchAllDocsQuery();
FacetSearchParams facetSearchParams = searchParamsWithRequests(numResults, partitionSize);
FacetsCollector fc = new FacetsCollector(facetSearchParams, indexReader, taxoReader) {
FacetsCollector fc = new StandardFacetsCollector(facetSearchParams, indexReader, taxoReader) {
@Override
protected FacetsAccumulator initFacetsAccumulator(
FacetSearchParams facetSearchParams, IndexReader indexReader,

View File

@ -110,7 +110,7 @@ public class AssociationsFacetRequestTest extends LuceneTestCase {
Query q = new MatchAllDocsQuery();
FacetsCollector fc = new FacetsCollector(fsp, reader, taxo);
FacetsCollector fc = FacetsCollector.create(fsp, reader, taxo);
IndexSearcher searcher = newSearcher(reader);
searcher.search(q, fc);
@ -135,7 +135,7 @@ public class AssociationsFacetRequestTest extends LuceneTestCase {
Query q = new MatchAllDocsQuery();
FacetsCollector fc = new FacetsCollector(fsp, reader, taxo);
FacetsCollector fc = FacetsCollector.create(fsp, reader, taxo);
IndexSearcher searcher = newSearcher(reader);
searcher.search(q, fc);
@ -165,7 +165,7 @@ public class AssociationsFacetRequestTest extends LuceneTestCase {
Query q = new MatchAllDocsQuery();
FacetsCollector fc = new FacetsCollector(fsp, reader, taxo);
FacetsCollector fc = FacetsCollector.create(fsp, reader, taxo);
IndexSearcher searcher = newSearcher(reader);
searcher.search(q, fc);

View File

@ -3,22 +3,22 @@ package org.apache.lucene.facet.search.sampling;
import java.util.List;
import java.util.Random;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.MultiCollector;
import org.apache.lucene.facet.search.BaseTestTopK;
import org.apache.lucene.facet.search.FacetsAccumulator;
import org.apache.lucene.facet.search.FacetsCollector;
import org.apache.lucene.facet.search.ScoredDocIDs;
import org.apache.lucene.facet.search.ScoredDocIdCollector;
import org.apache.lucene.facet.search.StandardFacetsCollector;
import org.apache.lucene.facet.search.params.FacetRequest;
import org.apache.lucene.facet.search.params.FacetRequest.ResultMode;
import org.apache.lucene.facet.search.params.FacetSearchParams;
import org.apache.lucene.facet.search.results.FacetResult;
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.MultiCollector;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -48,7 +48,7 @@ public abstract class BaseSampleTestTopK extends BaseTestTopK {
@Override
protected FacetSearchParams searchParamsWithRequests(int numResults, int partitionSize) {
FacetSearchParams res = super.searchParamsWithRequests(numResults, partitionSize);
for (FacetRequest req : res.getFacetRequests()) {
for (FacetRequest req : res.facetRequests) {
// randomize the way we aggregate results
if (random().nextBoolean()) {
req.setResultMode(ResultMode.GLOBAL_FLAT);
@ -78,7 +78,7 @@ public abstract class BaseSampleTestTopK extends BaseTestTopK {
ScoredDocIdCollector docCollector = ScoredDocIdCollector.create(indexReader.maxDoc(), false);
FacetSearchParams expectedSearchParams = searchParamsWithRequests(K, partitionSize);
FacetsCollector fc = new FacetsCollector(expectedSearchParams, indexReader, taxoReader);
FacetsCollector fc = FacetsCollector.create(expectedSearchParams, indexReader, taxoReader);
searcher.search(q, MultiCollector.wrap(docCollector, fc));
@ -120,7 +120,7 @@ public abstract class BaseSampleTestTopK extends BaseTestTopK {
private FacetsCollector samplingCollector(final boolean complement, final Sampler sampler,
FacetSearchParams samplingSearchParams) {
FacetsCollector samplingFC = new FacetsCollector(samplingSearchParams, indexReader, taxoReader) {
FacetsCollector samplingFC = new StandardFacetsCollector(samplingSearchParams, indexReader, taxoReader) {
@Override
protected FacetsAccumulator initFacetsAccumulator(FacetSearchParams facetSearchParams, IndexReader indexReader,
TaxonomyReader taxonomyReader) {

View File

@ -8,6 +8,7 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.facet.index.FacetFields;
import org.apache.lucene.facet.search.FacetsAccumulator;
import org.apache.lucene.facet.search.FacetsCollector;
import org.apache.lucene.facet.search.StandardFacetsCollector;
import org.apache.lucene.facet.search.params.CountFacetRequest;
import org.apache.lucene.facet.search.params.FacetRequest;
import org.apache.lucene.facet.search.params.FacetRequest.ResultMode;
@ -109,11 +110,10 @@ public class OversampleWithDepthTest extends LuceneTestCase {
}
/** search reader <code>r</code>*/
private FacetResult searchWithFacets(IndexReader r,
TaxonomyReader tr, FacetSearchParams fsp, final SamplingParams params)
throws IOException {
private FacetResult searchWithFacets(IndexReader r, TaxonomyReader tr, FacetSearchParams fsp,
final SamplingParams params) throws IOException {
// a FacetsCollector with a sampling accumulator
FacetsCollector fcWithSampling = new FacetsCollector(fsp, r, tr) {
FacetsCollector fcWithSampling = new StandardFacetsCollector(fsp, r, tr) {
@Override
protected FacetsAccumulator initFacetsAccumulator(FacetSearchParams facetSearchParams, IndexReader indexReader,
TaxonomyReader taxonomyReader) {