From 70a7363fdb2ab12feac7ab10065198602291594a Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 6 Oct 2011 20:09:36 +0000 Subject: [PATCH] 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 --- dev-tools/idea/modules/grouping/grouping.iml | 1 + .../index/values/VarSortedBytesImpl.java | 1 + modules/build.xml | 14 +- modules/grouping/CHANGES.txt | 5 + modules/grouping/build.xml | 16 +- .../grouping/BlockGroupingCollector.java | 2 +- .../search/grouping/SentinelIntSet.java | 8 +- .../FunctionAllGroupHeadsCollector.java | 147 ++++++++++++ .../function/FunctionAllGroupsCollector.java | 86 +++++++ .../FunctionFirstPassGroupingCollector.java | 88 +++++++ .../FunctionSecondPassGroupingCollector.java | 85 +++++++ .../lucene/search/grouping/package.html | 10 +- .../TermAllGroupHeadsCollector.java | 8 +- .../{ => term}/TermAllGroupsCollector.java | 4 +- .../TermFirstPassGroupingCollector.java | 5 +- .../TermSecondPassGroupingCollector.java | 9 +- ...t.java => AllGroupHeadsCollectorTest.java} | 30 ++- ...rTest.java => AllGroupsCollectorTest.java} | 25 +- .../lucene/search/grouping/TestGrouping.java | 202 ++++++++++++++-- .../valuesource/BytesRefFieldSource.java | 58 +++++ .../org/apache/solr/request/SimpleFacets.java | 4 - .../java/org/apache/solr/search/Grouping.java | 219 +----------------- .../solr/search/grouping/CommandHandler.java | 3 +- .../command/SearchGroupsFieldCommand.java | 2 +- .../command/TopGroupsFieldCommand.java | 4 +- 25 files changed, 757 insertions(+), 279 deletions(-) create mode 100644 modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java create mode 100644 modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java create mode 100644 modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java create mode 100644 modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java rename modules/grouping/src/java/org/apache/lucene/search/grouping/{ => term}/TermAllGroupHeadsCollector.java (98%) rename modules/grouping/src/java/org/apache/lucene/search/grouping/{ => term}/TermAllGroupsCollector.java (95%) rename modules/grouping/src/java/org/apache/lucene/search/grouping/{ => term}/TermFirstPassGroupingCollector.java (92%) rename modules/grouping/src/java/org/apache/lucene/search/grouping/{ => term}/TermSecondPassGroupingCollector.java (88%) rename modules/grouping/src/test/org/apache/lucene/search/grouping/{TermAllGroupHeadsCollectorTest.java => AllGroupHeadsCollectorTest.java} (93%) rename modules/grouping/src/test/org/apache/lucene/search/grouping/{TermAllGroupsCollectorTest.java => AllGroupsCollectorTest.java} (81%) create mode 100644 modules/queries/src/java/org/apache/lucene/queries/function/valuesource/BytesRefFieldSource.java diff --git a/dev-tools/idea/modules/grouping/grouping.iml b/dev-tools/idea/modules/grouping/grouping.iml index bbd6eabbd4e..2e0b6ff146b 100644 --- a/dev-tools/idea/modules/grouping/grouping.iml +++ b/dev-tools/idea/modules/grouping/grouping.iml @@ -12,5 +12,6 @@ + diff --git a/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java b/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java index 36caf0b5082..e352807b033 100644 --- a/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java +++ b/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java @@ -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 diff --git a/modules/build.xml b/modules/build.xml index 62db70ab0ed..6e7dc8c4d43 100644 --- a/modules/build.xml +++ b/modules/build.xml @@ -25,9 +25,9 @@ + - @@ -40,9 +40,9 @@ + - @@ -55,9 +55,9 @@ + - @@ -70,9 +70,9 @@ + - @@ -86,9 +86,9 @@ + - @@ -100,9 +100,9 @@ + - @@ -116,9 +116,9 @@ + - diff --git a/modules/grouping/CHANGES.txt b/modules/grouping/CHANGES.txt index 68e0cce51b6..014e77dd402 100644 --- a/modules/grouping/CHANGES.txt +++ b/modules/grouping/CHANGES.txt @@ -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) diff --git a/modules/grouping/build.xml b/modules/grouping/build.xml index 6aee1915476..b71f9d6b2c2 100644 --- a/modules/grouping/build.xml +++ b/modules/grouping/build.xml @@ -18,8 +18,9 @@ --> + - Collectors for grouping search results + Grouping module. Collectors for grouping search results @@ -29,5 +30,18 @@ + + + + + + + + + + + + + diff --git a/modules/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java b/modules/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java index 453607c3175..6f38fa6b666 100644 --- a/modules/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java +++ b/modules/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java @@ -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. * diff --git a/modules/grouping/src/java/org/apache/lucene/search/grouping/SentinelIntSet.java b/modules/grouping/src/java/org/apache/lucene/search/grouping/SentinelIntSet.java index 21da977fc95..a9c637027c2 100644 --- a/modules/grouping/src/java/org/apache/lucene/search/grouping/SentinelIntSet.java +++ b/modules/grouping/src/java/org/apache/lucene/search/grouping/SentinelIntSet.java @@ -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; diff --git a/modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java b/modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java new file mode 100644 index 00000000000..6acc90cf80a --- /dev/null +++ b/modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java @@ -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 { + + private final ValueSource groupBy; + private final Map vsContext; + private final Map 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(); + 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 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 { + + 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; + } + } +} diff --git a/modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java b/modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java new file mode 100644 index 00000000000..f17aba3af25 --- /dev/null +++ b/modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java @@ -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. + * + *

