LUCENE-3483: Move Function grouping collectors from Solr to grouping module.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1179808 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Martijn van Groningen 2011-10-06 20:09:36 +00:00
parent 026f36a734
commit 70a7363fdb
25 changed files with 757 additions and 279 deletions

View File

@ -12,5 +12,6 @@
<orderEntry type="sourceFolder" forTests="false" />
<orderEntry type="library" scope="TEST" name="JUnit" level="project" />
<orderEntry type="module" module-name="lucene" />
<orderEntry type="module" module-name="queries" />
</component>
</module>

View File

@ -30,6 +30,7 @@ import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.packed.Direct64;
import org.apache.lucene.util.packed.PackedInts;
// Stores variable-length byte[] by deref, ie when two docs

View File

@ -25,9 +25,9 @@
<fileset dir="analysis" includes="build.xml" />
<fileset dir="benchmark" includes="build.xml" />
<fileset dir="facet" includes="build.xml" />
<fileset dir="queries" includes="build.xml" />
<fileset dir="grouping" includes="build.xml" />
<fileset dir="join" includes="build.xml" />
<fileset dir="queries" includes="build.xml" />
<fileset dir="queryparser" includes="build.xml" />
<fileset dir="suggest" includes="build.xml" />
</subant>
@ -40,9 +40,9 @@
<fileset dir="analysis" includes="build.xml" />
<fileset dir="benchmark" includes="build.xml" />
<fileset dir="facet" includes="build.xml" />
<fileset dir="queries" includes="build.xml" />
<fileset dir="grouping" includes="build.xml" />
<fileset dir="join" includes="build.xml" />
<fileset dir="queries" includes="build.xml" />
<fileset dir="queryparser" includes="build.xml" />
<fileset dir="suggest" includes="build.xml" />
</subant>
@ -55,9 +55,9 @@
<fileset dir="analysis" includes="build.xml" />
<fileset dir="benchmark" includes="build.xml" />
<fileset dir="facet" includes="build.xml" />
<fileset dir="queries" includes="build.xml" />
<fileset dir="grouping" includes="build.xml" />
<fileset dir="join" includes="build.xml" />
<fileset dir="queries" includes="build.xml" />
<fileset dir="queryparser" includes="build.xml" />
<fileset dir="suggest" includes="build.xml" />
</subant>
@ -70,9 +70,9 @@
<fileset dir="analysis" includes="build.xml" />
<fileset dir="benchmark" includes="build.xml" />
<fileset dir="facet" includes="build.xml" />
<fileset dir="queries" includes="build.xml" />
<fileset dir="grouping" includes="build.xml" />
<fileset dir="join" includes="build.xml" />
<fileset dir="queries" includes="build.xml" />
<fileset dir="queryparser" includes="build.xml" />
<fileset dir="suggest" includes="build.xml" />
</subant>
@ -86,9 +86,9 @@
<fileset dir="analysis" includes="build.xml" />
<fileset dir="benchmark" includes="build.xml" />
<fileset dir="facet" includes="build.xml" />
<fileset dir="queries" includes="build.xml" />
<fileset dir="grouping" includes="build.xml" />
<fileset dir="join" includes="build.xml" />
<fileset dir="queries" includes="build.xml" />
<fileset dir="queryparser" includes="build.xml" />
<fileset dir="suggest" includes="build.xml" />
</subant>
@ -100,9 +100,9 @@
<fileset dir="analysis" includes="build.xml" />
<fileset dir="benchmark" includes="build.xml" />
<fileset dir="facet" includes="build.xml" />
<fileset dir="queries" includes="build.xml" />
<fileset dir="grouping" includes="build.xml" />
<fileset dir="join" includes="build.xml" />
<fileset dir="queries" includes="build.xml" />
<fileset dir="queryparser" includes="build.xml" />
<fileset dir="suggest" includes="build.xml" />
</subant>
@ -116,9 +116,9 @@
<fileset dir="analysis" includes="build.xml" />
<fileset dir="benchmark" includes="build.xml" />
<fileset dir="facet" includes="build.xml" />
<fileset dir="queries" includes="build.xml" />
<fileset dir="grouping" includes="build.xml" />
<fileset dir="join" includes="build.xml" />
<fileset dir="queries" includes="build.xml" />
<fileset dir="queryparser" includes="build.xml" />
<fileset dir="suggest" includes="build.xml" />
</subant>

View File

@ -9,3 +9,8 @@ Optimizations
LUCENE-3468: Replaced last() and remove() with pollLast() in
FirstPassGroupingCollector (Martijn van Groningen)
API Changes
LUCENE-3483: Move Function grouping collectors from Solr to
grouping module. (Martijn van Groningen)

View File

@ -18,8 +18,9 @@
-->
<project name="grouping" default="default">
<description>
Collectors for grouping search results
Grouping module. Collectors for grouping search results
</description>
<property name="build.dir" location="build/" />
@ -29,5 +30,18 @@
<import file="../../lucene/contrib/contrib-build.xml"/>
<property name="working.dir" location="work"/>
<path id="test.classpath">
<path refid="test.base.classpath" />
<pathelement path="${queries.jar}" />
</path>
<path id="classpath">
<pathelement path="${queries.jar}" />
<path refid="base.classpath"/>
</path>
<target name="dist-maven" depends="jar-core,javadocs,contrib-build.dist-maven" />
<target name="compile" depends="jar-queries,common.compile-core" description="Compiles facet classes" />
<target name="jar-core" depends="common.jar-core" />
</project>

View File

@ -49,7 +49,7 @@ import org.apache.lucene.util.PriorityQueue;
* being that the documents in each group must always be
* indexed as a block. This collector also fills in
* TopGroups.totalGroupCount without requiring the separate
* {@link TermAllGroupsCollector}. However, this collector does
* {@link org.apache.lucene.search.grouping.term.TermAllGroupsCollector}. However, this collector does
* not fill in the groupValue of each group; this field
* will always be null.
*

View File

@ -19,8 +19,12 @@ package org.apache.lucene.search.grouping;
import java.util.Arrays;
/** A native int set where one value is reserved to mean "EMPTY" */
class SentinelIntSet {
/**
* A native int set where one value is reserved to mean "EMPTY"
*
* @lucene.internal
*/
public class SentinelIntSet {
public int[] keys;
public int count;
public final int emptyVal;

View File

@ -0,0 +1,147 @@
package org.apache.lucene.search.grouping.function;
/*
* 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.
*/
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.queries.function.DocValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.grouping.AbstractAllGroupHeadsCollector;
import org.apache.lucene.util.mutable.MutableValue;
import java.io.IOException;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
/**
* An implementation of {@link AbstractAllGroupHeadsCollector} for retrieving the most relevant groups when grouping
* by {@link ValueSource}.
*
* @lucene.experimental
*/
public class FunctionAllGroupHeadsCollector extends AbstractAllGroupHeadsCollector<FunctionAllGroupHeadsCollector.GroupHead> {
private final ValueSource groupBy;
private final Map vsContext;
private final Map<MutableValue, GroupHead> groups;
private final Sort sortWithinGroup;
private DocValues.ValueFiller filler;
private MutableValue mval;
private IndexReader.AtomicReaderContext readerContext;
private Scorer scorer;
/**
* Constructs a {@link FunctionAllGroupHeadsCollector} instance.
*
* @param groupBy The {@link ValueSource} to group by
* @param vsContext The ValueSource context
* @param sortWithinGroup The sort within a group
*/
public FunctionAllGroupHeadsCollector(ValueSource groupBy, Map vsContext, Sort sortWithinGroup) {
super(sortWithinGroup.getSort().length);
groups = new HashMap<MutableValue, GroupHead>();
this.sortWithinGroup = sortWithinGroup;
this.groupBy = groupBy;
this.vsContext = vsContext;
final SortField[] sortFields = sortWithinGroup.getSort();
for (int i = 0; i < sortFields.length; i++) {
reversed[i] = sortFields[i].getReverse() ? -1 : 1;
}
}
/**
* {@inheritDoc}
*/
protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
filler.fillValue(doc);
GroupHead groupHead = groups.get(mval);
if (groupHead == null) {
MutableValue groupValue = mval.duplicate();
groupHead = new GroupHead(groupValue, sortWithinGroup, doc);
groups.put(groupValue, groupHead);
temporalResult.stop = true;
} else {
temporalResult.stop = false;
}
this.temporalResult.groupHead = groupHead;
}
/**
* {@inheritDoc}
*/
protected Collection<GroupHead> getCollectedGroupHeads() {
return groups.values();
}
public void setScorer(Scorer scorer) throws IOException {
this.scorer = scorer;
for (GroupHead groupHead : groups.values()) {
for (FieldComparator comparator : groupHead.comparators) {
comparator.setScorer(scorer);
}
}
}
public void setNextReader(IndexReader.AtomicReaderContext context) throws IOException {
this.readerContext = context;
DocValues docValues = groupBy.getValues(vsContext, context);
filler = docValues.getValueFiller();
mval = filler.getValue();
for (GroupHead groupHead : groups.values()) {
for (int i = 0; i < groupHead.comparators.length; i++) {
groupHead.comparators[i] = groupHead.comparators[i].setNextReader(context);
}
}
}
class GroupHead extends AbstractAllGroupHeadsCollector.GroupHead<MutableValue> {
final FieldComparator[] comparators;
private GroupHead(MutableValue groupValue, Sort sort, int doc) throws IOException {
super(groupValue, doc + readerContext.docBase);
final SortField[] sortFields = sort.getSort();
comparators = new FieldComparator[sortFields.length];
for (int i = 0; i < sortFields.length; i++) {
comparators[i] = sortFields[i].getComparator(1, i).setNextReader(readerContext);
comparators[i].setScorer(scorer);
comparators[i].copy(0, doc);
comparators[i].setBottom(0);
}
}
public int compare(int compIDX, int doc) throws IOException {
return comparators[compIDX].compareBottom(doc);
}
public void updateDocHead(int doc) throws IOException {
for (FieldComparator comparator : comparators) {
comparator.copy(0, doc);
comparator.setBottom(0);
}
this.doc = doc + readerContext.docBase;
}
}
}

View File

@ -0,0 +1,86 @@
package org.apache.lucene.search.grouping.function;
/*
* 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.
*/
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.queries.function.DocValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.grouping.AbstractAllGroupsCollector;
import org.apache.lucene.util.mutable.MutableValue;
import java.io.IOException;
import java.util.Collection;
import java.util.Map;
import java.util.SortedSet;
import java.util.TreeSet;
/**
* A collector that collects all groups that match the
* query. Only the group value is collected, and the order
* is undefined. This collector does not determine
* the most relevant document of a group.
*
* <p/>
* Implementation detail: Uses {@link ValueSource} and {@link DocValues} to retrieve the
* field values to group by.
*
* @lucene.experimental
*/
public class FunctionAllGroupsCollector extends AbstractAllGroupsCollector<MutableValue> {
private final Map vsContext;
private final ValueSource groupBy;
private final SortedSet<MutableValue> groups = new TreeSet<MutableValue>();
private DocValues.ValueFiller filler;
private MutableValue mval;
/**
* Constructs a {@link FunctionAllGroupsCollector} instance.
*
* @param groupBy The {@link ValueSource} to group by
* @param vsContext The ValueSource context
*/
public FunctionAllGroupsCollector(ValueSource groupBy, Map vsContext) {
this.vsContext = vsContext;
this.groupBy = groupBy;
}
/**
* {@inheritDoc}
*/
public Collection<MutableValue> getGroups() {
return groups;
}
public void collect(int doc) throws IOException {
filler.fillValue(doc);
if (!groups.contains(mval)) {
groups.add(mval.duplicate());
}
}
/**
* {@inheritDoc}
*/
public void setNextReader(IndexReader.AtomicReaderContext context) throws IOException {
DocValues docValues = groupBy.getValues(vsContext, context);
filler = docValues.getValueFiller();
mval = filler.getValue();
}
}

View File

@ -0,0 +1,88 @@
package org.apache.lucene.search.grouping.function;
/*
* 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.
*/
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.queries.function.DocValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.grouping.AbstractFirstPassGroupingCollector;
import org.apache.lucene.util.mutable.MutableValue;
import java.io.IOException;
import java.util.Map;
/**
* Concrete implementation of {@link AbstractFirstPassGroupingCollector} that groups based on
* {@link ValueSource} instances.
*
* @lucene.experimental
*/
public class FunctionFirstPassGroupingCollector extends AbstractFirstPassGroupingCollector<MutableValue> {
private final ValueSource groupByVS;
private final Map vsContext;
private DocValues docValues;
private DocValues.ValueFiller filler;
private MutableValue mval;
/**
* Creates a first pass collector.
*
* @param groupByVS The {@link ValueSource} instance to group by
* @param vsContext The ValueSource context
* @param groupSort The {@link Sort} used to sort the
* groups. The top sorted document within each group
* according to groupSort, determines how that group
* sorts against other groups. This must be non-null,
* ie, if you want to groupSort by relevance use
* Sort.RELEVANCE.
* @param topNGroups How many top groups to keep.
* @throws IOException When I/O related errors occur
*/
public FunctionFirstPassGroupingCollector(ValueSource groupByVS, Map vsContext, Sort groupSort, int topNGroups) throws IOException {
super(groupSort, topNGroups);
this.groupByVS = groupByVS;
this.vsContext = vsContext;
}
@Override
protected MutableValue getDocGroupValue(int doc) {
filler.fillValue(doc);
return mval;
}
@Override
protected MutableValue copyDocGroupValue(MutableValue groupValue, MutableValue reuse) {
if (reuse != null) {
reuse.copy(groupValue);
return reuse;
}
return groupValue.duplicate();
}
@Override
public void setNextReader(IndexReader.AtomicReaderContext readerContext) throws IOException {
super.setNextReader(readerContext);
docValues = groupByVS.getValues(vsContext, readerContext);
filler = docValues.getValueFiller();
mval = filler.getValue();
}
}

View File

@ -0,0 +1,85 @@
package org.apache.lucene.search.grouping.function;
/*
* 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.
*/
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.queries.function.DocValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.grouping.AbstractSecondPassGroupingCollector;
import org.apache.lucene.search.grouping.SearchGroup;
import org.apache.lucene.util.mutable.MutableValue;
import org.apache.lucene.search.grouping.TopGroups; //javadoc
import java.io.IOException;
import java.util.Collection;
import java.util.Map;
/**
* Concrete implementation of {@link AbstractSecondPassGroupingCollector} that groups based on
* {@link ValueSource} instances.
*
* @lucene.experimental
*/
public class FunctionSecondPassGroupingCollector extends AbstractSecondPassGroupingCollector<MutableValue> {
private final ValueSource groupByVS;
private final Map vsContext;
private DocValues.ValueFiller filler;
private MutableValue mval;
/**
* Constructs a {@link FunctionSecondPassGroupingCollector} instance.
*
* @param searchGroups The {@link SearchGroup} instances collected during the first phase.
* @param groupSort The group sort
* @param withinGroupSort The sort inside a group
* @param maxDocsPerGroup The maximum number of documents to collect inside a group
* @param getScores Whether to include the scores
* @param getMaxScores Whether to include the maximum score
* @param fillSortFields Whether to fill the sort values in {@link TopGroups#withinGroupSort}
* @param groupByVS The {@link ValueSource} to group by
* @param vsContext The value source context
* @throws IOException IOException When I/O related errors occur
*/
public FunctionSecondPassGroupingCollector(Collection<SearchGroup<MutableValue>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields, ValueSource groupByVS, Map vsContext) throws IOException {
super(searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
this.groupByVS = groupByVS;
this.vsContext = vsContext;
}
/**
* {@inheritDoc}
*/
protected SearchGroupDocs<MutableValue> retrieveGroup(int doc) throws IOException {
filler.fillValue(doc);
return groupMap.get(mval);
}
/**
* {@inheritDoc}
*/
public void setNextReader(IndexReader.AtomicReaderContext readerContext) throws IOException {
super.setNextReader(readerContext);
DocValues docValues = groupByVS.getValues(vsContext, readerContext);
filler = docValues.getValueFiller();
mval = filler.getValue();
}
}

View File

@ -44,9 +44,9 @@ field fall into a single group.</p>
</ul>
<p>The implementation is two-pass: the first pass ({@link
org.apache.lucene.search.grouping.TermFirstPassGroupingCollector})
org.apache.lucene.search.grouping.term.TermFirstPassGroupingCollector})
gathers the top groups, and the second pass ({@link
org.apache.lucene.search.grouping.TermSecondPassGroupingCollector})
org.apache.lucene.search.grouping.term.TermSecondPassGroupingCollector})
gathers documents within those groups. If the search is costly to
run you may want to use the {@link
org.apache.lucene.search.CachingCollector} class, which
@ -179,5 +179,11 @@ fields, <code>FieldCache</code>, etc.).
FixedBitSet groupHeadsBitSet = c.retrieveGroupHeads(maxDoc)
</pre>
<p>For each of the above collectors there is also a variant that works with <code>ValueSource</code> instead of
of fields. Concretely this means that these variants can work with functions. These variants are slower than
there term based counter parts. These implementations are located in the
<code>org.apache.lucene.search.grouping.function</code> package.
</p>
</body>
</html>

