mirror of https://github.com/apache/lucene.git
SOLR-7522: single valued numeric field faceting
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1678535 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
fb63b4c3db
commit
f41a65a457
|
@ -177,6 +177,10 @@ New Features
|
||||||
* SOLR-7461: stats.field now supports individual local params for 'countDistinct' and 'distinctValues'.
|
* SOLR-7461: stats.field now supports individual local params for 'countDistinct' and 'distinctValues'.
|
||||||
'calcdistinct' is still supported as an alias for both options (hossman)
|
'calcdistinct' is still supported as an alias for both options (hossman)
|
||||||
|
|
||||||
|
* SOLR-7522: Facet Module - Implement field/terms faceting over single-valued
|
||||||
|
numeric fields. (yonik)
|
||||||
|
|
||||||
|
|
||||||
Bug Fixes
|
Bug Fixes
|
||||||
----------------------
|
----------------------
|
||||||
|
|
||||||
|
|
|
@ -48,6 +48,7 @@ import org.apache.solr.common.SolrException;
|
||||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||||
import org.apache.solr.schema.FieldType;
|
import org.apache.solr.schema.FieldType;
|
||||||
import org.apache.solr.schema.SchemaField;
|
import org.apache.solr.schema.SchemaField;
|
||||||
|
import org.apache.solr.schema.TrieField;
|
||||||
import org.apache.solr.search.DocIterator;
|
import org.apache.solr.search.DocIterator;
|
||||||
import org.apache.solr.search.DocSet;
|
import org.apache.solr.search.DocSet;
|
||||||
import org.apache.solr.search.HashDocSet;
|
import org.apache.solr.search.HashDocSet;
|
||||||
|
@ -120,18 +121,28 @@ public class FacetField extends FacetRequest {
|
||||||
return new FacetFieldProcessorStream(fcontext, this, sf);
|
return new FacetFieldProcessorStream(fcontext, this, sf);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!multiToken || sf.hasDocValues()) {
|
org.apache.lucene.document.FieldType.NumericType ntype = ft.getNumericType();
|
||||||
|
|
||||||
|
if (sf.hasDocValues() && ntype==null) {
|
||||||
|
// single and multi-valued string docValues
|
||||||
return new FacetFieldProcessorDV(fcontext, this, sf);
|
return new FacetFieldProcessorDV(fcontext, this, sf);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (multiToken) {
|
if (!multiToken) {
|
||||||
return new FacetFieldProcessorUIF(fcontext, this, sf);
|
if (sf.getType().getNumericType() != null) {
|
||||||
|
// single valued numeric (docvalues or fieldcache)
|
||||||
|
return new FacetFieldProcessorNumeric(fcontext, this, sf);
|
||||||
} else {
|
} else {
|
||||||
// single valued string
|
// single valued string...
|
||||||
return new FacetFieldProcessorFC(fcontext, this, sf);
|
return new FacetFieldProcessorDV(fcontext, this, sf);
|
||||||
|
// what about FacetFieldProcessorFC?
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Multi-valued field cache (UIF)
|
||||||
|
return new FacetFieldProcessorUIF(fcontext, this, sf);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FacetMerger createFacetMerger(Object prototype) {
|
public FacetMerger createFacetMerger(Object prototype) {
|
||||||
return new FacetFieldMerger(this);
|
return new FacetFieldMerger(this);
|
||||||
|
@ -143,6 +154,7 @@ public class FacetField extends FacetRequest {
|
||||||
abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
|
abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
|
||||||
SchemaField sf;
|
SchemaField sf;
|
||||||
SlotAcc sortAcc;
|
SlotAcc sortAcc;
|
||||||
|
SlotAcc indexOrderAcc;
|
||||||
int effectiveMincount;
|
int effectiveMincount;
|
||||||
|
|
||||||
FacetFieldProcessor(FacetContext fcontext, FacetField freq, SchemaField sf) {
|
FacetFieldProcessor(FacetContext fcontext, FacetField freq, SchemaField sf) {
|
||||||
|
@ -157,6 +169,12 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
|
||||||
}
|
}
|
||||||
|
|
||||||
void setSortAcc(int numSlots) {
|
void setSortAcc(int numSlots) {
|
||||||
|
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);
|
||||||
|
}
|
||||||
|
|
||||||
String sortKey = freq.sortVariable;
|
String sortKey = freq.sortVariable;
|
||||||
sortAcc = accMap.get(sortKey);
|
sortAcc = accMap.get(sortKey);
|
||||||
|
|
||||||
|
@ -164,15 +182,16 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
|
||||||
if ("count".equals(sortKey)) {
|
if ("count".equals(sortKey)) {
|
||||||
sortAcc = countAcc;
|
sortAcc = countAcc;
|
||||||
} else if ("index".equals(sortKey)) {
|
} else if ("index".equals(sortKey)) {
|
||||||
sortAcc = new SortSlotAcc(fcontext);
|
sortAcc = indexOrderAcc;
|
||||||
// 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.
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static class Slot {
|
static class Slot {
|
||||||
int slot;
|
int slot;
|
||||||
|
public int tiebreakCompare(int slotA, int slotB) {
|
||||||
|
return slotB - slotA;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -249,7 +268,7 @@ abstract class FacetFieldProcessorFCBase extends FacetFieldProcessor {
|
||||||
// add a modest amount of over-request if this is a shard request
|
// 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 lim = freq.limit >= 0 ? (fcontext.isShard() ? (int)(freq.limit*1.1+4) : (int)freq.limit) : Integer.MAX_VALUE;
|
||||||
|
|
||||||
int maxsize = freq.limit > 0 ? off + lim : Integer.MAX_VALUE - 1;
|
int maxsize = (int)(freq.limit > 0 ? freq.offset + lim : Integer.MAX_VALUE - 1);
|
||||||
maxsize = Math.min(maxsize, nTerms);
|
maxsize = Math.min(maxsize, nTerms);
|
||||||
|
|
||||||
final int sortMul = freq.sortDirection.getMultiplier();
|
final int sortMul = freq.sortDirection.getMultiplier();
|
||||||
|
@ -612,6 +631,11 @@ class FacetFieldProcessorStream extends FacetFieldProcessor implements Closeable
|
||||||
if (freq.prefix != null) {
|
if (freq.prefix != null) {
|
||||||
String indexedPrefix = sf.getType().toInternal(freq.prefix);
|
String indexedPrefix = sf.getType().toInternal(freq.prefix);
|
||||||
startTermBytes = new BytesRef(indexedPrefix);
|
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();
|
Fields fields = fcontext.searcher.getLeafReader().fields();
|
||||||
|
@ -644,8 +668,6 @@ class FacetFieldProcessorStream extends FacetFieldProcessor implements Closeable
|
||||||
|
|
||||||
List<LeafReaderContext> leafList = fcontext.searcher.getTopReaderContext().leaves();
|
List<LeafReaderContext> leafList = fcontext.searcher.getTopReaderContext().leaves();
|
||||||
leaves = leafList.toArray( new LeafReaderContext[ leafList.size() ]);
|
leaves = leafList.toArray( new LeafReaderContext[ leafList.size() ]);
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,482 @@
|
||||||
|
package org.apache.solr.search.facet;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* 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.
|
||||||
|
*/
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import org.apache.lucene.index.DocValues;
|
||||||
|
import org.apache.lucene.index.LeafReaderContext;
|
||||||
|
import org.apache.lucene.index.NumericDocValues;
|
||||||
|
import org.apache.lucene.index.Term;
|
||||||
|
import org.apache.lucene.search.Query;
|
||||||
|
import org.apache.lucene.search.TermQuery;
|
||||||
|
import org.apache.lucene.util.BitUtil;
|
||||||
|
import org.apache.lucene.util.Bits;
|
||||||
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
import org.apache.lucene.util.PriorityQueue;
|
||||||
|
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||||
|
import org.apache.solr.schema.SchemaField;
|
||||||
|
import org.apache.solr.search.DocIterator;
|
||||||
|
import org.apache.solr.search.DocSet;
|
||||||
|
|
||||||
|
class FacetFieldProcessorNumeric 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 {
|
||||||
|
|
||||||
|
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;
|
||||||
|
|
||||||
|
int cardinality;
|
||||||
|
int threshold;
|
||||||
|
|
||||||
|
/** sz must be a power of two */
|
||||||
|
LongCounts(int sz) {
|
||||||
|
vals = new long[sz];
|
||||||
|
counts = new int[sz];
|
||||||
|
threshold = (int) (sz * LOAD_FACTOR);
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Current number of slots in the hash table */
|
||||||
|
public int numSlots() {
|
||||||
|
return vals.length;
|
||||||
|
}
|
||||||
|
|
||||||
|
private int hash(long val) {
|
||||||
|
// For floats: exponent bits start at bit 23 for single precision,
|
||||||
|
// and bit 52 for double precision.
|
||||||
|
// Many values will only have significant bits just to the right of that,
|
||||||
|
// and the leftmost bits will all be zero.
|
||||||
|
|
||||||
|
// For now, lets just settle to get first 8 significant mantissa bits of double or float in the lowest bits of our hash
|
||||||
|
// The upper bits of our hash will be irrelevant.
|
||||||
|
int h = (int) (val + (val >>> 44) + (val >>> 15));
|
||||||
|
return h;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** returns the slot */
|
||||||
|
int add(long val) {
|
||||||
|
if (cardinality >= threshold) {
|
||||||
|
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];
|
||||||
|
if (count == 0) {
|
||||||
|
counts[slot] = 1;
|
||||||
|
vals[slot] = val;
|
||||||
|
cardinality++;
|
||||||
|
return slot;
|
||||||
|
} else if (vals[slot] == val) {
|
||||||
|
// val is already in the set
|
||||||
|
counts[slot] = count + 1;
|
||||||
|
return slot;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void rehash() {
|
||||||
|
long[] oldVals = vals;
|
||||||
|
int[] oldCounts = counts; // after retrieving the count, this array is reused as a mapping to new array
|
||||||
|
int newCapacity = vals.length << 1;
|
||||||
|
vals = new long[newCapacity];
|
||||||
|
counts = new int[newCapacity];
|
||||||
|
threshold = (int) (newCapacity * LOAD_FACTOR);
|
||||||
|
|
||||||
|
for (int i=0; i<oldVals.length; i++) {
|
||||||
|
int count = oldCounts[i];
|
||||||
|
if (count == 0) {
|
||||||
|
oldCounts[i] = -1;
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
long val = oldVals[i];
|
||||||
|
|
||||||
|
int h = hash(val);
|
||||||
|
int slot = h & (vals.length-1);
|
||||||
|
while (counts[slot] != 0) {
|
||||||
|
slot = (slot + ((h>>7)|1)) & (vals.length-1);
|
||||||
|
}
|
||||||
|
counts[slot] = count;
|
||||||
|
vals[slot] = val;
|
||||||
|
oldCounts[i] = slot;
|
||||||
|
}
|
||||||
|
|
||||||
|
oldToNewMapping = oldCounts;
|
||||||
|
}
|
||||||
|
|
||||||
|
int cardinality() {
|
||||||
|
return cardinality;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
FacetFieldProcessorNumeric(FacetContext fcontext, FacetField freq, SchemaField sf) {
|
||||||
|
super(fcontext, freq, sf);
|
||||||
|
}
|
||||||
|
|
||||||
|
int missingSlot = -1;
|
||||||
|
int allBucketsSlot = -1;
|
||||||
|
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void process() throws IOException {
|
||||||
|
super.process();
|
||||||
|
response = calcFacets();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private void doRehash(LongCounts table) {
|
||||||
|
if (accs.length == 0) return; // TODO: FUTURE: only need to resize acc we will sort on
|
||||||
|
|
||||||
|
// Our "count" acc is backed by the hash table and will already be rehashed
|
||||||
|
|
||||||
|
int newTableSize = table.numSlots();
|
||||||
|
int numSlots = newTableSize;
|
||||||
|
final int oldMissingSlot = missingSlot;
|
||||||
|
final int oldAllBucketsSlot = allBucketsSlot;
|
||||||
|
if (oldMissingSlot >= 0) {
|
||||||
|
missingSlot = numSlots++;
|
||||||
|
}
|
||||||
|
if (allBucketsSlot >= 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 == oldMissingSlot) {
|
||||||
|
return missingSlot;
|
||||||
|
}
|
||||||
|
if (oldSlot == oldAllBucketsSlot) {
|
||||||
|
return allBucketsSlot;
|
||||||
|
}
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
for (SlotAcc acc : accs) {
|
||||||
|
acc.resize( resizer );
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public 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();
|
||||||
|
// 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) {
|
||||||
|
@Override
|
||||||
|
protected void rehash() {
|
||||||
|
super.rehash();
|
||||||
|
doRehash(this);
|
||||||
|
oldToNewMapping = null; // allow for gc
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
int numSlots = currHashSize;
|
||||||
|
|
||||||
|
int numMissing = 0;
|
||||||
|
|
||||||
|
if (freq.missing) {
|
||||||
|
missingSlot = numSlots++;
|
||||||
|
}
|
||||||
|
if (freq.allBuckets) {
|
||||||
|
allBucketsSlot = numSlots++;
|
||||||
|
}
|
||||||
|
|
||||||
|
indexOrderAcc = new SlotAcc(fcontext) {
|
||||||
|
@Override
|
||||||
|
public void collect(int doc, int slot) throws IOException {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int compare(int slotA, int slotB) {
|
||||||
|
long s1 = calc.bitsToSortableBits(table.vals[slotA]);
|
||||||
|
long s2 = calc.bitsToSortableBits(table.vals[slotB]);
|
||||||
|
return Long.compare(s1, s2);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object getValue(int slotNum) throws IOException {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void reset() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void resize(Resizer resizer) {
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
countAcc = new CountSlotAcc(fcontext) {
|
||||||
|
@Override
|
||||||
|
public void incrementCount(int slot, int count) {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getCount(int slot) {
|
||||||
|
return table.counts[slot];
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object getValue(int slotNum) {
|
||||||
|
return getCount(slotNum);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void reset() {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void collect(int doc, int slot) throws IOException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int compare(int slotA, int slotB) {
|
||||||
|
return Integer.compare( table.counts[slotA], table.counts[slotB] );
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void resize(Resizer resizer) {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
// we set the countAcc first so it won't be created here
|
||||||
|
createAccs(fcontext.base.size(), numSlots);
|
||||||
|
setSortAcc(numSlots);
|
||||||
|
prepareForCollection();
|
||||||
|
|
||||||
|
|
||||||
|
NumericDocValues values = null;
|
||||||
|
Bits docsWithField = null;
|
||||||
|
|
||||||
|
// 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;
|
||||||
|
setNextReader(ctx);
|
||||||
|
|
||||||
|
values = DocValues.getNumeric(ctx.reader(), sf.getName());
|
||||||
|
docsWithField = DocValues.getDocsWithField(ctx.reader(), sf.getName());
|
||||||
|
}
|
||||||
|
|
||||||
|
int segDoc = doc - segBase;
|
||||||
|
long val = values.get(segDoc);
|
||||||
|
if (val == 0 && !docsWithField.get(segDoc)) {
|
||||||
|
// missing
|
||||||
|
if (missingSlot >= 0) {
|
||||||
|
numMissing++;
|
||||||
|
collect(segDoc, missingSlot);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
int slot = table.add(val); // this can trigger a rehash rehash
|
||||||
|
|
||||||
|
collect(segDoc, slot);
|
||||||
|
|
||||||
|
if (allBucketsSlot >= 0) {
|
||||||
|
collect(segDoc, allBucketsSlot);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
//
|
||||||
|
// collection done, time to find the top slots
|
||||||
|
//
|
||||||
|
|
||||||
|
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, 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 res = new SimpleOrderedMap();
|
||||||
|
if (freq.numBuckets) {
|
||||||
|
if (!fcontext.isShard()) {
|
||||||
|
res.add("numBuckets", numBuckets);
|
||||||
|
} else {
|
||||||
|
SimpleOrderedMap map = new SimpleOrderedMap(2);
|
||||||
|
map.add("numBuckets", numBuckets);
|
||||||
|
map.add("vals", bucketVals);
|
||||||
|
res.add("numBuckets", map);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (freq.allBuckets) {
|
||||||
|
SimpleOrderedMap<Object> allBuckets = new SimpleOrderedMap<>();
|
||||||
|
// countAcc.setValues(allBuckets, allBucketsSlot);
|
||||||
|
allBuckets.add("count", table.numAdds);
|
||||||
|
for (SlotAcc acc : accs) {
|
||||||
|
acc.setValues(allBuckets, allBucketsSlot);
|
||||||
|
}
|
||||||
|
// allBuckets currently doesn't execute sub-facets (because it doesn't change the domain?)
|
||||||
|
res.add("allBuckets", allBuckets);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (freq.missing) {
|
||||||
|
SimpleOrderedMap<Object> missingBucket = new SimpleOrderedMap<>();
|
||||||
|
// countAcc.setValues(missingBucket, missingSlot);
|
||||||
|
missingBucket.add("count", numMissing);
|
||||||
|
for (SlotAcc acc : accs) {
|
||||||
|
acc.setValues(missingBucket, missingSlot);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (freq.getSubFacets().size() > 0) {
|
||||||
|
// TODO: we can do better than this!
|
||||||
|
DocSet missingDocSet = null;
|
||||||
|
if (missingDocSet == null) {
|
||||||
|
missingDocSet = getFieldMissing(fcontext.searcher, fcontext.base, freq.field);
|
||||||
|
}
|
||||||
|
processSubs(missingBucket, getFieldMissingQuery(fcontext.searcher, freq.field), missingDocSet);
|
||||||
|
}
|
||||||
|
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 bucketList = new ArrayList(collectCount);
|
||||||
|
res.add("buckets", bucketList);
|
||||||
|
|
||||||
|
|
||||||
|
for (int slotNum : sortedSlots) {
|
||||||
|
SimpleOrderedMap<Object> bucket = new SimpleOrderedMap<>();
|
||||||
|
Comparable val = calc.bitsToValue(table.vals[slotNum]);
|
||||||
|
bucket.add("val", val);
|
||||||
|
|
||||||
|
// add stats for this bucket
|
||||||
|
// TODO: this gets count from countAcc
|
||||||
|
// addStats(bucket, slotNum);
|
||||||
|
bucket.add("count", table.counts[slotNum]);
|
||||||
|
|
||||||
|
for (SlotAcc acc : accs) {
|
||||||
|
acc.setValues(bucket, slotNum);
|
||||||
|
}
|
||||||
|
|
||||||
|
// handle sub-facets for this bucket
|
||||||
|
if (freq.getSubFacets().size() > 0) {
|
||||||
|
Query filter = sf.getType().getFieldQuery(null, sf, calc.formatValue(val));
|
||||||
|
processSubs(bucket, filter, fcontext.searcher.getDocSet(filter, fcontext.base) );
|
||||||
|
}
|
||||||
|
|
||||||
|
bucketList.add(bucket);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
}
|
|
@ -24,6 +24,7 @@ import java.util.EnumSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.lucene.search.Query;
|
import org.apache.lucene.search.Query;
|
||||||
|
import org.apache.lucene.util.NumericUtils;
|
||||||
import org.apache.solr.common.SolrException;
|
import org.apache.solr.common.SolrException;
|
||||||
import org.apache.solr.common.params.FacetParams;
|
import org.apache.solr.common.params.FacetParams;
|
||||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||||
|
@ -101,6 +102,41 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static Calc getNumericCalc(SchemaField sf) {
|
||||||
|
Calc calc;
|
||||||
|
final FieldType ft = sf.getType();
|
||||||
|
|
||||||
|
if (ft instanceof TrieField) {
|
||||||
|
final TrieField trie = (TrieField)ft;
|
||||||
|
|
||||||
|
switch (trie.getType()) {
|
||||||
|
case FLOAT:
|
||||||
|
calc = new FloatCalc(sf);
|
||||||
|
break;
|
||||||
|
case DOUBLE:
|
||||||
|
calc = new DoubleCalc(sf);
|
||||||
|
break;
|
||||||
|
case INTEGER:
|
||||||
|
calc = new IntCalc(sf);
|
||||||
|
break;
|
||||||
|
case LONG:
|
||||||
|
calc = new LongCalc(sf);
|
||||||
|
break;
|
||||||
|
case DATE:
|
||||||
|
calc = new DateCalc(sf, null);
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new SolrException
|
||||||
|
(SolrException.ErrorCode.BAD_REQUEST,
|
||||||
|
"Expected numeric field type :" + sf);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
throw new SolrException
|
||||||
|
(SolrException.ErrorCode.BAD_REQUEST,
|
||||||
|
"Expected numeric field type :" + sf);
|
||||||
|
}
|
||||||
|
return calc;
|
||||||
|
}
|
||||||
|
|
||||||
private SimpleOrderedMap<Object> getRangeCounts() throws IOException {
|
private SimpleOrderedMap<Object> getRangeCounts() throws IOException {
|
||||||
final FieldType ft = sf.getType();
|
final FieldType ft = sf.getType();
|
||||||
|
@ -317,14 +353,22 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
|
||||||
* directly from some method -- but until then, keep this locked down
|
* directly from some method -- but until then, keep this locked down
|
||||||
* and private.
|
* and private.
|
||||||
*/
|
*/
|
||||||
private static abstract class Calc {
|
static abstract class Calc {
|
||||||
protected final SchemaField field;
|
protected final SchemaField field;
|
||||||
public Calc(final SchemaField field) {
|
public Calc(final SchemaField field) {
|
||||||
this.field = field;
|
this.field = field;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Comparable bitsToValue(long bits) {
|
||||||
|
return bits;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long bitsToSortableBits(long bits) {
|
||||||
|
return bits;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Formats a Range endpoint for use as a range label name in the response.
|
* Formats a value into a label used in a response
|
||||||
* Default Impl just uses toString()
|
* Default Impl just uses toString()
|
||||||
*/
|
*/
|
||||||
public String formatValue(final Comparable val) {
|
public String formatValue(final Comparable val) {
|
||||||
|
@ -332,7 +376,7 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Parses a String param into an Range endpoint value throwing
|
* Parses a String param into a value throwing
|
||||||
* an exception if not possible
|
* an exception if not possible
|
||||||
*/
|
*/
|
||||||
public final Comparable getValue(final String rawval) {
|
public final Comparable getValue(final String rawval) {
|
||||||
|
@ -346,7 +390,7 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Parses a String param into an Range endpoint.
|
* Parses a String param into a value.
|
||||||
* Can throw a low level format exception as needed.
|
* Can throw a low level format exception as needed.
|
||||||
*/
|
*/
|
||||||
protected abstract Comparable parseStr(final String rawval)
|
protected abstract Comparable parseStr(final String rawval)
|
||||||
|
@ -407,6 +451,16 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
|
||||||
|
|
||||||
private static class FloatCalc extends Calc {
|
private static class FloatCalc extends Calc {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Comparable bitsToValue(long bits) {
|
||||||
|
return Float.intBitsToFloat( (int)bits );
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long bitsToSortableBits(long bits) {
|
||||||
|
return NumericUtils.sortableDoubleBits(bits);
|
||||||
|
}
|
||||||
|
|
||||||
public FloatCalc(final SchemaField f) { super(f); }
|
public FloatCalc(final SchemaField f) { super(f); }
|
||||||
@Override
|
@Override
|
||||||
protected Float parseStr(String rawval) {
|
protected Float parseStr(String rawval) {
|
||||||
|
@ -418,6 +472,15 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
private static class DoubleCalc extends Calc {
|
private static class DoubleCalc extends Calc {
|
||||||
|
@Override
|
||||||
|
public Comparable bitsToValue(long bits) {
|
||||||
|
return Double.longBitsToDouble(bits);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long bitsToSortableBits(long bits) {
|
||||||
|
return NumericUtils.sortableDoubleBits(bits);
|
||||||
|
}
|
||||||
|
|
||||||
public DoubleCalc(final SchemaField f) { super(f); }
|
public DoubleCalc(final SchemaField f) { super(f); }
|
||||||
@Override
|
@Override
|
||||||
|
@ -463,6 +526,12 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
|
||||||
throw new IllegalArgumentException("SchemaField must use field type extending TrieDateField or DateRangeField");
|
throw new IllegalArgumentException("SchemaField must use field type extending TrieDateField or DateRangeField");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Comparable bitsToValue(long bits) {
|
||||||
|
return new Date(bits);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String formatValue(Comparable val) {
|
public String formatValue(Comparable val) {
|
||||||
return ((TrieDateField)field.getType()).toExternal( (Date)val );
|
return ((TrieDateField)field.getType()).toExternal( (Date)val );
|
||||||
|
|
|
@ -206,8 +206,13 @@ class FacetProcessor<FacetRequestT extends FacetRequest> {
|
||||||
|
|
||||||
protected void createAccs(int docCount, int slotCount) throws IOException {
|
protected void createAccs(int docCount, int slotCount) throws IOException {
|
||||||
accMap = new LinkedHashMap<String,SlotAcc>();
|
accMap = new LinkedHashMap<String,SlotAcc>();
|
||||||
|
|
||||||
|
// allow a custom count acc to be used
|
||||||
|
if (countAcc == null) {
|
||||||
countAcc = new CountSlotArrAcc(fcontext, slotCount);
|
countAcc = new CountSlotArrAcc(fcontext, slotCount);
|
||||||
countAcc.key = "count";
|
countAcc.key = "count";
|
||||||
|
}
|
||||||
|
|
||||||
for (Map.Entry<String,AggValueSource> entry : freq.getFacetStats().entrySet()) {
|
for (Map.Entry<String,AggValueSource> entry : freq.getFacetStats().entrySet()) {
|
||||||
SlotAcc acc = entry.getValue().createSlotAcc(fcontext, docCount, slotCount);
|
SlotAcc acc = entry.getValue().createSlotAcc(fcontext, docCount, slotCount);
|
||||||
acc.key = entry.getKey();
|
acc.key = entry.getKey();
|
||||||
|
|
|
@ -28,6 +28,7 @@ import java.util.Map;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
|
||||||
import com.tdunning.math.stats.AVLTreeDigest;
|
import com.tdunning.math.stats.AVLTreeDigest;
|
||||||
|
import org.apache.lucene.queryparser.flexible.standard.processors.NumericQueryNodeProcessor;
|
||||||
import org.apache.lucene.util.LuceneTestCase;
|
import org.apache.lucene.util.LuceneTestCase;
|
||||||
import org.apache.lucene.util.packed.GrowableWriter;
|
import org.apache.lucene.util.packed.GrowableWriter;
|
||||||
import org.apache.lucene.util.packed.PackedInts;
|
import org.apache.lucene.util.packed.PackedInts;
|
||||||
|
@ -44,10 +45,13 @@ import org.junit.Test;
|
||||||
public class TestJsonFacets extends SolrTestCaseHS {
|
public class TestJsonFacets extends SolrTestCaseHS {
|
||||||
|
|
||||||
private static SolrInstances servers; // for distributed testing
|
private static SolrInstances servers; // for distributed testing
|
||||||
|
private static int origTableSize;
|
||||||
|
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
public static void beforeTests() throws Exception {
|
public static void beforeTests() throws Exception {
|
||||||
JSONTestUtil.failRepeatedKeys = true;
|
JSONTestUtil.failRepeatedKeys = true;
|
||||||
|
origTableSize = FacetFieldProcessorNumeric.MAXIMUM_STARTING_TABLE_SIZE;
|
||||||
|
FacetFieldProcessorNumeric.MAXIMUM_STARTING_TABLE_SIZE=2; // stress test resizing
|
||||||
initCore("solrconfig-tlog.xml","schema_latest.xml");
|
initCore("solrconfig-tlog.xml","schema_latest.xml");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -60,6 +64,7 @@ public class TestJsonFacets extends SolrTestCaseHS {
|
||||||
@AfterClass
|
@AfterClass
|
||||||
public static void afterTests() throws Exception {
|
public static void afterTests() throws Exception {
|
||||||
JSONTestUtil.failRepeatedKeys = false;
|
JSONTestUtil.failRepeatedKeys = false;
|
||||||
|
FacetFieldProcessorNumeric.MAXIMUM_STARTING_TABLE_SIZE=origTableSize;
|
||||||
if (servers != null) {
|
if (servers != null) {
|
||||||
servers.stop();
|
servers.stop();
|
||||||
servers = null;
|
servers = null;
|
||||||
|
@ -368,6 +373,7 @@ public class TestJsonFacets extends SolrTestCaseHS {
|
||||||
client.commit();
|
client.commit();
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
// straight query facets
|
// straight query facets
|
||||||
client.testJQ(params(p, "q", "*:*"
|
client.testJQ(params(p, "q", "*:*"
|
||||||
, "json.facet", "{catA:{query:{q:'${cat_s}:A'}}, catA2:{query:{query:'${cat_s}:A'}}, catA3:{query:'${cat_s}:A'} }"
|
, "json.facet", "{catA:{query:{q:'${cat_s}:A'}}, catA2:{query:{query:'${cat_s}:A'}}, catA3:{query:'${cat_s}:A'} }"
|
||||||
|
@ -883,6 +889,50 @@ public class TestJsonFacets extends SolrTestCaseHS {
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
||||||
|
//
|
||||||
|
// facet on numbers
|
||||||
|
//
|
||||||
|
client.testJQ(params(p, "q", "*:*"
|
||||||
|
, "json.facet", "{" +
|
||||||
|
" f1:{ type:field, field:${num_i} }" +
|
||||||
|
",f2:{ type:field, field:${num_i}, sort:'count asc' }" +
|
||||||
|
",f3:{ type:field, field:${num_i}, sort:'index asc' }" +
|
||||||
|
",f4:{ type:field, field:${num_i}, sort:'index desc' }" +
|
||||||
|
",f5:{ type:field, field:${num_i}, sort:'index desc', limit:1, missing:true, allBuckets:true, numBuckets:true }" +
|
||||||
|
",f6:{ type:field, field:${num_i}, sort:'index desc', mincount:2, numBuckets:true }" + // mincount should lower numbuckets
|
||||||
|
",f7:{ type:field, field:${num_i}, sort:'index desc', offset:2, numBuckets:true }" + // test offset
|
||||||
|
",f8:{ type:field, field:${num_i}, sort:'index desc', offset:100, numBuckets:true }" + // test high offset
|
||||||
|
",f9:{ type:field, field:${num_i}, sort:'x desc', facet:{x:'avg(${num_d})'}, missing:true, allBuckets:true, numBuckets:true }" + // test stats
|
||||||
|
",f10:{ type:field, field:${num_i}, facet:{a:{query:'${cat_s}:A'}}, missing:true, allBuckets:true, numBuckets:true }" + // test subfacets
|
||||||
|
"}"
|
||||||
|
)
|
||||||
|
, "facets=={count:6 " +
|
||||||
|
",f1:{ buckets:[{val:-5,count:2},{val:2,count:1},{val:3,count:1},{val:7,count:1} ] } " +
|
||||||
|
",f2:{ buckets:[{val:2,count:1},{val:3,count:1},{val:7,count:1},{val:-5,count:2} ] } " +
|
||||||
|
",f3:{ buckets:[{val:-5,count:2},{val:2,count:1},{val:3,count:1},{val:7,count:1} ] } " +
|
||||||
|
",f4:{ buckets:[{val:7,count:1},{val:3,count:1},{val:2,count:1},{val:-5,count:2} ] } " +
|
||||||
|
",f5:{ buckets:[{val:7,count:1}] , numBuckets:4, allBuckets:{count:5}, missing:{count:1} } " +
|
||||||
|
",f6:{ buckets:[{val:-5,count:2}] , numBuckets:1 } " +
|
||||||
|
",f7:{ buckets:[{val:2,count:1},{val:-5,count:2}] , numBuckets:4 } " +
|
||||||
|
",f8:{ buckets:[] , numBuckets:4 } " +
|
||||||
|
",f9:{ buckets:[{val:7,count:1,x:11.0},{val:2,count:1,x:4.0},{val:3,count:1,x:2.0},{val:-5,count:2,x:-7.0} ], numBuckets:4, allBuckets:{count:5,x:0.6},missing:{count:1,x:0.0} } " + // TODO: should missing exclude "x" because no values were collected?
|
||||||
|
",f10:{ buckets:[{val:-5,count:2,a:{count:0}},{val:2,count:1,a:{count:1}},{val:3,count:1,a:{count:1}},{val:7,count:1,a:{count:0}} ], numBuckets:4, allBuckets:{count:5},missing:{count:1,a:{count:0}} } " +
|
||||||
|
"}"
|
||||||
|
);
|
||||||
|
|
||||||
|
|
||||||
|
// facet on a float field - shares same code with integers/longs currently, so we only need to test labels/sorting
|
||||||
|
client.testJQ(params(p, "q", "*:*"
|
||||||
|
, "json.facet", "{" +
|
||||||
|
" f1:{ type:field, field:${num_d} }" +
|
||||||
|
",f2:{ type:field, field:${num_d}, sort:'index desc' }" +
|
||||||
|
"}"
|
||||||
|
)
|
||||||
|
, "facets=={count:6 " +
|
||||||
|
",f1:{ buckets:[{val:-9.0,count:1},{val:-5.0,count:1},{val:2.0,count:1},{val:4.0,count:1},{val:11.0,count:1} ] } " +
|
||||||
|
",f2:{ buckets:[{val:11.0,count:1},{val:4.0,count:1},{val:2.0,count:1},{val:-5.0,count:1},{val:-9.0,count:1} ] } " +
|
||||||
|
"}"
|
||||||
|
);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue