mirror of https://github.com/apache/lucene.git
SOLR-9142: json.facet: new method=dvhash which works on terms. Also:
(1) method=stream now requires you set sort=index asc to work (2) faceting on numerics with prefix or mincount=0 will give you an error (3) refactored similar findTopSlots into one common one in FacetFieldProcessor (4) new DocSet.collectSortedDocSet utility
This commit is contained in:
parent
23825b2480
commit
7b5df8a103
|
@ -66,6 +66,15 @@ Jetty 9.3.8.v20160314
|
|||
Detailed Change List
|
||||
----------------------
|
||||
|
||||
Upgrade Notes
|
||||
----------------------
|
||||
|
||||
* If you use the JSON Facet API (json.facet) with method=stream, you must now set sort='index asc' to get the streaming
|
||||
behavior; otherwise it won't stream. Reminder: "method" is a hint that doesn't change defaults of other parameters.
|
||||
|
||||
* If you use the JSON Facet API (json.facet) to facet on a numeric field and if you use mincount=0 or if you set the
|
||||
prefix, then you will now get an error as these options are incompatible with numeric faceting.
|
||||
|
||||
New Features
|
||||
----------------------
|
||||
|
||||
|
@ -105,6 +114,8 @@ Optimizations
|
|||
|
||||
* SOLR-9452: JsonRecordReader should not deep copy document before handler.handle(). (noble, shalin)
|
||||
|
||||
* SOLR-9142: JSON Facet API: new method=dvhash can be chosen for fields with high cardinality. (David Smiley)
|
||||
|
||||
Other Changes
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -17,10 +17,12 @@
|
|||
package org.apache.solr.search;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.PostingsEnum;
|
||||
|
@ -29,7 +31,9 @@ import org.apache.lucene.index.Terms;
|
|||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.Collector;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.LeafCollector;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -208,4 +212,33 @@ public class DocSetUtil {
|
|||
return new SortedIntDocSet(docs);
|
||||
}
|
||||
|
||||
public static void collectSortedDocSet(DocSet docs, IndexReader reader, Collector collector) throws IOException {
|
||||
// TODO add SortedDocSet sub-interface and take that.
|
||||
// TODO collectUnsortedDocSet: iterate segment, then all docSet per segment.
|
||||
|
||||
final List<LeafReaderContext> leaves = reader.leaves();
|
||||
final Iterator<LeafReaderContext> ctxIt = leaves.iterator();
|
||||
int segBase = 0;
|
||||
int segMax;
|
||||
int adjustedMax = 0;
|
||||
LeafReaderContext ctx = null;
|
||||
LeafCollector leafCollector = null;
|
||||
for (DocIterator docsIt = docs.iterator(); docsIt.hasNext(); ) {
|
||||
final int doc = docsIt.nextDoc();
|
||||
if (doc >= adjustedMax) {
|
||||
do {
|
||||
ctx = ctxIt.next();
|
||||
segBase = ctx.docBase;
|
||||
segMax = ctx.reader().maxDoc();
|
||||
adjustedMax = segBase + segMax;
|
||||
} while (doc >= adjustedMax);
|
||||
leafCollector = collector.getLeafCollector(ctx);
|
||||
}
|
||||
if (doc < segBase) {
|
||||
throw new IllegalStateException("algorithm expects sorted DocSet but wasn't: " + docs.getClass());
|
||||
}
|
||||
leafCollector.collect(doc - segBase); // per-seg collectors
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -66,28 +66,29 @@ public class FacetField extends FacetRequestSorted {
|
|||
}
|
||||
|
||||
public enum FacetMethod {
|
||||
DV, // DocValues
|
||||
UIF, // UnInvertedField
|
||||
ENUM,
|
||||
STREAM,
|
||||
DV, // DocValues, collect into ordinal array
|
||||
UIF, // UnInvertedField, collect into ordinal array
|
||||
DVHASH, // DocValues, collect into hash
|
||||
ENUM, // TermsEnum then intersect DocSet (stream-able)
|
||||
STREAM, // presently equivalent to ENUM
|
||||
SMART,
|
||||
;
|
||||
|
||||
public static FacetMethod fromString(String method) {
|
||||
if (method == null || method.length()==0) return null;
|
||||
if ("dv".equals(method)) {
|
||||
return DV;
|
||||
} else if ("uif".equals(method)) {
|
||||
return UIF;
|
||||
} else if ("enum".equals(method)) {
|
||||
return ENUM;
|
||||
} else if ("smart".equals(method)) {
|
||||
return SMART;
|
||||
} else if ("stream".equals(method)) {
|
||||
return STREAM;
|
||||
if (method == null || method.length()==0) return DEFAULT_METHOD;
|
||||
switch (method) {
|
||||
case "dv": return DV;
|
||||
case "uif": return UIF;
|
||||
case "dvhash": return DVHASH;
|
||||
case "enum": return ENUM;
|
||||
case "stream": return STREAM; // TODO replace with enum?
|
||||
case "smart": return SMART;
|
||||
default:
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown FacetField method " + method);
|
||||
}
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown FacetField method " + method);
|
||||
}
|
||||
|
||||
static FacetMethod DEFAULT_METHOD = SMART; // non-final for tests to vary
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -96,21 +97,42 @@ public class FacetField extends FacetRequestSorted {
|
|||
FieldType ft = sf.getType();
|
||||
boolean multiToken = sf.multiValued() || ft.multiValuedFieldCache();
|
||||
|
||||
if (method == FacetMethod.ENUM && sf.indexed()) {
|
||||
throw new UnsupportedOperationException();
|
||||
} else if (method == FacetMethod.STREAM && sf.indexed()) {
|
||||
LegacyNumericType ntype = ft.getNumericType();
|
||||
// ensure we can support the requested options for numeric faceting:
|
||||
if (ntype != null) {
|
||||
if (prefix != null) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
"Doesn't make sense to set facet prefix on a numeric field");
|
||||
}
|
||||
if (mincount == 0) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
"Numeric fields do not support facet mincount=0; try indexing as terms");
|
||||
// TODO if indexed=true then we could add support
|
||||
}
|
||||
}
|
||||
|
||||
// TODO auto-pick ENUM/STREAM SOLR-9351 when index asc and DocSet cardinality is *not* much smaller than term cardinality
|
||||
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) {
|
||||
return new FacetFieldProcessorByEnumTermsStream(fcontext, this, sf);
|
||||
}
|
||||
|
||||
LegacyNumericType ntype = ft.getNumericType();
|
||||
// TODO if method=UIF and not single-valued numerics then simply choose that now? TODO add FieldType.getDocValuesType()
|
||||
|
||||
if (!multiToken) {
|
||||
if (ntype != null) {
|
||||
// single valued numeric (docvalues or fieldcache)
|
||||
if (mincount > 0 && prefix == null && (ntype != null || method == FacetMethod.DVHASH)) {
|
||||
// TODO can we auto-pick for strings when term cardinality is much greater than DocSet cardinality?
|
||||
// or if we don't know cardinality but DocSet size is very small
|
||||
return new FacetFieldProcessorByHashNumeric(fcontext, this, sf);
|
||||
} else {
|
||||
} else if (ntype == null) {
|
||||
// single valued string...
|
||||
return new FacetFieldProcessorByArrayDV(fcontext, this, sf);
|
||||
} else {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
|
||||
"Couldn't pick facet algorithm for field " + sf);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -18,12 +18,18 @@
|
|||
package org.apache.solr.search.facet;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.BiPredicate;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.IntFunction;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
import org.apache.solr.schema.SchemaField;
|
||||
import org.apache.solr.search.DocSet;
|
||||
|
@ -57,7 +63,7 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
|
|||
this.effectiveMincount = (int)(fcontext.isShard() ? Math.min(1 , freq.mincount) : freq.mincount);
|
||||
}
|
||||
|
||||
// This is used to create accs for second phase (or to create accs for all aggs)
|
||||
/** This is used to create accs for second phase (or to create accs for all aggs) */
|
||||
@Override
|
||||
protected void createAccs(int docCount, int slotCount) throws IOException {
|
||||
if (accMap == null) {
|
||||
|
@ -195,7 +201,140 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
|
|||
}
|
||||
}
|
||||
|
||||
void fillBucket(SimpleOrderedMap<Object> target, int count, int slotNum, DocSet subDomain, Query filter) throws IOException {
|
||||
/** Processes the collected data to finds the top slots, and composes it in the response NamedList. */
|
||||
SimpleOrderedMap<Object> findTopSlots(final int numSlots, final int slotCardinality,
|
||||
IntFunction<Comparable> bucketValFromSlotNumFunc,
|
||||
Function<Comparable, String> fieldQueryValFunc) throws IOException {
|
||||
int numBuckets = 0;
|
||||
List<Object> bucketVals = null;
|
||||
if (freq.numBuckets && fcontext.isShard()) {
|
||||
bucketVals = new ArrayList<>(100);
|
||||
}
|
||||
|
||||
final int off = fcontext.isShard() ? 0 : (int) freq.offset;
|
||||
// add a modest amount of over-request if this is a shard request
|
||||
final int lim = freq.limit >= 0 ? (fcontext.isShard() ? (int)(freq.limit*1.1+4) : (int)freq.limit) : Integer.MAX_VALUE;
|
||||
|
||||
final int sortMul = freq.sortDirection.getMultiplier();
|
||||
|
||||
int maxTopVals = (int) (lim >= 0 ? (long) off + lim : Integer.MAX_VALUE - 1);
|
||||
maxTopVals = Math.min(maxTopVals, slotCardinality);
|
||||
final SlotAcc sortAcc = this.sortAcc, indexOrderAcc = this.indexOrderAcc;
|
||||
final BiPredicate<Slot,Slot> orderPredicate;
|
||||
if (indexOrderAcc != null && indexOrderAcc != sortAcc) {
|
||||
orderPredicate = (a, b) -> {
|
||||
int cmp = sortAcc.compare(a.slot, b.slot) * sortMul;
|
||||
return cmp == 0 ? (indexOrderAcc.compare(a.slot, b.slot) > 0) : cmp < 0;
|
||||
};
|
||||
} else {
|
||||
orderPredicate = (a, b) -> {
|
||||
int cmp = sortAcc.compare(a.slot, b.slot) * sortMul;
|
||||
return cmp == 0 ? b.slot < a.slot : cmp < 0;
|
||||
};
|
||||
}
|
||||
final PriorityQueue<Slot> queue = new PriorityQueue<Slot>(maxTopVals) {
|
||||
@Override
|
||||
protected boolean lessThan(Slot a, Slot b) { return orderPredicate.test(a, b); }
|
||||
};
|
||||
|
||||
// note: We avoid object allocation by having a Slot and re-using the 'bottom'.
|
||||
Slot bottom = null;
|
||||
Slot scratchSlot = new Slot();
|
||||
for (int slotNum = 0; slotNum < numSlots; slotNum++) {
|
||||
// screen out buckets not matching mincount immediately (i.e. don't even increment numBuckets)
|
||||
if (effectiveMincount > 0 && countAcc.getCount(slotNum) < effectiveMincount) {
|
||||
continue;
|
||||
}
|
||||
|
||||
numBuckets++;
|
||||
if (bucketVals != null && bucketVals.size()<100) {
|
||||
Object val = bucketValFromSlotNumFunc.apply(slotNum);
|
||||
bucketVals.add(val);
|
||||
}
|
||||
|
||||
if (bottom != null) {
|
||||
scratchSlot.slot = slotNum; // scratchSlot is only used to hold this slotNum for the following line
|
||||
if (orderPredicate.test(bottom, scratchSlot)) {
|
||||
bottom.slot = slotNum;
|
||||
bottom = queue.updateTop();
|
||||
}
|
||||
} else if (lim > 0) {
|
||||
// queue not full
|
||||
Slot s = new Slot();
|
||||
s.slot = slotNum;
|
||||
queue.add(s);
|
||||
if (queue.size() >= maxTopVals) {
|
||||
bottom = queue.top();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
assert queue.size() <= numBuckets;
|
||||
|
||||
SimpleOrderedMap<Object> res = new SimpleOrderedMap<>();
|
||||
if (freq.numBuckets) {
|
||||
if (!fcontext.isShard()) {
|
||||
res.add("numBuckets", numBuckets);
|
||||
} else {
|
||||
SimpleOrderedMap<Object> map = new SimpleOrderedMap<>(2);
|
||||
map.add("numBuckets", numBuckets);
|
||||
map.add("vals", bucketVals);
|
||||
res.add("numBuckets", map);
|
||||
}
|
||||
}
|
||||
|
||||
FacetDebugInfo fdebug = fcontext.getDebugInfo();
|
||||
if (fdebug != null) fdebug.putInfoItem("numBuckets", (long) numBuckets);
|
||||
|
||||
if (freq.allBuckets) {
|
||||
SimpleOrderedMap<Object> allBuckets = new SimpleOrderedMap<>();
|
||||
// countAcc.setValues(allBuckets, allBucketsSlot);
|
||||
allBuckets.add("count", allBucketsAcc.getSpecialCount());
|
||||
allBucketsAcc.setValues(allBuckets, -1); // -1 slotNum is unused for SpecialSlotAcc
|
||||
// allBuckets currently doesn't execute sub-facets (because it doesn't change the domain?)
|
||||
res.add("allBuckets", allBuckets);
|
||||
}
|
||||
|
||||
if (freq.missing) {
|
||||
// TODO: it would be more efficient to build up a missing DocSet if we need it here anyway.
|
||||
SimpleOrderedMap<Object> missingBucket = new SimpleOrderedMap<>();
|
||||
fillBucket(missingBucket, getFieldMissingQuery(fcontext.searcher, freq.field), null);
|
||||
res.add("missing", missingBucket);
|
||||
}
|
||||
|
||||
// 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 <= lim;
|
||||
int[] sortedSlots = new int[collectCount];
|
||||
for (int i = collectCount - 1; i >= 0; i--) {
|
||||
sortedSlots[i] = queue.pop().slot;
|
||||
}
|
||||
|
||||
ArrayList<SimpleOrderedMap> bucketList = new ArrayList<>(collectCount);
|
||||
res.add("buckets", bucketList);
|
||||
|
||||
boolean needFilter = deferredAggs != null || freq.getSubFacets().size() > 0;
|
||||
|
||||
for (int slotNum : sortedSlots) {
|
||||
SimpleOrderedMap<Object> bucket = new SimpleOrderedMap<>();
|
||||
Comparable val = bucketValFromSlotNumFunc.apply(slotNum);
|
||||
bucket.add("val", val);
|
||||
|
||||
Query filter = needFilter ? sf.getType().getFieldQuery(null, sf, fieldQueryValFunc.apply(val)) : null;
|
||||
|
||||
fillBucket(bucket, countAcc.getCount(slotNum), slotNum, null, filter);
|
||||
|
||||
bucketList.add(bucket);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
private static class Slot {
|
||||
int slot;
|
||||
}
|
||||
|
||||
private void fillBucket(SimpleOrderedMap<Object> target, int count, int slotNum, DocSet subDomain, Query filter) throws IOException {
|
||||
target.add("count", count);
|
||||
if (count <= 0 && !freq.processEmpty) return;
|
||||
|
||||
|
@ -272,13 +411,6 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
|
|||
}
|
||||
}
|
||||
|
||||
static class Slot {
|
||||
int slot;
|
||||
public int tiebreakCompare(int slotA, int slotB) {
|
||||
return slotB - slotA;
|
||||
}
|
||||
}
|
||||
|
||||
static class SpecialSlotAcc extends SlotAcc {
|
||||
SlotAcc collectAcc;
|
||||
SlotAcc[] otherAccs;
|
||||
|
|
|
@ -18,19 +18,15 @@
|
|||
package org.apache.solr.search.facet;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
import org.apache.solr.schema.SchemaField;
|
||||
|
||||
/**
|
||||
* Base class for DV/UIF accumulating counts into an array by ordinal.
|
||||
* Base class for DV/UIF accumulating counts into an array by ordinal. It's
|
||||
* for {@link org.apache.lucene.index.SortedDocValues} and {@link org.apache.lucene.index.SortedSetDocValues} only.
|
||||
* It can handle terms (strings), not numbers directly but those encoded as terms, and is multi-valued capable.
|
||||
*/
|
||||
abstract class FacetFieldProcessorByArray extends FacetFieldProcessor {
|
||||
|
@ -57,11 +53,10 @@ abstract class FacetFieldProcessorByArray extends FacetFieldProcessor {
|
|||
@Override
|
||||
public void process() throws IOException {
|
||||
super.process();
|
||||
sf = fcontext.searcher.getSchema().getField(freq.field);
|
||||
response = getFieldCacheCounts();
|
||||
response = calcFacets();
|
||||
}
|
||||
|
||||
private SimpleOrderedMap<Object> getFieldCacheCounts() throws IOException {
|
||||
private SimpleOrderedMap<Object> calcFacets() throws IOException {
|
||||
String prefix = freq.prefix;
|
||||
if (prefix == null || prefix.length() == 0) {
|
||||
prefixRef = null;
|
||||
|
@ -86,128 +81,15 @@ abstract class FacetFieldProcessorByArray extends FacetFieldProcessor {
|
|||
|
||||
collectDocs();
|
||||
|
||||
return findTopSlots();
|
||||
}
|
||||
|
||||
private SimpleOrderedMap<Object> findTopSlots() throws IOException {
|
||||
SimpleOrderedMap<Object> res = new SimpleOrderedMap<>();
|
||||
|
||||
int numBuckets = 0;
|
||||
List<Object> bucketVals = null;
|
||||
if (freq.numBuckets && fcontext.isShard()) {
|
||||
bucketVals = new ArrayList<>(100);
|
||||
}
|
||||
|
||||
int off = fcontext.isShard() ? 0 : (int) freq.offset;
|
||||
// add a modest amount of over-request if this is a shard request
|
||||
int lim = freq.limit >= 0 ? (fcontext.isShard() ? (int)(freq.limit*1.1+4) : (int)freq.limit) : Integer.MAX_VALUE;
|
||||
|
||||
int maxsize = (int)(freq.limit >= 0 ? freq.offset + lim : Integer.MAX_VALUE - 1);
|
||||
maxsize = Math.min(maxsize, nTerms);
|
||||
|
||||
final int sortMul = freq.sortDirection.getMultiplier();
|
||||
final SlotAcc sortAcc = this.sortAcc;
|
||||
|
||||
PriorityQueue<Slot> queue = new PriorityQueue<Slot>(maxsize) {
|
||||
@Override
|
||||
protected boolean lessThan(Slot a, Slot b) {
|
||||
int cmp = sortAcc.compare(a.slot, b.slot) * sortMul;
|
||||
return cmp == 0 ? b.slot < a.slot : cmp < 0;
|
||||
}
|
||||
};
|
||||
|
||||
Slot bottom = null;
|
||||
for (int i = 0; i < nTerms; i++) {
|
||||
// screen out buckets not matching mincount immediately (i.e. don't even increment numBuckets)
|
||||
if (effectiveMincount > 0 && countAcc.getCount(i) < effectiveMincount) {
|
||||
continue;
|
||||
}
|
||||
|
||||
numBuckets++;
|
||||
if (bucketVals != null && bucketVals.size()<100) {
|
||||
int ord = startTermIndex + i;
|
||||
BytesRef br = lookupOrd(ord);
|
||||
Object val = sf.getType().toObject(sf, br);
|
||||
bucketVals.add(val);
|
||||
}
|
||||
|
||||
if (bottom != null) {
|
||||
if (sortAcc.compare(bottom.slot, i) * sortMul < 0) {
|
||||
bottom.slot = i;
|
||||
bottom = queue.updateTop();
|
||||
}
|
||||
} else if (lim > 0) {
|
||||
// queue not full
|
||||
Slot s = new Slot();
|
||||
s.slot = i;
|
||||
queue.add(s);
|
||||
if (queue.size() >= maxsize) {
|
||||
bottom = queue.top();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (freq.numBuckets) {
|
||||
if (!fcontext.isShard()) {
|
||||
res.add("numBuckets", numBuckets);
|
||||
} else {
|
||||
SimpleOrderedMap<Object> map = new SimpleOrderedMap<>(2);
|
||||
map.add("numBuckets", numBuckets);
|
||||
map.add("vals", bucketVals);
|
||||
res.add("numBuckets", map);
|
||||
}
|
||||
}
|
||||
|
||||
FacetDebugInfo fdebug = fcontext.getDebugInfo();
|
||||
if (fdebug != null) fdebug.putInfoItem("numBuckets", (long) numBuckets);
|
||||
|
||||
// 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 <= lim;
|
||||
int[] sortedSlots = new int[collectCount];
|
||||
for (int i = collectCount - 1; i >= 0; i--) {
|
||||
sortedSlots[i] = queue.pop().slot;
|
||||
}
|
||||
|
||||
if (freq.allBuckets) {
|
||||
SimpleOrderedMap<Object> allBuckets = new SimpleOrderedMap<>();
|
||||
allBuckets.add("count", allBucketsAcc.getSpecialCount());
|
||||
if (allBucketsAcc != null) {
|
||||
allBucketsAcc.setValues(allBuckets, allBucketsSlot);
|
||||
}
|
||||
res.add("allBuckets", allBuckets);
|
||||
}
|
||||
|
||||
ArrayList<SimpleOrderedMap<Object>> bucketList = new ArrayList<>(collectCount);
|
||||
res.add("buckets", bucketList);
|
||||
|
||||
// TODO: do this with a callback instead?
|
||||
boolean needFilter = deferredAggs != null || freq.getSubFacets().size() > 0;
|
||||
|
||||
for (int slotNum : sortedSlots) {
|
||||
SimpleOrderedMap<Object> bucket = new SimpleOrderedMap<>();
|
||||
|
||||
// get the ord of the slot...
|
||||
int ord = startTermIndex + slotNum;
|
||||
|
||||
BytesRef br = lookupOrd(ord);
|
||||
Object val = sf.getType().toObject(sf, br);
|
||||
|
||||
bucket.add("val", val);
|
||||
|
||||
TermQuery filter = needFilter ? new TermQuery(new Term(sf.getName(), br)) : null;
|
||||
fillBucket(bucket, countAcc.getCount(slotNum), slotNum, null, filter);
|
||||
|
||||
bucketList.add(bucket);
|
||||
}
|
||||
|
||||
if (freq.missing) {
|
||||
SimpleOrderedMap<Object> missingBucket = new SimpleOrderedMap<>();
|
||||
fillBucket(missingBucket, getFieldMissingQuery(fcontext.searcher, freq.field), null);
|
||||
res.add("missing", missingBucket);
|
||||
}
|
||||
|
||||
return res;
|
||||
return super.findTopSlots(nTerms, nTerms,
|
||||
slotNum -> { // getBucketValFromSlotNum
|
||||
try {
|
||||
return (Comparable) sf.getType().toObject(sf, lookupOrd(slotNum + startTermIndex));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
},
|
||||
Object::toString); // getFieldQueryVal
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -17,25 +17,37 @@
|
|||
package org.apache.solr.search.facet;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.text.ParseException;
|
||||
import java.util.function.IntFunction;
|
||||
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValuesType;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.MultiDocValues;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.search.SimpleCollector;
|
||||
import org.apache.lucene.util.BitUtil;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LongValues;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
import org.apache.solr.schema.SchemaField;
|
||||
import org.apache.solr.search.DocIterator;
|
||||
import org.apache.solr.search.DocSetUtil;
|
||||
|
||||
/**
|
||||
* Facets numbers into a hash table.
|
||||
* It currently only works with {@link NumericDocValues} (single-valued).
|
||||
* Facets numbers into a hash table. The number is either a raw numeric DocValues value, or
|
||||
* a term global ordinal integer.
|
||||
* Limitations:
|
||||
* <ul>
|
||||
* <li>doesn't handle multiValued, but could easily be added</li>
|
||||
* <li>doesn't handle prefix, but could easily be added</li>
|
||||
* <li>doesn't handle mincount==0 -- you're better off with an array alg</li>
|
||||
* </ul>
|
||||
*/
|
||||
// TODO rename: FacetFieldProcessorByHashDV
|
||||
class FacetFieldProcessorByHashNumeric extends FacetFieldProcessor {
|
||||
static int MAXIMUM_STARTING_TABLE_SIZE=1024; // must be a power of two, non-final to support setting by tests
|
||||
|
||||
|
@ -44,7 +56,6 @@ class FacetFieldProcessorByHashNumeric extends FacetFieldProcessor {
|
|||
|
||||
static final float LOAD_FACTOR = 0.7f;
|
||||
|
||||
long numAdds;
|
||||
long[] vals;
|
||||
int[] counts; // maintain the counts here since we need them to tell if there was actually a value anyway
|
||||
int[] oldToNewMapping;
|
||||
|
@ -82,7 +93,6 @@ class FacetFieldProcessorByHashNumeric extends FacetFieldProcessor {
|
|||
rehash();
|
||||
}
|
||||
|
||||
numAdds++;
|
||||
int h = hash(val);
|
||||
for (int slot = h & (vals.length-1); ;slot = (slot + ((h>>7)|1)) & (vals.length-1)) {
|
||||
int count = counts[slot];
|
||||
|
@ -135,29 +145,93 @@ class FacetFieldProcessorByHashNumeric extends FacetFieldProcessor {
|
|||
|
||||
}
|
||||
|
||||
/** A hack instance of Calc for Term ordinals in DocValues. */
|
||||
// TODO consider making FacetRangeProcessor.Calc facet top level; then less of a hack?
|
||||
private class TermOrdCalc extends FacetRangeProcessor.Calc {
|
||||
|
||||
IntFunction<BytesRef> lookupOrdFunction; // set in collectDocs()!
|
||||
|
||||
TermOrdCalc() throws IOException {
|
||||
super(sf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long bitsToSortableBits(long globalOrd) {
|
||||
return globalOrd;
|
||||
}
|
||||
|
||||
/** To be returned in "buckets"/"val" */
|
||||
@Override
|
||||
public Comparable bitsToValue(long globalOrd) {
|
||||
BytesRef bytesRef = lookupOrdFunction.apply((int) globalOrd);
|
||||
// note FacetFieldProcessorByArray.findTopSlots also calls SchemaFieldType.toObject
|
||||
return sf.getType().toObject(sf, bytesRef).toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String formatValue(Comparable val) {
|
||||
return (String) val;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Comparable parseStr(String rawval) throws ParseException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Comparable parseAndAddGap(Comparable value, String gap) throws ParseException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
FacetRangeProcessor.Calc calc;
|
||||
LongCounts table;
|
||||
int allBucketsSlot = -1;
|
||||
|
||||
FacetFieldProcessorByHashNumeric(FacetContext fcontext, FacetField freq, SchemaField sf) {
|
||||
super(fcontext, freq, sf);
|
||||
if (freq.mincount == 0) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
getClass()+" doesn't support mincount=0");
|
||||
}
|
||||
if (freq.prefix != null) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
getClass()+" doesn't support prefix"); // yet, but it could
|
||||
}
|
||||
FieldInfo fieldInfo = fcontext.searcher.getLeafReader().getFieldInfos().fieldInfo(sf.getName());
|
||||
if (fieldInfo != null &&
|
||||
fieldInfo.getDocValuesType() != DocValuesType.NUMERIC &&
|
||||
fieldInfo.getDocValuesType() != DocValuesType.SORTED) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
getClass()+" only support single valued number/string with docValues");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process() throws IOException {
|
||||
super.process();
|
||||
response = calcFacets();
|
||||
table = null;//gc
|
||||
}
|
||||
|
||||
private SimpleOrderedMap<Object> calcFacets() throws IOException {
|
||||
|
||||
final FacetRangeProcessor.Calc calc = FacetRangeProcessor.getNumericCalc(sf);
|
||||
if (sf.getType().getNumericType() != null) {
|
||||
calc = FacetRangeProcessor.getNumericCalc(sf);
|
||||
} else {
|
||||
calc = new TermOrdCalc(); // kind of a hack
|
||||
}
|
||||
|
||||
// TODO: it would be really nice to know the number of unique values!!!!
|
||||
// TODO: Use the number of indexed terms, if present, as an estimate!
|
||||
// Even for NumericDocValues, we could check for a terms index for an estimate.
|
||||
// Our estimation should aim high to avoid expensive rehashes.
|
||||
|
||||
int possibleValues = fcontext.base.size();
|
||||
// size smaller tables so that no resize will be necessary
|
||||
int currHashSize = BitUtil.nextHighestPowerOfTwo((int) (possibleValues * (1 / LongCounts.LOAD_FACTOR) + 1));
|
||||
currHashSize = Math.min(currHashSize, MAXIMUM_STARTING_TABLE_SIZE);
|
||||
final LongCounts table = new LongCounts(currHashSize) {
|
||||
table = new LongCounts(currHashSize) {
|
||||
@Override
|
||||
protected void rehash() {
|
||||
super.rehash();
|
||||
|
@ -166,9 +240,19 @@ class FacetFieldProcessorByHashNumeric extends FacetFieldProcessor {
|
|||
}
|
||||
};
|
||||
|
||||
int numSlots = currHashSize;
|
||||
// note: these methods/phases align with FacetFieldProcessorByArray's
|
||||
|
||||
int numMissing = 0;
|
||||
createCollectAcc();
|
||||
|
||||
collectDocs();
|
||||
|
||||
return super.findTopSlots(table.numSlots(), table.cardinality(),
|
||||
slotNum -> calc.bitsToValue(table.vals[slotNum]), // getBucketValFromSlotNum
|
||||
val -> calc.formatValue(val)); // getFieldQueryVal
|
||||
}
|
||||
|
||||
private void createCollectAcc() throws IOException {
|
||||
int numSlots = table.numSlots();
|
||||
|
||||
if (freq.allBuckets) {
|
||||
allBucketsSlot = numSlots++;
|
||||
|
@ -238,160 +322,80 @@ class FacetFieldProcessorByHashNumeric extends FacetFieldProcessor {
|
|||
};
|
||||
|
||||
// we set the countAcc & indexAcc first so generic ones won't be created for us.
|
||||
createCollectAcc(fcontext.base.size(), numSlots);
|
||||
super.createCollectAcc(fcontext.base.size(), numSlots);
|
||||
|
||||
if (freq.allBuckets) {
|
||||
allBucketsAcc = new SpecialSlotAcc(fcontext, collectAcc, allBucketsSlot, otherAccs, 0);
|
||||
}
|
||||
}
|
||||
|
||||
NumericDocValues values = null;
|
||||
Bits docsWithField = null;
|
||||
private void collectDocs() throws IOException {
|
||||
if (calc instanceof TermOrdCalc) { // Strings
|
||||
|
||||
// TODO: factor this code out so it can be shared...
|
||||
final List<LeafReaderContext> leaves = fcontext.searcher.getIndexReader().leaves();
|
||||
final Iterator<LeafReaderContext> ctxIt = leaves.iterator();
|
||||
LeafReaderContext ctx = null;
|
||||
int segBase = 0;
|
||||
int segMax;
|
||||
int adjustedMax = 0;
|
||||
for (DocIterator docsIt = fcontext.base.iterator(); docsIt.hasNext(); ) {
|
||||
final int doc = docsIt.nextDoc();
|
||||
if (doc >= adjustedMax) {
|
||||
do {
|
||||
ctx = ctxIt.next();
|
||||
segBase = ctx.docBase;
|
||||
segMax = ctx.reader().maxDoc();
|
||||
adjustedMax = segBase + segMax;
|
||||
} while (doc >= adjustedMax);
|
||||
assert doc >= ctx.docBase;
|
||||
setNextReaderFirstPhase(ctx);
|
||||
// TODO support SortedSetDocValues
|
||||
SortedDocValues globalDocValues = FieldUtil.getSortedDocValues(fcontext.qcontext, sf, null);
|
||||
((TermOrdCalc)calc).lookupOrdFunction = globalDocValues::lookupOrd;
|
||||
|
||||
values = DocValues.getNumeric(ctx.reader(), sf.getName());
|
||||
docsWithField = DocValues.getDocsWithField(ctx.reader(), sf.getName());
|
||||
}
|
||||
DocSetUtil.collectSortedDocSet(fcontext.base, fcontext.searcher.getIndexReader(), new SimpleCollector() {
|
||||
SortedDocValues docValues = globalDocValues; // this segment/leaf. NN
|
||||
LongValues toGlobal = LongValues.IDENTITY; // this segment to global ordinal. NN
|
||||
|
||||
int segDoc = doc - segBase;
|
||||
long val = values.get(segDoc);
|
||||
if (val != 0 || docsWithField.get(segDoc)) {
|
||||
int slot = table.add(val); // this can trigger a rehash rehash
|
||||
@Override public boolean needsScores() { return false; }
|
||||
|
||||
// countAcc.incrementCount(slot, 1);
|
||||
// our countAcc is virtual, so this is not needed
|
||||
@Override
|
||||
protected void doSetNextReader(LeafReaderContext ctx) throws IOException {
|
||||
setNextReaderFirstPhase(ctx);
|
||||
if (globalDocValues instanceof MultiDocValues.MultiSortedDocValues) {
|
||||
MultiDocValues.MultiSortedDocValues multiDocValues = (MultiDocValues.MultiSortedDocValues) globalDocValues;
|
||||
docValues = multiDocValues.values[ctx.ord];
|
||||
toGlobal = multiDocValues.mapping.getGlobalOrds(ctx.ord);
|
||||
}
|
||||
}
|
||||
|
||||
collectFirstPhase(segDoc, slot);
|
||||
}
|
||||
@Override
|
||||
public void collect(int segDoc) throws IOException {
|
||||
long ord = docValues.getOrd(segDoc);
|
||||
if (ord != -1) {
|
||||
long val = toGlobal.get(ord);
|
||||
collectValFirstPhase(segDoc, val);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
} else { // Numeric:
|
||||
|
||||
// TODO support SortedNumericDocValues
|
||||
DocSetUtil.collectSortedDocSet(fcontext.base, fcontext.searcher.getIndexReader(), new SimpleCollector() {
|
||||
NumericDocValues values = null; //NN
|
||||
Bits docsWithField = null; //NN
|
||||
|
||||
@Override public boolean needsScores() { return false; }
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(LeafReaderContext ctx) throws IOException {
|
||||
setNextReaderFirstPhase(ctx);
|
||||
values = DocValues.getNumeric(ctx.reader(), sf.getName());
|
||||
docsWithField = DocValues.getDocsWithField(ctx.reader(), sf.getName());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int segDoc) throws IOException {
|
||||
long val = values.get(segDoc);
|
||||
if (val != 0 || docsWithField.get(segDoc)) {
|
||||
collectValFirstPhase(segDoc, val);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
//
|
||||
// collection done, time to find the top slots
|
||||
//
|
||||
private void collectValFirstPhase(int segDoc, long val) throws IOException {
|
||||
int slot = table.add(val); // this can trigger a rehash
|
||||
|
||||
int numBuckets = 0;
|
||||
List<Object> bucketVals = null;
|
||||
if (freq.numBuckets && fcontext.isShard()) {
|
||||
bucketVals = new ArrayList<>(100);
|
||||
}
|
||||
// Our countAcc is virtual, so this is not needed:
|
||||
// countAcc.incrementCount(slot, 1);
|
||||
|
||||
int off = fcontext.isShard() ? 0 : (int) freq.offset;
|
||||
// add a modest amount of over-request if this is a shard request
|
||||
int lim = freq.limit >= 0 ? (fcontext.isShard() ? (int)(freq.limit*1.1+4) : (int)freq.limit) : Integer.MAX_VALUE;
|
||||
|
||||
int maxsize = (int)(freq.limit >= 0 ? freq.offset + lim : Integer.MAX_VALUE - 1);
|
||||
maxsize = Math.min(maxsize, table.cardinality);
|
||||
|
||||
final int sortMul = freq.sortDirection.getMultiplier();
|
||||
|
||||
PriorityQueue<Slot> queue = new PriorityQueue<Slot>(maxsize) {
|
||||
@Override
|
||||
protected boolean lessThan(Slot a, Slot b) {
|
||||
// TODO: sort-by-index-order
|
||||
int cmp = sortAcc.compare(a.slot, b.slot) * sortMul;
|
||||
return cmp == 0 ? (indexOrderAcc.compare(a.slot, b.slot) > 0) : cmp < 0;
|
||||
}
|
||||
};
|
||||
|
||||
// TODO: create a countAcc that wrapps the table so we can reuse more code?
|
||||
|
||||
Slot bottom = null;
|
||||
for (int i=0; i<table.counts.length; i++) {
|
||||
int count = table.counts[i];
|
||||
if (count < effectiveMincount) {
|
||||
// either not a valid slot, or count not high enough
|
||||
continue;
|
||||
}
|
||||
numBuckets++; // can be different from the table cardinality if mincount > 1
|
||||
|
||||
long val = table.vals[i];
|
||||
if (bucketVals != null && bucketVals.size()<100) {
|
||||
bucketVals.add( calc.bitsToValue(val) );
|
||||
}
|
||||
|
||||
if (bottom == null) {
|
||||
bottom = new Slot();
|
||||
}
|
||||
bottom.slot = i;
|
||||
|
||||
bottom = queue.insertWithOverflow(bottom);
|
||||
}
|
||||
|
||||
SimpleOrderedMap<Object> res = new SimpleOrderedMap<>();
|
||||
if (freq.numBuckets) {
|
||||
if (!fcontext.isShard()) {
|
||||
res.add("numBuckets", numBuckets);
|
||||
} else {
|
||||
SimpleOrderedMap<Object> map = new SimpleOrderedMap<>(2);
|
||||
map.add("numBuckets", numBuckets);
|
||||
map.add("vals", bucketVals);
|
||||
res.add("numBuckets", map);
|
||||
}
|
||||
}
|
||||
|
||||
FacetDebugInfo fdebug = fcontext.getDebugInfo();
|
||||
if (fdebug != null) fdebug.putInfoItem("numBuckets", (long) numBuckets);
|
||||
|
||||
if (freq.allBuckets) {
|
||||
SimpleOrderedMap<Object> allBuckets = new SimpleOrderedMap<>();
|
||||
// countAcc.setValues(allBuckets, allBucketsSlot);
|
||||
allBuckets.add("count", table.numAdds);
|
||||
allBucketsAcc.setValues(allBuckets, -1);
|
||||
// allBuckets currently doesn't execute sub-facets (because it doesn't change the domain?)
|
||||
res.add("allBuckets", allBuckets);
|
||||
}
|
||||
|
||||
if (freq.missing) {
|
||||
// TODO: it would be more efficient to buid up a missing DocSet if we need it here anyway.
|
||||
|
||||
SimpleOrderedMap<Object> missingBucket = new SimpleOrderedMap<>();
|
||||
fillBucket(missingBucket, getFieldMissingQuery(fcontext.searcher, freq.field), null);
|
||||
res.add("missing", missingBucket);
|
||||
}
|
||||
|
||||
// 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 <= lim;
|
||||
int[] sortedSlots = new int[collectCount];
|
||||
for (int i = collectCount - 1; i >= 0; i--) {
|
||||
sortedSlots[i] = queue.pop().slot;
|
||||
}
|
||||
|
||||
ArrayList<SimpleOrderedMap> bucketList = new ArrayList<>(collectCount);
|
||||
res.add("buckets", bucketList);
|
||||
|
||||
boolean needFilter = deferredAggs != null || freq.getSubFacets().size() > 0;
|
||||
|
||||
for (int slotNum : sortedSlots) {
|
||||
SimpleOrderedMap<Object> bucket = new SimpleOrderedMap<>();
|
||||
Comparable val = calc.bitsToValue(table.vals[slotNum]);
|
||||
bucket.add("val", val);
|
||||
|
||||
Query filter = needFilter ? sf.getType().getFieldQuery(null, sf, calc.formatValue(val)) : null;
|
||||
|
||||
fillBucket(bucket, table.counts[slotNum], slotNum, null, filter);
|
||||
|
||||
bucketList.add(bucket);
|
||||
}
|
||||
|
||||
return res;
|
||||
super.collectFirstPhase(segDoc, slot);
|
||||
}
|
||||
|
||||
private void doRehash(LongCounts table) {
|
||||
|
|
|
@ -32,7 +32,12 @@ import java.util.Arrays;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
|
||||
/**
|
||||
* Accumulates statistics separated by a slot number.
|
||||
* There is a separate statistic per slot. The slot is usually an ordinal into a set of values, e.g. tracking a count
|
||||
* frequency <em>per term</em>.
|
||||
* Sometimes there doesn't need to be a slot distinction, in which case there is just one nominal slot.
|
||||
*/
|
||||
public abstract class SlotAcc implements Closeable {
|
||||
String key; // todo...
|
||||
protected final FacetContext fcontext;
|
||||
|
@ -210,9 +215,7 @@ abstract class DoubleFuncSlotAcc extends FuncSlotAcc {
|
|||
|
||||
@Override
|
||||
public void reset() {
|
||||
for (int i=0; i<result.length; i++) {
|
||||
result[i] = initialValue;
|
||||
}
|
||||
Arrays.fill(result, initialValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -246,9 +249,7 @@ abstract class IntSlotAcc extends SlotAcc {
|
|||
|
||||
@Override
|
||||
public void reset() {
|
||||
for (int i=0; i<result.length; i++) {
|
||||
result[i] = initialValue;
|
||||
}
|
||||
Arrays.fill(result, initialValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
|
||||
import com.tdunning.math.stats.AVLTreeDigest;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.util.hll.HLL;
|
||||
|
@ -43,12 +44,19 @@ public class TestJsonFacets extends SolrTestCaseHS {
|
|||
|
||||
private static SolrInstances servers; // for distributed testing
|
||||
private static int origTableSize;
|
||||
private static FacetField.FacetMethod origDefaultFacetMethod;
|
||||
|
||||
@BeforeClass
|
||||
public static void beforeTests() throws Exception {
|
||||
JSONTestUtil.failRepeatedKeys = true;
|
||||
|
||||
origTableSize = FacetFieldProcessorByHashNumeric.MAXIMUM_STARTING_TABLE_SIZE;
|
||||
FacetFieldProcessorByHashNumeric.MAXIMUM_STARTING_TABLE_SIZE=2; // stress test resizing
|
||||
|
||||
origDefaultFacetMethod = FacetField.FacetMethod.DEFAULT_METHOD;
|
||||
// instead of the following, see the constructor
|
||||
//FacetField.FacetMethod.DEFAULT_METHOD = rand(FacetField.FacetMethod.values());
|
||||
|
||||
initCore("solrconfig-tlog.xml","schema_latest.xml");
|
||||
}
|
||||
|
||||
|
@ -62,12 +70,25 @@ public class TestJsonFacets extends SolrTestCaseHS {
|
|||
public static void afterTests() throws Exception {
|
||||
JSONTestUtil.failRepeatedKeys = false;
|
||||
FacetFieldProcessorByHashNumeric.MAXIMUM_STARTING_TABLE_SIZE=origTableSize;
|
||||
FacetField.FacetMethod.DEFAULT_METHOD = origDefaultFacetMethod;
|
||||
if (servers != null) {
|
||||
servers.stop();
|
||||
servers = null;
|
||||
}
|
||||
}
|
||||
|
||||
// tip: when debugging a test, comment out the @ParametersFactory and edit the constructor to be no-arg
|
||||
|
||||
@ParametersFactory
|
||||
public static Iterable<Object[]> parameters() {
|
||||
// wrap each enum val in an Object[] and return as Iterable
|
||||
return () -> Arrays.stream(FacetField.FacetMethod.values()).map(it -> new Object[]{it}).iterator();
|
||||
}
|
||||
|
||||
public TestJsonFacets(FacetField.FacetMethod defMethod) {
|
||||
FacetField.FacetMethod.DEFAULT_METHOD = defMethod; // note: the real default is restored in afterTests
|
||||
}
|
||||
|
||||
// attempt to reproduce https://github.com/Heliosearch/heliosearch/issues/33
|
||||
@Test
|
||||
public void testComplex() throws Exception {
|
||||
|
@ -180,8 +201,8 @@ public class TestJsonFacets extends SolrTestCaseHS {
|
|||
client.commit();
|
||||
}
|
||||
|
||||
|
||||
public void testStatsSimple() throws Exception {
|
||||
@Test
|
||||
public void testMethodStream() throws Exception {
|
||||
Client client = Client.localClient();
|
||||
indexSimple(client);
|
||||
|
||||
|
@ -196,15 +217,15 @@ public class TestJsonFacets extends SolrTestCaseHS {
|
|||
|
||||
// test streaming
|
||||
assertJQ(req("q", "*:*", "rows", "0"
|
||||
, "json.facet", "{ cat:{terms:{field:'cat_s', method:stream }}" +
|
||||
", cat2:{terms:{field:'cat_s', method:stream, sort:'index asc' }}" + // default sort
|
||||
", cat3:{terms:{field:'cat_s', method:stream, mincount:3 }}" + // mincount
|
||||
", cat4:{terms:{field:'cat_s', method:stream, prefix:B }}" + // prefix
|
||||
", cat5:{terms:{field:'cat_s', method:stream, offset:1 }}" + // offset
|
||||
, "json.facet", "{ cat:{terms:{field:'cat_s', method:stream }}" + // won't stream; need sort:index asc
|
||||
", cat2:{terms:{field:'cat_s', method:stream, sort:'index asc' }}" +
|
||||
", cat3:{terms:{field:'cat_s', method:stream, sort:'index asc', mincount:3 }}" + // mincount
|
||||
", cat4:{terms:{field:'cat_s', method:stream, sort:'index asc', prefix:B }}" + // prefix
|
||||
", cat5:{terms:{field:'cat_s', method:stream, sort:'index asc', offset:1 }}" + // offset
|
||||
" }"
|
||||
)
|
||||
, "facets=={count:6 " +
|
||||
", cat :{buckets:[{val:A, count:2},{val:B, count:3}]}" +
|
||||
", cat :{buckets:[{val:B, count:3},{val:A, count:2}]}" +
|
||||
", cat2:{buckets:[{val:A, count:2},{val:B, count:3}]}" +
|
||||
", cat3:{buckets:[{val:B, count:3}]}" +
|
||||
", cat4:{buckets:[{val:B, count:3}]}" +
|
||||
|
@ -215,7 +236,7 @@ public class TestJsonFacets extends SolrTestCaseHS {
|
|||
|
||||
// test nested streaming under non-streaming
|
||||
assertJQ(req("q", "*:*", "rows", "0"
|
||||
, "json.facet", "{ cat:{terms:{field:'cat_s', sort:'index asc', facet:{where:{terms:{field:where_s,method:stream}}} }}}"
|
||||
, "json.facet", "{ cat:{terms:{field:'cat_s', sort:'index asc', facet:{where:{terms:{field:where_s,method:stream,sort:'index asc'}}} }}}"
|
||||
)
|
||||
, "facets=={count:6 " +
|
||||
", cat :{buckets:[{val:A, count:2, where:{buckets:[{val:NJ,count:1},{val:NY,count:1}]} },{val:B, count:3, where:{buckets:[{val:NJ,count:2},{val:NY,count:1}]} }]}"
|
||||
|
@ -224,7 +245,7 @@ public class TestJsonFacets extends SolrTestCaseHS {
|
|||
|
||||
// test nested streaming under streaming
|
||||
assertJQ(req("q", "*:*", "rows", "0"
|
||||
, "json.facet", "{ cat:{terms:{field:'cat_s', method:stream, facet:{where:{terms:{field:where_s,method:stream}}} }}}"
|
||||
, "json.facet", "{ cat:{terms:{field:'cat_s', method:stream,sort:'index asc', facet:{where:{terms:{field:where_s,method:stream,sort:'index asc'}}} }}}"
|
||||
)
|
||||
, "facets=={count:6 " +
|
||||
", cat :{buckets:[{val:A, count:2, where:{buckets:[{val:NJ,count:1},{val:NY,count:1}]} },{val:B, count:3, where:{buckets:[{val:NJ,count:2},{val:NY,count:1}]} }]}"
|
||||
|
@ -233,7 +254,7 @@ public class TestJsonFacets extends SolrTestCaseHS {
|
|||
|
||||
// test nested streaming with stats under streaming
|
||||
assertJQ(req("q", "*:*", "rows", "0"
|
||||
, "json.facet", "{ cat:{terms:{field:'cat_s', method:stream, facet:{ where:{terms:{field:where_s,method:stream, facet:{x:'max(num_d)'} }}} }}}"
|
||||
, "json.facet", "{ cat:{terms:{field:'cat_s', method:stream,sort:'index asc', facet:{ where:{terms:{field:where_s,method:stream,sort:'index asc',sort:'index asc', facet:{x:'max(num_d)'} }}} }}}"
|
||||
)
|
||||
, "facets=={count:6 " +
|
||||
", cat :{buckets:[{val:A, count:2, where:{buckets:[{val:NJ,count:1,x:2.0},{val:NY,count:1,x:4.0}]} },{val:B, count:3, where:{buckets:[{val:NJ,count:2,x:11.0},{val:NY,count:1,x:-5.0}]} }]}"
|
||||
|
@ -243,7 +264,7 @@ public class TestJsonFacets extends SolrTestCaseHS {
|
|||
// test nested streaming with stats under streaming with stats
|
||||
assertJQ(req("q", "*:*", "rows", "0",
|
||||
"facet","true"
|
||||
, "json.facet", "{ cat:{terms:{field:'cat_s', method:stream, facet:{ y:'min(num_d)', where:{terms:{field:where_s,method:stream, facet:{x:'max(num_d)'} }}} }}}"
|
||||
, "json.facet", "{ cat:{terms:{field:'cat_s', method:stream,sort:'index asc', facet:{ y:'min(num_d)', where:{terms:{field:where_s,method:stream,sort:'index asc', facet:{x:'max(num_d)'} }}} }}}"
|
||||
)
|
||||
, "facets=={count:6 " +
|
||||
", cat :{buckets:[{val:A, count:2, y:2.0, where:{buckets:[{val:NJ,count:1,x:2.0},{val:NY,count:1,x:4.0}]} },{val:B, count:3, y:-9.0, where:{buckets:[{val:NJ,count:2,x:11.0},{val:NY,count:1,x:-5.0}]} }]}"
|
||||
|
@ -294,7 +315,7 @@ public class TestJsonFacets extends SolrTestCaseHS {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testDistrib() throws Exception {
|
||||
public void testStatsDistrib() throws Exception {
|
||||
initServers();
|
||||
Client client = servers.getClient(random().nextInt());
|
||||
client.queryDefaults().set( "shards", servers.getShards(), "debugQuery", Boolean.toString(random().nextBoolean()) );
|
||||
|
|
Loading…
Reference in New Issue