View File

@ -1,4 +1,4 @@
package org.apache.lucene.search.grouping;
package org.apache.lucene.search.grouping.term;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -19,13 +19,15 @@ package org.apache.lucene.search.grouping;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.*;
import org.apache.lucene.search.grouping.AbstractAllGroupHeadsCollector;
import org.apache.lucene.search.grouping.SentinelIntSet;
import org.apache.lucene.util.BytesRef;
import java.io.IOException;
import java.util.*;
/**
* A base implementation of {@link AbstractAllGroupHeadsCollector} for retrieving the most relevant groups when grouping
* A base implementation of {@link org.apache.lucene.search.grouping.AbstractAllGroupHeadsCollector} for retrieving the most relevant groups when grouping
* on a string based group field. More specifically this all concrete implementations of this base implementation
* use {@link org.apache.lucene.search.FieldCache.DocTermsIndex}.
*
@ -537,4 +539,4 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
}
}
}

View File

@ -1,4 +1,4 @@
package org.apache.lucene.search.grouping;
package org.apache.lucene.search.grouping.term;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -19,6 +19,8 @@ package org.apache.lucene.search.grouping;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.grouping.AbstractAllGroupsCollector;
import org.apache.lucene.search.grouping.SentinelIntSet;
import org.apache.lucene.util.BytesRef;
import java.io.IOException;

View File

@ -1,4 +1,4 @@
package org.apache.lucene.search.grouping;
package org.apache.lucene.search.grouping.term;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -20,12 +20,13 @@ package org.apache.lucene.search.grouping;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.grouping.AbstractFirstPassGroupingCollector;
import org.apache.lucene.util.BytesRef;
import java.io.IOException;
/**
* Concrete implementation of {@link AbstractFirstPassGroupingCollector} that groups based on
* Concrete implementation of {@link org.apache.lucene.search.grouping.AbstractFirstPassGroupingCollector} that groups based on
* field values and more specifically uses {@link org.apache.lucene.search.FieldCache.DocTermsIndex}
* to collect groups.
*

View File

@ -1,4 +1,4 @@
package org.apache.lucene.search.grouping;
package org.apache.lucene.search.grouping.term;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -20,13 +20,16 @@ package org.apache.lucene.search.grouping;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.grouping.AbstractSecondPassGroupingCollector;
import org.apache.lucene.search.grouping.SearchGroup;
import org.apache.lucene.search.grouping.SentinelIntSet;
import org.apache.lucene.util.BytesRef;
import java.io.IOException;
import java.util.Collection;
/**
* Concrete implementation of {@link AbstractSecondPassGroupingCollector} that groups based on
* Concrete implementation of {@link org.apache.lucene.search.grouping.AbstractSecondPassGroupingCollector} that groups based on
* field values and more specifically uses {@link org.apache.lucene.search.FieldCache.DocTermsIndex}
* to collect grouped docs.
*
@ -73,4 +76,4 @@ public class TermSecondPassGroupingCollector extends AbstractSecondPassGroupingC
}
return null;
}
}
}

View File

@ -26,7 +26,11 @@ import org.apache.lucene.document.TextField;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.valuesource.BytesRefFieldSource;
import org.apache.lucene.search.*;
import org.apache.lucene.search.grouping.function.FunctionAllGroupHeadsCollector;
import org.apache.lucene.search.grouping.term.TermAllGroupHeadsCollector;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
@ -36,7 +40,7 @@ import org.apache.lucene.util._TestUtil;
import java.io.IOException;
import java.util.*;
public class TermAllGroupHeadsCollectorTest extends LuceneTestCase {
public class AllGroupHeadsCollectorTest extends LuceneTestCase {
public void testBasic() throws Exception {
final String groupField = "author";
@ -107,30 +111,30 @@ public class TermAllGroupHeadsCollectorTest extends LuceneTestCase {
int maxDoc = indexSearcher.maxDoc();
Sort sortWithinGroup = new Sort(new SortField("id", SortField.Type.INT, true));
AbstractAllGroupHeadsCollector c1 = TermAllGroupHeadsCollector.create(groupField, sortWithinGroup);
AbstractAllGroupHeadsCollector c1 = createRandomCollector(groupField, sortWithinGroup);
indexSearcher.search(new TermQuery(new Term("content", "random")), c1);
assertTrue(arrayContains(new int[]{2, 3, 5, 7}, c1.retrieveGroupHeads()));
assertTrue(openBitSetContains(new int[]{2, 3, 5, 7}, c1.retrieveGroupHeads(maxDoc), maxDoc));
AbstractAllGroupHeadsCollector c2 = TermAllGroupHeadsCollector.create(groupField, sortWithinGroup);
AbstractAllGroupHeadsCollector c2 = createRandomCollector(groupField, sortWithinGroup);
indexSearcher.search(new TermQuery(new Term("content", "some")), c2);
assertTrue(arrayContains(new int[]{2, 3, 4}, c2.retrieveGroupHeads()));
assertTrue(openBitSetContains(new int[]{2, 3, 4}, c2.retrieveGroupHeads(maxDoc), maxDoc));
AbstractAllGroupHeadsCollector c3 = TermAllGroupHeadsCollector.create(groupField, sortWithinGroup);
AbstractAllGroupHeadsCollector c3 = createRandomCollector(groupField, sortWithinGroup);
indexSearcher.search(new TermQuery(new Term("content", "blob")), c3);
assertTrue(arrayContains(new int[]{1, 5}, c3.retrieveGroupHeads()));
assertTrue(openBitSetContains(new int[]{1, 5}, c3.retrieveGroupHeads(maxDoc), maxDoc));
// STRING sort type triggers different implementation
Sort sortWithinGroup2 = new Sort(new SortField("id", SortField.Type.STRING, true));
AbstractAllGroupHeadsCollector c4 = TermAllGroupHeadsCollector.create(groupField, sortWithinGroup2);
AbstractAllGroupHeadsCollector c4 = createRandomCollector(groupField, sortWithinGroup2);
indexSearcher.search(new TermQuery(new Term("content", "random")), c4);
assertTrue(arrayContains(new int[]{2, 3, 5, 7}, c4.retrieveGroupHeads()));
assertTrue(openBitSetContains(new int[]{2, 3, 5, 7}, c4.retrieveGroupHeads(maxDoc), maxDoc));
Sort sortWithinGroup3 = new Sort(new SortField("id", SortField.Type.STRING, false));
AbstractAllGroupHeadsCollector c5 = TermAllGroupHeadsCollector.create(groupField, sortWithinGroup3);
AbstractAllGroupHeadsCollector c5 = createRandomCollector(groupField, sortWithinGroup3);
indexSearcher.search(new TermQuery(new Term("content", "random")), c5);
// 7 b/c higher doc id wins, even if order of field is in not in reverse.
assertTrue(arrayContains(new int[]{0, 3, 4, 6}, c5.retrieveGroupHeads()));
@ -279,7 +283,7 @@ public class TermAllGroupHeadsCollectorTest extends LuceneTestCase {
final String searchTerm = "real" + random.nextInt(3);
boolean sortByScoreOnly = random.nextBoolean();
Sort sortWithinGroup = getRandomSort(sortByScoreOnly);
AbstractAllGroupHeadsCollector allGroupHeadsCollector = TermAllGroupHeadsCollector.create("group", sortWithinGroup);
AbstractAllGroupHeadsCollector allGroupHeadsCollector = createRandomCollector("group", sortWithinGroup);
s.search(new TermQuery(new Term("content", searchTerm)), allGroupHeadsCollector);
int[] expectedGroupHeads = createExpectedGroupHeads(searchTerm, groupDocs, sortWithinGroup, sortByScoreOnly, fieldIdToDocID);
int[] actualGroupHeads = allGroupHeadsCollector.retrieveGroupHeads();
@ -340,6 +344,7 @@ public class TermAllGroupHeadsCollectorTest extends LuceneTestCase {
private boolean arrayContains(int[] expected, int[] actual) {
Arrays.sort(actual); // in some cases the actual docs aren't sorted by docid. This method expects that.
if (expected.length != actual.length) {
return false;
}
@ -469,6 +474,15 @@ public class TermAllGroupHeadsCollectorTest extends LuceneTestCase {
};
}
private AbstractAllGroupHeadsCollector createRandomCollector(String groupField, Sort sortWithinGroup) throws IOException {
if (random.nextBoolean()) {
ValueSource vs = new BytesRefFieldSource(groupField);
return new FunctionAllGroupHeadsCollector(vs, new HashMap(), sortWithinGroup);
} else {
return TermAllGroupHeadsCollector.create(groupField, sortWithinGroup);
}
}
private static class GroupDoc {
final int id;
@ -491,4 +505,4 @@ public class TermAllGroupHeadsCollectorTest extends LuceneTestCase {
}
}
}

View File

@ -24,12 +24,19 @@ import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.valuesource.BytesRefFieldSource;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.grouping.function.FunctionAllGroupsCollector;
import org.apache.lucene.search.grouping.term.TermAllGroupsCollector;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
public class TermAllGroupsCollectorTest extends LuceneTestCase {
import java.io.IOException;
import java.util.HashMap;
public class AllGroupsCollectorTest extends LuceneTestCase {
public void testTotalGroupCount() throws Exception {
@ -95,19 +102,29 @@ public class TermAllGroupsCollectorTest extends LuceneTestCase {
IndexSearcher indexSearcher = new IndexSearcher(w.getReader());
w.close();
TermAllGroupsCollector c1 = new TermAllGroupsCollector(groupField);
AbstractAllGroupsCollector c1 = createRandomCollector(groupField);
indexSearcher.search(new TermQuery(new Term("content", "random")), c1);
assertEquals(4, c1.getGroupCount());
TermAllGroupsCollector c2 = new TermAllGroupsCollector(groupField);
AbstractAllGroupsCollector c2 = createRandomCollector(groupField);
indexSearcher.search(new TermQuery(new Term("content", "some")), c2);
assertEquals(3, c2.getGroupCount());
TermAllGroupsCollector c3 = new TermAllGroupsCollector(groupField);
AbstractAllGroupsCollector c3 = createRandomCollector(groupField);
indexSearcher.search(new TermQuery(new Term("content", "blob")), c3);
assertEquals(2, c3.getGroupCount());
indexSearcher.getIndexReader().close();
dir.close();
}
private AbstractAllGroupsCollector createRandomCollector(String groupField) throws IOException {
if (random.nextBoolean()) {
return new TermAllGroupsCollector(groupField);
} else {
ValueSource vs = new BytesRefFieldSource(groupField);
return new FunctionAllGroupsCollector(vs, new HashMap());
}
}
}

View File

@ -31,12 +31,22 @@ import org.apache.lucene.document.TextField;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.valuesource.BytesRefFieldSource;
import org.apache.lucene.search.*;
import org.apache.lucene.search.grouping.function.FunctionAllGroupsCollector;
import org.apache.lucene.search.grouping.function.FunctionFirstPassGroupingCollector;
import org.apache.lucene.search.grouping.function.FunctionSecondPassGroupingCollector;
import org.apache.lucene.search.grouping.term.TermAllGroupsCollector;
import org.apache.lucene.search.grouping.term.TermFirstPassGroupingCollector;
import org.apache.lucene.search.grouping.term.TermSecondPassGroupingCollector;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.util.mutable.MutableValue;
import org.apache.lucene.util.mutable.MutableValueStr;
// TODO
// - should test relevance sort too
@ -111,10 +121,10 @@ public class TestGrouping extends LuceneTestCase {
w.close();
final Sort groupSort = Sort.RELEVANCE;
final TermFirstPassGroupingCollector c1 = new TermFirstPassGroupingCollector(groupField, groupSort, 10);
final AbstractFirstPassGroupingCollector c1 = createRandomFirstPassCollector(groupField, groupSort, 10);
indexSearcher.search(new TermQuery(new Term("content", "random")), c1);
final TermSecondPassGroupingCollector c2 = new TermSecondPassGroupingCollector(groupField, c1.getTopGroups(0, true), groupSort, null, 5, true, false, true);
final AbstractSecondPassGroupingCollector c2 = createSecondPassCollector(c1, groupField, groupSort, null, 0, 5, true, false, true);
indexSearcher.search(new TermQuery(new Term("content", "random")), c2);
final TopGroups groups = c2.getTopGroups(0);
@ -128,14 +138,14 @@ public class TestGrouping extends LuceneTestCase {
// the later a document is added the higher this docId
// value
GroupDocs group = groups.groups[0];
assertEquals(new BytesRef("author3"), group.groupValue);
compareGroupValue("author3", group);
assertEquals(2, group.scoreDocs.length);
assertEquals(5, group.scoreDocs[0].doc);
assertEquals(4, group.scoreDocs[1].doc);
assertTrue(group.scoreDocs[0].score > group.scoreDocs[1].score);
group = groups.groups[1];
assertEquals(new BytesRef("author1"), group.groupValue);
compareGroupValue("author1", group);
assertEquals(3, group.scoreDocs.length);
assertEquals(0, group.scoreDocs[0].doc);
assertEquals(1, group.scoreDocs[1].doc);
@ -144,12 +154,12 @@ public class TestGrouping extends LuceneTestCase {
assertTrue(group.scoreDocs[1].score > group.scoreDocs[2].score);
group = groups.groups[2];
assertEquals(new BytesRef("author2"), group.groupValue);
compareGroupValue("author2", group);
assertEquals(1, group.scoreDocs.length);
assertEquals(3, group.scoreDocs[0].doc);
group = groups.groups[3];
assertNull(group.groupValue);
compareGroupValue(null, group);
assertEquals(1, group.scoreDocs.length);
assertEquals(6, group.scoreDocs[0].doc);
@ -157,6 +167,140 @@ public class TestGrouping extends LuceneTestCase {
dir.close();
}
private AbstractFirstPassGroupingCollector createRandomFirstPassCollector(String groupField, Sort groupSort, int topDocs) throws IOException {
if (random.nextBoolean()) {
ValueSource vs = new BytesRefFieldSource(groupField);
return new FunctionFirstPassGroupingCollector(vs, new HashMap(), groupSort, topDocs);
} else {
return new TermFirstPassGroupingCollector(groupField, groupSort, topDocs);
}
}
private AbstractSecondPassGroupingCollector createSecondPassCollector(AbstractFirstPassGroupingCollector firstPassGroupingCollector,
String groupField,
Sort groupSort,
Sort sortWithinGroup,
int groupOffset,
int maxDocsPerGroup,
boolean getScores,
boolean getMaxScores,
boolean fillSortFields) throws IOException {
if (firstPassGroupingCollector.getClass().isAssignableFrom(TermFirstPassGroupingCollector.class)) {
@SuppressWarnings("unchecked")
Collection<SearchGroup<BytesRef>> searchGroups = firstPassGroupingCollector.getTopGroups(groupOffset, fillSortFields);
return new TermSecondPassGroupingCollector(groupField, searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup , getScores, getMaxScores, fillSortFields);
} else {
ValueSource vs = new BytesRefFieldSource(groupField);
@SuppressWarnings("unchecked")
Collection<SearchGroup<MutableValue>> searchGroups = firstPassGroupingCollector.getTopGroups(groupOffset, fillSortFields);
return new FunctionSecondPassGroupingCollector(searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields, vs, new HashMap());
}
}
// Basically converts searchGroups from MutableValue to BytesRef if grouping by ValueSource
private AbstractSecondPassGroupingCollector createSecondPassCollector(AbstractFirstPassGroupingCollector firstPassGroupingCollector,
String groupField,
Collection<SearchGroup<BytesRef>> searchGroups,
Sort groupSort,
Sort sortWithinGroup,
int maxDocsPerGroup,
boolean getScores,
boolean getMaxScores,
boolean fillSortFields) throws IOException {
if (firstPassGroupingCollector.getClass().isAssignableFrom(TermFirstPassGroupingCollector.class)) {
return new TermSecondPassGroupingCollector(groupField, searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup , getScores, getMaxScores, fillSortFields);
} else {
ValueSource vs = new BytesRefFieldSource(groupField);
List<SearchGroup<MutableValue>> mvalSearchGroups = new ArrayList<SearchGroup<MutableValue>>(searchGroups.size());
for (SearchGroup<BytesRef> mergedTopGroup : searchGroups) {
SearchGroup<MutableValue> sg = new SearchGroup<MutableValue>();
MutableValueStr groupValue = new MutableValueStr();
if (mergedTopGroup.groupValue != null) {
groupValue.value = mergedTopGroup.groupValue;
} else {
groupValue.value = new BytesRef();
groupValue.exists = false;
}
sg.groupValue = groupValue;
sg.sortValues = mergedTopGroup.sortValues;
mvalSearchGroups.add(sg);
}
return new FunctionSecondPassGroupingCollector(mvalSearchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields, vs, new HashMap());
}
}
private AbstractAllGroupsCollector createAllGroupsCollector(AbstractFirstPassGroupingCollector firstPassGroupingCollector, String groupField) {
if (firstPassGroupingCollector.getClass().isAssignableFrom(TermFirstPassGroupingCollector.class)) {
return new TermAllGroupsCollector(groupField);
} else {
ValueSource vs = new BytesRefFieldSource(groupField);
return new FunctionAllGroupsCollector(vs, new HashMap());
}
}
private void compareGroupValue(String expected, GroupDocs group) {
if (expected == null) {
if (group.groupValue == null) {
return;
} else if (group.groupValue.getClass().isAssignableFrom(MutableValueStr.class)) {
return;
}
fail();
}
if (group.groupValue.getClass().isAssignableFrom(BytesRef.class)) {
assertEquals(new BytesRef(expected), group.groupValue);
} else if (group.groupValue.getClass().isAssignableFrom(MutableValueStr.class)) {
MutableValueStr v = new MutableValueStr();
v.value = new BytesRef(expected);
assertEquals(v, group.groupValue);
} else {
fail();
}
}
private Collection<SearchGroup<BytesRef>> getSearchGroups(AbstractFirstPassGroupingCollector c, int groupOffset, boolean fillFields) {
if (c.getClass().isAssignableFrom(TermFirstPassGroupingCollector.class)) {
return ((TermFirstPassGroupingCollector) c).getTopGroups(groupOffset, fillFields);
} else if (c.getClass().isAssignableFrom(FunctionFirstPassGroupingCollector.class)) {
Collection<SearchGroup<MutableValue>> mutableValueGroups = ((FunctionFirstPassGroupingCollector) c).getTopGroups(groupOffset, fillFields);
if (mutableValueGroups == null) {
return null;
}
List<SearchGroup<BytesRef>> groups = new ArrayList<SearchGroup<BytesRef>>(mutableValueGroups.size());
for (SearchGroup<MutableValue> mutableValueGroup : mutableValueGroups) {
SearchGroup<BytesRef> sg = new SearchGroup<BytesRef>();
sg.groupValue = mutableValueGroup.groupValue.exists() ? ((MutableValueStr) mutableValueGroup.groupValue).value : null;
sg.sortValues = mutableValueGroup.sortValues;
groups.add(sg);
}
return groups;
}
fail();
return null;
}
@SuppressWarnings("unchecked")
private TopGroups<BytesRef> getTopGroups(AbstractSecondPassGroupingCollector c, int withinGroupOffset) {
if (c.getClass().isAssignableFrom(TermSecondPassGroupingCollector.class)) {
return ((TermSecondPassGroupingCollector) c).getTopGroups(withinGroupOffset);
} else if (c.getClass().isAssignableFrom(FunctionSecondPassGroupingCollector.class)) {
TopGroups<MutableValue> mvalTopGroups = ((FunctionSecondPassGroupingCollector) c).getTopGroups(withinGroupOffset);
List<GroupDocs<BytesRef>> groups = new ArrayList<GroupDocs<BytesRef>>(mvalTopGroups.groups.length);
for (GroupDocs<MutableValue> mvalGd : mvalTopGroups.groups) {
BytesRef groupValue = mvalGd.groupValue.exists() ? ((MutableValueStr) mvalGd.groupValue).value : null;
groups.add(new GroupDocs<BytesRef>(mvalGd.maxScore, mvalGd.totalHits, mvalGd.scoreDocs, groupValue, mvalGd.groupSortValues));
}
return new TopGroups<BytesRef>(mvalTopGroups.groupSort, mvalTopGroups.withinGroupSort, mvalTopGroups.totalHitCount, mvalTopGroups.totalGroupedHitCount, groups.toArray(new GroupDocs[groups.size()]));
}
fail();
return null;
}
private static class GroupDoc {
final int id;
final BytesRef group;
@ -662,17 +806,17 @@ public class TestGrouping extends LuceneTestCase {
System.out.println("TEST: groupSort=" + groupSort + " docSort=" + docSort + " searchTerm=" + searchTerm + " dF=" + r.docFreq("content", new BytesRef(searchTerm)) +" dFBlock=" + rBlocks.docFreq("content", new BytesRef(searchTerm)) + " topNGroups=" + topNGroups + " groupOffset=" + groupOffset + " docOffset=" + docOffset + " doCache=" + doCache + " docsPerGroup=" + docsPerGroup + " doAllGroups=" + doAllGroups + " getScores=" + getScores + " getMaxScores=" + getMaxScores);
}
final TermAllGroupsCollector allGroupsCollector;
final AbstractFirstPassGroupingCollector c1 = createRandomFirstPassCollector("group", groupSort, groupOffset+topNGroups);
final CachingCollector cCache;
final Collector c;
final AbstractAllGroupsCollector allGroupsCollector;
if (doAllGroups) {
allGroupsCollector = new TermAllGroupsCollector("group");
allGroupsCollector = createAllGroupsCollector(c1, "group");
} else {
allGroupsCollector = null;
}
final TermFirstPassGroupingCollector c1 = new TermFirstPassGroupingCollector("group", groupSort, groupOffset+topNGroups);
final CachingCollector cCache;
final Collector c;
final boolean useWrappingCollector = random.nextBoolean();
if (doCache) {
@ -723,8 +867,8 @@ public class TestGrouping extends LuceneTestCase {
}
// Get 1st pass top groups
final Collection<SearchGroup<BytesRef>> topGroups = c1.getTopGroups(groupOffset, fillFields);
final Collection<SearchGroup<BytesRef>> topGroups = getSearchGroups(c1, groupOffset, fillFields);
final TopGroups<BytesRef> groupsResult;
if (VERBOSE) {
System.out.println("TEST: first pass topGroups");
if (topGroups == null) {
@ -738,12 +882,17 @@ public class TestGrouping extends LuceneTestCase {
// Get 1st pass top groups using shards
final TopGroups<BytesRef> topGroupsShards = searchShards(s, shards.subSearchers, query, groupSort, docSort, groupOffset, topNGroups, docOffset, docsPerGroup, getScores, getMaxScores);
final TopGroups<BytesRef> groupsResult;
final AbstractSecondPassGroupingCollector c2;
if (topGroups != null) {
// Get 2nd pass grouped result:
final TermSecondPassGroupingCollector c2 = new TermSecondPassGroupingCollector("group", topGroups, groupSort, docSort, docOffset+docsPerGroup, getScores, getMaxScores, fillFields);
if (VERBOSE) {
System.out.println("TEST: topGroups");
for (SearchGroup<BytesRef> searchGroup : topGroups) {
System.out.println(" " + (searchGroup.groupValue == null ? "null" : searchGroup.groupValue.utf8ToString()) + ": " + Arrays.deepToString(searchGroup.sortValues));
}
}
c2 = createSecondPassCollector(c1, "group", groupSort, docSort, groupOffset, docOffset + docsPerGroup, getScores, getMaxScores, fillFields);
if (doCache) {
if (cCache.isCached()) {
if (VERBOSE) {
@ -761,12 +910,13 @@ public class TestGrouping extends LuceneTestCase {
}
if (doAllGroups) {
TopGroups<BytesRef> tempTopGroups = c2.getTopGroups(docOffset);
TopGroups<BytesRef> tempTopGroups = getTopGroups(c2, docOffset);
groupsResult = new TopGroups<BytesRef>(tempTopGroups, allGroupsCollector.getGroupCount());
} else {
groupsResult = c2.getTopGroups(docOffset);
groupsResult = getTopGroups(c2, docOffset);
}
} else {
c2 = null;
groupsResult = null;
if (VERBOSE) {
System.out.println("TEST: no results");
@ -962,10 +1112,12 @@ public class TestGrouping extends LuceneTestCase {
// Run 1st pass collector to get top groups per shard
final Weight w = topSearcher.createNormalizedWeight(query);
final List<Collection<SearchGroup<BytesRef>>> shardGroups = new ArrayList<Collection<SearchGroup<BytesRef>>>();
List<AbstractFirstPassGroupingCollector> firstPassGroupingCollectors = new ArrayList<AbstractFirstPassGroupingCollector>();
for(int shardIDX=0;shardIDX<subSearchers.length;shardIDX++) {
final TermFirstPassGroupingCollector c = new TermFirstPassGroupingCollector("group", groupSort, groupOffset+topNGroups);
final AbstractFirstPassGroupingCollector c = createRandomFirstPassCollector("group", groupSort, groupOffset+topNGroups);
firstPassGroupingCollectors.add(c);
subSearchers[shardIDX].search(w, c);
final Collection<SearchGroup<BytesRef>> topGroups = c.getTopGroups(0, true);
final Collection<SearchGroup<BytesRef>> topGroups = getSearchGroups(c, 0, true);
if (topGroups != null) {
if (VERBOSE) {
System.out.println(" shard " + shardIDX + " s=" + subSearchers[shardIDX] + " " + topGroups.size() + " groups:");
@ -995,10 +1147,10 @@ public class TestGrouping extends LuceneTestCase {
@SuppressWarnings("unchecked")
final TopGroups<BytesRef>[] shardTopGroups = new TopGroups[subSearchers.length];
for(int shardIDX=0;shardIDX<subSearchers.length;shardIDX++) {
final TermSecondPassGroupingCollector c = new TermSecondPassGroupingCollector("group", mergedTopGroups, groupSort, docSort,
docOffset + topNDocs, getScores, getMaxScores, true);
final AbstractSecondPassGroupingCollector c = createSecondPassCollector(firstPassGroupingCollectors.get(shardIDX),
"group", mergedTopGroups, groupSort, docSort, docOffset + topNDocs, getScores, getMaxScores, true);
subSearchers[shardIDX].search(w, c);
shardTopGroups[shardIDX] = c.getTopGroups(0);
shardTopGroups[shardIDX] = getTopGroups(c, 0);
}
return TopGroups.merge(shardTopGroups, groupSort, docSort, docOffset, topNDocs);

View File

@ -0,0 +1,58 @@
package org.apache.lucene.queries.function.valuesource;
/*
* 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.
*/
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.queries.function.DocValues;
import org.apache.lucene.queries.function.docvalues.StringIndexDocValues;
import org.apache.lucene.queries.function.ValueSource; //javadoc
import java.io.IOException;
import java.util.Map;
/**
* An implementation for retrieving {@link DocValues} instances for string based fields.
*/
public class BytesRefFieldSource extends FieldCacheSource {
public BytesRefFieldSource(String field) {
super(field);
}
@Override
public DocValues getValues(Map context, IndexReader.AtomicReaderContext readerContext) throws IOException {
return new StringIndexDocValues(this, readerContext, field) {
@Override
protected String toTerm(String readableValue) {
return readableValue;
}
@Override
public Object objectVal(int doc) {
return strVal(doc);
}
@Override
public String toString(int doc) {
return description() + '=' + strVal(doc);
}
};
}
}

