mirror of https://github.com/apache/lucene.git
SOLR-9404: Refactor move/renames in JSON FacetProcessor and FacetFieldProcessor.
This commit is contained in:
parent
6d1f1f6c78
commit
7072458ea4
|
@ -280,6 +280,8 @@ Other Changes
|
|||
|
||||
* SOLR-9410: Make ReRankQParserPlugin's private ReRankWeight a public class of its own. (Christine Poerschke)
|
||||
|
||||
* SOLR-9404: Refactor move/renames in JSON FacetProcessor and FacetFieldProcessor. (David Smiley)
|
||||
|
||||
================== 6.1.0 ==================
|
||||
|
||||
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,369 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.solr.search.facet;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
import org.apache.solr.schema.SchemaField;
|
||||
import org.apache.solr.search.DocSet;
|
||||
|
||||
/**
|
||||
* Facet processing based on field values. (not range nor by query)
|
||||
* @see FacetField
|
||||
*/
|
||||
abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
|
||||
SchemaField sf;
|
||||
SlotAcc indexOrderAcc;
|
||||
int effectiveMincount;
|
||||
|
||||
Map<String,AggValueSource> deferredAggs; // null if none
|
||||
|
||||
// TODO: push any of this down to base class?
|
||||
|
||||
//
|
||||
// For sort="x desc", collectAcc would point to "x", and sortAcc would also point to "x".
|
||||
// collectAcc would be used to accumulate all buckets, and sortAcc would be used to sort those buckets.
|
||||
//
|
||||
SlotAcc collectAcc; // Accumulator to collect across entire domain (in addition to the countAcc). May be null.
|
||||
SlotAcc sortAcc; // Accumulator to use for sorting *only* (i.e. not used for collection). May be an alias of countAcc, collectAcc, or indexOrderAcc
|
||||
SlotAcc[] otherAccs; // Accumulators that do not need to be calculated across all buckets.
|
||||
|
||||
SpecialSlotAcc allBucketsAcc; // this can internally refer to otherAccs and/or collectAcc. setNextReader should be called on otherAccs directly if they exist.
|
||||
|
||||
FacetFieldProcessor(FacetContext fcontext, FacetField freq, SchemaField sf) {
|
||||
super(fcontext, freq);
|
||||
this.sf = sf;
|
||||
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)
|
||||
@Override
|
||||
protected void createAccs(int docCount, int slotCount) throws IOException {
|
||||
if (accMap == null) {
|
||||
accMap = new LinkedHashMap<>();
|
||||
}
|
||||
|
||||
// allow a custom count acc to be used
|
||||
if (countAcc == null) {
|
||||
countAcc = new CountSlotArrAcc(fcontext, slotCount);
|
||||
countAcc.key = "count";
|
||||
}
|
||||
|
||||
if (accs != null) {
|
||||
// reuse these accs, but reset them first
|
||||
for (SlotAcc acc : accs) {
|
||||
acc.reset();
|
||||
}
|
||||
return;
|
||||
} else {
|
||||
accs = new SlotAcc[ freq.getFacetStats().size() ];
|
||||
}
|
||||
|
||||
int accIdx = 0;
|
||||
for (Map.Entry<String,AggValueSource> entry : freq.getFacetStats().entrySet()) {
|
||||
SlotAcc acc = null;
|
||||
if (slotCount == 1) {
|
||||
acc = accMap.get(entry.getKey());
|
||||
if (acc != null) {
|
||||
acc.reset();
|
||||
}
|
||||
}
|
||||
if (acc == null) {
|
||||
acc = entry.getValue().createSlotAcc(fcontext, docCount, slotCount);
|
||||
acc.key = entry.getKey();
|
||||
accMap.put(acc.key, acc);
|
||||
}
|
||||
accs[accIdx++] = acc;
|
||||
}
|
||||
}
|
||||
|
||||
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)) {
|
||||
// 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();
|
||||
} else {
|
||||
AggValueSource sortAgg = freq.getFacetStats().get(freq.sortVariable);
|
||||
if (sortAgg != null) {
|
||||
collectAcc = sortAgg.createSlotAcc(fcontext, numDocs, numSlots);
|
||||
collectAcc.key = freq.sortVariable; // TODO: improve this
|
||||
}
|
||||
sortAcc = collectAcc;
|
||||
deferredAggs = new HashMap<>(freq.getFacetStats());
|
||||
deferredAggs.remove(freq.sortVariable);
|
||||
}
|
||||
|
||||
if (deferredAggs.size() == 0) {
|
||||
deferredAggs = null;
|
||||
}
|
||||
|
||||
boolean needOtherAccs = freq.allBuckets; // TODO: use for missing too...
|
||||
|
||||
if (!needOtherAccs) {
|
||||
// we may need them later, but we don't want to create them now
|
||||
// otherwise we won't know if we need to call setNextReader on them.
|
||||
return;
|
||||
}
|
||||
|
||||
// create the deferred aggs up front for use by allBuckets
|
||||
createOtherAccs(numDocs, 1);
|
||||
}
|
||||
|
||||
private void createOtherAccs(int numDocs, int numSlots) throws IOException {
|
||||
if (otherAccs != null) {
|
||||
// reuse existing accumulators
|
||||
for (SlotAcc acc : otherAccs) {
|
||||
acc.reset(); // todo - make reset take numDocs and numSlots?
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
int numDeferred = deferredAggs == null ? 0 : deferredAggs.size();
|
||||
if (numDeferred <= 0) return;
|
||||
|
||||
otherAccs = new SlotAcc[ numDeferred ];
|
||||
|
||||
int otherAccIdx = 0;
|
||||
for (Map.Entry<String,AggValueSource> entry : deferredAggs.entrySet()) {
|
||||
AggValueSource agg = entry.getValue();
|
||||
SlotAcc acc = agg.createSlotAcc(fcontext, numDocs, numSlots);
|
||||
acc.key = entry.getKey();
|
||||
accMap.put(acc.key, acc);
|
||||
otherAccs[otherAccIdx++] = acc;
|
||||
}
|
||||
|
||||
if (numDeferred == freq.getFacetStats().size()) {
|
||||
// accs and otherAccs are the same...
|
||||
accs = otherAccs;
|
||||
}
|
||||
}
|
||||
|
||||
int collectFirstPhase(DocSet docs, int slot) throws IOException {
|
||||
int num = -1;
|
||||
if (collectAcc != null) {
|
||||
num = collectAcc.collect(docs, slot);
|
||||
}
|
||||
if (allBucketsAcc != null) {
|
||||
num = allBucketsAcc.collect(docs, slot);
|
||||
}
|
||||
return num >= 0 ? num : docs.size();
|
||||
}
|
||||
|
||||
void collectFirstPhase(int segDoc, int slot) throws IOException {
|
||||
if (collectAcc != null) {
|
||||
collectAcc.collect(segDoc, slot);
|
||||
}
|
||||
if (allBucketsAcc != null) {
|
||||
allBucketsAcc.collect(segDoc, slot);
|
||||
}
|
||||
}
|
||||
|
||||
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;
|
||||
|
||||
if (collectAcc != null && slotNum >= 0) {
|
||||
collectAcc.setValues(target, slotNum);
|
||||
}
|
||||
|
||||
createOtherAccs(-1, 1);
|
||||
|
||||
if (otherAccs == null && freq.subFacets.isEmpty()) return;
|
||||
|
||||
if (subDomain == null) {
|
||||
subDomain = fcontext.searcher.getDocSet(filter, fcontext.base);
|
||||
}
|
||||
|
||||
// if no subFacets, we only need a DocSet
|
||||
// otherwise we need more?
|
||||
// TODO: save something generic like "slotNum" in the context and use that to implement things like filter exclusion if necessary?
|
||||
// Hmmm, but we need to look up some stuff anyway (for the label?)
|
||||
// have a method like "DocSet applyConstraint(facet context, DocSet parent)"
|
||||
// that's needed for domain changing things like joins anyway???
|
||||
|
||||
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);
|
||||
acc.setValues(target, 0);
|
||||
}
|
||||
}
|
||||
|
||||
processSubs(target, filter, subDomain);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void processStats(SimpleOrderedMap<Object> bucket, DocSet docs, int docCount) throws IOException {
|
||||
if (docCount == 0 && !freq.processEmpty || freq.getFacetStats().size() == 0) {
|
||||
bucket.add("count", docCount);
|
||||
return;
|
||||
}
|
||||
createAccs(docCount, 1);
|
||||
int collected = collect(docs, 0);
|
||||
|
||||
// countAcc.incrementCount(0, collected); // should we set the counton the acc instead of just passing it?
|
||||
|
||||
assert collected == docCount;
|
||||
addStats(bucket, collected, 0);
|
||||
}
|
||||
|
||||
// overrides but with different signature!
|
||||
private void addStats(SimpleOrderedMap<Object> target, int count, int slotNum) throws IOException {
|
||||
target.add("count", count);
|
||||
if (count > 0 || freq.processEmpty) {
|
||||
for (SlotAcc acc : accs) {
|
||||
acc.setValues(target, slotNum);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
void setNextReader(LeafReaderContext ctx) throws IOException {
|
||||
// base class calls this (for missing bucket...) ... go over accs[] in that case
|
||||
super.setNextReader(ctx);
|
||||
}
|
||||
|
||||
void setNextReaderFirstPhase(LeafReaderContext ctx) throws IOException {
|
||||
if (collectAcc != null) {
|
||||
collectAcc.setNextReader(ctx);
|
||||
}
|
||||
if (otherAccs != null) {
|
||||
for (SlotAcc acc : otherAccs) {
|
||||
acc.setNextReader(ctx);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static class Slot {
|
||||
int slot;
|
||||
public int tiebreakCompare(int slotA, int slotB) {
|
||||
return slotB - slotA;
|
||||
}
|
||||
}
|
||||
|
||||
static class SpecialSlotAcc extends SlotAcc {
|
||||
SlotAcc collectAcc;
|
||||
SlotAcc[] otherAccs;
|
||||
int collectAccSlot;
|
||||
int otherAccsSlot;
|
||||
long count;
|
||||
|
||||
SpecialSlotAcc(FacetContext fcontext, SlotAcc collectAcc, int collectAccSlot, SlotAcc[] otherAccs, int otherAccsSlot) {
|
||||
super(fcontext);
|
||||
this.collectAcc = collectAcc;
|
||||
this.collectAccSlot = collectAccSlot;
|
||||
this.otherAccs = otherAccs;
|
||||
this.otherAccsSlot = otherAccsSlot;
|
||||
}
|
||||
|
||||
public int getCollectAccSlot() { return collectAccSlot; }
|
||||
public int getOtherAccSlot() { return otherAccsSlot; }
|
||||
|
||||
long getSpecialCount() {
|
||||
return count;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int doc, int slot) throws IOException {
|
||||
assert slot != collectAccSlot || slot < 0;
|
||||
count++;
|
||||
if (collectAcc != null) {
|
||||
collectAcc.collect(doc, collectAccSlot);
|
||||
}
|
||||
if (otherAccs != null) {
|
||||
for (SlotAcc otherAcc : otherAccs) {
|
||||
otherAcc.collect(doc, otherAccsSlot);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setNextReader(LeafReaderContext readerContext) throws IOException {
|
||||
// collectAcc and otherAccs will normally have setNextReader called directly on them.
|
||||
// This, however, will be used when collect(DocSet,slot) variant is used on this Acc.
|
||||
if (collectAcc != null) {
|
||||
collectAcc.setNextReader(readerContext);
|
||||
}
|
||||
if (otherAccs != null) {
|
||||
for (SlotAcc otherAcc : otherAccs) {
|
||||
otherAcc.setNextReader(readerContext);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(int slotA, int slotB) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getValue(int slotNum) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setValues(SimpleOrderedMap<Object> bucket, int slotNum) throws IOException {
|
||||
if (collectAcc != null) {
|
||||
collectAcc.setValues(bucket, collectAccSlot);
|
||||
}
|
||||
if (otherAccs != null) {
|
||||
for (SlotAcc otherAcc : otherAccs) {
|
||||
otherAcc.setValues(bucket, otherAccsSlot);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() {
|
||||
// reset should be called on underlying accs
|
||||
// TODO: but in case something does need to be done here, should we require this method to be called but do nothing for now?
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void resize(Resizer resizer) {
|
||||
// someone else will call resize on collectAcc directly
|
||||
if (collectAccSlot >= 0) {
|
||||
collectAccSlot = resizer.getNewSlot(collectAccSlot);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,213 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
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.
|
||||
* It can handle terms (strings), not numbers directly but those encoded as terms, and is multi-valued capable.
|
||||
*/
|
||||
abstract class FacetFieldProcessorByArray extends FacetFieldProcessor {
|
||||
BytesRefBuilder prefixRef;
|
||||
int startTermIndex;
|
||||
int endTermIndex;
|
||||
int nTerms;
|
||||
int nDocs;
|
||||
int maxSlots;
|
||||
|
||||
int allBucketsSlot = -1; // slot for the primary Accs (countAcc, collectAcc)
|
||||
|
||||
FacetFieldProcessorByArray(FacetContext fcontext, FacetField freq, SchemaField sf) {
|
||||
super(fcontext, freq, sf);
|
||||
}
|
||||
|
||||
abstract protected void findStartAndEndOrds() throws IOException;
|
||||
|
||||
abstract protected void collectDocs() throws IOException;
|
||||
|
||||
/** this BytesRef may be shared across calls and should be deep-cloned if necessary */
|
||||
abstract protected BytesRef lookupOrd(int ord) throws IOException;
|
||||
|
||||
@Override
|
||||
public void process() throws IOException {
|
||||
super.process();
|
||||
sf = fcontext.searcher.getSchema().getField(freq.field);
|
||||
response = getFieldCacheCounts();
|
||||
}
|
||||
|
||||
private SimpleOrderedMap<Object> getFieldCacheCounts() throws IOException {
|
||||
String prefix = freq.prefix;
|
||||
if (prefix == null || prefix.length() == 0) {
|
||||
prefixRef = null;
|
||||
} else {
|
||||
prefixRef = new BytesRefBuilder();
|
||||
prefixRef.copyChars(prefix);
|
||||
}
|
||||
|
||||
findStartAndEndOrds();
|
||||
|
||||
maxSlots = nTerms;
|
||||
|
||||
if (freq.allBuckets) {
|
||||
allBucketsSlot = maxSlots++;
|
||||
}
|
||||
|
||||
createCollectAcc(nDocs, maxSlots);
|
||||
|
||||
if (freq.allBuckets) {
|
||||
allBucketsAcc = new SpecialSlotAcc(fcontext, collectAcc, allBucketsSlot, otherAccs, 0);
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
}
|
|
@ -34,23 +34,22 @@ import org.apache.solr.common.SolrException;
|
|||
import org.apache.solr.schema.SchemaField;
|
||||
import org.apache.solr.search.Filter;
|
||||
|
||||
class FacetFieldProcessorDV extends FacetFieldProcessorFCBase {
|
||||
/**
|
||||
* Grabs values from {@link DocValues}.
|
||||
*/
|
||||
class FacetFieldProcessorByArrayDV extends FacetFieldProcessorByArray {
|
||||
static boolean unwrap_singleValued_multiDv = true; // only set to false for test coverage
|
||||
|
||||
boolean multiValuedField;
|
||||
SortedSetDocValues si; // only used for term lookups (for both single and multi-valued)
|
||||
MultiDocValues.OrdinalMap ordinalMap = null; // maps per-segment ords to global ords
|
||||
|
||||
|
||||
public FacetFieldProcessorDV(FacetContext fcontext, FacetField freq, SchemaField sf) {
|
||||
FacetFieldProcessorByArrayDV(FacetContext fcontext, FacetField freq, SchemaField sf) {
|
||||
super(fcontext, freq, sf);
|
||||
multiValuedField = sf.multiValued() || sf.getType().multiValuedFieldCache();
|
||||
}
|
||||
|
||||
protected BytesRef lookupOrd(int ord) throws IOException {
|
||||
return si.lookupOrd(ord);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void findStartAndEndOrds() throws IOException {
|
||||
if (multiValuedField) {
|
||||
si = FieldUtil.getSortedSetDocValues(fcontext.qcontext, sf, null);
|
||||
|
@ -175,16 +174,9 @@ class FacetFieldProcessorDV extends FacetFieldProcessorFCBase {
|
|||
reuse = null; // better GC
|
||||
}
|
||||
|
||||
private int[] reuse;
|
||||
private int[] getCountArr(int maxNeeded) {
|
||||
if (reuse == null) {
|
||||
// make the count array large enough for any segment
|
||||
// FUTURE: (optionally) directly use the array of the CountAcc for an optimized index..
|
||||
reuse = new int[(int) si.getValueCount() + 1];
|
||||
} else {
|
||||
Arrays.fill(reuse, 0, maxNeeded, 0);
|
||||
}
|
||||
return reuse;
|
||||
@Override
|
||||
protected BytesRef lookupOrd(int ord) throws IOException {
|
||||
return si.lookupOrd(ord);
|
||||
}
|
||||
|
||||
private void collectPerSeg(SortedDocValues singleDv, DocIdSetIterator disi, LongValues toGlobal) throws IOException {
|
||||
|
@ -205,7 +197,6 @@ class FacetFieldProcessorDV extends FacetFieldProcessorFCBase {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
private void collectPerSeg(SortedSetDocValues multiDv, DocIdSetIterator disi, LongValues toGlobal) throws IOException {
|
||||
int segMax = (int)multiDv.getValueCount();
|
||||
final int[] counts = getCountArr( segMax );
|
||||
|
@ -229,6 +220,18 @@ class FacetFieldProcessorDV extends FacetFieldProcessorFCBase {
|
|||
}
|
||||
}
|
||||
|
||||
private int[] reuse;
|
||||
private int[] getCountArr(int maxNeeded) {
|
||||
if (reuse == null) {
|
||||
// make the count array large enough for any segment
|
||||
// FUTURE: (optionally) directly use the array of the CountAcc for an optimized index..
|
||||
reuse = new int[(int) si.getValueCount() + 1];
|
||||
} else {
|
||||
Arrays.fill(reuse, 0, maxNeeded, 0);
|
||||
}
|
||||
return reuse;
|
||||
}
|
||||
|
||||
private void collectDocs(SortedDocValues singleDv, DocIdSetIterator disi, LongValues toGlobal) throws IOException {
|
||||
int doc;
|
||||
while ((doc = disi.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
|
@ -0,0 +1,71 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.solr.search.facet;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.apache.solr.schema.SchemaField;
|
||||
|
||||
/** {@link UnInvertedField} implementation of field faceting.
|
||||
* It's a top-level term cache. */
|
||||
class FacetFieldProcessorByArrayUIF extends FacetFieldProcessorByArray {
|
||||
UnInvertedField uif;
|
||||
TermsEnum te;
|
||||
|
||||
FacetFieldProcessorByArrayUIF(FacetContext fcontext, FacetField freq, SchemaField sf) {
|
||||
super(fcontext, freq, sf);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void findStartAndEndOrds() throws IOException {
|
||||
uif = UnInvertedField.getUnInvertedField(freq.field, fcontext.searcher);
|
||||
te = uif.getOrdTermsEnum( fcontext.searcher.getLeafReader() ); // "te" can be null
|
||||
|
||||
startTermIndex = 0;
|
||||
endTermIndex = uif.numTerms(); // one past the end
|
||||
|
||||
if (prefixRef != null && te != null) {
|
||||
if (te.seekCeil(prefixRef.get()) == TermsEnum.SeekStatus.END) {
|
||||
startTermIndex = uif.numTerms();
|
||||
} else {
|
||||
startTermIndex = (int) te.ord();
|
||||
}
|
||||
prefixRef.append(UnicodeUtil.BIG_TERM);
|
||||
if (te.seekCeil(prefixRef.get()) == TermsEnum.SeekStatus.END) {
|
||||
endTermIndex = uif.numTerms();
|
||||
} else {
|
||||
endTermIndex = (int) te.ord();
|
||||
}
|
||||
}
|
||||
|
||||
nTerms = endTermIndex - startTermIndex;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void collectDocs() throws IOException {
|
||||
uif.collectDocs(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected BytesRef lookupOrd(int ord) throws IOException {
|
||||
return uif.getTermValue(te, ord);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,356 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.solr.search.facet;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.MultiPostingsEnum;
|
||||
import org.apache.lucene.index.PostingsEnum;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
import org.apache.solr.schema.SchemaField;
|
||||
import org.apache.solr.schema.TrieField;
|
||||
import org.apache.solr.search.DocSet;
|
||||
import org.apache.solr.search.HashDocSet;
|
||||
import org.apache.solr.search.SolrIndexSearcher;
|
||||
import org.apache.solr.search.SortedIntDocSet;
|
||||
|
||||
/**
|
||||
* Enumerates indexed terms in order in a streaming fashion.
|
||||
* It's able to stream since no data needs to be accumulated so long as it's index order.
|
||||
*/
|
||||
class FacetFieldProcessorByEnumTermsStream extends FacetFieldProcessor implements Closeable {
|
||||
long bucketsToSkip;
|
||||
long bucketsReturned;
|
||||
|
||||
boolean closed;
|
||||
boolean countOnly;
|
||||
boolean hasSubFacets; // true if there are subfacets
|
||||
int minDfFilterCache;
|
||||
DocSet docs;
|
||||
DocSet fastForRandomSet;
|
||||
TermsEnum termsEnum = null;
|
||||
SolrIndexSearcher.DocsEnumState deState = null;
|
||||
PostingsEnum postingsEnum;
|
||||
BytesRef startTermBytes;
|
||||
BytesRef term;
|
||||
LeafReaderContext[] leaves;
|
||||
|
||||
FacetFieldProcessorByEnumTermsStream(FacetContext fcontext, FacetField freq, SchemaField sf) {
|
||||
super(fcontext, freq, sf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (!closed) {
|
||||
closed = true;
|
||||
// fcontext.base.decref(); // OFF-HEAP
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process() throws IOException {
|
||||
super.process();
|
||||
|
||||
// We need to keep the fcontext open after processing is done (since we will be streaming in the response writer).
|
||||
// But if the connection is broken, we want to clean up.
|
||||
// fcontext.base.incref(); // OFF-HEAP
|
||||
fcontext.qcontext.addCloseHook(this);
|
||||
|
||||
setup();
|
||||
response = new SimpleOrderedMap<>();
|
||||
response.add("buckets", new Iterator() {
|
||||
boolean retrieveNext = true;
|
||||
Object val;
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
if (retrieveNext) {
|
||||
val = nextBucket();
|
||||
}
|
||||
retrieveNext = false;
|
||||
return val != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object next() {
|
||||
if (retrieveNext) {
|
||||
val = nextBucket();
|
||||
}
|
||||
retrieveNext = true;
|
||||
if (val == null) {
|
||||
// Last value, so clean up. In the case that we are doing streaming facets within streaming facets,
|
||||
// the number of close hooks could grow very large, so we want to remove ourselves.
|
||||
boolean removed = fcontext.qcontext.removeCloseHook(FacetFieldProcessorByEnumTermsStream.this);
|
||||
assert removed;
|
||||
try {
|
||||
close();
|
||||
} catch (IOException e) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Error during facet streaming close", e);
|
||||
}
|
||||
}
|
||||
return val;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private void setup() throws IOException {
|
||||
|
||||
countOnly = freq.facetStats.size() == 0 || freq.facetStats.values().iterator().next() instanceof CountAgg;
|
||||
hasSubFacets = freq.subFacets.size() > 0;
|
||||
bucketsToSkip = freq.offset;
|
||||
|
||||
createAccs(-1, 1);
|
||||
|
||||
// Minimum term docFreq in order to use the filterCache for that term.
|
||||
if (freq.cacheDf == -1) { // -1 means never cache
|
||||
minDfFilterCache = Integer.MAX_VALUE;
|
||||
} else if (freq.cacheDf == 0) { // default; compute as fraction of maxDoc
|
||||
minDfFilterCache = Math.max(fcontext.searcher.maxDoc() >> 4, 3); // (minimum of 3 is for test coverage purposes)
|
||||
} else {
|
||||
minDfFilterCache = freq.cacheDf;
|
||||
}
|
||||
|
||||
docs = fcontext.base;
|
||||
fastForRandomSet = null;
|
||||
|
||||
if (freq.prefix != null) {
|
||||
String indexedPrefix = sf.getType().toInternal(freq.prefix);
|
||||
startTermBytes = new BytesRef(indexedPrefix);
|
||||
} else if (sf.getType().getNumericType() != null) {
|
||||
String triePrefix = TrieField.getMainValuePrefix(sf.getType());
|
||||
if (triePrefix != null) {
|
||||
startTermBytes = new BytesRef(triePrefix);
|
||||
}
|
||||
}
|
||||
|
||||
Fields fields = fcontext.searcher.getLeafReader().fields();
|
||||
Terms terms = fields == null ? null : fields.terms(sf.getName());
|
||||
|
||||
termsEnum = null;
|
||||
deState = null;
|
||||
term = null;
|
||||
|
||||
|
||||
if (terms != null) {
|
||||
|
||||
termsEnum = terms.iterator();
|
||||
|
||||
// TODO: OPT: if seek(ord) is supported for this termsEnum, then we could use it for
|
||||
// facet.offset when sorting by index order.
|
||||
|
||||
if (startTermBytes != null) {
|
||||
if (termsEnum.seekCeil(startTermBytes) == TermsEnum.SeekStatus.END) {
|
||||
termsEnum = null;
|
||||
} else {
|
||||
term = termsEnum.term();
|
||||
}
|
||||
} else {
|
||||
// position termsEnum on first term
|
||||
term = termsEnum.next();
|
||||
}
|
||||
}
|
||||
|
||||
List<LeafReaderContext> leafList = fcontext.searcher.getTopReaderContext().leaves();
|
||||
leaves = leafList.toArray( new LeafReaderContext[ leafList.size() ]);
|
||||
}
|
||||
|
||||
private SimpleOrderedMap<Object> nextBucket() {
|
||||
try {
|
||||
return _nextBucket();
|
||||
} catch (Exception e) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Error during facet streaming", e);
|
||||
}
|
||||
}
|
||||
|
||||
private SimpleOrderedMap<Object> _nextBucket() throws IOException {
|
||||
DocSet termSet = null;
|
||||
|
||||
try {
|
||||
while (term != null) {
|
||||
|
||||
if (startTermBytes != null && !StringHelper.startsWith(term, startTermBytes)) {
|
||||
break;
|
||||
}
|
||||
|
||||
int df = termsEnum.docFreq();
|
||||
if (df < effectiveMincount) {
|
||||
term = termsEnum.next();
|
||||
continue;
|
||||
}
|
||||
|
||||
if (termSet != null) {
|
||||
// termSet.decref(); // OFF-HEAP
|
||||
termSet = null;
|
||||
}
|
||||
|
||||
int c = 0;
|
||||
|
||||
if (hasSubFacets || df >= minDfFilterCache) {
|
||||
// use the filter cache
|
||||
|
||||
if (deState == null) {
|
||||
deState = new SolrIndexSearcher.DocsEnumState();
|
||||
deState.fieldName = sf.getName();
|
||||
deState.liveDocs = fcontext.searcher.getLeafReader().getLiveDocs();
|
||||
deState.termsEnum = termsEnum;
|
||||
deState.postingsEnum = postingsEnum;
|
||||
deState.minSetSizeCached = minDfFilterCache;
|
||||
}
|
||||
|
||||
if (hasSubFacets || !countOnly) {
|
||||
DocSet termsAll = fcontext.searcher.getDocSet(deState);
|
||||
termSet = docs.intersection(termsAll);
|
||||
// termsAll.decref(); // OFF-HEAP
|
||||
c = termSet.size();
|
||||
} else {
|
||||
c = fcontext.searcher.numDocs(docs, deState);
|
||||
}
|
||||
postingsEnum = deState.postingsEnum;
|
||||
|
||||
resetStats();
|
||||
|
||||
if (!countOnly) {
|
||||
collect(termSet, 0);
|
||||
}
|
||||
|
||||
} else {
|
||||
// We don't need the docset here (meaning no sub-facets).
|
||||
// if countOnly, then we are calculating some other stats...
|
||||
resetStats();
|
||||
|
||||
// lazy convert to fastForRandomSet
|
||||
if (fastForRandomSet == null) {
|
||||
fastForRandomSet = docs;
|
||||
if (docs instanceof SortedIntDocSet) { // OFF-HEAP todo: also check for native version
|
||||
SortedIntDocSet sset = (SortedIntDocSet) docs;
|
||||
fastForRandomSet = new HashDocSet(sset.getDocs(), 0, sset.size());
|
||||
}
|
||||
}
|
||||
// iterate over TermDocs to calculate the intersection
|
||||
postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
|
||||
|
||||
if (postingsEnum instanceof MultiPostingsEnum) {
|
||||
MultiPostingsEnum.EnumWithSlice[] subs = ((MultiPostingsEnum) postingsEnum).getSubs();
|
||||
int numSubs = ((MultiPostingsEnum) postingsEnum).getNumSubs();
|
||||
for (int subindex = 0; subindex < numSubs; subindex++) {
|
||||
MultiPostingsEnum.EnumWithSlice sub = subs[subindex];
|
||||
if (sub.postingsEnum == null) continue;
|
||||
int base = sub.slice.start;
|
||||
int docid;
|
||||
|
||||
if (countOnly) {
|
||||
while ((docid = sub.postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
if (fastForRandomSet.exists(docid + base)) c++;
|
||||
}
|
||||
} else {
|
||||
setNextReader(leaves[sub.slice.readerIndex]);
|
||||
while ((docid = sub.postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
if (fastForRandomSet.exists(docid + base)) {
|
||||
c++;
|
||||
collect(docid, 0);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
} else {
|
||||
int docid;
|
||||
if (countOnly) {
|
||||
while ((docid = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
if (fastForRandomSet.exists(docid)) c++;
|
||||
}
|
||||
} else {
|
||||
setNextReader(leaves[0]);
|
||||
while ((docid = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
if (fastForRandomSet.exists(docid)) {
|
||||
c++;
|
||||
collect(docid, 0);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
if (c < effectiveMincount) {
|
||||
term = termsEnum.next();
|
||||
continue;
|
||||
}
|
||||
|
||||
// handle offset and limit
|
||||
if (bucketsToSkip > 0) {
|
||||
bucketsToSkip--;
|
||||
term = termsEnum.next();
|
||||
continue;
|
||||
}
|
||||
|
||||
if (freq.limit >= 0 && ++bucketsReturned > freq.limit) {
|
||||
return null;
|
||||
}
|
||||
|
||||
// set count in case other stats depend on it
|
||||
countAcc.incrementCount(0, c);
|
||||
|
||||
// OK, we have a good bucket to return... first get bucket value before moving to next term
|
||||
Object bucketVal = sf.getType().toObject(sf, term);
|
||||
TermQuery bucketQuery = hasSubFacets ? new TermQuery(new Term(freq.field, term)) : null;
|
||||
term = termsEnum.next();
|
||||
|
||||
SimpleOrderedMap<Object> bucket = new SimpleOrderedMap<>();
|
||||
bucket.add("val", bucketVal);
|
||||
addStats(bucket, 0);
|
||||
if (hasSubFacets) {
|
||||
processSubs(bucket, bucketQuery, termSet);
|
||||
}
|
||||
|
||||
// TODO... termSet needs to stick around for streaming sub-facets?
|
||||
|
||||
return bucket;
|
||||
|
||||
}
|
||||
|
||||
} finally {
|
||||
if (termSet != null) {
|
||||
// termSet.decref(); // OFF-HEAP
|
||||
termSet = null;
|
||||
}
|
||||
}
|
||||
|
||||
// end of the iteration
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
|
@ -32,10 +32,15 @@ import org.apache.solr.common.util.SimpleOrderedMap;
|
|||
import org.apache.solr.schema.SchemaField;
|
||||
import org.apache.solr.search.DocIterator;
|
||||
|
||||
class FacetFieldProcessorNumeric extends FacetFieldProcessor {
|
||||
/**
|
||||
* Facets numbers into a hash table.
|
||||
* It currently only works with {@link NumericDocValues} (single-valued).
|
||||
*/
|
||||
class FacetFieldProcessorByHashNumeric extends FacetFieldProcessor {
|
||||
static int MAXIMUM_STARTING_TABLE_SIZE=1024; // must be a power of two, non-final to support setting by tests
|
||||
|
||||
static class LongCounts {
|
||||
/** a hash table with long keys (what we're counting) and integer values (counts) */
|
||||
private static class LongCounts {
|
||||
|
||||
static final float LOAD_FACTOR = 0.7f;
|
||||
|
||||
|
@ -55,7 +60,7 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
|
|||
}
|
||||
|
||||
/** Current number of slots in the hash table */
|
||||
public int numSlots() {
|
||||
int numSlots() {
|
||||
return vals.length;
|
||||
}
|
||||
|
||||
|
@ -130,69 +135,22 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
|
|||
|
||||
}
|
||||
|
||||
int allBucketsSlot = -1;
|
||||
|
||||
|
||||
FacetFieldProcessorNumeric(FacetContext fcontext, FacetField freq, SchemaField sf) {
|
||||
FacetFieldProcessorByHashNumeric(FacetContext fcontext, FacetField freq, SchemaField sf) {
|
||||
super(fcontext, freq, sf);
|
||||
}
|
||||
|
||||
int allBucketsSlot = -1;
|
||||
|
||||
@Override
|
||||
public void process() throws IOException {
|
||||
super.process();
|
||||
response = calcFacets();
|
||||
}
|
||||
|
||||
private void doRehash(LongCounts table) {
|
||||
if (collectAcc == null && allBucketsAcc == null) return;
|
||||
|
||||
// Our "count" acc is backed by the hash table and will already be rehashed
|
||||
// otherAccs don't need to be rehashed
|
||||
|
||||
int newTableSize = table.numSlots();
|
||||
int numSlots = newTableSize;
|
||||
final int oldAllBucketsSlot = allBucketsSlot;
|
||||
if (oldAllBucketsSlot >= 0) {
|
||||
allBucketsSlot = numSlots++;
|
||||
}
|
||||
|
||||
final int finalNumSlots = numSlots;
|
||||
final int[] mapping = table.oldToNewMapping;
|
||||
|
||||
SlotAcc.Resizer resizer = new SlotAcc.Resizer() {
|
||||
@Override
|
||||
public int getNewSize() {
|
||||
return finalNumSlots;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNewSlot(int oldSlot) {
|
||||
if (oldSlot < mapping.length) {
|
||||
return mapping[oldSlot];
|
||||
}
|
||||
if (oldSlot == oldAllBucketsSlot) {
|
||||
return allBucketsSlot;
|
||||
}
|
||||
return -1;
|
||||
}
|
||||
};
|
||||
|
||||
// NOTE: resizing isn't strictly necessary for missing/allBuckets... we could just set the new slot directly
|
||||
if (collectAcc != null) {
|
||||
collectAcc.resize(resizer);
|
||||
}
|
||||
if (allBucketsAcc != null) {
|
||||
allBucketsAcc.resize(resizer);
|
||||
}
|
||||
}
|
||||
|
||||
public SimpleOrderedMap<Object> calcFacets() throws IOException {
|
||||
|
||||
private SimpleOrderedMap<Object> calcFacets() throws IOException {
|
||||
|
||||
final FacetRangeProcessor.Calc calc = FacetRangeProcessor.getNumericCalc(sf);
|
||||
|
||||
|
||||
// TODO: it would be really nice to know the number of unique values!!!!
|
||||
|
||||
int possibleValues = fcontext.base.size();
|
||||
|
@ -212,7 +170,6 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
|
|||
|
||||
int numMissing = 0;
|
||||
|
||||
|
||||
if (freq.allBuckets) {
|
||||
allBucketsSlot = numSlots++;
|
||||
}
|
||||
|
@ -325,7 +282,6 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
//
|
||||
// collection done, time to find the top slots
|
||||
//
|
||||
|
@ -333,7 +289,7 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
|
|||
int numBuckets = 0;
|
||||
List<Object> bucketVals = null;
|
||||
if (freq.numBuckets && fcontext.isShard()) {
|
||||
bucketVals = new ArrayList(100);
|
||||
bucketVals = new ArrayList<>(100);
|
||||
}
|
||||
|
||||
int off = fcontext.isShard() ? 0 : (int) freq.offset;
|
||||
|
@ -378,13 +334,12 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
|
|||
bottom = queue.insertWithOverflow(bottom);
|
||||
}
|
||||
|
||||
|
||||
SimpleOrderedMap res = new SimpleOrderedMap();
|
||||
SimpleOrderedMap<Object> res = new SimpleOrderedMap<>();
|
||||
if (freq.numBuckets) {
|
||||
if (!fcontext.isShard()) {
|
||||
res.add("numBuckets", numBuckets);
|
||||
} else {
|
||||
SimpleOrderedMap map = new SimpleOrderedMap(2);
|
||||
SimpleOrderedMap<Object> map = new SimpleOrderedMap<>(2);
|
||||
map.add("numBuckets", numBuckets);
|
||||
map.add("vals", bucketVals);
|
||||
res.add("numBuckets", map);
|
||||
|
@ -392,7 +347,7 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
|
|||
}
|
||||
|
||||
FacetDebugInfo fdebug = fcontext.getDebugInfo();
|
||||
if (fdebug != null) fdebug.putInfoItem("numBuckets", new Long(numBuckets));
|
||||
if (fdebug != null) fdebug.putInfoItem("numBuckets", (long) numBuckets);
|
||||
|
||||
if (freq.allBuckets) {
|
||||
SimpleOrderedMap<Object> allBuckets = new SimpleOrderedMap<>();
|
||||
|
@ -419,7 +374,7 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
|
|||
sortedSlots[i] = queue.pop().slot;
|
||||
}
|
||||
|
||||
ArrayList bucketList = new ArrayList(collectCount);
|
||||
ArrayList<SimpleOrderedMap> bucketList = new ArrayList<>(collectCount);
|
||||
res.add("buckets", bucketList);
|
||||
|
||||
boolean needFilter = deferredAggs != null || freq.getSubFacets().size() > 0;
|
||||
|
@ -436,8 +391,49 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
|
|||
bucketList.add(bucket);
|
||||
}
|
||||
|
||||
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
private void doRehash(LongCounts table) {
|
||||
if (collectAcc == null && allBucketsAcc == null) return;
|
||||
|
||||
// Our "count" acc is backed by the hash table and will already be rehashed
|
||||
// otherAccs don't need to be rehashed
|
||||
|
||||
int newTableSize = table.numSlots();
|
||||
int numSlots = newTableSize;
|
||||
final int oldAllBucketsSlot = allBucketsSlot;
|
||||
if (oldAllBucketsSlot >= 0) {
|
||||
allBucketsSlot = numSlots++;
|
||||
}
|
||||
|
||||
final int finalNumSlots = numSlots;
|
||||
final int[] mapping = table.oldToNewMapping;
|
||||
|
||||
SlotAcc.Resizer resizer = new SlotAcc.Resizer() {
|
||||
@Override
|
||||
public int getNewSize() {
|
||||
return finalNumSlots;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNewSlot(int oldSlot) {
|
||||
if (oldSlot < mapping.length) {
|
||||
return mapping[oldSlot];
|
||||
}
|
||||
if (oldSlot == oldAllBucketsSlot) {
|
||||
return allBucketsSlot;
|
||||
}
|
||||
return -1;
|
||||
}
|
||||
};
|
||||
|
||||
// NOTE: resizing isn't strictly necessary for missing/allBuckets... we could just set the new slot directly
|
||||
if (collectAcc != null) {
|
||||
collectAcc.resize(resizer);
|
||||
}
|
||||
if (allBucketsAcc != null) {
|
||||
allBucketsAcc.resize(resizer);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -45,27 +45,18 @@ import org.apache.solr.search.SolrIndexSearcher;
|
|||
import org.apache.solr.search.SyntaxError;
|
||||
import org.apache.solr.util.RTimer;
|
||||
|
||||
public class FacetProcessor<FacetRequestT extends FacetRequest> {
|
||||
protected SimpleOrderedMap<Object> response;
|
||||
protected FacetContext fcontext;
|
||||
protected FacetRequestT freq;
|
||||
public abstract class FacetProcessor<FacetRequestT extends FacetRequest> {
|
||||
SimpleOrderedMap<Object> response;
|
||||
FacetContext fcontext;
|
||||
FacetRequestT freq;
|
||||
|
||||
LinkedHashMap<String,SlotAcc> accMap;
|
||||
protected SlotAcc[] accs;
|
||||
protected CountSlotAcc countAcc;
|
||||
|
||||
FacetProcessor(FacetContext fcontext, FacetRequestT freq) {
|
||||
this.fcontext = fcontext;
|
||||
this.freq = freq;
|
||||
}
|
||||
|
||||
public void process() throws IOException {
|
||||
handleDomainChanges();
|
||||
}
|
||||
SlotAcc[] accs;
|
||||
CountSlotAcc countAcc;
|
||||
|
||||
/** factory method for invoking json facet framework as whole */
|
||||
public static FacetProcessor<?> createProcessor(SolrQueryRequest req,
|
||||
Map<String, Object> params, DocSet docs){
|
||||
Map<String, Object> params, DocSet docs){
|
||||
FacetParser parser = new FacetTopParser(req);
|
||||
FacetRequest facetRequest = null;
|
||||
try {
|
||||
|
@ -83,39 +74,25 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
|
|||
return facetRequest.createFacetProcessor(fcontext);
|
||||
}
|
||||
|
||||
protected void handleDomainChanges() throws IOException {
|
||||
FacetProcessor(FacetContext fcontext, FacetRequestT freq) {
|
||||
this.fcontext = fcontext;
|
||||
this.freq = freq;
|
||||
}
|
||||
|
||||
public Object getResponse() {
|
||||
return response;
|
||||
}
|
||||
|
||||
public void process() throws IOException {
|
||||
handleDomainChanges();
|
||||
}
|
||||
|
||||
private void handleDomainChanges() throws IOException {
|
||||
if (freq.domain == null) return;
|
||||
handleFilterExclusions();
|
||||
handleBlockJoin();
|
||||
}
|
||||
|
||||
private void handleBlockJoin() throws IOException {
|
||||
if (!(freq.domain.toChildren || freq.domain.toParent)) return;
|
||||
|
||||
// TODO: avoid query parsing per-bucket somehow...
|
||||
String parentStr = freq.domain.parents;
|
||||
Query parentQuery;
|
||||
try {
|
||||
QParser parser = QParser.getParser(parentStr, fcontext.req);
|
||||
parentQuery = parser.getQuery();
|
||||
} catch (SyntaxError err) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Error parsing block join parent specification: " + parentStr);
|
||||
}
|
||||
|
||||
BitDocSet parents = fcontext.searcher.getDocSetBits(parentQuery);
|
||||
DocSet input = fcontext.base;
|
||||
DocSet result;
|
||||
|
||||
if (freq.domain.toChildren) {
|
||||
DocSet filt = fcontext.searcher.getDocSetBits( new MatchAllDocsQuery() );
|
||||
result = BlockJoin.toChildren(input, parents, filt, fcontext.qcontext);
|
||||
} else {
|
||||
result = BlockJoin.toParents(input, parents, fcontext.qcontext);
|
||||
}
|
||||
|
||||
fcontext.base = result;
|
||||
}
|
||||
|
||||
private void handleFilterExclusions() throws IOException {
|
||||
List<String> excludeTags = freq.domain.excludeTags;
|
||||
|
||||
|
@ -177,11 +154,44 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
|
|||
fcontext.base = fcontext.searcher.getDocSet(qlist);
|
||||
}
|
||||
|
||||
private void handleBlockJoin() throws IOException {
|
||||
if (!(freq.domain.toChildren || freq.domain.toParent)) return;
|
||||
|
||||
public Object getResponse() {
|
||||
return null;
|
||||
// TODO: avoid query parsing per-bucket somehow...
|
||||
String parentStr = freq.domain.parents;
|
||||
Query parentQuery;
|
||||
try {
|
||||
QParser parser = QParser.getParser(parentStr, fcontext.req);
|
||||
parentQuery = parser.getQuery();
|
||||
} catch (SyntaxError err) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Error parsing block join parent specification: " + parentStr);
|
||||
}
|
||||
|
||||
BitDocSet parents = fcontext.searcher.getDocSetBits(parentQuery);
|
||||
DocSet input = fcontext.base;
|
||||
DocSet result;
|
||||
|
||||
if (freq.domain.toChildren) {
|
||||
DocSet filt = fcontext.searcher.getDocSetBits( new MatchAllDocsQuery() );
|
||||
result = BlockJoin.toChildren(input, parents, filt, fcontext.qcontext);
|
||||
} else {
|
||||
result = BlockJoin.toParents(input, parents, fcontext.qcontext);
|
||||
}
|
||||
|
||||
fcontext.base = result;
|
||||
}
|
||||
|
||||
protected void processStats(SimpleOrderedMap<Object> bucket, DocSet docs, int docCount) throws IOException {
|
||||
if (docCount == 0 && !freq.processEmpty || freq.getFacetStats().size() == 0) {
|
||||
bucket.add("count", docCount);
|
||||
return;
|
||||
}
|
||||
createAccs(docCount, 1);
|
||||
int collected = collect(docs, 0);
|
||||
countAcc.incrementCount(0, collected);
|
||||
assert collected == docCount;
|
||||
addStats(bucket, 0);
|
||||
}
|
||||
|
||||
protected void createAccs(int docCount, int slotCount) throws IOException {
|
||||
accMap = new LinkedHashMap<>();
|
||||
|
@ -198,7 +208,6 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
|
|||
accMap.put(acc.key, acc);
|
||||
}
|
||||
|
||||
|
||||
accs = new SlotAcc[accMap.size()];
|
||||
int i=0;
|
||||
for (SlotAcc acc : accMap.values()) {
|
||||
|
@ -206,63 +215,14 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
protected void resetStats() {
|
||||
// note: only called by enum/stream prior to collect
|
||||
void resetStats() {
|
||||
countAcc.reset();
|
||||
for (SlotAcc acc : accs) {
|
||||
acc.reset();
|
||||
}
|
||||
}
|
||||
|
||||
protected void processStats(SimpleOrderedMap<Object> bucket, DocSet docs, int docCount) throws IOException {
|
||||
if (docCount == 0 && !freq.processEmpty || freq.getFacetStats().size() == 0) {
|
||||
bucket.add("count", docCount);
|
||||
return;
|
||||
}
|
||||
createAccs(docCount, 1);
|
||||
int collected = collect(docs, 0);
|
||||
countAcc.incrementCount(0, collected);
|
||||
assert collected == docCount;
|
||||
addStats(bucket, 0);
|
||||
}
|
||||
|
||||
|
||||
protected void processSubs(SimpleOrderedMap<Object> response, Query filter, DocSet domain) throws IOException {
|
||||
|
||||
// TODO: what if a zero bucket has a sub-facet with an exclusion that would yield results?
|
||||
// should we check for domain-altering exclusions, or even ask the sub-facet for
|
||||
// it's domain and then only skip it if it's 0?
|
||||
|
||||
if (domain == null || domain.size() == 0 && !freq.processEmpty) {
|
||||
return;
|
||||
}
|
||||
|
||||
for (Map.Entry<String,FacetRequest> sub : freq.getSubFacets().entrySet()) {
|
||||
// make a new context for each sub-facet since they can change the domain
|
||||
FacetContext subContext = fcontext.sub(filter, domain);
|
||||
FacetProcessor subProcessor = sub.getValue().createFacetProcessor(subContext);
|
||||
if (fcontext.getDebugInfo() != null) { // if fcontext.debugInfo != null, it means rb.debug() == true
|
||||
FacetDebugInfo fdebug = new FacetDebugInfo();
|
||||
subContext.setDebugInfo(fdebug);
|
||||
fcontext.getDebugInfo().addChild(fdebug);
|
||||
|
||||
fdebug.setReqDescription(sub.getValue().getFacetDescription());
|
||||
fdebug.setProcessor(subProcessor.getClass().getSimpleName());
|
||||
if (subContext.filter != null) fdebug.setFilter(subContext.filter.toString());
|
||||
|
||||
final RTimer timer = new RTimer();
|
||||
subProcessor.process();
|
||||
long timeElapsed = (long) timer.getTime();
|
||||
fdebug.setElapse(timeElapsed);
|
||||
fdebug.putInfoItem("domainSize", (long)subContext.base.size());
|
||||
} else {
|
||||
subProcessor.process();
|
||||
}
|
||||
|
||||
response.add( sub.getKey(), subProcessor.getResponse() );
|
||||
}
|
||||
}
|
||||
|
||||
int collect(DocSet docs, int slot) throws IOException {
|
||||
int count = 0;
|
||||
SolrIndexSearcher searcher = fcontext.searcher;
|
||||
|
@ -310,7 +270,6 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
void addStats(SimpleOrderedMap<Object> target, int slotNum) throws IOException {
|
||||
int count = countAcc.getCount(slotNum);
|
||||
target.add("count", count);
|
||||
|
@ -321,8 +280,7 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
public void fillBucket(SimpleOrderedMap<Object> bucket, Query q, DocSet result) throws IOException {
|
||||
void fillBucket(SimpleOrderedMap<Object> bucket, Query q, DocSet result) throws IOException {
|
||||
boolean needDocSet = freq.getFacetStats().size() > 0 || freq.getSubFacets().size() > 0;
|
||||
|
||||
// TODO: always collect counts or not???
|
||||
|
@ -348,7 +306,7 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
|
|||
}
|
||||
|
||||
try {
|
||||
processStats(bucket, result, (int) count);
|
||||
processStats(bucket, result, count);
|
||||
processSubs(bucket, q, result);
|
||||
} finally {
|
||||
if (result != null) {
|
||||
|
@ -358,7 +316,44 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
|
|||
}
|
||||
}
|
||||
|
||||
public static DocSet getFieldMissing(SolrIndexSearcher searcher, DocSet docs, String fieldName) throws IOException {
|
||||
void processSubs(SimpleOrderedMap<Object> response, Query filter, DocSet domain) throws IOException {
|
||||
|
||||
// TODO: what if a zero bucket has a sub-facet with an exclusion that would yield results?
|
||||
// should we check for domain-altering exclusions, or even ask the sub-facet for
|
||||
// it's domain and then only skip it if it's 0?
|
||||
|
||||
if (domain == null || domain.size() == 0 && !freq.processEmpty) {
|
||||
return;
|
||||
}
|
||||
|
||||
for (Map.Entry<String,FacetRequest> sub : freq.getSubFacets().entrySet()) {
|
||||
// make a new context for each sub-facet since they can change the domain
|
||||
FacetContext subContext = fcontext.sub(filter, domain);
|
||||
FacetProcessor subProcessor = sub.getValue().createFacetProcessor(subContext);
|
||||
if (fcontext.getDebugInfo() != null) { // if fcontext.debugInfo != null, it means rb.debug() == true
|
||||
FacetDebugInfo fdebug = new FacetDebugInfo();
|
||||
subContext.setDebugInfo(fdebug);
|
||||
fcontext.getDebugInfo().addChild(fdebug);
|
||||
|
||||
fdebug.setReqDescription(sub.getValue().getFacetDescription());
|
||||
fdebug.setProcessor(subProcessor.getClass().getSimpleName());
|
||||
if (subContext.filter != null) fdebug.setFilter(subContext.filter.toString());
|
||||
|
||||
final RTimer timer = new RTimer();
|
||||
subProcessor.process();
|
||||
long timeElapsed = (long) timer.getTime();
|
||||
fdebug.setElapse(timeElapsed);
|
||||
fdebug.putInfoItem("domainSize", (long)subContext.base.size());
|
||||
} else {
|
||||
subProcessor.process();
|
||||
}
|
||||
|
||||
response.add( sub.getKey(), subProcessor.getResponse() );
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
static DocSet getFieldMissing(SolrIndexSearcher searcher, DocSet docs, String fieldName) throws IOException {
|
||||
SchemaField sf = searcher.getSchema().getField(fieldName);
|
||||
DocSet hasVal = searcher.getDocSet(sf.getType().getRangeQuery(null, sf, null, null, false, false));
|
||||
DocSet answer = docs.andNot(hasVal);
|
||||
|
@ -366,7 +361,7 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
|
|||
return answer;
|
||||
}
|
||||
|
||||
public static Query getFieldMissingQuery(SolrIndexSearcher searcher, String fieldName) throws IOException {
|
||||
static Query getFieldMissingQuery(SolrIndexSearcher searcher, String fieldName) throws IOException {
|
||||
SchemaField sf = searcher.getSchema().getField(fieldName);
|
||||
Query hasVal = sf.getType().getRangeQuery(null, sf, null, null, false, false);
|
||||
BooleanQuery.Builder noVal = new BooleanQuery.Builder();
|
||||
|
|
|
@ -53,11 +53,6 @@ class FacetQueryProcessor extends FacetProcessor<FacetQuery> {
|
|||
super(fcontext, freq);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getResponse() {
|
||||
return response;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process() throws IOException {
|
||||
super.process();
|
||||
|
|
|
@ -93,11 +93,6 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
|
|||
response = getRangeCounts();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getResponse() {
|
||||
return response;
|
||||
}
|
||||
|
||||
private static class Range {
|
||||
Object label;
|
||||
Comparable low;
|
||||
|
|
|
@ -305,7 +305,7 @@ public class UnInvertedField extends DocTermOrds {
|
|||
|
||||
|
||||
|
||||
private void getCounts(FacetFieldProcessorUIF processor, CountSlotAcc counts) throws IOException {
|
||||
private void getCounts(FacetFieldProcessorByArrayUIF processor, CountSlotAcc counts) throws IOException {
|
||||
DocSet docs = processor.fcontext.base;
|
||||
int baseSize = docs.size();
|
||||
int maxDoc = searcher.maxDoc();
|
||||
|
@ -397,7 +397,7 @@ public class UnInvertedField extends DocTermOrds {
|
|||
|
||||
|
||||
|
||||
public void collectDocs(FacetFieldProcessorUIF processor) throws IOException {
|
||||
public void collectDocs(FacetFieldProcessorByArrayUIF processor) throws IOException {
|
||||
if (processor.collectAcc==null && processor.allBucketsAcc == null && processor.startTermIndex == 0 && processor.endTermIndex >= numTermsInField) {
|
||||
getCounts(processor, processor.countAcc);
|
||||
return;
|
||||
|
@ -408,7 +408,7 @@ public class UnInvertedField extends DocTermOrds {
|
|||
|
||||
// called from FieldFacetProcessor
|
||||
// TODO: do a callback version that can be specialized!
|
||||
public void collectDocsGeneric(FacetFieldProcessorUIF processor) throws IOException {
|
||||
public void collectDocsGeneric(FacetFieldProcessorByArrayUIF processor) throws IOException {
|
||||
use.incrementAndGet();
|
||||
|
||||
int startTermIndex = processor.startTermIndex;
|
||||
|
|
|
@ -47,8 +47,8 @@ public class TestJsonFacets extends SolrTestCaseHS {
|
|||
@BeforeClass
|
||||
public static void beforeTests() throws Exception {
|
||||
JSONTestUtil.failRepeatedKeys = true;
|
||||
origTableSize = FacetFieldProcessorNumeric.MAXIMUM_STARTING_TABLE_SIZE;
|
||||
FacetFieldProcessorNumeric.MAXIMUM_STARTING_TABLE_SIZE=2; // stress test resizing
|
||||
origTableSize = FacetFieldProcessorByHashNumeric.MAXIMUM_STARTING_TABLE_SIZE;
|
||||
FacetFieldProcessorByHashNumeric.MAXIMUM_STARTING_TABLE_SIZE=2; // stress test resizing
|
||||
initCore("solrconfig-tlog.xml","schema_latest.xml");
|
||||
}
|
||||
|
||||
|
@ -61,7 +61,7 @@ public class TestJsonFacets extends SolrTestCaseHS {
|
|||
@AfterClass
|
||||
public static void afterTests() throws Exception {
|
||||
JSONTestUtil.failRepeatedKeys = false;
|
||||
FacetFieldProcessorNumeric.MAXIMUM_STARTING_TABLE_SIZE=origTableSize;
|
||||
FacetFieldProcessorByHashNumeric.MAXIMUM_STARTING_TABLE_SIZE=origTableSize;
|
||||
if (servers != null) {
|
||||
servers.stop();
|
||||
servers = null;
|
||||
|
@ -349,11 +349,11 @@ public class TestJsonFacets extends SolrTestCaseHS {
|
|||
doStatsTemplated(client, params(p, "rows","0", "noexist","noexist_sd", "cat_s","cat_sd", "where_s","where_sd", "num_d","num_dd", "num_i","num_id", "num_is","num_lds", "num_fs","num_dds", "super_s","super_sd", "val_b","val_b", "date","date_dtd", "sparse_s","sparse_sd" ,"multi_ss","multi_sds") );
|
||||
|
||||
// multi-valued docvalues
|
||||
FacetFieldProcessorDV.unwrap_singleValued_multiDv = false; // better multi-valued coverage
|
||||
FacetFieldProcessorByArrayDV.unwrap_singleValued_multiDv = false; // better multi-valued coverage
|
||||
doStatsTemplated(client, params(p, "rows","0", "noexist","noexist_sds", "cat_s","cat_sds", "where_s","where_sds", "num_d","num_d", "num_i","num_i", "num_is","num_ids", "num_fs","num_fds", "super_s","super_sds", "val_b","val_b", "date","date_dtds", "sparse_s","sparse_sds" ,"multi_ss","multi_sds") );
|
||||
|
||||
// multi-valued docvalues
|
||||
FacetFieldProcessorDV.unwrap_singleValued_multiDv = true;
|
||||
FacetFieldProcessorByArrayDV.unwrap_singleValued_multiDv = true;
|
||||
doStatsTemplated(client, params(p, "rows","0", "noexist","noexist_sds", "cat_s","cat_sds", "where_s","where_sds", "num_d","num_d", "num_i","num_i", "num_is","num_ids", "num_fs","num_fds", "super_s","super_sds", "val_b","val_b", "date","date_dtds", "sparse_s","sparse_sds" ,"multi_ss","multi_sds") );
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue