From e94b8c6382af64bcdd084eb9e92094afda297f09 Mon Sep 17 00:00:00 2001 From: Judith Silverman Date: Tue, 28 Mar 2017 19:56:20 -0700 Subject: [PATCH 001/113] DistributedQueryComponentCustomSortTest.java: replacing duplicated tests --- ...stributedQueryComponentCustomSortTest.java | 45 ++++++++++--------- 1 file changed, 23 insertions(+), 22 deletions(-) diff --git a/solr/core/src/test/org/apache/solr/handler/component/DistributedQueryComponentCustomSortTest.java b/solr/core/src/test/org/apache/solr/handler/component/DistributedQueryComponentCustomSortTest.java index 7ce171c63a6c..e77d4488392e 100644 --- a/solr/core/src/test/org/apache/solr/handler/component/DistributedQueryComponentCustomSortTest.java +++ b/solr/core/src/test/org/apache/solr/handler/component/DistributedQueryComponentCustomSortTest.java @@ -124,7 +124,7 @@ public static void setUpBeforeClass() throws Exception { assertFieldValues(rsp.getResults(), id, 11, 13, 8, 9, 5, 3, 12, 10,16,17,18, 2, 4,14,15, 6, 1, 7); -// function sorting +// non-function sorting { rsp = query("q", "id:[1 TO 10]", "fl", "id", "rows", "20", "sort", "payload desc, id asc", "group", "true", "group.field", "val", "group.sort", "id asc"); @@ -190,12 +190,11 @@ public static void setUpBeforeClass() throws Exception { assertFieldValues((SolrDocumentList)groupList2.get("doclist"), id, 2); } -// Dup - // function sorting + + // function sorting, limit 1 { rsp = query("q", "id:[1 TO 10]", "fl", "id", "rows", "20", - "sort", "abs(sub(5,id)) asc, id desc", "group", "true", "group.field", "val", "group.sort", "id asc"); - + "sort", "sub(5,id) asc", "group", "true", "group.field", "val", "group.sort", "id asc", "group.limit", 1); NamedList grouped = (NamedList)rsp.getResponse().get("grouped"); assertNotNull(grouped); @@ -209,18 +208,18 @@ public static void setUpBeforeClass() throws Exception { NamedList groupList0 = (NamedList)groupLists.get(0); assertNotNull(groupList0); - assertFieldValues((SolrDocumentList)groupList0.get("doclist"), id, 1); + assertFieldValues((SolrDocumentList)groupList0.get("doclist"), id, 2); NamedList groupList1 = (NamedList)groupLists.get(1); assertNotNull(groupList1); - assertFieldValues((SolrDocumentList)groupList1.get("doclist"), id, 6); + assertFieldValues((SolrDocumentList)groupList1.get("doclist"), id, 1); NamedList groupList2 = (NamedList)groupLists.get(2); assertNotNull(groupList2); - assertFieldValues((SolrDocumentList)groupList2.get("doclist"), id, 2); + assertFieldValues((SolrDocumentList)groupList2.get("doclist"), id, 6); } @@ -256,10 +255,12 @@ public static void setUpBeforeClass() throws Exception { assertFieldValues((SolrDocumentList)groupList2.get("doclist"), id, 10); } - // function sorting (dup) + + + // function sorting (group.sort desc) { rsp = query("q", "id:[1 TO 10]", "fl", "id", "rows", "20", - "sort", "abs(sub(5,id)) asc, id desc", "group", "true", "group.field", "val", "group.sort", "id desc"); + "sort", "abs(sub(5,id)) asc, val desc", "group", "true", "group.field", "val", "group.sort", "id desc"); NamedList grouped = (NamedList)rsp.getResponse().get("grouped"); @@ -288,7 +289,8 @@ public static void setUpBeforeClass() throws Exception { assertFieldValues((SolrDocumentList)groupList2.get("doclist"), id, 10); } - // function sorting + + // function sorting (group.limit=2) { rsp = query("q", "id:[1 TO 10]", "fl", "id", "rows", "20", "sort", "abs(sub(5,id)) desc, id desc", "group", "true", "group.field", "val", "group.sort", "sum(3,id) asc", "group.limit", "2"); @@ -322,7 +324,7 @@ public static void setUpBeforeClass() throws Exception { } - // function sorting + // function sorting (group.limit=3) { rsp = query("q", "id:[1 TO 10]", "fl", "id", "rows", "20", "sort", "abs(sub(5,id)) asc, id desc", "group", "true", "group.field", "val", "group.sort", "sum(3,id) asc", "group.limit", "3"); @@ -356,7 +358,7 @@ public static void setUpBeforeClass() throws Exception { } - // function sorting + // function sorting (start=2) // Pagination is on level of groups, not individual offers. { rsp = query("q", "id:[1 TO 10]", "fl", "id", "rows", "20", "start", "2", @@ -379,7 +381,7 @@ public static void setUpBeforeClass() throws Exception { } - // function sorting + // function sorting (group.offset=1) { rsp = query("q", "id:[1 TO 10]", "fl", "id", "rows", "20", "sort", "abs(sub(5,id)) asc, id desc", "group", "true", "group.field", "val", "group.sort", "sum(3,id) asc", "group.limit", "3", "group.offset", "1"); @@ -411,7 +413,7 @@ public static void setUpBeforeClass() throws Exception { assertFieldValues((SolrDocumentList)groupList2.get("doclist"), id, 4, 10); } - // function sorting + // function sorting (group.offset=2) { rsp = query("q", "id:[1 TO 10]", "fl", "id", "rows", "20", "sort", "abs(sub(5,id)) asc, id desc", "group", "true", "group.field", "val", "group.sort", "sum(3,id) asc", "group.limit", "3", "group.offset", "2"); @@ -443,10 +445,10 @@ public static void setUpBeforeClass() throws Exception { assertFieldValues((SolrDocumentList)groupList2.get("doclist"), id, 10); } - // function sorting + // function sorting (different sort and group.sort functions) { rsp = query("q", "id:[1 TO 10]", "fl", "id", "rows", "20", - "sort", "abs(sub(5,id)) asc, id desc", "group", "true", "group.field", "val", "group.sort", "sum(3,id) desc", "group.limit", "3"); + "sort", "sum(val,id)) asc, id desc", "group", "true", "group.field", "val", "group.sort", "id asc", "group.limit", "3"); NamedList grouped = (NamedList)rsp.getResponse().get("grouped"); @@ -460,17 +462,18 @@ public static void setUpBeforeClass() throws Exception { NamedList groupList0 = (NamedList)groupLists.get(0); assertNotNull(groupList0); - assertFieldValues((SolrDocumentList)groupList0.get("doclist"), id, 9, 5, 3); + assertFieldValues((SolrDocumentList)groupList0.get("doclist"), id, 2, 4, 10); NamedList groupList1 = (NamedList)groupLists.get(1); assertNotNull(groupList1); - assertFieldValues((SolrDocumentList)groupList1.get("doclist"), id, 8, 7, 6); + assertFieldValues((SolrDocumentList)groupList1.get("doclist"), id, 1, 3, 5); + NamedList groupList2 = (NamedList)groupLists.get(2); assertNotNull(groupList2); + assertFieldValues((SolrDocumentList)groupList2.get("doclist"), id, 6, 7, 8); - assertFieldValues((SolrDocumentList)groupList2.get("doclist"), id, 10, 4, 2); } // function sorting (id:[1 TO 2]) @@ -498,7 +501,5 @@ public static void setUpBeforeClass() throws Exception { assertFieldValues((SolrDocumentList)groupList1.get("doclist"), id, 1); } - - } } From b2a631079ad2722f28cd6ee77779903e354d7a56 Mon Sep 17 00:00:00 2001 From: Joel Bernstein Date: Fri, 7 Apr 2017 12:36:19 -0400 Subject: [PATCH 002/113] Add version 6.5.1 --- lucene/core/src/java/org/apache/lucene/util/Version.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/lucene/core/src/java/org/apache/lucene/util/Version.java b/lucene/core/src/java/org/apache/lucene/util/Version.java index da6d653bce71..f6e6adcc5ed1 100644 --- a/lucene/core/src/java/org/apache/lucene/util/Version.java +++ b/lucene/core/src/java/org/apache/lucene/util/Version.java @@ -101,6 +101,13 @@ public final class Version { @Deprecated public static final Version LUCENE_6_5_0 = new Version(6, 5, 0); + /** + * Match settings and bugs in Lucene's 6.5.1 release. + * @deprecated Use latest + */ + @Deprecated + public static final Version LUCENE_6_5_1 = new Version(6, 5, 1); + /** * Match settings and bugs in Lucene's 6.6.0 release. * @deprecated Use latest From ce75eb62e9fd3256bcab23e7f1d901fec7c65115 Mon Sep 17 00:00:00 2001 From: Tomas Fernandez Lobbe Date: Fri, 7 Apr 2017 11:36:22 -0700 Subject: [PATCH 003/113] SOLR-10437: Improve test coverage of useDocValuesAsStored=false --- .../solr/collection1/conf/schema-point.xml | 5 ++ .../apache/solr/schema/TestPointFields.java | 71 ++++++++++--------- .../solr/schema/TestUseDocValuesAsStored.java | 28 ++++++++ 3 files changed, 71 insertions(+), 33 deletions(-) diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-point.xml b/solr/core/src/test-files/solr/collection1/conf/schema-point.xml index 1a936a04c643..ae6a11ecfef4 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-point.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-point.xml @@ -54,6 +54,7 @@ + @@ -66,6 +67,7 @@ + @@ -78,6 +80,7 @@ + @@ -90,6 +93,7 @@ + @@ -102,6 +106,7 @@ + diff --git a/solr/core/src/test/org/apache/solr/schema/TestPointFields.java b/solr/core/src/test/org/apache/solr/schema/TestPointFields.java index de458445fe4d..18ac3b257f5b 100644 --- a/solr/core/src/test/org/apache/solr/schema/TestPointFields.java +++ b/solr/core/src/test/org/apache/solr/schema/TestPointFields.java @@ -17,6 +17,7 @@ package org.apache.solr.schema; import java.io.IOException; +import java.text.SimpleDateFormat; import java.util.Arrays; import java.util.Collections; import java.util.Date; @@ -57,7 +58,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.ibm.icu.text.SimpleDateFormat; /** * Tests for PointField functionality @@ -885,7 +885,7 @@ public void testIndexOrDocValuesQuery() throws Exception { public void testInternals() throws IOException { String[] types = new String[]{"i", "l", "f", "d"}; - String[] suffixes = new String[]{"", "_dv", "_mv", "_mv_dv", "_ni", "_ni_dv", "_ni_dv_ns", "_ni_mv", "_ni_mv_dv", "_ni_ns", "_ni_ns_mv", "_dv_ns", "_ni_ns_dv", "_dv_ns_mv"}; + String[] suffixes = new String[]{"", "_dv", "_mv", "_mv_dv", "_ni", "_ni_dv", "_ni_dv_ns", "_ni_dv_ns_mv", "_ni_mv", "_ni_mv_dv", "_ni_ns", "_ni_ns_mv", "_dv_ns", "_ni_ns_dv", "_dv_ns_mv"}; Set typesTested = new HashSet<>(); for (String type:types) { for (String suffix:suffixes) { @@ -2696,38 +2696,36 @@ private void doTestInternals(String field, String[] values) throws IOException { public void testNonReturnable() throws Exception { - doTestNonReturnable("foo_p_i_ni_ns", "42"); - doTestNonReturnable("foo_p_i_ni_ns_mv", "42", "666"); + doTestReturnNonStored("foo_p_i_ni_ns", false, "42"); + doTestReturnNonStored("foo_p_i_ni_dv_ns", true, "42"); + doTestReturnNonStored("foo_p_i_ni_ns_mv", false, "42", "666"); + doTestReturnNonStored("foo_p_i_ni_dv_ns_mv", true, "42", "666"); - doTestNonReturnable("foo_p_l_ni_ns", "3333333333"); - doTestNonReturnable("foo_p_l_ni_ns_mv", "3333333333", "-4444444444"); + doTestReturnNonStored("foo_p_l_ni_ns", false, "3333333333"); + doTestReturnNonStored("foo_p_l_ni_dv_ns", true, "3333333333"); + doTestReturnNonStored("foo_p_l_ni_ns_mv", false, "3333333333", "-4444444444"); + doTestReturnNonStored("foo_p_l_ni_dv_ns_mv", true, "3333333333", "-4444444444"); - doTestNonReturnable("foo_p_f_ni_ns", "42.3"); - doTestNonReturnable("foo_p_f_ni_ns_mv", "42.3", "-66.6"); + doTestReturnNonStored("foo_p_f_ni_ns", false, "42.3"); + doTestReturnNonStored("foo_p_f_ni_dv_ns", true, "42.3"); + doTestReturnNonStored("foo_p_f_ni_ns_mv", false, "42.3", "-66.6"); + doTestReturnNonStored("foo_p_f_ni_dv_ns_mv", true, "42.3", "-66.6"); - doTestNonReturnable("foo_p_d_ni_ns", "42.3"); - doTestNonReturnable("foo_p_d_ni_ns_mv", "42.3", "-66.6"); + doTestReturnNonStored("foo_p_d_ni_ns", false, "42.3"); + doTestReturnNonStored("foo_p_d_ni_dv_ns", true, "42.3"); + doTestReturnNonStored("foo_p_d_ni_ns_mv", false, "42.3", "-66.6"); + doTestReturnNonStored("foo_p_d_ni_dv_ns_mv", true, "42.3", "-66.6"); - doTestNonReturnable("foo_p_dt_ni_ns", "1995-12-31T23:59:59Z"); - doTestNonReturnable("foo_p_dt_ni_ns_mv", "1995-12-31T23:59:59Z", "2000-12-31T23:59:59Z+3DAYS"); - - } - - @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-10437") - public void testNonReturnableDocValues() throws Exception { - // TODO: once SOLR-10437 is resolved, this test method can be folded into testNonReturnable() - - // these fields are stored=false, docValues=true, useDocValuesAsStored=false and yet they are - // still returned and failing this test. - - doTestNonReturnable("foo_p_i_ni_dv_ns", "42"); - doTestNonReturnable("foo_p_l_ni_dv_ns", "3333333333"); - doTestNonReturnable("foo_p_f_ni_dv_ns", "42.3"); - doTestNonReturnable("foo_p_d_ni_dv_ns", "42.3"); - doTestNonReturnable("foo_p_dt_ni_dv_ns", "1995-12-31T23:59:59Z"); + doTestReturnNonStored("foo_p_dt_ni_ns", false, "1995-12-31T23:59:59Z"); + doTestReturnNonStored("foo_p_dt_ni_dv_ns", true, "1995-12-31T23:59:59Z"); + doTestReturnNonStored("foo_p_dt_ni_ns_mv", false, "1995-12-31T23:59:59Z", "2000-12-31T23:59:59Z+3DAYS"); + doTestReturnNonStored("foo_p_dt_ni_dv_ns_mv", true, "1995-12-31T23:59:59Z", "2000-12-31T23:59:59Z+3DAYS"); } - public void doTestNonReturnable(final String fieldName, final String... values) throws Exception { + public void doTestReturnNonStored(final String fieldName, boolean shouldReturnFieldIfRequested, final String... values) throws Exception { + final String RETURN_FIELD = "count(//doc/*[@name='" + fieldName + "'])=10"; + final String DONT_RETURN_FIELD = "count(//doc/*[@name='" + fieldName + "'])=0"; + assertFalse(h.getCore().getLatestSchema().getField(fieldName).stored()); for (int i=0; i < 10; i++) { SolrInputDocument doc = sdoc("id", String.valueOf(i)); for (String value : values) { @@ -2739,17 +2737,24 @@ public void doTestNonReturnable(final String fieldName, final String... values) assertQ(req("q", "*:*", "rows", "100", "fl", "id," + fieldName), "//*[@numFound='10']", "count(//doc)=10", // exactly 10 docs in response - "count(//doc/*)=10", // exactly 10 fields across all docs - "count(//doc/*[@name!='id'])=0"); // no field in any doc other then 'id' + (shouldReturnFieldIfRequested?RETURN_FIELD:DONT_RETURN_FIELD)); // no field in any doc other then 'id' + + assertQ(req("q", "*:*", "rows", "100", "fl", "*"), + "//*[@numFound='10']", + "count(//doc)=10", // exactly 10 docs in response + DONT_RETURN_FIELD); // no field in any doc other then 'id' + + assertQ(req("q", "*:*", "rows", "100"), + "//*[@numFound='10']", + "count(//doc)=10", // exactly 10 docs in response + DONT_RETURN_FIELD); // no field in any doc other then 'id' clearIndex(); assertU(commit()); } public void testWhiteboxCreateFields() throws Exception { - // TODO: we should have a "coverage" assert that we're looping over all the dynamic (point) fields in the schema - String[] typeNames = new String[]{"i", "l", "f", "d", "dt"}; - String[] suffixes = new String[]{"", "_dv", "_mv", "_mv_dv", "_ni", "_ni_dv", "_ni_dv_ns", "_ni_mv", "_ni_mv_dv", "_ni_ns", "_ni_ns_mv", "_dv_ns", "_ni_ns_dv", "_dv_ns_mv"}; + String[] suffixes = new String[]{"", "_dv", "_mv", "_mv_dv", "_ni", "_ni_dv", "_ni_dv_ns", "_ni_dv_ns_mv", "_ni_mv", "_ni_mv_dv", "_ni_ns", "_ni_ns_mv", "_dv_ns", "_ni_ns_dv", "_dv_ns_mv"}; Class[] expectedClasses = new Class[]{IntPoint.class, LongPoint.class, FloatPoint.class, DoublePoint.class, LongPoint.class}; Date dateToTest = new Date(); diff --git a/solr/core/src/test/org/apache/solr/schema/TestUseDocValuesAsStored.java b/solr/core/src/test/org/apache/solr/schema/TestUseDocValuesAsStored.java index 48a3f2266bc7..9cababc10447 100644 --- a/solr/core/src/test/org/apache/solr/schema/TestUseDocValuesAsStored.java +++ b/solr/core/src/test/org/apache/solr/schema/TestUseDocValuesAsStored.java @@ -250,6 +250,34 @@ public void testMultipleSearchResults() throws Exception { + "{'id':'myid6','test_s_dvo':'hello'}" + "]"); } + + @Test + public void testUseDocValuesAsStoredFalse() throws Exception { + SchemaField sf = h.getCore().getLatestSchema().getField("nonstored_dv_str"); + assertNotNull(sf); + assertTrue(sf.hasDocValues()); + assertFalse(sf.useDocValuesAsStored()); + assertFalse(sf.stored()); + assertU(adoc("id", "myid", "nonstored_dv_str", "dont see me")); + assertU(commit()); + + assertJQ(req("q", "id:myid"), + "/response/docs==[" + + "{'id':'myid'}" + + "]"); + assertJQ(req("q", "id:myid", "fl", "*"), + "/response/docs==[" + + "{'id':'myid'}" + + "]"); + assertJQ(req("q", "id:myid", "fl", "id,nonstored_dv_*"), + "/response/docs==[" + + "{'id':'myid'}" + + "]"); + assertJQ(req("q", "id:myid", "fl", "id,nonstored_dv_str"), + "/response/docs==[" + + "{'id':'myid','nonstored_dv_str':'dont see me'}" + + "]"); + } public void testManagedSchema() throws Exception { IndexSchema oldSchema = h.getCore().getLatestSchema(); From d671273bc396faa4c3815e0a4614e88d3e0373e1 Mon Sep 17 00:00:00 2001 From: Tomas Fernandez Lobbe Date: Fri, 7 Apr 2017 14:11:25 -0700 Subject: [PATCH 004/113] SOLR-10443: Improvements to TestPointFields * Fixes testInternals, index needs to be cleaned after each field * Validate that SolrQueryParser generates a PointInSetQuery when possible --- .../apache/solr/schema/TestPointFields.java | 68 +++++++++++++------ 1 file changed, 49 insertions(+), 19 deletions(-) diff --git a/solr/core/src/test/org/apache/solr/schema/TestPointFields.java b/solr/core/src/test/org/apache/solr/schema/TestPointFields.java index 18ac3b257f5b..e5dc99091c89 100644 --- a/solr/core/src/test/org/apache/solr/schema/TestPointFields.java +++ b/solr/core/src/test/org/apache/solr/schema/TestPointFields.java @@ -19,7 +19,6 @@ import java.io.IOException; import java.text.SimpleDateFormat; import java.util.Arrays; -import java.util.Collections; import java.util.Date; import java.util.HashSet; import java.util.LinkedHashSet; @@ -48,8 +47,10 @@ import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrInputDocument; +import org.apache.solr.common.params.CommonParams; import org.apache.solr.schema.IndexSchema.DynamicField; import org.apache.solr.search.SolrIndexSearcher; +import org.apache.solr.search.SolrQueryParser; import org.apache.solr.util.DateMathParser; import org.apache.solr.util.RefCounted; import org.junit.After; @@ -218,9 +219,9 @@ public void testMultiValuedIntPointFieldsAtomicUpdates() throws Exception { @Test public void testIntPointSetQuery() throws Exception { - doTestSetQueries("number_p_i", getRandomStringArrayWithInts(10, false), false); - doTestSetQueries("number_p_i_mv", getRandomStringArrayWithInts(10, false), true); - doTestSetQueries("number_p_i_ni_dv", getRandomStringArrayWithInts(10, false), false); + doTestSetQueries("number_p_i", getRandomStringArrayWithInts(20, false), false); + doTestSetQueries("number_p_i_mv", getRandomStringArrayWithInts(20, false), true); + doTestSetQueries("number_p_i_ni_dv", getRandomStringArrayWithInts(20, false), false); } // DoublePointField @@ -408,9 +409,9 @@ private void doTestFloatPointFieldsAtomicUpdates(String field, String type) thro @Test public void testDoublePointSetQuery() throws Exception { - doTestSetQueries("number_p_d", getRandomStringArrayWithDoubles(10, false), false); - doTestSetQueries("number_p_d_mv", getRandomStringArrayWithDoubles(10, false), true); - doTestSetQueries("number_p_d_ni_dv", getRandomStringArrayWithDoubles(10, false), false); + doTestSetQueries("number_p_d", getRandomStringArrayWithDoubles(20, false), false); + doTestSetQueries("number_p_d_mv", getRandomStringArrayWithDoubles(20, false), true); + doTestSetQueries("number_p_d_ni_dv", getRandomStringArrayWithDoubles(20, false), false); } // Float @@ -557,9 +558,9 @@ public void testMultiValuedFloatePointFieldsAtomicUpdates() throws Exception { @Test public void testFloatPointSetQuery() throws Exception { - doTestSetQueries("number_p_f", getRandomStringArrayWithFloats(10, false), false); - doTestSetQueries("number_p_f_mv", getRandomStringArrayWithFloats(10, false), true); - doTestSetQueries("number_p_f_ni_dv", getRandomStringArrayWithFloats(10, false), false); + doTestSetQueries("number_p_f", getRandomStringArrayWithFloats(20, false), false); + doTestSetQueries("number_p_f_mv", getRandomStringArrayWithFloats(20, false), true); + doTestSetQueries("number_p_f_ni_dv", getRandomStringArrayWithFloats(20, false), false); } @Test @@ -705,9 +706,9 @@ public void testMultiValuedLongPointFieldsAtomicUpdates() throws Exception { @Test public void testLongPointSetQuery() throws Exception { - doTestSetQueries("number_p_l", getRandomStringArrayWithLongs(10, false), false); - doTestSetQueries("number_p_l_mv", getRandomStringArrayWithLongs(10, false), true); - doTestSetQueries("number_p_l_ni_dv", getRandomStringArrayWithLongs(10, false), false); + doTestSetQueries("number_p_l", getRandomStringArrayWithLongs(20, false), false); + doTestSetQueries("number_p_l_mv", getRandomStringArrayWithLongs(20, false), true); + doTestSetQueries("number_p_l_ni_dv", getRandomStringArrayWithLongs(20, false), false); } @Test @@ -850,9 +851,9 @@ public void testMultiValuedDatePointFieldsAtomicUpdates() throws Exception { @Test public void testDatePointSetQuery() throws Exception { - doTestSetQueries("number_p_dt", getRandomStringArrayWithDates(10, false), false); - doTestSetQueries("number_p_dt_mv", getRandomStringArrayWithDates(10, false), true); - doTestSetQueries("number_p_dt_ni_dv", getRandomStringArrayWithDates(10, false), false); + doTestSetQueries("number_p_dt", getRandomStringArrayWithDates(20, false), false); + doTestSetQueries("number_p_dt_mv", getRandomStringArrayWithDates(20, false), true); + doTestSetQueries("number_p_dt_ni_dv", getRandomStringArrayWithDates(20, false), false); } @@ -2087,7 +2088,8 @@ private void doTestSetQueries(String fieldName, String[] values, boolean multiVa assertU(adoc("id", String.valueOf(i), fieldName, values[i])); } assertU(commit()); - assertTrue(h.getCore().getLatestSchema().getField(fieldName).getType() instanceof PointField); + SchemaField sf = h.getCore().getLatestSchema().getField(fieldName); + assertTrue(sf.getType() instanceof PointField); for (int i = 0; i < values.length; i++) { assertQ(req("q", "{!term f='" + fieldName + "'}" + values[i], "fl", "id," + fieldName), @@ -2099,6 +2101,27 @@ private void doTestSetQueries(String fieldName, String[] values, boolean multiVa "//*[@numFound='2']"); } + assertTrue(values.length > SolrQueryParser.TERMS_QUERY_THRESHOLD); + int numTerms = SolrQueryParser.TERMS_QUERY_THRESHOLD + 1; + StringBuilder builder = new StringBuilder(fieldName + ":("); + for (int i = 0; i < numTerms; i++) { + if (sf.getType().getNumberType() == NumberType.DATE) { + builder.append(String.valueOf(values[i]).replace(":", "\\:") + ' '); + } else { + builder.append(String.valueOf(values[i]).replace("-", "\\-") + ' '); + } + } + builder.append(')'); + if (sf.indexed()) { // SolrQueryParser should also be generating a PointInSetQuery if indexed + assertQ(req(CommonParams.DEBUG, CommonParams.QUERY, "q", "*:*", "fq", builder.toString(), "fl", "id," + fieldName), + "//*[@numFound='" + numTerms + "']", + "//*[@name='parsed_filter_queries']/str[.='(" + getSetQueryToString(fieldName, values, numTerms) + ")']"); + } else { + // Won't use PointInSetQuery if the fiels is not indexed, but should match the same docs + assertQ(req(CommonParams.DEBUG, CommonParams.QUERY, "q", "*:*", "fq", builder.toString(), "fl", "id," + fieldName), + "//*[@numFound='" + numTerms + "']"); + } + if (multiValued) { clearIndex(); assertU(commit()); @@ -2118,6 +2141,11 @@ private void doTestSetQueries(String fieldName, String[] values, boolean multiVa } } + private String getSetQueryToString(String fieldName, String[] values, int numTerms) { + SchemaField sf = h.getCore().getLatestSchema().getField(fieldName); + return sf.getType().getSetQuery(null, sf, Arrays.asList(Arrays.copyOf(values, numTerms))).toString(); + } + private void doTestDoublePointFieldMultiValuedRangeFacet(String docValuesField, String nonDocValuesField) throws Exception { for (int i = 0; i < 10; i++) { assertU(adoc("id", String.valueOf(i), docValuesField, String.valueOf(i), docValuesField, String.valueOf(i + 10), @@ -2681,9 +2709,9 @@ private void doTestInternals(String field, String[] values) throws IOException { for (LeafReaderContext leave:ir.leaves()) { LeafReader reader = leave.reader(); for (int i = 0; i < reader.numDocs(); i++) { - Document doc = reader.document(i, Collections.singleton(field)); + Document doc = reader.document(i); if (sf.stored()) { - assertNotNull(doc.get(field)); + assertNotNull("Field " + field + " not found. Doc: " + doc, doc.get(field)); } else { assertNull(doc.get(field)); } @@ -2692,6 +2720,8 @@ private void doTestInternals(String field, String[] values) throws IOException { } finally { ref.decref(); } + clearIndex(); + assertU(commit()); } public void testNonReturnable() throws Exception { From 550a32495257634859ac6eae3858cbce71be06f2 Mon Sep 17 00:00:00 2001 From: Noble Paul Date: Mon, 10 Apr 2017 06:43:27 +0930 Subject: [PATCH 005/113] SOLR-10429: UpdateRequest#getRoutes()should copy the response parser --- solr/CHANGES.txt | 2 ++ .../org/apache/solr/client/solrj/request/UpdateRequest.java | 3 ++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 62e91899b772..ec25b90b4e52 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -207,6 +207,8 @@ Other Changes * SOLR-10440: LBHttpSolrClient.doRequest is now always wrapped in a Mapped Diagnostic Context (MDC). (Christine Poerschke) +* SOLR-10429: UpdateRequest#getRoutes()should copy the response parser (noble) + ================== 6.5.1 ================== Bug Fixes diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java index 142710a04250..0d9867cc86c9 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java @@ -269,7 +269,7 @@ public Map getRoutes(DocRouter router, return null; } String leaderUrl = urls.get(0); - LBHttpSolrClient.Req request = (LBHttpSolrClient.Req) routes + LBHttpSolrClient.Req request = routes .get(leaderUrl); if (request == null) { UpdateRequest updateRequest = new UpdateRequest(); @@ -278,6 +278,7 @@ public Map getRoutes(DocRouter router, updateRequest.setParams(params); updateRequest.setPath(getPath()); updateRequest.setBasicAuthCredentials(getBasicAuthUser(), getBasicAuthPassword()); + updateRequest.setResponseParser(getResponseParser()); request = new LBHttpSolrClient.Req(updateRequest, urls); routes.put(leaderUrl, request); } From caf329717ef643b2a5a8b7ef0bfb7bd5e096c849 Mon Sep 17 00:00:00 2001 From: Alan Woodward Date: Tue, 28 Mar 2017 19:52:53 +0100 Subject: [PATCH 006/113] LUCENE-7701: Refactor grouping collectors --- lucene/CHANGES.txt | 3 + lucene/MIGRATE.txt | 13 + .../lucene/search/TopFieldCollector.java | 4 +- .../grouping/AllGroupHeadsCollector.java | 239 +++++- .../search/grouping/AllGroupsCollector.java | 42 +- .../grouping/BlockGroupingCollector.java | 2 +- .../grouping/DistinctValuesCollector.java | 92 ++- .../grouping/FirstPassGroupingCollector.java | 63 +- .../lucene/search/grouping/GroupReducer.java | 112 +++ .../lucene/search/grouping/GroupSelector.java | 73 ++ .../lucene/search/grouping/Grouper.java | 56 -- .../search/grouping/GroupingSearch.java | 24 +- .../grouping/SecondPassGroupingCollector.java | 144 +--- .../{term => }/TermGroupFacetCollector.java | 17 +- .../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 -- .../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 --- .../search/grouping/term/TermGrouper.java | 81 -- .../term/TermSecondPassGroupingCollector.java | 91 --- .../search/grouping/term/package-info.java | 21 - .../grouping/AllGroupHeadsCollectorTest.java | 13 +- .../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 +- 44 files changed, 1095 insertions(+), 2422 deletions(-) create mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupReducer.java create mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupSelector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java rename lucene/grouping/src/java/org/apache/lucene/search/grouping/{term => }/TermGroupFacetCollector.java (97%) create mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupSelector.java create mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java create mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/ValueSourceGroupSelector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/function/package-info.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGrouper.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/term/package-info.java diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 1f3f30c79e40..cd25deeac6bd 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 diff --git a/lucene/MIGRATE.txt b/lucene/MIGRATE.txt index 1ccf56f62787..c7936a4bd7af 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. 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 1ec322fe5d9d..97589c430dee 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 boolean needsScores() { * trackDocScores 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); } 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 b5fbdc36f1a8..503b9529bd65 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 @@ 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 extends SimpleCollector { + private final GroupSelector 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> 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 the group value type + */ + public static AllGroupHeadsCollector newCollector(GroupSelector selector, Sort sort) { + if (sort.equals(Sort.RELEVANCE)) + return new ScoringGroupHeadsCollector<>(selector, sort); + return new SortingGroupHeadsCollector<>(selector, sort); + } + + private AllGroupHeadsCollector(GroupSelector 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; } /** @@ -78,35 +113,28 @@ public int groupHeadsSize() { return getCollectedGroupHeads().size(); } - /** - * 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. - *

- * The {@link TemporalResult#stop} property will be true if the group head wasn't encountered before - * otherwise false. - * - * @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> getCollectedGroupHeads(); + protected Collection> 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 groupHead = temporalResult.groupHead; - // Ok now we need to check if the current doc is more relevant then current doc for this group + GroupHead 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,17 +153,33 @@ public void collect(int doc) throws IOException { 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 groupHead; - public boolean stop; + @Override + protected void doSetNextReader(LeafReaderContext context) throws IOException { + groupSelector.setNextReader(context); + this.context = context; + for (GroupHead head : heads.values()) { + head.setNextReader(context); + } + } + @Override + public void setScorer(Scorer scorer) throws IOException { + this.scorer = scorer; + for (GroupHead 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 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,11 +191,29 @@ public static abstract class GroupHead { 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. * @@ -173,4 +235,117 @@ protected GroupHead(T groupValue, int doc) { } + /** + * General implementation using a {@link FieldComparator} to select the group head + */ + private static class SortingGroupHeadsCollector extends AllGroupHeadsCollector { + + protected SortingGroupHeadsCollector(GroupSelector selector, Sort sort) { + super(selector, sort); + } + + @Override + protected GroupHead newGroupHead(int doc, T value, LeafReaderContext ctx, Scorer scorer) throws IOException { + return new SortingGroupHead<>(sort, value, doc, ctx, scorer); + } + } + + private static class SortingGroupHead extends GroupHead { + + 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 extends AllGroupHeadsCollector { + + protected ScoringGroupHeadsCollector(GroupSelector selector, Sort sort) { + super(selector, sort); + } + + @Override + protected GroupHead newGroupHead(int doc, T value, LeafReaderContext context, Scorer scorer) throws IOException { + return new ScoringGroupHead<>(scorer, value, doc, context.docBase); + } + } + + private static class ScoringGroupHead extends GroupHead { + + 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; + } + } + } 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 af697af5a843..8434534df4de 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 @@ 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. - *

- * 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 extends SimpleCollector { +public class AllGroupsCollector extends SimpleCollector { + + private final GroupSelector groupSelector; + + private final Set groups = new HashSet(); + + /** + * Create a new AllGroupsCollector + * @param groupSelector the GroupSelector to determine groups + */ + public AllGroupsCollector(GroupSelector groupSelector) { + this.groupSelector = groupSelector; + } /** * Returns the total number of groups for the executed search. @@ -49,17 +60,30 @@ public int getGroupCount() { /** * Returns the group values *

- * 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 getGroups(); + public Collection 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 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 c965042a8553..a50fda103d95 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 @@ * 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. * 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 54d752c58766..103b0d26a707 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 @@ * * @lucene.experimental */ -public abstract class DistinctValuesCollector extends SimpleCollector { +public class DistinctValuesCollector extends SecondPassGroupingCollector { + + /** + * 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 groupSelector, Collection> groups, + GroupSelector valueSelector) { + super(groupSelector, groups, new DistinctValuesReducer<>(valueSelector)); + } + + private static class ValuesCollector extends SimpleCollector { + + final GroupSelector valueSelector; + final Set values = new HashSet<>(); + + private ValuesCollector(GroupSelector 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 extends GroupReducer> { + + final GroupSelector valueSelector; + + private DistinctValuesReducer(GroupSelector valueSelector) { + this.valueSelector = valueSelector; + } + + @Override + public boolean needsScores() { + return false; + } + + @Override + protected ValuesCollector 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> getGroups(); + public List> getGroups() { + List> counts = new ArrayList<>(); + for (SearchGroup group : groups) { + @SuppressWarnings("unchecked") + ValuesCollector vc = (ValuesCollector) 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 { + public static class GroupCount { public final T groupValue; - public final Set uniqueValues; + public final Set uniqueValues; - public GroupCount(T groupValue) { + public GroupCount(T groupValue, Set values) { this.groupValue = groupValue; - this.uniqueValues = new HashSet<>(); + this.uniqueValues = values; } } - @Override - public boolean needsScores() { - return false; // not needed to fetch all values - } - } 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 02bb1a276379..bd47adbb1f7f 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 @@ /** 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} * *

See {@link org.apache.lucene.search.grouping} for more * details including a full code example.

* * @lucene.experimental */ -abstract public class FirstPassGroupingCollector extends SimpleCollector { +public class FirstPassGroupingCollector extends SimpleCollector { + + private final GroupSelector groupSelector; private final FieldComparator[] comparators; private final LeafFieldComparator[] leafComparators; @@ -60,16 +61,18 @@ abstract public class FirstPassGroupingCollector 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 groupSelector, Sort groupSort, int topNGroups) { + this.groupSelector = groupSelector; if (topNGroups < 1) { throw new IllegalArgumentException("topNGroups must be >= 1 (got " + topNGroups + ")"); } @@ -133,7 +136,7 @@ public Collection> getTopGroups(int groupOffset, boolean fillFiel 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 searchGroup = new SearchGroup<>(); searchGroup.groupValue = group.groupValue; if (fillFields) { @@ -155,14 +158,11 @@ public void setScorer(Scorer scorer) throws IOException { } } - @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 @@ public void collect(int doc) throws IOException { 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 @@ public void collect(int doc) throws IOException { // 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 group = groupMap.get(groupValue); @@ -207,7 +216,7 @@ public void collect(int doc) throws IOException { // Add a new CollectedSearchGroup: CollectedSearchGroup 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 @@ public void collect(int doc) throws IOException { 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 @@ protected void doSetNextReader(LeafReaderContext readerContext) throws IOExcepti for (int i=0; i getGroupSelector() { + return groupSelector; + } } 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 000000000000..4366e9123f78 --- /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 the type of the value used for grouping + * @param the type of {@link Collector} used to reduce each group + */ +public abstract class GroupReducer { + + private final Map> groups = new HashMap<>(); + + /** + * Define which groups should be reduced. + * + * Called by {@link SecondPassGroupingCollector} + */ + public void setGroups(Collection> groups) { + for (SearchGroup 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 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 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 collector : groups.values()) { + collector.leafCollector = collector.collector.getLeafCollector(ctx); + } + } + + private static final class GroupCollector { + + final C collector; + LeafCollector leafCollector; + + private GroupCollector(C collector) { + this.collector = collector; + } + } + +} 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 000000000000..dbb09329f8ae --- /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 the type of the group value + */ +public abstract class GroupSelector { + + /** + * 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> groups); + +} 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 2ff79a1331b2..000000000000 --- 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 the type the group value - */ -public abstract class Grouper { - - /** - * 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 getFirstPassCollector(Sort sort, int count) throws IOException; - - /** - * Create an {@link AllGroupsCollector} - */ - public abstract AllGroupsCollector getAllGroupsCollector(); - - /** - * Create an {@link AllGroupHeadsCollector} - * @param sort a within-group sort order to determine which doc is the group head - */ - public abstract AllGroupHeadsCollector getGroupHeadsCollector(Sort sort); - - /** - * Create a second-pass collector - */ - public abstract SecondPassGroupingCollector getSecondPassCollector( - Collection> groups, Sort groupSort, Sort withinGroupSort, - int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException; - -} 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 f4319d578d35..a36917d742ac 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.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 @@ */ 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 GroupingSearch(String groupField, int initialSize) { * @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 GroupingSearch(Query groupEndDocs) { 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 TopGroups search(IndexSearcher searcher, Query query, int groupOff 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 @@ protected TopGroups groupByFieldOrFunction(IndexSearcher searcher, Query query, } 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()) { 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 f8feb756f540..c54c8eee4a81 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 @@ 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 * - *

See {@link org.apache.lucene.search.grouping} for more - * details including a full code example.

+ * @see TopGroupsCollector + * @see DistinctValuesCollector * * @lucene.experimental */ -public abstract class SecondPassGroupingCollector extends SimpleCollector { +public class SecondPassGroupingCollector extends SimpleCollector { - private final Collection> groups; - private final Sort groupSort; - private final Sort withinGroupSort; - private final int maxDocsPerGroup; - private final boolean needsScores; - protected final Map> groupMap; + protected final GroupSelector groupSelector; + protected final Collection> groups; + protected final GroupReducer groupReducer; - protected SearchGroupDocs[] groupDocs; + protected int totalHitCount; + protected int totalGroupedHitCount; - private int totalHitCount; - private int totalGroupedHitCount; - - public SecondPassGroupingCollector(Collection> 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 groupSelector, Collection> groups, GroupReducer 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 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 getGroupSelector() { + return groupSelector; } @Override public boolean needsScores() { - return needsScores; + return groupReducer.needsScores(); } @Override public void setScorer(Scorer scorer) throws IOException { - for (SearchGroupDocs group : groupMap.values()) { - group.leafCollector.setScorer(scorer); - } + groupReducer.setScorer(scorer); } @Override public void collect(int doc) throws IOException { totalHitCount++; - SearchGroupDocs 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 null if no group could be retrieved. - * - * @param doc The specified doc - * @return the group the specified doc belongs to or null if no group could be retrieved - * @throws IOException If an I/O related error occurred - */ - protected abstract SearchGroupDocs retrieveGroup(int doc) throws IOException; - @Override protected void doSetNextReader(LeafReaderContext readerContext) throws IOException { - //System.out.println("SP.setNextReader"); - for (SearchGroupDocs group : groupMap.values()) { - group.leafCollector = group.collector.getLeafCollector(readerContext); - } - } - - public TopGroups getTopGroups(int withinGroupOffset) { - @SuppressWarnings({"unchecked","rawtypes"}) - final GroupDocs[] groupDocsResult = (GroupDocs[]) new GroupDocs[groups.size()]; - - int groupIDX = 0; - float maxScore = Float.MIN_VALUE; - for(SearchGroup group : groups) { - final SearchGroupDocs 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 { - - public final T groupValue; - public final TopDocsCollector collector; - public LeafCollector leafCollector; - - public SearchGroupDocs(T groupValue, TopDocsCollector collector) { - this.groupValue = groupValue; - this.collector = collector; - } - } } 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/TermGroupFacetCollector.java similarity index 97% rename from lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java rename to lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java index cee327cbf473..39d28a579962 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.search.grouping.term; +package org.apache.lucene.search.grouping; import java.io.IOException; import java.util.ArrayList; @@ -25,7 +25,6 @@ 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; @@ -401,15 +400,15 @@ protected void nextTerm() throws IOException { } } } -} -class GroupedFacetHit { + private static class GroupedFacetHit { - final BytesRef groupValue; - final BytesRef facetValue; + final BytesRef groupValue; + final BytesRef facetValue; - GroupedFacetHit(BytesRef groupValue, BytesRef facetValue) { - this.groupValue = groupValue; - this.facetValue = facetValue; + GroupedFacetHit(BytesRef groupValue, BytesRef facetValue) { + this.groupValue = groupValue; + this.facetValue = facetValue; + } } } 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 000000000000..5b8f77c2ce83 --- /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 { + + private final String field; + private final BytesRefHash values = new BytesRefHash(); + private final Map 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> searchGroups) { + this.values.clear(); + this.values.reinit(); + for (SearchGroup sg : searchGroups) { + if (sg.groupValue == null) + includeEmpty = true; + else + this.values.add(sg.groupValue); + } + this.secondPass = true; + } +} 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 000000000000..b6c71d37a8d2 --- /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 the type of the group value + */ +public class TopGroupsCollector extends SecondPassGroupingCollector { + + 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 groupSelector, Collection> 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 extends GroupReducer> { + + private final Supplier> 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 getTopGroups(int withinGroupOffset) { + @SuppressWarnings({"unchecked","rawtypes"}) + final GroupDocs[] groupDocsResult = (GroupDocs[]) new GroupDocs[groups.size()]; + + int groupIDX = 0; + float maxScore = Float.MIN_VALUE; + for(SearchGroup 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); + } + + +} 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 000000000000..249016018855 --- /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 { + + private final ValueSource valueSource; + private final Map context; + + private Set 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> searchGroups) { + secondPassGroups = new HashSet<>(); + for (SearchGroup group : searchGroups) { + secondPassGroups.add(group.groupValue); + } + } +} 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 f4d46682497a..000000000000 --- 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 { - - private final ValueSource groupBy; - private final Map vsContext; - private final Map 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 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 { - - 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(); - } -} 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 1609d4d7bcff..000000000000 --- 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. - *

- * Implementation detail: Uses {@link ValueSource} and {@link FunctionValues} to retrieve the - * field values to group by. - * - * @lucene.experimental - */ -public class FunctionAllGroupsCollector extends AllGroupsCollector { - - private final Map vsContext; - private final ValueSource groupBy; - private final SortedSet 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 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(); - } - -} 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 69ead076d6fc..000000000000 --- 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 { - - private final Map vsContext; - private final ValueSource groupSource; - private final ValueSource countSource; - private final Map> groupMap; - - private FunctionValues.ValueFiller groupFiller; - private FunctionValues.ValueFiller countFiller; - private MutableValue groupMval; - private MutableValue countMval; - - public FunctionDistinctValuesCollector(Map vsContext, ValueSource groupSource, ValueSource countSource, Collection> groups) { - this.vsContext = vsContext; - this.groupSource = groupSource; - this.countSource = countSource; - groupMap = new LinkedHashMap<>(); - for (SearchGroup group : groups) { - groupMap.put(group.groupValue, new GroupCount<>(group.groupValue)); - } - } - - @Override - public List> getGroups() { - return new ArrayList<>(groupMap.values()); - } - - @Override - public void collect(int doc) throws IOException { - groupFiller.fillValue(doc); - GroupCount 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(); - } - -} 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 85376e6a066f..000000000000 --- 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 { - - 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(); - } - -} 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 5204dc266570..000000000000 --- 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 { - - 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 getFirstPassCollector(Sort sort, int count) throws IOException { - return new FunctionFirstPassGroupingCollector(valueSource, context, sort, count); - } - - @Override - public AllGroupHeadsCollector getGroupHeadsCollector(Sort sort) { - return new FunctionAllGroupHeadsCollector(valueSource, context, sort); - } - - @Override - public AllGroupsCollector getAllGroupsCollector() { - return new FunctionAllGroupsCollector(valueSource, context); - } - - @Override - public SecondPassGroupingCollector getSecondPassCollector(Collection> 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); - } -} 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 45f2b37887ec..000000000000 --- 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 { - - 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> 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 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(); - } - -} 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 73588ce24639..000000000000 --- 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; 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 824a98e31bf5..7e3745e61f28 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 @@ * * *

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.

* - *

- * 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. - *

+ *

Groups are defined by {@link org.apache.lucene.search.grouping.GroupSelector} + * implementations:

+ *
    + *
  • {@link org.apache.lucene.search.grouping.TermGroupSelector} groups based on + * the value of a {@link org.apache.lucene.index.SortedDocValues} field
  • + *
  • {@link org.apache.lucene.search.grouping.ValueSourceGroupSelector} groups based on + * the value of a {@link org.apache.lucene.queries.function.ValueSource}
  • + *
* *

Known limitations:

*