From 5dc988f5eeff78464d852f54ce7f06a801dcbfee Mon Sep 17 00:00:00 2001 From: Chris Hostetter Date: Fri, 30 Nov 2018 15:49:06 -0700 Subject: [PATCH] SOLR-12839: JSON 'terms' Faceting now supports a 'prelim_sort' option to use when initially selecting the top ranking buckets, prior to the final 'sort' option used after refinement. --- solr/CHANGES.txt | 3 +- .../apache/solr/search/facet/FacetField.java | 25 +- .../solr/search/facet/FacetFieldMerger.java | 2 +- .../search/facet/FacetFieldProcessor.java | 338 +++++++++--- .../solr/search/facet/FacetRangeMerger.java | 4 +- .../solr/search/facet/FacetRequest.java | 70 ++- .../facet/FacetRequestSortedMerger.java | 25 +- .../apache/solr/search/facet/DebugAgg.java | 39 +- .../search/facet/TestJsonFacetRefinement.java | 127 +++++ .../solr/search/facet/TestJsonFacets.java | 495 +++++++++++++++++- .../facet/TestJsonFacetsStatsParsing.java | 15 + solr/solr-ref-guide/src/json-facet-api.adoc | 36 +- 12 files changed, 1039 insertions(+), 140 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 9cb681f9a81..e4fb80526e9 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -130,7 +130,8 @@ Upgrade Notes New Features ---------------------- -(No Changes) +* SOLR-12839: JSON 'terms' Faceting now supports a 'prelim_sort' option to use when initially selecting + the top ranking buckets, prior to the final 'sort' option used after refinement. (hossman) Bug Fixes ---------------------- diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetField.java b/solr/core/src/java/org/apache/solr/search/facet/FacetField.java index a5ca1dff7f1..f2a3c2d07d0 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetField.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetField.java @@ -41,8 +41,18 @@ abstract class FacetRequestSorted extends FacetRequest { */ int overrefine = -1; long mincount; - String sortVariable; - SortDirection sortDirection; + /** + * The basic sorting to do on buckets, defaults to {@link FacetRequest.FacetSort#COUNT_DESC} + * @see #prelim_sort + */ + FacetSort sort; + /** + * An optional "Pre-Sort" that defaults to null. + * If specified, then the prelim_sort is used as an optimization in place of {@link #sort} + * during collection, and the full {@link #sort} values are only computed for the top candidate buckets + * (after refinement) + */ + FacetSort prelim_sort; RefineMethod refine; // null, NONE, or SIMPLE @Override @@ -137,8 +147,15 @@ public class FacetField extends FacetRequestSorted { if (method == FacetMethod.ENUM) {// at the moment these two are the same method = FacetMethod.STREAM; } - if (method == FacetMethod.STREAM && sf.indexed() && - "index".equals(sortVariable) && sortDirection == SortDirection.asc && !ft.isPointField()) { + if (method == FacetMethod.STREAM && sf.indexed() && !ft.isPointField() && + // wether we can use stream processing depends on wether this is a shard request, wether + // re-sorting has been requested, and if the effective sort during collection is "index asc" + ( fcontext.isShard() + // for a shard request, the effective per-shard sort must be index asc + ? FacetSort.INDEX_ASC.equals(null == prelim_sort ? sort : prelim_sort) + // for a non-shard request, we can only use streaming if there is no pre-sorting + : (null == prelim_sort && FacetSort.INDEX_ASC.equals( sort ) ) ) ) { + return new FacetFieldProcessorByEnumTermsStream(fcontext, this, sf); } diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldMerger.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldMerger.java index 2aa961f3cff..f6276b5fd2c 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldMerger.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldMerger.java @@ -102,7 +102,7 @@ public class FacetFieldMerger extends FacetRequestSortedMerger { result.add("numBuckets", ((Number)numBuckets.getMergedResult()).longValue()); } - sortBuckets(); + sortBuckets(freq.sort); long first = freq.offset; long end = freq.limit >=0 ? first + (int) freq.limit : Integer.MAX_VALUE; 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 83ea39f3632..40eb7854bfa 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,8 +18,10 @@ package org.apache.solr.search.facet; import java.io.IOException; +import java.util.Arrays; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.Date; import java.util.HashMap; import java.util.LinkedHashMap; @@ -48,8 +50,11 @@ abstract class FacetFieldProcessor extends FacetProcessor { SchemaField sf; SlotAcc indexOrderAcc; int effectiveMincount; - - Map deferredAggs; // null if none + final boolean singlePassSlotAccCollection; + final FacetRequest.FacetSort sort; // never null (may be the user's requested sort, or the prelim_sort) + final FacetRequest.FacetSort resort; // typically null (unless the user specified a prelim_sort) + + final Map deferredAggs = new HashMap(); // TODO: push any of this down to base class? @@ -67,6 +72,37 @@ abstract class FacetFieldProcessor extends FacetProcessor { super(fcontext, freq); this.sf = sf; this.effectiveMincount = (int)(fcontext.isShard() ? Math.min(1 , freq.mincount) : freq.mincount); + this.singlePassSlotAccCollection = (freq.limit == -1 && freq.subFacets.size() == 0); + + if ( null == freq.prelim_sort ) { + // If the user has not specified any preliminary sort, then things are very simple. + // Just use the "sort" as is w/o needing any re-sorting + this.sort = freq.sort; + this.resort = null; + } else { + assert null != freq.prelim_sort; + + if ( fcontext.isShard() ) { + // for a shard request, we can ignore the users requested "sort" and focus solely on the prelim_sort + // the merger will worry about the final sorting -- we don't need to resort anything... + this.sort = freq.prelim_sort; + this.resort = null; + + } else { // non shard... + if ( singlePassSlotAccCollection ) { // special case situation... + // when we can do a single pass SlotAcc collection on non-shard request, there is + // no point re-sorting. Ignore the freq.prelim_sort and use the freq.sort option as is... + this.sort = freq.sort; + this.resort = null; + } else { + // for a non-shard request, we will use the prelim_sort as our initial sort option if it exists + // then later we will re-sort on the final desired sort... + this.sort = freq.prelim_sort; + this.resort = freq.sort; + } + } + } + assert null != this.sort; } /** This is used to create accs for second phase (or to create accs for all aggs) */ @@ -86,17 +122,7 @@ abstract class FacetFieldProcessor extends FacetProcessor { // reuse these accs, but reset them first and resize since size could be different for (SlotAcc acc : accs) { acc.reset(); - acc.resize(new SlotAcc.Resizer() { - @Override - public int getNewSize() { - return slotCount; - } - - @Override - public int getNewSlot(int oldSlot) { - return 0; - } - }); + acc.resize(new FlatteningResizer(slotCount)); } return; } else { @@ -121,33 +147,47 @@ abstract class FacetFieldProcessor extends FacetProcessor { } } - void createCollectAcc(int numDocs, int numSlots) throws IOException { - accMap = new LinkedHashMap<>(); - - // we always count... - // allow a subclass to set a custom counter. - if (countAcc == null) { - countAcc = new CountSlotArrAcc(fcontext, numSlots); - } - - if ("count".equals(freq.sortVariable)) { - sortAcc = countAcc; - deferredAggs = freq.getFacetStats(); - } else if ("index".equals(freq.sortVariable)) { + /** + * Simple helper for checking if a {@FacetRequest.FacetSort} is on "count" or "index" and picking + * the existing SlotAcc + * @return an existing SlotAcc for sorting, else null if it should be built from the Aggs + */ + private SlotAcc getTrivialSortingSlotAcc(FacetRequest.FacetSort fsort) { + if ("count".equals(fsort.sortVariable)) { + assert null != countAcc; + return countAcc; + } else if ("index".equals(fsort.sortVariable)) { // allow subclass to set indexOrderAcc first if (indexOrderAcc == null) { // This sorting accumulator just goes by the slot number, so does not need to be collected // and hence does not need to find it's way into the accMap or accs array. indexOrderAcc = new SortSlotAcc(fcontext); } - sortAcc = indexOrderAcc; - deferredAggs = freq.getFacetStats(); + return indexOrderAcc; + } + return null; + } + + void createCollectAcc(int numDocs, int numSlots) throws IOException { + accMap = new LinkedHashMap<>(); + + // start with the assumption that we're going to defer the computation of all stats + deferredAggs.putAll(freq.getFacetStats()); + + // we always count... + // allow a subclass to set a custom counter. + if (countAcc == null) { + countAcc = new CountSlotArrAcc(fcontext, numSlots); } - // If we are going to return all buckets and if there are no subfacets (that would need a domain), then don't defer - // any aggregation calculations to a second phase. This way we can avoid calculating domains for each bucket, which - // can be expensive. - if (freq.limit == -1 && freq.subFacets.size() == 0) { + sortAcc = getTrivialSortingSlotAcc(this.sort); + + if (this.singlePassSlotAccCollection) { + // If we are going to return all buckets, and if there are no subfacets (that would need a domain), + // then don't defer any aggregation calculations to a second phase. + // This way we can avoid calculating domains for each bucket, which can be expensive. + + // TODO: BEGIN: why can't we just call createAccs here ? accs = new SlotAcc[ freq.getFacetStats().size() ]; int otherAccIdx = 0; for (Map.Entry entry : freq.getFacetStats().entrySet()) { @@ -157,6 +197,7 @@ abstract class FacetFieldProcessor extends FacetProcessor { accMap.put(acc.key, acc); accs[otherAccIdx++] = acc; } + // TODO: END: why can't we just call createAccs here ? if (accs.length == 1) { collectAcc = accs[0]; } else { @@ -164,26 +205,21 @@ abstract class FacetFieldProcessor extends FacetProcessor { } if (sortAcc == null) { - sortAcc = accMap.get(freq.sortVariable); + sortAcc = accMap.get(sort.sortVariable); assert sortAcc != null; } - deferredAggs = null; + deferredAggs.clear(); } if (sortAcc == null) { - AggValueSource sortAgg = freq.getFacetStats().get(freq.sortVariable); + AggValueSource sortAgg = freq.getFacetStats().get(sort.sortVariable); if (sortAgg != null) { collectAcc = sortAgg.createSlotAcc(fcontext, numDocs, numSlots); - collectAcc.key = freq.sortVariable; // TODO: improve this + collectAcc.key = sort.sortVariable; // TODO: improve this } sortAcc = collectAcc; - deferredAggs = new HashMap<>(freq.getFacetStats()); - deferredAggs.remove(freq.sortVariable); - } - - if (deferredAggs == null || deferredAggs.size() == 0) { - deferredAggs = null; + deferredAggs.remove(sort.sortVariable); } boolean needOtherAccs = freq.allBuckets; // TODO: use for missing too... @@ -207,7 +243,7 @@ abstract class FacetFieldProcessor extends FacetProcessor { return; } - int numDeferred = deferredAggs == null ? 0 : deferredAggs.size(); + final int numDeferred = deferredAggs.size(); if (numDeferred <= 0) return; otherAccs = new SlotAcc[ numDeferred ]; @@ -267,11 +303,13 @@ abstract class FacetFieldProcessor extends FacetProcessor { } else { effectiveLimit += freq.overrequest; } + } else if (null != resort && 0 < freq.overrequest) { + // in non-shard situations, if we have a 'resort' we check for explicit overrequest > 0 + effectiveLimit += freq.overrequest; } } - - final int sortMul = freq.sortDirection.getMultiplier(); + final int sortMul = sort.sortDirection.getMultiplier(); int maxTopVals = (int) (effectiveLimit >= 0 ? Math.min(freq.offset + effectiveLimit, Integer.MAX_VALUE - 1) : Integer.MAX_VALUE - 1); maxTopVals = Math.min(maxTopVals, slotCardinality); @@ -358,31 +396,53 @@ abstract class FacetFieldProcessor extends FacetProcessor { // moved missing fillBucket after we fill facet since it will reset all the accumulators. } - // if we are deep paging, we don't have to order the highest "offset" counts. - int collectCount = Math.max(0, queue.size() - off); - assert collectCount <= maxTopVals; - int[] sortedSlots = new int[collectCount]; - for (int i = collectCount - 1; i >= 0; i--) { - sortedSlots[i] = queue.pop().slot; + final boolean needFilter = (!deferredAggs.isEmpty()) || freq.getSubFacets().size() > 0; + if (needFilter) { + createOtherAccs(-1, 1); } - ArrayList bucketList = new ArrayList<>(collectCount); - res.add("buckets", bucketList); + // if we are deep paging, we don't have to order the highest "offset" counts... + // ...unless we need to resort. + int collectCount = Math.max(0, queue.size() - (null == this.resort ? off : 0)); + // + assert collectCount <= maxTopVals; + Slot[] sortedSlots = new Slot[collectCount]; + for (int i = collectCount - 1; i >= 0; i--) { + Slot slot = sortedSlots[i] = queue.pop(); + // At this point we know we're either returning this Slot as a Bucket, or resorting it, + // so definitely fill in the bucket value -- we'll need it either way + slot.bucketVal = bucketValFromSlotNumFunc.apply(slot.slot); + + if (needFilter || null != this.resort) { + slot.bucketFilter = makeBucketQuery(fieldQueryValFunc.apply(slot.bucketVal)); + } + } + + final SlotAcc resortAccForFill = resortSlots(sortedSlots); // No-Op if not needed + + if (null != this.resort) { + // now that we've completely resorted, throw away extra docs from possible offset/overrequest... + final int endOffset = (int)Math.min((long) sortedSlots.length, + // NOTE: freq.limit is long, so no risk of overflow here + off + (freq.limit < 0 ? Integer.MAX_VALUE : freq.limit)); + if (0 < off || endOffset < sortedSlots.length) { + sortedSlots = Arrays.copyOfRange(sortedSlots, off, endOffset); + } + } + List bucketList = new ArrayList<>(sortedSlots.length); - boolean needFilter = deferredAggs != null || freq.getSubFacets().size() > 0; - - for (int slotNum : sortedSlots) { + for (Slot slot : sortedSlots) { SimpleOrderedMap bucket = new SimpleOrderedMap<>(); - Comparable val = bucketValFromSlotNumFunc.apply(slotNum); - bucket.add("val", val); + bucket.add("val", slot.bucketVal); - Query filter = needFilter ? makeBucketQuery(fieldQueryValFunc.apply(val)) : null; - - fillBucket(bucket, countAcc.getCount(slotNum), slotNum, null, filter); + fillBucketFromSlot(bucket, slot, resortAccForFill); bucketList.add(bucket); } + res.add("buckets", bucketList); + + if (fcontext.isShard() && shardHasMoreBuckets) { // Currently, "more" is an internal implementation detail and only returned for distributed sub-requests res.add("more", true); @@ -420,24 +480,38 @@ abstract class FacetFieldProcessor extends FacetProcessor { } private static class Slot { + /** The Slot number used during collection */ int slot; + + /** filled in only once we know the bucket will either be involved in resorting, or returned */ + Comparable bucketVal; + + /** Filled in if and only if needed for resorting, deferred stats, or subfacets */ + Query bucketFilter; + // TODO: we could potentially store the bucket's (DocSet)subDomain as well, + // but that's much bigger object to hang onto for every slot at the sametime + // Probably best to just trust the filterCache to do it's job + + /** The Slot number used during resorting */ + int resortSlotNum; } - private void fillBucket(SimpleOrderedMap target, int count, int slotNum, DocSet subDomain, Query filter) throws IOException { + /** Helper method used solely when looping over buckets to be returned in findTopSlots */ + private void fillBucketFromSlot(SimpleOrderedMap target, Slot slot, + SlotAcc resortAcc) throws IOException { + final int count = countAcc.getCount(slot.slot); target.add("count", count); if (count <= 0 && !freq.processEmpty) return; - if (collectAcc != null && slotNum >= 0) { - collectAcc.setValues(target, slotNum); + if (collectAcc != null && slot.slot >= 0) { + collectAcc.setValues(target, slot.slot); } - createOtherAccs(-1, 1); - if (otherAccs == null && freq.subFacets.isEmpty()) return; - if (subDomain == null) { - subDomain = fcontext.searcher.getDocSet(filter, fcontext.base); - } + assert null != slot.bucketFilter; + final Query filter = slot.bucketFilter; + final DocSet subDomain = fcontext.searcher.getDocSet(filter, fcontext.base); // if no subFacets, we only need a DocSet // otherwise we need more? @@ -449,15 +523,119 @@ abstract class FacetFieldProcessor extends FacetProcessor { if (otherAccs != null) { // do acc at a time (traversing domain each time) or do all accs for each doc? for (SlotAcc acc : otherAccs) { - acc.reset(); // TODO: only needed if we previously used for allBuckets or missing - acc.collect(subDomain, 0, slot -> { return new SlotContext(filter); }); - acc.setValues(target, 0); + if (acc == resortAcc) { + // already collected, just need to get the value from the correct slot + acc.setValues(target, slot.resortSlotNum); + } else { + acc.reset(); // TODO: only needed if we previously used for allBuckets or missing + acc.collect(subDomain, 0, s -> { return new SlotContext(filter); }); + acc.setValues(target, 0); + } } } processSubs(target, filter, subDomain, false, null); } + /** + * Helper method that resorts the slots (if needed). + * + * @return a SlotAcc that should be used {@link SlotAcc#setValues} on the final buckets via + * {@link Slot#resortSlotNum} or null if no special SlotAcc was needed (ie: no resorting, or resorting + * on something already known/collected) + */ + private SlotAcc resortSlots(Slot[] slots) throws IOException { + if (null == this.resort) { + return null; // Nothing to do. + } + assert ! fcontext.isShard(); + + // NOTE: getMultiplier() is confusing and weird and ment for use in PriorityQueue.lessThan, + // so it's backwards from what you'd expect in a Comparator... + final int resortMul = -1 * resort.sortDirection.getMultiplier(); + + SlotAcc resortAcc = getTrivialSortingSlotAcc(this.resort); + if (null != resortAcc) { + // resorting on count or index is rare (and not particularly useful) but if someone chooses to do + // either of these we don't need to re-collect ... instead just re-sort the slots based on + // the previously collected values using the originally collected slot numbers... + if (resortAcc.equals(countAcc)) { + final Comparator comparator = null != indexOrderAcc ? + (new Comparator() { + public int compare(Slot x, Slot y) { + final int cmp = resortMul * countAcc.compare(x.slot, y.slot); + return cmp != 0 ? cmp : indexOrderAcc.compare(x.slot, y.slot); + } + }) + : (new Comparator() { + public int compare(Slot x, Slot y) { + final int cmp = resortMul * countAcc.compare(x.slot, y.slot); + return cmp != 0 ? cmp : Integer.compare(x.slot, y.slot); + } + }); + Arrays.sort(slots, comparator); + return null; + } + if (resortAcc.equals(indexOrderAcc)) { + // obviously indexOrderAcc is not null, and no need for a fancy tie breaker... + Arrays.sort(slots, new Comparator() { + public int compare(Slot x, Slot y) { + return resortMul * indexOrderAcc.compare(x.slot, y.slot); + } + }); + return null; + } + // nothing else should be possible + assert false : "trivial resort isn't count or index: " + this.resort; + } + + assert null == resortAcc; + for (SlotAcc acc : otherAccs) { + if (acc.key.equals(this.resort.sortVariable)) { + resortAcc = acc; + break; + } + } + // TODO: what if resortAcc is still null, ie: bad input? ... throw an error? (see SOLR-13022) + // looks like equivilent sort code path silently ignores sorting if sortVariable isn't in accMap... + // ...and we get a deffered NPE when trying to collect. + assert null != resortAcc; + + final SlotAcc acc = resortAcc; + + // reset resortAcc to be (just) big enough for all the slots we care about... + acc.reset(); + acc.resize(new FlatteningResizer(slots.length)); + + // give each existing Slot a new resortSlotNum and let the resortAcc collect it... + for (int slotNum = 0; slotNum < slots.length; slotNum++) { + Slot slot = slots[slotNum]; + slot.resortSlotNum = slotNum; + + assert null != slot.bucketFilter : "null filter for slot=" +slot.bucketVal; + + final DocSet subDomain = fcontext.searcher.getDocSet(slot.bucketFilter, fcontext.base); + acc.collect(subDomain, slotNum, s -> { return new SlotContext(slot.bucketFilter); } ); + } + + // now resort all the Slots according to the new collected values... + final Comparator comparator = null != indexOrderAcc ? + (new Comparator() { + public int compare(Slot x, Slot y) { + final int cmp = resortMul * acc.compare(x.resortSlotNum, y.resortSlotNum); + return cmp != 0 ? cmp : indexOrderAcc.compare(x.slot, y.slot); + } + }) + : (new Comparator() { + public int compare(Slot x, Slot y) { + final int cmp = resortMul * acc.compare(x.resortSlotNum, y.resortSlotNum); + return cmp != 0 ? cmp : Integer.compare(x.slot, y.slot); + } + }); + Arrays.sort(slots, comparator); + return acc; + } + @Override protected void processStats(SimpleOrderedMap bucket, Query bucketQ, DocSet docs, int docCount) throws IOException { if (docCount == 0 && !freq.processEmpty || freq.getFacetStats().size() == 0) { @@ -733,4 +911,20 @@ abstract class FacetFieldProcessor extends FacetProcessor { return bucket; } + /** Resizes to the specified size, remapping all existing slots to slot 0 */ + private static final class FlatteningResizer extends SlotAcc.Resizer { + private final int slotCount; + public FlatteningResizer(int slotCount) { + this.slotCount = slotCount; + } + @Override + public int getNewSize() { + return slotCount; + } + + @Override + public int getNewSlot(int oldSlot) { + return 0; + } + } } diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetRangeMerger.java b/solr/core/src/java/org/apache/solr/search/facet/FacetRangeMerger.java index 452652f9f0b..aa7112e3018 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetRangeMerger.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetRangeMerger.java @@ -44,8 +44,8 @@ public class FacetRangeMerger extends FacetRequestSortedMerger { } @Override - public void sortBuckets() { - // regardless of mincount, every shard returns a consistent set of buckets which are already in the correct order + public void sortBuckets(final FacetRequest.FacetSort sort) { + // regardless of sort or mincount, every shard returns a consistent set of buckets which are already in the correct order sortedBuckets = new ArrayList<>( buckets.values() ); } diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java b/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java index 4135c87db68..07a10f35ce3 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.EnumSet; import java.util.LinkedHashMap; import java.util.List; +import java.util.Objects; import java.util.Map; import org.apache.lucene.search.Query; @@ -54,8 +55,40 @@ import static org.apache.solr.search.facet.FacetRequest.RefineMethod.NONE; */ public abstract class FacetRequest { + /** Simple structure for encapsulating a sort variable and a direction */ + public static final class FacetSort { + final String sortVariable; + final SortDirection sortDirection; + public FacetSort(final String sortVariable, final SortDirection sortDirection) { + assert null != sortVariable; + assert null != sortDirection; + + this.sortVariable = sortVariable; + this.sortDirection = sortDirection; + } + public boolean equals(Object other) { + if (other instanceof FacetSort) { + final FacetSort that = (FacetSort)other; + return this.sortVariable.equals(that.sortVariable) + && this.sortDirection.equals(that.sortDirection); + } + return false; + } + public int hashCode() { + return Objects.hash(sortVariable, sortDirection); + } + public String toString() { + return sortVariable + " " + sortDirection; + } + + /** Commonly Re-used "count desc" (default) */ + public static final FacetSort COUNT_DESC = new FacetSort("count", SortDirection.desc); + /** Commonly Re-used "index asc" (index order / streaming) */ + public static final FacetSort INDEX_ASC = new FacetSort("index", SortDirection.asc); + } + public static enum SortDirection { - asc(-1) , + asc(-1), desc(1); private final int multiplier; @@ -893,8 +926,7 @@ class FacetFieldParser extends FacetParser { if (arg instanceof String) { // just the field name... facet.field = (String)arg; - parseSort( null ); // TODO: defaults - + } else if (arg instanceof Map) { Map m = (Map) arg; facet.field = getField(m); @@ -921,7 +953,13 @@ class FacetFieldParser extends FacetParser { Object o = m.get("facet"); parseSubs(o); - parseSort( m.get(SORT) ); + // TODO: SOLR-13022 ... validate the sortVariabls against the subs. + facet.sort = parseSort( m.get(SORT) ); + facet.prelim_sort = parseSort( m.get("prelim_sort") ); + } + + if (null == facet.sort) { + facet.sort = FacetRequest.FacetSort.COUNT_DESC; } return facet; @@ -932,21 +970,23 @@ class FacetFieldParser extends FacetParser { // sort : 'mystat desc' // OR // sort : { mystat : 'desc' } - private void parseSort(Object sort) { + private static FacetRequest.FacetSort parseSort(Object sort) { if (sort == null) { - facet.sortVariable = "count"; - facet.sortDirection = FacetRequest.SortDirection.desc; + return null; } else if (sort instanceof String) { String sortStr = (String)sort; if (sortStr.endsWith(" asc")) { - facet.sortVariable = sortStr.substring(0, sortStr.length()-" asc".length()); - facet.sortDirection = FacetRequest.SortDirection.asc; + return new FacetRequest.FacetSort(sortStr.substring(0, sortStr.length()-" asc".length()), + FacetRequest.SortDirection.asc); } else if (sortStr.endsWith(" desc")) { - facet.sortVariable = sortStr.substring(0, sortStr.length()-" desc".length()); - facet.sortDirection = FacetRequest.SortDirection.desc; + return new FacetRequest.FacetSort(sortStr.substring(0, sortStr.length()-" desc".length()), + FacetRequest.SortDirection.desc); } else { - facet.sortVariable = sortStr; - facet.sortDirection = "index".equals(facet.sortVariable) ? FacetRequest.SortDirection.asc : FacetRequest.SortDirection.desc; // default direction for "index" is ascending + return new FacetRequest.FacetSort(sortStr, + // default direction for "index" is ascending + ("index".equals(sortStr) + ? FacetRequest.SortDirection.asc + : FacetRequest.SortDirection.desc)); } } else { // sort : { myvar : 'desc' } @@ -955,10 +995,8 @@ class FacetFieldParser extends FacetParser { Map.Entry entry = map.entrySet().iterator().next(); String k = entry.getKey(); Object v = entry.getValue(); - facet.sortVariable = k; - facet.sortDirection = FacetRequest.SortDirection.valueOf(v.toString()); + return new FacetRequest.FacetSort(k, FacetRequest.SortDirection.valueOf(v.toString())); } - } } diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetRequestSortedMerger.java b/solr/core/src/java/org/apache/solr/search/facet/FacetRequestSortedMerger.java index 1d30a0e3897..c1e5631cc86 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetRequestSortedMerger.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetRequestSortedMerger.java @@ -79,25 +79,27 @@ abstract class FacetRequestSortedMerger( buckets.values() ); Comparator comparator = null; - final FacetRequest.SortDirection direction = freq.sortDirection; + final FacetRequest.SortDirection direction = sort.sortDirection; final int sortMul = direction.getMultiplier(); - if ("count".equals(freq.sortVariable)) { + if ("count".equals(sort.sortVariable)) { comparator = (o1, o2) -> { int v = -Long.compare(o1.count, o2.count) * sortMul; return v == 0 ? o1.bucketValue.compareTo(o2.bucketValue) : v; }; Collections.sort(sortedBuckets, comparator); - } else if ("index".equals(freq.sortVariable)) { + } else if ("index".equals(sort.sortVariable)) { comparator = (o1, o2) -> -o1.bucketValue.compareTo(o2.bucketValue) * sortMul; Collections.sort(sortedBuckets, comparator); } else { - final String key = freq.sortVariable; + final String key = sort.sortVariable; /** final FacetSortableMerger[] arr = new FacetSortableMerger[buckets.size()]; @@ -154,6 +156,7 @@ abstract class FacetRequestSortedMerger tagsWithPartial = mcontext.getSubsWithPartial(freq); @@ -206,9 +211,9 @@ abstract class FacetRequestSortedMerger 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; + + // client 0 // shard1: A=1,B=1,C=2 ... + c0.add(sdoc("id", id++, "cat_s","A", "price_i","1")); + c0.add(sdoc("id", id++, "cat_s","B", "price_i","1")); + c0.add(sdoc("id", id++, "cat_s","C", "price_i","1")); + c0.add(sdoc("id", id++, "cat_s","C", "price_i","1")); + // ... X=3,Y=3 + c0.add(sdoc("id", id++, "cat_s","X", "price_i","1")); + c0.add(sdoc("id", id++, "cat_s","X", "price_i","1")); + c0.add(sdoc("id", id++, "cat_s","X", "price_i","1")); + c0.add(sdoc("id", id++, "cat_s","Y", "price_i","1")); + c0.add(sdoc("id", id++, "cat_s","Y", "price_i","1")); + c0.add(sdoc("id", id++, "cat_s","Y", "price_i","1")); + + // client 1 // shard2: X=1,Y=2,Z=2 ... + c1.add(sdoc("id", id++, "cat_s","X", "price_i","1")); + c1.add(sdoc("id", id++, "cat_s","Y", "price_i","1")); + c1.add(sdoc("id", id++, "cat_s","Y", "price_i","1")); + c1.add(sdoc("id", id++, "cat_s","Z", "price_i","1")); + c1.add(sdoc("id", id++, "cat_s","Z", "price_i","1")); + // ... C=4 + c1.add(sdoc("id", id++, "cat_s","C", "price_i","1")); + c1.add(sdoc("id", id++, "cat_s","C", "price_i","1")); + c1.add(sdoc("id", id++, "cat_s","C", "price_i","1")); + c1.add(sdoc("id", id++, "cat_s","C", "price_i","1")); + + // Whole Collection: A=1,B=1,Z=2,X=4,Y=5,C=6 + client.commit(); + + // in both cases, neither C nor Z make the cut for the top3 buckets in phase#1 (due to tie breaker), + // so they aren't refined -- after refinement the re-sorting re-orders the buckets + client.testJQ(params("q", "*:*", "rows", "0", "json.facet", "{" + + " cat_1 : { type:terms, field:cat_s, limit:3, overrequest:0" + + " , refine:true, prelim_sort:'count asc', sort:'index desc' }, " + + " cat_2 : { type:terms, field:cat_s, limit:3, overrequest:0" + + " , refine:true, prelim_sort:'sum_p asc', sort:'count desc' " + + " , facet: { sum_p: 'sum(price_i)' } }" + + "}") + , "facets=={ count: "+id+"," + + " cat_1:{ buckets:[ " + + " {val:X,count:4}," // index desc + + " {val:B,count:1}," + + " {val:A,count:1}," + + " ] }," + + " cat_2:{ buckets:[ " + + " {val:X,count:4,sum_p:4.0}," // count desc + + " {val:A,count:1,sum_p:1.0}," // index order tie break + + " {val:B,count:1,sum_p:1.0}," + + " ] }" + + "}" + ); + + // with some explicit overrefinement=2, we also refine C and Y, giving us those additional + // (fully populated) buckets to consider during re-sorting... + client.testJQ(params("q", "*:*", "rows", "0", "json.facet", "{" + + " cat_1 : { type:terms, field:cat_s, limit:3, overrequest:0, overrefine:2" + + " , refine:true, prelim_sort:'count asc', sort:'index desc' }, " + + " cat_2 : { type:terms, field:cat_s, limit:3, overrequest:0, overrefine:2" + + " , refine:true, prelim_sort:'sum_p asc', sort:'count desc' " + + " , facet: { sum_p: 'sum(price_i)' } }" + + "}") + , "facets=={ count: "+id+"," + + " cat_1:{ buckets:[ " + + " {val:Y,count:5}," // index desc + + " {val:X,count:4}," + + " {val:C,count:6}," + + " ] }," + + " cat_2:{ buckets:[ " + + " {val:C,count:6,sum_p:6.0}," // count desc + + " {val:Y,count:5,sum_p:5.0}," + + " {val:X,count:4,sum_p:4.0}," + + " ] }" + + "}" + ); + } + @Test public void testSortedFacetRefinementPushingNonRefinedBucketBackIntoTopN() throws Exception { 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 e21c6d84811..89098150cab 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 @@ -25,6 +25,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Random; +import java.util.concurrent.atomic.AtomicLong; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; import com.tdunning.math.stats.AVLTreeDigest; @@ -50,7 +51,7 @@ import org.junit.Test; @LuceneTestCase.SuppressCodecs({"Lucene3x","Lucene40","Lucene41","Lucene42","Lucene45","Appending"}) public class TestJsonFacets extends SolrTestCaseHS { - + private static SolrInstances servers; // for distributed testing private static int origTableSize; private static FacetField.FacetMethod origDefaultFacetMethod; @@ -89,13 +90,20 @@ public class TestJsonFacets extends SolrTestCaseHS { } } - // tip: when debugging a test, comment out the @ParametersFactory and edit the constructor to be no-arg + // tip: when debugging failures, change this variable to DEFAULT_METHOD + // (or if only one method is problematic, set to that explicitly) + private static final FacetField.FacetMethod TEST_ONLY_ONE_FACET_METHOD + = null; // FacetField.FacetMethod.DEFAULT_METHOD; @ParametersFactory public static Iterable parameters() { + if (null != TEST_ONLY_ONE_FACET_METHOD) { + return Arrays.asList(new Object[] { TEST_ONLY_ONE_FACET_METHOD }); + } + // wrap each enum val in an Object[] and return as Iterable return () -> Arrays.stream(FacetField.FacetMethod.values()) - .map(it -> new Object[]{it}).iterator(); + .map(it -> new Object[]{it}).iterator(); } public TestJsonFacets(FacetField.FacetMethod defMethod) { @@ -435,18 +443,28 @@ public class TestJsonFacets extends SolrTestCaseHS { + " } }" ); - // simple single level facet w/skg stat & sorting - for (String sort : Arrays.asList("index asc", "skg desc")) { - // the relatedness score of each of our cat_s values is (conviniently) also alphabetical order - // so both of these sort options should produce identical output - // and testinging "index" sort allows the randomized use of "stream" processor as default to be tested + // simple single level facet w/skg stat & (re)sorting + for (String sort : Arrays.asList("sort:'index asc'", + "sort:'y desc'", + "sort:'z desc'", + "sort:'skg desc'", + "prelim_sort:'count desc', sort:'index asc'", + "prelim_sort:'count desc', sort:'y desc'", + "prelim_sort:'count desc', sort:'z 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 assertJQ(req("q", "cat_s:[* TO *]", "rows", "0", "fore", "where_s:NY", "back", "*:*", "json.facet", "" - + "{x: { type: terms, field: 'cat_s', sort: '"+sort+"', " - + " facet: { skg: 'relatedness($fore,$back)' } } }") + + "{x: { type: terms, field: 'cat_s', "+sort+", " + + " facet: { skg: 'relatedness($fore,$back)', y:'sum(num_i)', z:'min(num_i)' } } }") , "facets=={count:5, x:{ buckets:[" - + " { val:'A', count:2, " + + " { val:'A', count:2, y:5.0, z:2, " + " skg : { relatedness: 0.00554, " //+ " foreground_count: 1, " //+ " foreground_size: 2, " @@ -455,7 +473,7 @@ public class TestJsonFacets extends SolrTestCaseHS { + " foreground_popularity: 0.16667," + " background_popularity: 0.33333, }," + " }, " - + " { val:'B', count:3, " + + " { val:'B', count:3, y:-3.0, z:-5, " + " skg : { relatedness: 0.0, " // perfectly average and uncorrolated //+ " foreground_count: 1, " //+ " foreground_size: 2, " @@ -467,6 +485,37 @@ public class TestJsonFacets extends SolrTestCaseHS { ); } + // 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) + for (int overrequest : Arrays.asList(0, 1, 42)) { + // based on our counts & relatedness values, the blackbox output should be the same for both + // overrequest values ... only DebugAgg stats should change... + DebugAgg.Acc.collectDocs.set(0); + DebugAgg.Acc.collectDocSets.set(0); + + assertJQ(req("q", "cat_s:[* TO *]", "rows", "0", + "fore", "where_s:NJ", "back", "*:*", + "json.facet", "" + + "{x: { type: terms, field: 'cat_s', prelim_sort: 'count desc', sort:'skg desc', " + + " limit: 1, overrequest: " + overrequest + ", " + + " facet: { skg: 'debug(wrap,relatedness($fore,$back))' } } }") + , "facets=={count:5, x:{ buckets:[" + + " { val:'B', count:3, " + + " skg : { relatedness: 0.00638, " + //+ " foreground_count: 2, " + //+ " foreground_size: 3, " + //+ " background_count: 3, " + //+ " background_size: 6," + + " foreground_popularity: 0.33333," + + " background_popularity: 0.5 }," + + " }, " + + " ] } } " + ); + // at most 2 buckets, regardless of overrequest... + assertEqualsAndReset(0 < overrequest ? 2 : 1, DebugAgg.Acc.collectDocSets); + assertEqualsAndReset(0, DebugAgg.Acc.collectDocs); + } + // SKG used in multiple nested facets // // we'll re-use these params in 2 requests, one will simulate a shard request @@ -936,7 +985,6 @@ public class TestJsonFacets extends SolrTestCaseHS { } public void doStats(Client client, ModifiableSolrParams p) throws Exception { - Map> fieldLists = new HashMap<>(); fieldLists.put("noexist", getAlternatives("noexist_s")); fieldLists.put("cat_s", getAlternatives("cat_s")); @@ -1165,6 +1213,31 @@ public class TestJsonFacets extends SolrTestCaseHS { ", f2:{ 'buckets':[{ val:'B', count:3, n1:-3.0}, { val:'A', count:2, n1:6.0 }]} }" ); + // test trivial re-sorting by stats + // (there are other more indepth tests of this in doTestPrelimSorting, but this let's us sanity check + // small responses with multiple templatized params of diff real types) + client.testJQ(params(p, "q", "*:*", "json.facet" // num_d + , "{f1:{terms:{${terms} field:'${cat_s}', " + + " prelim_sort:'count desc', sort:'n1 desc', facet:{n1:'sum(${num_d})'} }}," + + " f2:{terms:{${terms} field:'${cat_s}', " + + " prelim_sort:'count asc', sort:'n1 asc', facet:{n1:'sum(${num_d})'} }} }" + ) + , "facets=={ 'count':6 " + + ", f1:{ 'buckets':[{ val:'A', count:2, n1:6.0 }, { val:'B', count:3, n1:-3.0}]}" + + ", f2:{ 'buckets':[{ val:'B', count:3, n1:-3.0}, { val:'A', count:2, n1:6.0 }]} }" + ); + client.testJQ(params(p, "q", "*:*", "json.facet" // num_i + , "{f1:{terms:{${terms} field:'${cat_s}', " + + " prelim_sort:'count desc', sort:'n1 desc', facet:{n1:'sum(${num_i})'} }}," + + " f2:{terms:{${terms} field:'${cat_s}', " + + " prelim_sort:'count asc', sort:'n1 asc', facet:{n1:'sum(${num_i})'} }} }" + ) + , "facets=={ 'count':6 " + + ", f1:{ 'buckets':[{ val:'A', count:2, n1:5.0 }, { val:'B', count:3, n1:-3.0}]}" + + ", f2:{ 'buckets':[{ val:'B', count:3, n1:-3.0}, { val:'A', count:2, n1:5.0 }]} }" + ); + + // test sorting by other stats and more than one facet client.testJQ(params(p, "q", "*:*" , "json.facet", "{f1:{terms:{${terms} field:'${cat_s}', sort:'n1 desc', facet:{n1:'sum(${num_d})', n2:'avg(${num_d})'} }}" + @@ -2193,10 +2266,390 @@ public class TestJsonFacets extends SolrTestCaseHS { long refineParses = DebugAgg.parses.get() - startParses; assertEquals(noRefineParses, refineParses); } - - } + public void testPrelimSortingSingleNode() throws Exception { + doTestPrelimSortingSingleNode(false, false); + } + + public void testPrelimSortingSingleNodeExtraStat() throws Exception { + doTestPrelimSortingSingleNode(true, false); + } + + public void testPrelimSortingSingleNodeExtraFacet() throws Exception { + doTestPrelimSortingSingleNode(false, true); + } + + public void testPrelimSortingSingleNodeExtraStatAndFacet() throws Exception { + doTestPrelimSortingSingleNode(true, true); + } + + /** @see #doTestPrelimSorting */ + public void doTestPrelimSortingSingleNode(final boolean extraAgg, final boolean extraSubFacet) throws Exception { + // we're not using Client.localClient because it doesn't provide a SolrClient to + // use in doTestPrelimSorting -- so instead we make a single node, and don't use any shards param... + final SolrInstances nodes = new SolrInstances(1, "solrconfig-tlog.xml", "schema_latest.xml"); + try { + final Client client = nodes.getClient(random().nextInt()); + client.queryDefaults().set("debugQuery", Boolean.toString(random().nextBoolean()) ); + doTestPrelimSorting(client, extraAgg, extraSubFacet); + } finally { + nodes.stop(); + } + } + + public void testPrelimSortingDistrib() throws Exception { + doTestPrelimSortingDistrib(false, false); + } + + public void testPrelimSortingDistribExtraStat() throws Exception { + doTestPrelimSortingDistrib(true, false); + } + + public void testPrelimSortingDistribExtraFacet() throws Exception { + doTestPrelimSortingDistrib(false, true); + } + + public void testPrelimSortingDistribExtraStatAndFacet() throws Exception { + doTestPrelimSortingDistrib(true, true); + } + + /** @see #doTestPrelimSorting */ + public void doTestPrelimSortingDistrib(final boolean extraAgg, final boolean extraSubFacet) throws Exception { + // we only use 2 shards, but we also want to to sanity check code paths if one (additional) shard is empty + final int totalShards = random().nextBoolean() ? 2 : 3; + final SolrInstances nodes = new SolrInstances(totalShards, "solrconfig-tlog.xml", "schema_latest.xml"); + try { + final Client client = nodes.getClient(random().nextInt()); + client.queryDefaults().set( "shards", nodes.getShards(), + "debugQuery", Boolean.toString(random().nextBoolean()) ); + doTestPrelimSorting(client, extraAgg, extraSubFacet); + } finally { + nodes.stop(); + } + } + + /** + * Helper method that indexes a fixed set of docs to exactly two of the SolrClients + * involved in the current Client such that each shard is identical for the purposes of simplified + * doc/facet counting/assertions -- if there is only one SolrClient (Client.local) then it sends that + * single shard twice as many docs so the counts/assertions will be consistent. + * + * Note: this test doesn't demonstrate practical uses of prelim_sort. + * The scenerios it tests are actualy fairly absurd, but help to ensure that edge cases are covered. + * + * @param client client to use -- may be local or multishard + * @param extraAgg if an extra aggregation function should be included, this hits slightly diff code paths + * @param extraSubFacet if an extra sub facet should be included, this hits slightly diff code paths + */ + public void doTestPrelimSorting(final Client client, + final boolean extraAgg, + final boolean extraSubFacet) throws Exception { + + client.deleteByQuery("*:*", null); + + List clients = client.getClientProvider().all(); + + // carefully craft two balanced shards (assuming we have at least two) and leave any other shards + // empty to help check the code paths of some shards returning no buckets. + // + // if we are in a single node sitaution, these clients will be the same, and we'll have the same + // total docs in our collection, but the numShardsWithData will be diff + // (which will affect some assertions) + final SolrClient shardA = clients.get(0); + final SolrClient shardB = clients.get(clients.size()-1); + final int numShardsWithData = (shardA == shardB) ? 1 : 2; + + // for simplicity, each foo_s "term" exists on each shard in the same number of docs as it's numeric + // value (so count should be double the term) and bar_i is always 1 per doc (so sum(bar_i) + // should always be the same as count) + int id = 0; + for (int i = 1; i <= 20; i++) { + for (int j = 1; j <= i; j++) { + shardA.add(new SolrInputDocument("id", ""+(++id), "foo_s", "foo_" + i, "bar_i", "1")); + shardB.add(new SolrInputDocument("id", ""+(++id), "foo_s", "foo_" + i, "bar_i", "1")); + } + } + assertEquals(420, id); // sanity check + client.commit(); + DebugAgg.Acc.collectDocs.set(0); + DebugAgg.Acc.collectDocSets.set(0); + + // NOTE: sorting by index can cause some optimizations when using type=enum|stream + // that cause our stat to be collected differently, so we have to account for that when + // looking at DebugAdd collect stats if/when the test framework picks those + // ...BUT... this only affects cloud, for single node prelim_sort overrides streaming + final boolean indexSortDebugAggFudge = ( 1 < numShardsWithData ) && + (FacetField.FacetMethod.DEFAULT_METHOD.equals(FacetField.FacetMethod.STREAM) || + FacetField.FacetMethod.DEFAULT_METHOD.equals(FacetField.FacetMethod.ENUM)); + + + final String common = "refine:true, type:field, field:'foo_s', facet: { " + + "x: 'debug(wrap,sum(bar_i))' " + + (extraAgg ? ", y:'min(bar_i)'" : "") + + (extraSubFacet ? ", z:{type:query, q:'bar_i:0'}" : "") + + "}"; + final String yz = (extraAgg ? "y:1, " : "") + (extraSubFacet ? "z:{count:0}, " : ""); + + // really basic: top 5 by (prelim_sort) count, (re)sorted by a stat + client.testJQ(params("q", "*:*", "rows", "0", "json.facet" + , "{ foo_a:{ "+ common+", limit:5, overrequest:0, " + + " prelim_sort:'count desc', sort:'x asc' }" + + " foo_b:{ "+ common+", limit:5, overrequest:0, " + + " prelim_sort:'count asc', sort:'x desc' } }") + , "facets=={ 'count':420, " + + " 'foo_a':{ 'buckets':[" + + " { val:foo_16, count:32, " + yz + "x:32.0}," + + " { val:foo_17, count:34, " + yz + "x:34.0}," + + " { val:foo_18, count:36, " + yz + "x:36.0}," + + " { val:foo_19, count:38, " + yz + "x:38.0}," + + " { val:foo_20, count:40, " + yz + "x:40.0}," + + "] }," + + " 'foo_b':{ 'buckets':[" + + " { val:foo_5, count:10, " + yz + "x:10.0}," + + " { val:foo_4, count:8, " + yz + "x:8.0}," + + " { val:foo_3, count:6, " + yz + "x:6.0}," + + " { val:foo_2, count:4, " + yz + "x:4.0}," + + " { val:foo_1, count:2, " + yz + "x:2.0}," + + "] }," + + "}" + ); + // (re)sorting should prevent 'sum(bar_i)' from being computed for every doc + // only the choosen buckets should be collected (as a set) once per node... + assertEqualsAndReset(0, DebugAgg.Acc.collectDocs); + // 2 facets, 5 bucket, on each shard + assertEqualsAndReset(numShardsWithData * 2 * 5, DebugAgg.Acc.collectDocSets); + + { // same really basic top 5 by (prelim_sort) count, (re)sorted by a stat -- w/allBuckets:true + // check code paths with and w/o allBuckets + // NOTE: allBuckets includes stats, but not other sub-facets... + final String aout = "allBuckets:{ count:420, "+ (extraAgg ? "y:1, " : "") + "x:420.0 }"; + client.testJQ(params("q", "*:*", "rows", "0", "json.facet" + , "{ foo_a:{ " + common+", allBuckets:true, limit:5, overrequest:0, " + + " prelim_sort:'count desc', sort:'x asc' }" + + " foo_b:{ " + common+", allBuckets:true, limit:5, overrequest:0, " + + " prelim_sort:'count asc', sort:'x desc' } }") + , "facets=={ 'count':420, " + + " 'foo_a':{ " + aout + " 'buckets':[" + + " { val:foo_16, count:32, " + yz + "x:32.0}," + + " { val:foo_17, count:34, " + yz + "x:34.0}," + + " { val:foo_18, count:36, " + yz + "x:36.0}," + + " { val:foo_19, count:38, " + yz + "x:38.0}," + + " { val:foo_20, count:40, " + yz + "x:40.0}," + + "] }," + + " 'foo_b':{ " + aout + " 'buckets':[" + + " { val:foo_5, count:10, " + yz + "x:10.0}," + + " { val:foo_4, count:8, " + yz + "x:8.0}," + + " { val:foo_3, count:6, " + yz + "x:6.0}," + + " { val:foo_2, count:4, " + yz + "x:4.0}," + + " { val:foo_1, count:2, " + yz + "x:2.0}," + + "] }," + + "}" + ); + // because of allBuckets, we collect every doc on everyshard (x2 facets) in a single "all" slot... + assertEqualsAndReset(2 * 420, DebugAgg.Acc.collectDocs); + // ... in addition to collecting each of the choosen buckets (as sets) once per node... + // 2 facets, 5 bucket, on each shard + assertEqualsAndReset(numShardsWithData * 2 * 5, DebugAgg.Acc.collectDocSets); + } + + // pagination (with offset) should happen against the re-sorted list (up to the effective limit) + client.testJQ(params("q", "*:*", "rows", "0", "json.facet" + , "{ foo_a:{ "+common+", offset:2, limit:3, overrequest:0, " + + " prelim_sort:'count desc', sort:'x asc' }" + + " foo_b:{ "+common+", offset:2, limit:3, overrequest:0, " + + " prelim_sort:'count asc', sort:'x desc' } }") + , "facets=={ 'count':420, " + + " 'foo_a':{ 'buckets':[" + + " { val:foo_18, count:36, " + yz + "x:36.0}," + + " { val:foo_19, count:38, " + yz + "x:38.0}," + + " { val:foo_20, count:40, " + yz + "x:40.0}," + + "] }," + + " 'foo_b':{ 'buckets':[" + + " { val:foo_3, count:6, " + yz + "x:6.0}," + + " { val:foo_2, count:4, " + yz + "x:4.0}," + + " { val:foo_1, count:2, " + yz + "x:2.0}," + + "] }," + + "}" + ); + assertEqualsAndReset(0, DebugAgg.Acc.collectDocs); + // 2 facets, 5 buckets (including offset), on each shard + assertEqualsAndReset(numShardsWithData * 2 * 5, DebugAgg.Acc.collectDocSets); + + // when overrequesting is used, the full list of candidate buckets should be considered + client.testJQ(params("q", "*:*", "rows", "0", "json.facet" + , "{ foo_a:{ "+common+", limit:5, overrequest:5, " + + " prelim_sort:'count desc', sort:'x asc' }" + + " foo_b:{ "+common+", limit:5, overrequest:5, " + + " prelim_sort:'count asc', sort:'x desc' } }") + , "facets=={ 'count':420, " + + " 'foo_a':{ 'buckets':[" + + " { val:foo_11, count:22, " + yz + "x:22.0}," + + " { val:foo_12, count:24, " + yz + "x:24.0}," + + " { val:foo_13, count:26, " + yz + "x:26.0}," + + " { val:foo_14, count:28, " + yz + "x:28.0}," + + " { val:foo_15, count:30, " + yz + "x:30.0}," + + "] }," + + " 'foo_b':{ 'buckets':[" + + " { val:foo_10, count:20, " + yz + "x:20.0}," + + " { val:foo_9, count:18, " + yz + "x:18.0}," + + " { val:foo_8, count:16, " + yz + "x:16.0}," + + " { val:foo_7, count:14, " + yz + "x:14.0}," + + " { val:foo_6, count:12, " + yz + "x:12.0}," + + "] }," + + "}" + ); + assertEqualsAndReset(0, DebugAgg.Acc.collectDocs); + // 2 facets, 10 buckets (including overrequest), on each shard + assertEqualsAndReset(numShardsWithData * 2 * 10, DebugAgg.Acc.collectDocSets); + + { // for an (effectively) unlimited facet, then from the black box perspective of the client, + // preliminary sorting should be completely ignored... + final StringBuilder expected = new StringBuilder("facets=={ 'count':420, 'foo_a':{ 'buckets':[\n"); + for (int i = 20; 0 < i; i--) { + final int x = i * 2; + expected.append("{ val:foo_"+i+", count:"+x+", " + yz + "x:"+x+".0},\n"); + } + expected.append("] } }"); + for (int limit : Arrays.asList(-1, 100000)) { + for (String sortOpts : Arrays.asList("sort:'x desc'", + "prelim_sort:'count asc', sort:'x desc'", + "prelim_sort:'index asc', sort:'x desc'")) { + final String snippet = "limit: " + limit + ", " + sortOpts; + client.testJQ(params("q", "*:*", "rows", "0", "json.facet" + , "{ foo_a:{ "+common+", " + snippet + "}}") + , expected.toString()); + + // the only difference from a white box perspective, is when/if we are + // optimized to use the sort SlotAcc during collection instead of the prelim_sort SlotAcc.. + // (ie: sub facet preventing single pass (re)sort in single node mode) + if (((0 < limit || extraSubFacet) && snippet.contains("prelim_sort")) && + ! (indexSortDebugAggFudge && snippet.contains("index asc"))) { + // by-pass single pass collection, do everything as sets... + assertEqualsAndReset(snippet, numShardsWithData * 20, DebugAgg.Acc.collectDocSets); + assertEqualsAndReset(snippet, 0, DebugAgg.Acc.collectDocs); + } else { // simple sort on x, or optimized single pass (re)sort, or indexSortDebugAggFudge + // no sets should have been (post) collected for our stat + assertEqualsAndReset(snippet, 0, DebugAgg.Acc.collectDocSets); + // every doc should be collected... + assertEqualsAndReset(snippet, 420, DebugAgg.Acc.collectDocs); + } + } + } + } + + // test all permutations of (prelim_sort | sort) on (index | count | stat) since there are + // custom sort codepaths for index & count that work differnetly then general stats + // + // NOTE: there's very little value in re-sort by count/index after prelim_sort on something more complex, + // typically better to just ignore the prelim_sort, but we're testing it for completeness + // (and because you *might* want to prelim_sort by some function, for the purpose of "sampling" the + // top results and then (re)sorting by count/index) + for (String numSort : Arrays.asList("count", "x")) { // equivilent ordering + client.testJQ(params("q", "*:*", "rows", "0", "json.facet" + , "{ foo_a:{ "+common+", limit:10, overrequest:0, " + + " prelim_sort:'"+numSort+" asc', sort:'index desc' }" + + " foo_b:{ "+common+", limit:10, overrequest:0, " + + " prelim_sort:'index asc', sort:'"+numSort+" desc' } }") + , "facets=={ 'count':420, " + + " 'foo_a':{ 'buckets':[" + + " { val:foo_9, count:18, " + yz + "x:18.0}," + + " { val:foo_8, count:16, " + yz + "x:16.0}," + + " { val:foo_7, count:14, " + yz + "x:14.0}," + + " { val:foo_6, count:12, " + yz + "x:12.0}," + + " { val:foo_5, count:10, " + yz + "x:10.0}," + + " { val:foo_4, count:8, " + yz + "x:8.0}," + + " { val:foo_3, count:6, " + yz + "x:6.0}," + + " { val:foo_2, count:4, " + yz + "x:4.0}," + + " { val:foo_10, count:20, " + yz + "x:20.0}," + + " { val:foo_1, count:2, " + yz + "x:2.0}," + + "] }," + + " 'foo_b':{ 'buckets':[" + + " { val:foo_18, count:36, " + yz + "x:36.0}," + + " { val:foo_17, count:34, " + yz + "x:34.0}," + + " { val:foo_16, count:32, " + yz + "x:32.0}," + + " { val:foo_15, count:30, " + yz + "x:30.0}," + + " { val:foo_14, count:28, " + yz + "x:28.0}," + + " { val:foo_13, count:26, " + yz + "x:26.0}," + + " { val:foo_12, count:24, " + yz + "x:24.0}," + + " { val:foo_11, count:22, " + yz + "x:22.0}," + + " { val:foo_10, count:20, " + yz + "x:20.0}," + + " { val:foo_1, count:2, " + yz + "x:2.0}," + + "] }," + + "}" + ); + // since these behave differently, defer DebugAgg counter checks until all are done... + } + // These 3 permutations defer the compuation of x as docsets, + // so it's 3 x (10 buckets on each shard) (but 0 direct docs) + // prelim_sort:count, sort:index + // prelim_sort:index, sort:x + // prelim_sort:index, sort:count + // ...except when streaming, prelim_sort:index does no docsets. + assertEqualsAndReset((indexSortDebugAggFudge ? 1 : 3) * numShardsWithData * 10, + DebugAgg.Acc.collectDocSets); + // This is the only situation that should (always) result in every doc being collected (but 0 docsets)... + // prelim_sort:x, sort:index + // ...but the (2) prelim_sort:index streaming situations above will also cause all the docs in the first + // 10+1 buckets to be collected (enum checks limit+1 to know if there are "more"... + assertEqualsAndReset(420 + (indexSortDebugAggFudge ? + 2 * numShardsWithData * (1+10+11+12+13+14+15+16+17+18+19) : 0), + DebugAgg.Acc.collectDocs); + + // sanity check of prelim_sorting in a sub facet + client.testJQ(params("q", "*:*", "rows", "0", "json.facet" + , "{ bar:{ type:query, query:'foo_s:[foo_10 TO foo_19]', facet: {" + + " foo:{ "+ common+", limit:5, overrequest:0, " + + " prelim_sort:'count desc', sort:'x asc' } } } }") + , "facets=={ 'count':420, " + + " 'bar':{ 'count':290, " + + " 'foo':{ 'buckets':[" + + " { val:foo_15, count:30, " + yz + "x:30.0}," + + " { val:foo_16, count:32, " + yz + "x:32.0}," + + " { val:foo_17, count:34, " + yz + "x:34.0}," + + " { val:foo_18, count:36, " + yz + "x:36.0}," + + " { val:foo_19, count:38, " + yz + "x:38.0}," + + " ] }," + + " }," + + "}" + ); + // the prelim_sort should prevent 'sum(bar_i)' from being computed for every doc + // only the choosen buckets should be collected (as a set) once per node... + assertEqualsAndReset(0, DebugAgg.Acc.collectDocs); + // 5 bucket, on each shard + assertEqualsAndReset(numShardsWithData * 5, DebugAgg.Acc.collectDocSets); + + { // sanity check how defered stats are handled + + // here we'll prelim_sort & sort on things that are both "not x" and using the debug() counters + // (wrapping x) to assert that 'x' is correctly defered and only collected for the final top buckets + final List sorts = new ArrayList(Arrays.asList("index asc", "count asc")); + if (extraAgg) { + sorts.add("y asc"); // same for every bucket, but index order tie breaker should kick in + } + for (String s : sorts) { + client.testJQ(params("q", "*:*", "rows", "0", "json.facet" + , "{ foo:{ "+ common+", limit:5, overrequest:0, " + + " prelim_sort:'count desc', sort:'"+s+"' } }") + , "facets=={ 'count':420, " + + " 'foo':{ 'buckets':[" + + " { val:foo_16, count:32, " + yz + "x:32.0}," + + " { val:foo_17, count:34, " + yz + "x:34.0}," + + " { val:foo_18, count:36, " + yz + "x:36.0}," + + " { val:foo_19, count:38, " + yz + "x:38.0}," + + " { val:foo_20, count:40, " + yz + "x:40.0}," + + "] } }" + ); + // Neither prelim_sort nor sort should need 'sum(bar_i)' to be computed for every doc + // only the choosen buckets should be collected (as a set) once per node... + assertEqualsAndReset(0, DebugAgg.Acc.collectDocs); + // 5 bucket, on each shard + assertEqualsAndReset(numShardsWithData * 5, DebugAgg.Acc.collectDocSets); + } + } + } + + @Test public void testOverrequest() throws Exception { initServers(); @@ -2796,4 +3249,16 @@ public class TestJsonFacets extends SolrTestCaseHS { hll.addRaw(987654321); } + + /** atomicly resets the acctual AtomicLong value matches the expected and resets it to 0 */ + private static final void assertEqualsAndReset(String msg, long expected, AtomicLong actual) { + final long current = actual.getAndSet(0); + assertEquals(msg, expected, current); + } + /** atomicly resets the acctual AtomicLong value matches the expected and resets it to 0 */ + private static final void assertEqualsAndReset(long expected, AtomicLong actual) { + final long current = actual.getAndSet(0); + assertEquals(expected, current); + } + } diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetsStatsParsing.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetsStatsParsing.java index 438c545a3f1..2fe3f8d766c 100644 --- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetsStatsParsing.java +++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetsStatsParsing.java @@ -24,6 +24,7 @@ import org.apache.lucene.queries.function.valuesource.IntFieldSource; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.request.SolrQueryRequest; import org.junit.BeforeClass; +import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.core.IsInstanceOf.instanceOf; import org.noggit.ObjectBuilder; @@ -36,6 +37,20 @@ public class TestJsonFacetsStatsParsing extends SolrTestCaseJ4 { initCore("solrconfig-tlog.xml","schema15.xml"); } + public void testSortEquality() throws Exception { + assertEquals(new FacetRequest.FacetSort("count", FacetRequest.SortDirection.desc), + FacetRequest.FacetSort.COUNT_DESC); + assertEquals(new FacetRequest.FacetSort("index", FacetRequest.SortDirection.asc), + FacetRequest.FacetSort.INDEX_ASC); + assertEquals(new FacetRequest.FacetSort("foo", FacetRequest.SortDirection.asc), + new FacetRequest.FacetSort("foo", FacetRequest.SortDirection.asc)); + // negative assertions... + assertThat(new FacetRequest.FacetSort("foo", FacetRequest.SortDirection.desc), + not(new FacetRequest.FacetSort("foo", FacetRequest.SortDirection.asc))); + assertThat(new FacetRequest.FacetSort("bar", FacetRequest.SortDirection.desc), + not(new FacetRequest.FacetSort("foo", FacetRequest.SortDirection.desc))); + } + public void testEquality() throws IOException { try (SolrQueryRequest req = req("custom_req_param","foo_i", "overridden_param","xxxxx_i")) { diff --git a/solr/solr-ref-guide/src/json-facet-api.adoc b/solr/solr-ref-guide/src/json-facet-api.adoc index fd40538e8b0..87e44875431 100644 --- a/solr/solr-ref-guide/src/json-facet-api.adoc +++ b/solr/solr-ref-guide/src/json-facet-api.adoc @@ -218,7 +218,9 @@ json.facet={ |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. -|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 <> 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” +|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 <> 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. @@ -248,6 +250,7 @@ This parameter indicates the facet algorithm to use: * "stream" Presently equivalent to "enum" * "smart" Pick the best method for the field type (this is the default) +|prelim_sort |An optional parameter for specifying an approximation of the final `sort` to use during initial collection of top buckets when the <>. |=== == Query Facet @@ -532,13 +535,13 @@ By default "top authors" is defined by simple document count descending, but we === Sorting Facets By Nested Functions -The default sort for a field or terms facet is by bucket count descending. We can optionally sort ascending or descending by any facet function that appears in each bucket. +The default sort for a field or terms facet is by bucket count descending. We can optionally `sort` ascending or descending by any facet function that appears in each bucket. [source,java] ---- { categories:{ - type : terms // terms facet creates a bucket for each indexed term in the field + type : terms, // terms facet creates a bucket for each indexed term in the field field : cat, sort : "x desc", // can also use sort:{x:desc} facet : { @@ -549,6 +552,28 @@ The default sort for a field or terms facet is by bucket count descending. We ca } ---- +In some situations the desired `sort` may be an aggregation function that is very costly to compute for every bucket. A `prelim_sort` option can be used to specify an approximation of the `sort`, for initially ranking the buckets to determine the top candidates (based on the `limit` and `overrequest`). Only after the top candidate buckets have been refined, will the actual `sort` be used. + +[source,java] +---- +{ + categories:{ + type : terms, + field : cat, + refine: true, + limit: 10, + overrequest: 100, + prelim_sort: "sales_rank desc", + sort : "prod_quality desc", + facet : { + prod_quality : "avg(div(prod(rating,sales_rank),prod(num_returns,price)))" + sales_rank : "sum(sales_rank)" + } + } +} +---- + + == Changing the Domain As discussed above, facets compute buckets or statistics based on a "domain" which is typically implicit: @@ -805,6 +830,11 @@ When using the extended `type:func` syntax for specifying a `relatedness()` aggr This can be particularly useful when using a descending sorting on `relatedness()` with foreground and background queries that are disjoint, to ensure the "top buckets" are all relevant to both sets. +[TIP] +==== +When sorting on `relatedness(...)` requests can be processed much more quickly by adding a `prelim_sort: "count desc"` option. Increasing the `overrequest` can help improve the accuracy of the top buckets. +==== + === Semantic Knowledge Graph Example .Sample Documents