You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2017/04/10 10:11:49 UTC
[1/3] lucene-solr:master: LUCENE-7701: Refactor grouping collectors
Repository: lucene-solr
Updated Branches:
refs/heads/master 550a32495 -> caf329717
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGrouper.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGrouper.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGrouper.java
deleted file mode 100644
index 246ee78..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGrouper.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.lucene.search.grouping.term;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.AllGroupsCollector;
-import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
-import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
-import org.apache.lucene.search.grouping.Grouper;
-import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.BytesRef;
-
-/**
- * Collector factory for grouping by term
- */
-public class TermGrouper extends Grouper<BytesRef> {
-
- private final String field;
- private final int initialSize;
-
- /**
- * Create a new TermGrouper
- * @param field the field to group on
- */
- public TermGrouper(String field) {
- this(field, 128);
- }
-
- /**
- * Create a new TermGrouper
- * @param field the field to group on
- * @param initialSize the initial size of various internal datastructures
- */
- public TermGrouper(String field, int initialSize) {
- this.field = field;
- this.initialSize = initialSize;
- }
-
- @Override
- public FirstPassGroupingCollector<BytesRef> getFirstPassCollector(Sort sort, int count) throws IOException {
- return new TermFirstPassGroupingCollector(field, sort, count);
- }
-
- @Override
- public AllGroupHeadsCollector<BytesRef> getGroupHeadsCollector(Sort sort) {
- return TermAllGroupHeadsCollector.create(field, sort, initialSize);
- }
-
- @Override
- public AllGroupsCollector<BytesRef> getAllGroupsCollector() {
- return new TermAllGroupsCollector(field, initialSize);
- }
-
- @Override
- public SecondPassGroupingCollector<BytesRef> getSecondPassCollector(
- Collection<SearchGroup<BytesRef>> groups, Sort groupSort, Sort withinGroupSort,
- int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
- return new TermSecondPassGroupingCollector(field, groups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java
deleted file mode 100644
index 75d2210..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.term;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
-import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.SentinelIntSet;
-
-/**
- * Concrete implementation of {@link SecondPassGroupingCollector} that groups based on
- * field values and more specifically uses {@link SortedDocValues}
- * to collect grouped docs.
- *
- * @lucene.experimental
- */
-public class TermSecondPassGroupingCollector extends SecondPassGroupingCollector<BytesRef> {
-
- private final String groupField;
- private final SentinelIntSet ordSet;
-
- private SortedDocValues index;
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- public TermSecondPassGroupingCollector(String groupField, Collection<SearchGroup<BytesRef>> groups, Sort groupSort, Sort withinGroupSort,
- int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields)
- throws IOException {
- super(groups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
- this.groupField = groupField;
- this.ordSet = new SentinelIntSet(groupMap.size(), -2);
- super.groupDocs = (SearchGroupDocs<BytesRef>[]) new SearchGroupDocs[ordSet.keys.length];
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
- super.doSetNextReader(readerContext);
- index = DocValues.getSorted(readerContext.reader(), groupField);
-
- // Rebuild ordSet
- ordSet.clear();
- for (SearchGroupDocs<BytesRef> group : groupMap.values()) {
-// System.out.println(" group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
- int ord = group.groupValue == null ? -1 : index.lookupTerm(group.groupValue);
- if (group.groupValue == null || ord >= 0) {
- groupDocs[ordSet.put(ord)] = group;
- }
- }
- }
-
- @Override
- protected SearchGroupDocs<BytesRef> retrieveGroup(int doc) throws IOException {
- if (doc > index.docID()) {
- index.advance(doc);
- }
-
- int ord;
- if (doc == index.docID()) {
- ord = index.ordValue();
- } else {
- ord = -1;
- }
-
- int slot = ordSet.find(ord);
- if (slot >= 0) {
- return groupDocs[slot];
- }
- return null;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/package-info.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/package-info.java
deleted file mode 100644
index 2732011..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/package-info.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Support for grouping by indexed terms via {@link org.apache.lucene.index.DocValues}.
- */
-package org.apache.lucene.search.grouping.term;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
index 0c99456..5ab4b5f 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
@@ -49,8 +49,6 @@ import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TermQuery;
-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.Bits;
import org.apache.lucene.util.BytesRef;
@@ -513,19 +511,12 @@ public class AllGroupHeadsCollectorTest extends LuceneTestCase {
@SuppressWarnings({"unchecked","rawtypes"})
private AllGroupHeadsCollector<?> createRandomCollector(String groupField, Sort sortWithinGroup) {
- AllGroupHeadsCollector<?> collector;
if (random().nextBoolean()) {
ValueSource vs = new BytesRefFieldSource(groupField);
- collector = new FunctionAllGroupHeadsCollector(vs, new HashMap<>(), sortWithinGroup);
+ return AllGroupHeadsCollector.newCollector(new ValueSourceGroupSelector(vs, new HashMap<>()), sortWithinGroup);
} else {
- collector = TermAllGroupHeadsCollector.create(groupField, sortWithinGroup);
+ return AllGroupHeadsCollector.newCollector(new TermGroupSelector(groupField), sortWithinGroup);
}
-
- if (VERBOSE) {
- System.out.println("Selected implementation: " + collector.getClass().getSimpleName());
- }
-
- return collector;
}
private void addGroupField(Document doc, String groupField, String value, DocValuesType valueType) {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java
index ab70fad..0d777f6 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java
@@ -16,6 +16,8 @@
*/
package org.apache.lucene.search.grouping;
+import java.util.HashMap;
+
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@@ -28,14 +30,10 @@ 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.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
-import java.util.HashMap;
-
public class AllGroupsCollectorTest extends LuceneTestCase {
public void testTotalGroupCount() throws Exception {
@@ -124,19 +122,13 @@ public class AllGroupsCollectorTest extends LuceneTestCase {
}
private AllGroupsCollector<?> createRandomCollector(String groupField) {
- AllGroupsCollector<?> selected;
if (random().nextBoolean()) {
- selected = new TermAllGroupsCollector(groupField);
- } else {
- ValueSource vs = new BytesRefFieldSource(groupField);
- selected = new FunctionAllGroupsCollector(vs, new HashMap<>());
+ return new AllGroupsCollector<>(new TermGroupSelector(groupField));
}
-
- if (VERBOSE) {
- System.out.println("Selected implementation: " + selected.getClass().getName());
+ else {
+ ValueSource vs = new BytesRefFieldSource(groupField);
+ return new AllGroupsCollector<>(new ValueSourceGroupSelector(vs, new HashMap<>()));
}
-
- return selected;
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java
index b5d67cf..89d9a6e 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java
@@ -44,17 +44,12 @@ import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.grouping.function.FunctionDistinctValuesCollector;
-import org.apache.lucene.search.grouping.function.FunctionFirstPassGroupingCollector;
-import org.apache.lucene.search.grouping.term.TermDistinctValuesCollector;
-import org.apache.lucene.search.grouping.term.TermFirstPassGroupingCollector;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.mutable.MutableValue;
import org.apache.lucene.util.mutable.MutableValueStr;
-
public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
private final static NullComparator nullComparator = new NullComparator();
@@ -126,32 +121,27 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
IndexSearcher indexSearcher = newSearcher(w.getReader());
w.close();
- Comparator<DistinctValuesCollector.GroupCount<Comparable<Object>>> cmp = new Comparator<DistinctValuesCollector.GroupCount<Comparable<Object>>>() {
-
- @Override
- public int compare(DistinctValuesCollector.GroupCount<Comparable<Object>> groupCount1, DistinctValuesCollector.GroupCount<Comparable<Object>> groupCount2) {
- if (groupCount1.groupValue == null) {
- if (groupCount2.groupValue == null) {
- return 0;
- }
- return -1;
- } else if (groupCount2.groupValue == null) {
- return 1;
- } else {
- return groupCount1.groupValue.compareTo(groupCount2.groupValue);
+ Comparator<DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>>> cmp = (groupCount1, groupCount2) -> {
+ if (groupCount1.groupValue == null) {
+ if (groupCount2.groupValue == null) {
+ return 0;
}
+ return -1;
+ } else if (groupCount2.groupValue == null) {
+ return 1;
+ } else {
+ return groupCount1.groupValue.compareTo(groupCount2.groupValue);
}
-
};
// === Search for content:random
FirstPassGroupingCollector<Comparable<Object>> firstCollector = createRandomFirstPassCollector(new Sort(), GROUP_FIELD, 10);
indexSearcher.search(new TermQuery(new Term("content", "random")), firstCollector);
- DistinctValuesCollector<Comparable<Object>> distinctValuesCollector
- = createDistinctCountCollector(firstCollector, GROUP_FIELD, COUNT_FIELD);
+ DistinctValuesCollector<Comparable<Object>, Comparable<Object>> distinctValuesCollector
+ = createDistinctCountCollector(firstCollector, COUNT_FIELD);
indexSearcher.search(new TermQuery(new Term("content", "random")), distinctValuesCollector);
- List<DistinctValuesCollector.GroupCount<Comparable<Object>>> gcs = distinctValuesCollector.getGroups();
+ List<DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>>> gcs = distinctValuesCollector.getGroups();
Collections.sort(gcs, cmp);
assertEquals(4, gcs.size());
@@ -180,7 +170,7 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
// === Search for content:some
firstCollector = createRandomFirstPassCollector(new Sort(), GROUP_FIELD, 10);
indexSearcher.search(new TermQuery(new Term("content", "some")), firstCollector);
- distinctValuesCollector = createDistinctCountCollector(firstCollector, GROUP_FIELD, COUNT_FIELD);
+ distinctValuesCollector = createDistinctCountCollector(firstCollector, COUNT_FIELD);
indexSearcher.search(new TermQuery(new Term("content", "some")), distinctValuesCollector);
gcs = distinctValuesCollector.getGroups();
@@ -207,7 +197,7 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
// === Search for content:blob
firstCollector = createRandomFirstPassCollector(new Sort(), GROUP_FIELD, 10);
indexSearcher.search(new TermQuery(new Term("content", "blob")), firstCollector);
- distinctValuesCollector = createDistinctCountCollector(firstCollector, GROUP_FIELD, COUNT_FIELD);
+ distinctValuesCollector = createDistinctCountCollector(firstCollector, COUNT_FIELD);
indexSearcher.search(new TermQuery(new Term("content", "blob")), distinctValuesCollector);
gcs = distinctValuesCollector.getGroups();
@@ -240,15 +230,15 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
Sort groupSort = new Sort(new SortField("id", SortField.Type.STRING));
int topN = 1 + random.nextInt(10);
- List<DistinctValuesCollector.GroupCount<Comparable<?>>> expectedResult = createExpectedResult(context, term, groupSort, topN);
+ List<DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>>> expectedResult = createExpectedResult(context, term, groupSort, topN);
- FirstPassGroupingCollector<Comparable<?>> firstCollector = createRandomFirstPassCollector(groupSort, GROUP_FIELD, topN);
+ FirstPassGroupingCollector<Comparable<Object>> firstCollector = createRandomFirstPassCollector(groupSort, GROUP_FIELD, topN);
searcher.search(new TermQuery(new Term("content", term)), firstCollector);
- DistinctValuesCollector<Comparable<?>> distinctValuesCollector
- = createDistinctCountCollector(firstCollector, GROUP_FIELD, COUNT_FIELD);
+ DistinctValuesCollector<Comparable<Object>, Comparable<Object>> distinctValuesCollector
+ = createDistinctCountCollector(firstCollector, COUNT_FIELD);
searcher.search(new TermQuery(new Term("content", term)), distinctValuesCollector);
@SuppressWarnings("unchecked")
- List<DistinctValuesCollector.GroupCount<Comparable<?>>> actualResult = distinctValuesCollector.getGroups();
+ List<DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>>> actualResult = distinctValuesCollector.getGroups();
if (VERBOSE) {
System.out.println("Index iter=" + indexIter);
@@ -265,8 +255,8 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
assertEquals(expectedResult.size(), actualResult.size());
for (int i = 0; i < expectedResult.size(); i++) {
- DistinctValuesCollector.GroupCount<Comparable<?>> expected = expectedResult.get(i);
- DistinctValuesCollector.GroupCount<Comparable<?>> actual = actualResult.get(i);
+ DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>> expected = expectedResult.get(i);
+ DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>> actual = actualResult.get(i);
assertValues(expected.groupValue, actual.groupValue);
assertEquals(expected.uniqueValues.size(), actual.uniqueValues.size());
List<Comparable<?>> expectedUniqueValues = new ArrayList<>(expected.uniqueValues);
@@ -283,9 +273,9 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
}
}
- private void printGroups(List<? extends DistinctValuesCollector.GroupCount<Comparable<?>>> results) {
+ private void printGroups(List<DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>>> results) {
for(int i=0;i<results.size();i++) {
- DistinctValuesCollector.GroupCount<Comparable<?>> group = results.get(i);
+ DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>> group = results.get(i);
Object gv = group.groupValue;
if (gv instanceof BytesRef) {
System.out.println(i + ": groupValue=" + ((BytesRef) gv).utf8ToString());
@@ -350,15 +340,16 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
}
@SuppressWarnings({"unchecked","rawtypes"})
- private <T extends Comparable> DistinctValuesCollector<T> createDistinctCountCollector(FirstPassGroupingCollector<T> firstPassGroupingCollector,
- String groupField,
- String countField) throws IOException {
- Random random = random();
+ private <T extends Comparable<Object>, R extends Comparable<Object>> DistinctValuesCollector<T, R> createDistinctCountCollector(FirstPassGroupingCollector<T> firstPassGroupingCollector,
+ String countField) throws IOException {
Collection<SearchGroup<T>> searchGroups = firstPassGroupingCollector.getTopGroups(0, false);
- if (FunctionFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
- return (DistinctValuesCollector) new FunctionDistinctValuesCollector(new HashMap<>(), new BytesRefFieldSource(groupField), new BytesRefFieldSource(countField), (Collection) searchGroups);
+ GroupSelector<T> selector = firstPassGroupingCollector.getGroupSelector();
+ if (ValueSourceGroupSelector.class.isAssignableFrom(selector.getClass())) {
+ GroupSelector gs = new ValueSourceGroupSelector(new BytesRefFieldSource(countField), new HashMap<>());
+ return new DistinctValuesCollector<>(selector, searchGroups, gs);
} else {
- return (DistinctValuesCollector) new TermDistinctValuesCollector(groupField, countField, (Collection) searchGroups);
+ GroupSelector ts = new TermGroupSelector(countField);
+ return new DistinctValuesCollector<>(selector, searchGroups, ts);
}
}
@@ -366,21 +357,14 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
private <T> FirstPassGroupingCollector<T> createRandomFirstPassCollector(Sort groupSort, String groupField, int topNGroups) throws IOException {
Random random = random();
if (random.nextBoolean()) {
- return (FirstPassGroupingCollector<T>) new FunctionFirstPassGroupingCollector(new BytesRefFieldSource(groupField), new HashMap<>(), groupSort, topNGroups);
+ return (FirstPassGroupingCollector<T>) new FirstPassGroupingCollector<>(new ValueSourceGroupSelector(new BytesRefFieldSource(groupField), new HashMap<>()), groupSort, topNGroups);
} else {
- return (FirstPassGroupingCollector<T>) new TermFirstPassGroupingCollector(groupField, groupSort, topNGroups);
+ return (FirstPassGroupingCollector<T>) new FirstPassGroupingCollector<>(new TermGroupSelector(groupField), groupSort, topNGroups);
}
}
@SuppressWarnings({"unchecked","rawtypes"})
- private List<DistinctValuesCollector.GroupCount<Comparable<?>>> createExpectedResult(IndexContext context, String term, Sort groupSort, int topN) {
- class GroupCount extends DistinctValuesCollector.GroupCount<BytesRef> {
- GroupCount(BytesRef groupValue, Collection<BytesRef> uniqueValues) {
- super(groupValue);
- this.uniqueValues.addAll(uniqueValues);
- }
- }
-
+ private List<DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>>> createExpectedResult(IndexContext context, String term, Sort groupSort, int topN) {
List result = new ArrayList();
Map<String, Set<String>> groupCounts = context.searchTermToGroupCounts.get(term);
int i = 0;
@@ -392,7 +376,7 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
for (String val : groupCounts.get(group)) {
uniqueValues.add(val != null ? new BytesRef(val) : null);
}
- result.add(new GroupCount(group != null ? new BytesRef(group) : null, uniqueValues));
+ result.add(new DistinctValuesCollector.GroupCount(group != null ? new BytesRef(group) : null, uniqueValues));
}
return result;
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java
index c590502..968ce5a 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java
@@ -44,7 +44,6 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.grouping.term.TermGroupFacetCollector;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.TestUtil;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
index f079b85..b322fba 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
@@ -58,12 +58,6 @@ import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.Weight;
-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;
@@ -147,7 +141,7 @@ public class TestGrouping extends LuceneTestCase {
final FirstPassGroupingCollector<?> c1 = createRandomFirstPassCollector(groupField, groupSort, 10);
indexSearcher.search(new TermQuery(new Term("content", "random")), c1);
- final SecondPassGroupingCollector<?> c2 = createSecondPassCollector(c1, groupField, groupSort, Sort.RELEVANCE, 0, 5, true, true, true);
+ final TopGroupsCollector<?> c2 = createSecondPassCollector(c1, groupSort, Sort.RELEVANCE, 0, 5, true, true, true);
indexSearcher.search(new TermQuery(new Term("content", "random")), c2);
final TopGroups<?> groups = c2.getTopGroups(0);
@@ -196,31 +190,26 @@ public class TestGrouping extends LuceneTestCase {
}
private FirstPassGroupingCollector<?> createRandomFirstPassCollector(String groupField, Sort groupSort, int topDocs) throws IOException {
- FirstPassGroupingCollector<?> selected;
if (random().nextBoolean()) {
ValueSource vs = new BytesRefFieldSource(groupField);
- selected = new FunctionFirstPassGroupingCollector(vs, new HashMap<>(), groupSort, topDocs);
+ return new FirstPassGroupingCollector<>(new ValueSourceGroupSelector(vs, new HashMap<>()), groupSort, topDocs);
} else {
- selected = new TermFirstPassGroupingCollector(groupField, groupSort, topDocs);
+ return new FirstPassGroupingCollector<>(new TermGroupSelector(groupField), groupSort, topDocs);
}
- if (VERBOSE) {
- System.out.println("Selected implementation: " + selected.getClass().getName());
- }
- return selected;
}
private FirstPassGroupingCollector<?> createFirstPassCollector(String groupField, Sort groupSort, int topDocs, FirstPassGroupingCollector<?> firstPassGroupingCollector) throws IOException {
- if (TermFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
+ GroupSelector<?> selector = firstPassGroupingCollector.getGroupSelector();
+ if (TermGroupSelector.class.isAssignableFrom(selector.getClass())) {
ValueSource vs = new BytesRefFieldSource(groupField);
- return new FunctionFirstPassGroupingCollector(vs, new HashMap<>(), groupSort, topDocs);
+ return new FirstPassGroupingCollector<>(new ValueSourceGroupSelector(vs, new HashMap<>()), groupSort, topDocs);
} else {
- return new TermFirstPassGroupingCollector(groupField, groupSort, topDocs);
+ return new FirstPassGroupingCollector<>(new TermGroupSelector(groupField), groupSort, topDocs);
}
}
@SuppressWarnings({"unchecked","rawtypes"})
- private <T> SecondPassGroupingCollector<T> createSecondPassCollector(FirstPassGroupingCollector firstPassGroupingCollector,
- String groupField,
+ private <T> TopGroupsCollector<T> createSecondPassCollector(FirstPassGroupingCollector firstPassGroupingCollector,
Sort groupSort,
Sort sortWithinGroup,
int groupOffset,
@@ -229,19 +218,13 @@ public class TestGrouping extends LuceneTestCase {
boolean getMaxScores,
boolean fillSortFields) throws IOException {
- if (TermFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
- Collection<SearchGroup<BytesRef>> searchGroups = firstPassGroupingCollector.getTopGroups(groupOffset, fillSortFields);
- return (SecondPassGroupingCollector) new TermSecondPassGroupingCollector(groupField, searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup , getScores, getMaxScores, fillSortFields);
- } else {
- ValueSource vs = new BytesRefFieldSource(groupField);
- Collection<SearchGroup<MutableValue>> searchGroups = firstPassGroupingCollector.getTopGroups(groupOffset, fillSortFields);
- return (SecondPassGroupingCollector) new FunctionSecondPassGroupingCollector(searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields, vs, new HashMap());
- }
+ Collection<SearchGroup<T>> searchGroups = firstPassGroupingCollector.getTopGroups(groupOffset, fillSortFields);
+ return new TopGroupsCollector<>(firstPassGroupingCollector.getGroupSelector(), searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
}
// Basically converts searchGroups from MutableValue to BytesRef if grouping by ValueSource
@SuppressWarnings("unchecked")
- private SecondPassGroupingCollector<?> createSecondPassCollector(FirstPassGroupingCollector<?> firstPassGroupingCollector,
+ private TopGroupsCollector<?> createSecondPassCollector(FirstPassGroupingCollector<?> firstPassGroupingCollector,
String groupField,
Collection<SearchGroup<BytesRef>> searchGroups,
Sort groupSort,
@@ -250,8 +233,9 @@ public class TestGrouping extends LuceneTestCase {
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);
+ if (firstPassGroupingCollector.getGroupSelector().getClass().isAssignableFrom(TermGroupSelector.class)) {
+ GroupSelector<BytesRef> selector = (GroupSelector<BytesRef>) firstPassGroupingCollector.getGroupSelector();
+ return new TopGroupsCollector<>(selector, searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup , getScores, getMaxScores, fillSortFields);
} else {
ValueSource vs = new BytesRefFieldSource(groupField);
List<SearchGroup<MutableValue>> mvalSearchGroups = new ArrayList<>(searchGroups.size());
@@ -267,19 +251,14 @@ public class TestGrouping extends LuceneTestCase {
sg.sortValues = mergedTopGroup.sortValues;
mvalSearchGroups.add(sg);
}
-
- return new FunctionSecondPassGroupingCollector(mvalSearchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields, vs, new HashMap<>());
+ ValueSourceGroupSelector selector = new ValueSourceGroupSelector(vs, new HashMap<>());
+ return new TopGroupsCollector<>(selector, mvalSearchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
}
}
private AllGroupsCollector<?> createAllGroupsCollector(FirstPassGroupingCollector<?> 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<>());
- }
+ return new AllGroupsCollector<>(firstPassGroupingCollector.getGroupSelector());
}
private void compareGroupValue(String expected, GroupDocs<?> group) {
@@ -306,10 +285,12 @@ public class TestGrouping extends LuceneTestCase {
}
private Collection<SearchGroup<BytesRef>> getSearchGroups(FirstPassGroupingCollector<?> c, int groupOffset, boolean fillFields) throws IOException {
- if (TermFirstPassGroupingCollector.class.isAssignableFrom(c.getClass())) {
- return ((TermFirstPassGroupingCollector) c).getTopGroups(groupOffset, fillFields);
- } else if (FunctionFirstPassGroupingCollector.class.isAssignableFrom(c.getClass())) {
- Collection<SearchGroup<MutableValue>> mutableValueGroups = ((FunctionFirstPassGroupingCollector) c).getTopGroups(groupOffset, fillFields);
+ if (TermGroupSelector.class.isAssignableFrom(c.getGroupSelector().getClass())) {
+ FirstPassGroupingCollector<BytesRef> collector = (FirstPassGroupingCollector<BytesRef>) c;
+ return collector.getTopGroups(groupOffset, fillFields);
+ } else if (ValueSourceGroupSelector.class.isAssignableFrom(c.getGroupSelector().getClass())) {
+ FirstPassGroupingCollector<MutableValue> collector = (FirstPassGroupingCollector<MutableValue>) c;
+ Collection<SearchGroup<MutableValue>> mutableValueGroups = collector.getTopGroups(groupOffset, fillFields);
if (mutableValueGroups == null) {
return null;
}
@@ -328,11 +309,13 @@ public class TestGrouping extends LuceneTestCase {
}
@SuppressWarnings({"unchecked", "rawtypes"})
- private TopGroups<BytesRef> getTopGroups(SecondPassGroupingCollector 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);
+ private TopGroups<BytesRef> getTopGroups(TopGroupsCollector c, int withinGroupOffset) {
+ if (c.getGroupSelector().getClass().isAssignableFrom(TermGroupSelector.class)) {
+ TopGroupsCollector<BytesRef> collector = (TopGroupsCollector<BytesRef>) c;
+ return collector.getTopGroups(withinGroupOffset);
+ } else if (c.getGroupSelector().getClass().isAssignableFrom(ValueSourceGroupSelector.class)) {
+ TopGroupsCollector<MutableValue> collector = (TopGroupsCollector<MutableValue>) c;
+ TopGroups<MutableValue> mvalTopGroups = collector.getTopGroups(withinGroupOffset);
List<GroupDocs<BytesRef>> groups = new ArrayList<>(mvalTopGroups.groups.length);
for (GroupDocs<MutableValue> mvalGd : mvalTopGroups.groups) {
BytesRef groupValue = mvalGd.groupValue.exists() ? ((MutableValueStr) mvalGd.groupValue).value.get() : null;
@@ -952,8 +935,8 @@ 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, true, false);
- final SecondPassGroupingCollector<?> c2;
+ groupOffset, topNGroups, docOffset, docsPerGroup, getScores, getMaxScores, true, true);
+ final TopGroupsCollector<?> c2;
if (topGroups != null) {
if (VERBOSE) {
@@ -963,7 +946,7 @@ public class TestGrouping extends LuceneTestCase {
}
}
- c2 = createSecondPassCollector(c1, groupField, groupSort, docSort, groupOffset, docOffset + docsPerGroup, getScores, getMaxScores, fillFields);
+ c2 = createSecondPassCollector(c1, groupSort, docSort, groupOffset, docOffset + docsPerGroup, getScores, getMaxScores, fillFields);
if (doCache) {
if (cCache.isCached()) {
if (VERBOSE) {
@@ -1050,13 +1033,13 @@ public class TestGrouping extends LuceneTestCase {
final boolean needsScores = getScores || getMaxScores || docSort == null;
final BlockGroupingCollector c3 = new BlockGroupingCollector(groupSort, groupOffset+topNGroups, needsScores, sBlocks.createNormalizedWeight(lastDocInBlock, false));
- final TermAllGroupsCollector allGroupsCollector2;
+ final AllGroupsCollector<BytesRef> allGroupsCollector2;
final Collector c4;
if (doAllGroups) {
// NOTE: must be "group" and not "group_dv"
// (groupField) because we didn't index doc
// values in the block index:
- allGroupsCollector2 = new TermAllGroupsCollector("group");
+ allGroupsCollector2 = new AllGroupsCollector<>(new TermGroupSelector("group"));
c4 = MultiCollector.wrap(c3, allGroupsCollector2);
} else {
allGroupsCollector2 = null;
@@ -1223,7 +1206,7 @@ public class TestGrouping extends LuceneTestCase {
@SuppressWarnings({"unchecked","rawtypes"})
final TopGroups<BytesRef>[] shardTopGroups = new TopGroups[subSearchers.length];
for(int shardIDX=0;shardIDX<subSearchers.length;shardIDX++) {
- final SecondPassGroupingCollector<?> secondPassCollector = createSecondPassCollector(firstPassGroupingCollectors.get(shardIDX),
+ final TopGroupsCollector<?> secondPassCollector = createSecondPassCollector(firstPassGroupingCollectors.get(shardIDX),
groupField, mergedTopGroups, groupSort, docSort, docOffset + topNDocs, getScores, getMaxScores, true);
subSearchers[shardIDX].search(w, secondPassCollector);
shardTopGroups[shardIDX] = getTopGroups(secondPassCollector, 0);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/solr/contrib/velocity/src/java/org/apache/solr/response/SolrVelocityResourceLoader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/velocity/src/java/org/apache/solr/response/SolrVelocityResourceLoader.java b/solr/contrib/velocity/src/java/org/apache/solr/response/SolrVelocityResourceLoader.java
index a659d0c..c83a5a7 100644
--- a/solr/contrib/velocity/src/java/org/apache/solr/response/SolrVelocityResourceLoader.java
+++ b/solr/contrib/velocity/src/java/org/apache/solr/response/SolrVelocityResourceLoader.java
@@ -16,15 +16,15 @@
*/
package org.apache.solr.response;
-import org.apache.velocity.runtime.resource.loader.ResourceLoader;
-import org.apache.velocity.runtime.resource.Resource;
-import org.apache.velocity.exception.ResourceNotFoundException;
-import org.apache.commons.collections.ExtendedProperties;
-import org.apache.solr.core.SolrResourceLoader;
-
import java.io.IOException;
import java.io.InputStream;
+import org.apache.commons.collections.ExtendedProperties;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.velocity.exception.ResourceNotFoundException;
+import org.apache.velocity.runtime.resource.Resource;
+import org.apache.velocity.runtime.resource.loader.ResourceLoader;
+
/**
* Velocity resource loader wrapper around Solr resource loader
*/
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
index 0bfef4c..406a4ed 100644
--- a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
@@ -52,8 +52,9 @@ import org.apache.lucene.search.FilterCollector;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.term.TermAllGroupsCollector;
-import org.apache.lucene.search.grouping.term.TermGroupFacetCollector;
+import org.apache.lucene.search.grouping.AllGroupsCollector;
+import org.apache.lucene.search.grouping.TermGroupFacetCollector;
+import org.apache.lucene.search.grouping.TermGroupSelector;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRefBuilder;
import org.apache.lucene.util.StringHelper;
@@ -330,7 +331,7 @@ public class SimpleFacets {
);
}
- TermAllGroupsCollector collector = new TermAllGroupsCollector(groupField);
+ AllGroupsCollector collector = new AllGroupsCollector<>(new TermGroupSelector(groupField));
Filter mainQueryFilter = docSet.getTopFilter(); // This returns a filter that only matches documents matching with q param and fq params
Query filteredFacetQuery = new BooleanQuery.Builder()
.add(facetQuery, Occur.MUST)
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/solr/core/src/java/org/apache/solr/search/Grouping.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/Grouping.java b/solr/core/src/java/org/apache/solr/search/Grouping.java
index 302383a..eeb6b66 100644
--- a/solr/core/src/java/org/apache/solr/search/Grouping.java
+++ b/solr/core/src/java/org/apache/solr/search/Grouping.java
@@ -47,17 +47,14 @@ import org.apache.lucene.search.TopFieldCollector;
import org.apache.lucene.search.TopScoreDocCollector;
import org.apache.lucene.search.TotalHitCountCollector;
import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
+import org.apache.lucene.search.grouping.AllGroupsCollector;
+import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
import org.apache.lucene.search.grouping.GroupDocs;
import org.apache.lucene.search.grouping.SearchGroup;
+import org.apache.lucene.search.grouping.TermGroupSelector;
import org.apache.lucene.search.grouping.TopGroups;
-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.search.grouping.TopGroupsCollector;
+import org.apache.lucene.search.grouping.ValueSourceGroupSelector;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.mutable.MutableValue;
import org.apache.solr.common.SolrException;
@@ -700,10 +697,10 @@ public class Grouping {
public class CommandField extends Command<BytesRef> {
public String groupBy;
- TermFirstPassGroupingCollector firstPass;
- TermSecondPassGroupingCollector secondPass;
+ FirstPassGroupingCollector<BytesRef> firstPass;
+ TopGroupsCollector<BytesRef> secondPass;
- TermAllGroupsCollector allGroupsCollector;
+ AllGroupsCollector<BytesRef> allGroupsCollector;
// If offset falls outside the number of documents a group can provide use this collector instead of secondPass
TotalHitCountCollector fallBackCollector;
@@ -729,7 +726,7 @@ public class Grouping {
}
groupSort = groupSort == null ? Sort.RELEVANCE : groupSort;
- firstPass = new TermFirstPassGroupingCollector(groupBy, groupSort, actualGroupsToFind);
+ firstPass = new FirstPassGroupingCollector<>(new TermGroupSelector(groupBy), groupSort, actualGroupsToFind);
return firstPass;
}
@@ -739,14 +736,14 @@ public class Grouping {
@Override
protected Collector createSecondPassCollector() throws IOException {
if (actualGroupsToFind <= 0) {
- allGroupsCollector = new TermAllGroupsCollector(groupBy);
+ allGroupsCollector = new AllGroupsCollector<>(new TermGroupSelector(groupBy));
return totalCount == TotalCount.grouped ? allGroupsCollector : null;
}
topGroups = format == Format.grouped ? firstPass.getTopGroups(offset, false) : firstPass.getTopGroups(0, false);
if (topGroups == null) {
if (totalCount == TotalCount.grouped) {
- allGroupsCollector = new TermAllGroupsCollector(groupBy);
+ allGroupsCollector = new AllGroupsCollector<>(new TermGroupSelector(groupBy));
fallBackCollector = new TotalHitCountCollector();
return MultiCollector.wrap(allGroupsCollector, fallBackCollector);
} else {
@@ -758,12 +755,12 @@ public class Grouping {
int groupedDocsToCollect = getMax(groupOffset, docsPerGroup, maxDoc);
groupedDocsToCollect = Math.max(groupedDocsToCollect, 1);
Sort withinGroupSort = this.withinGroupSort != null ? this.withinGroupSort : Sort.RELEVANCE;
- secondPass = new TermSecondPassGroupingCollector(
- groupBy, topGroups, groupSort, withinGroupSort, groupedDocsToCollect, needScores, needScores, false
+ secondPass = new TopGroupsCollector<>(new TermGroupSelector(groupBy),
+ topGroups, groupSort, withinGroupSort, groupedDocsToCollect, needScores, needScores, false
);
if (totalCount == TotalCount.grouped) {
- allGroupsCollector = new TermAllGroupsCollector(groupBy);
+ allGroupsCollector = new AllGroupsCollector<>(new TermGroupSelector(groupBy));
return MultiCollector.wrap(secondPass, allGroupsCollector);
} else {
return secondPass;
@@ -776,7 +773,7 @@ public class Grouping {
@Override
public AllGroupHeadsCollector<?> createAllGroupCollector() throws IOException {
Sort sortWithinGroup = withinGroupSort != null ? withinGroupSort : Sort.RELEVANCE;
- return TermAllGroupHeadsCollector.create(groupBy, sortWithinGroup);
+ return AllGroupHeadsCollector.newCollector(new TermGroupSelector(groupBy), sortWithinGroup);
}
/**
@@ -921,11 +918,15 @@ public class Grouping {
public ValueSource groupBy;
Map context;
- FunctionFirstPassGroupingCollector firstPass;
- FunctionSecondPassGroupingCollector secondPass;
+ private ValueSourceGroupSelector newSelector() {
+ return new ValueSourceGroupSelector(groupBy, context);
+ }
+
+ FirstPassGroupingCollector<MutableValue> firstPass;
+ TopGroupsCollector<MutableValue> secondPass;
// If offset falls outside the number of documents a group can provide use this collector instead of secondPass
TotalHitCountCollector fallBackCollector;
- FunctionAllGroupsCollector allGroupsCollector;
+ AllGroupsCollector<MutableValue> allGroupsCollector;
Collection<SearchGroup<MutableValue>> topGroups;
/**
@@ -950,7 +951,7 @@ public class Grouping {
}
groupSort = groupSort == null ? Sort.RELEVANCE : groupSort;
- firstPass = new FunctionFirstPassGroupingCollector(groupBy, context, searcher.weightSort(groupSort), actualGroupsToFind);
+ firstPass = new FirstPassGroupingCollector<>(newSelector(), searcher.weightSort(groupSort), actualGroupsToFind);
return firstPass;
}
@@ -960,14 +961,14 @@ public class Grouping {
@Override
protected Collector createSecondPassCollector() throws IOException {
if (actualGroupsToFind <= 0) {
- allGroupsCollector = new FunctionAllGroupsCollector(groupBy, context);
+ allGroupsCollector = new AllGroupsCollector<>(newSelector());
return totalCount == TotalCount.grouped ? allGroupsCollector : null;
}
topGroups = format == Format.grouped ? firstPass.getTopGroups(offset, false) : firstPass.getTopGroups(0, false);
if (topGroups == null) {
if (totalCount == TotalCount.grouped) {
- allGroupsCollector = new FunctionAllGroupsCollector(groupBy, context);
+ allGroupsCollector = new AllGroupsCollector<>(newSelector());
fallBackCollector = new TotalHitCountCollector();
return MultiCollector.wrap(allGroupsCollector, fallBackCollector);
} else {
@@ -979,12 +980,12 @@ public class Grouping {
int groupdDocsToCollect = getMax(groupOffset, docsPerGroup, maxDoc);
groupdDocsToCollect = Math.max(groupdDocsToCollect, 1);
Sort withinGroupSort = this.withinGroupSort != null ? this.withinGroupSort : Sort.RELEVANCE;
- secondPass = new FunctionSecondPassGroupingCollector(
- topGroups, groupSort, withinGroupSort, groupdDocsToCollect, needScores, needScores, false, groupBy, context
+ secondPass = new TopGroupsCollector<>(newSelector(),
+ topGroups, groupSort, withinGroupSort, groupdDocsToCollect, needScores, needScores, false
);
if (totalCount == TotalCount.grouped) {
- allGroupsCollector = new FunctionAllGroupsCollector(groupBy, context);
+ allGroupsCollector = new AllGroupsCollector<>(newSelector());
return MultiCollector.wrap(secondPass, allGroupsCollector);
} else {
return secondPass;
@@ -994,7 +995,7 @@ public class Grouping {
@Override
public AllGroupHeadsCollector<?> createAllGroupCollector() throws IOException {
Sort sortWithinGroup = withinGroupSort != null ? withinGroupSort : Sort.RELEVANCE;
- return new FunctionAllGroupHeadsCollector(groupBy, context, sortWithinGroup);
+ return AllGroupHeadsCollector.newCollector(newSelector(), sortWithinGroup);
}
/**
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java b/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java
index 8ba0a6a..4ec01db 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java
@@ -32,8 +32,8 @@ import org.apache.lucene.search.Query;
import org.apache.lucene.search.TimeLimitingCollector;
import org.apache.lucene.search.TotalHitCountCollector;
import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.function.FunctionAllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.term.TermAllGroupHeadsCollector;
+import org.apache.lucene.search.grouping.TermGroupSelector;
+import org.apache.lucene.search.grouping.ValueSourceGroupSelector;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.schema.FieldType;
import org.apache.solr.schema.SchemaField;
@@ -174,9 +174,11 @@ public class CommandHandler {
final AllGroupHeadsCollector allGroupHeadsCollector;
if (fieldType.getNumberType() != null) {
ValueSource vs = fieldType.getValueSource(sf, null);
- allGroupHeadsCollector = new FunctionAllGroupHeadsCollector(vs, new HashMap(), firstCommand.getWithinGroupSort());
+ allGroupHeadsCollector = AllGroupHeadsCollector.newCollector(new ValueSourceGroupSelector(vs, new HashMap<>()),
+ firstCommand.getWithinGroupSort());
} else {
- allGroupHeadsCollector = TermAllGroupHeadsCollector.create(firstCommand.getKey(), firstCommand.getWithinGroupSort());
+ allGroupHeadsCollector
+ = AllGroupHeadsCollector.newCollector(new TermGroupSelector(firstCommand.getKey()), firstCommand.getWithinGroupSort());
}
if (collectors.isEmpty()) {
searchWithTimeLimiter(query, filter, allGroupHeadsCollector);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java
index 7f92382..b81dda0 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java
@@ -16,24 +16,26 @@
*/
package org.apache.solr.search.grouping.distributed.command;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.grouping.AllGroupsCollector;
import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.search.grouping.function.FunctionAllGroupsCollector;
-import org.apache.lucene.search.grouping.function.FunctionFirstPassGroupingCollector;
-import org.apache.lucene.search.grouping.term.TermAllGroupsCollector;
-import org.apache.lucene.search.grouping.term.TermFirstPassGroupingCollector;
+import org.apache.lucene.search.grouping.TermGroupSelector;
+import org.apache.lucene.search.grouping.ValueSourceGroupSelector;
import org.apache.lucene.util.BytesRef;
import org.apache.solr.schema.FieldType;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.search.grouping.Command;
-import java.io.IOException;
-import java.util.*;
-
/**
* Creates all the collectors needed for the first phase and how to handle the results.
*/
@@ -98,18 +100,20 @@ public class SearchGroupsFieldCommand implements Command<SearchGroupsFieldComman
if (topNGroups > 0) {
if (fieldType.getNumberType() != null) {
ValueSource vs = fieldType.getValueSource(field, null);
- firstPassGroupingCollector = new FunctionFirstPassGroupingCollector(vs, new HashMap<Object,Object>(), groupSort, topNGroups);
+ firstPassGroupingCollector
+ = new FirstPassGroupingCollector<>(new ValueSourceGroupSelector(vs, new HashMap<>()), groupSort, topNGroups);
} else {
- firstPassGroupingCollector = new TermFirstPassGroupingCollector(field.getName(), groupSort, topNGroups);
+ firstPassGroupingCollector
+ = new FirstPassGroupingCollector<>(new TermGroupSelector(field.getName()), groupSort, topNGroups);
}
collectors.add(firstPassGroupingCollector);
}
if (includeGroupCount) {
if (fieldType.getNumberType() != null) {
ValueSource vs = fieldType.getValueSource(field, null);
- allGroupsCollector = new FunctionAllGroupsCollector(vs, new HashMap<Object,Object>());
+ allGroupsCollector = new AllGroupsCollector<>(new ValueSourceGroupSelector(vs, new HashMap<>()));
} else {
- allGroupsCollector = new TermAllGroupsCollector(field.getName());
+ allGroupsCollector = new AllGroupsCollector<>(new TermGroupSelector(field.getName()));
}
collectors.add(allGroupsCollector);
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java
index e34e278..80ea4cd 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java
@@ -16,28 +16,28 @@
*/
package org.apache.solr.search.grouping.distributed.command;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
import org.apache.lucene.search.grouping.GroupDocs;
import org.apache.lucene.search.grouping.SearchGroup;
+import org.apache.lucene.search.grouping.TermGroupSelector;
import org.apache.lucene.search.grouping.TopGroups;
-import org.apache.lucene.search.grouping.function.FunctionSecondPassGroupingCollector;
-import org.apache.lucene.search.grouping.term.TermSecondPassGroupingCollector;
+import org.apache.lucene.search.grouping.TopGroupsCollector;
+import org.apache.lucene.search.grouping.ValueSourceGroupSelector;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.mutable.MutableValue;
import org.apache.solr.schema.FieldType;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.search.grouping.Command;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-
/**
* Defines all collectors for retrieving the second phase and how to handle the collector result.
*/
@@ -106,7 +106,7 @@ public class TopGroupsFieldCommand implements Command<TopGroups<BytesRef>> {
private final int maxDocPerGroup;
private final boolean needScores;
private final boolean needMaxScore;
- private SecondPassGroupingCollector secondPassCollector;
+ private TopGroupsCollector secondPassCollector;
private TopGroupsFieldCommand(SchemaField field,
Sort groupSort,
@@ -135,12 +135,12 @@ public class TopGroupsFieldCommand implements Command<TopGroups<BytesRef>> {
if (fieldType.getNumberType() != null) {
ValueSource vs = fieldType.getValueSource(field, null);
Collection<SearchGroup<MutableValue>> v = GroupConverter.toMutable(field, firstPhaseGroups);
- secondPassCollector = new FunctionSecondPassGroupingCollector(
- v, groupSort, sortWithinGroup, maxDocPerGroup, needScores, needMaxScore, true, vs, new HashMap<Object,Object>()
+ secondPassCollector = new TopGroupsCollector<>(new ValueSourceGroupSelector(vs, new HashMap<>()),
+ v, groupSort, sortWithinGroup, maxDocPerGroup, needScores, needMaxScore, true
);
} else {
- secondPassCollector = new TermSecondPassGroupingCollector(
- field.getName(), firstPhaseGroups, groupSort, sortWithinGroup, maxDocPerGroup, needScores, needMaxScore, true
+ secondPassCollector = new TopGroupsCollector<>(new TermGroupSelector(field.getName()),
+ firstPhaseGroups, groupSort, sortWithinGroup, maxDocPerGroup, needScores, needMaxScore, true
);
}
collectors.add(secondPassCollector);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/TestGroupingSearch.java b/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
index e659727..2d46551 100644
--- a/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
+++ b/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
@@ -838,7 +838,7 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
Object realResponse = ObjectBuilder.fromJSON(strResponse);
String err = JSONTestUtil.matchObj("/grouped/" + groupField, realResponse, modelResponse);
if (err != null) {
- log.error("GROUPING MISMATCH: " + err
+ log.error("GROUPING MISMATCH (" + queryIter + "): " + err
+ "\n\trequest="+req
+ "\n\tresult="+strResponse
+ "\n\texpected="+ JSONUtil.toJSON(modelResponse)
@@ -854,7 +854,7 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
// assert post / pre grouping facets
err = JSONTestUtil.matchObj("/facet_counts/facet_fields/"+FOO_STRING_FIELD, realResponse, expectedFacetResponse);
if (err != null) {
- log.error("GROUPING MISMATCH: " + err
+ log.error("GROUPING MISMATCH (" + queryIter + "): " + err
+ "\n\trequest="+req
+ "\n\tresult="+strResponse
+ "\n\texpected="+ JSONUtil.toJSON(expectedFacetResponse)
[3/3] lucene-solr:master: LUCENE-7701: Refactor grouping collectors
Posted by ro...@apache.org.
LUCENE-7701: Refactor grouping collectors
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/caf32971
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/caf32971
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/caf32971
Branch: refs/heads/master
Commit: caf329717ef643b2a5a8b7ef0bfb7bd5e096c849
Parents: 550a324
Author: Alan Woodward <ro...@apache.org>
Authored: Tue Mar 28 19:52:53 2017 +0100
Committer: Alan Woodward <ro...@apache.org>
Committed: Mon Apr 10 11:10:18 2017 +0100
----------------------------------------------------------------------
lucene/CHANGES.txt | 3 +
lucene/MIGRATE.txt | 13 +
.../apache/lucene/search/TopFieldCollector.java | 4 +-
.../search/grouping/AllGroupHeadsCollector.java | 239 +++++-
.../search/grouping/AllGroupsCollector.java | 42 +-
.../search/grouping/BlockGroupingCollector.java | 2 +-
.../grouping/DistinctValuesCollector.java | 92 ++-
.../grouping/FirstPassGroupingCollector.java | 63 +-
.../lucene/search/grouping/GroupReducer.java | 112 +++
.../lucene/search/grouping/GroupSelector.java | 73 ++
.../apache/lucene/search/grouping/Grouper.java | 56 --
.../lucene/search/grouping/GroupingSearch.java | 24 +-
.../grouping/SecondPassGroupingCollector.java | 144 +---
.../grouping/TermGroupFacetCollector.java | 414 ++++++++++
.../search/grouping/TermGroupSelector.java | 114 +++
.../search/grouping/TopGroupsCollector.java | 116 +++
.../grouping/ValueSourceGroupSelector.java | 86 +++
.../FunctionAllGroupHeadsCollector.java | 159 ----
.../function/FunctionAllGroupsCollector.java | 82 --
.../FunctionDistinctValuesCollector.java | 85 --
.../FunctionFirstPassGroupingCollector.java | 86 ---
.../grouping/function/FunctionGrouper.java | 69 --
.../FunctionSecondPassGroupingCollector.java | 80 --
.../search/grouping/function/package-info.java | 21 -
.../lucene/search/grouping/package-info.java | 34 +-
.../term/TermAllGroupHeadsCollector.java | 767 -------------------
.../grouping/term/TermAllGroupsCollector.java | 128 ----
.../term/TermDistinctValuesCollector.java | 162 ----
.../term/TermFirstPassGroupingCollector.java | 96 ---
.../grouping/term/TermGroupFacetCollector.java | 415 ----------
.../search/grouping/term/TermGrouper.java | 81 --
.../term/TermSecondPassGroupingCollector.java | 91 ---
.../search/grouping/term/package-info.java | 21 -
.../grouping/AllGroupHeadsCollectorTest.java | 13 +-
.../search/grouping/AllGroupsCollectorTest.java | 20 +-
.../grouping/DistinctValuesCollectorTest.java | 86 +--
.../grouping/GroupFacetCollectorTest.java | 1 -
.../lucene/search/grouping/TestGrouping.java | 89 +--
.../response/SolrVelocityResourceLoader.java | 12 +-
.../org/apache/solr/request/SimpleFacets.java | 7 +-
.../java/org/apache/solr/search/Grouping.java | 57 +-
.../solr/search/grouping/CommandHandler.java | 10 +-
.../command/SearchGroupsFieldCommand.java | 26 +-
.../command/TopGroupsFieldCommand.java | 30 +-
.../org/apache/solr/TestGroupingSearch.java | 4 +-
45 files changed, 1501 insertions(+), 2828 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 1f3f30c..cd25dee 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -47,6 +47,9 @@ API Changes
* LUCENE-7734: FieldType's copy constructor was widened to accept any IndexableFieldType.
(David Smiley)
+* LUCENE-7701: Grouping collectors have been refactored, such that groups are
+ now defined by a GroupSelector implementation. (Alan Woodward)
+
Bug Fixes
* LUCENE-7626: IndexWriter will no longer accept broken token offsets
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/MIGRATE.txt
----------------------------------------------------------------------
diff --git a/lucene/MIGRATE.txt b/lucene/MIGRATE.txt
index 1ccf56f..c7936a4 100644
--- a/lucene/MIGRATE.txt
+++ b/lucene/MIGRATE.txt
@@ -61,3 +61,16 @@ do not take deleted docs and doc values updates into account.
Index-time boosts are not supported anymore. As a replacement, index-time
scoring factors should be indexed in a doc value field and combined with the
score at query time using FunctionScoreQuery for instance.
+
+## Grouping collector refactoring (LUCENE-7701)
+
+Groups are now defined by GroupSelector classes, making it easier to define new
+types of groups. Rather than having term or function specific collection
+classes, FirstPassGroupingCollector, AllGroupsCollector and
+AllGroupHeadsCollector are now concrete classes taking a GroupSelector.
+
+SecondPassGroupingCollector is no longer specifically aimed at
+collecting TopDocs for each group, but instead takes a GroupReducer that will
+perform any type of reduction on the top groups collected on a first-pass. To
+reproduce the old behaviour of SecondPassGroupingCollector, you should instead
+use TopGroupsCollector.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
index 1ec322f..97589c4 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
@@ -320,11 +320,9 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
* <code>trackDocScores</code> to true as well.
* @return a {@link TopFieldCollector} instance which will sort the results by
* the sort criteria.
- * @throws IOException if there is a low-level I/O error
*/
public static TopFieldCollector create(Sort sort, int numHits,
- boolean fillFields, boolean trackDocScores, boolean trackMaxScore)
- throws IOException {
+ boolean fillFields, boolean trackDocScores, boolean trackMaxScore) {
return create(sort, numHits, null, fillFields, trackDocScores, trackMaxScore);
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java
index b5fbdc3..503b952 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java
@@ -18,27 +18,62 @@ package org.apache.lucene.search.grouping;
import java.io.IOException;
import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.LeafFieldComparator;
+import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
import org.apache.lucene.util.FixedBitSet;
/**
- * This collector specializes in collecting the most relevant document (group head) for each group that match the query.
+ * This collector specializes in collecting the most relevant document (group head) for each
+ * group that matches the query.
+ *
+ * Clients should create new collectors by calling {@link #newCollector(GroupSelector, Sort)}
*
* @lucene.experimental
*/
@SuppressWarnings({"unchecked","rawtypes"})
public abstract class AllGroupHeadsCollector<T> extends SimpleCollector {
+ private final GroupSelector<T> groupSelector;
+ protected final Sort sort;
+
protected final int[] reversed;
protected final int compIDXEnd;
- protected final TemporalResult temporalResult;
- protected AllGroupHeadsCollector(int numberOfSorts) {
- this.reversed = new int[numberOfSorts];
- this.compIDXEnd = numberOfSorts - 1;
- temporalResult = new TemporalResult();
+ protected Map<T, GroupHead<T>> heads = new HashMap<>();
+
+ protected LeafReaderContext context;
+ protected Scorer scorer;
+
+ /**
+ * Create a new AllGroupHeadsCollector based on the type of within-group Sort required
+ * @param selector a GroupSelector to define the groups
+ * @param sort the within-group sort to use to choose the group head document
+ * @param <T> the group value type
+ */
+ public static <T> AllGroupHeadsCollector<T> newCollector(GroupSelector<T> selector, Sort sort) {
+ if (sort.equals(Sort.RELEVANCE))
+ return new ScoringGroupHeadsCollector<>(selector, sort);
+ return new SortingGroupHeadsCollector<>(selector, sort);
+ }
+
+ private AllGroupHeadsCollector(GroupSelector<T> selector, Sort sort) {
+ this.groupSelector = selector;
+ this.sort = sort;
+ this.reversed = new int[sort.getSort().length];
+ final SortField[] sortFields = sort.getSort();
+ for (int i = 0; i < sortFields.length; i++) {
+ reversed[i] = sortFields[i].getReverse() ? -1 : 1;
+ }
+ this.compIDXEnd = this.reversed.length - 1;
}
/**
@@ -79,34 +114,27 @@ public abstract class AllGroupHeadsCollector<T> extends SimpleCollector {
}
/**
- * Returns the group head and puts it into {@link #temporalResult}.
- * If the group head wasn't encountered before then it will be added to the collected group heads.
- * <p>
- * The {@link TemporalResult#stop} property will be <code>true</code> if the group head wasn't encountered before
- * otherwise <code>false</code>.
- *
- * @param doc The document to retrieve the group head for.
- * @throws IOException If I/O related errors occur
- */
- protected abstract void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException;
-
- /**
* Returns the collected group heads.
* Subsequent calls should return the same group heads.
*
* @return the collected group heads
*/
- protected abstract Collection<? extends GroupHead<T>> getCollectedGroupHeads();
+ protected Collection<? extends GroupHead<T>> getCollectedGroupHeads() {
+ return heads.values();
+ }
@Override
public void collect(int doc) throws IOException {
- retrieveGroupHeadAndAddIfNotExist(doc);
- if (temporalResult.stop) {
+ groupSelector.advanceTo(doc);
+ T groupValue = groupSelector.currentValue();
+ if (heads.containsKey(groupValue) == false) {
+ groupValue = groupSelector.copyValue();
+ heads.put(groupValue, newGroupHead(doc, groupValue, context, scorer));
return;
}
- GroupHead<T> groupHead = temporalResult.groupHead;
- // Ok now we need to check if the current doc is more relevant then current doc for this group
+ GroupHead<T> groupHead = heads.get(groupValue);
+ // Ok now we need to check if the current doc is more relevant than top doc for this group
for (int compIDX = 0; ; compIDX++) {
final int c = reversed[compIDX] * groupHead.compare(compIDX, doc);
if (c < 0) {
@@ -125,18 +153,34 @@ public abstract class AllGroupHeadsCollector<T> extends SimpleCollector {
groupHead.updateDocHead(doc);
}
- /**
- * Contains the result of group head retrieval.
- * To prevent new object creations of this class for every collect.
- */
- protected class TemporalResult {
+ @Override
+ public boolean needsScores() {
+ return sort.needsScores();
+ }
- public GroupHead<T> groupHead;
- public boolean stop;
+ @Override
+ protected void doSetNextReader(LeafReaderContext context) throws IOException {
+ groupSelector.setNextReader(context);
+ this.context = context;
+ for (GroupHead<T> head : heads.values()) {
+ head.setNextReader(context);
+ }
+ }
+ @Override
+ public void setScorer(Scorer scorer) throws IOException {
+ this.scorer = scorer;
+ for (GroupHead<T> head : heads.values()) {
+ head.setScorer(scorer);
+ }
}
/**
+ * Create a new GroupHead for the given group value, initialized with a doc, context and scorer
+ */
+ protected abstract GroupHead<T> newGroupHead(int doc, T value, LeafReaderContext context, Scorer scorer) throws IOException;
+
+ /**
* Represents a group head. A group head is the most relevant document for a particular group.
* The relevancy is based is usually based on the sort.
*
@@ -147,12 +191,30 @@ public abstract class AllGroupHeadsCollector<T> extends SimpleCollector {
public final T groupValue;
public int doc;
- protected GroupHead(T groupValue, int doc) {
+ protected int docBase;
+
+ /**
+ * Create a new GroupHead for the given value
+ */
+ protected GroupHead(T groupValue, int doc, int docBase) {
this.groupValue = groupValue;
- this.doc = doc;
+ this.doc = doc + docBase;
+ this.docBase = docBase;
+ }
+
+ /**
+ * Called for each segment
+ */
+ protected void setNextReader(LeafReaderContext ctx) throws IOException {
+ this.docBase = ctx.docBase;
}
/**
+ * Called for each segment
+ */
+ protected abstract void setScorer(Scorer scorer) throws IOException;
+
+ /**
* Compares the specified document for a specified comparator against the current most relevant document.
*
* @param compIDX The comparator index of the specified comparator.
@@ -173,4 +235,117 @@ public abstract class AllGroupHeadsCollector<T> extends SimpleCollector {
}
+ /**
+ * General implementation using a {@link FieldComparator} to select the group head
+ */
+ private static class SortingGroupHeadsCollector<T> extends AllGroupHeadsCollector<T> {
+
+ protected SortingGroupHeadsCollector(GroupSelector<T> selector, Sort sort) {
+ super(selector, sort);
+ }
+
+ @Override
+ protected GroupHead<T> newGroupHead(int doc, T value, LeafReaderContext ctx, Scorer scorer) throws IOException {
+ return new SortingGroupHead<>(sort, value, doc, ctx, scorer);
+ }
+ }
+
+ private static class SortingGroupHead<T> extends GroupHead<T> {
+
+ final FieldComparator[] comparators;
+ final LeafFieldComparator[] leafComparators;
+
+ protected SortingGroupHead(Sort sort, T groupValue, int doc, LeafReaderContext context, Scorer scorer) throws IOException {
+ super(groupValue, doc, context.docBase);
+ final SortField[] sortFields = sort.getSort();
+ comparators = new FieldComparator[sortFields.length];
+ leafComparators = new LeafFieldComparator[sortFields.length];
+ for (int i = 0; i < sortFields.length; i++) {
+ comparators[i] = sortFields[i].getComparator(1, i);
+ leafComparators[i] = comparators[i].getLeafComparator(context);
+ leafComparators[i].setScorer(scorer);
+ leafComparators[i].copy(0, doc);
+ leafComparators[i].setBottom(0);
+ }
+ }
+
+ @Override
+ public void setNextReader(LeafReaderContext ctx) throws IOException {
+ super.setNextReader(ctx);
+ for (int i = 0; i < comparators.length; i++) {
+ leafComparators[i] = comparators[i].getLeafComparator(ctx);
+ }
+ }
+
+ @Override
+ protected void setScorer(Scorer scorer) throws IOException {
+ for (LeafFieldComparator c : leafComparators) {
+ c.setScorer(scorer);
+ }
+ }
+
+ @Override
+ public int compare(int compIDX, int doc) throws IOException {
+ return leafComparators[compIDX].compareBottom(doc);
+ }
+
+ @Override
+ public void updateDocHead(int doc) throws IOException {
+ for (LeafFieldComparator comparator : leafComparators) {
+ comparator.copy(0, doc);
+ comparator.setBottom(0);
+ }
+ this.doc = doc + docBase;
+ }
+ }
+
+ /**
+ * Specialized implementation for sorting by score
+ */
+ private static class ScoringGroupHeadsCollector<T> extends AllGroupHeadsCollector<T> {
+
+ protected ScoringGroupHeadsCollector(GroupSelector<T> selector, Sort sort) {
+ super(selector, sort);
+ }
+
+ @Override
+ protected GroupHead<T> newGroupHead(int doc, T value, LeafReaderContext context, Scorer scorer) throws IOException {
+ return new ScoringGroupHead<>(scorer, value, doc, context.docBase);
+ }
+ }
+
+ private static class ScoringGroupHead<T> extends GroupHead<T> {
+
+ private Scorer scorer;
+ private float topScore;
+
+ protected ScoringGroupHead(Scorer scorer, T groupValue, int doc, int docBase) throws IOException {
+ super(groupValue, doc, docBase);
+ assert scorer.docID() == doc;
+ this.scorer = scorer;
+ this.topScore = scorer.score();
+ }
+
+ @Override
+ protected void setScorer(Scorer scorer) {
+ this.scorer = scorer;
+ }
+
+ @Override
+ protected int compare(int compIDX, int doc) throws IOException {
+ assert scorer.docID() == doc;
+ assert compIDX == 0;
+ float score = scorer.score();
+ int c = Float.compare(score, topScore);
+ if (c > 0)
+ topScore = score;
+ return c;
+ }
+
+ @Override
+ protected void updateDocHead(int doc) throws IOException {
+ this.doc = doc + docBase;
+ }
+ }
+
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java
index af697af..8434534 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java
@@ -18,23 +18,34 @@ package org.apache.lucene.search.grouping;
import java.io.IOException;
import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
-import org.apache.lucene.util.BytesRef;
/**
* 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>
- * This is an abstract version. Concrete implementations define
- * what a group actually is and how it is internally collected.
*
* @lucene.experimental
*/
-public abstract class AllGroupsCollector<T> extends SimpleCollector {
+public class AllGroupsCollector<T> extends SimpleCollector {
+
+ private final GroupSelector<T> groupSelector;
+
+ private final Set<T> groups = new HashSet<T>();
+
+ /**
+ * Create a new AllGroupsCollector
+ * @param groupSelector the GroupSelector to determine groups
+ */
+ public AllGroupsCollector(GroupSelector<T> groupSelector) {
+ this.groupSelector = groupSelector;
+ }
/**
* Returns the total number of groups for the executed search.
@@ -49,18 +60,31 @@ public abstract class AllGroupsCollector<T> extends SimpleCollector {
/**
* Returns the group values
* <p>
- * This is an unordered collections of group values. For each group that matched the query there is a {@link BytesRef}
- * representing a group value.
+ * This is an unordered collections of group values.
*
* @return the group values
*/
- public abstract Collection<T> getGroups();
+ public Collection<T> getGroups() {
+ return groups;
+ }
- // Empty not necessary
@Override
public void setScorer(Scorer scorer) throws IOException {}
@Override
+ protected void doSetNextReader(LeafReaderContext context) throws IOException {
+ groupSelector.setNextReader(context);
+ }
+
+ @Override
+ public void collect(int doc) throws IOException {
+ groupSelector.advanceTo(doc);
+ if (groups.contains(groupSelector.currentValue()))
+ return;
+ groups.add(groupSelector.copyValue());
+ }
+
+ @Override
public boolean needsScores() {
return false; // the result is unaffected by relevancy
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
index c965042..a50fda1 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
@@ -50,7 +50,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 org.apache.lucene.search.grouping.term.TermAllGroupsCollector}. However, this collector does
+ * {@link org.apache.lucene.search.grouping.AllGroupsCollector}. However, this collector does
* not fill in the groupValue of each group; this field
* will always be null.
*
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java
index 54d752c..103b0d2 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java
@@ -16,10 +16,14 @@
*/
package org.apache.lucene.search.grouping;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
+import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.SimpleCollector;
/**
@@ -27,33 +31,99 @@ import org.apache.lucene.search.SimpleCollector;
*
* @lucene.experimental
*/
-public abstract class DistinctValuesCollector<T> extends SimpleCollector {
+public class DistinctValuesCollector<T, R> extends SecondPassGroupingCollector<T> {
+
+ /**
+ * Create a DistinctValuesCollector
+ * @param groupSelector the group selector to determine the top-level groups
+ * @param groups the top-level groups to collect for
+ * @param valueSelector a group selector to determine which values to collect per-group
+ */
+ public DistinctValuesCollector(GroupSelector<T> groupSelector, Collection<SearchGroup<T>> groups,
+ GroupSelector<R> valueSelector) {
+ super(groupSelector, groups, new DistinctValuesReducer<>(valueSelector));
+ }
+
+ private static class ValuesCollector<R> extends SimpleCollector {
+
+ final GroupSelector<R> valueSelector;
+ final Set<R> values = new HashSet<>();
+
+ private ValuesCollector(GroupSelector<R> valueSelector) {
+ this.valueSelector = valueSelector;
+ }
+
+ @Override
+ public void collect(int doc) throws IOException {
+ if (valueSelector.advanceTo(doc) == GroupSelector.State.ACCEPT) {
+ R value = valueSelector.currentValue();
+ if (values.contains(value) == false)
+ values.add(valueSelector.copyValue());
+ }
+ else {
+ if (values.contains(null) == false)
+ values.add(null);
+ }
+ }
+
+ @Override
+ protected void doSetNextReader(LeafReaderContext context) throws IOException {
+ valueSelector.setNextReader(context);
+ }
+
+ @Override
+ public boolean needsScores() {
+ return false;
+ }
+ }
+
+ private static class DistinctValuesReducer<T, R> extends GroupReducer<T, ValuesCollector<R>> {
+
+ final GroupSelector<R> valueSelector;
+
+ private DistinctValuesReducer(GroupSelector<R> valueSelector) {
+ this.valueSelector = valueSelector;
+ }
+
+ @Override
+ public boolean needsScores() {
+ return false;
+ }
+
+ @Override
+ protected ValuesCollector<R> newCollector() {
+ return new ValuesCollector<>(valueSelector);
+ }
+ }
/**
* Returns all unique values for each top N group.
*
* @return all unique values for each top N group
*/
- public abstract List<GroupCount<T>> getGroups();
+ public List<GroupCount<T, R>> getGroups() {
+ List<GroupCount<T, R>> counts = new ArrayList<>();
+ for (SearchGroup<T> group : groups) {
+ @SuppressWarnings("unchecked")
+ ValuesCollector<R> vc = (ValuesCollector<R>) groupReducer.getCollector(group.groupValue);
+ counts.add(new GroupCount<>(group.groupValue, vc.values));
+ }
+ return counts;
+ }
/**
* Returned by {@link DistinctValuesCollector#getGroups()},
* representing the value and set of distinct values for the group.
*/
- public static class GroupCount<T> {
+ public static class GroupCount<T, R> {
public final T groupValue;
- public final Set<T> uniqueValues;
+ public final Set<R> uniqueValues;
- public GroupCount(T groupValue) {
+ public GroupCount(T groupValue, Set<R> values) {
this.groupValue = groupValue;
- this.uniqueValues = new HashSet<>();
+ this.uniqueValues = values;
}
}
- @Override
- public boolean needsScores() {
- return false; // not needed to fetch all values
- }
-
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
index 02bb1a2..bd47adb 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
@@ -33,15 +33,16 @@ import org.apache.lucene.search.SortField;
/** FirstPassGroupingCollector is the first of two passes necessary
* to collect grouped hits. This pass gathers the top N sorted
- * groups. Concrete subclasses define what a group is and how it
- * is internally collected.
+ * groups. Groups are defined by a {@link GroupSelector}
*
* <p>See {@link org.apache.lucene.search.grouping} for more
* details including a full code example.</p>
*
* @lucene.experimental
*/
-abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
+public class FirstPassGroupingCollector<T> extends SimpleCollector {
+
+ private final GroupSelector<T> groupSelector;
private final FieldComparator<?>[] comparators;
private final LeafFieldComparator[] leafComparators;
@@ -60,16 +61,18 @@ abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
/**
* Create the first pass collector.
*
- * @param groupSort The {@link Sort} used to sort the
+ * @param groupSelector a GroupSelector used to defined groups
+ * @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.
+ * @param topNGroups How many top groups to keep.
*/
@SuppressWarnings({"unchecked", "rawtypes"})
- public FirstPassGroupingCollector(Sort groupSort, int topNGroups) {
+ public FirstPassGroupingCollector(GroupSelector<T> groupSelector, Sort groupSort, int topNGroups) {
+ this.groupSelector = groupSelector;
if (topNGroups < 1) {
throw new IllegalArgumentException("topNGroups must be >= 1 (got " + topNGroups + ")");
}
@@ -133,7 +136,7 @@ abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
if (upto++ < groupOffset) {
continue;
}
- //System.out.println(" group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
+ // System.out.println(" group=" + (group.groupValue == null ? "null" : group.groupValue.toString()));
SearchGroup<T> searchGroup = new SearchGroup<>();
searchGroup.groupValue = group.groupValue;
if (fillFields) {
@@ -155,14 +158,11 @@ abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
}
}
- @Override
- public void collect(int doc) throws IOException {
- //System.out.println("FP.collect doc=" + doc);
-
+ private boolean isCompetitive(int doc) throws IOException {
// If orderedGroups != null we already have collected N groups and
// can short circuit by comparing this document to the bottom group,
// without having to find what group this document belongs to.
-
+
// Even if this document belongs to a group in the top N, we'll know that
// we don't have to update that group.
@@ -173,7 +173,7 @@ abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
final int c = reversed[compIDX] * leafComparators[compIDX].compareBottom(doc);
if (c < 0) {
// Definitely not competitive. So don't even bother to continue
- return;
+ return false;
} else if (c > 0) {
// Definitely competitive.
break;
@@ -181,15 +181,24 @@ abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
// Here c=0. If we're at the last comparator, this doc is not
// competitive, since docs are visited in doc Id order, which means
// this doc cannot compete with any other document in the queue.
- return;
+ return false;
}
}
}
+ return true;
+ }
+
+ @Override
+ public void collect(int doc) throws IOException {
+
+ if (isCompetitive(doc) == false)
+ return;
// TODO: should we add option to mean "ignore docs that
// don't have the group field" (instead of stuffing them
// under null group)?
- final T groupValue = getDocGroupValue(doc);
+ groupSelector.advanceTo(doc);
+ T groupValue = groupSelector.currentValue();
final CollectedSearchGroup<T> group = groupMap.get(groupValue);
@@ -207,7 +216,7 @@ abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
// Add a new CollectedSearchGroup:
CollectedSearchGroup<T> sg = new CollectedSearchGroup<>();
- sg.groupValue = copyDocGroupValue(groupValue, null);
+ sg.groupValue = groupSelector.copyValue();
sg.comparatorSlot = groupMap.size();
sg.topDoc = docBase + doc;
for (LeafFieldComparator fc : leafComparators) {
@@ -233,7 +242,7 @@ abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
groupMap.remove(bottomGroup.groupValue);
// reuse the removed CollectedSearchGroup
- bottomGroup.groupValue = copyDocGroupValue(groupValue, bottomGroup.groupValue);
+ bottomGroup.groupValue = groupSelector.copyValue();
bottomGroup.topDoc = docBase + doc;
for (LeafFieldComparator fc : leafComparators) {
@@ -338,25 +347,15 @@ abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
for (int i=0; i<comparators.length; i++) {
leafComparators[i] = comparators[i].getLeafComparator(readerContext);
}
+ groupSelector.setNextReader(readerContext);
}
/**
- * Returns the group value for the specified doc.
- *
- * @param doc The specified doc
- * @return the group value for the specified doc
- */
- protected abstract T getDocGroupValue(int doc) throws IOException;
-
- /**
- * Returns a copy of the specified group value by creating a new instance and copying the value from the specified
- * groupValue in the new instance. Or optionally the reuse argument can be used to copy the group value in.
- *
- * @param groupValue The group value to copy
- * @param reuse Optionally a reuse instance to prevent a new instance creation
- * @return a copy of the specified group value
+ * @return the GroupSelector used for this Collector
*/
- protected abstract T copyDocGroupValue(T groupValue, T reuse);
+ public GroupSelector<T> getGroupSelector() {
+ return groupSelector;
+ }
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupReducer.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupReducer.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupReducer.java
new file mode 100644
index 0000000..4366e91
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupReducer.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.grouping;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.search.Scorer;
+
+/**
+ * Concrete implementations of this class define what to collect for individual
+ * groups during the second-pass of a grouping search.
+ *
+ * Each group is assigned a Collector returned by {@link #newCollector()}, and
+ * {@link LeafCollector#collect(int)} is called for each document that is in
+ * a group
+ *
+ * @see SecondPassGroupingCollector
+ *
+ * @param <T> the type of the value used for grouping
+ * @param <C> the type of {@link Collector} used to reduce each group
+ */
+public abstract class GroupReducer<T, C extends Collector> {
+
+ private final Map<T, GroupCollector<C>> groups = new HashMap<>();
+
+ /**
+ * Define which groups should be reduced.
+ *
+ * Called by {@link SecondPassGroupingCollector}
+ */
+ public void setGroups(Collection<SearchGroup<T>> groups) {
+ for (SearchGroup<T> group : groups) {
+ this.groups.put(group.groupValue, new GroupCollector<>(newCollector()));
+ }
+ }
+
+ /**
+ * Whether or not this reducer requires collected documents to be scored
+ */
+ public abstract boolean needsScores();
+
+ /**
+ * Creates a new Collector for each group
+ */
+ protected abstract C newCollector();
+
+ /**
+ * Get the Collector for a given group
+ */
+ public final C getCollector(T value) {
+ return groups.get(value).collector;
+ }
+
+ /**
+ * Collect a given document into a given group
+ * @throws IOException on error
+ */
+ public final void collect(T value, int doc) throws IOException {
+ GroupCollector<C> collector = groups.get(value);
+ collector.leafCollector.collect(doc);
+ }
+
+ /**
+ * Set the Scorer on all group collectors
+ */
+ public final void setScorer(Scorer scorer) throws IOException {
+ for (GroupCollector<C> collector : groups.values()) {
+ collector.leafCollector.setScorer(scorer);
+ }
+ }
+
+ /**
+ * Called when the parent {@link SecondPassGroupingCollector} moves to a new segment
+ */
+ public final void setNextReader(LeafReaderContext ctx) throws IOException {
+ for (GroupCollector<C> collector : groups.values()) {
+ collector.leafCollector = collector.collector.getLeafCollector(ctx);
+ }
+ }
+
+ private static final class GroupCollector<C extends Collector> {
+
+ final C collector;
+ LeafCollector leafCollector;
+
+ private GroupCollector(C collector) {
+ this.collector = collector;
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupSelector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupSelector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupSelector.java
new file mode 100644
index 0000000..dbb0932
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupSelector.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.grouping;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.lucene.index.LeafReaderContext;
+
+/**
+ * Defines a group, for use by grouping collectors
+ *
+ * A GroupSelector acts as an iterator over documents. For each segment, clients
+ * should call {@link #setNextReader(LeafReaderContext)}, and then {@link #advanceTo(int)}
+ * for each matching document.
+ *
+ * @param <T> the type of the group value
+ */
+public abstract class GroupSelector<T> {
+
+ /**
+ * What to do with the current value
+ */
+ public enum State { SKIP, ACCEPT }
+
+ /**
+ * Set the LeafReaderContext
+ */
+ public abstract void setNextReader(LeafReaderContext readerContext) throws IOException;
+
+ /**
+ * Advance the GroupSelector's iterator to the given document
+ */
+ public abstract State advanceTo(int doc) throws IOException;
+
+ /**
+ * Get the group value of the current document
+ *
+ * N.B. this object may be reused, for a persistent version use {@link #copyValue()}
+ */
+ public abstract T currentValue();
+
+ /**
+ * @return a copy of the group value of the current document
+ */
+ public abstract T copyValue();
+
+ /**
+ * Set a restriction on the group values returned by this selector
+ *
+ * If the selector is positioned on a document whose group value is not contained
+ * within this set, then {@link #advanceTo(int)} will return {@link State#SKIP}
+ *
+ * @param groups a set of {@link SearchGroup} objects to limit selections to
+ */
+ public abstract void setGroups(Collection<SearchGroup<T>> groups);
+
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java
deleted file mode 100644
index 2ff79a1..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.lucene.search.grouping;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.lucene.search.Sort;
-
-/**
- * A factory object to create first and second-pass collectors, run by a {@link GroupingSearch}
- * @param <T> the type the group value
- */
-public abstract class Grouper<T> {
-
- /**
- * Create a first-pass collector
- * @param sort the order in which groups should be returned
- * @param count how many groups to return
- */
- public abstract FirstPassGroupingCollector<T> getFirstPassCollector(Sort sort, int count) throws IOException;
-
- /**
- * Create an {@link AllGroupsCollector}
- */
- public abstract AllGroupsCollector<T> getAllGroupsCollector();
-
- /**
- * Create an {@link AllGroupHeadsCollector}
- * @param sort a within-group sort order to determine which doc is the group head
- */
- public abstract AllGroupHeadsCollector<T> getGroupHeadsCollector(Sort sort);
-
- /**
- * Create a second-pass collector
- */
- public abstract SecondPassGroupingCollector<T> getSecondPassCollector(
- Collection<SearchGroup<T>> groups, Sort groupSort, Sort withinGroupSort,
- int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException;
-
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
index f4319d5..a36917d 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
@@ -30,8 +30,6 @@ import org.apache.lucene.search.Query;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.Weight;
-import org.apache.lucene.search.grouping.function.FunctionGrouper;
-import org.apache.lucene.search.grouping.term.TermGrouper;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.mutable.MutableValue;
@@ -43,7 +41,7 @@ import org.apache.lucene.util.mutable.MutableValue;
*/
public class GroupingSearch {
- private final Grouper grouper;
+ private final GroupSelector grouper;
private final Query groupEndDocs;
private Sort groupSort = Sort.RELEVANCE;
@@ -71,11 +69,7 @@ public class GroupingSearch {
* @param groupField The name of the field to group by.
*/
public GroupingSearch(String groupField) {
- this(new TermGrouper(groupField, 128), null);
- }
-
- public GroupingSearch(String groupField, int initialSize) {
- this(new TermGrouper(groupField, initialSize), null);
+ this(new TermGroupSelector(groupField), null);
}
/**
@@ -86,7 +80,7 @@ public class GroupingSearch {
* @param valueSourceContext The context of the specified groupFunction
*/
public GroupingSearch(ValueSource groupFunction, Map<?, ?> valueSourceContext) {
- this(new FunctionGrouper(groupFunction, valueSourceContext), null);
+ this(new ValueSourceGroupSelector(groupFunction, valueSourceContext), null);
}
/**
@@ -99,7 +93,7 @@ public class GroupingSearch {
this(null, groupEndDocs);
}
- private GroupingSearch(Grouper grouper, Query groupEndDocs) {
+ private GroupingSearch(GroupSelector grouper, Query groupEndDocs) {
this.grouper = grouper;
this.groupEndDocs = groupEndDocs;
}
@@ -129,10 +123,10 @@ public class GroupingSearch {
protected TopGroups groupByFieldOrFunction(IndexSearcher searcher, Query query, int groupOffset, int groupLimit) throws IOException {
int topN = groupOffset + groupLimit;
- final FirstPassGroupingCollector firstPassCollector = grouper.getFirstPassCollector(groupSort, topN);
- final AllGroupsCollector allGroupsCollector = allGroups ? grouper.getAllGroupsCollector() : null;
+ final FirstPassGroupingCollector firstPassCollector = new FirstPassGroupingCollector(grouper, groupSort, topN);
+ final AllGroupsCollector allGroupsCollector = allGroups ? new AllGroupsCollector(grouper) : null;
final AllGroupHeadsCollector allGroupHeadsCollector
- = allGroupHeads ? grouper.getGroupHeadsCollector(sortWithinGroup) : null;
+ = allGroupHeads ? AllGroupHeadsCollector.newCollector(grouper, sortWithinGroup) : null;
final Collector firstRound = MultiCollector.wrap(firstPassCollector, allGroupsCollector, allGroupHeadsCollector);
@@ -158,8 +152,8 @@ public class GroupingSearch {
}
int topNInsideGroup = groupDocsOffset + groupDocsLimit;
- SecondPassGroupingCollector secondPassCollector
- = grouper.getSecondPassCollector(topSearchGroups, groupSort, sortWithinGroup, topNInsideGroup,
+ TopGroupsCollector secondPassCollector
+ = new TopGroupsCollector(grouper, topSearchGroups, groupSort, sortWithinGroup, topNInsideGroup,
includeScores, includeMaxScore, fillSortFields);
if (cachedCollector != null && cachedCollector.isCached()) {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java
index f8feb75..c54c8ee 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java
@@ -18,152 +18,82 @@ package org.apache.lucene.search.grouping;
import java.io.IOException;
import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
import java.util.Objects;
import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.search.TopDocsCollector;
-import org.apache.lucene.search.TopFieldCollector;
-import org.apache.lucene.search.TopScoreDocCollector;
/**
- * SecondPassGroupingCollector is the second of two passes
- * necessary to collect grouped docs. This pass gathers the
- * top N documents per top group computed from the
- * first pass. Concrete subclasses define what a group is and how it
- * is internally collected.
+ * SecondPassGroupingCollector runs over an already collected set of
+ * groups, further applying a {@link GroupReducer} to each group
*
- * <p>See {@link org.apache.lucene.search.grouping} for more
- * details including a full code example.</p>
+ * @see TopGroupsCollector
+ * @see DistinctValuesCollector
*
* @lucene.experimental
*/
-public abstract class SecondPassGroupingCollector<T> extends SimpleCollector {
+public class SecondPassGroupingCollector<T> extends SimpleCollector {
- private final Collection<SearchGroup<T>> groups;
- private final Sort groupSort;
- private final Sort withinGroupSort;
- private final int maxDocsPerGroup;
- private final boolean needsScores;
- protected final Map<T, SearchGroupDocs<T>> groupMap;
+ protected final GroupSelector<T> groupSelector;
+ protected final Collection<SearchGroup<T>> groups;
+ protected final GroupReducer<T, ?> groupReducer;
- protected SearchGroupDocs<T>[] groupDocs;
+ protected int totalHitCount;
+ protected int totalGroupedHitCount;
- private int totalHitCount;
- private int totalGroupedHitCount;
-
- public SecondPassGroupingCollector(Collection<SearchGroup<T>> groups, Sort groupSort, Sort withinGroupSort,
- int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields)
- throws IOException {
+ /**
+ * Create a new SecondPassGroupingCollector
+ * @param groupSelector the GroupSelector that defines groups for this search
+ * @param groups the groups to collect documents for
+ * @param reducer the reducer to apply to each group
+ */
+ public SecondPassGroupingCollector(GroupSelector<T> groupSelector, Collection<SearchGroup<T>> groups, GroupReducer<T, ?> reducer) {
//System.out.println("SP init");
if (groups.isEmpty()) {
throw new IllegalArgumentException("no groups to collect (groups is empty)");
}
+ this.groupSelector = Objects.requireNonNull(groupSelector);
+ this.groupSelector.setGroups(groups);
+
this.groups = Objects.requireNonNull(groups);
- this.groupSort = Objects.requireNonNull(groupSort);
- this.withinGroupSort = Objects.requireNonNull(withinGroupSort);
- this.maxDocsPerGroup = maxDocsPerGroup;
- this.needsScores = getScores || getMaxScores || withinGroupSort.needsScores();
+ this.groupReducer = reducer;
+ reducer.setGroups(groups);
+ }
- this.groupMap = new HashMap<>(groups.size());
- for (SearchGroup<T> group : groups) {
- //System.out.println(" prep group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
- final TopDocsCollector<?> collector;
- if (withinGroupSort.equals(Sort.RELEVANCE)) { // optimize to use TopScoreDocCollector
- // Sort by score
- collector = TopScoreDocCollector.create(maxDocsPerGroup);
- } else {
- // Sort by fields
- collector = TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores);
- }
- groupMap.put(group.groupValue, new SearchGroupDocs<>(group.groupValue, collector));
- }
+ /**
+ * @return the GroupSelector used in this collector
+ */
+ public GroupSelector<T> getGroupSelector() {
+ return groupSelector;
}
@Override
public boolean needsScores() {
- return needsScores;
+ return groupReducer.needsScores();
}
@Override
public void setScorer(Scorer scorer) throws IOException {
- for (SearchGroupDocs<T> group : groupMap.values()) {
- group.leafCollector.setScorer(scorer);
- }
+ groupReducer.setScorer(scorer);
}
@Override
public void collect(int doc) throws IOException {
totalHitCount++;
- SearchGroupDocs<T> group = retrieveGroup(doc);
- if (group != null) {
- totalGroupedHitCount++;
- group.leafCollector.collect(doc);
- }
+ if (groupSelector.advanceTo(doc) == GroupSelector.State.SKIP)
+ return;
+ totalGroupedHitCount++;
+ T value = groupSelector.currentValue();
+ groupReducer.collect(value, doc);
}
- /**
- * Returns the group the specified doc belongs to or <code>null</code> if no group could be retrieved.
- *
- * @param doc The specified doc
- * @return the group the specified doc belongs to or <code>null</code> if no group could be retrieved
- * @throws IOException If an I/O related error occurred
- */
- protected abstract SearchGroupDocs<T> retrieveGroup(int doc) throws IOException;
-
@Override
protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
- //System.out.println("SP.setNextReader");
- for (SearchGroupDocs<T> group : groupMap.values()) {
- group.leafCollector = group.collector.getLeafCollector(readerContext);
- }
- }
-
- public TopGroups<T> getTopGroups(int withinGroupOffset) {
- @SuppressWarnings({"unchecked","rawtypes"})
- final GroupDocs<T>[] groupDocsResult = (GroupDocs<T>[]) new GroupDocs[groups.size()];
-
- int groupIDX = 0;
- float maxScore = Float.MIN_VALUE;
- for(SearchGroup<?> group : groups) {
- final SearchGroupDocs<T> groupDocs = groupMap.get(group.groupValue);
- final TopDocs topDocs = groupDocs.collector.topDocs(withinGroupOffset, maxDocsPerGroup);
- groupDocsResult[groupIDX++] = new GroupDocs<>(Float.NaN,
- topDocs.getMaxScore(),
- topDocs.totalHits,
- topDocs.scoreDocs,
- groupDocs.groupValue,
- group.sortValues);
- maxScore = Math.max(maxScore, topDocs.getMaxScore());
- }
-
- return new TopGroups<>(groupSort.getSort(),
- withinGroupSort.getSort(),
- totalHitCount, totalGroupedHitCount, groupDocsResult,
- maxScore);
+ groupReducer.setNextReader(readerContext);
+ groupSelector.setNextReader(readerContext);
}
-
- // TODO: merge with SearchGroup or not?
- // ad: don't need to build a new hashmap
- // disad: blows up the size of SearchGroup if we need many of them, and couples implementations
- public class SearchGroupDocs<T> {
-
- public final T groupValue;
- public final TopDocsCollector<?> collector;
- public LeafCollector leafCollector;
-
- public SearchGroupDocs(T groupValue, TopDocsCollector<?> collector) {
- this.groupValue = groupValue;
- this.collector = collector;
- }
- }
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java
new file mode 100644
index 0000000..39d28a5
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java
@@ -0,0 +1,414 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search.grouping;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.SentinelIntSet;
+import org.apache.lucene.util.UnicodeUtil;
+
+/**
+ * An implementation of {@link GroupFacetCollector} that computes grouped facets based on the indexed terms
+ * from DocValues.
+ *
+ * @lucene.experimental
+ */
+public abstract class TermGroupFacetCollector extends GroupFacetCollector {
+
+ final List<GroupedFacetHit> groupedFacetHits;
+ final SentinelIntSet segmentGroupedFacetHits;
+
+ SortedDocValues groupFieldTermsIndex;
+
+ /**
+ * Factory method for creating the right implementation based on the fact whether the facet field contains
+ * multiple tokens per documents.
+ *
+ * @param groupField The group field
+ * @param facetField The facet field
+ * @param facetFieldMultivalued Whether the facet field has multiple tokens per document
+ * @param facetPrefix The facet prefix a facet entry should start with to be included.
+ * @param initialSize The initial allocation size of the internal int set and group facet list which should roughly
+ * match the total number of expected unique groups. Be aware that the heap usage is
+ * 4 bytes * initialSize.
+ * @return <code>TermGroupFacetCollector</code> implementation
+ */
+ public static TermGroupFacetCollector createTermGroupFacetCollector(String groupField,
+ String facetField,
+ boolean facetFieldMultivalued,
+ BytesRef facetPrefix,
+ int initialSize) {
+ if (facetFieldMultivalued) {
+ return new MV(groupField, facetField, facetPrefix, initialSize);
+ } else {
+ return new SV(groupField, facetField, facetPrefix, initialSize);
+ }
+ }
+
+ TermGroupFacetCollector(String groupField, String facetField, BytesRef facetPrefix, int initialSize) {
+ super(groupField, facetField, facetPrefix);
+ groupedFacetHits = new ArrayList<>(initialSize);
+ segmentGroupedFacetHits = new SentinelIntSet(initialSize, Integer.MIN_VALUE);
+ }
+
+ // Implementation for single valued facet fields.
+ static class SV extends TermGroupFacetCollector {
+
+ private SortedDocValues facetFieldTermsIndex;
+
+ SV(String groupField, String facetField, BytesRef facetPrefix, int initialSize) {
+ super(groupField, facetField, facetPrefix, initialSize);
+ }
+
+ @Override
+ public void collect(int doc) throws IOException {
+ if (doc > facetFieldTermsIndex.docID()) {
+ facetFieldTermsIndex.advance(doc);
+ }
+
+ int facetOrd;
+ if (doc == facetFieldTermsIndex.docID()) {
+ facetOrd = facetFieldTermsIndex.ordValue();
+ } else {
+ facetOrd = -1;
+ }
+
+ if (facetOrd < startFacetOrd || facetOrd >= endFacetOrd) {
+ return;
+ }
+
+ if (doc > groupFieldTermsIndex.docID()) {
+ groupFieldTermsIndex.advance(doc);
+ }
+
+ int groupOrd;
+ if (doc == groupFieldTermsIndex.docID()) {
+ groupOrd = groupFieldTermsIndex.ordValue();
+ } else {
+ groupOrd = -1;
+ }
+ int segmentGroupedFacetsIndex = groupOrd * (facetFieldTermsIndex.getValueCount()+1) + facetOrd;
+ if (segmentGroupedFacetHits.exists(segmentGroupedFacetsIndex)) {
+ return;
+ }
+
+ segmentTotalCount++;
+ segmentFacetCounts[facetOrd+1]++;
+
+ segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
+
+ BytesRef groupKey;
+ if (groupOrd == -1) {
+ groupKey = null;
+ } else {
+ groupKey = BytesRef.deepCopyOf(groupFieldTermsIndex.lookupOrd(groupOrd));
+ }
+
+ BytesRef facetKey;
+ if (facetOrd == -1) {
+ facetKey = null;
+ } else {
+ facetKey = BytesRef.deepCopyOf(facetFieldTermsIndex.lookupOrd(facetOrd));
+ }
+
+ groupedFacetHits.add(new GroupedFacetHit(groupKey, facetKey));
+ }
+
+ @Override
+ protected void doSetNextReader(LeafReaderContext context) throws IOException {
+ if (segmentFacetCounts != null) {
+ segmentResults.add(createSegmentResult());
+ }
+
+ groupFieldTermsIndex = DocValues.getSorted(context.reader(), groupField);
+ facetFieldTermsIndex = DocValues.getSorted(context.reader(), facetField);
+
+ // 1+ to allow for the -1 "not set":
+ segmentFacetCounts = new int[facetFieldTermsIndex.getValueCount()+1];
+ segmentTotalCount = 0;
+
+ segmentGroupedFacetHits.clear();
+ for (GroupedFacetHit groupedFacetHit : groupedFacetHits) {
+ int facetOrd = groupedFacetHit.facetValue == null ? -1 : facetFieldTermsIndex.lookupTerm(groupedFacetHit.facetValue);
+ if (groupedFacetHit.facetValue != null && facetOrd < 0) {
+ continue;
+ }
+
+ int groupOrd = groupedFacetHit.groupValue == null ? -1 : groupFieldTermsIndex.lookupTerm(groupedFacetHit.groupValue);
+ if (groupedFacetHit.groupValue != null && groupOrd < 0) {
+ continue;
+ }
+
+ int segmentGroupedFacetsIndex = groupOrd * (facetFieldTermsIndex.getValueCount()+1) + facetOrd;
+ segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
+ }
+
+ if (facetPrefix != null) {
+ startFacetOrd = facetFieldTermsIndex.lookupTerm(facetPrefix);
+ if (startFacetOrd < 0) {
+ // Points to the ord one higher than facetPrefix
+ startFacetOrd = -startFacetOrd - 1;
+ }
+ BytesRefBuilder facetEndPrefix = new BytesRefBuilder();
+ facetEndPrefix.append(facetPrefix);
+ facetEndPrefix.append(UnicodeUtil.BIG_TERM);
+ endFacetOrd = facetFieldTermsIndex.lookupTerm(facetEndPrefix.get());
+ assert endFacetOrd < 0;
+ endFacetOrd = -endFacetOrd - 1; // Points to the ord one higher than facetEndPrefix
+ } else {
+ startFacetOrd = -1;
+ endFacetOrd = facetFieldTermsIndex.getValueCount();
+ }
+ }
+
+ @Override
+ protected SegmentResult createSegmentResult() throws IOException {
+ return new SegmentResult(segmentFacetCounts, segmentTotalCount, facetFieldTermsIndex.termsEnum(), startFacetOrd, endFacetOrd);
+ }
+
+ private static class SegmentResult extends GroupFacetCollector.SegmentResult {
+
+ final TermsEnum tenum;
+
+ SegmentResult(int[] counts, int total, TermsEnum tenum, int startFacetOrd, int endFacetOrd) throws IOException {
+ super(counts, total - counts[0], counts[0], endFacetOrd+1);
+ this.tenum = tenum;
+ this.mergePos = startFacetOrd == -1 ? 1 : startFacetOrd+1;
+ if (mergePos < maxTermPos) {
+ assert tenum != null;
+ tenum.seekExact(startFacetOrd == -1 ? 0 : startFacetOrd);
+ mergeTerm = tenum.term();
+ }
+ }
+
+ @Override
+ protected void nextTerm() throws IOException {
+ mergeTerm = tenum.next();
+ }
+ }
+ }
+
+ // Implementation for multi valued facet fields.
+ static class MV extends TermGroupFacetCollector {
+
+ private SortedSetDocValues facetFieldDocTermOrds;
+ private TermsEnum facetOrdTermsEnum;
+ private int facetFieldNumTerms;
+
+ MV(String groupField, String facetField, BytesRef facetPrefix, int initialSize) {
+ super(groupField, facetField, facetPrefix, initialSize);
+ }
+
+ @Override
+ public void collect(int doc) throws IOException {
+ if (doc > groupFieldTermsIndex.docID()) {
+ groupFieldTermsIndex.advance(doc);
+ }
+
+ int groupOrd;
+ if (doc == groupFieldTermsIndex.docID()) {
+ groupOrd = groupFieldTermsIndex.ordValue();
+ } else {
+ groupOrd = -1;
+ }
+
+ if (facetFieldNumTerms == 0) {
+ int segmentGroupedFacetsIndex = groupOrd * (facetFieldNumTerms + 1);
+ if (facetPrefix != null || segmentGroupedFacetHits.exists(segmentGroupedFacetsIndex)) {
+ return;
+ }
+
+ segmentTotalCount++;
+ segmentFacetCounts[facetFieldNumTerms]++;
+
+ segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
+ BytesRef groupKey;
+ if (groupOrd == -1) {
+ groupKey = null;
+ } else {
+ groupKey = BytesRef.deepCopyOf(groupFieldTermsIndex.lookupOrd(groupOrd));
+ }
+ groupedFacetHits.add(new GroupedFacetHit(groupKey, null));
+ return;
+ }
+
+ if (doc > facetFieldDocTermOrds.docID()) {
+ facetFieldDocTermOrds.advance(doc);
+ }
+ boolean empty = true;
+ if (doc == facetFieldDocTermOrds.docID()) {
+ long ord;
+ while ((ord = facetFieldDocTermOrds.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+ process(groupOrd, (int) ord);
+ empty = false;
+ }
+ }
+
+ if (empty) {
+ process(groupOrd, facetFieldNumTerms); // this facet ord is reserved for docs not containing facet field.
+ }
+ }
+
+ private void process(int groupOrd, int facetOrd) throws IOException {
+ if (facetOrd < startFacetOrd || facetOrd >= endFacetOrd) {
+ return;
+ }
+
+ int segmentGroupedFacetsIndex = groupOrd * (facetFieldNumTerms + 1) + facetOrd;
+ if (segmentGroupedFacetHits.exists(segmentGroupedFacetsIndex)) {
+ return;
+ }
+
+ segmentTotalCount++;
+ segmentFacetCounts[facetOrd]++;
+
+ segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
+
+ BytesRef groupKey;
+ if (groupOrd == -1) {
+ groupKey = null;
+ } else {
+ groupKey = BytesRef.deepCopyOf(groupFieldTermsIndex.lookupOrd(groupOrd));
+ }
+
+ final BytesRef facetValue;
+ if (facetOrd == facetFieldNumTerms) {
+ facetValue = null;
+ } else {
+ facetValue = BytesRef.deepCopyOf(facetFieldDocTermOrds.lookupOrd(facetOrd));
+ }
+ groupedFacetHits.add(new GroupedFacetHit(groupKey, facetValue));
+ }
+
+ @Override
+ protected void doSetNextReader(LeafReaderContext context) throws IOException {
+ if (segmentFacetCounts != null) {
+ segmentResults.add(createSegmentResult());
+ }
+
+ groupFieldTermsIndex = DocValues.getSorted(context.reader(), groupField);
+ facetFieldDocTermOrds = DocValues.getSortedSet(context.reader(), facetField);
+ facetFieldNumTerms = (int) facetFieldDocTermOrds.getValueCount();
+ if (facetFieldNumTerms == 0) {
+ facetOrdTermsEnum = null;
+ } else {
+ facetOrdTermsEnum = facetFieldDocTermOrds.termsEnum();
+ }
+ // [facetFieldNumTerms() + 1] for all possible facet values and docs not containing facet field
+ segmentFacetCounts = new int[facetFieldNumTerms + 1];
+ segmentTotalCount = 0;
+
+ segmentGroupedFacetHits.clear();
+ for (GroupedFacetHit groupedFacetHit : groupedFacetHits) {
+ int groupOrd = groupedFacetHit.groupValue == null ? -1 : groupFieldTermsIndex.lookupTerm(groupedFacetHit.groupValue);
+ if (groupedFacetHit.groupValue != null && groupOrd < 0) {
+ continue;
+ }
+
+ int facetOrd;
+ if (groupedFacetHit.facetValue != null) {
+ if (facetOrdTermsEnum == null || !facetOrdTermsEnum.seekExact(groupedFacetHit.facetValue)) {
+ continue;
+ }
+ facetOrd = (int) facetOrdTermsEnum.ord();
+ } else {
+ facetOrd = facetFieldNumTerms;
+ }
+
+ // (facetFieldDocTermOrds.numTerms() + 1) for all possible facet values and docs not containing facet field
+ int segmentGroupedFacetsIndex = groupOrd * (facetFieldNumTerms + 1) + facetOrd;
+ segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
+ }
+
+ if (facetPrefix != null) {
+ TermsEnum.SeekStatus seekStatus;
+ if (facetOrdTermsEnum != null) {
+ seekStatus = facetOrdTermsEnum.seekCeil(facetPrefix);
+ } else {
+ seekStatus = TermsEnum.SeekStatus.END;
+ }
+
+ if (seekStatus != TermsEnum.SeekStatus.END) {
+ startFacetOrd = (int) facetOrdTermsEnum.ord();
+ } else {
+ startFacetOrd = 0;
+ endFacetOrd = 0;
+ return;
+ }
+
+ BytesRefBuilder facetEndPrefix = new BytesRefBuilder();
+ facetEndPrefix.append(facetPrefix);
+ facetEndPrefix.append(UnicodeUtil.BIG_TERM);
+ seekStatus = facetOrdTermsEnum.seekCeil(facetEndPrefix.get());
+ if (seekStatus != TermsEnum.SeekStatus.END) {
+ endFacetOrd = (int) facetOrdTermsEnum.ord();
+ } else {
+ endFacetOrd = facetFieldNumTerms; // Don't include null...
+ }
+ } else {
+ startFacetOrd = 0;
+ endFacetOrd = facetFieldNumTerms + 1;
+ }
+ }
+
+ @Override
+ protected SegmentResult createSegmentResult() throws IOException {
+ return new SegmentResult(segmentFacetCounts, segmentTotalCount, facetFieldNumTerms, facetOrdTermsEnum, startFacetOrd, endFacetOrd);
+ }
+
+ private static class SegmentResult extends GroupFacetCollector.SegmentResult {
+
+ final TermsEnum tenum;
+
+ SegmentResult(int[] counts, int total, int missingCountIndex, TermsEnum tenum, int startFacetOrd, int endFacetOrd) throws IOException {
+ super(counts, total - counts[missingCountIndex], counts[missingCountIndex],
+ endFacetOrd == missingCountIndex + 1 ? missingCountIndex : endFacetOrd);
+ this.tenum = tenum;
+ this.mergePos = startFacetOrd;
+ if (tenum != null) {
+ tenum.seekExact(mergePos);
+ mergeTerm = tenum.term();
+ }
+ }
+
+ @Override
+ protected void nextTerm() throws IOException {
+ mergeTerm = tenum.next();
+ }
+ }
+ }
+
+ private static class GroupedFacetHit {
+
+ final BytesRef groupValue;
+ final BytesRef facetValue;
+
+ GroupedFacetHit(BytesRef groupValue, BytesRef facetValue) {
+ this.groupValue = groupValue;
+ this.facetValue = facetValue;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupSelector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupSelector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupSelector.java
new file mode 100644
index 0000000..5b8f77c
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupSelector.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.grouping;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefHash;
+
+/**
+ * A GroupSelector implementation that groups via SortedDocValues
+ */
+public class TermGroupSelector extends GroupSelector<BytesRef> {
+
+ private final String field;
+ private final BytesRefHash values = new BytesRefHash();
+ private final Map<Integer, Integer> ordsToGroupIds = new HashMap<>();
+
+ private SortedDocValues docValues;
+ private int groupId;
+
+ private boolean secondPass;
+ private boolean includeEmpty;
+
+ /**
+ * Create a new TermGroupSelector
+ * @param field the SortedDocValues field to use for grouping
+ */
+ public TermGroupSelector(String field) {
+ this.field = field;
+ }
+
+ @Override
+ public void setNextReader(LeafReaderContext readerContext) throws IOException {
+ this.docValues = DocValues.getSorted(readerContext.reader(), field);
+ this.ordsToGroupIds.clear();
+ BytesRef scratch = new BytesRef();
+ for (int i = 0; i < values.size(); i++) {
+ values.get(i, scratch);
+ int ord = this.docValues.lookupTerm(scratch);
+ if (ord >= 0)
+ ordsToGroupIds.put(ord, i);
+ }
+ }
+
+ @Override
+ public State advanceTo(int doc) throws IOException {
+ if (this.docValues.advanceExact(doc) == false) {
+ groupId = -1;
+ return includeEmpty ? State.ACCEPT : State.SKIP;
+ }
+ int ord = docValues.ordValue();
+ if (ordsToGroupIds.containsKey(ord)) {
+ groupId = ordsToGroupIds.get(ord);
+ return State.ACCEPT;
+ }
+ if (secondPass)
+ return State.SKIP;
+ groupId = values.add(docValues.binaryValue());
+ ordsToGroupIds.put(ord, groupId);
+ return State.ACCEPT;
+ }
+
+ private BytesRef scratch = new BytesRef();
+
+ @Override
+ public BytesRef currentValue() {
+ if (groupId == -1)
+ return null;
+ values.get(groupId, scratch);
+ return scratch;
+ }
+
+ @Override
+ public BytesRef copyValue() {
+ if (groupId == -1)
+ return null;
+ return BytesRef.deepCopyOf(currentValue());
+ }
+
+ @Override
+ public void setGroups(Collection<SearchGroup<BytesRef>> searchGroups) {
+ this.values.clear();
+ this.values.reinit();
+ for (SearchGroup<BytesRef> sg : searchGroups) {
+ if (sg.groupValue == null)
+ includeEmpty = true;
+ else
+ this.values.add(sg.groupValue);
+ }
+ this.secondPass = true;
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java
new file mode 100644
index 0000000..b6c71d3
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.grouping;
+
+import java.util.Collection;
+import java.util.Objects;
+import java.util.function.Supplier;
+
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TopDocsCollector;
+import org.apache.lucene.search.TopFieldCollector;
+import org.apache.lucene.search.TopScoreDocCollector;
+
+/**
+ * A second-pass collector that collects the TopDocs for each group, and
+ * returns them as a {@link TopGroups} object
+ *
+ * @param <T> the type of the group value
+ */
+public class TopGroupsCollector<T> extends SecondPassGroupingCollector<T> {
+
+ private final Sort groupSort;
+ private final Sort withinGroupSort;
+ private final int maxDocsPerGroup;
+
+ /**
+ * Create a new TopGroupsCollector
+ * @param groupSelector the group selector used to define groups
+ * @param groups the groups to collect TopDocs for
+ * @param groupSort the order in which groups are returned
+ * @param withinGroupSort the order in which documents are sorted in each group
+ * @param maxDocsPerGroup the maximum number of docs to collect for each group
+ * @param getScores if true, record the scores of all docs in each group
+ * @param getMaxScores if true, record the maximum score for each group
+ * @param fillSortFields if true, record the sort field values for all docs
+ */
+ public TopGroupsCollector(GroupSelector<T> groupSelector, Collection<SearchGroup<T>> groups, Sort groupSort, Sort withinGroupSort,
+ int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) {
+ super(groupSelector, groups,
+ new TopDocsReducer<>(withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields));
+ this.groupSort = Objects.requireNonNull(groupSort);
+ this.withinGroupSort = Objects.requireNonNull(withinGroupSort);
+ this.maxDocsPerGroup = maxDocsPerGroup;
+
+ }
+
+ private static class TopDocsReducer<T> extends GroupReducer<T, TopDocsCollector<?>> {
+
+ private final Supplier<TopDocsCollector<?>> supplier;
+ private final boolean needsScores;
+
+ TopDocsReducer(Sort withinGroupSort,
+ int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) {
+ this.needsScores = getScores || getMaxScores || withinGroupSort.needsScores();
+ this.supplier = withinGroupSort == Sort.RELEVANCE ?
+ () -> TopScoreDocCollector.create(maxDocsPerGroup) :
+ () -> TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores);
+ }
+
+ @Override
+ public boolean needsScores() {
+ return needsScores;
+ }
+
+ @Override
+ protected TopDocsCollector<?> newCollector() {
+ return supplier.get();
+ }
+ }
+
+ /**
+ * Get the TopGroups recorded by this collector
+ * @param withinGroupOffset the offset within each group to start collecting documents
+ */
+ public TopGroups<T> getTopGroups(int withinGroupOffset) {
+ @SuppressWarnings({"unchecked","rawtypes"})
+ final GroupDocs<T>[] groupDocsResult = (GroupDocs<T>[]) new GroupDocs[groups.size()];
+
+ int groupIDX = 0;
+ float maxScore = Float.MIN_VALUE;
+ for(SearchGroup<T> group : groups) {
+ TopDocsCollector<?> collector = (TopDocsCollector<?>) groupReducer.getCollector(group.groupValue);
+ final TopDocs topDocs = collector.topDocs(withinGroupOffset, maxDocsPerGroup);
+ groupDocsResult[groupIDX++] = new GroupDocs<>(Float.NaN,
+ topDocs.getMaxScore(),
+ topDocs.totalHits,
+ topDocs.scoreDocs,
+ group.groupValue,
+ group.sortValues);
+ maxScore = Math.max(maxScore, topDocs.getMaxScore());
+ }
+
+ return new TopGroups<>(groupSort.getSort(),
+ withinGroupSort.getSort(),
+ totalHitCount, totalGroupedHitCount, groupDocsResult,
+ maxScore);
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/ValueSourceGroupSelector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/ValueSourceGroupSelector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/ValueSourceGroupSelector.java
new file mode 100644
index 0000000..2490160
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/ValueSourceGroupSelector.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.grouping;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.util.mutable.MutableValue;
+
+/**
+ * A GroupSelector that groups via a ValueSource
+ */
+public class ValueSourceGroupSelector extends GroupSelector<MutableValue> {
+
+ private final ValueSource valueSource;
+ private final Map<?, ?> context;
+
+ private Set<MutableValue> secondPassGroups;
+
+ /**
+ * Create a new ValueSourceGroupSelector
+ * @param valueSource the ValueSource to group by
+ * @param context a context map for the ValueSource
+ */
+ public ValueSourceGroupSelector(ValueSource valueSource, Map<?, ?> context) {
+ this.valueSource = valueSource;
+ this.context = context;
+ }
+
+ private FunctionValues.ValueFiller filler;
+
+ @Override
+ public void setNextReader(LeafReaderContext readerContext) throws IOException {
+ FunctionValues values = valueSource.getValues(context, readerContext);
+ this.filler = values.getValueFiller();
+ }
+
+ @Override
+ public State advanceTo(int doc) throws IOException {
+ this.filler.fillValue(doc);
+ if (secondPassGroups != null) {
+ if (secondPassGroups.contains(filler.getValue()) == false)
+ return State.SKIP;
+ }
+ return State.ACCEPT;
+ }
+
+ @Override
+ public MutableValue currentValue() {
+ return filler.getValue();
+ }
+
+ @Override
+ public MutableValue copyValue() {
+ return filler.getValue().duplicate();
+ }
+
+ @Override
+ public void setGroups(Collection<SearchGroup<MutableValue>> searchGroups) {
+ secondPassGroups = new HashSet<>();
+ for (SearchGroup<MutableValue> group : searchGroups) {
+ secondPassGroups.add(group.groupValue);
+ }
+ }
+}
[2/3] lucene-solr:master: LUCENE-7701: Refactor grouping collectors
Posted by ro...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java
deleted file mode 100644
index f4d4668..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.function;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.FieldComparator;
-import org.apache.lucene.search.LeafFieldComparator;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
-import org.apache.lucene.util.mutable.MutableValue;
-
-/**
- * An implementation of {@link AllGroupHeadsCollector} for retrieving the most relevant groups when grouping
- * by {@link ValueSource}.
- *
- * @lucene.experimental
- */
-public class FunctionAllGroupHeadsCollector extends AllGroupHeadsCollector<MutableValue> {
-
- private final ValueSource groupBy;
- private final Map<?, ?> vsContext;
- private final Map<MutableValue, FunctionGroupHead> groups;
- private final Sort sortWithinGroup;
-
- private FunctionValues.ValueFiller filler;
- private MutableValue mval;
- private LeafReaderContext 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;
- }
- }
-
- @Override
- protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
- filler.fillValue(doc);
- FunctionGroupHead groupHead = groups.get(mval);
- if (groupHead == null) {
- MutableValue groupValue = mval.duplicate();
- groupHead = new FunctionGroupHead(groupValue, sortWithinGroup, doc);
- groups.put(groupValue, groupHead);
- temporalResult.stop = true;
- } else {
- temporalResult.stop = false;
- }
- this.temporalResult.groupHead = groupHead;
- }
-
- @Override
- protected Collection<FunctionGroupHead> getCollectedGroupHeads() {
- return groups.values();
- }
-
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- this.scorer = scorer;
- for (FunctionGroupHead groupHead : groups.values()) {
- for (LeafFieldComparator comparator : groupHead.leafComparators) {
- comparator.setScorer(scorer);
- }
- }
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- this.readerContext = context;
- FunctionValues values = groupBy.getValues(vsContext, context);
- filler = values.getValueFiller();
- mval = filler.getValue();
-
- for (FunctionGroupHead groupHead : groups.values()) {
- for (int i = 0; i < groupHead.comparators.length; i++) {
- groupHead.leafComparators[i] = groupHead.comparators[i].getLeafComparator(context);
- }
- }
- }
-
- /** Holds current head document for a single group.
- *
- * @lucene.experimental */
- public class FunctionGroupHead extends AllGroupHeadsCollector.GroupHead<MutableValue> {
-
- final FieldComparator<?>[] comparators;
- final LeafFieldComparator[] leafComparators;
-
- @SuppressWarnings({"unchecked","rawtypes"})
- private FunctionGroupHead(MutableValue groupValue, Sort sort, int doc) throws IOException {
- super(groupValue, doc + readerContext.docBase);
- final SortField[] sortFields = sort.getSort();
- comparators = new FieldComparator[sortFields.length];
- leafComparators = new LeafFieldComparator[sortFields.length];
- for (int i = 0; i < sortFields.length; i++) {
- comparators[i] = sortFields[i].getComparator(1, i);
- leafComparators[i] = comparators[i].getLeafComparator(readerContext);
- leafComparators[i].setScorer(scorer);
- leafComparators[i].copy(0, doc);
- leafComparators[i].setBottom(0);
- }
- }
-
- @Override
- public int compare(int compIDX, int doc) throws IOException {
- return leafComparators[compIDX].compareBottom(doc);
- }
-
- @Override
- public void updateDocHead(int doc) throws IOException {
- for (LeafFieldComparator comparator : leafComparators) {
- comparator.copy(0, doc);
- comparator.setBottom(0);
- }
- this.doc = doc + readerContext.docBase;
- }
- }
-
- @Override
- public boolean needsScores() {
- return sortWithinGroup.needsScores();
- }
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java
deleted file mode 100644
index 1609d4d..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.function;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.grouping.AllGroupsCollector;
-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 FunctionValues} to retrieve the
- * field values to group by.
- *
- * @lucene.experimental
- */
-public class FunctionAllGroupsCollector extends AllGroupsCollector<MutableValue> {
-
- private final Map<?, ?> vsContext;
- private final ValueSource groupBy;
- private final SortedSet<MutableValue> groups = new TreeSet<>();
-
- private FunctionValues.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;
- }
-
- @Override
- public Collection<MutableValue> getGroups() {
- return groups;
- }
-
- @Override
- public void collect(int doc) throws IOException {
- filler.fillValue(doc);
- if (!groups.contains(mval)) {
- groups.add(mval.duplicate());
- }
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- FunctionValues values = groupBy.getValues(vsContext, context);
- filler = values.getValueFiller();
- mval = filler.getValue();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java
deleted file mode 100644
index 69ead07..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.function;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.grouping.DistinctValuesCollector;
-import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.mutable.MutableValue;
-
-/**
- * Function based implementation of {@link DistinctValuesCollector}.
- *
- * @lucene.experimental
- */
-public class FunctionDistinctValuesCollector extends DistinctValuesCollector<MutableValue> {
-
- private final Map<?, ?> vsContext;
- private final ValueSource groupSource;
- private final ValueSource countSource;
- private final Map<MutableValue, GroupCount<MutableValue>> groupMap;
-
- private FunctionValues.ValueFiller groupFiller;
- private FunctionValues.ValueFiller countFiller;
- private MutableValue groupMval;
- private MutableValue countMval;
-
- public FunctionDistinctValuesCollector(Map<?, ?> vsContext, ValueSource groupSource, ValueSource countSource, Collection<SearchGroup<MutableValue>> groups) {
- this.vsContext = vsContext;
- this.groupSource = groupSource;
- this.countSource = countSource;
- groupMap = new LinkedHashMap<>();
- for (SearchGroup<MutableValue> group : groups) {
- groupMap.put(group.groupValue, new GroupCount<>(group.groupValue));
- }
- }
-
- @Override
- public List<GroupCount<MutableValue>> getGroups() {
- return new ArrayList<>(groupMap.values());
- }
-
- @Override
- public void collect(int doc) throws IOException {
- groupFiller.fillValue(doc);
- GroupCount<MutableValue> groupCount = groupMap.get(groupMval);
- if (groupCount != null) {
- countFiller.fillValue(doc);
- groupCount.uniqueValues.add(countMval.duplicate());
- }
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- FunctionValues values = groupSource.getValues(vsContext, context);
- groupFiller = values.getValueFiller();
- groupMval = groupFiller.getValue();
- values = countSource.getValues(vsContext, context);
- countFiller = values.getValueFiller();
- countMval = countFiller.getValue();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java
deleted file mode 100644
index 85376e6..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.function;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
-import org.apache.lucene.util.mutable.MutableValue;
-
-import java.io.IOException;
-import java.util.Map;
-
-/**
- * Concrete implementation of {@link FirstPassGroupingCollector} that groups based on
- * {@link ValueSource} instances.
- *
- * @lucene.experimental
- */
-public class FunctionFirstPassGroupingCollector extends FirstPassGroupingCollector<MutableValue> {
-
- private final ValueSource groupByVS;
- private final Map<?, ?> vsContext;
-
- private FunctionValues.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) throws IOException {
- 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
- protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
- super.doSetNextReader(readerContext);
- FunctionValues values = groupByVS.getValues(vsContext, readerContext);
- filler = values.getValueFiller();
- mval = filler.getValue();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java
deleted file mode 100644
index 5204dc2..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.lucene.search.grouping.function;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.AllGroupsCollector;
-import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
-import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
-import org.apache.lucene.search.grouping.Grouper;
-import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.mutable.MutableValue;
-
-/**
- * Collector factory for grouping by ValueSource
- */
-public class FunctionGrouper extends Grouper<MutableValue> {
-
- private final ValueSource valueSource;
- private final Map<?, ?> context;
-
- /**
- * Create a Grouper for the provided ValueSource and context
- */
- public FunctionGrouper(ValueSource valueSource, Map<?, ?> context) {
- this.valueSource = valueSource;
- this.context = context;
- }
-
- @Override
- public FirstPassGroupingCollector<MutableValue> getFirstPassCollector(Sort sort, int count) throws IOException {
- return new FunctionFirstPassGroupingCollector(valueSource, context, sort, count);
- }
-
- @Override
- public AllGroupHeadsCollector<MutableValue> getGroupHeadsCollector(Sort sort) {
- return new FunctionAllGroupHeadsCollector(valueSource, context, sort);
- }
-
- @Override
- public AllGroupsCollector<MutableValue> getAllGroupsCollector() {
- return new FunctionAllGroupsCollector(valueSource, context);
- }
-
- @Override
- public SecondPassGroupingCollector<MutableValue> getSecondPassCollector(Collection<SearchGroup<MutableValue>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
- return new FunctionSecondPassGroupingCollector(searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields, valueSource, context);
- }
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java
deleted file mode 100644
index 45f2b37..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.function;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
-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 SecondPassGroupingCollector} that groups based on
- * {@link ValueSource} instances.
- *
- * @lucene.experimental
- */
-public class FunctionSecondPassGroupingCollector extends SecondPassGroupingCollector<MutableValue> {
-
- private final ValueSource groupByVS;
- private final Map<?, ?> vsContext;
-
- private FunctionValues.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;
- }
-
- @Override
- protected SearchGroupDocs<MutableValue> retrieveGroup(int doc) throws IOException {
- filler.fillValue(doc);
- return groupMap.get(mval);
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
- super.doSetNextReader(readerContext);
- FunctionValues values = groupByVS.getValues(vsContext, readerContext);
- filler = values.getValueFiller();
- mval = filler.getValue();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/package-info.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/package-info.java
deleted file mode 100644
index 73588ce..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/package-info.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Support for grouping by {@link org.apache.lucene.queries.function.ValueSource}.
- */
-package org.apache.lucene.search.grouping.function;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java
index 824a98e..7e3745e 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java
@@ -62,9 +62,9 @@
* </ul>
*
* <p>The implementation is two-pass: the first pass ({@link
- * org.apache.lucene.search.grouping.term.TermFirstPassGroupingCollector})
+ * org.apache.lucene.search.grouping.FirstPassGroupingCollector})
* gathers the top groups, and the second pass ({@link
- * org.apache.lucene.search.grouping.term.TermSecondPassGroupingCollector})
+ * org.apache.lucene.search.grouping.SecondPassGroupingCollector})
* 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
@@ -73,18 +73,17 @@
* hold all hits. Results are returned as a {@link
* org.apache.lucene.search.grouping.TopGroups} instance.</p>
*
- * <p>
- * This module abstracts away what defines group and how it is collected. All grouping collectors
- * are abstract and have currently term based implementations. One can implement
- * collectors that for example group on multiple fields.
- * </p>
+ * <p>Groups are defined by {@link org.apache.lucene.search.grouping.GroupSelector}
+ * implementations:</p>
+ * <ul>
+ * <li>{@link org.apache.lucene.search.grouping.TermGroupSelector} groups based on
+ * the value of a {@link org.apache.lucene.index.SortedDocValues} field</li>
+ * <li>{@link org.apache.lucene.search.grouping.ValueSourceGroupSelector} groups based on
+ * the value of a {@link org.apache.lucene.queries.function.ValueSource}</li>
+ * </ul>
*
* <p>Known limitations:</p>
* <ul>
- * <li> For the two-pass grouping search, the group field must be a
- * indexed as a {@link org.apache.lucene.document.SortedDocValuesField}).
- * <li> Although Solr support grouping by function and this module has abstraction of what a group is, there are currently only
- * implementations for grouping based on terms.
* <li> Sharding is not directly supported, though is not too
* difficult, if you can merge the top groups and top documents per
* group yourself.
@@ -174,14 +173,15 @@
* have to separately retrieve it (for example using stored
* fields, <code>FieldCache</code>, etc.).
*
- * <p>Another collector is the <code>TermAllGroupHeadsCollector</code> that can be used to retrieve all most relevant
+ * <p>Another collector is the <code>AllGroupHeadsCollector</code> that can be used to retrieve all most relevant
* documents per group. Also known as group heads. This can be useful in situations when one wants to compute group
* based facets / statistics on the complete query result. The collector can be executed during the first or second
* phase. This collector can also be used with the <code>GroupingSearch</code> convenience utility, but when if one only
* wants to compute the most relevant documents per group it is better to just use the collector as done here below.</p>
*
* <pre class="prettyprint">
- * AbstractAllGroupHeadsCollector c = TermAllGroupHeadsCollector.create(groupField, sortWithinGroup);
+ * TermGroupSelector grouper = new TermGroupSelector(groupField);
+ * AllGroupHeadsCollector c = AllGroupHeadsCollector.newCollector(grouper, sortWithinGroup);
* s.search(new TermQuery(new Term("content", searchTerm)), c);
* // Return all group heads as int array
* int[] groupHeadsArray = c.retrieveGroupHeads()
@@ -189,12 +189,6 @@
* int maxDoc = s.maxDoc();
* FixedBitSet groupHeadsBitSet = c.retrieveGroupHeads(maxDoc)
* </pre>
- *
- * <p>For each of the above collector types 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, but can also be used with the
- * <code>GroupingSearch</code> convenience utility
- * </p>
+ *
*/
package org.apache.lucene.search.grouping;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java
deleted file mode 100644
index 54e2399..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java
+++ /dev/null
@@ -1,767 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.term;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.FieldComparator;
-import org.apache.lucene.search.LeafFieldComparator;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.SentinelIntSet;
-
-/**
- * A base implementation of {@link AllGroupHeadsCollector} 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 SortedDocValues}.
- *
- * @lucene.experimental
- */
-public abstract class TermAllGroupHeadsCollector extends AllGroupHeadsCollector<BytesRef> {
-
- private static final int DEFAULT_INITIAL_SIZE = 128;
-
- final String groupField;
-
- SortedDocValues groupIndex;
- LeafReaderContext readerContext;
-
- protected TermAllGroupHeadsCollector(String groupField, int numberOfSorts) {
- super(numberOfSorts);
- this.groupField = groupField;
- }
-
- /**
- * Creates an <code>AbstractAllGroupHeadsCollector</code> instance based on the supplied arguments.
- * This factory method decides with implementation is best suited.
- *
- * Delegates to {@link #create(String, org.apache.lucene.search.Sort, int)} with an initialSize of 128.
- *
- * @param groupField The field to group by
- * @param sortWithinGroup The sort within each group
- * @return an <code>AbstractAllGroupHeadsCollector</code> instance based on the supplied arguments
- */
- public static AllGroupHeadsCollector<BytesRef> create(String groupField, Sort sortWithinGroup) {
- return create(groupField, sortWithinGroup, DEFAULT_INITIAL_SIZE);
- }
-
- /**
- * Creates an <code>AbstractAllGroupHeadsCollector</code> instance based on the supplied arguments.
- * This factory method decides with implementation is best suited.
- *
- * @param groupField The field to group by
- * @param sortWithinGroup The sort within each group
- * @param initialSize The initial allocation size of the internal int set and group list which should roughly match
- * the total number of expected unique groups. Be aware that the heap usage is
- * 4 bytes * initialSize.
- * @return an <code>AbstractAllGroupHeadsCollector</code> instance based on the supplied arguments
- */
- public static AllGroupHeadsCollector<BytesRef> create(String groupField, Sort sortWithinGroup, int initialSize) {
- boolean sortAllScore = true;
- boolean sortAllFieldValue = true;
-
- for (SortField sortField : sortWithinGroup.getSort()) {
- if (sortField.getType() == SortField.Type.SCORE) {
- sortAllFieldValue = false;
- } else if (needGeneralImpl(sortField)) {
- return new GeneralAllGroupHeadsCollector(groupField, sortWithinGroup);
- } else {
- sortAllScore = false;
- }
- }
-
- if (sortAllScore) {
- return new ScoreAllGroupHeadsCollector(groupField, sortWithinGroup, initialSize);
- } else if (sortAllFieldValue) {
- return new OrdAllGroupHeadsCollector(groupField, sortWithinGroup, initialSize);
- } else {
- return new OrdScoreAllGroupHeadsCollector(groupField, sortWithinGroup, initialSize);
- }
- }
-
- // Returns when a sort field needs the general impl.
- private static boolean needGeneralImpl(SortField sortField) {
- SortField.Type sortType = sortField.getType();
- // Note (MvG): We can also make an optimized impl when sorting is SortField.DOC
- return sortType != SortField.Type.STRING_VAL && sortType != SortField.Type.STRING && sortType != SortField.Type.SCORE;
- }
-
- // A general impl that works for any group sort.
- static class GeneralAllGroupHeadsCollector extends TermAllGroupHeadsCollector {
-
- private final Sort sortWithinGroup;
- private final Map<BytesRef, GroupHead> groups;
-
- Scorer scorer;
-
- GeneralAllGroupHeadsCollector(String groupField, Sort sortWithinGroup) {
- super(groupField, sortWithinGroup.getSort().length);
- this.sortWithinGroup = sortWithinGroup;
- groups = new HashMap<>();
-
- final SortField[] sortFields = sortWithinGroup.getSort();
- for (int i = 0; i < sortFields.length; i++) {
- reversed[i] = sortFields[i].getReverse() ? -1 : 1;
- }
- }
-
- protected int getOrdForDoc(int doc) throws IOException {
- int curDocID = groupIndex.docID();
- if (curDocID < doc) {
- curDocID = groupIndex.advance(doc);
- }
-
- if (curDocID == doc) {
- return groupIndex.ordValue();
- } else {
- return -1;
- }
- }
-
- @Override
- protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
- int ord = getOrdForDoc(doc);
-
- BytesRef groupValue;
- if (ord == -1) {
- groupValue = null;
- } else {
- groupValue = groupIndex.lookupOrd(ord);
- }
-
- GroupHead groupHead = groups.get(groupValue);
- if (groupHead == null) {
- groupValue = groupValue == null ? null : BytesRef.deepCopyOf(groupValue);
- groupHead = new GroupHead(groupValue, sortWithinGroup, doc);
- groups.put(groupValue, groupHead);
- temporalResult.stop = true;
- } else {
- temporalResult.stop = false;
- }
- temporalResult.groupHead = groupHead;
- }
-
- @Override
- protected Collection<GroupHead> getCollectedGroupHeads() {
- return groups.values();
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- this.readerContext = context;
- groupIndex = DocValues.getSorted(context.reader(), groupField);
-
- for (GroupHead groupHead : groups.values()) {
- for (int i = 0; i < groupHead.comparators.length; i++) {
- groupHead.leafComparators[i] = groupHead.comparators[i].getLeafComparator(context);
- }
- }
- }
-
- @Override
- public boolean needsScores() {
- return sortWithinGroup.needsScores();
- }
-
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- this.scorer = scorer;
- for (GroupHead groupHead : groups.values()) {
- for (LeafFieldComparator comparator : groupHead.leafComparators) {
- comparator.setScorer(scorer);
- }
- }
- }
-
- class GroupHead extends AllGroupHeadsCollector.GroupHead<BytesRef> {
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- final FieldComparator[] comparators;
-
- final LeafFieldComparator[] leafComparators;
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- GroupHead(BytesRef groupValue, Sort sort, int doc) throws IOException {
- super(groupValue, doc + readerContext.docBase);
- final SortField[] sortFields = sort.getSort();
- comparators = new FieldComparator[sortFields.length];
- leafComparators = new LeafFieldComparator[sortFields.length];
- for (int i = 0; i < sortFields.length; i++) {
- comparators[i] = sortFields[i].getComparator(1, i);
- leafComparators[i] = comparators[i].getLeafComparator(readerContext);
- leafComparators[i].setScorer(scorer);
- leafComparators[i].copy(0, doc);
- leafComparators[i].setBottom(0);
- }
- }
-
- @Override
- public int compare(int compIDX, int doc) throws IOException {
- return leafComparators[compIDX].compareBottom(doc);
- }
-
- @Override
- public void updateDocHead(int doc) throws IOException {
- for (LeafFieldComparator comparator : leafComparators) {
- comparator.copy(0, doc);
- comparator.setBottom(0);
- }
- this.doc = doc + readerContext.docBase;
- }
- }
- }
-
-
- // AbstractAllGroupHeadsCollector optimized for ord fields and scores.
- static class OrdScoreAllGroupHeadsCollector extends TermAllGroupHeadsCollector {
-
- private final SentinelIntSet ordSet;
- private final List<GroupHead> collectedGroups;
- final SortField[] fields;
-
- SortedDocValues[] sortsIndex;
- Scorer scorer;
- private GroupHead[] segmentGroupHeads;
-
- OrdScoreAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
- super(groupField, sortWithinGroup.getSort().length);
- ordSet = new SentinelIntSet(initialSize, -2);
- collectedGroups = new ArrayList<>(initialSize);
-
- final SortField[] sortFields = sortWithinGroup.getSort();
- fields = new SortField[sortFields.length];
- sortsIndex = new SortedDocValues[sortFields.length];
- for (int i = 0; i < sortFields.length; i++) {
- reversed[i] = sortFields[i].getReverse() ? -1 : 1;
- fields[i] = sortFields[i];
- }
- }
-
- @Override
- protected Collection<GroupHead> getCollectedGroupHeads() {
- return collectedGroups;
- }
-
- @Override
- public boolean needsScores() {
- return true;
- }
-
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- this.scorer = scorer;
- }
-
- private int getOrdForDoc(int doc) throws IOException {
- int curDocID = groupIndex.docID();
- if (curDocID < doc) {
- curDocID = groupIndex.advance(doc);
- }
-
- if (curDocID == doc) {
- return groupIndex.ordValue();
- } else {
- return -1;
- }
- }
-
- @Override
- protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
- int key = getOrdForDoc(doc);
- GroupHead groupHead;
- if (!ordSet.exists(key)) {
- ordSet.put(key);
- final BytesRef term;
- if (key == -1) {
- term = null;
- } else {
- term = BytesRef.deepCopyOf(groupIndex.lookupOrd(key));
- }
- groupHead = new GroupHead(doc, term);
- collectedGroups.add(groupHead);
- segmentGroupHeads[key+1] = groupHead;
- temporalResult.stop = true;
- } else {
- temporalResult.stop = false;
- groupHead = segmentGroupHeads[key+1];
- }
- temporalResult.groupHead = groupHead;
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- this.readerContext = context;
- groupIndex = DocValues.getSorted(context.reader(), groupField);
- for (int i = 0; i < fields.length; i++) {
- if (fields[i].getType() == SortField.Type.SCORE) {
- continue;
- }
-
- sortsIndex[i] = DocValues.getSorted(context.reader(), fields[i].getField());
- }
-
- // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
- ordSet.clear();
- segmentGroupHeads = new GroupHead[groupIndex.getValueCount()+1];
- for (GroupHead collectedGroup : collectedGroups) {
- int ord;
- if (collectedGroup.groupValue == null) {
- ord = -1;
- } else {
- ord = groupIndex.lookupTerm(collectedGroup.groupValue);
- }
- if (collectedGroup.groupValue == null || ord >= 0) {
- ordSet.put(ord);
- segmentGroupHeads[ord+1] = collectedGroup;
-
- for (int i = 0; i < sortsIndex.length; i++) {
- if (fields[i].getType() == SortField.Type.SCORE) {
- continue;
- }
- int sortOrd;
- if (collectedGroup.sortValues[i] == null) {
- sortOrd = -1;
- } else {
- sortOrd = sortsIndex[i].lookupTerm(collectedGroup.sortValues[i].get());
- }
- collectedGroup.sortOrds[i] = sortOrd;
- }
- }
- }
- }
-
- void setDoc(int docID) throws IOException {
- for (int i = 0; i < sortsIndex.length; i++) {
- SortedDocValues values = sortsIndex[i];
- if (values != null && docID > values.docID()) {
- values.advance(docID);
- }
- }
- }
-
- class GroupHead extends AllGroupHeadsCollector.GroupHead<BytesRef> {
-
- BytesRefBuilder[] sortValues;
- int[] sortOrds;
- float[] scores;
-
- GroupHead(int doc, BytesRef groupValue) throws IOException {
- super(groupValue, doc + readerContext.docBase);
- sortValues = new BytesRefBuilder[sortsIndex.length];
- sortOrds = new int[sortsIndex.length];
- scores = new float[sortsIndex.length];
- setDoc(doc);
- for (int i = 0; i < sortsIndex.length; i++) {
- if (fields[i].getType() == SortField.Type.SCORE) {
- scores[i] = scorer.score();
- } else {
- if (doc == sortsIndex[i].docID()) {
- sortOrds[i] = sortsIndex[i].ordValue();
- } else {
- sortOrds[i] = -1;
- }
- sortValues[i] = new BytesRefBuilder();
- if (sortOrds[i] != -1) {
- sortValues[i].copyBytes(sortsIndex[i].binaryValue());
- }
- }
- }
- }
-
- @Override
- public int compare(int compIDX, int doc) throws IOException {
- if (fields[compIDX].getType() == SortField.Type.SCORE) {
- float score = scorer.score();
- if (scores[compIDX] < score) {
- return 1;
- } else if (scores[compIDX] > score) {
- return -1;
- }
- return 0;
- } else {
- if (sortsIndex[compIDX].docID() < doc) {
- sortsIndex[compIDX].advance(doc);
- }
- if (sortOrds[compIDX] < 0) {
- // The current segment doesn't contain the sort value we encountered before. Therefore the ord is negative.
- final BytesRef term;
- if (sortsIndex[compIDX].docID() == doc) {
- term = sortsIndex[compIDX].binaryValue();
- } else {
- term = new BytesRef(BytesRef.EMPTY_BYTES);
- }
- return sortValues[compIDX].get().compareTo(term);
- } else {
- int ord;
- if (sortsIndex[compIDX].docID() == doc) {
- ord = sortsIndex[compIDX].ordValue();
- } else {
- ord = -1;
- }
- return sortOrds[compIDX] - ord;
- }
- }
- }
-
- @Override
- public void updateDocHead(int doc) throws IOException {
- setDoc(doc);
- for (int i = 0; i < sortsIndex.length; i++) {
- if (fields[i].getType() == SortField.Type.SCORE) {
- scores[i] = scorer.score();
- } else {
- if (sortsIndex[i].docID() == doc) {
- sortOrds[i] = sortsIndex[i].ordValue();
- sortValues[i].copyBytes(sortsIndex[i].binaryValue());
- } else {
- sortOrds[i] = -1;
- sortValues[i].clear();
- }
- }
- }
- this.doc = doc + readerContext.docBase;
- }
- }
- }
-
-
- // AbstractAllGroupHeadsCollector optimized for ord fields.
- static class OrdAllGroupHeadsCollector extends TermAllGroupHeadsCollector {
-
- private final SentinelIntSet ordSet;
- private final List<GroupHead> collectedGroups;
- private final SortField[] fields;
-
- SortedDocValues[] sortsIndex;
- GroupHead[] segmentGroupHeads;
-
- OrdAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
- super(groupField, sortWithinGroup.getSort().length);
- ordSet = new SentinelIntSet(initialSize, -2);
- collectedGroups = new ArrayList<>(initialSize);
-
- final SortField[] sortFields = sortWithinGroup.getSort();
- fields = new SortField[sortFields.length];
- sortsIndex = new SortedDocValues[sortFields.length];
- for (int i = 0; i < sortFields.length; i++) {
- reversed[i] = sortFields[i].getReverse() ? -1 : 1;
- fields[i] = sortFields[i];
- }
- }
-
- @Override
- protected Collection<GroupHead> getCollectedGroupHeads() {
- return collectedGroups;
- }
-
- @Override
- public boolean needsScores() {
- return false;
- }
-
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- }
-
- @Override
- protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
- if (doc > groupIndex.docID()) {
- groupIndex.advance(doc);
- }
-
- int key;
- if (doc == groupIndex.docID()) {
- key = groupIndex.ordValue();
- } else {
- key = -1;
- }
-
- GroupHead groupHead;
- if (!ordSet.exists(key)) {
- ordSet.put(key);
- final BytesRef term;
- if (key == -1) {
- term = null;
- } else {
- term = BytesRef.deepCopyOf(groupIndex.lookupOrd(key));
- }
- groupHead = new GroupHead(doc, term);
- collectedGroups.add(groupHead);
- segmentGroupHeads[key+1] = groupHead;
- temporalResult.stop = true;
- } else {
- temporalResult.stop = false;
- groupHead = segmentGroupHeads[key+1];
- }
- temporalResult.groupHead = groupHead;
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- this.readerContext = context;
- groupIndex = DocValues.getSorted(context.reader(), groupField);
- for (int i = 0; i < fields.length; i++) {
- sortsIndex[i] = DocValues.getSorted(context.reader(), fields[i].getField());
- }
-
- // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
- ordSet.clear();
- segmentGroupHeads = new GroupHead[groupIndex.getValueCount()+1];
- for (GroupHead collectedGroup : collectedGroups) {
- int groupOrd;
- if (collectedGroup.groupValue == null) {
- groupOrd = -1;
- } else {
- groupOrd = groupIndex.lookupTerm(collectedGroup.groupValue);
- }
- if (collectedGroup.groupValue == null || groupOrd >= 0) {
- ordSet.put(groupOrd);
- segmentGroupHeads[groupOrd+1] = collectedGroup;
-
- for (int i = 0; i < sortsIndex.length; i++) {
- int sortOrd;
- if (collectedGroup.sortOrds[i] == -1) {
- sortOrd = -1;
- } else {
- sortOrd = sortsIndex[i].lookupTerm(collectedGroup.sortValues[i].get());
- }
- collectedGroup.sortOrds[i] = sortOrd;
- }
- }
- }
- }
-
- void setDoc(int docID) throws IOException {
- for (int i = 0; i < sortsIndex.length; i++) {
- SortedDocValues values = sortsIndex[i];
- if (docID > values.docID()) {
- values.advance(docID);
- }
- }
- }
-
- class GroupHead extends AllGroupHeadsCollector.GroupHead<BytesRef> {
-
- BytesRefBuilder[] sortValues;
- int[] sortOrds;
-
- GroupHead(int doc, BytesRef groupValue) throws IOException {
- super(groupValue, doc + readerContext.docBase);
- sortValues = new BytesRefBuilder[sortsIndex.length];
- sortOrds = new int[sortsIndex.length];
- setDoc(doc);
- for (int i = 0; i < sortsIndex.length; i++) {
- if (doc == sortsIndex[i].docID()) {
- sortOrds[i] = sortsIndex[i].ordValue();
- } else {
- sortOrds[i] = -1;
- }
- sortValues[i] = new BytesRefBuilder();
- if (sortOrds[i] != -1) {
- sortValues[i].copyBytes(sortsIndex[i].binaryValue());
- }
- }
- }
-
- @Override
- public int compare(int compIDX, int doc) throws IOException {
- if (sortsIndex[compIDX].docID() < doc) {
- sortsIndex[compIDX].advance(doc);
- }
- if (sortOrds[compIDX] < 0) {
- // The current segment doesn't contain the sort value we encountered before. Therefore the ord is negative.
- final BytesRef term;
- if (sortsIndex[compIDX].docID() == doc) {
- term = sortsIndex[compIDX].binaryValue();
- } else {
- term = new BytesRef(BytesRef.EMPTY_BYTES);
- }
- return sortValues[compIDX].get().compareTo(term);
- } else {
- int ord;
- if (sortsIndex[compIDX].docID() == doc) {
- ord = sortsIndex[compIDX].ordValue();
- } else {
- ord = -1;
- }
- return sortOrds[compIDX] - ord;
- }
- }
-
- @Override
- public void updateDocHead(int doc) throws IOException {
- setDoc(doc);
- for (int i = 0; i < sortsIndex.length; i++) {
- if (sortsIndex[i].docID() == doc) {
- sortOrds[i] = sortsIndex[i].ordValue();
- sortValues[i].copyBytes(sortsIndex[i].binaryValue());
- } else {
- sortOrds[i] = -1;
- sortValues[i].clear();
- }
- }
- this.doc = doc + readerContext.docBase;
- }
-
- }
-
- }
-
-
- // AbstractAllGroupHeadsCollector optimized for scores.
- static class ScoreAllGroupHeadsCollector extends TermAllGroupHeadsCollector {
-
- final SentinelIntSet ordSet;
- final List<GroupHead> collectedGroups;
- final SortField[] fields;
-
- Scorer scorer;
- GroupHead[] segmentGroupHeads;
-
- ScoreAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
- super(groupField, sortWithinGroup.getSort().length);
- ordSet = new SentinelIntSet(initialSize, -2);
- collectedGroups = new ArrayList<>(initialSize);
-
- final SortField[] sortFields = sortWithinGroup.getSort();
- fields = new SortField[sortFields.length];
- for (int i = 0; i < sortFields.length; i++) {
- reversed[i] = sortFields[i].getReverse() ? -1 : 1;
- fields[i] = sortFields[i];
- }
- }
-
- @Override
- protected Collection<GroupHead> getCollectedGroupHeads() {
- return collectedGroups;
- }
-
- @Override
- public boolean needsScores() {
- return true;
- }
-
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- this.scorer = scorer;
- }
-
- @Override
- protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
- if (doc > groupIndex.docID()) {
- groupIndex.advance(doc);
- }
-
- int key;
- if (doc == groupIndex.docID()) {
- key = groupIndex.ordValue();
- } else {
- key = -1;
- }
-
- GroupHead groupHead;
- if (!ordSet.exists(key)) {
- ordSet.put(key);
- final BytesRef term;
- if (key == -1) {
- term = null;
- } else {
- term = BytesRef.deepCopyOf(groupIndex.lookupOrd(key));
- }
- groupHead = new GroupHead(doc, term);
- collectedGroups.add(groupHead);
- segmentGroupHeads[key+1] = groupHead;
- temporalResult.stop = true;
- } else {
- temporalResult.stop = false;
- groupHead = segmentGroupHeads[key+1];
- }
- temporalResult.groupHead = groupHead;
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- this.readerContext = context;
- groupIndex = DocValues.getSorted(context.reader(), groupField);
-
- // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
- ordSet.clear();
- segmentGroupHeads = new GroupHead[groupIndex.getValueCount()+1];
- for (GroupHead collectedGroup : collectedGroups) {
- int ord;
- if (collectedGroup.groupValue == null) {
- ord = -1;
- } else {
- ord = groupIndex.lookupTerm(collectedGroup.groupValue);
- }
- if (collectedGroup.groupValue == null || ord >= 0) {
- ordSet.put(ord);
- segmentGroupHeads[ord+1] = collectedGroup;
- }
- }
- }
-
- class GroupHead extends AllGroupHeadsCollector.GroupHead<BytesRef> {
-
- float[] scores;
-
- GroupHead(int doc, BytesRef groupValue) throws IOException {
- super(groupValue, doc + readerContext.docBase);
- scores = new float[fields.length];
- float score = scorer.score();
- for (int i = 0; i < scores.length; i++) {
- scores[i] = score;
- }
- }
-
- @Override
- public int compare(int compIDX, int doc) throws IOException {
- float score = scorer.score();
- if (scores[compIDX] < score) {
- return 1;
- } else if (scores[compIDX] > score) {
- return -1;
- }
- return 0;
- }
-
- @Override
- public void updateDocHead(int doc) throws IOException {
- float score = scorer.score();
- for (int i = 0; i < scores.length; i++) {
- scores[i] = score;
- }
- this.doc = doc + readerContext.docBase;
- }
-
- }
-
- }
-
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java
deleted file mode 100644
index 125555a..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.term;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.grouping.AllGroupsCollector;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.SentinelIntSet;
-
-/**
- * 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: an int hash set (SentinelIntSet)
- * is used to detect if a group is already added to the
- * total count. For each segment the int set is cleared and filled
- * with previous counted groups that occur in the new
- * segment.
- *
- * @lucene.experimental
- */
-public class TermAllGroupsCollector extends AllGroupsCollector<BytesRef> {
-
- private static final int DEFAULT_INITIAL_SIZE = 128;
-
- private final String groupField;
- private final SentinelIntSet ordSet;
- private final List<BytesRef> groups;
-
- private SortedDocValues index;
-
- /**
- * Expert: Constructs a {@link AllGroupsCollector}
- *
- * @param groupField The field to group by
- * @param initialSize The initial allocation size of the
- * internal int set and group list
- * which should roughly match the total
- * number of expected unique groups. Be aware that the
- * heap usage is 4 bytes * initialSize.
- */
- public TermAllGroupsCollector(String groupField, int initialSize) {
- ordSet = new SentinelIntSet(initialSize, -2);
- groups = new ArrayList<>(initialSize);
- this.groupField = groupField;
- }
-
- /**
- * Constructs a {@link AllGroupsCollector}. This sets the
- * initial allocation size for the internal int set and group
- * list to 128.
- *
- * @param groupField The field to group by
- */
- public TermAllGroupsCollector(String groupField) {
- this(groupField, DEFAULT_INITIAL_SIZE);
- }
-
- @Override
- public void collect(int doc) throws IOException {
- if (doc > index.docID()) {
- index.advance(doc);
- }
- int key;
- if (doc == index.docID()) {
- key = index.ordValue();
- } else {
- key = -1;
- }
- if (!ordSet.exists(key)) {
- ordSet.put(key);
- final BytesRef term;
- if (key == -1) {
- term = null;
- } else {
- term = BytesRef.deepCopyOf(index.lookupOrd(key));
- }
- groups.add(term);
- }
- }
-
- @Override
- public Collection<BytesRef> getGroups() {
- return groups;
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- index = DocValues.getSorted(context.reader(), groupField);
-
- // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
- ordSet.clear();
- for (BytesRef countedGroup : groups) {
- if (countedGroup == null) {
- ordSet.put(-1);
- } else {
- int ord = index.lookupTerm(countedGroup);
- if (ord >= 0) {
- ordSet.put(ord);
- }
- }
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java
deleted file mode 100644
index e5356a3..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.term;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.grouping.DistinctValuesCollector;
-import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.SentinelIntSet;
-
-/**
- * A term based implementation of {@link DistinctValuesCollector} that relies
- * on {@link SortedDocValues} to count the distinct values per group.
- *
- * @lucene.experimental
- */
-public class TermDistinctValuesCollector extends DistinctValuesCollector<BytesRef> {
-
- private final String groupField;
- private final String countField;
- private final List<TermGroupCount> groups;
- private final SentinelIntSet ordSet;
- private final TermGroupCount groupCounts[];
-
- private SortedDocValues groupFieldTermIndex;
- private SortedDocValues countFieldTermIndex;
-
- /**
- * Constructs {@link TermDistinctValuesCollector} instance.
- *
- * @param groupField The field to group by
- * @param countField The field to count distinct values for
- * @param groups The top N groups, collected during the first phase search
- */
- public TermDistinctValuesCollector(String groupField, String countField, Collection<SearchGroup<BytesRef>> groups) {
- this.groupField = groupField;
- this.countField = countField;
- this.groups = new ArrayList<>(groups.size());
- for (SearchGroup<BytesRef> group : groups) {
- this.groups.add(new TermGroupCount(group.groupValue));
- }
- ordSet = new SentinelIntSet(groups.size(), -2);
- groupCounts = new TermGroupCount[ordSet.keys.length];
- }
-
- @Override
- public void collect(int doc) throws IOException {
- if (doc > groupFieldTermIndex.docID()) {
- groupFieldTermIndex.advance(doc);
- }
- int ord;
- if (doc == groupFieldTermIndex.docID()) {
- ord = groupFieldTermIndex.ordValue();
- } else {
- ord = -1;
- }
- int slot = ordSet.find(ord);
- if (slot < 0) {
- return;
- }
-
- TermGroupCount gc = groupCounts[slot];
- if (doc > countFieldTermIndex.docID()) {
- countFieldTermIndex.advance(doc);
- }
-
- int countOrd;
- if (doc == countFieldTermIndex.docID()) {
- countOrd = countFieldTermIndex.ordValue();
- } else {
- countOrd = -1;
- }
-
- if (doesNotContainOrd(countOrd, gc.ords)) {
- if (countOrd == -1) {
- gc.uniqueValues.add(null);
- } else {
- BytesRef term = BytesRef.deepCopyOf(countFieldTermIndex.lookupOrd(countOrd));
- gc.uniqueValues.add(term);
- }
-
- gc.ords = Arrays.copyOf(gc.ords, gc.ords.length + 1);
- gc.ords[gc.ords.length - 1] = countOrd;
- if (gc.ords.length > 1) {
- Arrays.sort(gc.ords);
- }
- }
- }
-
- private boolean doesNotContainOrd(int ord, int[] ords) {
- if (ords.length == 0) {
- return true;
- } else if (ords.length == 1) {
- return ord != ords[0];
- }
- return Arrays.binarySearch(ords, ord) < 0;
- }
-
- @Override
- public List<GroupCount<BytesRef>> getGroups() {
- return new ArrayList<>(groups);
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- groupFieldTermIndex = DocValues.getSorted(context.reader(), groupField);
- countFieldTermIndex = DocValues.getSorted(context.reader(), countField);
- ordSet.clear();
- for (TermGroupCount group : groups) {
- int groupOrd = group.groupValue == null ? -1 : groupFieldTermIndex.lookupTerm(group.groupValue);
- if (group.groupValue != null && groupOrd < 0) {
- continue;
- }
-
- groupCounts[ordSet.put(groupOrd)] = group;
- group.ords = new int[group.uniqueValues.size()];
- Arrays.fill(group.ords, -2);
- int i = 0;
- for (BytesRef value : group.uniqueValues) {
- int countOrd = value == null ? -1 : countFieldTermIndex.lookupTerm(value);
- if (value == null || countOrd >= 0) {
- group.ords[i++] = countOrd;
- }
- }
- }
- }
-
- /** Holds distinct values for a single group.
- *
- * @lucene.experimental */
- public static class TermGroupCount extends DistinctValuesCollector.GroupCount<BytesRef> {
-
- int[] ords;
-
- TermGroupCount(BytesRef groupValue) {
- super(groupValue);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java
deleted file mode 100644
index 3c35fa8..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.term;
-
-import java.io.IOException;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BytesRef;
-
-/**
- * Concrete implementation of {@link FirstPassGroupingCollector} that groups based on
- * field values and more specifically uses {@link SortedDocValues}
- * to collect groups.
- *
- * @lucene.experimental
- */
-public class TermFirstPassGroupingCollector extends FirstPassGroupingCollector<BytesRef> {
-
- private SortedDocValues index;
-
- private String groupField;
-
- /**
- * Create the first pass collector.
- *
- * @param groupField The field used to group
- * documents. This field must be single-valued and
- * indexed (DocValues is used to access its value
- * per-document).
- * @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 TermFirstPassGroupingCollector(String groupField, Sort groupSort, int topNGroups) throws IOException {
- super(groupSort, topNGroups);
- this.groupField = groupField;
- }
-
- @Override
- protected BytesRef getDocGroupValue(int doc) throws IOException {
- if (doc > index.docID()) {
- index.advance(doc);
- }
- if (doc == index.docID()) {
- return index.binaryValue();
- } else {
- return null;
- }
- }
-
- @Override
- protected BytesRef copyDocGroupValue(BytesRef groupValue, BytesRef reuse) {
- if (groupValue == null) {
- return null;
- } else if (reuse != null) {
- reuse.bytes = ArrayUtil.grow(reuse.bytes, groupValue.length);
- reuse.offset = 0;
- reuse.length = groupValue.length;
- System.arraycopy(groupValue.bytes, groupValue.offset, reuse.bytes, 0, groupValue.length);
- return reuse;
- } else {
- return BytesRef.deepCopyOf(groupValue);
- }
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
- super.doSetNextReader(readerContext);
- index = DocValues.getSorted(readerContext.reader(), groupField);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java
deleted file mode 100644
index cee327c..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java
+++ /dev/null
@@ -1,415 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.term;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.grouping.GroupFacetCollector;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.SentinelIntSet;
-import org.apache.lucene.util.UnicodeUtil;
-
-/**
- * An implementation of {@link GroupFacetCollector} that computes grouped facets based on the indexed terms
- * from DocValues.
- *
- * @lucene.experimental
- */
-public abstract class TermGroupFacetCollector extends GroupFacetCollector {
-
- final List<GroupedFacetHit> groupedFacetHits;
- final SentinelIntSet segmentGroupedFacetHits;
-
- SortedDocValues groupFieldTermsIndex;
-
- /**
- * Factory method for creating the right implementation based on the fact whether the facet field contains
- * multiple tokens per documents.
- *
- * @param groupField The group field
- * @param facetField The facet field
- * @param facetFieldMultivalued Whether the facet field has multiple tokens per document
- * @param facetPrefix The facet prefix a facet entry should start with to be included.
- * @param initialSize The initial allocation size of the internal int set and group facet list which should roughly
- * match the total number of expected unique groups. Be aware that the heap usage is
- * 4 bytes * initialSize.
- * @return <code>TermGroupFacetCollector</code> implementation
- */
- public static TermGroupFacetCollector createTermGroupFacetCollector(String groupField,
- String facetField,
- boolean facetFieldMultivalued,
- BytesRef facetPrefix,
- int initialSize) {
- if (facetFieldMultivalued) {
- return new MV(groupField, facetField, facetPrefix, initialSize);
- } else {
- return new SV(groupField, facetField, facetPrefix, initialSize);
- }
- }
-
- TermGroupFacetCollector(String groupField, String facetField, BytesRef facetPrefix, int initialSize) {
- super(groupField, facetField, facetPrefix);
- groupedFacetHits = new ArrayList<>(initialSize);
- segmentGroupedFacetHits = new SentinelIntSet(initialSize, Integer.MIN_VALUE);
- }
-
- // Implementation for single valued facet fields.
- static class SV extends TermGroupFacetCollector {
-
- private SortedDocValues facetFieldTermsIndex;
-
- SV(String groupField, String facetField, BytesRef facetPrefix, int initialSize) {
- super(groupField, facetField, facetPrefix, initialSize);
- }
-
- @Override
- public void collect(int doc) throws IOException {
- if (doc > facetFieldTermsIndex.docID()) {
- facetFieldTermsIndex.advance(doc);
- }
-
- int facetOrd;
- if (doc == facetFieldTermsIndex.docID()) {
- facetOrd = facetFieldTermsIndex.ordValue();
- } else {
- facetOrd = -1;
- }
-
- if (facetOrd < startFacetOrd || facetOrd >= endFacetOrd) {
- return;
- }
-
- if (doc > groupFieldTermsIndex.docID()) {
- groupFieldTermsIndex.advance(doc);
- }
-
- int groupOrd;
- if (doc == groupFieldTermsIndex.docID()) {
- groupOrd = groupFieldTermsIndex.ordValue();
- } else {
- groupOrd = -1;
- }
- int segmentGroupedFacetsIndex = groupOrd * (facetFieldTermsIndex.getValueCount()+1) + facetOrd;
- if (segmentGroupedFacetHits.exists(segmentGroupedFacetsIndex)) {
- return;
- }
-
- segmentTotalCount++;
- segmentFacetCounts[facetOrd+1]++;
-
- segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
-
- BytesRef groupKey;
- if (groupOrd == -1) {
- groupKey = null;
- } else {
- groupKey = BytesRef.deepCopyOf(groupFieldTermsIndex.lookupOrd(groupOrd));
- }
-
- BytesRef facetKey;
- if (facetOrd == -1) {
- facetKey = null;
- } else {
- facetKey = BytesRef.deepCopyOf(facetFieldTermsIndex.lookupOrd(facetOrd));
- }
-
- groupedFacetHits.add(new GroupedFacetHit(groupKey, facetKey));
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- if (segmentFacetCounts != null) {
- segmentResults.add(createSegmentResult());
- }
-
- groupFieldTermsIndex = DocValues.getSorted(context.reader(), groupField);
- facetFieldTermsIndex = DocValues.getSorted(context.reader(), facetField);
-
- // 1+ to allow for the -1 "not set":
- segmentFacetCounts = new int[facetFieldTermsIndex.getValueCount()+1];
- segmentTotalCount = 0;
-
- segmentGroupedFacetHits.clear();
- for (GroupedFacetHit groupedFacetHit : groupedFacetHits) {
- int facetOrd = groupedFacetHit.facetValue == null ? -1 : facetFieldTermsIndex.lookupTerm(groupedFacetHit.facetValue);
- if (groupedFacetHit.facetValue != null && facetOrd < 0) {
- continue;
- }
-
- int groupOrd = groupedFacetHit.groupValue == null ? -1 : groupFieldTermsIndex.lookupTerm(groupedFacetHit.groupValue);
- if (groupedFacetHit.groupValue != null && groupOrd < 0) {
- continue;
- }
-
- int segmentGroupedFacetsIndex = groupOrd * (facetFieldTermsIndex.getValueCount()+1) + facetOrd;
- segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
- }
-
- if (facetPrefix != null) {
- startFacetOrd = facetFieldTermsIndex.lookupTerm(facetPrefix);
- if (startFacetOrd < 0) {
- // Points to the ord one higher than facetPrefix
- startFacetOrd = -startFacetOrd - 1;
- }
- BytesRefBuilder facetEndPrefix = new BytesRefBuilder();
- facetEndPrefix.append(facetPrefix);
- facetEndPrefix.append(UnicodeUtil.BIG_TERM);
- endFacetOrd = facetFieldTermsIndex.lookupTerm(facetEndPrefix.get());
- assert endFacetOrd < 0;
- endFacetOrd = -endFacetOrd - 1; // Points to the ord one higher than facetEndPrefix
- } else {
- startFacetOrd = -1;
- endFacetOrd = facetFieldTermsIndex.getValueCount();
- }
- }
-
- @Override
- protected SegmentResult createSegmentResult() throws IOException {
- return new SegmentResult(segmentFacetCounts, segmentTotalCount, facetFieldTermsIndex.termsEnum(), startFacetOrd, endFacetOrd);
- }
-
- private static class SegmentResult extends GroupFacetCollector.SegmentResult {
-
- final TermsEnum tenum;
-
- SegmentResult(int[] counts, int total, TermsEnum tenum, int startFacetOrd, int endFacetOrd) throws IOException {
- super(counts, total - counts[0], counts[0], endFacetOrd+1);
- this.tenum = tenum;
- this.mergePos = startFacetOrd == -1 ? 1 : startFacetOrd+1;
- if (mergePos < maxTermPos) {
- assert tenum != null;
- tenum.seekExact(startFacetOrd == -1 ? 0 : startFacetOrd);
- mergeTerm = tenum.term();
- }
- }
-
- @Override
- protected void nextTerm() throws IOException {
- mergeTerm = tenum.next();
- }
- }
- }
-
- // Implementation for multi valued facet fields.
- static class MV extends TermGroupFacetCollector {
-
- private SortedSetDocValues facetFieldDocTermOrds;
- private TermsEnum facetOrdTermsEnum;
- private int facetFieldNumTerms;
-
- MV(String groupField, String facetField, BytesRef facetPrefix, int initialSize) {
- super(groupField, facetField, facetPrefix, initialSize);
- }
-
- @Override
- public void collect(int doc) throws IOException {
- if (doc > groupFieldTermsIndex.docID()) {
- groupFieldTermsIndex.advance(doc);
- }
-
- int groupOrd;
- if (doc == groupFieldTermsIndex.docID()) {
- groupOrd = groupFieldTermsIndex.ordValue();
- } else {
- groupOrd = -1;
- }
-
- if (facetFieldNumTerms == 0) {
- int segmentGroupedFacetsIndex = groupOrd * (facetFieldNumTerms + 1);
- if (facetPrefix != null || segmentGroupedFacetHits.exists(segmentGroupedFacetsIndex)) {
- return;
- }
-
- segmentTotalCount++;
- segmentFacetCounts[facetFieldNumTerms]++;
-
- segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
- BytesRef groupKey;
- if (groupOrd == -1) {
- groupKey = null;
- } else {
- groupKey = BytesRef.deepCopyOf(groupFieldTermsIndex.lookupOrd(groupOrd));
- }
- groupedFacetHits.add(new GroupedFacetHit(groupKey, null));
- return;
- }
-
- if (doc > facetFieldDocTermOrds.docID()) {
- facetFieldDocTermOrds.advance(doc);
- }
- boolean empty = true;
- if (doc == facetFieldDocTermOrds.docID()) {
- long ord;
- while ((ord = facetFieldDocTermOrds.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
- process(groupOrd, (int) ord);
- empty = false;
- }
- }
-
- if (empty) {
- process(groupOrd, facetFieldNumTerms); // this facet ord is reserved for docs not containing facet field.
- }
- }
-
- private void process(int groupOrd, int facetOrd) throws IOException {
- if (facetOrd < startFacetOrd || facetOrd >= endFacetOrd) {
- return;
- }
-
- int segmentGroupedFacetsIndex = groupOrd * (facetFieldNumTerms + 1) + facetOrd;
- if (segmentGroupedFacetHits.exists(segmentGroupedFacetsIndex)) {
- return;
- }
-
- segmentTotalCount++;
- segmentFacetCounts[facetOrd]++;
-
- segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
-
- BytesRef groupKey;
- if (groupOrd == -1) {
- groupKey = null;
- } else {
- groupKey = BytesRef.deepCopyOf(groupFieldTermsIndex.lookupOrd(groupOrd));
- }
-
- final BytesRef facetValue;
- if (facetOrd == facetFieldNumTerms) {
- facetValue = null;
- } else {
- facetValue = BytesRef.deepCopyOf(facetFieldDocTermOrds.lookupOrd(facetOrd));
- }
- groupedFacetHits.add(new GroupedFacetHit(groupKey, facetValue));
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- if (segmentFacetCounts != null) {
- segmentResults.add(createSegmentResult());
- }
-
- groupFieldTermsIndex = DocValues.getSorted(context.reader(), groupField);
- facetFieldDocTermOrds = DocValues.getSortedSet(context.reader(), facetField);
- facetFieldNumTerms = (int) facetFieldDocTermOrds.getValueCount();
- if (facetFieldNumTerms == 0) {
- facetOrdTermsEnum = null;
- } else {
- facetOrdTermsEnum = facetFieldDocTermOrds.termsEnum();
- }
- // [facetFieldNumTerms() + 1] for all possible facet values and docs not containing facet field
- segmentFacetCounts = new int[facetFieldNumTerms + 1];
- segmentTotalCount = 0;
-
- segmentGroupedFacetHits.clear();
- for (GroupedFacetHit groupedFacetHit : groupedFacetHits) {
- int groupOrd = groupedFacetHit.groupValue == null ? -1 : groupFieldTermsIndex.lookupTerm(groupedFacetHit.groupValue);
- if (groupedFacetHit.groupValue != null && groupOrd < 0) {
- continue;
- }
-
- int facetOrd;
- if (groupedFacetHit.facetValue != null) {
- if (facetOrdTermsEnum == null || !facetOrdTermsEnum.seekExact(groupedFacetHit.facetValue)) {
- continue;
- }
- facetOrd = (int) facetOrdTermsEnum.ord();
- } else {
- facetOrd = facetFieldNumTerms;
- }
-
- // (facetFieldDocTermOrds.numTerms() + 1) for all possible facet values and docs not containing facet field
- int segmentGroupedFacetsIndex = groupOrd * (facetFieldNumTerms + 1) + facetOrd;
- segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
- }
-
- if (facetPrefix != null) {
- TermsEnum.SeekStatus seekStatus;
- if (facetOrdTermsEnum != null) {
- seekStatus = facetOrdTermsEnum.seekCeil(facetPrefix);
- } else {
- seekStatus = TermsEnum.SeekStatus.END;
- }
-
- if (seekStatus != TermsEnum.SeekStatus.END) {
- startFacetOrd = (int) facetOrdTermsEnum.ord();
- } else {
- startFacetOrd = 0;
- endFacetOrd = 0;
- return;
- }
-
- BytesRefBuilder facetEndPrefix = new BytesRefBuilder();
- facetEndPrefix.append(facetPrefix);
- facetEndPrefix.append(UnicodeUtil.BIG_TERM);
- seekStatus = facetOrdTermsEnum.seekCeil(facetEndPrefix.get());
- if (seekStatus != TermsEnum.SeekStatus.END) {
- endFacetOrd = (int) facetOrdTermsEnum.ord();
- } else {
- endFacetOrd = facetFieldNumTerms; // Don't include null...
- }
- } else {
- startFacetOrd = 0;
- endFacetOrd = facetFieldNumTerms + 1;
- }
- }
-
- @Override
- protected SegmentResult createSegmentResult() throws IOException {
- return new SegmentResult(segmentFacetCounts, segmentTotalCount, facetFieldNumTerms, facetOrdTermsEnum, startFacetOrd, endFacetOrd);
- }
-
- private static class SegmentResult extends GroupFacetCollector.SegmentResult {
-
- final TermsEnum tenum;
-
- SegmentResult(int[] counts, int total, int missingCountIndex, TermsEnum tenum, int startFacetOrd, int endFacetOrd) throws IOException {
- super(counts, total - counts[missingCountIndex], counts[missingCountIndex],
- endFacetOrd == missingCountIndex + 1 ? missingCountIndex : endFacetOrd);
- this.tenum = tenum;
- this.mergePos = startFacetOrd;
- if (tenum != null) {
- tenum.seekExact(mergePos);
- mergeTerm = tenum.term();
- }
- }
-
- @Override
- protected void nextTerm() throws IOException {
- mergeTerm = tenum.next();
- }
- }
- }
-}
-
-class GroupedFacetHit {
-
- final BytesRef groupValue;
- final BytesRef facetValue;
-
- GroupedFacetHit(BytesRef groupValue, BytesRef facetValue) {
- this.groupValue = groupValue;
- this.facetValue = facetValue;
- }
-}