You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by yo...@apache.org on 2015/03/16 05:40:24 UTC
svn commit: r1666856 [2/5] - in /lucene/dev/trunk/solr: ./
contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/
core/src/java/org/apache/solr/handler/
core/src/java/org/apache/solr/handler/component/
core/src/java/org/apache/solr/requ...
Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetField.java?rev=1666856&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetField.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetField.java Mon Mar 16 04:40:23 2015
@@ -0,0 +1,813 @@
+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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+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.SortedDocValues;
+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.BytesRefBuilder;
+import org.apache.lucene.util.PriorityQueue;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.UnicodeUtil;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.request.SimpleFacets;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.DocIterator;
+import org.apache.solr.search.DocSet;
+import org.apache.solr.search.HashDocSet;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.search.SortedIntDocSet;
+
+
+public class FacetField extends FacetRequest {
+ String field;
+ long offset;
+ long limit = 10;
+ long mincount = 1;
+ boolean missing;
+ boolean numBuckets;
+ String prefix;
+ String sortVariable;
+ SortDirection sortDirection;
+ FacetMethod method;
+ boolean allBuckets; // show cumulative stats across all buckets (this can be different than non-bucketed stats across all docs because of multi-valued docs)
+ int cacheDf; // 0 means "default", -1 means "never cache"
+
+ // TODO: put this somewhere more generic?
+ public static enum SortDirection {
+ asc(-1) ,
+ desc(1);
+
+ private final int multiplier;
+ private SortDirection(int multiplier) {
+ this.multiplier = multiplier;
+ }
+
+ // asc==-1, desc==1
+ public int getMultiplier() {
+ return multiplier;
+ }
+ }
+
+ public static enum FacetMethod {
+ ENUM,
+ STREAM,
+ FIELDCACHE,
+ SMART,
+ ;
+
+ public static FacetMethod fromString(String method) {
+ if (method == null || method.length()==0) return null;
+ if ("enum".equals(method)) {
+ return ENUM;
+ } else if ("fc".equals(method) || "fieldcache".equals(method)) {
+ return FIELDCACHE;
+ } else if ("smart".equals(method)) {
+ return SMART;
+ } else if ("stream".equals(method)) {
+ return STREAM;
+ }
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown FacetField method " + method);
+ }
+ }
+
+
+ @Override
+ public FacetProcessor createFacetProcessor(FacetContext fcontext) {
+ SchemaField sf = fcontext.searcher.getSchema().getField(field);
+ FieldType ft = sf.getType();
+ boolean multiToken = sf.multiValued() || ft.multiValuedFieldCache();
+
+ if (method == FacetMethod.ENUM && sf.indexed()) {
+ throw new UnsupportedOperationException();
+ } else if (method == FacetMethod.STREAM && sf.indexed()) {
+ return new FacetFieldProcessorStream(fcontext, this, sf);
+ }
+
+ if (multiToken) {
+ return new FacetFieldProcessorUIF(fcontext, this, sf);
+ } else {
+ // single valued string
+ return new FacetFieldProcessorFC(fcontext, this, sf);
+ }
+ }
+
+ @Override
+ public FacetMerger createFacetMerger(Object prototype) {
+ return new FacetFieldMerger(this);
+ }
+}
+
+
+
+abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
+ SchemaField sf;
+ SlotAcc sortAcc;
+
+ FacetFieldProcessor(FacetContext fcontext, FacetField freq, SchemaField sf) {
+ super(fcontext, freq);
+ this.sf = sf;
+ }
+
+ @Override
+ public Object getResponse() {
+ return response;
+ }
+
+ void setSortAcc(int numSlots) {
+ String sortKey = freq.sortVariable;
+ sortAcc = accMap.get(sortKey);
+
+ if (sortAcc == null) {
+ if ("count".equals(sortKey)) {
+ sortAcc = countAcc;
+ } else if ("index".equals(sortKey)) {
+ sortAcc = new SortSlotAcc(fcontext);
+ // 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 {
+ int slot;
+ }
+}
+
+
+// base class for FC style of facet counting (single and multi-valued strings)
+abstract class FacetFieldProcessorFCBase extends FacetFieldProcessor {
+ BytesRefBuilder prefixRef;
+ int startTermIndex;
+ int endTermIndex;
+ int nTerms;
+ int nDocs;
+ int maxSlots;
+ int allBucketsSlot;
+
+
+ public FacetFieldProcessorFCBase(FacetContext fcontext, FacetField freq, SchemaField sf) {
+ super(fcontext, freq, sf);
+ }
+
+ @Override
+ public void process() throws IOException {
+ sf = fcontext.searcher.getSchema().getField(freq.field);
+ response = getFieldCacheCounts();
+ }
+
+
+ abstract protected BytesRef lookupOrd(int ord) throws IOException;
+ abstract protected void findStartAndEndOrds() throws IOException;
+ abstract protected void collectDocs() throws IOException;
+
+
+ public 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();
+
+ // if we need an extra slot for the "missing" bucket, and it wasn't able to be tacked onto the beginning,
+ // then lets add room for it at the end.
+ maxSlots = (freq.missing && startTermIndex != -1) ? nTerms + 1 : nTerms;
+
+ if (freq.allBuckets) {
+ allBucketsSlot = maxSlots;
+ maxSlots++;
+ } else {
+ allBucketsSlot = -1;
+ }
+ createAccs(nDocs, maxSlots);
+ setSortAcc(maxSlots);
+ prepareForCollection();
+
+ collectDocs();
+
+ return findTopSlots();
+ }
+
+
+ protected 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 = freq.limit > 0 ? off + 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;
+ int effectiveMincount = (int)(fcontext.isShard() ? Math.min(1 , freq.mincount) : freq.mincount);
+ for (int i = (startTermIndex == -1) ? 1 : 0; i < nTerms; i++) {
+ if (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 {
+ // 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 map = new SimpleOrderedMap(2);
+ map.add("numBuckets", numBuckets);
+ map.add("vals", bucketVals);
+ res.add("numBuckets", map);
+ }
+ }
+
+ // 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<>();
+ for (SlotAcc acc : accs) {
+ countAcc.setValues(allBuckets, allBucketsSlot);
+ acc.setValues(allBuckets, allBucketsSlot);
+ }
+ res.add("allBuckets", allBuckets);
+ }
+
+ ArrayList bucketList = new ArrayList(collectCount);
+ res.add("buckets", bucketList);
+
+
+ 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);
+ // add stats for this bucket
+ addStats(bucket, slotNum);
+
+ // handle sub-facets for this bucket
+ if (freq.getSubFacets().size() > 0) {
+ FacetContext subContext = fcontext.sub();
+ subContext.base = fcontext.searcher.getDocSet(new TermQuery(new Term(sf.getName(), br.clone())), fcontext.base);
+ try {
+ fillBucketSubs(bucket, subContext);
+ } finally {
+ // subContext.base.decref(); // OFF-HEAP
+ // subContext.base = null; // do not modify context after creation... there may be deferred execution (i.e. streaming)
+ }
+ }
+
+ bucketList.add(bucket);
+ }
+
+ if (freq.missing) {
+ SimpleOrderedMap<Object> missingBucket = new SimpleOrderedMap<>();
+ DocSet missingDocSet = null;
+ try {
+ if (startTermIndex == -1) {
+ addStats(missingBucket, 0);
+ } else {
+ missingDocSet = getFieldMissing(fcontext.searcher, fcontext.base, freq.field);
+ // an extra slot was added to the end for this missing bucket
+ countAcc.incrementCount(nTerms, missingDocSet.size());
+ collect(missingDocSet, nTerms);
+ addStats(missingBucket, nTerms);
+ }
+
+ if (freq.getSubFacets().size() > 0) {
+ FacetContext subContext = fcontext.sub();
+ // TODO: we can do better than this!
+ if (missingDocSet == null) {
+ missingDocSet = getFieldMissing(fcontext.searcher, fcontext.base, freq.field);
+ }
+ subContext.base = missingDocSet;
+ fillBucketSubs(missingBucket, subContext);
+ }
+
+ res.add("missing", missingBucket);
+ } finally {
+ if (missingDocSet != null) {
+ // missingDocSet.decref(); // OFF-HEAP
+ missingDocSet = null;
+ }
+ }
+ }
+
+ return res;
+ }
+
+
+}
+
+
+class FacetFieldProcessorFC extends FacetFieldProcessorFCBase {
+ SortedDocValues sortedDocValues;
+
+
+ public FacetFieldProcessorFC(FacetContext fcontext, FacetField freq, SchemaField sf) {
+ super(fcontext, freq, sf);
+ }
+
+ protected BytesRef lookupOrd(int ord) throws IOException {
+ return sortedDocValues.lookupOrd(ord);
+ }
+
+ protected void findStartAndEndOrds() throws IOException {
+ sortedDocValues = FieldUtil.getSortedDocValues(fcontext.qcontext, sf, null);
+
+ if (prefixRef != null) {
+ startTermIndex = sortedDocValues.lookupTerm(prefixRef.get());
+ if (startTermIndex < 0) startTermIndex = -startTermIndex - 1;
+ prefixRef.append(UnicodeUtil.BIG_TERM);
+ endTermIndex = sortedDocValues.lookupTerm(prefixRef.get());
+ assert endTermIndex < 0;
+ endTermIndex = -endTermIndex - 1;
+ } else {
+ startTermIndex = 0;
+ endTermIndex = sortedDocValues.getValueCount();
+ }
+
+ // optimize collecting the "missing" bucket when startTermindex is 0 (since the "missing" ord is -1)
+ startTermIndex = startTermIndex==0 && freq.missing ? -1 : startTermIndex;
+
+ nTerms = endTermIndex - startTermIndex;
+ }
+
+ protected void collectDocs() throws IOException {
+ 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);
+ }
+
+ int term = sortedDocValues.getOrd( doc );
+ int arrIdx = term - startTermIndex;
+ if (arrIdx>=0 && arrIdx<nTerms) {
+ countAcc.incrementCount(arrIdx, 1);
+ collect(doc - segBase, arrIdx); // per-seg collectors
+ if (allBucketsSlot >= 0 && term >= 0) {
+ countAcc.incrementCount(allBucketsSlot, 1);
+ collect(doc - segBase, allBucketsSlot); // per-seg collectors
+ }
+ }
+ }
+ }
+
+}
+
+// UnInvertedField implementation of field faceting
+class FacetFieldProcessorUIF extends FacetFieldProcessorFC {
+ UnInvertedField uif;
+ TermsEnum te;
+
+ FacetFieldProcessorUIF(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 BytesRef lookupOrd(int ord) throws IOException {
+ return uif.getTermValue(te, ord);
+ }
+
+ @Override
+ protected void collectDocs() throws IOException {
+ uif.collectDocs(this);
+ }
+}
+
+
+
+class FacetFieldProcessorStream 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;
+
+
+
+ FacetFieldProcessorStream(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 {
+ // 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(FacetFieldProcessorStream.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();
+ }
+ });
+ }
+
+
+
+ public 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);
+ prepareForCollection();
+
+ // Minimum term docFreq in order to use the filterCache for that term.
+ int defaultMinDf = Math.max(fcontext.searcher.maxDoc() >> 4, 3); // (minimum of 3 is for test coverage purposes)
+ int minDfFilterCache = freq.cacheDf == 0 ? defaultMinDf : freq.cacheDf;
+ if (minDfFilterCache == -1) minDfFilterCache = Integer.MAX_VALUE; // -1 means never cache
+
+ docs = fcontext.base;
+ fastForRandomSet = null;
+
+ if (freq.prefix != null) {
+ String indexedPrefix = sf.getType().toInternal(freq.prefix);
+ startTermBytes = new BytesRef(indexedPrefix);
+ }
+
+ 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(null);
+
+ // 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() ]);
+
+
+ }
+
+
+ public SimpleOrderedMap<Object> nextBucket() {
+ try {
+ return _nextBucket();
+ } catch (Exception e) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Error during facet streaming", e);
+ }
+ }
+
+ public SimpleOrderedMap<Object> _nextBucket() throws IOException {
+ int effectiveMincount = (int)(fcontext.isShard() ? Math.min(1 , freq.mincount) : freq.mincount);
+ 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(null, 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);
+ term = termsEnum.next();
+
+ SimpleOrderedMap<Object> bucket = new SimpleOrderedMap<>();
+ bucket.add("val", bucketVal);
+ addStats(bucket, 0);
+ if (hasSubFacets) {
+ processSubs(bucket, 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;
+ }
+
+
+
+}
\ No newline at end of file
Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java?rev=1666856&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java Mon Mar 16 04:40:23 2015
@@ -0,0 +1,824 @@
+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.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.FacetParams;
+import org.apache.solr.common.params.ShardParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.handler.component.ResponseBuilder;
+import org.apache.solr.handler.component.SearchComponent;
+import org.apache.solr.handler.component.ShardRequest;
+import org.apache.solr.handler.component.ShardResponse;
+import org.apache.solr.search.QueryContext;
+import org.apache.solr.search.SyntaxError;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FacetModule extends SearchComponent {
+ public static Logger log = LoggerFactory.getLogger(FacetModule.class);
+
+ public static final String COMPONENT_NAME = "facet_module";
+
+ // Ensure these don't overlap with other PURPOSE flags in ShardRequest
+ // The largest current flag in ShardRequest is 0x00002000
+ // We'll put our bits in the middle to avoid future ones in ShardRequest and
+ // custom ones that may start at the top.
+ public final static int PURPOSE_GET_JSON_FACETS = 0x00100000;
+ public final static int PURPOSE_REFINE_JSON_FACETS = 0x00200000;
+
+ // Internal information passed down from the top level to shards for distributed faceting.
+ private final static String FACET_STATE = "_facet_";
+
+
+ public FacetComponentState getFacetComponentState(ResponseBuilder rb) {
+ // TODO: put a map on ResponseBuilder?
+ // rb.componentInfo.get(FacetComponentState.class);
+ return (FacetComponentState) rb.req.getContext().get(FacetComponentState.class);
+ }
+
+ @Override
+ public void process(ResponseBuilder rb) throws IOException {
+ // if this is null, faceting is not enabled
+ FacetComponentState facetState = getFacetComponentState(rb);
+ if (facetState == null) return;
+
+ boolean isShard = rb.req.getParams().getBool(ShardParams.IS_SHARD, false);
+
+ FacetContext fcontext = new FacetContext();
+ fcontext.base = rb.getResults().docSet;
+ fcontext.req = rb.req;
+ fcontext.searcher = rb.req.getSearcher();
+ fcontext.qcontext = QueryContext.newContext(fcontext.searcher);
+ if (isShard) {
+ fcontext.flags |= FacetContext.IS_SHARD;
+ }
+
+ FacetProcessor fproc = facetState.facetRequest.createFacetProcessor(fcontext);
+ fproc.process();
+ rb.rsp.add("facets", fproc.getResponse());
+ }
+
+
+ @Override
+ public void prepare(ResponseBuilder rb) throws IOException {
+ Map<String,Object> json = rb.req.getJSON();
+ Map<String,Object> jsonFacet = null;
+ if (json == null) {
+ int version = rb.req.getParams().getInt("facet.version",1);
+ if (version <= 1) return;
+ boolean facetsEnabled = rb.req.getParams().getBool(FacetParams.FACET, false);
+ if (!facetsEnabled) return;
+ jsonFacet = new LegacyFacet(rb.req.getParams()).getLegacy();
+ } else {
+ jsonFacet = (Map<String, Object>) json.get("facet");
+ }
+ if (jsonFacet == null) return;
+
+ SolrParams params = rb.req.getParams();
+
+ boolean isShard = params.getBool(ShardParams.IS_SHARD, false);
+ if (isShard) {
+ String jfacet = params.get(FACET_STATE);
+ if (jfacet == null) {
+ // if this is a shard request, but there is no facet state, then don't do anything.
+ return;
+ }
+ }
+
+ // At this point, we know we need to do something. Create and save the state.
+ rb.setNeedDocSet(true);
+
+ // Parse the facet in the prepare phase?
+ FacetParser parser = new FacetTopParser(rb.req);
+ FacetRequest facetRequest = null;
+ try {
+ facetRequest = parser.parse(jsonFacet);
+ } catch (SyntaxError syntaxError) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, syntaxError);
+ }
+
+ FacetComponentState fcState = new FacetComponentState();
+ fcState.rb = rb;
+ fcState.isShard = isShard;
+ fcState.facetCommands = jsonFacet;
+ fcState.facetRequest = facetRequest;
+
+ rb.req.getContext().put(FacetComponentState.class, fcState);
+ }
+
+
+
+ @Override
+ public int distributedProcess(ResponseBuilder rb) throws IOException {
+ FacetComponentState facetState = getFacetComponentState(rb);
+ if (facetState == null) return ResponseBuilder.STAGE_DONE;
+
+ return ResponseBuilder.STAGE_DONE;
+ }
+
+ @Override
+ public void modifyRequest(ResponseBuilder rb, SearchComponent who,ShardRequest sreq) {
+ FacetComponentState facetState = getFacetComponentState(rb);
+ if (facetState == null) return;
+
+ if ((sreq.purpose & ShardRequest.PURPOSE_GET_TOP_IDS) != 0) {
+ sreq.purpose |= FacetModule.PURPOSE_GET_JSON_FACETS;
+ sreq.params.set(FACET_STATE, "{}");
+ } else {
+ // turn off faceting on other requests
+ sreq.params.remove("json.facet");
+ sreq.params.remove(FACET_STATE);
+ }
+ }
+
+ @Override
+ public void handleResponses(ResponseBuilder rb, ShardRequest sreq) {
+ FacetComponentState facetState = getFacetComponentState(rb);
+ if (facetState == null) return;
+
+ for (ShardResponse shardRsp : sreq.responses) {
+ SolrResponse rsp = shardRsp.getSolrResponse();
+ NamedList<Object> top = rsp.getResponse();
+ Object facet = top.get("facets");
+ if (facet == null) continue;
+ if (facetState.merger == null) {
+ facetState.merger = facetState.facetRequest.createFacetMerger(facet);
+ }
+ facetState.merger.merge(facet);
+ }
+ }
+
+ @Override
+ public void finishStage(ResponseBuilder rb) {
+ if (rb.stage != ResponseBuilder.STAGE_GET_FIELDS) return;
+
+ FacetComponentState facetState = getFacetComponentState(rb);
+ if (facetState == null) return;
+
+ if (facetState.merger != null) {
+ rb.rsp.add("facets", facetState.merger.getMergedResult());
+ }
+ }
+
+ @Override
+ public String getDescription() {
+ return "Heliosearch Faceting";
+ }
+
+ @Override
+ public String getSource() {
+ return null;
+ }
+
+}
+
+
+class FacetComponentState {
+ ResponseBuilder rb;
+ Map<String,Object> facetCommands;
+ FacetRequest facetRequest;
+ boolean isShard;
+
+ //
+ // Only used for distributed search
+ //
+ FacetMerger merger;
+}
+
+//
+// The FacetMerger code is in the prototype stage, and this is the reason that
+// many implementations are all in this file. They can be moved to separate
+// files after the interfaces are locked down more.
+//
+
+class FacetMerger {
+ public void merge(Object facetResult) {
+
+ }
+
+ public Object getMergedResult() {
+ return null; // TODO
+ }
+}
+
+
+abstract class FacetSortableMerger extends FacetMerger {
+ public void prepareSort() {
+ }
+
+ /** Return the normal comparison sort order. The sort direction is only to be used in special circumstances (such as making NaN sort
+ * last regardless of sort order.) Normal sorters do not need to pay attention to direction.
+ */
+ public abstract int compareTo(FacetSortableMerger other, FacetField.SortDirection direction);
+}
+
+class FacetDoubleMerger extends FacetSortableMerger {
+ double val;
+
+ @Override
+ public void merge(Object facetResult) {
+ val += ((Number)facetResult).doubleValue();
+ }
+
+ @Override
+ public Object getMergedResult() {
+ return val;
+ }
+
+ @Override
+ public int compareTo(FacetSortableMerger other, FacetField.SortDirection direction) {
+ return compare(val, ((FacetDoubleMerger)other).val, direction);
+ }
+
+
+ public static int compare(double a, double b, FacetField.SortDirection direction) {
+ if (a < b) return -1;
+ if (a > b) return 1;
+
+ if (a != a) { // a==NaN
+ if (b != b) {
+ return 0; // both NaN
+ }
+ return -1 * direction.getMultiplier(); // asc==-1, so this will put NaN at end of sort
+ }
+
+ if (b != b) { // b is NaN so a is greater
+ return 1 * direction.getMultiplier(); // if sorting asc, make a less so NaN is at end
+ }
+
+ // consider +-0 to be equal
+ return 0;
+ }
+}
+
+
+class FacetLongMerger extends FacetSortableMerger {
+ long val;
+
+ @Override
+ public void merge(Object facetResult) {
+ val += ((Number)facetResult).longValue();
+ }
+
+ @Override
+ public Object getMergedResult() {
+ return val;
+ }
+
+ @Override
+ public int compareTo(FacetSortableMerger other, FacetField.SortDirection direction) {
+ return Long.compare(val, ((FacetLongMerger)other).val);
+ }
+}
+
+
+// base class for facets that create buckets (and can hence have sub-facets)
+class FacetBucketMerger<FacetRequestT extends FacetRequest> extends FacetMerger {
+ FacetRequestT freq;
+
+ public FacetBucketMerger(FacetRequestT freq) {
+ this.freq = freq;
+ }
+
+ /** Bucketval is the representative value for the bucket. Only applicable to terms and range queries to distinguish buckets. */
+ FacetBucket newBucket(Comparable bucketVal) {
+ return new FacetBucket(this, bucketVal);
+ }
+
+ // do subs...
+
+ // callback stuff for buckets?
+ // passing object gives us a chance to specialize based on value
+ FacetMerger createFacetMerger(String key, Object val) {
+ FacetRequest sub = freq.getSubFacets().get(key);
+ if (sub != null) {
+ return sub.createFacetMerger(val);
+ }
+
+ AggValueSource subStat = freq.getFacetStats().get(key);
+ if (subStat != null) {
+ return subStat.createFacetMerger(val);
+ }
+
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "no merger for key=" + key + " , val=" + val);
+ }
+}
+
+class FacetQueryMerger extends FacetBucketMerger<FacetQuery> {
+ FacetBucket bucket;
+
+ public FacetQueryMerger(FacetQuery freq) {
+ super(freq);
+ }
+
+ @Override
+ public void merge(Object facet) {
+ if (bucket == null) {
+ bucket = newBucket(null);
+ }
+ bucket.mergeBucket((SimpleOrderedMap) facet);
+ }
+
+ @Override
+ public Object getMergedResult() {
+ return bucket.getMergedBucket();
+ }
+}
+
+
+
+class FacetBucket {
+ FacetBucketMerger parent;
+ Comparable bucketValue;
+ long count;
+ Map<String, FacetMerger> subs;
+
+ public FacetBucket(FacetBucketMerger parent, Comparable bucketValue) {
+ this.parent = parent;
+ this.bucketValue = bucketValue;
+ }
+
+ public long getCount() {
+ return count;
+ }
+
+ /** returns the existing merger for the given key, or null if none yet exists */
+ FacetMerger getExistingMerger(String key) {
+ if (subs == null) return null;
+ return subs.get(key);
+ }
+
+ private FacetMerger getMerger(String key, Object prototype) {
+ FacetMerger merger = null;
+ if (subs != null) {
+ merger = subs.get(key);
+ if (merger != null) return merger;
+ }
+
+ merger = parent.createFacetMerger(key, prototype);
+
+ if (merger != null) {
+ if (subs == null) {
+ subs = new HashMap<>();
+ }
+ subs.put(key, merger);
+ }
+
+ return merger;
+ }
+
+ public void mergeBucket(SimpleOrderedMap bucket) {
+ // todo: for refinements, we want to recurse, but not re-do stats for intermediate buckets
+
+ // drive merging off the received bucket?
+ for (int i=0; i<bucket.size(); i++) {
+ String key = bucket.getName(i);
+ Object val = bucket.getVal(i);
+ if ("count".equals(key)) {
+ count += ((Number)val).longValue();
+ continue;
+ }
+ if ("val".equals(key)) {
+ // this is taken care of at a higher level...
+ continue;
+ }
+
+ FacetMerger merger = getMerger(key, val);
+
+ if (merger != null) {
+ merger.merge( val );
+ }
+ }
+ }
+
+
+ public SimpleOrderedMap getMergedBucket() {
+ SimpleOrderedMap out = new SimpleOrderedMap( (subs == null ? 0 : subs.size()) + 2 );
+ if (bucketValue != null) {
+ out.add("val", bucketValue);
+ }
+ out.add("count", count);
+ if (subs != null) {
+ for (Map.Entry<String,FacetMerger> mergerEntry : subs.entrySet()) {
+ FacetMerger subMerger = mergerEntry.getValue();
+ out.add(mergerEntry.getKey(), subMerger.getMergedResult());
+ }
+ }
+
+ return out;
+ }
+}
+
+
+
+class FacetFieldMerger extends FacetBucketMerger<FacetField> {
+ FacetBucket missingBucket;
+ FacetBucket allBuckets;
+ FacetMerger numBuckets;
+
+ LinkedHashMap<Object,FacetBucket> buckets = new LinkedHashMap<Object,FacetBucket>();
+ List<FacetBucket> sortedBuckets;
+ int numReturnedBuckets; // the number of buckets in the bucket lists returned from all of the shards
+
+ private static class SortVal implements Comparable<SortVal> {
+ FacetBucket bucket;
+ FacetSortableMerger merger;
+ FacetField.SortDirection direction;
+
+ @Override
+ public int compareTo(SortVal o) {
+ int c = -merger.compareTo(o.merger, direction) * direction.getMultiplier();
+ return c == 0 ? bucket.bucketValue.compareTo(o.bucket.bucketValue) : c;
+ }
+ }
+
+ public FacetFieldMerger(FacetField freq) {
+ super(freq);
+ }
+
+ @Override
+ public void merge(Object facetResult) {
+ merge((SimpleOrderedMap)facetResult);
+ }
+
+ public void merge(SimpleOrderedMap facetResult) {
+ if (freq.missing) {
+ Object o = facetResult.get("missing");
+ if (o != null) {
+ if (missingBucket == null) {
+ missingBucket = newBucket(null);
+ }
+ missingBucket.mergeBucket((SimpleOrderedMap)o);
+ }
+ }
+
+ if (freq.allBuckets) {
+ Object o = facetResult.get("allBuckets");
+ if (o != null) {
+ if (allBuckets == null) {
+ allBuckets = newBucket(null);
+ }
+ allBuckets.mergeBucket((SimpleOrderedMap)o);
+ }
+ }
+
+ List<SimpleOrderedMap> bucketList = (List<SimpleOrderedMap>) facetResult.get("buckets");
+ numReturnedBuckets += bucketList.size();
+ mergeBucketList(bucketList);
+
+ if (freq.numBuckets) {
+ Object nb = facetResult.get("numBuckets");
+ if (nb != null) {
+ if (numBuckets == null) {
+ numBuckets = new FacetNumBucketsMerger();
+ }
+ numBuckets.merge(nb);
+ }
+ }
+
+ }
+
+ public void mergeBucketList(List<SimpleOrderedMap> bucketList) {
+ for (SimpleOrderedMap bucketRes : bucketList) {
+ Comparable bucketVal = (Comparable)bucketRes.get("val");
+ FacetBucket bucket = buckets.get(bucketVal);
+ if (bucket == null) {
+ bucket = newBucket(bucketVal);
+ buckets.put(bucketVal, bucket);
+ }
+ bucket.mergeBucket( bucketRes );
+ }
+ }
+
+ public void sortBuckets() {
+ sortedBuckets = new ArrayList<>( buckets.values() );
+
+ Comparator<FacetBucket> comparator = null;
+
+ final FacetField.SortDirection direction = freq.sortDirection;
+ final int sortMul = direction.getMultiplier();
+
+ if ("count".equals(freq.sortVariable)) {
+ comparator = new Comparator<FacetBucket>() {
+ @Override
+ public int compare(FacetBucket o1, FacetBucket o2) {
+ int v = -Long.compare(o1.count, o2.count) * sortMul;
+ return v == 0 ? o1.bucketValue.compareTo(o2.bucketValue) : v;
+ }
+ };
+ Collections.sort(sortedBuckets, comparator);
+ } else if ("index".equals(freq.sortVariable)) {
+ comparator = new Comparator<FacetBucket>() {
+ @Override
+ public int compare(FacetBucket o1, FacetBucket o2) {
+ return -o1.bucketValue.compareTo(o2.bucketValue) * sortMul;
+ }
+ };
+ Collections.sort(sortedBuckets, comparator);
+ } else {
+ final String key = freq.sortVariable;
+
+ /**
+ final FacetSortableMerger[] arr = new FacetSortableMerger[buckets.size()];
+ final int[] index = new int[arr.length];
+ int start = 0;
+ int nullStart = index.length;
+ int i=0;
+ for (FacetBucket bucket : buckets.values()) {
+ FacetMerger merger = bucket.getExistingMerger(key);
+ if (merger == null) {
+ index[--nullStart] = i;
+ }
+ if (merger != null) {
+ arr[start] = (FacetSortableMerger)merger;
+ index[start] = i;
+ start++;
+ }
+ i++;
+ }
+
+ PrimUtils.sort(0, nullStart, index, new PrimUtils.IntComparator() {
+ @Override
+ public int compare(int a, int b) {
+ return arr[index[a]].compareTo(arr[index[b]], direction);
+ }
+ });
+ **/
+
+ // timsort may do better here given that the lists may be partially sorted.
+
+ List<SortVal> lst = new ArrayList<SortVal>(buckets.size());
+ List<FacetBucket> nulls = new ArrayList<FacetBucket>(buckets.size()>>1);
+ for (int i=0; i<sortedBuckets.size(); i++) {
+ FacetBucket bucket = sortedBuckets.get(i);
+ FacetMerger merger = bucket.getExistingMerger(key);
+ if (merger == null) {
+ nulls.add(bucket);
+ }
+ if (merger != null) {
+ SortVal sv = new SortVal();
+ sv.bucket = bucket;
+ sv.merger = (FacetSortableMerger)merger;
+ sv.direction = direction;
+ // sv.pos = i; // if we need position in the future...
+ lst.add(sv);
+ }
+ }
+ Collections.sort(lst);
+ Collections.sort(nulls, new Comparator<FacetBucket>() {
+ @Override
+ public int compare(FacetBucket o1, FacetBucket o2) {
+ return o1.bucketValue.compareTo(o2.bucketValue);
+ }
+ });
+
+ ArrayList<FacetBucket> out = new ArrayList<>(buckets.size());
+ for (SortVal sv : lst) {
+ out.add( sv.bucket );
+ }
+ out.addAll(nulls);
+ sortedBuckets = out;
+ }
+ }
+
+ @Override
+ public Object getMergedResult() {
+ SimpleOrderedMap result = new SimpleOrderedMap();
+
+ if (numBuckets != null) {
+ int removed = 0;
+ if (freq.mincount > 1) {
+ for (FacetBucket bucket : buckets.values()) {
+ if (bucket.count < freq.mincount) removed++;
+ }
+ }
+ result.add("numBuckets", ((Number)numBuckets.getMergedResult()).longValue() - removed);
+
+ // TODO: we can further increase this estimate.
+ // If not sorting by count, use a simple ratio to scale
+ // If sorting by count desc, then add up the highest_possible_missing_count from each shard
+ }
+
+ sortBuckets();
+
+ int first = (int)freq.offset;
+ int end = freq.limit >=0 ? first + (int) freq.limit : Integer.MAX_VALUE;
+ int last = Math.min(sortedBuckets.size(), end);
+
+ List<SimpleOrderedMap> resultBuckets = new ArrayList<>(Math.max(0, (last - first)));
+
+ /** this only works if there are no filters (like mincount)
+ for (int i=first; i<last; i++) {
+ FacetBucket bucket = sortedBuckets.get(i);
+ resultBuckets.add( bucket.getMergedBucket() );
+ }
+ ***/
+
+ // TODO: change effective offsets + limits at shards...
+
+ int off = (int)freq.offset;
+ int lim = freq.limit >= 0 ? (int)freq.limit : Integer.MAX_VALUE;
+ for (FacetBucket bucket : sortedBuckets) {
+ if (bucket.getCount() < freq.mincount) {
+ continue;
+ }
+
+ if (off > 0) {
+ --off;
+ continue;
+ }
+
+ if (resultBuckets.size() >= lim) {
+ break;
+ }
+
+ resultBuckets.add( bucket.getMergedBucket() );
+ }
+
+
+ result.add("buckets", resultBuckets);
+ if (missingBucket != null) {
+ result.add("missing", missingBucket.getMergedBucket());
+ }
+ if (allBuckets != null) {
+ result.add("allBuckets", allBuckets.getMergedBucket());
+ }
+
+ return result;
+ }
+
+
+ private class FacetNumBucketsMerger extends FacetMerger {
+ long sumBuckets;
+ long shardsMissingSum;
+ long shardsTruncatedSum;
+ Set<Object> values;
+
+ @Override
+ public void merge(Object facetResult) {
+ SimpleOrderedMap map = (SimpleOrderedMap)facetResult;
+ long numBuckets = ((Number)map.get("numBuckets")).longValue();
+ sumBuckets += numBuckets;
+
+ List vals = (List)map.get("vals");
+ if (vals != null) {
+ if (values == null) {
+ values = new HashSet<>(vals.size()*4);
+ }
+ values.addAll(vals);
+ if (numBuckets > values.size()) {
+ shardsTruncatedSum += numBuckets - values.size();
+ }
+ } else {
+ shardsMissingSum += numBuckets;
+ }
+ }
+
+ @Override
+ public Object getMergedResult() {
+ long exactCount = values == null ? 0 : values.size();
+ return exactCount + shardsMissingSum + shardsTruncatedSum;
+ // TODO: reduce count by (at least) number of buckets that fail to hit mincount (after merging)
+ // that should make things match for most of the small tests at least
+ }
+ }
+}
+
+
+class FacetRangeMerger extends FacetBucketMerger<FacetRange> {
+ FacetBucket beforeBucket;
+ FacetBucket afterBucket;
+ FacetBucket betweenBucket;
+
+ LinkedHashMap<Object, FacetBucket> buckets = new LinkedHashMap<Object, FacetBucket>();
+
+
+ public FacetRangeMerger(FacetRange freq) {
+ super(freq);
+ }
+
+ @Override
+ FacetBucket newBucket(Comparable bucketVal) {
+ return super.newBucket(bucketVal);
+ }
+
+ @Override
+ FacetMerger createFacetMerger(String key, Object val) {
+ return super.createFacetMerger(key, val);
+ }
+
+ @Override
+ public void merge(Object facetResult) {
+ merge((SimpleOrderedMap) facetResult);
+ }
+
+ public void merge(SimpleOrderedMap facetResult) {
+ boolean all = freq.others.contains(FacetParams.FacetRangeOther.ALL);
+
+ if (all || freq.others.contains(FacetParams.FacetRangeOther.BEFORE)) {
+ Object o = facetResult.get("before");
+ if (o != null) {
+ if (beforeBucket == null) {
+ beforeBucket = newBucket(null);
+ }
+ beforeBucket.mergeBucket((SimpleOrderedMap)o);
+ }
+ }
+
+ if (all || freq.others.contains(FacetParams.FacetRangeOther.AFTER)) {
+ Object o = facetResult.get("after");
+ if (o != null) {
+ if (afterBucket == null) {
+ afterBucket = newBucket(null);
+ }
+ afterBucket.mergeBucket((SimpleOrderedMap)o);
+ }
+ }
+
+ if (all || freq.others.contains(FacetParams.FacetRangeOther.BETWEEN)) {
+ Object o = facetResult.get("between");
+ if (o != null) {
+ if (betweenBucket == null) {
+ betweenBucket = newBucket(null);
+ }
+ betweenBucket.mergeBucket((SimpleOrderedMap)o);
+ }
+ }
+
+ List<SimpleOrderedMap> bucketList = (List<SimpleOrderedMap>) facetResult.get("buckets");
+ mergeBucketList(bucketList);
+ }
+
+ public void mergeBucketList(List<SimpleOrderedMap> bucketList) {
+ for (SimpleOrderedMap bucketRes : bucketList) {
+ Comparable bucketVal = (Comparable)bucketRes.get("val");
+ FacetBucket bucket = buckets.get(bucketVal);
+ if (bucket == null) {
+ bucket = newBucket(bucketVal);
+ buckets.put(bucketVal, bucket);
+ }
+ bucket.mergeBucket( bucketRes );
+ }
+ }
+
+ @Override
+ public Object getMergedResult() {
+ SimpleOrderedMap result = new SimpleOrderedMap(4);
+
+ List<SimpleOrderedMap> resultBuckets = new ArrayList<>(buckets.size());
+ // TODO: if we implement mincount for ranges, we'll need to sort buckets (see FacetFieldMerger)
+
+ for (FacetBucket bucket : buckets.values()) {
+ /***
+ if (bucket.getCount() < freq.mincount) {
+ continue;
+ }
+ ***/
+ resultBuckets.add( bucket.getMergedBucket() );
+ }
+
+ result.add("buckets", resultBuckets);
+
+ if (beforeBucket != null) {
+ result.add("before", beforeBucket.getMergedBucket());
+ }
+ if (afterBucket != null) {
+ result.add("after", afterBucket.getMergedBucket());
+ }
+ if (betweenBucket != null) {
+ result.add("between", betweenBucket.getMergedBucket());
+ }
+ return result;
+
+ }
+}
\ No newline at end of file
Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetQuery.java?rev=1666856&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetQuery.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetQuery.java Mon Mar 16 04:40:23 2015
@@ -0,0 +1,60 @@
+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 org.apache.lucene.search.Query;
+import org.apache.solr.common.util.SimpleOrderedMap;
+
+public class FacetQuery extends FacetRequest {
+ // query string or query?
+ Query q;
+
+ @Override
+ public FacetProcessor createFacetProcessor(FacetContext fcontext) {
+ return new FacetQueryProcessor(fcontext, this);
+ }
+
+ @Override
+ public FacetMerger createFacetMerger(Object prototype) {
+ return new FacetQueryMerger(this);
+ }
+}
+
+
+
+
+class FacetQueryProcessor extends FacetProcessor<FacetQuery> {
+ FacetQueryProcessor(FacetContext fcontext, FacetQuery freq) {
+ super(fcontext, freq);
+ }
+
+ @Override
+ public Object getResponse() {
+ return response;
+ }
+
+ @Override
+ public void process() throws IOException {
+ response = new SimpleOrderedMap<>();
+ fillBucket(response, freq.q);
+ }
+
+
+}
Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java?rev=1666856&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java Mon Mar 16 04:40:23 2015
@@ -0,0 +1,374 @@
+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.EnumSet;
+import java.util.List;
+
+import org.apache.lucene.search.Query;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.FacetParams;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.schema.TrieField;
+
+public class FacetRange extends FacetRequest {
+ String field;
+ Object start;
+ Object end;
+ Object gap;
+ boolean hardend = false;
+ EnumSet<FacetParams.FacetRangeInclude> include;
+ EnumSet<FacetParams.FacetRangeOther> others;
+
+ @Override
+ public FacetProcessor createFacetProcessor(FacetContext fcontext) {
+ return new FacetRangeProcessor(fcontext, this);
+ }
+
+ @Override
+ public FacetMerger createFacetMerger(Object prototype) {
+ return new FacetRangeMerger(this);
+ }
+}
+
+
+class FacetRangeProcessor extends FacetProcessor<FacetRange> {
+ SchemaField sf;
+
+
+ FacetRangeProcessor(FacetContext fcontext, FacetRange freq) {
+ super(fcontext, freq);
+ }
+
+ @Override
+ public void process() throws IOException {
+ sf = fcontext.searcher.getSchema().getField(freq.field);
+
+ response = getRangeCountsIndexed();
+ }
+
+ @Override
+ public Object getResponse() {
+ return response;
+ }
+
+
+ SimpleOrderedMap<Object> getRangeCountsIndexed() throws IOException {
+ final FieldType ft = sf.getType();
+
+ RangeEndpointCalculator<?> calc = null;
+
+ if (ft instanceof TrieField) {
+ final TrieField trie = (TrieField)ft;
+
+ switch (trie.getType()) {
+ case FLOAT:
+ calc = new FloatRangeEndpointCalculator(sf);
+ break;
+ case DOUBLE:
+ calc = new DoubleRangeEndpointCalculator(sf);
+ break;
+ case INTEGER:
+ calc = new IntegerRangeEndpointCalculator(sf);
+ break;
+ case LONG:
+ calc = new LongRangeEndpointCalculator(sf);
+ break;
+ default:
+ throw new SolrException
+ (SolrException.ErrorCode.BAD_REQUEST,
+ "Unable to range facet on tried field of unexpected type:" + freq.field);
+ }
+ } else {
+ throw new SolrException
+ (SolrException.ErrorCode.BAD_REQUEST,
+ "Unable to range facet on field:" + sf);
+ }
+
+ return getRangeCountsIndexed(calc);
+ }
+
+ private <T extends Comparable<T>> SimpleOrderedMap getRangeCountsIndexed(RangeEndpointCalculator<T> calc) throws IOException {
+
+ final SimpleOrderedMap<Object> res = new SimpleOrderedMap<>();
+
+ List<SimpleOrderedMap<Object>> buckets = null;
+
+ buckets = new ArrayList<>();
+ res.add("buckets", buckets);
+
+ T start = calc.getValue(freq.start.toString());
+ T end = calc.getValue(freq.end.toString());
+ EnumSet<FacetParams.FacetRangeInclude> include = freq.include;
+
+ String gap = freq.gap.toString();
+
+ final int minCount = 0;
+
+ T low = start;
+
+ while (low.compareTo(end) < 0) {
+ T high = calc.addGap(low, gap);
+ if (end.compareTo(high) < 0) {
+ if (freq.hardend) {
+ high = end;
+ } else {
+ end = high;
+ }
+ }
+ if (high.compareTo(low) < 0) {
+ throw new SolrException
+ (SolrException.ErrorCode.BAD_REQUEST,
+ "range facet infinite loop (is gap negative? did the math overflow?)");
+ }
+ if (high.compareTo(low) == 0) {
+ throw new SolrException
+ (SolrException.ErrorCode.BAD_REQUEST,
+ "range facet infinite loop: gap is either zero, or too small relative start/end and caused underflow: " + low + " + " + gap + " = " + high );
+ }
+
+ final boolean includeLower =
+ (include.contains(FacetParams.FacetRangeInclude.LOWER) ||
+ (include.contains(FacetParams.FacetRangeInclude.EDGE) &&
+ 0 == low.compareTo(start)));
+ final boolean includeUpper =
+ (include.contains(FacetParams.FacetRangeInclude.UPPER) ||
+ (include.contains(FacetParams.FacetRangeInclude.EDGE) &&
+ 0 == high.compareTo(end)));
+
+ final String lowS = calc.formatValue(low);
+ final String highS = calc.formatValue(high);
+
+ Object label = low;
+ buckets.add( rangeStats(low, minCount,lowS, highS, includeLower, includeUpper) );
+
+ low = high;
+ }
+
+ // no matter what other values are listed, we don't do
+ // anything if "none" is specified.
+ if (! freq.others.contains(FacetParams.FacetRangeOther.NONE) ) {
+
+ boolean all = freq.others.contains(FacetParams.FacetRangeOther.ALL);
+ final String startS = calc.formatValue(start);
+ final String endS = calc.formatValue(end);
+
+ if (all || freq.others.contains(FacetParams.FacetRangeOther.BEFORE)) {
+ // include upper bound if "outer" or if first gap doesn't already include it
+ res.add(FacetParams.FacetRangeOther.BEFORE.toString(),
+ rangeStats(null, 0, null, startS,
+ false,
+ (include.contains(FacetParams.FacetRangeInclude.OUTER) ||
+ (!(include.contains(FacetParams.FacetRangeInclude.LOWER) ||
+ include.contains(FacetParams.FacetRangeInclude.EDGE))))));
+
+ }
+ if (all || freq.others.contains(FacetParams.FacetRangeOther.AFTER)) {
+ // include lower bound if "outer" or if last gap doesn't already include it
+ res.add(FacetParams.FacetRangeOther.AFTER.toString(),
+ rangeStats(null, 0, endS, null,
+ (include.contains(FacetParams.FacetRangeInclude.OUTER) ||
+ (!(include.contains(FacetParams.FacetRangeInclude.UPPER) ||
+ include.contains(FacetParams.FacetRangeInclude.EDGE)))),
+ false));
+ }
+ if (all || freq.others.contains(FacetParams.FacetRangeOther.BETWEEN)) {
+ res.add(FacetParams.FacetRangeOther.BETWEEN.toString(),
+ rangeStats(null, 0, startS, endS,
+ (include.contains(FacetParams.FacetRangeInclude.LOWER) ||
+ include.contains(FacetParams.FacetRangeInclude.EDGE)),
+ (include.contains(FacetParams.FacetRangeInclude.UPPER) ||
+ include.contains(FacetParams.FacetRangeInclude.EDGE))));
+
+ }
+ }
+
+
+ return res;
+ }
+
+ private SimpleOrderedMap<Object> rangeStats(Object label, int mincount, String low, String high, boolean iLow, boolean iHigh) throws IOException {
+ SimpleOrderedMap<Object> bucket = new SimpleOrderedMap<>();
+
+ // typically the start value of the range, but null for before/after/between
+ if (label != null) {
+ bucket.add("val", label);
+ }
+
+ Query rangeQ = sf.getType().getRangeQuery(null, sf, low, high, iLow, iHigh);
+ fillBucket(bucket, rangeQ);
+
+ return bucket;
+ }
+
+
+
+
+ // Essentially copied from SimpleFacets...
+ // would be nice to unify this stuff w/ analytics component...
+ /**
+ * Perhaps someday instead of having a giant "instanceof" case
+ * statement to pick an impl, we can add a "RangeFacetable" marker
+ * interface to FieldTypes and they can return instances of these
+ * directly from some method -- but until then, keep this locked down
+ * and private.
+ */
+ private static abstract class RangeEndpointCalculator<T extends Comparable<T>> {
+ protected final SchemaField field;
+ public RangeEndpointCalculator(final SchemaField field) {
+ this.field = field;
+ }
+
+ /**
+ * Formats a Range endpoint for use as a range label name in the response.
+ * Default Impl just uses toString()
+ */
+ public String formatValue(final T val) {
+ return val.toString();
+ }
+ /**
+ * Parses a String param into an Range endpoint value throwing
+ * a useful exception if not possible
+ */
+ public final T getValue(final String rawval) {
+ try {
+ return parseVal(rawval);
+ } catch (Exception e) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+ "Can't parse value "+rawval+" for field: " +
+ field.getName(), e);
+ }
+ }
+ /**
+ * Parses a String param into an Range endpoint.
+ * Can throw a low level format exception as needed.
+ */
+ protected abstract T parseVal(final String rawval)
+ throws java.text.ParseException;
+
+ /**
+ * Parses a String param into a value that represents the gap and
+ * can be included in the response, throwing
+ * a useful exception if not possible.
+ *
+ * Note: uses Object as the return type instead of T for things like
+ * Date where gap is just a DateMathParser string
+ */
+ public final Object getGap(final String gap) {
+ try {
+ return parseGap(gap);
+ } catch (Exception e) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+ "Can't parse gap "+gap+" for field: " +
+ field.getName(), e);
+ }
+ }
+
+ /**
+ * Parses a String param into a value that represents the gap and
+ * can be included in the response.
+ * Can throw a low level format exception as needed.
+ *
+ * Default Impl calls parseVal
+ */
+ protected Object parseGap(final String rawval)
+ throws java.text.ParseException {
+ return parseVal(rawval);
+ }
+
+ /**
+ * Adds the String gap param to a low Range endpoint value to determine
+ * the corrisponding high Range endpoint value, throwing
+ * a useful exception if not possible.
+ */
+ public final T addGap(T value, String gap) {
+ try {
+ return parseAndAddGap(value, gap);
+ } catch (Exception e) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+ "Can't add gap "+gap+" to value " + value +
+ " for field: " + field.getName(), e);
+ }
+ }
+ /**
+ * Adds the String gap param to a low Range endpoint value to determine
+ * the corrisponding high Range endpoint value.
+ * Can throw a low level format exception as needed.
+ */
+ protected abstract T parseAndAddGap(T value, String gap)
+ throws java.text.ParseException;
+
+ }
+
+ private static class FloatRangeEndpointCalculator
+ extends RangeEndpointCalculator<Float> {
+
+ public FloatRangeEndpointCalculator(final SchemaField f) { super(f); }
+ @Override
+ protected Float parseVal(String rawval) {
+ return Float.valueOf(rawval);
+ }
+ @Override
+ public Float parseAndAddGap(Float value, String gap) {
+ return new Float(value.floatValue() + Float.valueOf(gap).floatValue());
+ }
+ }
+ private static class DoubleRangeEndpointCalculator
+ extends RangeEndpointCalculator<Double> {
+
+ public DoubleRangeEndpointCalculator(final SchemaField f) { super(f); }
+ @Override
+ protected Double parseVal(String rawval) {
+ return Double.valueOf(rawval);
+ }
+ @Override
+ public Double parseAndAddGap(Double value, String gap) {
+ return new Double(value.doubleValue() + Double.valueOf(gap).doubleValue());
+ }
+ }
+ private static class IntegerRangeEndpointCalculator
+ extends RangeEndpointCalculator<Integer> {
+
+ public IntegerRangeEndpointCalculator(final SchemaField f) { super(f); }
+ @Override
+ protected Integer parseVal(String rawval) {
+ return Integer.valueOf(rawval);
+ }
+ @Override
+ public Integer parseAndAddGap(Integer value, String gap) {
+ return new Integer(value.intValue() + Integer.valueOf(gap).intValue());
+ }
+ }
+ private static class LongRangeEndpointCalculator
+ extends RangeEndpointCalculator<Long> {
+
+ public LongRangeEndpointCalculator(final SchemaField f) { super(f); }
+ @Override
+ protected Long parseVal(String rawval) {
+ return Long.valueOf(rawval);
+ }
+ @Override
+ public Long parseAndAddGap(Long value, String gap) {
+ return new Long(value.longValue() + Long.valueOf(gap).longValue());
+ }
+ }
+}
Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java?rev=1666856&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java Mon Mar 16 04:40:23 2015
@@ -0,0 +1,720 @@
+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.EnumSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.Query;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.FacetParams;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.DocIterator;
+import org.apache.solr.search.DocSet;
+import org.apache.solr.search.FunctionQParser;
+import org.apache.solr.search.FunctionQParserPlugin;
+import org.apache.solr.search.QParser;
+import org.apache.solr.search.QueryContext;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.search.SyntaxError;
+
+
+public abstract class FacetRequest {
+ protected Map<String,AggValueSource> facetStats; // per-bucket statistics
+ protected Map<String,FacetRequest> subFacets; // list of facets
+ protected List<String> excludeFilters;
+ protected boolean processEmpty;
+
+ public FacetRequest() {
+ facetStats = new LinkedHashMap<>();
+ subFacets = new LinkedHashMap<>();
+ }
+
+ public Map<String, AggValueSource> getFacetStats() {
+ return facetStats;
+ }
+
+ public Map<String, FacetRequest> getSubFacets() {
+ return subFacets;
+ }
+
+ public void addStat(String key, AggValueSource stat) {
+ facetStats.put(key, stat);
+ }
+
+ public void addSubFacet(String key, FacetRequest facetRequest) {
+ subFacets.put(key, facetRequest);
+ }
+
+ public abstract FacetProcessor createFacetProcessor(FacetContext fcontext);
+
+ public abstract FacetMerger createFacetMerger(Object prototype);
+}
+
+
+class FacetContext {
+ // Context info for actually executing a local facet command
+ public static final int IS_SHARD=0x01;
+
+ QueryContext qcontext;
+ SolrQueryRequest req; // TODO: replace with params?
+ SolrIndexSearcher searcher;
+ DocSet base;
+ FacetContext parent;
+ int flags;
+
+ public boolean isShard() {
+ return (flags & IS_SHARD) != 0;
+ }
+
+ public FacetContext sub() {
+ FacetContext ctx = new FacetContext();
+ ctx.flags = flags;
+ ctx.qcontext = qcontext;
+ ctx.req = req;
+ ctx.searcher = searcher;
+ ctx.base = base;
+
+ ctx.parent = this;
+ return ctx;
+ }
+}
+
+
+class FacetProcessor<FacetRequestT extends FacetRequest> {
+ protected SimpleOrderedMap<Object> response;
+ protected FacetContext fcontext;
+ protected 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 {
+
+
+ }
+
+ public Object getResponse() {
+ return null;
+ }
+
+
+ protected void createAccs(int docCount, int slotCount) throws IOException {
+ accMap = new LinkedHashMap<String,SlotAcc>();
+ countAcc = new CountSlotAcc(fcontext, slotCount);
+ countAcc.key = "count";
+ for (Map.Entry<String,AggValueSource> entry : freq.getFacetStats().entrySet()) {
+ SlotAcc acc = entry.getValue().createSlotAcc(fcontext, docCount, slotCount);
+ acc.key = entry.getKey();
+ accMap.put(acc.key, acc);
+ }
+ }
+
+ /** Create the actual accs array from accMap before starting to collect stats. */
+ protected void prepareForCollection() {
+ accs = new SlotAcc[accMap.size()];
+ int i=0;
+ for (SlotAcc acc : accMap.values()) {
+ accs[i++] = acc;
+ }
+ }
+
+ protected 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);
+ prepareForCollection();
+ int collected = collect(docs, 0);
+ countAcc.incrementCount(0, collected);
+ assert collected == docCount;
+ addStats(bucket, 0);
+ }
+
+
+ protected void fillBucketSubs(SimpleOrderedMap<Object> response, FacetContext subContext) throws IOException {
+ for (Map.Entry<String,FacetRequest> sub : freq.getSubFacets().entrySet()) {
+ FacetProcessor subProcessor = sub.getValue().createFacetProcessor(subContext);
+ subProcessor.process();
+ response.add( sub.getKey(), subProcessor.getResponse() );
+ }
+ }
+
+ int collect(DocSet docs, int slot) throws IOException {
+ int count = 0;
+ SolrIndexSearcher searcher = fcontext.searcher;
+
+ final List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
+ final Iterator<LeafReaderContext> ctxIt = leaves.iterator();
+ LeafReaderContext ctx = null;
+ int segBase = 0;
+ int segMax;
+ int adjustedMax = 0;
+ for (DocIterator docsIt = docs.iterator(); docsIt.hasNext(); ) {
+ final int doc = docsIt.nextDoc();
+ if (doc >= adjustedMax) {
+ do {
+ ctx = ctxIt.next();
+ if (ctx == null) {
+ // should be impossible
+ throw new RuntimeException("INTERNAL FACET ERROR");
+ }
+ segBase = ctx.docBase;
+ segMax = ctx.reader().maxDoc();
+ adjustedMax = segBase + segMax;
+ } while (doc >= adjustedMax);
+ assert doc >= ctx.docBase;
+ setNextReader(ctx);
+ }
+ count++;
+ collect(doc - segBase, slot); // per-seg collectors
+ }
+ return count;
+ }
+
+ void collect(int segDoc, int slot) throws IOException {
+ for (SlotAcc acc : accs) {
+ acc.collect(segDoc, slot);
+ }
+ }
+
+ void setNextReader(LeafReaderContext ctx) throws IOException {
+ // countAcc.setNextReader is a no-op
+ for (SlotAcc acc : accs) {
+ acc.setNextReader(ctx);
+ }
+ }
+
+ void addStats(SimpleOrderedMap<Object> target, int slotNum) throws IOException {
+ int count = countAcc.getCount(slotNum);
+ target.add("count", count);
+ if (count > 0 || freq.processEmpty) {
+ for (SlotAcc acc : accs) {
+ acc.setValues(target, slotNum);
+ }
+ }
+ }
+
+
+
+
+
+ public void fillBucket(SimpleOrderedMap<Object> bucket, Query q) throws IOException {
+ boolean needDocSet = freq.getFacetStats().size() > 0 || freq.getSubFacets().size() > 0;
+
+ // TODO: always collect counts or not???
+
+ DocSet result = null;
+ int count;
+
+ if (needDocSet) {
+ if (q == null) {
+ result = fcontext.base;
+ // result.incref(); // OFF-HEAP
+ } else {
+ result = fcontext.searcher.getDocSet(q, fcontext.base);
+ }
+ count = result.size();
+ } else {
+ if (q == null) {
+ count = fcontext.base.size();
+ } else {
+ count = fcontext.searcher.numDocs(q, fcontext.base);
+ }
+ }
+
+ try {
+ processStats(bucket, result, (int) count);
+ processSubs(bucket, result);
+ } finally {
+ if (result != null) {
+ // result.decref(); // OFF-HEAP
+ result = null;
+ }
+ }
+ }
+
+
+
+
+ protected void processSubs(SimpleOrderedMap<Object> bucket, DocSet result) throws IOException {
+ // TODO: process exclusions, etc
+
+ if (result == null || result.size() == 0 && !freq.processEmpty) {
+ return;
+ }
+
+ FacetContext subContext = fcontext.sub();
+ subContext.base = result;
+
+ fillBucketSubs(bucket, subContext);
+ }
+
+
+ public 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);
+ // hasVal.decref(); // OFF-HEAP
+ return answer;
+ }
+
+}
+
+
+
+
+
+abstract class FacetParser<FacetRequestT extends FacetRequest> {
+ protected FacetRequestT facet;
+ protected FacetParser parent;
+ protected String key;
+
+ public FacetParser(FacetParser parent,String key) {
+ this.parent = parent;
+ this.key = key;
+ }
+
+ public String getKey() {
+ return key;
+ }
+
+ public String getPathStr() {
+ if (parent == null) {
+ return "/" + key;
+ }
+ return parent.getKey() + "/" + key;
+ }
+
+ protected RuntimeException err(String msg) {
+ return new SolrException(SolrException.ErrorCode.BAD_REQUEST, msg + " ,path="+getPathStr());
+ }
+
+ public abstract FacetRequest parse(Object o) throws SyntaxError;
+
+ // TODO: put the FacetRequest on the parser object?
+ public void parseSubs(Object o) throws SyntaxError {
+ if (o==null) return;
+ if (o instanceof Map) {
+ Map<String,Object> m = (Map<String, Object>) o;
+ for (Map.Entry<String,Object> entry : m.entrySet()) {
+ String key = entry.getKey();
+ Object value = entry.getValue();
+
+ if ("processEmpty".equals(key)) {
+ facet.processEmpty = getBoolean(m, "processEmpty", false);
+ continue;
+ }
+
+ // "my_prices" : { "range" : { "field":...
+ // key="my_prices", value={"range":..
+
+ Object parsedValue = parseFacetOrStat(key, value);
+
+ // TODO: have parseFacetOrStat directly add instead of return?
+ if (parsedValue instanceof FacetRequest) {
+ facet.addSubFacet(key, (FacetRequest)parsedValue);
+ } else if (parsedValue instanceof AggValueSource) {
+ facet.addStat(key, (AggValueSource)parsedValue);
+ } else {
+ throw new RuntimeException("Huh? TODO: " + parsedValue);
+ }
+ }
+ } else {
+ // facet : my_field?
+ throw err("Expected map for facet/stat");
+ }
+ }
+
+ public Object parseFacetOrStat(String key, Object o) throws SyntaxError {
+ if (o instanceof String) {
+ return parseStringFacetOrStat(key, (String)o);
+ }
+
+ if (!(o instanceof Map)) {
+ throw err("expected Map but got " + o);
+ }
+
+ // { "range" : { "field":...
+ Map<String,Object> m = (Map<String,Object>)o;
+ if (m.size() != 1) {
+ throw err("expected facet/stat type name, like {range:{... but got " + m);
+ }
+
+ // Is this most efficient way?
+ Map.Entry<String,Object> entry = m.entrySet().iterator().next();
+ String type = entry.getKey();
+ Object args = entry.getValue();
+ return parseFacetOrStat(key, type, args);
+ }
+
+ public Object parseFacetOrStat(String key, String type, Object args) throws SyntaxError {
+ // TODO: a place to register all these facet types?
+
+ if ("field".equals(type) || "terms".equals(type)) {
+ return parseFieldFacet(key, args);
+ } else if ("query".equals(type)) {
+ return parseQueryFacet(key, args);
+ } else if ("range".equals(type)) {
+ return parseRangeFacet(key, args);
+ }
+
+ return parseStat(key, type, args);
+ }
+
+
+
+ FacetField parseFieldFacet(String key, Object args) throws SyntaxError {
+ FacetFieldParser parser = new FacetFieldParser(this, key);
+ return parser.parse(args);
+ }
+
+ FacetQuery parseQueryFacet(String key, Object args) throws SyntaxError {
+ FacetQueryParser parser = new FacetQueryParser(this, key);
+ return parser.parse(args);
+ }
+
+ FacetRange parseRangeFacet(String key, Object args) throws SyntaxError {
+ FacetRangeParser parser = new FacetRangeParser(this, key);
+ return parser.parse(args);
+ }
+
+ public Object parseStringFacetOrStat(String key, String s) throws SyntaxError {
+ // "avg(myfield)"
+ return parseStringStat(key, s);
+ // TODO - simple string representation of facets
+ }
+
+ // parses avg(x)
+ private AggValueSource parseStringStat(String key, String stat) throws SyntaxError {
+ FunctionQParser parser = (FunctionQParser)QParser.getParser(stat, FunctionQParserPlugin.NAME, getSolrRequest());
+ AggValueSource agg = parser.parseAgg(FunctionQParser.FLAG_DEFAULT);
+ return agg;
+ }
+
+ public AggValueSource parseStat(String key, String type, Object args) throws SyntaxError {
+ return null;
+ }
+
+
+ public String getField(Map<String,Object> args) {
+ Object fieldName = args.get("field"); // TODO: pull out into defined constant
+ if (fieldName == null) {
+ fieldName = args.get("f"); // short form
+ }
+ if (fieldName == null) {
+ throw err("Missing 'field'");
+ }
+
+ if (!(fieldName instanceof String)) {
+ throw err("Expected string for 'field', got" + fieldName);
+ }
+
+ return (String)fieldName;
+ }
+
+
+ public Long getLongOrNull(Map<String,Object> args, String paramName, boolean required) {
+ Object o = args.get(paramName);
+ if (o == null) {
+ if (required) {
+ throw err("Missing required parameter '" + paramName + "'");
+ }
+ return null;
+ }
+ if (!(o instanceof Long || o instanceof Integer || o instanceof Short || o instanceof Byte)) {
+ throw err("Expected integer type for param '"+paramName + "' but got " + o);
+ }
+
+ return ((Number)o).longValue();
+ }
+
+ public long getLong(Map<String,Object> args, String paramName, long defVal) {
+ Object o = args.get(paramName);
+ if (o == null) {
+ return defVal;
+ }
+ if (!(o instanceof Long || o instanceof Integer || o instanceof Short || o instanceof Byte)) {
+ throw err("Expected integer type for param '"+paramName + "' but got " + o.getClass().getSimpleName() + " = " + o);
+ }
+
+ return ((Number)o).longValue();
+ }
+
+ public boolean getBoolean(Map<String,Object> args, String paramName, boolean defVal) {
+ Object o = args.get(paramName);
+ if (o == null) {
+ return defVal;
+ }
+ // TODO: should we be more flexible and accept things like "true" (strings)?
+ // Perhaps wait until the use case comes up.
+ if (!(o instanceof Boolean)) {
+ throw err("Expected boolean type for param '"+paramName + "' but got " + o.getClass().getSimpleName() + " = " + o);
+ }
+
+ return (Boolean)o;
+ }
+
+ public String getString(Map<String,Object> args, String paramName, String defVal) {
+ Object o = args.get(paramName);
+ if (o == null) {
+ return defVal;
+ }
+ if (!(o instanceof String)) {
+ throw err("Expected string type for param '"+paramName + "' but got " + o.getClass().getSimpleName() + " = " + o);
+ }
+
+ return (String)o;
+ }
+
+
+ public IndexSchema getSchema() {
+ return parent.getSchema();
+ }
+
+ public SolrQueryRequest getSolrRequest() {
+ return parent.getSolrRequest();
+ }
+
+}
+
+
+class FacetTopParser extends FacetParser<FacetQuery> {
+ private SolrQueryRequest req;
+
+ public FacetTopParser(SolrQueryRequest req) {
+ super(null, "facet");
+ this.facet = new FacetQuery();
+ this.req = req;
+ }
+
+ @Override
+ public FacetQuery parse(Object args) throws SyntaxError {
+ parseSubs(args);
+ return facet;
+ }
+
+ @Override
+ public SolrQueryRequest getSolrRequest() {
+ return req;
+ }
+
+ @Override
+ public IndexSchema getSchema() {
+ return req.getSchema();
+ }
+}
+
+class FacetQueryParser extends FacetParser<FacetQuery> {
+ public FacetQueryParser(FacetParser parent, String key) {
+ super(parent, key);
+ facet = new FacetQuery();
+ }
+
+ @Override
+ public FacetQuery parse(Object arg) throws SyntaxError {
+ String qstring = null;
+ if (arg instanceof String) {
+ // just the field name...
+ qstring = (String)arg;
+
+ } else if (arg instanceof Map) {
+ Map<String, Object> m = (Map<String, Object>) arg;
+ qstring = getString(m, "q", null);
+ if (qstring == null) {
+ qstring = getString(m, "query", null);
+ }
+
+ // OK to parse subs before we have parsed our own query?
+ // as long as subs don't need to know about it.
+ parseSubs( m.get("facet") );
+ }
+
+ // TODO: substats that are from defaults!!!
+
+ if (qstring != null) {
+ QParser parser = QParser.getParser(qstring, null, getSolrRequest());
+ facet.q = parser.getQuery();
+ }
+
+ return facet;
+ }
+}
+
+class FacetFieldParser extends FacetParser<FacetField> {
+ public FacetFieldParser(FacetParser parent, String key) {
+ super(parent, key);
+ facet = new FacetField();
+ }
+
+ public FacetField parse(Object arg) throws SyntaxError {
+
+ if (arg instanceof String) {
+ // just the field name...
+ facet.field = (String)arg;
+ parseSort( null ); // TODO: defaults
+
+ } else if (arg instanceof Map) {
+ Map<String, Object> m = (Map<String, Object>) arg;
+ facet.field = getField(m);
+ facet.offset = getLong(m, "offset", facet.offset);
+ facet.limit = getLong(m, "limit", facet.limit);
+ facet.mincount = getLong(m, "mincount", facet.mincount);
+ facet.missing = getBoolean(m, "missing", facet.missing);
+ facet.numBuckets = getBoolean(m, "numBuckets", facet.numBuckets);
+ facet.prefix = getString(m, "prefix", facet.prefix);
+ facet.allBuckets = getBoolean(m, "allBuckets", facet.allBuckets);
+ facet.method = FacetField.FacetMethod.fromString(getString(m, "method", null));
+ facet.cacheDf = (int)getLong(m, "cacheDf", facet.cacheDf);
+
+ // facet.sort may depend on a facet stat...
+ // should we be parsing / validating this here, or in the execution environment?
+ Object o = m.get("facet");
+ parseSubs(o);
+
+ parseSort( m.get("sort") );
+ }
+
+ return facet;
+ }
+
+
+ // Sort specification is currently
+ // sort : 'mystat desc'
+ // OR
+ // sort : { mystat : 'desc' }
+ private void parseSort(Object sort) {
+ if (sort == null) {
+ facet.sortVariable = "count";
+ facet.sortDirection = FacetField.SortDirection.desc;
+ } else if (sort instanceof String) {
+ String sortStr = (String)sort;
+ if (sortStr.endsWith(" asc")) {
+ facet.sortVariable = sortStr.substring(0, sortStr.length()-" asc".length());
+ facet.sortDirection = FacetField.SortDirection.asc;
+ } else if (sortStr.endsWith(" desc")) {
+ facet.sortVariable = sortStr.substring(0, sortStr.length()-" desc".length());
+ facet.sortDirection = FacetField.SortDirection.desc;
+ } else {
+ facet.sortDirection = "index".equals(facet.sortVariable) ? FacetField.SortDirection.asc : FacetField.SortDirection.desc; // default direction for "index" is ascending
+ }
+ } else {
+ // sort : { myvar : 'desc' }
+ Map<String,Object> map = (Map<String,Object>)sort;
+ // TODO: validate
+ Map.Entry<String,Object> entry = map.entrySet().iterator().next();
+ String k = entry.getKey();
+ Object v = entry.getValue();
+ facet.sortVariable = k;
+ facet.sortDirection = FacetField.SortDirection.valueOf(v.toString());
+ }
+
+ }
+}
+
+
+
+class FacetRangeParser extends FacetParser<FacetRange> {
+ public FacetRangeParser(FacetParser parent, String key) {
+ super(parent, key);
+ facet = new FacetRange();
+ }
+
+ public FacetRange parse(Object arg) throws SyntaxError {
+ if (!(arg instanceof Map)) {
+ throw err("Missing range facet arguments");
+ }
+
+ Map<String, Object> m = (Map<String, Object>) arg;
+
+ facet.field = getString(m, "field", null);
+
+ facet.start = m.get("start");
+ facet.end = m.get("end");
+ facet.gap = m.get("gap");
+ facet.hardend = getBoolean(m, "hardend", facet.hardend);
+
+ // TODO: refactor list-of-options code
+
+ Object o = m.get("include");
+ String[] includeList = null;
+ if (o != null) {
+ List lst = null;
+
+ if (o instanceof List) {
+ lst = (List)o;
+ } else if (o instanceof String) {
+ lst = StrUtils.splitSmart((String)o, ',');
+ }
+
+ includeList = (String[])lst.toArray(new String[lst.size()]);
+ }
+ facet.include = FacetParams.FacetRangeInclude.parseParam( includeList );
+
+ facet.others = EnumSet.noneOf(FacetParams.FacetRangeOther.class);
+
+ o = m.get("other");
+ if (o != null) {
+ List<String> lst = null;
+
+ if (o instanceof List) {
+ lst = (List)o;
+ } else if (o instanceof String) {
+ lst = StrUtils.splitSmart((String)o, ',');
+ }
+
+ for (String otherStr : lst) {
+ facet.others.add( FacetParams.FacetRangeOther.get(otherStr) );
+ }
+ }
+
+
+ Object facetObj = m.get("facet");
+ parseSubs(facetObj);
+
+ return facet;
+ }
+
+}
+
+
+