diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 8941f5685b0..474df844c02 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -229,6 +229,9 @@ Bug Fixes Now we sync with ZooKeeper and try one more time. V1 partially did this but only for aliases; now it does both. (David Smiley) +* SOLR-12170: JSON Facet API: Terms facet on a date field sometimes failed with an exception complaining + about "Invalid Date String". (yonik) + Optimizations ---------------------- diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java index 9b47d66d62a..f872db35e24 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java @@ -18,7 +18,6 @@ package org.apache.solr.search.facet; import java.io.IOException; -import java.time.Instant; import java.util.ArrayList; import java.util.Collections; import java.util.Date; @@ -708,7 +707,7 @@ abstract class FacetFieldProcessor extends FacetProcessor { bucket.add("val", bucketVal); // fieldQuery currently relies on a string input of the value... - String bucketStr = bucketVal instanceof Date ? Instant.ofEpochMilli(((Date)bucketVal).getTime()).toString() : bucketVal.toString(); + String bucketStr = bucketVal instanceof Date ? ((Date)bucketVal).toInstant().toString() : bucketVal.toString(); Query domainQ = ft.getFieldQuery(null, sf, bucketStr); fillBucket(bucket, domainQ, null, skip, facetInfo); diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java index 228678b7c83..43bafd46ca2 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java @@ -18,6 +18,7 @@ package org.apache.solr.search.facet; import java.io.IOException; +import java.util.Date; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefBuilder; @@ -115,7 +116,8 @@ abstract class FacetFieldProcessorByArray extends FacetFieldProcessor { throw new RuntimeException(e); } }, - Object::toString); // getFieldQueryVal + obj -> obj instanceof Date ? ((Date)obj).toInstant().toString() : obj.toString() + ); } } diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java index 9d94281ea27..0e4266efcdc 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java @@ -17,7 +17,6 @@ package org.apache.solr.search.facet; import java.io.IOException; -import java.time.Instant; import java.util.Collection; import java.util.Date; import java.util.HashMap; @@ -245,7 +244,7 @@ public class FacetModule extends SearchComponent { public void handleUnknownClass(Object o) { // handle date formatting correctly if (o instanceof Date) { - String s = Instant.ofEpochMilli(((Date)o).getTime()).toString(); + String s = ((Date)o).toInstant().toString(); writeString(s); return; } @@ -308,6 +307,7 @@ public class FacetModule extends SearchComponent { return; } + // System.err.println("MERGING FACET RESULT FROM SHARD = " + facet); facetState.mcontext.root = facet; facetState.mcontext.newShard(shardRsp.getShard()); facetState.merger.merge(facet , facetState.mcontext); diff --git a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml index 934451fbe21..70abb394e97 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml @@ -382,6 +382,7 @@ + - diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java index c8bf5ce7073..1e62cb2bd11 100644 --- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java +++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java @@ -239,18 +239,20 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { @Test public void testBasicRefinement() throws Exception { - ModifiableSolrParams p = params("cat_s", "cat_s", "cat_i", "cat_i", "xy_s", "xy_s", "num_d", "num_d", "qw_s", "qw_s", "er_s", "er_s"); + ModifiableSolrParams p; + p = params("cat_s", "cat_s", "cat_i", "cat_i", "date","cat_dt", "xy_s", "xy_s", "num_d", "num_d", "qw_s", "qw_s", "er_s", "er_s"); doBasicRefinement(p); - p.set("terms", "method:dvhash,"); - doBasicRefinement(p); - - // multi-valued - p = params("cat_s", "cat_ss", "cat_i", "cat_is", "xy_s", "xy_ss", "num_d", "num_d", "qw_s", "qw_ss", "er_s", "er_ss"); + // multi-valued (except num_d) + p = params("cat_s", "cat_ss", "cat_i", "cat_is", "date","cat_dts", "xy_s", "xy_ss", "num_d", "num_d", "qw_s", "qw_ss", "er_s", "er_ss"); doBasicRefinement(p); // single valued docvalues - p = params("cat_s", "cat_sd", "cat_i", "cat_id", "xy_s", "xy_sd", "num_d", "num_dd", "qw_s", "qw_sd", "er_s", "er_sd"); + p = params("cat_s", "cat_sd", "cat_i", "cat_id", "date","cat_dtd", "xy_s", "xy_sd", "num_d", "num_dd", "qw_s", "qw_sd", "er_s", "er_sd"); + doBasicRefinement(p); + + // multi valued docvalues (except num_d) + p = params("cat_s", "cat_sds", "cat_i", "cat_ids", "date","cat_dtds", "xy_s", "xy_sds", "num_d", "num_dd", "qw_s", "qw_sds", "er_s", "er_sds"); doBasicRefinement(p); } @@ -270,17 +272,18 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { String qw_s = p.get("qw_s"); String er_s = p.get("er_s"); // this field is designed to test numBuckets refinement... the first phase will only have a single bucket returned for the top count bucket of cat_s String num_d = p.get("num_d"); + String date = p.get("date"); - clients.get(0).add(sdoc("id", "01", "all_s", "all", cat_s, "A", cat_i, 1, xy_s, "X", num_d, -1, qw_s, "Q", er_s, "E")); // A wins count tie - clients.get(0).add(sdoc("id", "02", "all_s", "all", cat_s, "B", cat_i, 2, xy_s, "Y", num_d, 3)); + clients.get(0).add(sdoc("id", "01", "all_s", "all", cat_s, "A", cat_i, 1, date, "2001-01-01T01:01:01Z", xy_s, "X", num_d, -1, qw_s, "Q", er_s, "E")); // A wins count tie + clients.get(0).add(sdoc("id", "02", "all_s", "all", cat_s, "B", cat_i, 2, date, "2002-02-02T02:02:02Z", xy_s, "Y", num_d, 3)); - clients.get(1).add(sdoc("id", "11", "all_s", "all", cat_s, "B", cat_i, 2, xy_s, "X", num_d, -5, er_s, "E")); // B highest count - clients.get(1).add(sdoc("id", "12", "all_s", "all", cat_s, "B", cat_i, 2, xy_s, "Y", num_d, -11, qw_s, "W")); - clients.get(1).add(sdoc("id", "13", "all_s", "all", cat_s, "A", cat_i, 1, xy_s, "X", num_d, 7, er_s, "R")); // "R" will only be picked up via refinement when parent facet is cat_s + clients.get(1).add(sdoc("id", "11", "all_s", "all", cat_s, "B", cat_i, 2, date, "2002-02-02T02:02:02Z", xy_s, "X", num_d, -5, er_s, "E")); // B highest count + clients.get(1).add(sdoc("id", "12", "all_s", "all", cat_s, "B", cat_i, 2, date, "2002-02-02T02:02:02Z", xy_s, "Y", num_d, -11, qw_s, "W")); + clients.get(1).add(sdoc("id", "13", "all_s", "all", cat_s, "A", cat_i, 1, date, "2001-01-01T01:01:01Z", xy_s, "X", num_d, 7, er_s, "R")); // "R" will only be picked up via refinement when parent facet is cat_s - clients.get(2).add(sdoc("id", "21", "all_s", "all", cat_s, "A", cat_i, 1, xy_s, "X", num_d, 17, qw_s, "W", er_s, "E")); // A highest count - clients.get(2).add(sdoc("id", "22", "all_s", "all", cat_s, "A", cat_i, 1, xy_s, "Y", num_d, -19)); - clients.get(2).add(sdoc("id", "23", "all_s", "all", cat_s, "B", cat_i, 2, xy_s, "X", num_d, 11)); + clients.get(2).add(sdoc("id", "21", "all_s", "all", cat_s, "A", cat_i, 1, date, "2001-01-01T01:01:01Z", xy_s, "X", num_d, 17, qw_s, "W", er_s, "E")); // A highest count + clients.get(2).add(sdoc("id", "22", "all_s", "all", cat_s, "A", cat_i, 1, date, "2001-01-01T01:01:01Z", xy_s, "Y", num_d, -19)); + clients.get(2).add(sdoc("id", "23", "all_s", "all", cat_s, "B", cat_i, 2, date, "2002-02-02T02:02:02Z", xy_s, "X", num_d, 11)); client.commit(); @@ -298,229 +301,252 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { , "facets=={foo:555}" ); ****/ + for (String method : new String[]{"","dvhash","stream","uif","enum","stream","smart"}) { + if (method.equals("")) { + p.remove("terms"); + } else { + p.set("terms", "method:" + method+", "); + } - client.testJQ(params(p, "q", "*:*", - "json.facet", "{" + - "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:false}" + - "}" - ) - , "facets=={ count:8" + - ", cat0:{ buckets:[ {val:A,count:3} ] }" + // w/o overrequest and refinement, count is lower than it should be (we don't see the A from the middle shard) - "}" - ); - client.testJQ(params(p, "q", "*:*", - "json.facet", "{" + - "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true}" + - "}" - ) - , "facets=={ count:8" + - ", cat0:{ buckets:[ {val:A,count:4} ] }" + // w/o overrequest, we need refining to get the correct count. - "}" - ); + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:false}" + + "}" + ) + , "facets=={ count:8" + + ", cat0:{ buckets:[ {val:A,count:3} ] }" + // w/o overrequest and refinement, count is lower than it should be (we don't see the A from the middle shard) + "}" + ); - // same as above, but with an integer field instead of a string - client.testJQ(params(p, "q", "*:*", - "json.facet", "{" + - "cat0:{${terms} type:terms, field:${cat_i}, sort:'count desc', limit:1, overrequest:0, refine:true}" + - "}" - ) - , "facets=={ count:8" + - ", cat0:{ buckets:[ {val:1,count:4} ] }" + // w/o overrequest, we need refining to get the correct count. - "}" - ); + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true}" + + "}" + ) + , "facets=={ count:8" + + ", cat0:{ buckets:[ {val:A,count:4} ] }" + // w/o overrequest, we need refining to get the correct count. + "}" + ); - // basic refining test through/under a query facet - client.testJQ(params(p, "q", "*:*", - "json.facet", "{" + - "q1 : { type:query, q:'*:*', facet:{" + - "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true}" + - "}}" + - "}" - ) - , "facets=={ count:8" + - ", q1:{ count:8, cat0:{ buckets:[ {val:A,count:4} ] } }" + - "}" - ); + // same as above, but with an integer field instead of a string + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + "cat0:{${terms} type:terms, field:${cat_i}, sort:'count desc', limit:1, overrequest:0, refine:true}" + + "}" + ) + , "facets=={ count:8" + + ", cat0:{ buckets:[ {val:1,count:4} ] }" + // w/o overrequest, we need refining to get the correct count. + "}" + ); - // basic refining test through/under a range facet - client.testJQ(params(p, "q", "*:*", - "json.facet", "{" + - "r1 : { type:range, field:${num_d} start:-20, end:20, gap:40 , facet:{" + - "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true}" + - "}}" + - "}" - ) - , "facets=={ count:8" + - ", r1:{ buckets:[{val:-20.0,count:8, cat0:{buckets:[{val:A,count:4}]} }] }" + - "}" - ); + // same as above, but with a date field + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + "cat0:{${terms} type:terms, field:${date}, sort:'count desc', limit:1, overrequest:0, refine:true}" + + "}" + ) + , "facets=={ count:8" + + ", cat0:{ buckets:[ {val:'2001-01-01T01:01:01Z',count:4} ] }" + // w/o overrequest, we need refining to get the correct count. + "}" + ); - // test that basic stats work for refinement - client.testJQ(params(p, "q", "*:*", - "json.facet", "{" + - "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true, facet:{ stat1:'sum(${num_d})'} }" + - "}" - ) - , "facets=={ count:8" + - ", cat0:{ buckets:[ {val:A,count:4, stat1:4.0} ] }" + - "}" - ); + // basic refining test through/under a query facet + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + "q1 : { type:query, q:'*:*', facet:{" + + "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true}" + + "}}" + + "}" + ) + , "facets=={ count:8" + + ", q1:{ count:8, cat0:{ buckets:[ {val:A,count:4} ] } }" + + "}" + ); - // test sorting buckets by a different stat - client.testJQ(params(p, "q", "*:*", - "json.facet", "{" + - " cat0:{${terms} type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:false, facet:{ min1:'min(${num_d})'} }" + - ",cat1:{${terms} type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:true, facet:{ min1:'min(${num_d})'} }" + - ",qfacet:{type:query, q:'*:*', facet:{ cat2:{${terms} type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:true, facet:{ min1:'min(${num_d})'} } }}" + // refinement needed through a query facet - ",allf:{${terms} type:terms, field:all_s, facet:{ cat3:{${terms} type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:true, facet:{ min1:'min(${num_d})'} } }}" + // refinement needed through field facet - ",sum1:'sum(${num_d})'" + // make sure that root bucket stats aren't affected by refinement - "}" - ) - , "facets=={ count:8" + - ", cat0:{ buckets:[ {val:A,count:3, min1:-19.0} ] }" + // B wins in shard2, so we're missing the "A" count for that shard w/o refinement. - ", cat1:{ buckets:[ {val:A,count:4, min1:-19.0} ] }" + // with refinement, we get the right count - ", qfacet:{ count:8, cat2:{ buckets:[ {val:A,count:4, min1:-19.0} ] } }" + // just like the previous response, just nested under a query facet - ", allf:{ buckets:[ {cat3:{ buckets:[ {val:A,count:4, min1:-19.0} ] } ,count:8,val:all }] }" + // just like the previous response, just nested under a field facet - ", sum1:2.0" + - "}" - ); + // basic refining test through/under a range facet + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + "r1 : { type:range, field:${num_d} start:-20, end:20, gap:40 , facet:{" + + "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true}" + + "}}" + + "}" + ) + , "facets=={ count:8" + + ", r1:{ buckets:[{val:-20.0,count:8, cat0:{buckets:[{val:A,count:4}]} }] }" + + "}" + ); - // test partial buckets (field facet within field facet) - client.testJQ(params(p, "q", "*:*", - "json.facet", "{" + - "ab:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true, facet:{ xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true } }}" + - "}" - ) - , "facets=={ count:8" + - ", ab:{ buckets:[ {val:A, count:4, xy:{buckets:[ {val:X,count:3}]} }] }" + // just like the previous response, just nested under a field facet - "}" - ); + // test that basic stats work for refinement + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + "cat0:{${terms} type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true, facet:{ stat1:'sum(${num_d})'} }" + + "}" + ) + , "facets=={ count:8" + + ", cat0:{ buckets:[ {val:A,count:4, stat1:4.0} ] }" + + "}" + ); - // test that sibling facets and stats are included for _p buckets, but skipped for _s buckets - client.testJQ(params(p, "q", "*:*", - "json.facet", "{" + - " ab :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true, facet:{ xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})' }}" + - ",ab2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, facet:{ xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})' }}" + // top level refine=false shouldn't matter - ",allf :{${terms} type:terms, field:all_s, limit:1, overrequest:0, refine:true, facet:{cat:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})' }}" + - ",allf2:{${terms} type:terms, field:all_s, limit:1, overrequest:0, refine:false, facet:{cat:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})' }}" + // top level refine=false shouldn't matter - "}" - ) - , "facets=={ count:8" + - ", ab:{ buckets:[ {val:A, count:4, xy:{buckets:[ {val:X,count:3}]} ,qq:{count:4}, ww:4.0 }] }" + // make sure qq and ww are included for _p buckets - ", allf:{ buckets:[ {count:8, val:all, cat:{buckets:[{val:A,count:4}]} ,qq:{count:8}, ww:2.0 }] }" + // make sure qq and ww are excluded (not calculated again in another phase) for _s buckets - ", ab2:{ buckets:[ {val:A, count:4, xy:{buckets:[ {val:X,count:3}]} ,qq:{count:4}, ww:4.0 }] }" + // make sure qq and ww are included for _p buckets - ", allf2:{ buckets:[ {count:8, val:all, cat:{buckets:[{val:A,count:4}]} ,qq:{count:8}, ww:2.0 }] }" + // make sure qq and ww are excluded (not calculated again in another phase) for _s buckets - "}" - ); + // test sorting buckets by a different stat + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + " cat0:{${terms} type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:false, facet:{ min1:'min(${num_d})'} }" + + ",cat1:{${terms} type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:true, facet:{ min1:'min(${num_d})'} }" + + ",qfacet:{type:query, q:'*:*', facet:{ cat2:{${terms} type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:true, facet:{ min1:'min(${num_d})'} } }}" + // refinement needed through a query facet + ",allf:{${terms} type:terms, field:all_s, facet:{ cat3:{${terms} type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:true, facet:{ min1:'min(${num_d})'} } }}" + // refinement needed through field facet + ",sum1:'sum(${num_d})'" + // make sure that root bucket stats aren't affected by refinement + "}" + ) + , "facets=={ count:8" + + ", cat0:{ buckets:[ {val:A,count:3, min1:-19.0} ] }" + // B wins in shard2, so we're missing the "A" count for that shard w/o refinement. + ", cat1:{ buckets:[ {val:A,count:4, min1:-19.0} ] }" + // with refinement, we get the right count + ", qfacet:{ count:8, cat2:{ buckets:[ {val:A,count:4, min1:-19.0} ] } }" + // just like the previous response, just nested under a query facet + ", allf:{ buckets:[ {cat3:{ buckets:[ {val:A,count:4, min1:-19.0} ] } ,count:8,val:all }] }" + // just like the previous response, just nested under a field facet + ", sum1:2.0" + + "}" + ); - // test refining under the special "missing" bucket of a field facet - client.testJQ(params(p, "q", "*:*", - "json.facet", "{" + - "f:{${terms} type:terms, field:missing_s, limit:1, overrequest:0, missing:true, refine:true, facet:{ cat:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true } }}" + - "}" - ) - , "facets=={ count:8" + - ", f:{ buckets:[], missing:{count:8, cat:{buckets:[{val:A,count:4}]} } }" + // just like the previous response, just nested under a field facet - "}" - ); + // test partial buckets (field facet within field facet) + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + " ab:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true, facet:{ xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true } }}" + + ",cd:{${terms} type:terms, field:${cat_i}, limit:1, overrequest:0, refine:true, facet:{ xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true } }}" + + ",ef:{${terms} type:terms, field:${date}, limit:1, overrequest:0, refine:true, facet:{ xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true } }}" + + "}" + ) + , "facets=={ count:8" + + ", ab:{ buckets:[ {val:A, count:4, xy:{buckets:[ {val:X,count:3}]} }] }" + // just like the previous response, just nested under a field facet + ", cd:{ buckets:[ {val:1, count:4, xy:{buckets:[ {val:X,count:3}]} }] }" + // just like the previous response, just nested under a field facet (int type) + ", ef:{ buckets:[ {val:'2001-01-01T01:01:01Z', count:4, xy:{buckets:[ {val:X,count:3}]} }] }" + // just like the previous response, just nested under a field facet (date type) + "}" + ); - // test filling in "missing" bucket for partially refined facets - client.testJQ(params(p, "q", "*:*", - "json.facet", "{" + - // test all values missing in sub-facet - " ab :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, facet:{ zz:{${terms} type:terms, field:missing_s, limit:1, overrequest:0, refine:false, missing:true} }}" + - ",ab2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , facet:{ zz:{${terms} type:terms, field:missing_s, limit:1, overrequest:0, refine:true , missing:true} }}" + - // test some values missing in sub-facet (and test that this works with normal partial bucket refinement) - ", cd :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, facet:{ qw:{${terms} type:terms, field:${qw_s}, limit:1, overrequest:0, refine:false, missing:true, facet:{qq:{query:'*:*'}} } }}" + - ", cd2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , facet:{ qw:{${terms} type:terms, field:${qw_s}, limit:1, overrequest:0, refine:true , missing:true, facet:{qq:{query:'*:*'}} } }}" + + // test that sibling facets and stats are included for _p buckets, but skipped for _s buckets + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + " ab :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true, facet:{ xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})' }}" + + ",ab2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, facet:{ xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})' }}" + // top level refine=false shouldn't matter + ",allf :{${terms} type:terms, field:all_s, limit:1, overrequest:0, refine:true, facet:{cat:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})' }}" + + ",allf2:{${terms} type:terms, field:all_s, limit:1, overrequest:0, refine:false, facet:{cat:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})' }}" + // top level refine=false shouldn't matter + "}" + ) + , "facets=={ count:8" + + ", ab:{ buckets:[ {val:A, count:4, xy:{buckets:[ {val:X,count:3}]} ,qq:{count:4}, ww:4.0 }] }" + // make sure qq and ww are included for _p buckets + ", allf:{ buckets:[ {count:8, val:all, cat:{buckets:[{val:A,count:4}]} ,qq:{count:8}, ww:2.0 }] }" + // make sure qq and ww are excluded (not calculated again in another phase) for _s buckets + ", ab2:{ buckets:[ {val:A, count:4, xy:{buckets:[ {val:X,count:3}]} ,qq:{count:4}, ww:4.0 }] }" + // make sure qq and ww are included for _p buckets + ", allf2:{ buckets:[ {count:8, val:all, cat:{buckets:[{val:A,count:4}]} ,qq:{count:8}, ww:2.0 }] }" + // make sure qq and ww are excluded (not calculated again in another phase) for _s buckets + "}" + ); - "}" - ) - , "facets=={ count:8" + - ", ab:{ buckets:[ {val:A, count:3, zz:{buckets:[], missing:{count:3}}}] }" + - ",ab2:{ buckets:[ {val:A, count:4, zz:{buckets:[], missing:{count:4}}}] }" + - ", cd:{ buckets:[ {val:A, count:3, qw:{buckets:[{val:Q, count:1, qq:{count:1}}], missing:{count:1,qq:{count:1}}}}] }" + - ",cd2:{ buckets:[ {val:A, count:4, qw:{buckets:[{val:Q, count:1, qq:{count:1}}], missing:{count:2,qq:{count:2}}}}] }" + - "}" - ); + // test refining under the special "missing" bucket of a field facet + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + "f:{${terms} type:terms, field:missing_s, limit:1, overrequest:0, missing:true, refine:true, facet:{ cat:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true } }}" + + "}" + ) + , "facets=={ count:8" + + ", f:{ buckets:[], missing:{count:8, cat:{buckets:[{val:A,count:4}]} } }" + // just like the previous response, just nested under a field facet + "}" + ); - // test filling in missing "allBuckets" - client.testJQ(params(p, "q", "*:*", - "json.facet", "{" + - " cat :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, allBuckets:true, facet:{ xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, allBuckets:true, refine:false} } }" + - ", cat2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , allBuckets:true, facet:{ xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, allBuckets:true, refine:true } } }" + - ", cat3:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , allBuckets:true, facet:{ xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, allBuckets:true, refine:true , facet:{f:'sum(${num_d})'} } } }" + - "}" - ) - , "facets=={ count:8" + - ", cat:{ allBuckets:{count:8}, buckets:[ {val:A, count:3, xy:{buckets:[{count:2, val:X}], allBuckets:{count:3}}}] }" + - ",cat2:{ allBuckets:{count:8}, buckets:[ {val:A, count:4, xy:{buckets:[{count:3, val:X}], allBuckets:{count:4}}}] }" + - ",cat3:{ allBuckets:{count:8}, buckets:[ {val:A, count:4, xy:{buckets:[{count:3, val:X, f:23.0}], allBuckets:{count:4, f:4.0}}}] }" + - "}" - ); + // test filling in "missing" bucket for partially refined facets + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + // test all values missing in sub-facet + " ab :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, facet:{ zz:{${terms} type:terms, field:missing_s, limit:1, overrequest:0, refine:false, missing:true} }}" + + ",ab2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , facet:{ zz:{${terms} type:terms, field:missing_s, limit:1, overrequest:0, refine:true , missing:true} }}" + + // test some values missing in sub-facet (and test that this works with normal partial bucket refinement) + ", cd :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, facet:{ qw:{${terms} type:terms, field:${qw_s}, limit:1, overrequest:0, refine:false, missing:true, facet:{qq:{query:'*:*'}} } }}" + + ", cd2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , facet:{ qw:{${terms} type:terms, field:${qw_s}, limit:1, overrequest:0, refine:true , missing:true, facet:{qq:{query:'*:*'}} } }}" + - // test filling in missing numBuckets - client.testJQ(params(p, "q", "*:*", - "json.facet", "{" + - " cat :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, numBuckets:true, facet:{ er:{${terms} type:terms, field:${er_s}, limit:1, overrequest:0, numBuckets:true, refine:false} } }" + - ", cat2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , numBuckets:true, facet:{ er:{${terms} type:terms, field:${er_s}, limit:1, overrequest:0, numBuckets:true, refine:true } } }" + - "}" - ) - , "facets=={ count:8" + - ", cat:{ numBuckets:2, buckets:[ {val:A, count:3, er:{numBuckets:1,buckets:[{count:2, val:E}] }}] }" + // the "R" bucket will not be seen w/o refinement - ",cat2:{ numBuckets:2, buckets:[ {val:A, count:4, er:{numBuckets:2,buckets:[{count:2, val:E}] }}] }" + - "}" - ); + "}" + ) + , "facets=={ count:8" + + ", ab:{ buckets:[ {val:A, count:3, zz:{buckets:[], missing:{count:3}}}] }" + + ",ab2:{ buckets:[ {val:A, count:4, zz:{buckets:[], missing:{count:4}}}] }" + + ", cd:{ buckets:[ {val:A, count:3, qw:{buckets:[{val:Q, count:1, qq:{count:1}}], missing:{count:1,qq:{count:1}}}}] }" + + ",cd2:{ buckets:[ {val:A, count:4, qw:{buckets:[{val:Q, count:1, qq:{count:1}}], missing:{count:2,qq:{count:2}}}}] }" + + "}" + ); - final String sort_limit_over = "sort:'count desc', limit:1, overrequest:0, "; - // simplistic join domain testing: no refinement == low count - client.testJQ(params(p, "q", "${xy_s}:Y", // query only matches one doc per shard - "json.facet", "{" + - " cat0:{${terms} type:terms, field:${cat_s}, " + sort_limit_over + " refine:false," + - // self join on all_s ensures every doc on every shard included in facets - " domain: { join: { from:all_s, to:all_s } } }" + - "}" - ) - , - "/response/numFound==3", - "facets=={ count:3, " + - // w/o overrequest and refinement, count for 'A' is lower than it should be - // (we don't see the A from the middle shard) - " cat0:{ buckets:[ {val:A,count:3} ] } }"); - // simplistic join domain testing: refinement == correct count - client.testJQ(params(p, "q", "${xy_s}:Y", // query only matches one doc per shard - "json.facet", "{" + - " cat0:{${terms} type:terms, field:${cat_s}, " + sort_limit_over + " refine:true," + - // self join on all_s ensures every doc on every shard included in facets - " domain: { join: { from:all_s, to:all_s } } }" + - "}" - ) - , - "/response/numFound==3", - "facets=={ count:3," + - // w/o overrequest, we need refining to get the correct count for 'A'. - " cat0:{ buckets:[ {val:A,count:4} ] } }"); + // test filling in missing "allBuckets" + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + " cat :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, allBuckets:true, facet:{ xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, allBuckets:true, refine:false} } }" + + ", cat2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , allBuckets:true, facet:{ xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, allBuckets:true, refine:true } } }" + + ", cat3:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , allBuckets:true, facet:{ xy:{${terms} type:terms, field:${xy_s}, limit:1, overrequest:0, allBuckets:true, refine:true , facet:{f:'sum(${num_d})'} } } }" + + "}" + ) + , "facets=={ count:8" + + ", cat:{ allBuckets:{count:8}, buckets:[ {val:A, count:3, xy:{buckets:[{count:2, val:X}], allBuckets:{count:3}}}] }" + + ",cat2:{ allBuckets:{count:8}, buckets:[ {val:A, count:4, xy:{buckets:[{count:3, val:X}], allBuckets:{count:4}}}] }" + + ",cat3:{ allBuckets:{count:8}, buckets:[ {val:A, count:4, xy:{buckets:[{count:3, val:X, f:23.0}], allBuckets:{count:4, f:4.0}}}] }" + + "}" + ); - // contrived join domain + refinement (at second level) + testing - client.testJQ(params(p, "q", "${xy_s}:Y", // query only matches one doc per shard - "json.facet", "{" + - // top level facet has a single term - " all:{${terms} type:terms, field:all_s, " + sort_limit_over + " refine:true, " + - " facet:{ " + - // subfacet will facet on cat after joining on all (so all docs should be included in subfacet) - " cat0:{${terms} type:terms, field:${cat_s}, " + sort_limit_over + " refine:true," + - " domain: { join: { from:all_s, to:all_s } } } } }" + - "}" - ) - , - "/response/numFound==3", - "facets=={ count:3," + - // all 3 docs matching base query have same 'all' value in top facet - " all:{ buckets:[ { val:all, count:3, " + - // sub facet has refinement, so count for 'A' should be correct - " cat0:{ buckets: [{val:A,count:4}] } } ] } }"); + // test filling in missing numBuckets + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + " cat :{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, numBuckets:true, facet:{ er:{${terms} type:terms, field:${er_s}, limit:1, overrequest:0, numBuckets:true, refine:false} } }" + + ", cat2:{${terms} type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , numBuckets:true, facet:{ er:{${terms} type:terms, field:${er_s}, limit:1, overrequest:0, numBuckets:true, refine:true } } }" + + "}" + ) + , "facets=={ count:8" + + ", cat:{ numBuckets:2, buckets:[ {val:A, count:3, er:{numBuckets:1,buckets:[{count:2, val:E}] }}] }" + // the "R" bucket will not be seen w/o refinement + ",cat2:{ numBuckets:2, buckets:[ {val:A, count:4, er:{numBuckets:2,buckets:[{count:2, val:E}] }}] }" + + "}" + ); + final String sort_limit_over = "sort:'count desc', limit:1, overrequest:0, "; + // simplistic join domain testing: no refinement == low count + client.testJQ(params(p, "q", "${xy_s}:Y", // query only matches one doc per shard + "json.facet", "{" + + " cat0:{${terms} type:terms, field:${cat_s}, " + sort_limit_over + " refine:false," + + // self join on all_s ensures every doc on every shard included in facets + " domain: { join: { from:all_s, to:all_s } } }" + + "}" + ) + , + "/response/numFound==3", + "facets=={ count:3, " + + // w/o overrequest and refinement, count for 'A' is lower than it should be + // (we don't see the A from the middle shard) + " cat0:{ buckets:[ {val:A,count:3} ] } }"); + // simplistic join domain testing: refinement == correct count + client.testJQ(params(p, "q", "${xy_s}:Y", // query only matches one doc per shard + "json.facet", "{" + + " cat0:{${terms} type:terms, field:${cat_s}, " + sort_limit_over + " refine:true," + + // self join on all_s ensures every doc on every shard included in facets + " domain: { join: { from:all_s, to:all_s } } }" + + "}" + ) + , + "/response/numFound==3", + "facets=={ count:3," + + // w/o overrequest, we need refining to get the correct count for 'A'. + " cat0:{ buckets:[ {val:A,count:4} ] } }"); + + // contrived join domain + refinement (at second level) + testing + client.testJQ(params(p, "q", "${xy_s}:Y", // query only matches one doc per shard + "json.facet", "{" + + // top level facet has a single term + " all:{${terms} type:terms, field:all_s, " + sort_limit_over + " refine:true, " + + " facet:{ " + + // subfacet will facet on cat after joining on all (so all docs should be included in subfacet) + " cat0:{${terms} type:terms, field:${cat_s}, " + sort_limit_over + " refine:true," + + " domain: { join: { from:all_s, to:all_s } } } } }" + + "}" + ) + , + "/response/numFound==3", + "facets=={ count:3," + + // all 3 docs matching base query have same 'all' value in top facet + " all:{ buckets:[ { val:all, count:3, " + + // sub facet has refinement, so count for 'A' should be correct + " cat0:{ buckets: [{val:A,count:4}] } } ] } }"); + + } // end method loop } // Unlike solrconfig.xml this test using solrconfig-tlog.xml should not fail with too-many-exceptions (see TestSolrQueryParser.testManyClauses) diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java index 65d4e75ccd3..b6afdb8bc62 100644 --- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java +++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java @@ -760,10 +760,15 @@ public class TestJsonFacets extends SolrTestCaseHS { client.testJQ(params(p, "q", "*:*" , "json.facet", "{" + " f1:{${terms} type:field, field:${date}}" + + ",f2:{${terms} type:field, field:${date} sort:'index asc'}" + + ",f3:{${terms} type:field, field:${date} sort:'index desc'}" + + // ",f4:{${terms} type:field, field:${date}, facet:{x:{type:field,field:${num_is},limit:1}} }" + "}" ) , "facets=={count:6 " + ",f1:{ buckets:[ {val:'2001-01-01T01:01:01Z', count:1},{val:'2001-02-03T01:02:03Z', count:1},{val:'2002-02-02T02:02:02Z', count:1},{val:'2002-03-01T03:02:01Z', count:1},{val:'2003-03-03T03:03:03Z', count:1} ] }" + + ",f2:{ buckets:[ {val:'2001-01-01T01:01:01Z', count:1},{val:'2001-02-03T01:02:03Z', count:1},{val:'2002-02-02T02:02:02Z', count:1},{val:'2002-03-01T03:02:01Z', count:1},{val:'2003-03-03T03:03:03Z', count:1} ] }" + + ",f3:{ buckets:[ {val:'2003-03-03T03:03:03Z', count:1},{val:'2002-03-01T03:02:01Z', count:1},{val:'2002-02-02T02:02:02Z', count:1},{val:'2001-02-03T01:02:03Z', count:1},{val:'2001-01-01T01:01:01Z', count:1} ] }" + "}" );