View File

@ -18,13 +18,9 @@
package org.apache.solr.request;
import org.apache.lucene.index.*;
import org.apache.lucene.queries.function.FunctionQuery;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.valuesource.QueryValueSource;
import org.apache.lucene.queryparser.classic.ParseException;
import org.apache.lucene.search.*;
import org.apache.lucene.search.grouping.AbstractAllGroupHeadsCollector;
import org.apache.lucene.search.grouping.TermAllGroupHeadsCollector;
import org.apache.lucene.util.*;
import org.apache.lucene.util.packed.Direct16;
import org.apache.lucene.util.packed.Direct32;

View File

@ -18,15 +18,21 @@
package org.apache.solr.search;
import org.apache.commons.lang.ArrayUtils;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.queries.function.DocValues;
import org.apache.lucene.queries.function.FunctionQuery;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.valuesource.QueryValueSource;
import org.apache.lucene.queryparser.classic.ParseException;
import org.apache.lucene.search.*;
import org.apache.lucene.search.grouping.*;
import org.apache.lucene.search.grouping.function.FunctionAllGroupHeadsCollector;
import org.apache.lucene.search.grouping.function.FunctionAllGroupsCollector;
import org.apache.lucene.search.grouping.function.FunctionFirstPassGroupingCollector;
import org.apache.lucene.search.grouping.function.FunctionSecondPassGroupingCollector;
import org.apache.lucene.search.grouping.term.TermAllGroupHeadsCollector;
import org.apache.lucene.search.grouping.term.TermAllGroupsCollector;
import org.apache.lucene.search.grouping.term.TermFirstPassGroupingCollector;
import org.apache.lucene.search.grouping.term.TermSecondPassGroupingCollector;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.OpenBitSet;
@ -976,213 +982,4 @@ public class Grouping {
}
static class FunctionFirstPassGroupingCollector extends AbstractFirstPassGroupingCollector<MutableValue> {
private final ValueSource groupByVS;
private final Map vsContext;
private DocValues docValues;
private DocValues.ValueFiller filler;
private MutableValue mval;
FunctionFirstPassGroupingCollector(ValueSource groupByVS, Map vsContext, Sort groupSort, int topNGroups) throws IOException {
super(groupSort, topNGroups);
this.groupByVS = groupByVS;
this.vsContext = vsContext;
}
@Override
protected MutableValue getDocGroupValue(int doc) {
filler.fillValue(doc);
return mval;
}
@Override
protected MutableValue copyDocGroupValue(MutableValue groupValue, MutableValue reuse) {
if (reuse != null) {
reuse.copy(groupValue);
return reuse;
}
return groupValue.duplicate();
}
@Override
public void setNextReader(AtomicReaderContext readerContext) throws IOException {
super.setNextReader(readerContext);
docValues = groupByVS.getValues(vsContext, readerContext);
filler = docValues.getValueFiller();
mval = filler.getValue();
}
}
static class FunctionSecondPassGroupingCollector extends AbstractSecondPassGroupingCollector<MutableValue> {
private final ValueSource groupByVS;
private final Map vsContext;
private DocValues docValues;
private DocValues.ValueFiller filler;
private MutableValue mval;
FunctionSecondPassGroupingCollector(Collection<SearchGroup<MutableValue>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields, ValueSource groupByVS, Map vsContext) throws IOException {
super(searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
this.groupByVS = groupByVS;
this.vsContext = vsContext;
}
/**
* {@inheritDoc}
*/
protected SearchGroupDocs<MutableValue> retrieveGroup(int doc) throws IOException {
filler.fillValue(doc);
return groupMap.get(mval);
}
/**
* {@inheritDoc}
*/
public void setNextReader(AtomicReaderContext readerContext) throws IOException {
super.setNextReader(readerContext);
docValues = groupByVS.getValues(vsContext, readerContext);
filler = docValues.getValueFiller();
mval = filler.getValue();
}
}
static class FunctionAllGroupsCollector extends AbstractAllGroupsCollector<MutableValue> {
private final Map vsContext;
private final ValueSource groupBy;
private final SortedSet<MutableValue> groups = new TreeSet<MutableValue>();
private DocValues docValues;
private DocValues.ValueFiller filler;
private MutableValue mval;
FunctionAllGroupsCollector(ValueSource groupBy, Map vsContext) {
this.vsContext = vsContext;
this.groupBy = groupBy;
}
public Collection<MutableValue> getGroups() {
return groups;
}
public void collect(int doc) throws IOException {
filler.fillValue(doc);
if (!groups.contains(mval)) {
groups.add(mval.duplicate());
}
}
/**
* {@inheritDoc}
*/
public void setNextReader(AtomicReaderContext context) throws IOException {
docValues = groupBy.getValues(vsContext, context);
filler = docValues.getValueFiller();
mval = filler.getValue();
}
}
static class FunctionAllGroupHeadsCollector extends AbstractAllGroupHeadsCollector<FunctionAllGroupHeadsCollector.GroupHead> {
private final ValueSource groupBy;
private final Map vsContext;
private final Map<MutableValue, GroupHead> groups;
private final Sort sortWithinGroup;
private DocValues docValues;
private DocValues.ValueFiller filler;
private MutableValue mval;
private AtomicReaderContext readerContext;
private Scorer scorer;
FunctionAllGroupHeadsCollector(ValueSource groupBy, Map vsContext, Sort sortWithinGroup) {
super(sortWithinGroup.getSort().length);
groups = new HashMap<MutableValue, GroupHead>();
this.sortWithinGroup = sortWithinGroup;
this.groupBy = groupBy;
this.vsContext = vsContext;
final SortField[] sortFields = sortWithinGroup.getSort();
for (int i = 0; i < sortFields.length; i++) {
reversed[i] = sortFields[i].getReverse() ? -1 : 1;
}
}
protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
filler.fillValue(doc);
GroupHead groupHead = groups.get(mval);
if (groupHead == null) {
MutableValue groupValue = mval.duplicate();
groupHead = new GroupHead(groupValue, sortWithinGroup, doc);
groups.put(groupValue, groupHead);
temporalResult.stop = true;
} else {
temporalResult.stop = false;
}
this.temporalResult.groupHead = groupHead;
}
protected Collection<GroupHead> getCollectedGroupHeads() {
return groups.values();
}
public void setScorer(Scorer scorer) throws IOException {
this.scorer = scorer;
for (GroupHead groupHead : groups.values()) {
for (FieldComparator comparator : groupHead.comparators) {
comparator.setScorer(scorer);
}
}
}
public void setNextReader(AtomicReaderContext context) throws IOException {
this.readerContext = context;
docValues = groupBy.getValues(vsContext, context);
filler = docValues.getValueFiller();
mval = filler.getValue();
for (GroupHead groupHead : groups.values()) {
for (int i = 0; i < groupHead.comparators.length; i++) {
groupHead.comparators[i] = groupHead.comparators[i].setNextReader(context);
}
}
}
class GroupHead extends AbstractAllGroupHeadsCollector.GroupHead<MutableValue> {
final FieldComparator[] comparators;
private GroupHead(MutableValue groupValue, Sort sort, int doc) throws IOException {
super(groupValue, doc + readerContext.docBase);
final SortField[] sortFields = sort.getSort();
comparators = new FieldComparator[sortFields.length];
for (int i = 0; i < sortFields.length; i++) {
comparators[i] = sortFields[i].getComparator(1, i).setNextReader(readerContext);
comparators[i].setScorer(scorer);
comparators[i].copy(0, doc);
comparators[i].setBottom(0);
}
}
public int compare(int compIDX, int doc) throws IOException {
return comparators[compIDX].compareBottom(doc);
}
public void updateDocHead(int doc) throws IOException {
for (FieldComparator comparator : comparators) {
comparator.copy(0, doc);
comparator.setBottom(0);
}
this.doc = doc + readerContext.docBase;
}
}
}
}