+ * Implementation detail: Uses {@link ValueSource} and {@link DocValues} to retrieve the + * field values to group by. + * + * @lucene.experimental + */ +public class FunctionAllGroupsCollector extends AbstractAllGroupsCollector { + + private final Map vsContext; + private final ValueSource groupBy; + private final SortedSet groups = new TreeSet(); + + 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 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(); + } +} diff --git a/modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java b/modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java new file mode 100644 index 00000000000..4ece6223f4e --- /dev/null +++ b/modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java @@ -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 { + + 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(); + } + +} diff --git a/modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java b/modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java new file mode 100644 index 00000000000..a5464701b77 --- /dev/null +++ b/modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java @@ -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 { + + 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> 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 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(); + } + +} diff --git a/modules/grouping/src/java/org/apache/lucene/search/grouping/package.html b/modules/grouping/src/java/org/apache/lucene/search/grouping/package.html index 4d9e4bc657a..647c2be1993 100644 --- a/modules/grouping/src/java/org/apache/lucene/search/grouping/package.html +++ b/modules/grouping/src/java/org/apache/lucene/search/grouping/package.html @@ -44,9 +44,9 @@ field fall into a single group.

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, FieldCache, etc.). FixedBitSet groupHeadsBitSet = c.retrieveGroupHeads(maxDoc) +

For each of the above collectors there is also a variant that works with ValueSource 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 + org.apache.lucene.search.grouping.function package. +

+ diff --git a/modules/grouping/src/java/org/apache/lucene/search/grouping/TermAllGroupHeadsCollector.java b/modules/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java similarity index 98% rename from modules/grouping/src/java/org/apache/lucene/search/grouping/TermAllGroupHeadsCollector.java rename to modules/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java index 50067d5fa0d..81f26b26e82 100644 --- a/modules/grouping/src/java/org/apache/lucene/search/grouping/TermAllGroupHeadsCollector.java +++ b/modules/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java @@ -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 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> 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> 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> 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> mvalSearchGroups = new ArrayList>(searchGroups.size()); + for (SearchGroup mergedTopGroup : searchGroups) { + SearchGroup sg = new SearchGroup(); + 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> 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> mutableValueGroups = ((FunctionFirstPassGroupingCollector) c).getTopGroups(groupOffset, fillFields); + if (mutableValueGroups == null) { + return null; + } + + List> groups = new ArrayList>(mutableValueGroups.size()); + for (SearchGroup mutableValueGroup : mutableValueGroups) { + SearchGroup sg = new SearchGroup(); + 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 getTopGroups(AbstractSecondPassGroupingCollector c, int withinGroupOffset) { + if (c.getClass().isAssignableFrom(TermSecondPassGroupingCollector.class)) { + return ((TermSecondPassGroupingCollector) c).getTopGroups(withinGroupOffset); + } else if (c.getClass().isAssignableFrom(FunctionSecondPassGroupingCollector.class)) { + TopGroups mvalTopGroups = ((FunctionSecondPassGroupingCollector) c).getTopGroups(withinGroupOffset); + List> groups = new ArrayList>(mvalTopGroups.groups.length); + for (GroupDocs mvalGd : mvalTopGroups.groups) { + BytesRef groupValue = mvalGd.groupValue.exists() ? ((MutableValueStr) mvalGd.groupValue).value : null; + groups.add(new GroupDocs(mvalGd.maxScore, mvalGd.totalHits, mvalGd.scoreDocs, groupValue, mvalGd.groupSortValues)); + } + return new TopGroups(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> topGroups = c1.getTopGroups(groupOffset, fillFields); - + final Collection> topGroups = getSearchGroups(c1, groupOffset, fillFields); + final TopGroups 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 topGroupsShards = searchShards(s, shards.subSearchers, query, groupSort, docSort, groupOffset, topNGroups, docOffset, docsPerGroup, getScores, getMaxScores); - - final TopGroups 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 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 tempTopGroups = c2.getTopGroups(docOffset); + TopGroups tempTopGroups = getTopGroups(c2, docOffset); groupsResult = new TopGroups(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>> shardGroups = new ArrayList>>(); + List firstPassGroupingCollectors = new ArrayList(); for(int shardIDX=0;shardIDX> topGroups = c.getTopGroups(0, true); + final Collection> 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[] shardTopGroups = new TopGroups[subSearchers.length]; for(int shardIDX=0;shardIDX