mirror of https://github.com/apache/lucene.git
SOLR-14467: Fix relatedness() stat in json.facets to no longer cause server errors (or nonsense results) when combined with allBuckets:true
This commit is contained in:
parent
0c4d8fb116
commit
b055c7489f
solr
CHANGES.txt
core/src
java/org/apache/solr/search/facet
test/org/apache/solr/search/facet
solr-ref-guide/src
|
@ -215,6 +215,10 @@ Bug Fixes
|
|||
* SOLR-14520: Fixed server errors from the json.facet allBuckets:true option when combined with refine:true
|
||||
(Michael Gibney, hossman)
|
||||
|
||||
* SOLR-14467: Fix relatedness() stat in json.facets to no longer cause server errors (or nonsense results)
|
||||
when combined with allBuckets:true. (Michael Gibney, hossman)
|
||||
|
||||
|
||||
Other Changes
|
||||
---------------------
|
||||
* SOLR-14197: SolrResourceLoader: marked many methods as deprecated, and in some cases rerouted exiting logic to avoid
|
||||
|
|
|
@ -106,7 +106,10 @@ public class FacetBucket {
|
|||
if (subs != null) {
|
||||
for (Map.Entry<String,FacetMerger> mergerEntry : subs.entrySet()) {
|
||||
FacetMerger subMerger = mergerEntry.getValue();
|
||||
out.add(mergerEntry.getKey(), subMerger.getMergedResult());
|
||||
Object mergedResult = subMerger.getMergedResult();
|
||||
if (null != mergedResult) {
|
||||
out.add(mergerEntry.getKey(), mergedResult);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -743,6 +743,22 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
|
|||
}
|
||||
}
|
||||
|
||||
private static final SlotContext ALL_BUCKETS_SLOT_CONTEXT = new SlotContext(null) {
|
||||
@Override
|
||||
public Query getSlotQuery() {
|
||||
throw new IllegalStateException("getSlotQuery() is mutually exclusive with isAllBuckets==true");
|
||||
}
|
||||
@Override
|
||||
public boolean isAllBuckets() {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
private static final IntFunction<SlotContext> ALL_BUCKETS_SLOT_FUNCTION = new IntFunction<SlotContext>() {
|
||||
@Override
|
||||
public SlotContext apply(int value) {
|
||||
return ALL_BUCKETS_SLOT_CONTEXT;
|
||||
}
|
||||
};
|
||||
|
||||
static class SpecialSlotAcc extends SlotAcc {
|
||||
SlotAcc collectAcc;
|
||||
|
@ -771,11 +787,11 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
|
|||
assert slot != collectAccSlot || slot < 0;
|
||||
count++;
|
||||
if (collectAcc != null) {
|
||||
collectAcc.collect(doc, collectAccSlot, slotContext);
|
||||
collectAcc.collect(doc, collectAccSlot, ALL_BUCKETS_SLOT_FUNCTION);
|
||||
}
|
||||
if (otherAccs != null) {
|
||||
for (SlotAcc otherAcc : otherAccs) {
|
||||
otherAcc.collect(doc, otherAccsSlot, slotContext);
|
||||
otherAcc.collect(doc, otherAccsSlot, ALL_BUCKETS_SLOT_FUNCTION);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.ArrayList;
|
|||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.Map;
|
||||
import java.util.function.IntFunction;
|
||||
|
||||
|
@ -36,7 +37,6 @@ import org.apache.solr.common.util.NamedList;
|
|||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
import org.apache.solr.search.DocSet;
|
||||
import org.apache.solr.search.QParser;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -163,6 +163,8 @@ public class RelatednessAgg extends AggValueSource {
|
|||
return new Merger(this);
|
||||
}
|
||||
|
||||
private static final String IMPLIED_KEY = "implied";
|
||||
|
||||
private static final class SKGSlotAcc extends SlotAcc {
|
||||
private final RelatednessAgg agg;
|
||||
private BucketData[] slotvalues;
|
||||
|
@ -186,7 +188,22 @@ public class RelatednessAgg extends AggValueSource {
|
|||
|
||||
assert null != slotContext;
|
||||
|
||||
Query slotQ = slotContext.apply(slot).getSlotQuery();
|
||||
final BucketData slotVal = new BucketData(agg);
|
||||
slotvalues[slot] = slotVal;
|
||||
|
||||
final SlotContext ctx = slotContext.apply(slot);
|
||||
if (ctx.isAllBuckets()) {
|
||||
// relatedness is meaningless for allBuckets (see SOLR-14467)
|
||||
// our current (implied & empty) BucketData is all we need
|
||||
//
|
||||
// NOTE: it might be temping to use 'slotvalues[slot] = null' in this case
|
||||
// since getValue() will also ultimately generate an implied bucket in that case,
|
||||
// but by using a non-null bucket we let collect(int,...) know it doesn't need to keep calling
|
||||
// processSlot over and over.
|
||||
return;
|
||||
}
|
||||
|
||||
Query slotQ = ctx.getSlotQuery();
|
||||
if (null == slotQ) {
|
||||
// extremeley special edge case...
|
||||
// the only way this should be possible is if our relatedness() function is used as a "top level"
|
||||
|
@ -198,11 +215,9 @@ public class RelatednessAgg extends AggValueSource {
|
|||
// ...and in which case we should just use the current base
|
||||
final DocSet slotSet = null == slotQ ? fcontext.base : fcontext.searcher.getDocSet(slotQ);
|
||||
|
||||
final BucketData slotVal = new BucketData(agg);
|
||||
slotVal.incSizes(fgSize, bgSize);
|
||||
slotVal.incCounts(fgSet.intersectionSize(slotSet),
|
||||
bgSet.intersectionSize(slotSet));
|
||||
slotvalues[slot] = slotVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -247,7 +262,7 @@ public class RelatednessAgg extends AggValueSource {
|
|||
public Object getValue(int slotNum) {
|
||||
BucketData slotVal = slotvalues[slotNum];
|
||||
if (null == slotVal) {
|
||||
// since we haven't been told about any docs for this slot, use a slot w/no counts,
|
||||
// since we haven't collected any docs for this slot, use am (implied) slot w/no counts,
|
||||
// just the known fg/bg sizes. (this is most likely a refinement request for a bucket we dont have)
|
||||
slotVal = new BucketData(agg);
|
||||
slotVal.incSizes(fgSize, bgSize);
|
||||
|
@ -280,12 +295,22 @@ public class RelatednessAgg extends AggValueSource {
|
|||
* @see SKGSlotAcc
|
||||
* @see Merger
|
||||
*/
|
||||
private static final class BucketData implements Comparable<BucketData> {
|
||||
private static class BucketData implements Comparable<BucketData> {
|
||||
private RelatednessAgg agg;
|
||||
private long fg_size = 0;
|
||||
private long bg_size = 0;
|
||||
private long fg_count = 0;
|
||||
private long bg_count = 0;
|
||||
|
||||
/**
|
||||
* Buckets are implied until/unless counts are explicitly incremented (even if those counts are 0)
|
||||
* An implied bucket means we have no real data for it -- it may be useful for a per-Shard request
|
||||
* to return "size" info of a bucket that doesn't exist on the current shard, or it may represent
|
||||
* the <code>allBuckets</code> bucket.
|
||||
*
|
||||
* @see #incCounts
|
||||
*/
|
||||
private boolean implied;
|
||||
|
||||
/**
|
||||
* NaN indicates that <b>all</a> derived values need (re)-computed
|
||||
|
@ -306,6 +331,7 @@ public class RelatednessAgg extends AggValueSource {
|
|||
|
||||
public BucketData(final RelatednessAgg agg) {
|
||||
this.agg = agg;
|
||||
this.implied = true;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -313,9 +339,10 @@ public class RelatednessAgg extends AggValueSource {
|
|||
* derived values that may be cached
|
||||
*/
|
||||
public void incCounts(final long fgInc, final long bgInc) {
|
||||
this.relatedness = Double.NaN;
|
||||
fg_count += fgInc;
|
||||
bg_count += bgInc;
|
||||
this.implied = false;
|
||||
this.relatedness = Double.NaN;
|
||||
fg_count += fgInc;
|
||||
bg_count += bgInc;
|
||||
}
|
||||
/**
|
||||
* Increment both the foreground & background <em>sizes</em> for the current bucket, reseting any
|
||||
|
@ -329,7 +356,7 @@ public class RelatednessAgg extends AggValueSource {
|
|||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(this.getClass(), fg_count, bg_count, fg_size, bg_size, agg);
|
||||
return Objects.hash(this.getClass(), implied, fg_count, bg_count, fg_size, bg_size, agg);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -339,7 +366,8 @@ public class RelatednessAgg extends AggValueSource {
|
|||
}
|
||||
BucketData that = (BucketData)other;
|
||||
// we will most certainly be compared to other buckets of the same Agg instance, so compare counts first
|
||||
return Objects.equals(this.fg_count, that.fg_count)
|
||||
return Objects.equals(this.implied, that.implied)
|
||||
&& Objects.equals(this.fg_count, that.fg_count)
|
||||
&& Objects.equals(this.bg_count, that.bg_count)
|
||||
&& Objects.equals(this.fg_size, that.fg_size)
|
||||
&& Objects.equals(this.bg_size, that.bg_size)
|
||||
|
@ -408,16 +436,36 @@ public class RelatednessAgg extends AggValueSource {
|
|||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
public SimpleOrderedMap externalize(final boolean isShardRequest) {
|
||||
SimpleOrderedMap result = new SimpleOrderedMap<Number>();
|
||||
|
||||
// if counts are non-zero, then this bucket must not be implied
|
||||
assert 0 == fg_count || ! implied : "Implied bucket has non-zero fg_count";
|
||||
assert 0 == bg_count || ! implied : "Implied bucket has non-zero bg_count";
|
||||
|
||||
if (isShardRequest) {
|
||||
result.add(FG_COUNT, fg_count);
|
||||
result.add(BG_COUNT, bg_count);
|
||||
// shard responses must include size info, but don't need the derived stats
|
||||
//
|
||||
// NOTE: sizes will be the same for every slot...
|
||||
// TODO: it would be nice to put them directly in the parent facet, instead of every bucket,
|
||||
// in order to reduce the size of the response.
|
||||
result.add(FG_SIZE, fg_size);
|
||||
result.add(BG_SIZE, bg_size);
|
||||
|
||||
if (implied) {
|
||||
// for an implied bucket on this shard, we don't need to bother returning the (empty)
|
||||
// counts, just the flag explaining that this bucket is (locally) implied...
|
||||
result.add(IMPLIED_KEY, Boolean.TRUE);
|
||||
} else {
|
||||
result.add(FG_COUNT, fg_count);
|
||||
result.add(BG_COUNT, bg_count);
|
||||
}
|
||||
} else {
|
||||
if (implied) {
|
||||
// When returning results to an external client, any bucket still 'implied' shouldn't return
|
||||
// any results at all.
|
||||
// (practically speaking this should only happen for the 'allBuckets' bucket
|
||||
return null;
|
||||
}
|
||||
|
||||
// there's no need to bother computing these when returning results *to* a shard coordinator
|
||||
// only useful to external clients
|
||||
result.add(RELATEDNESS, this.getRelatedness());
|
||||
|
@ -441,9 +489,22 @@ public class RelatednessAgg extends AggValueSource {
|
|||
@Override
|
||||
public void merge(Object facetResult, Context mcontext) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
NamedList<Object> shardData = (NamedList<Object>)facetResult;
|
||||
final NamedList<Object> shardData = (NamedList<Object>)facetResult;
|
||||
|
||||
final boolean shardImplied = Optional.ofNullable((Boolean)shardData.remove(IMPLIED_KEY)).orElse(false);
|
||||
|
||||
// regardless of wether this shard is implied, we want to know it's size info...
|
||||
mergedData.incSizes((Long)shardData.remove(FG_SIZE), (Long)shardData.remove(BG_SIZE));
|
||||
mergedData.incCounts((Long)shardData.remove(FG_COUNT), (Long)shardData.remove(BG_COUNT));
|
||||
|
||||
if (! shardImplied) {
|
||||
// only merge in counts from non-implied shard buckets...
|
||||
mergedData.incCounts((Long)shardData.remove(FG_COUNT), (Long)shardData.remove(BG_COUNT));
|
||||
} else {
|
||||
// if this shard is implied, we shouldn't have even gotten counts...
|
||||
assert shardImplied;
|
||||
assert null == shardData.remove(FG_COUNT);
|
||||
assert null == shardData.remove(BG_COUNT);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -249,16 +249,27 @@ public abstract class SlotAcc implements Closeable {
|
|||
* Incapsulates information about the current slot, for Accumulators that may want
|
||||
* additional info during collection.
|
||||
*/
|
||||
public static final class SlotContext {
|
||||
public static class SlotContext {
|
||||
private final Query slotQuery;
|
||||
|
||||
public SlotContext(Query slotQuery) {
|
||||
this.slotQuery = slotQuery;
|
||||
}
|
||||
|
||||
/**
|
||||
* behavior of this method is undefined if {@link #isAllBuckets} returns <code>true</code>
|
||||
*/
|
||||
public Query getSlotQuery() {
|
||||
return slotQuery;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true if and only if this slot corrisponds to the <code>allBuckets</code> bucket.
|
||||
* @see #getSlotQuery
|
||||
*/
|
||||
public boolean isAllBuckets() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -21,6 +21,8 @@ import java.lang.invoke.MethodHandles;
|
|||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
|
@ -287,6 +289,17 @@ public class TestCloudJSONFacetSKG extends SolrCloudTestCase {
|
|||
assertTrue("Didn't check a single bucket???", maxBuckets.get() < UNIQUE_FIELD_VALS);
|
||||
}
|
||||
}
|
||||
{ // allBuckets should have no impact...
|
||||
for (Boolean allBuckets : Arrays.asList( null, false, true )) {
|
||||
Map<String,TermFacet> facets = new LinkedHashMap<>();
|
||||
facets.put("allb__" + allBuckets, new TermFacet(multiStrField(9),
|
||||
map("allBuckets", allBuckets,
|
||||
"sort", "skg desc")));
|
||||
final AtomicInteger maxBuckets = new AtomicInteger(UNIQUE_FIELD_VALS);
|
||||
assertFacetSKGsAreCorrect(maxBuckets, facets, multiStrField(7)+":11", multiStrField(5)+":9", "*:*");
|
||||
assertTrue("Didn't check a single bucket???", maxBuckets.get() < UNIQUE_FIELD_VALS);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testRandom() throws Exception {
|
||||
|
@ -402,6 +415,15 @@ public class TestCloudJSONFacetSKG extends SolrCloudTestCase {
|
|||
|
||||
final NamedList results = (NamedList) actualFacetResponse.get(facetKey);
|
||||
assertNotNull(facetKey + " key missing from: " + actualFacetResponse, results);
|
||||
|
||||
if (null != results.get("allBuckets")) {
|
||||
// if the response includes an allBuckets bucket, then there must not be an skg value
|
||||
|
||||
// 'skg' key must not exist in th allBuckets bucket
|
||||
assertEquals(facetKey + " has skg in allBuckets: " + results.get("allBuckets"),
|
||||
Collections.emptyList(),
|
||||
((NamedList)results.get("allBuckets")).getAll("skg"));
|
||||
}
|
||||
final List<NamedList> buckets = (List<NamedList>) results.get("buckets");
|
||||
assertNotNull(facetKey + " has null buckets: " + actualFacetResponse, buckets);
|
||||
|
||||
|
@ -658,6 +680,7 @@ public class TestCloudJSONFacetSKG extends SolrCloudTestCase {
|
|||
*
|
||||
* @return a sort string (w/direction), or null to specify nothing (trigger default behavior)
|
||||
* @see #randomLimitParam
|
||||
* @see #randomAllBucketsParam
|
||||
* @see #randomPrelimSortParam
|
||||
*/
|
||||
public static String randomSortParam(Random r) {
|
||||
|
@ -726,6 +749,35 @@ public class TestCloudJSONFacetSKG extends SolrCloudTestCase {
|
|||
// else.... either leave param unspecified (or redundently specify the -1 default)
|
||||
return r.nextBoolean() ? null : -1;
|
||||
}
|
||||
|
||||
/**
|
||||
* picks a random value for the "allBuckets" param, biased in favor of interesting test cases.
|
||||
* This bucket should be ignored by relatedness, but inclusion should not cause any problems
|
||||
* (or change the results)
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE:</b> allBuckets is meaningless in conjunction with the <code>STREAM</code> processor, so
|
||||
* this method always returns null if sort is <code>index asc</code>.
|
||||
* </p>
|
||||
*
|
||||
*
|
||||
* @return a Boolean, may be null
|
||||
* @see <a href="https://issues.apache.org/jira/browse/SOLR-14514">SOLR-14514: allBuckets ignored by method:stream</a>
|
||||
*/
|
||||
public static Boolean randomAllBucketsParam(final Random r, final String sort) {
|
||||
|
||||
if ("index asc".equals(sort)) {
|
||||
return null;
|
||||
}
|
||||
|
||||
switch(r.nextInt(4)) {
|
||||
case 0: return true;
|
||||
case 1: return false;
|
||||
case 2:
|
||||
case 3: return null;
|
||||
default: throw new RuntimeException("Broken case statement");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* recursive helper method for building random facets
|
||||
|
@ -747,6 +799,7 @@ public class TestCloudJSONFacetSKG extends SolrCloudTestCase {
|
|||
"limit", randomLimitParam(random(), sort),
|
||||
"overrequest", randomOverrequestParam(random()),
|
||||
"prefix", randomPrefixParam(random(), facetField),
|
||||
"allBuckets", randomAllBucketsParam(random(), sort),
|
||||
"perSeg", randomPerSegParam(random())));
|
||||
|
||||
|
||||
|
|
|
@ -373,11 +373,6 @@ public class TestCloudJSONFacetSKGEquiv extends SolrCloudTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* If/when we can re-enable this test, make sure to update {@link TermFacet#buildRandom}
|
||||
* and {@link #testBespokeStructures} to start doing randomized testing of <code>allBuckets</code>
|
||||
*/
|
||||
@AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-14467")
|
||||
public void testBespokeAllBuckets() throws Exception {
|
||||
{ // single level facet w/sorting on skg and allBuckets
|
||||
Map<String,TermFacet> facets = new LinkedHashMap<>();
|
||||
|
@ -722,7 +717,7 @@ public class TestCloudJSONFacetSKGEquiv extends SolrCloudTestCase {
|
|||
"perSeg", randomPerSegParam(random()),
|
||||
"sort", sort,
|
||||
"prelim_sort", randomPrelimSortParam(random(), sort),
|
||||
// SOLR-14467 // "allBuckets", randomAllBucketsParam(random()),
|
||||
"allBuckets", randomAllBucketsParam(random(), sort),
|
||||
"refine", randomRefineParam(random())));
|
||||
}
|
||||
|
||||
|
@ -759,12 +754,23 @@ public class TestCloudJSONFacetSKGEquiv extends SolrCloudTestCase {
|
|||
|
||||
/**
|
||||
* picks a random value for the "allBuckets" param, biased in favor of interesting test cases
|
||||
* This bucket should be ignored by relatedness, but inclusion should not cause any problems
|
||||
* (or change the results)
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE:</b> allBuckets is meaningless in conjunction with the <code>STREAM</code> processor, so
|
||||
* this method always returns null if sort is <code>index asc</code>.
|
||||
* </p>
|
||||
*
|
||||
* @return a Boolean, may be null
|
||||
* @see #testBespokeAllBuckets
|
||||
* @see <a href="https://issues.apache.org/jira/browse/SOLR-14514">SOLR-14514: allBuckets ignored by method:stream</a>
|
||||
*/
|
||||
public static Boolean randomAllBucketsParam(final Random r) {
|
||||
public static Boolean randomAllBucketsParam(final Random r, final String sort) {
|
||||
|
||||
if ("index asc".equals(sort)) {
|
||||
return null;
|
||||
}
|
||||
|
||||
switch(r.nextInt(4)) {
|
||||
case 0: return true;
|
||||
case 1: return false;
|
||||
|
@ -834,6 +840,7 @@ public class TestCloudJSONFacetSKGEquiv extends SolrCloudTestCase {
|
|||
* Assumes every TermFacet will have at least one "skg" stat
|
||||
*
|
||||
* @return a sort string (w/direction), or null to specify nothing (trigger default behavior)
|
||||
* @see #randomAllBucketsParam
|
||||
* @see #randomPrelimSortParam
|
||||
*/
|
||||
public static String randomSortParam(final Random r) {
|
||||
|
|
|
@ -1212,10 +1212,13 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS {
|
|||
"{ type:func, func:'relatedness($fore,$back)', min_popularity:0.2 }")) {
|
||||
client.testJQ(params(p, "rows", "0", "q", "*:*", "fore", "${xy_s}:X", "back", "${num_d}:[0 TO 100]",
|
||||
"json.facet", "{"
|
||||
+ " cat0:{ ${terms} type:terms, field: ${cat_s}, "
|
||||
+ " cat0:{ ${terms} type:terms, field: ${cat_s}, allBuckets:true, "
|
||||
+ " sort:'count desc', limit:1, overrequest:0, refine:true, "
|
||||
+ " facet:{ s:"+s+"} } }")
|
||||
, "facets=={ count:8, cat0:{ buckets:[ "
|
||||
, "facets=={ count:8, cat0:{ "
|
||||
// 's' key must not exist in the allBuckets bucket
|
||||
+ " allBuckets: { count:8 }"
|
||||
+ " buckets:[ "
|
||||
+ " { val:A, count:4, "
|
||||
+ " s : { relatedness: 0.00496, "
|
||||
//+ " foreground_count: 3, "
|
||||
|
@ -1231,11 +1234,14 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS {
|
|||
// same query with a high min_pop should result in a -Infinity relatedness score
|
||||
client.testJQ(params(p, "rows", "0", "q", "*:*", "fore", "${xy_s}:X", "back", "${num_d}:[0 TO 100]",
|
||||
"json.facet", "{"
|
||||
+ " cat0:{ ${terms} type:terms, field: ${cat_s}, "
|
||||
+ " cat0:{ ${terms} type:terms, field: ${cat_s}, allBuckets:true,"
|
||||
+ " sort:'count desc', limit:1, overrequest:0, refine:true, "
|
||||
+ " facet:{ s:{ type:func, func:'relatedness($fore,$back)', "
|
||||
+ " min_popularity:0.6 } } } }")
|
||||
, "facets=={ count:8, cat0:{ buckets:[ "
|
||||
, "facets=={ count:8, cat0:{ "
|
||||
// 's' key must not exist in the allBuckets bucket
|
||||
+ " allBuckets: { count:8 }"
|
||||
+ " buckets:[ "
|
||||
+ " { val:A, count:4, "
|
||||
+ " s : { relatedness: '-Infinity', "
|
||||
//+ " foreground_count: 3, "
|
||||
|
@ -1248,6 +1254,22 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS {
|
|||
"}"
|
||||
);
|
||||
|
||||
// really special case: allBuckets when there are no regular buckets...
|
||||
for (String refine : Arrays.asList("", "refine: true,", "refine:false,")) {
|
||||
client.testJQ(params(p, "rows", "0", "q", "*:*", "fore", "${xy_s}:X", "back", "${num_d}:[0 TO 100]",
|
||||
"json.facet", "{"
|
||||
+ " cat0:{ ${terms} type:terms, field: bogus_field_s, allBuckets:true, "
|
||||
+ refine
|
||||
+ " facet:{ s:{ type:func, func:'relatedness($fore,$back)' } } } }")
|
||||
, "facets=={ count:8, cat0:{ "
|
||||
// 's' key must not exist in the allBuckets bucket
|
||||
+ " allBuckets: { count:0 }"
|
||||
+ " buckets:[ ]"
|
||||
+ "} }"
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
// SKG under nested facet where some terms only exist on one shard
|
||||
{
|
||||
// sub-bucket order should change as sort direction changes
|
||||
|
|
|
@ -484,10 +484,73 @@ public class TestJsonFacets extends SolrTestCaseHS {
|
|||
+ " background_popularity: 0.5 },"
|
||||
+ " } ] } } "
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
// relatedness shouldn't be computed for allBuckets, but it also shouldn't cause any problems
|
||||
//
|
||||
// NOTE: we can't test this with 'index asc' because STREAM processor
|
||||
// (which test may randomize as default) doesn't support allBuckets
|
||||
// see: https://issues.apache.org/jira/browse/SOLR-14514
|
||||
//
|
||||
for (String sort : Arrays.asList("sort:'y desc'",
|
||||
"sort:'z desc'",
|
||||
"sort:'skg desc'",
|
||||
"prelim_sort:'count desc', sort:'skg desc'")) {
|
||||
// the relatedness score of each of our cat_s values is (conviniently) also alphabetical order,
|
||||
// (and the same order as 'sum(num_i) desc' & 'min(num_i) desc')
|
||||
//
|
||||
// So all of these re/sort options should produce identical output (since the num buckets is < limit)
|
||||
// - Testing "index" sort allows the randomized use of "stream" processor as default to be tested.
|
||||
// - Testing (re)sorts on other stats sanity checks code paths where relatedness() is a "defered" Agg
|
||||
for (String limit : Arrays.asList(", ", ", limit:5, ", ", limit:-1, ")) {
|
||||
// results shouldn't change regardless of our limit param"
|
||||
assertJQ(req("q", "cat_s:[* TO *]", "rows", "0",
|
||||
"fore", "where_s:NY", "back", "*:*",
|
||||
"json.facet", ""
|
||||
+ "{x: { type: terms, field: 'cat_s', allBuckets:true, "+sort + limit
|
||||
+ " facet: { skg: 'relatedness($fore,$back)', y:'sum(num_i)', z:'min(num_i)' } } }")
|
||||
, "facets=={count:5, x:{ "
|
||||
// 'skg' key must not exist in th allBuckets bucket
|
||||
+ " allBuckets: { count:5, y:2.0, z:-5 },"
|
||||
+ "buckets:["
|
||||
+ " { val:'A', count:2, y:5.0, z:2, "
|
||||
+ " skg : { relatedness: 0.00554, "
|
||||
//+ " foreground_count: 1, "
|
||||
//+ " foreground_size: 2, "
|
||||
//+ " background_count: 2, "
|
||||
//+ " background_size: 6,"
|
||||
+ " foreground_popularity: 0.16667,"
|
||||
+ " background_popularity: 0.33333, },"
|
||||
+ " }, "
|
||||
+ " { val:'B', count:3, y:-3.0, z:-5, "
|
||||
+ " skg : { relatedness: 0.0, " // perfectly average and uncorrolated
|
||||
//+ " foreground_count: 1, "
|
||||
//+ " foreground_size: 2, "
|
||||
//+ " background_count: 3, "
|
||||
//+ " background_size: 6,"
|
||||
+ " foreground_popularity: 0.16667,"
|
||||
+ " background_popularity: 0.5 },"
|
||||
+ " } ] } } "
|
||||
);
|
||||
|
||||
// really special case: allBuckets when there are no regular buckets...
|
||||
assertJQ(req("q", "cat_s:[* TO *]", "rows", "0",
|
||||
"fore", "where_s:NY", "back", "*:*",
|
||||
"json.facet", ""
|
||||
+ "{x: { type: terms, field: 'bogus_field_s', allBuckets:true, "+sort + limit
|
||||
+ " facet: { skg: 'relatedness($fore,$back)', y:'sum(num_i)', z:'min(num_i)' } } }")
|
||||
, "facets=={count:5, x:{ "
|
||||
// 'skg' key (as well as 'z' since it's a min) must not exist in the allBuckets bucket
|
||||
+ " allBuckets: { count:0, y:0.0 },"
|
||||
+ "buckets:[ ]"
|
||||
+ " } } "
|
||||
);
|
||||
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// trivial sanity check that we can (re)sort on SKG after pre-sorting on count...
|
||||
// ...and it's only computed for the top N buckets (based on our pre-sort)
|
||||
|
|
|
@ -910,6 +910,8 @@ The `relatedness(...)` function is used to "score" these relationships, relative
|
|||
|
||||
Unlike most aggregation functions, the `relatedness(...)` function is aware of whether and how it's used in <<nested-facets,Nested Facets>>. It evaluates the query defining the current bucket _independently_ from its parent/ancestor buckets, and intersects those documents with a "Foreground Set" defined by the foreground query _combined with the ancestor buckets_. The result is then compared to a similar intersection done against the "Background Set" (defined exclusively by background query) to see if there is a positive, or negative, correlation between the current bucket and the Foreground Set, relative to the Background Set.
|
||||
|
||||
NOTE: The semantics of `relatedness(...)` in an `allBuckets` context is currently undefined. Accordingly, although the `relatedness(...)` stat may be specified for a facet request that also specifies `allBuckets:true`, the `allBuckets` bucket itself will not include a relatedness calculation.
|
||||
|
||||
NOTE: While it's very common to define the Background Set as `\*:*`, or some other super-set of the Foreground Query, it is not strictly required. The `relatedness(...)` function can be used to compare the statistical relatedness of sets of documents to orthogonal foreground/background queries.
|
||||
|
||||
[[relatedness-options]]
|
||||
|
|
Loading…
Reference in New Issue