View File

@ -22,8 +22,7 @@ import org.apache.lucene.search.Filter;
import org.apache.lucene.search.MultiCollector;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.grouping.AbstractAllGroupHeadsCollector;
import org.apache.lucene.search.grouping.TermAllGroupHeadsCollector;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.search.grouping.term.TermAllGroupHeadsCollector;
import org.apache.lucene.util.OpenBitSet;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.search.*;

View File

@ -20,7 +20,7 @@ package org.apache.solr.search.grouping.distributed.command;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.grouping.SearchGroup;
import org.apache.lucene.search.grouping.TermFirstPassGroupingCollector;
import org.apache.lucene.search.grouping.term.TermFirstPassGroupingCollector;
import org.apache.lucene.util.BytesRef;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.search.grouping.Command;

View File

@ -20,8 +20,8 @@ package org.apache.solr.search.grouping.distributed.command;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.grouping.SearchGroup;
import org.apache.lucene.search.grouping.TermAllGroupsCollector;
import org.apache.lucene.search.grouping.TermSecondPassGroupingCollector;
import org.apache.lucene.search.grouping.term.TermAllGroupsCollector;
import org.apache.lucene.search.grouping.term.TermSecondPassGroupingCollector;
import org.apache.lucene.search.grouping.TopGroups;
import org.apache.lucene.util.BytesRef;
import org.apache.solr.schema.SchemaField;