mirror of https://github.com/apache/lucene.git
SOLR-12343: Fixed a bug in JSON Faceting that could cause incorrect counts/stats when using non default sort options
This also adds a new configurable "overrefine" option
This commit is contained in:
parent
d443ed088d
commit
3a5d4a25df
|
@ -154,6 +154,9 @@ Bug Fixes
|
|||
* SOLR-12516: Fix some bugs in 'type:range' Facet refinement when sub-facets are combined with non
|
||||
default values for the 'other' and 'include' options. (hossman)
|
||||
|
||||
* SOLR-12343: Fixed a bug in JSON Faceting that could cause incorrect counts/stats when using non default
|
||||
sort options. This also adds a new configurable "overrefine" option. (Yonik Seeley, hossman)
|
||||
|
||||
Optimizations
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -29,7 +29,16 @@ import org.apache.solr.schema.SchemaField;
|
|||
abstract class FacetRequestSorted extends FacetRequest {
|
||||
long offset;
|
||||
long limit;
|
||||
int overrequest = -1; // Number of buckets to request beyond the limit to do internally during distributed search. -1 means default.
|
||||
/**
|
||||
* Number of buckets to request beyond the limit to do internally during initial distributed search.
|
||||
* -1 means default heuristic.
|
||||
*/
|
||||
int overrequest = -1;
|
||||
/**
|
||||
* Number of buckets to fill in beyond the limit to do internally during refinement of distributed search.
|
||||
* -1 means default heuristic.
|
||||
*/
|
||||
int overrefine = -1;
|
||||
long mincount;
|
||||
String sortVariable;
|
||||
SortDirection sortDirection;
|
||||
|
|
|
@ -119,12 +119,17 @@ public class FacetFieldMerger extends FacetRequestSortedMerger<FacetField> {
|
|||
|
||||
// TODO: change effective offsets + limits at shards...
|
||||
|
||||
boolean refine = freq.refine != null && freq.refine != FacetRequest.RefineMethod.NONE;
|
||||
|
||||
int off = (int)freq.offset;
|
||||
int lim = freq.limit >= 0 ? (int)freq.limit : Integer.MAX_VALUE;
|
||||
for (FacetBucket bucket : sortedBuckets) {
|
||||
if (bucket.getCount() < freq.mincount) {
|
||||
continue;
|
||||
}
|
||||
if (refine && !isBucketComplete(bucket,mcontext)) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (off > 0) {
|
||||
--off;
|
||||
|
|
|
@ -36,7 +36,7 @@ public abstract class FacetMerger {
|
|||
return null;
|
||||
}
|
||||
public abstract void finish(Context mcontext);
|
||||
public abstract Object getMergedResult();
|
||||
public abstract Object getMergedResult(); // TODO: we should pass mcontext through here as well
|
||||
|
||||
// This class lets mergers know overall context such as what shard is being merged
|
||||
// and what buckets have been seen by what shard.
|
||||
|
@ -74,10 +74,14 @@ public abstract class FacetMerger {
|
|||
sawShard.set( bucketNum * numShards + shardNum );
|
||||
}
|
||||
|
||||
public boolean getShardFlag(int bucketNum) {
|
||||
public boolean getShardFlag(int bucketNum, int shardNum) {
|
||||
return sawShard.get( bucketNum * numShards + shardNum );
|
||||
}
|
||||
|
||||
public boolean getShardFlag(int bucketNum) {
|
||||
return getShardFlag(bucketNum, shardNum);
|
||||
}
|
||||
|
||||
public boolean bucketWasMissing() {
|
||||
return bucketWasMissing;
|
||||
}
|
||||
|
|
|
@ -879,6 +879,7 @@ class FacetFieldParser extends FacetParser<FacetField> {
|
|||
facet.offset = getLong(m, "offset", facet.offset);
|
||||
facet.limit = getLong(m, "limit", facet.limit);
|
||||
facet.overrequest = (int) getLong(m, "overrequest", facet.overrequest);
|
||||
facet.overrefine = (int) getLong(m, "overrefine", facet.overrefine);
|
||||
if (facet.limit == 0) facet.offset = 0; // normalize. an offset with a limit of non-zero isn't useful.
|
||||
facet.mincount = getLong(m, "mincount", facet.mincount);
|
||||
facet.missing = getBoolean(m, "missing", facet.missing);
|
||||
|
|
|
@ -35,7 +35,7 @@ abstract class FacetRequestSortedMerger<FacetRequestT extends FacetRequestSorted
|
|||
LinkedHashMap<Object,FacetBucket> buckets = new LinkedHashMap<>();
|
||||
List<FacetBucket> sortedBuckets;
|
||||
BitSet shardHasMoreBuckets; // null, or "true" if we saw a result from this shard and it indicated that there are more results
|
||||
|
||||
Context mcontext; // HACK: this should be passed in getMergedResult as well!
|
||||
|
||||
public FacetRequestSortedMerger(FacetRequestT freq) {
|
||||
super(freq);
|
||||
|
@ -43,6 +43,7 @@ abstract class FacetRequestSortedMerger<FacetRequestT extends FacetRequestSorted
|
|||
|
||||
@Override
|
||||
public void merge(Object facetResult, Context mcontext) {
|
||||
this.mcontext = mcontext;
|
||||
SimpleOrderedMap res = (SimpleOrderedMap)facetResult;
|
||||
Boolean more = (Boolean)res.get("more");
|
||||
if (more != null && more) {
|
||||
|
@ -155,6 +156,16 @@ abstract class FacetRequestSortedMerger<FacetRequestT extends FacetRequestSorted
|
|||
}
|
||||
}
|
||||
|
||||
boolean isBucketComplete(FacetBucket bucket, Context mcontext) {
|
||||
if (mcontext.numShards <= 1 || shardHasMoreBuckets==null) return true;
|
||||
for (int shard=0; shard < mcontext.numShards; shard++) {
|
||||
// bucket is incomplete if we didn't see the bucket for this shard, and the shard has more buckets
|
||||
if (!mcontext.getShardFlag(bucket.bucketNumber, shard) && shardHasMoreBuckets!=null && shardHasMoreBuckets.get(shard)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Object> getRefinement(Context mcontext) {
|
||||
|
@ -188,13 +199,45 @@ abstract class FacetRequestSortedMerger<FacetRequestT extends FacetRequestSorted
|
|||
return null;
|
||||
}
|
||||
|
||||
long numBucketsToCheck = Integer.MAX_VALUE; // use max-int instead of max-long to avoid overflow
|
||||
if (freq.limit >= 0) {
|
||||
numBucketsToCheck = freq.offset + freq.limit; // effective limit
|
||||
if (-1 == freq.overrefine) { // DEFAULT: use heuristic for overrefinement
|
||||
|
||||
int num = freq.limit < 0 ? Integer.MAX_VALUE : (int)(freq.offset + freq.limit);
|
||||
int numBucketsToCheck = Math.min(buckets.size(), num);
|
||||
// when we don't have to worry about mincount pruning, there is no need for any
|
||||
// over refinement for these sorts..
|
||||
if (freq.mincount <= 1 && ("index".equals(freq.sortVariable)
|
||||
|| ("count".equals(freq.sortVariable)
|
||||
&& FacetRequest.SortDirection.desc == freq.sortDirection))) {
|
||||
// No-Op
|
||||
} else if (0 <= freq.overrequest) {
|
||||
// if user asked for an explicit amount of overrequesting,
|
||||
// (but did not provide an explicit amount of overrefinement)
|
||||
// then use the same amount for overrefinement
|
||||
numBucketsToCheck += freq.overrequest;
|
||||
} else {
|
||||
// default: add 10% plus 4
|
||||
numBucketsToCheck = (long) (numBucketsToCheck * 1.1 +4);
|
||||
}
|
||||
|
||||
// TODO: should we scale our 'overrefine' (heuristic) value based on 'mincount' ?
|
||||
//
|
||||
// If mincount=M > 1 should we be doing something like numBucketsToCheck *= M ?
|
||||
// Perhaps that would make more sense in the 'overrequest' heuristic calc?
|
||||
//
|
||||
// Maybe we should look at how many buckets were fully populated in phase#1 AND
|
||||
// already meet the 'mincount', and use the the difference between that number
|
||||
// and 'limit' to decide a scaling factor for 'overrefine' ?
|
||||
|
||||
} else { // user requested an explicit amount of overrefinement
|
||||
numBucketsToCheck += freq.overrefine;
|
||||
}
|
||||
}
|
||||
numBucketsToCheck = Math.min(buckets.size(), numBucketsToCheck);
|
||||
|
||||
Collection<FacetBucket> bucketList;
|
||||
if (buckets.size() < num) {
|
||||
// no need to sort
|
||||
if (buckets.size() < numBucketsToCheck) {
|
||||
// no need to sort (yet)
|
||||
// todo: but we may need to filter.... simplify by always sorting?
|
||||
bucketList = buckets.values();
|
||||
} else {
|
||||
|
|
|
@ -554,7 +554,7 @@ public class TestCloudJSONFacetSKG extends SolrCloudTestCase {
|
|||
|
||||
// IMPORTANT!!!
|
||||
// if this method is modified to produce new sorts, make sure to update
|
||||
// randomLimitParam to account for them if they are impacted by SOLR-12343
|
||||
// randomLimitParam to account for them if they are impacted by SOLR-12556
|
||||
final String dir = random().nextBoolean() ? "asc" : "desc";
|
||||
switch(r.nextInt(4)) {
|
||||
case 0: return null;
|
||||
|
@ -568,7 +568,7 @@ public class TestCloudJSONFacetSKG extends SolrCloudTestCase {
|
|||
* picks a random value for the "limit" param, biased in favor of interesting test cases
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE:</b> Due to SOLR-12343, we have to force an overrequest of "all" possible terms for
|
||||
* <b>NOTE:</b> Due to SOLR-12556, we have to force an overrequest of "all" possible terms for
|
||||
* some sort values.
|
||||
* </p>
|
||||
*
|
||||
|
@ -579,7 +579,7 @@ public class TestCloudJSONFacetSKG extends SolrCloudTestCase {
|
|||
public static Integer randomLimitParam(Random r, final String sort) {
|
||||
if (null != sort) {
|
||||
if (sort.equals("count asc") || sort.startsWith("skg")) {
|
||||
// of the known types of sorts produced, these are at risk of SOLR-12343
|
||||
// of the known types of sorts produced, these are at risk of SOLR-12556
|
||||
// so request (effectively) unlimited num buckets
|
||||
return r.nextBoolean() ? UNIQUE_FIELD_VALS : -1;
|
||||
}
|
||||
|
|
|
@ -38,7 +38,14 @@ public class DebugAgg extends AggValueSource {
|
|||
@Override
|
||||
public ValueSource parse(FunctionQParser fp) throws SyntaxError {
|
||||
parses.incrementAndGet();
|
||||
return new DebugAgg();
|
||||
final String what = fp.hasMoreArguments() ? fp.parseId() : "debug";
|
||||
|
||||
switch (what) {
|
||||
case "debug": return new DebugAgg();
|
||||
case "numShards": return new DebugAggNumShards();
|
||||
default: /* No-Op */
|
||||
}
|
||||
throw new RuntimeException("No idea what to do with " + what);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -150,4 +157,63 @@ public class DebugAgg extends AggValueSource {
|
|||
return new FacetLongMerger();
|
||||
}
|
||||
|
||||
/** A simple agg that just returns the number of shards contributing to a bucket */
|
||||
public static class DebugAggNumShards extends AggValueSource {
|
||||
public DebugAggNumShards() {
|
||||
super("debugNumShards");
|
||||
}
|
||||
|
||||
@Override
|
||||
public SlotAcc createSlotAcc(FacetContext fcontext, int numDocs, int numSlots) {
|
||||
return new NumShardsAcc(fcontext, numDocs, numSlots);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String description() {
|
||||
return "debug(numShards)";
|
||||
}
|
||||
|
||||
public static class NumShardsAcc extends SlotAcc {
|
||||
public NumShardsAcc(FacetContext fcontext, int numDocs, int numSlots) {
|
||||
super(fcontext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int doc, int slot, IntFunction<SlotContext> slotContext) throws IOException {
|
||||
// No-Op
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(int slotA, int slotB) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getValue(int slotNum) throws IOException {
|
||||
return 1L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() throws IOException {
|
||||
// No-Op
|
||||
}
|
||||
|
||||
@Override
|
||||
public void resize(Resizer resizer) {
|
||||
// No-op
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public FacetMerger createFacetMerger(Object prototype) {
|
||||
return new FacetLongMerger();
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,6 +21,8 @@ import java.io.IOException;
|
|||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
import org.apache.solr.JSONTestUtil;
|
||||
import org.apache.solr.SolrTestCaseHS;
|
||||
import org.apache.solr.client.solrj.SolrClient;
|
||||
|
@ -160,6 +162,7 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS {
|
|||
|
||||
@Test
|
||||
public void testMerge() throws Exception {
|
||||
|
||||
doTestRefine("{x : {type:terms, field:X, limit:2, refine:true} }", // the facet request
|
||||
"{x: {buckets:[{val:x1, count:5}, {val:x2, count:3}], more:true } }", // shard0 response
|
||||
"{x: {buckets:[{val:x2, count:4}, {val:x3, count:2}], more:true } }", // shard1 response
|
||||
|
@ -168,24 +171,24 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS {
|
|||
);
|
||||
|
||||
// same test as above, but shard1 indicates it doesn't have any more results, so there shouldn't be any refinement
|
||||
doTestRefine("{x : {type:terms, field:X, limit:2, refine:true} }", // the facet request
|
||||
"{x: {buckets:[{val:x1, count:5}, {val:x2, count:3}], more:true } }", // shard0 response
|
||||
"{x: {buckets:[{val:x2, count:4}, {val:x3, count:2}] } }", // shard1 response
|
||||
null, // shard0 expected refinement info
|
||||
null // shard1 expected refinement info
|
||||
);
|
||||
|
||||
doTestRefine("{x : {type:terms, field:X, limit:2, refine:true} }", // the facet request
|
||||
"{x: {buckets:[{val:x1, count:5}, {val:x2, count:3}],more:true } }", // shard0 response
|
||||
"{x: {buckets:[{val:x2, count:4}, {val:x3, count:2}] } }", // shard1 response
|
||||
null, // shard0 expected refinement info
|
||||
null // shard1 expected refinement info // without more:true, we should not attempt to get extra bucket
|
||||
);
|
||||
// same but with processEmpty:true we should check for refinement even if there isn't "more"
|
||||
doTestRefine("{x : {type:terms, field:X, limit:2, refine:true, facet: { processEmpty:true } } }",
|
||||
"{x: {buckets:[{val:x1, count:5}, {val:x2, count:3}],more:true } }", // shard0 response
|
||||
"{x: {buckets:[{val:x2, count:4}] } }", // shard1 response -- NO "more"
|
||||
null, // shard0 expected refinement info
|
||||
"=={x:{_l:[x1]}}" // shard1 expected refinement info
|
||||
);
|
||||
|
||||
// same test w/o refinement turned on
|
||||
// same test w/o refinement turned on (even though shards say they have more)
|
||||
doTestRefine("{x : {type:terms, field:X, limit:2} }", // the facet request
|
||||
"{x: {buckets:[{val:x1, count:5}, {val:x2, count:3}] } }", // shard0 response
|
||||
"{x: {buckets:[{val:x2, count:4}, {val:x3, count:2}] } }", // shard1 response
|
||||
"{x: {buckets:[{val:x1, count:5}, {val:x2, count:3}], more:true } }", // shard0 response
|
||||
"{x: {buckets:[{val:x2, count:4}, {val:x3, count:2}], more:true } }", // shard1 response
|
||||
null, // shard0 expected refinement info
|
||||
null // shard1 expected refinement info
|
||||
);
|
||||
|
@ -205,7 +208,7 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS {
|
|||
null,
|
||||
null
|
||||
);
|
||||
|
||||
|
||||
// same test, but nested in a terms facet
|
||||
doTestRefine("{top:{type:terms, field:Afield, facet:{x : {type:terms, field:X, limit:2, refine:true} } } }",
|
||||
"{top: {buckets:[{val:'A', count:2, x:{buckets:[{val:x1, count:5},{val:x2, count:3}], more:true} } ] } }",
|
||||
|
@ -310,7 +313,549 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS {
|
|||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMergeWithOverrefine() throws Exception {
|
||||
// overrefine hueristic should use explicit overrequest as default
|
||||
doTestRefine("{x : {type:terms, field:X, limit:1, overrequest:1, sort:'count asc', refine:true} }",
|
||||
//
|
||||
"{x: {buckets:[{val:x1, count:3}, {val:x2, count:5}, {val:x9, count:42}, {val:x0, count:42}], more:true } }",
|
||||
"{x: {buckets:[{val:x2, count:2}, {val:x3, count:4}, {val:x7, count:66}, {val:x8, count:66}], more:true } }",
|
||||
//
|
||||
"=={x:{_l:[x3]}}",
|
||||
"=={x:{_l:[x1]}}");
|
||||
doTestRefine("{x : {type:terms, field:X, limit:1, overrequest:0, sort:'count asc', refine:true} }",
|
||||
//
|
||||
"{x: {buckets:[{val:x1, count:3}, {val:x2, count:5}, {val:x9, count:42}, {val:x0, count:42}], more:true } }",
|
||||
"{x: {buckets:[{val:x2, count:2}, {val:x3, count:4}, {val:x7, count:66}, {val:x8, count:66}], more:true } }",
|
||||
//
|
||||
null,
|
||||
"=={x:{_l:[x1]}}");
|
||||
|
||||
// completely implicit hueristic when no explicit overrequest
|
||||
// limit=1 + 10% + 4 =~ 5 total, but x2 is fully populated so only the other 4 "lowest" should be refined
|
||||
doTestRefine("{x : {type:terms, field:X, limit:1, sort:'count asc', refine:true} }",
|
||||
//
|
||||
"{x: {buckets:[{val:x1, count:3}, {val:x2, count:5}, {val:x9, count:42}, {val:x0, count:42}], more:true } }",
|
||||
"{x: {buckets:[{val:x2, count:2}, {val:x3, count:4}, {val:x7, count:66}, {val:x8, count:66}], more:true } }",
|
||||
//
|
||||
"=={x:{_l:[x3]}}",
|
||||
"=={x:{_l:[x1,x0,x9]}}");
|
||||
|
||||
// when using (default) mincount (or mincount=0) sort="count desc" should eliminate need for overrefine
|
||||
// (regardless of whether any explicit overrequest is specified)
|
||||
for (String extra : Arrays.asList("", ", mincount:0", ", mincount:1",
|
||||
", overrequest:3", ", overrequest:3, mincount:0")) {
|
||||
// w/o any overrefinement, we should only need to backfill x1 & x3 (x2 already fully populated)
|
||||
doTestRefine("{x : {type:terms, field:X, limit:3, sort:'count desc', refine:true"+extra+" } }",
|
||||
//
|
||||
"{x: {buckets:[{val:x1, count:29}, {val:x2, count:15}, {val:x9, count:7}, {val:x0, count:7}], more:true } }",
|
||||
"{x: {buckets:[{val:x2, count:20}, {val:x3, count:12}, {val:x7, count:7}, {val:x8, count:7}], more:true } }",
|
||||
//
|
||||
"=={x:{_l:[x3]}}",
|
||||
"=={x:{_l:[x1]}}");
|
||||
}
|
||||
|
||||
// with 1<mincount, even sort="count desc" should trigger hueristic overrefinement
|
||||
// limit=1 + 10% + 4 =~ 5 total, but x2 is fully populated so only the other 4 "highest" should be refined
|
||||
doTestRefine("{x : {type:terms, field:X, limit:1, mincount:5, sort:'count desc', refine:true } }",
|
||||
//
|
||||
"{x: {buckets:[{val:x1, count:29}, {val:x2, count:15}, {val:x9, count:5}, {val:x0, count:3}], more:true } }",
|
||||
"{x: {buckets:[{val:x2, count:20}, {val:x3, count:12}, {val:x7, count:7}, {val:x8, count:4}], more:true } }",
|
||||
//
|
||||
"=={x:{_l:[x3,x7]}}",
|
||||
"=={x:{_l:[x1,x9]}}");
|
||||
|
||||
// explicit overrefine
|
||||
// limit=1 + overrefine=2 == 3 total, but x2 is fully populated, so only x1 & x3 need refined
|
||||
doTestRefine("{x : {type:terms, field:X, limit:1, overrequest:1, overrefine:2, sort:'count asc', refine:true} }",
|
||||
//
|
||||
"{x: {buckets:[{val:x1, count:3}, {val:x2, count:5}, {val:x9, count:42}, {val:x0, count:42}], more:true } }",
|
||||
"{x: {buckets:[{val:x2, count:2}, {val:x3, count:9}, {val:x7, count:66}, {val:x8, count:66}], more:true } }",
|
||||
//
|
||||
"=={x:{_l:[x3]}}",
|
||||
"=={x:{_l:[x1]}}");
|
||||
|
||||
// explicit overrefine with 0<offset
|
||||
// offset=1 + limit=1 + overrefine=2 == 4 total, but x2 is fully populated, so only x1,x3,x9 need refined
|
||||
doTestRefine("{x : {type:terms, field:X, limit:1, offset:1, overrequest:1, overrefine:2, sort:'count asc', refine:true} }",
|
||||
//
|
||||
"{x: {buckets:[{val:x1, count:3}, {val:x2, count:5}, {val:x9, count:42}, {val:x0, count:43}], more:true } }",
|
||||
"{x: {buckets:[{val:x2, count:2}, {val:x3, count:9}, {val:x7, count:66}, {val:x8, count:67}], more:true } }",
|
||||
//
|
||||
"=={x:{_l:[x3]}}",
|
||||
"=={x:{_l:[x1,x9]}}");
|
||||
|
||||
// hueristic refinement of nested facets
|
||||
// limit=1 + 10% + 4 =~ 5 total (at each level)
|
||||
// -> x2 is fully populated and child buckets are consistent - no refinement needed at all
|
||||
// -> x4 has counts from both shards, but child buckets don't align perfectly
|
||||
//
|
||||
// For (test) simplicity, only x3 and x4 have enough (total) y buckets to prove that the sub-facet
|
||||
// overrefine hueristic is finite...
|
||||
// -> x3 has 6 total sub-facet buckets, only "lowest 5" should be refined on missing shard
|
||||
// -> x4 also has 6 total sub-facet buckets, but only 3 need refined since 2 already fully populated
|
||||
doTestRefine("{x:{type:terms, field:X, limit:1, sort:'count asc', refine:true, " +
|
||||
" facet:{y : {type:terms, field:X, limit:1, sort:'count asc', refine:true} } } }",
|
||||
//
|
||||
"{x: {buckets:[" +
|
||||
" {val:'x1', count:1, y:{buckets:[{val:y11, count:1},{val:y12, count:3}], more:true} }, "+
|
||||
" {val:'x2', count:2, y:{buckets:[{val:y21, count:1},{val:y22, count:3}], more:true} }, "+
|
||||
" {val:'x4', count:3, y:{buckets:[{val:y41, count:1},{val:y4a, count:3}, "+
|
||||
" {val:y42, count:4},{val:y4d, count:5}], more:true} }, "+
|
||||
" {val:'x5', count:4, y:{buckets:[{val:y51, count:1},{val:y52, count:3}], more:true} }, "+
|
||||
" ], more:true } }",
|
||||
"{x: {buckets:[" +
|
||||
" {val:'x3', count:1, y:{buckets:[{val:y31, count:1},{val:y32, count:2}, "+
|
||||
" {val:y33, count:3},{val:y34, count:4}, "+
|
||||
" {val:y35, count:5},{val:y36, count:6}], more:true} }, "+
|
||||
" {val:'x2', count:2, y:{buckets:[{val:y21, count:1},{val:y22, count:3}], more:true} }, "+
|
||||
" {val:'x4', count:3, y:{buckets:[{val:y41, count:1},{val:y4b, count:3}, "+
|
||||
" {val:y42, count:4},{val:y4c, count:9}], more:true} }, "+
|
||||
" {val:'x9', count:9, y:{buckets:[{val:y91, count:1},{val:y92, count:3}], more:true} }, "+
|
||||
" ], more:true } }",
|
||||
//
|
||||
"=={x: {" +
|
||||
" _p:[ ['x3' , {y:{_l:[y31,y32,y33,y34,y35]}} ] ]," +
|
||||
" _s:[ ['x4' , {y:{_l:[y4b]}} ] ]," +
|
||||
" } }",
|
||||
"=={x: {" +
|
||||
" _p:[ ['x1' , {y:{_l:[y11,y12]}} ], " +
|
||||
" ['x5' , {y:{_l:[y51,y52]}} ] ]," +
|
||||
" _s:[ ['x4' , {y:{_l:[y4a,y4d]}} ] ]," +
|
||||
" } }");
|
||||
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSortedFacetRefinementPushingNonRefinedBucketBackIntoTopN() throws Exception {
|
||||
initServers();
|
||||
final Client client = servers.getClient(random().nextInt());
|
||||
client.queryDefaults().set("shards", servers.getShards(), "debugQuery", Boolean.toString(random().nextBoolean()));
|
||||
|
||||
List<SolrClient> clients = client.getClientProvider().all();
|
||||
assertTrue(clients.size() >= 3); // we only use 2, but assert 3 to also test empty shard
|
||||
final SolrClient c0 = clients.get(0);
|
||||
final SolrClient c1 = clients.get(1);
|
||||
|
||||
client.deleteByQuery("*:*", null);
|
||||
int id = 0;
|
||||
|
||||
// all_ss is only used for sub-faceting...
|
||||
// every doc will be in all_ss:z_all, (most c1 docs will be in all_ss:some
|
||||
// (with index order tie breaker, c1 should return "some" when limit:1
|
||||
// but "z_all" should have a higher count from c0)
|
||||
|
||||
// client 0 // shard1: A=1,B=1,C=2 ...
|
||||
c0.add(sdoc("id", id++, "cat_s","A", "price_i","1", "all_ss","z_all"));
|
||||
c0.add(sdoc("id", id++, "cat_s","B", "price_i","1", "all_ss","z_all"));
|
||||
c0.add(sdoc("id", id++, "cat_s","C", "price_i","1", "all_ss","z_all"));
|
||||
c0.add(sdoc("id", id++, "cat_s","C", "price_i","1", "all_ss","z_all"));
|
||||
// ... X=3,Y=3
|
||||
c0.add(sdoc("id", id++, "cat_s","X", "price_i","1", "all_ss","z_all"));
|
||||
c0.add(sdoc("id", id++, "cat_s","X", "price_i","1", "all_ss","z_all"));
|
||||
c0.add(sdoc("id", id++, "cat_s","X", "price_i","1", "all_ss","z_all"));
|
||||
c0.add(sdoc("id", id++, "cat_s","Y", "price_i","1", "all_ss","z_all"));
|
||||
c0.add(sdoc("id", id++, "cat_s","Y", "price_i","1", "all_ss","z_all"));
|
||||
c0.add(sdoc("id", id++, "cat_s","Y", "price_i","1", "all_ss","z_all"));
|
||||
|
||||
// client 1 // shard2: X=1,Y=2,Z=2 ...
|
||||
c1.add(sdoc("id", id++, "cat_s","X", "price_i","1", "all_ss","z_all","all_ss","some"));
|
||||
c1.add(sdoc("id", id++, "cat_s","Y", "price_i","1", "all_ss","z_all","all_ss","some"));
|
||||
c1.add(sdoc("id", id++, "cat_s","Y", "price_i","1", "all_ss","z_all","all_ss","some"));
|
||||
c1.add(sdoc("id", id++, "cat_s","Z", "price_i","1", "all_ss","z_all","all_ss","some"));
|
||||
c1.add(sdoc("id", id++, "cat_s","Z", "price_i","1", "all_ss","z_all","all_ss","some"));
|
||||
// ... C=4
|
||||
c1.add(sdoc("id", id++, "cat_s","C", "price_i","1", "all_ss","z_all","all_ss","some"));
|
||||
c1.add(sdoc("id", id++, "cat_s","C", "price_i","1", "all_ss","z_all","all_ss","some"));
|
||||
c1.add(sdoc("id", id++, "cat_s","C", "price_i","1", "all_ss","z_all","all_ss","some"));
|
||||
c1.add(sdoc("id", id++, "cat_s","C", "price_i","1", "all_ss","z_all","all_ss","some"));
|
||||
|
||||
// the amount of overrequest shouldn't matter for demonstrating the issue...
|
||||
// it only changes how many C_fillerN & Z_fillerN terms are needed on each shard
|
||||
final int overreq = TestUtil.nextInt(random(),0,20);
|
||||
|
||||
// for overreq=n: C_n:(x2 on client0 + x4 on client1); Z_n:(x2 on client1)
|
||||
for (int i = 0; i < overreq; i++) {
|
||||
for (int t = 0; t < 2; t++) {
|
||||
c0.add(sdoc("id", id++, "cat_s","C_filler"+i, "price_i","1", "all_ss","z_all"));
|
||||
c1.add(sdoc("id", id++, "cat_s","Z_filler"+i, "price_i","1", "all_ss","z_all","all_ss","some"));
|
||||
}
|
||||
for (int t = 0; t < 4; t++) {
|
||||
c1.add(sdoc("id", id++, "cat_s","C_filler"+i, "price_i","1", "all_ss","z_all","all_ss","some"));
|
||||
// extra c0 docs that don't contribute to the cat_s facet,...
|
||||
// just so "z_all" will win overall on parent facet
|
||||
c0.add(sdoc("id", id++, "all_ss","z_all"));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// Whole Collection: A=1,B=1,Z=2,X=4,Y=5,C=6
|
||||
client.commit();
|
||||
|
||||
// In an ideal world, 'Z:2' would be returned as the 3rd value,
|
||||
// but neither C or Z make the topN cut in phase#1, so only A,B,X get refined.
|
||||
// After refinement, X's increased values should *NOT* push it out of the (original) topN
|
||||
// to let "C" bubble back up into the topN, with incomplete/inaccurate count/stats
|
||||
// (NOTE: hueristic for num buckets refined is based on 'overrequest' unless explicit 'overrefine')
|
||||
client.testJQ(params("q", "*:*", "rows", "0", "json.facet", "{"
|
||||
+ " cat_count:{ type:terms, field:cat_s, limit:3, overrequest:"+overreq
|
||||
+ " , refine:true, sort:'count asc' },"
|
||||
+ " cat_price:{ type:terms, field:cat_s, limit:3, overrequest:"+overreq
|
||||
+ " , refine:true, sort:'sum_p asc' "
|
||||
+ " , facet: { sum_p: 'sum(price_i)' } }"
|
||||
+ "}")
|
||||
, "facets=={ count: "+id+","
|
||||
+ " cat_count:{ buckets:[ "
|
||||
+ " {val:A,count:1},"
|
||||
+ " {val:B,count:1},"
|
||||
+ " {val:X,count:4},"
|
||||
+ " ] },"
|
||||
+ " cat_price:{ buckets:[ "
|
||||
+ " {val:A,count:1,sum_p:1.0},"
|
||||
+ " {val:B,count:1,sum_p:1.0},"
|
||||
+ " {val:X,count:4,sum_p:4.0},"
|
||||
+ " ] }"
|
||||
+ "}"
|
||||
);
|
||||
|
||||
// if we do the same query but explicitly request enough overrefinement to get past the filler
|
||||
// terms, we should get accurate counts for (C and) Z which should push X out
|
||||
client.testJQ(params("q", "*:*", "rows", "0", "json.facet", "{"
|
||||
+ " cat_count:{ type:terms, field:cat_s, limit:3, overrequest:"+overreq
|
||||
+ " , overrefine:"+((1+overreq)*3)+", refine:true, sort:'count asc' },"
|
||||
+ " cat_price:{ type:terms, field:cat_s, limit:3, overrequest:"+overreq
|
||||
+ " , overrefine:"+((1+overreq)*3)+", refine:true, sort:'sum_p asc' "
|
||||
+ " , facet: { sum_p: 'sum(price_i)' } }"
|
||||
+ "}")
|
||||
, "facets=={ count: "+id+","
|
||||
+ " cat_count:{ buckets:[ "
|
||||
+ " {val:A,count:1},"
|
||||
+ " {val:B,count:1},"
|
||||
+ " {val:Z,count:2},"
|
||||
+ " ] },"
|
||||
+ " cat_price:{ buckets:[ "
|
||||
+ " {val:A,count:1,sum_p:1.0},"
|
||||
+ " {val:B,count:1,sum_p:1.0},"
|
||||
+ " {val:Z,count:2,sum_p:2.0},"
|
||||
+ " ] }"
|
||||
+ "}"
|
||||
);
|
||||
|
||||
// if we use mincount=2, such that A & B get filtered out, then we should have buckets.size() < limit
|
||||
// rather then buckets w/inaccurate counts/stats.
|
||||
// (explicitly disabling overrefine & overrequest to prevent filler terms)
|
||||
client.testJQ(params("q", "*:*", "rows", "0", "json.facet", "{"
|
||||
+ " cat_count:{ type:terms, field:cat_s, limit:3, overrequest: 0, overrefine: 0"
|
||||
+ " , mincount: 2, refine:true, sort:'count asc' },"
|
||||
+ " cat_price:{ type:terms, field:cat_s, limit:3, overrequest: 0, overrefine: 0"
|
||||
+ " , mincount: 2, refine:true, sort:'sum_p asc' "
|
||||
+ " , facet: { sum_p: 'sum(price_i)' } }"
|
||||
+ "}")
|
||||
, "facets=={ count: "+id+","
|
||||
+ " cat_count:{ buckets:[ "
|
||||
+ " {val:X,count:4},"
|
||||
+ " ] },"
|
||||
+ " cat_price:{ buckets:[ "
|
||||
+ " {val:X,count:4,sum_p:4.0},"
|
||||
+ " ] }"
|
||||
+ "}"
|
||||
);
|
||||
|
||||
// When our 'cat_s' facets are nested under an 'all_ss' facet, we should likewise not get
|
||||
// any (sub) buckets with incomplete/inaccurate counts
|
||||
//
|
||||
// NOTE: parent facet limit is 1, testing with various top level overrequest/refine params to see
|
||||
// how different refinement code paths of parent effect the child refinement
|
||||
for (String top_refine : Arrays.asList("true", "false")) {
|
||||
// if our top level facet does *NO* overrequesting, then our shard1 will return "some" as it's
|
||||
// (only) top term, which will lose to "z_all" from shard0, and the (single pass) refinement
|
||||
// logic will have no choice but to choose & refine the child facet terms from shard0: A,B,C
|
||||
client.testJQ(params("q", "*:*", "rows", "0", "json.facet", "{"
|
||||
+ " all:{ type:terms, field:all_ss, limit:1, refine:"+top_refine
|
||||
+ ", overrequest:0"
|
||||
+ " , facet:{"
|
||||
+ " cat_count:{ type:terms, field:cat_s, limit:3, overrequest:"+overreq
|
||||
+ " , refine:true, sort:'count asc' },"
|
||||
+ " cat_price:{ type:terms, field:cat_s, limit:3, overrequest:"+overreq
|
||||
+ " , refine:true, sort:'sum_p asc' "
|
||||
+ " , facet: { sum_p: 'sum(price_i)' } }"
|
||||
+ "} } }")
|
||||
, "facets=={ count: "+id+","
|
||||
+ "all:{ buckets:[ "
|
||||
+ " { val:z_all, count: "+id+","
|
||||
+ " cat_count:{ buckets:[ "
|
||||
+ " {val:A,count:1},"
|
||||
+ " {val:B,count:1},"
|
||||
+ " {val:C,count:6},"
|
||||
+ " ] },"
|
||||
+ " cat_price:{ buckets:[ "
|
||||
+ " {val:A,count:1,sum_p:1.0},"
|
||||
+ " {val:B,count:1,sum_p:1.0},"
|
||||
+ " {val:C,count:6,sum_p:6.0},"
|
||||
+ " ] }"
|
||||
+ "} ] } }"
|
||||
);
|
||||
|
||||
// With any overrequest param > 0 on the parent facet, both shards will return "z_all" as a
|
||||
// viable candidate and the merge logic should recoginize that X is a better choice,
|
||||
// even though the (single shard) stats for "C" will be lower
|
||||
final int top_over = TestUtil.nextInt(random(), 1, 999);
|
||||
client.testJQ(params("q", "*:*", "rows", "0", "json.facet", "{"
|
||||
+ " all:{ type:terms, field:all_ss, limit:1, refine:"+top_refine
|
||||
+ ", overrequest:" + top_over
|
||||
+ " , facet:{"
|
||||
+ " cat_count:{ type:terms, field:cat_s, limit:3, overrequest:"+overreq
|
||||
+ " , refine:true, sort:'count asc' },"
|
||||
+ " cat_price:{ type:terms, field:cat_s, limit:3, overrequest:"+overreq
|
||||
+ " , refine:true, sort:'sum_p asc' "
|
||||
+ " , facet: { sum_p: 'sum(price_i)' } }"
|
||||
+ "} } }")
|
||||
, "facets=={ count: "+id+","
|
||||
+ "all:{ buckets:[ "
|
||||
+ " { val:z_all, count: "+id+","
|
||||
+ " cat_count:{ buckets:[ "
|
||||
+ " {val:A,count:1},"
|
||||
+ " {val:B,count:1},"
|
||||
+ " {val:X,count:4},"
|
||||
+ " ] },"
|
||||
+ " cat_price:{ buckets:[ "
|
||||
+ " {val:A,count:1,sum_p:1.0},"
|
||||
+ " {val:B,count:1,sum_p:1.0},"
|
||||
+ " {val:X,count:4,sum_p:4.0},"
|
||||
+ " ] }"
|
||||
+ "} ] } }"
|
||||
);
|
||||
|
||||
// if we do the same query but explicitly request enough overrefinement on the child facet
|
||||
// to get past the filler terms, we should get accurate counts for (C and) Z which should push X out
|
||||
client.testJQ(params("q", "*:*", "rows", "0", "json.facet", "{"
|
||||
+ " all:{ type:terms, field:all_ss, limit:1, refine:"+top_refine
|
||||
+ ", overrequest:" + top_over
|
||||
+ " , facet:{"
|
||||
+ " cat_count:{ type:terms, field:cat_s, limit:3, overrequest:"+((1+overreq)*3)
|
||||
+ " , refine:true, sort:'count asc' },"
|
||||
+ " cat_price:{ type:terms, field:cat_s, limit:3, overrequest:"+((1+overreq)*3)
|
||||
+ " , refine:true, sort:'sum_p asc' "
|
||||
+ " , facet: { sum_p: 'sum(price_i)' } }"
|
||||
+ "} } }")
|
||||
, "facets=={ count: "+id+","
|
||||
+ "all:{ buckets:[ "
|
||||
+ " { val:z_all, count: "+id+","
|
||||
+ " cat_count:{ buckets:[ "
|
||||
+ " {val:A,count:1},"
|
||||
+ " {val:B,count:1},"
|
||||
+ " {val:Z,count:2},"
|
||||
+ " ] },"
|
||||
+ " cat_price:{ buckets:[ "
|
||||
+ " {val:A,count:1,sum_p:1.0},"
|
||||
+ " {val:B,count:1,sum_p:1.0},"
|
||||
+ " {val:Z,count:2,sum_p:2.0},"
|
||||
+ " ] }"
|
||||
+ "} ] } }"
|
||||
);
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-12556")
|
||||
@Test
|
||||
public void testProcessEmptyRefinement() throws Exception {
|
||||
initServers();
|
||||
final Client client = servers.getClient(random().nextInt());
|
||||
client.queryDefaults().set("shards", servers.getShards(), "debugQuery", Boolean.toString(random().nextBoolean()));
|
||||
|
||||
List<SolrClient> clients = client.getClientProvider().all();
|
||||
assertTrue(clients.size() >= 3); // we only use 2, but assert at least 3 to also test empty shard
|
||||
final SolrClient c0 = clients.get(0);
|
||||
final SolrClient c1 = clients.get(1);
|
||||
|
||||
client.deleteByQuery("*:*", null);
|
||||
int id = 0;
|
||||
|
||||
c0.add(sdoc("id", id++, "cat_s", "Ax"));
|
||||
c0.add(sdoc("id", id++, "cat_s", "Bx"));
|
||||
c0.add(sdoc("id", id++, "cat_s", "Cx"));
|
||||
|
||||
c1.add(sdoc("id", id++, "cat_s", "Ay"));
|
||||
c1.add(sdoc("id", id++, "cat_s", "By"));
|
||||
c1.add(sdoc("id", id++, "cat_s", "Cy"));
|
||||
c1.add(sdoc("id", id++, "cat_s", "Dy"));
|
||||
|
||||
client.commit();
|
||||
|
||||
// regardless of how much overrequest there is, in phase#1
|
||||
// all terms will tie on the sort criteria, and "Ax" should win the tiebreaker.
|
||||
//
|
||||
// When Ax is refined against c1, it's 'debug' sort value will increase, but regardless
|
||||
// of the value of processEmpty, no other term should be returned in it's place
|
||||
// (because if they are also correctly refined, then their 'debug' sort values will also increase
|
||||
// and Ax will stll win the tie breaker -- and if they are not refined they shouldn't be returned)
|
||||
for (int overrequest = 0; overrequest < 5; overrequest++) {
|
||||
for (boolean pe : Arrays.asList(false, true)) {
|
||||
ModifiableSolrParams p
|
||||
= params("q", "*:*", "rows", "0", "json.facet"
|
||||
, "{"
|
||||
+ " top:{ type:terms, field:cat_s, limit:1, overrequest:"+overrequest+", "
|
||||
+ " refine:true, sort: 'debug asc', "
|
||||
+ " facet:{ debug:'debug(numShards)', processEmpty:"+pe+" } } }");
|
||||
try {
|
||||
client.testJQ(p
|
||||
, "facets=={ count: "+id+","
|
||||
+ " top:{ buckets:[ "
|
||||
+ " { val:Ax, count: 1, "
|
||||
+ " debug:"+(pe ? 2 : 1)
|
||||
+ " }"
|
||||
+ " ] } }"
|
||||
);
|
||||
} catch (AssertionError | RuntimeException e) {
|
||||
throw new AssertionError(p + " --> " + e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Helper method used by multiple tests to look at same data diff ways */
|
||||
private int initSomeDocsWhere1ShardHasOnlyParentFacetField() throws Exception {
|
||||
initServers();
|
||||
final Client client = servers.getClient(random().nextInt());
|
||||
client.queryDefaults().set("shards", servers.getShards(), "debugQuery", Boolean.toString(random().nextBoolean()));
|
||||
|
||||
final List<SolrClient> clients = client.getClientProvider().all();
|
||||
assertTrue(clients.size() >= 2);
|
||||
final SolrClient c0 = clients.get(0);
|
||||
final SolrClient c1 = clients.get(1);
|
||||
|
||||
client.deleteByQuery("*:*", null);
|
||||
int id = 0;
|
||||
|
||||
// client 0 // shard1
|
||||
// only terms pX & pY (with high counts) from the parent_s facet, no child_s values
|
||||
for (int i = 0; i < 10; i++) {
|
||||
c0.add(sdoc("id", id++, "parent_s", "pX"));
|
||||
for (int j =0; j < 2; j++) {
|
||||
c0.add(sdoc("id", id++, "parent_s", "pY"));
|
||||
}
|
||||
}
|
||||
|
||||
// client 1 // shard2
|
||||
// some docs with pX & pY, but on this shard, pA & pB have higher counts
|
||||
// (but not as high as pX/py on shard1)
|
||||
// all docs on this shard also have values in child_s
|
||||
for (int i = 0; i < 2; i++) {
|
||||
for (int j = 0; j < 3 ; j++) {
|
||||
c1.add(sdoc("id", id++, "parent_s", "pA", "child_s", "c"+i));
|
||||
c1.add(sdoc("id", id++, "parent_s", "pB", "child_s", "c"+i));
|
||||
}
|
||||
c1.add(sdoc("id", id++, "parent_s", "pX", "child_s", "c"+i));
|
||||
c1.add(sdoc("id", id++, "parent_s", "pY", "child_s", "c"+i));
|
||||
}
|
||||
c1.add(sdoc("id", id++, "parent_s", "pX", "child_s", "c0"));
|
||||
c1.add(sdoc("id", id++, "parent_s", "pY", "child_s", "c1"));
|
||||
c1.add(sdoc("id", id++, "parent_s", "pY", "child_s", "c1"));
|
||||
|
||||
client.commit();
|
||||
return id;
|
||||
}
|
||||
|
||||
/** @see #testSortedSubFacetRefinementWhenParentOnlyReturnedByOneShardProcessEmpty */
|
||||
@Test
|
||||
public void testSortedSubFacetRefinementWhenParentOnlyReturnedByOneShard() throws Exception {
|
||||
final int numDocs = initSomeDocsWhere1ShardHasOnlyParentFacetField();
|
||||
final Client client = servers.getClient(random().nextInt());
|
||||
final List<SolrClient> clients = client.getClientProvider().all();
|
||||
|
||||
assertTrue(clients.size() >= 3); // we only use 2, but assert at least 3 to also test empty shard
|
||||
final SolrClient c0 = clients.get(0);
|
||||
final SolrClient c1 = clients.get(1);
|
||||
|
||||
// during the initial request...
|
||||
// - shard1 should return "high" count pX & pY w/o any child buckets (no "more" child)
|
||||
// - shard2 should return "lower" count pA & pB w/some child buckets
|
||||
// - any other shards should indicate they have no parent buckets (no "more" parent)
|
||||
// during refinement:
|
||||
// - shard2 should be asked to backfill any known children of pX&pY
|
||||
// - these children from shard2 will be the only (possibly) contributors to the child buckets
|
||||
//
|
||||
// - the numShards for all parent buckets should be 2, but for the child buckets it should be 1
|
||||
client.testJQ(params("q", "*:*", "rows", "0", "json.facet", "{"
|
||||
+ "parent:{ type:terms, field:parent_s, limit:2, overrequest:0, refine:true, facet:{"
|
||||
+ " debug:'debug(numShards)',"
|
||||
+ " child:{ type:terms, field:child_s, limit:2, overrequest:0, refine: true,"
|
||||
+ " facet:{ debug:'debug(numShards)' } }"
|
||||
+ "} } }")
|
||||
, "facets=={ count: "+numDocs+","
|
||||
+ " parent:{ buckets:[ "
|
||||
+ " { val:pY, count: 24,"
|
||||
+ " debug:2, "
|
||||
+ " child:{ buckets:[ "
|
||||
+ " {val:c1,count:3, debug:1},"
|
||||
+ " {val:c0,count:1, debug:1},"
|
||||
+ " ] } },"
|
||||
+ " { val:pX, count: 13,"
|
||||
+ " debug:2, "
|
||||
+ " child:{ buckets:[ "
|
||||
+ " {val:c0,count:2, debug:1},"
|
||||
+ " {val:c1,count:1, debug:1},"
|
||||
+ " ] } },"
|
||||
+ " ] } }"
|
||||
);
|
||||
}
|
||||
|
||||
/** @see #testSortedSubFacetRefinementWhenParentOnlyReturnedByOneShard */
|
||||
@AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-12556")
|
||||
@Test
|
||||
public void testSortedSubFacetRefinementWhenParentOnlyReturnedByOneShardProcessEmpty() throws Exception {
|
||||
final int numDocs = initSomeDocsWhere1ShardHasOnlyParentFacetField();
|
||||
final Client client = servers.getClient(random().nextInt());
|
||||
final List<SolrClient> clients = client.getClientProvider().all();
|
||||
final int numClients = clients.size();
|
||||
|
||||
assertTrue(numClients >= 3); // we only use 2, but assert at least 3 to also test empty shard
|
||||
final SolrClient c0 = clients.get(0);
|
||||
final SolrClient c1 = clients.get(1);
|
||||
|
||||
// if we do the same request as testSortedSubFacetRefinementWhenParentOnlyReturnedByOneShard,
|
||||
// but with processEmpty:true, then ideally we should get the same buckets & counts as before,
|
||||
// but the debug info should indicate that every shard contributed to every bucket (either initially,
|
||||
// or during refinement)
|
||||
//
|
||||
// The problem comes in with how "empty" bucket lists are dealt with...
|
||||
// - child debug counts never get higher then '2' because even with the forced "_l" refinement of
|
||||
// the parent buckets against the "empty" shards we don't explicitly ask those shards to
|
||||
// evaluate the child buckets
|
||||
// - perhaps we should reconsider the value of "_l" ?
|
||||
// - why aren't we just specifying all the buckets (and child buckets) chosen in phase#1 using "_p" ?
|
||||
// - or at the very least, if the purpose of "_l" is to give other buckets a chance to "bubble up"
|
||||
// in phase#2, then shouldn't a "_l" refinement requests still include the buckets choosen in
|
||||
// phase#1, and request that the shard fill them in in addition to returning it's own top buckets?
|
||||
client.testJQ(params("q", "*:*", "rows", "0", "json.facet", "{"
|
||||
+ "processEmpty:true,"
|
||||
+ "parent:{ type:terms, field:parent_s, limit:2, overrequest:0, refine:true, facet:{"
|
||||
+ " processEmpty:true,"
|
||||
+ " debug:'debug(numShards)',"
|
||||
+ " child:{ type:terms, field:child_s, limit:2, overrequest:0, refine: true,"
|
||||
+ " facet:{ processEmpty:true, debug:'debug(numShards)' } }"
|
||||
+ "} } }")
|
||||
, "facets=={ count: "+numDocs+","
|
||||
+ " parent:{ buckets:[ "
|
||||
+ " { val:pY, count: 24,"
|
||||
+ " debug:"+numClients+", "
|
||||
+ " child:{ buckets:[ "
|
||||
+ " {val:c1,count:3, debug:"+numClients+"},"
|
||||
+ " {val:c0,count:1, debug:"+numClients+"},"
|
||||
+ " ] } },"
|
||||
+ " { val:pX, count: 13,"
|
||||
+ " debug:"+numClients+", "
|
||||
+ " child:{ buckets:[ "
|
||||
+ " {val:c0,count:2, debug:"+numClients+"},"
|
||||
+ " {val:c1,count:1, debug:"+numClients+"},"
|
||||
+ " ] } },"
|
||||
+ " ] } }"
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testBasicRefinement() throws Exception {
|
||||
ModifiableSolrParams p;
|
||||
|
|
|
@ -160,16 +160,27 @@ json.facet={
|
|||
|
||||
// TODO: This table has cells that won't work with PDF: https://github.com/ctargett/refguide-asciidoc-poc/issues/13
|
||||
|
||||
[width="100%",cols="10%,90%",options="header",]
|
||||
[width="100%",cols="20%,90%",options="header",]
|
||||
|===
|
||||
|Parameter |Description
|
||||
|field |The field name to facet over.
|
||||
|offset |Used for paging, this skips the first N buckets. Defaults to 0.
|
||||
|limit |Limits the number of buckets returned. Defaults to 10.
|
||||
|refine |If true, turns on distributed facet refining. This uses a second phase to retrieve selected stats from shards so that every shard contributes to every returned bucket in this facet and any sub-facets. This makes stats for returned buckets exact.
|
||||
|overrequest |Number of buckets beyond the limit to request internally during distributed search. -1 means default.
|
||||
|mincount |Only return buckets with a count of at least this number. Defaults to 1.
|
||||
|sort |Specifies how to sort the buckets produced. “count” specifies document count, “index” sorts by the index (natural) order of the bucket value. One can also sort by any <<json-facet-api.adoc#aggregation-functions,facet function / statistic>> that occurs in the bucket. The default is “count desc”. This parameter may also be specified in JSON like `sort:{count:desc}`. The sort order may either be “asc” or “desc”
|
||||
|overrequest a|
|
||||
Number of buckets beyond the `limit` to internally request from shards during a distributed search.
|
||||
|
||||
Larger values can increase the accuracy of the final "Top Terms" returned when the individual shards have very diff top terms.
|
||||
|
||||
The default of `-1` causes a hueristic to be applied based on the other options specified.
|
||||
|refine |If `true`, turns on distributed facet refining. This uses a second phase to retrieve any buckets needed for the final result from shards that did not include those buckets in their initial internal results, so that every shard contributes to every returned bucket in this facet and any sub-facets. This makes counts & stats for returned buckets exact.
|
||||
|overrefine a|
|
||||
Number of buckets beyond the `limit` to consider internally during a distributed search when determining which buckets to refine.
|
||||
|
||||
Larger values can increase the accuracy of the final "Top Terms" returned when the individual shards have very diff top terms, and the current `sort` option can result in refinement pushing terms lower down the sorted list (ex: `sort:"count asc"`)
|
||||
|
||||
The default of `-1` causes a hueristic to be applied based on other options specified.
|
||||
|mincount |Only return buckets with a count of at least this number. Defaults to 1.
|
||||
|missing |A boolean that specifies if a special “missing” bucket should be returned that is defined by documents without a value in the field. Defaults to false.
|
||||
|numBuckets |A boolean. If true, adds “numBuckets” to the response, an integer representing the number of buckets for the facet (as opposed to the number of buckets returned). Defaults to false.
|
||||
|allBuckets |A boolean. If true, adds an “allBuckets” bucket to the response, representing the union of all of the buckets. For multi-valued fields, this is different than a bucket for all of the documents in the domain since a single document can belong to multiple buckets. Defaults to false.
|
||||
|
|
Loading…
Reference in New Issue