SOLR-7214: JSON Facet API

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1666856 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Yonik Seeley 2015-03-16 04:40:23 +00:00
parent bb95b48546
commit 3dc5ed33c5
52 changed files with 8205 additions and 339 deletions

View File

@ -166,6 +166,15 @@ New Features
* SOLR-7218: lucene/solr query syntax to give any query clause a constant score. * SOLR-7218: lucene/solr query syntax to give any query clause a constant score.
General Form: <clause>^=<constant_score> General Form: <clause>^=<constant_score>
Example: (color:blue color:green)^=2.0 text:shoes Example: (color:blue color:green)^=2.0 text:shoes
(yonik)
* SOLR-7214: New Facet module with a JSON API, facet functions, aggregations, and analytics.
Any facet type can have sub facets, and facets can be sorted by arbitrary aggregation functions.
Examples:
json.facet={x:'avg(price)', y:'unique(color)'}
json.facet={count1:{query:"price:[10 TO 20]"}, count2:{query:"color:blue AND popularity:[0 TO 50]"} }
json.facet={categories:{terms:{field:cat, sort:"x desc", facet:{x:"avg(price)", y:"sum(price)"}}}}
(yonik)
Bug Fixes Bug Fixes

View File

@ -21,6 +21,7 @@ import static org.apache.solr.handler.dataimport.DataImporter.IMPORT_CMD;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.MapSolrParams;
import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.params.UpdateParams; import org.apache.solr.common.params.UpdateParams;
@ -84,11 +85,14 @@ public class DataImportHandler extends RequestHandlerBase implements
public DataImporter getImporter() { public DataImporter getImporter() {
return this.importer; return this.importer;
} }
@Override @Override
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public void init(NamedList args) { public void init(NamedList args) {
super.init(args); super.init(args);
Map<String,String> macro = new HashMap<>();
macro.put("expandMacros", "false");
defaults = SolrParams.wrapDefaults(defaults, new MapSolrParams(macro));
} }
@Override @Override

View File

@ -25,6 +25,7 @@ import org.apache.solr.core.PluginInfo;
import org.apache.solr.core.PluginBag; import org.apache.solr.core.PluginBag;
import org.apache.solr.core.SolrCore; import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrInfoMBean; import org.apache.solr.core.SolrInfoMBean;
import org.apache.solr.handler.component.SearchHandler;
import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrRequestHandler; import org.apache.solr.request.SolrRequestHandler;
import org.apache.solr.response.SolrQueryResponse; import org.apache.solr.response.SolrQueryResponse;
@ -136,7 +137,7 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
TimerContext timer = requestTimes.time(); TimerContext timer = requestTimes.time();
try { try {
if(pluginInfo != null && pluginInfo.attributes.containsKey(USEPARAM)) req.getContext().put(USEPARAM,pluginInfo.attributes.get(USEPARAM)); if(pluginInfo != null && pluginInfo.attributes.containsKey(USEPARAM)) req.getContext().put(USEPARAM,pluginInfo.attributes.get(USEPARAM));
SolrPluginUtils.setDefaults(req,defaults,appends,invariants); SolrPluginUtils.setDefaults(this, req, defaults, appends, invariants);
req.getContext().remove(USEPARAM); req.getContext().remove(USEPARAM);
rsp.setHttpCaching(httpCaching); rsp.setHttpCaching(httpCaching);
handleRequestBody( req, rsp ); handleRequestBody( req, rsp );

View File

@ -25,6 +25,7 @@ import java.util.Map;
import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.NamedList;
import org.apache.solr.core.SolrInfoMBean; import org.apache.solr.core.SolrInfoMBean;
import org.apache.solr.search.facet.FacetModule;
import org.apache.solr.util.plugin.NamedListInitializedPlugin; import org.apache.solr.util.plugin.NamedListInitializedPlugin;
/** /**
@ -134,6 +135,7 @@ public abstract class SearchComponent implements SolrInfoMBean, NamedListInitial
map.put(HighlightComponent.COMPONENT_NAME, HighlightComponent.class); map.put(HighlightComponent.COMPONENT_NAME, HighlightComponent.class);
map.put(QueryComponent.COMPONENT_NAME, QueryComponent.class); map.put(QueryComponent.COMPONENT_NAME, QueryComponent.class);
map.put(FacetComponent.COMPONENT_NAME, FacetComponent.class); map.put(FacetComponent.COMPONENT_NAME, FacetComponent.class);
map.put(FacetModule.COMPONENT_NAME, FacetModule.class);
map.put(MoreLikeThisComponent.COMPONENT_NAME, MoreLikeThisComponent.class); map.put(MoreLikeThisComponent.COMPONENT_NAME, MoreLikeThisComponent.class);
map.put(StatsComponent.COMPONENT_NAME, StatsComponent.class); map.put(StatsComponent.COMPONENT_NAME, StatsComponent.class);
map.put(DebugComponent.COMPONENT_NAME, DebugComponent.class); map.put(DebugComponent.COMPONENT_NAME, DebugComponent.class);

View File

@ -35,6 +35,7 @@ import org.apache.solr.handler.RequestHandlerBase;
import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse; import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.search.SolrQueryTimeoutImpl; import org.apache.solr.search.SolrQueryTimeoutImpl;
import org.apache.solr.search.facet.FacetModule;
import org.apache.solr.util.RTimer; import org.apache.solr.util.RTimer;
import org.apache.solr.util.SolrPluginUtils; import org.apache.solr.util.SolrPluginUtils;
import org.apache.solr.util.plugin.PluginInfoInitialized; import org.apache.solr.util.plugin.PluginInfoInitialized;
@ -75,6 +76,7 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware ,
ArrayList<String> names = new ArrayList<>(6); ArrayList<String> names = new ArrayList<>(6);
names.add( QueryComponent.COMPONENT_NAME ); names.add( QueryComponent.COMPONENT_NAME );
names.add( FacetComponent.COMPONENT_NAME ); names.add( FacetComponent.COMPONENT_NAME );
names.add( FacetModule.COMPONENT_NAME );
names.add( MoreLikeThisComponent.COMPONENT_NAME ); names.add( MoreLikeThisComponent.COMPONENT_NAME );
names.add( HighlightComponent.COMPONENT_NAME ); names.add( HighlightComponent.COMPONENT_NAME );
names.add( StatsComponent.COMPONENT_NAME ); names.add( StatsComponent.COMPONENT_NAME );

View File

@ -97,6 +97,13 @@ public interface SolrQueryRequest {
* @param params The new set of parameter * @param params The new set of parameter
*/ */
public void forward(String handler, SolrParams params, SolrQueryResponse rsp); public void forward(String handler, SolrParams params, SolrQueryResponse rsp);
/** Returns any associated JSON (or null if none) in deserialized generic form.
* Java classes used to represent the JSON are as follows: Map, List, String, Long, Double, Boolean
*/
public Map<String,Object> getJSON();
public void setJSON(Map<String,Object> json);
} }

View File

@ -50,6 +50,7 @@ public abstract class SolrQueryRequestBase implements SolrQueryRequest, Closeabl
protected SolrParams params; protected SolrParams params;
protected Map<Object,Object> context; protected Map<Object,Object> context;
protected Iterable<ContentStream> streams; protected Iterable<ContentStream> streams;
protected Map<String,Object> json;
private final RTimer requestTimer; private final RTimer requestTimer;
@ -172,4 +173,14 @@ public abstract class SolrQueryRequestBase implements SolrQueryRequest, Closeabl
} }
@Override
public Map<String, Object> getJSON() {
return json;
}
@Override
public void setJSON(Map<String, Object> json) {
this.json = json;
}
} }

View File

@ -1,281 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.request;
import java.io.IOException;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.uninverting.DocTermOrds;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRefBuilder;
import org.apache.solr.common.SolrException;
import org.apache.solr.core.SolrCore;
import org.apache.solr.schema.FieldType;
import org.apache.solr.schema.TrieField;
import org.apache.solr.search.DocSet;
import org.apache.solr.search.SolrCache;
import org.apache.solr.search.SolrIndexSearcher;
/**
*
* Final form of the un-inverted field:
* Each document points to a list of term numbers that are contained in that document.
*
* Term numbers are in sorted order, and are encoded as variable-length deltas from the
* previous term number. Real term numbers start at 2 since 0 and 1 are reserved. A
* term number of 0 signals the end of the termNumber list.
*
* There is a single int[maxDoc()] which either contains a pointer into a byte[] for
* the termNumber lists, or directly contains the termNumber list if it fits in the 4
* bytes of an integer. If the first byte in the integer is 1, the next 3 bytes
* are a pointer into a byte[] where the termNumber list starts.
*
* There are actually 256 byte arrays, to compensate for the fact that the pointers
* into the byte arrays are only 3 bytes long. The correct byte array for a document
* is a function of its id.
*
* To save space and speed up faceting, any term that matches enough documents will
* not be un-inverted... it will be skipped while building the un-inverted field structure,
* and will use a set intersection method during faceting.
*
* To further save memory, the terms (the actual string values) are not all stored in
* memory, but a TermIndex is used to convert term numbers to term values only
* for the terms needed after faceting has completed. Only every 128th term value
* is stored, along with its corresponding term number, and this is used as an
* index to find the closest term and iterate until the desired number is hit (very
* much like Lucene's own internal term index).
*
*/
public class UnInvertedField extends DocTermOrds {
static class TopTerm {
BytesRef term;
int termNum;
long memSize() {
return 8 + // obj header
8 + 8 +term.length + //term
4; // int
}
}
long memsz;
final AtomicLong use = new AtomicLong(); // number of uses
int[] maxTermCounts = new int[1024];
final Map<Integer,TopTerm> bigTerms = new LinkedHashMap<>();
private SolrIndexSearcher.DocsEnumState deState;
private final SolrIndexSearcher searcher;
private static UnInvertedField uifPlaceholder = new UnInvertedField();
private UnInvertedField() { // Dummy for synchronization.
super("fake", 0, 0); // cheapest initialization I can find.
searcher = null;
}
@Override
protected void visitTerm(TermsEnum te, int termNum) throws IOException {
if (termNum >= maxTermCounts.length) {
// resize by doubling - for very large number of unique terms, expanding
// by 4K and resultant GC will dominate uninvert times. Resize at end if material
int[] newMaxTermCounts = new int[maxTermCounts.length*2];
System.arraycopy(maxTermCounts, 0, newMaxTermCounts, 0, termNum);
maxTermCounts = newMaxTermCounts;
}
final BytesRef term = te.term();
if (te.docFreq() > maxTermDocFreq) {
TopTerm topTerm = new TopTerm();
topTerm.term = BytesRef.deepCopyOf(term);
topTerm.termNum = termNum;
bigTerms.put(topTerm.termNum, topTerm);
if (deState == null) {
deState = new SolrIndexSearcher.DocsEnumState();
deState.fieldName = field;
deState.liveDocs = searcher.getLeafReader().getLiveDocs();
deState.termsEnum = te; // TODO: check for MultiTermsEnum in SolrIndexSearcher could now fail?
deState.postingsEnum = postingsEnum;
deState.minSetSizeCached = maxTermDocFreq;
}
postingsEnum = deState.postingsEnum;
DocSet set = searcher.getDocSet(deState);
maxTermCounts[termNum] = set.size();
}
}
@Override
protected void setActualDocFreq(int termNum, int docFreq) {
maxTermCounts[termNum] = docFreq;
}
public long memSize() {
// can cache the mem size since it shouldn't change
if (memsz!=0) return memsz;
long sz = super.ramBytesUsed();
sz += 8*8 + 32; // local fields
sz += bigTerms.size() * 64;
for (TopTerm tt : bigTerms.values()) {
sz += tt.memSize();
}
if (maxTermCounts != null)
sz += maxTermCounts.length * 4;
if (indexedTermsArray != null) {
// assume 8 byte references?
sz += 8+8+8+8+(indexedTermsArray.length<<3)+sizeOfIndexedStrings;
}
memsz = sz;
return sz;
}
public UnInvertedField(String field, SolrIndexSearcher searcher) throws IOException {
super(field,
// threshold, over which we use set intersections instead of counting
// to (1) save memory, and (2) speed up faceting.
// Add 2 for testing purposes so that there will always be some terms under
// the threshold even when the index is very
// small.
searcher.maxDoc()/20 + 2,
DEFAULT_INDEX_INTERVAL_BITS);
//System.out.println("maxTermDocFreq=" + maxTermDocFreq + " maxDoc=" + searcher.maxDoc());
final String prefix = TrieField.getMainValuePrefix(searcher.getSchema().getFieldType(field));
this.searcher = searcher;
try {
LeafReader r = searcher.getLeafReader();
uninvert(r, r.getLiveDocs(), prefix == null ? null : new BytesRef(prefix));
} catch (IllegalStateException ise) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, ise.getMessage());
}
if (tnums != null) {
for(byte[] target : tnums) {
if (target != null && target.length > (1<<24)*.9) {
SolrCore.log.warn("Approaching too many values for UnInvertedField faceting on field '"+field+"' : bucket size=" + target.length);
}
}
}
// free space if outrageously wasteful (tradeoff memory/cpu)
if ((maxTermCounts.length - numTermsInField) > 1024) { // too much waste!
int[] newMaxTermCounts = new int[numTermsInField];
System.arraycopy(maxTermCounts, 0, newMaxTermCounts, 0, numTermsInField);
maxTermCounts = newMaxTermCounts;
}
SolrCore.log.info("UnInverted multi-valued field " + toString());
//System.out.println("CREATED: " + toString() + " ti.index=" + ti.index);
}
public int getNumTerms() {
return numTermsInField;
}
String getReadableValue(BytesRef termval, FieldType ft, CharsRefBuilder charsRef) {
return ft.indexedToReadable(termval, charsRef).toString();
}
/** may return a reused BytesRef */
BytesRef getTermValue(TermsEnum te, int termNum) throws IOException {
//System.out.println("getTermValue termNum=" + termNum + " this=" + this + " numTerms=" + numTermsInField);
if (bigTerms.size() > 0) {
// see if the term is one of our big terms.
TopTerm tt = bigTerms.get(termNum);
if (tt != null) {
//System.out.println(" return big " + tt.term);
return tt.term;
}
}
return lookupTerm(te, termNum);
}
@Override
public String toString() {
final long indexSize = indexedTermsArray == null ? 0 : (8+8+8+8+(indexedTermsArray.length<<3)+sizeOfIndexedStrings); // assume 8 byte references?
return "{field=" + field
+ ",memSize="+memSize()
+ ",tindexSize="+indexSize
+ ",time="+total_time
+ ",phase1="+phase1_time
+ ",nTerms="+numTermsInField
+ ",bigTerms="+bigTerms.size()
+ ",termInstances="+termInstances
+ ",uses="+use.get()
+ "}";
}
//////////////////////////////////////////////////////////////////
//////////////////////////// caching /////////////////////////////
//////////////////////////////////////////////////////////////////
public static UnInvertedField getUnInvertedField(String field, SolrIndexSearcher searcher) throws IOException {
SolrCache<String,UnInvertedField> cache = searcher.getFieldValueCache();
if (cache == null) {
return new UnInvertedField(field, searcher);
}
UnInvertedField uif = null;
Boolean doWait = false;
synchronized (cache) {
uif = cache.get(field);
if (uif == null) {
/**
* We use this place holder object to pull the UninvertedField construction out of the sync
* so that if many fields are accessed in a short time, the UninvertedField can be
* built for these fields in parallel rather than sequentially.
*/
cache.put(field, uifPlaceholder);
} else {
if (uif != uifPlaceholder) {
return uif;
}
doWait = true; // Someone else has put the place holder in, wait for that to complete.
}
}
while (doWait) {
try {
synchronized (cache) {
uif = cache.get(field); // Should at least return the placeholder, NPE if not is OK.
if (uif != uifPlaceholder) { // OK, another thread put this in the cache we should be good.
return uif;
}
cache.wait();
}
} catch (InterruptedException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Thread interrupted in getUninvertedField.");
}
}
uif = new UnInvertedField(field, searcher);
synchronized (cache) {
cache.put(field, uif); // Note, this cleverly replaces the placeholder.
cache.notifyAll();
}
return uif;
}
}

View File

@ -0,0 +1,87 @@
package org.apache.solr.request.json;
/*
* 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.Collection;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import org.noggit.JSONParser;
public class JSONUtil {
public static boolean advanceToMapKey(JSONParser parser, String key, boolean deepSearch) throws IOException {
for (;;) {
int event = parser.nextEvent();
switch (event) {
case JSONParser.STRING:
if (key != null && parser.wasKey()) {
String val = parser.getString();
if (key.equals(val)) {
return true;
}
}
break;
case JSONParser.OBJECT_END:
return false;
case JSONParser.OBJECT_START:
if (deepSearch) {
boolean found = advanceToMapKey(parser, key, true);
if (found) {
return true;
}
} else {
advanceToMapKey(parser, null, false);
}
break;
case JSONParser.ARRAY_START:
skipArray(parser, key, deepSearch);
break;
}
}
}
public static void skipArray(JSONParser parser, String key, boolean deepSearch) throws IOException {
for (;;) {
int event = parser.nextEvent();
switch (event) {
case JSONParser.OBJECT_START:
advanceToMapKey(parser, key, deepSearch);
break;
case JSONParser.ARRAY_START:
skipArray(parser, key, deepSearch);
break;
case JSONParser.ARRAY_END:
return;
}
}
}
public static void expect(JSONParser parser, int parserEventType) throws IOException {
int event = parser.nextEvent();
if (event != parserEventType) {
throw new IOException("JSON Parser: expected " + JSONParser.getEventString(parserEventType) + " but got " + JSONParser.getEventString(event) );
}
}
}

View File

@ -0,0 +1,118 @@
package org.apache.solr.request.json;
/*
* 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.Collection;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import org.noggit.JSONParser;
public class ObjectUtil {
public static class ConflictHandler {
protected boolean isList(Map<String,Object> container, List<String> path, String key, Object current, Object previous) {
return key!=null && ("fields".equals(key) || "filter".equals(key));
}
public void handleConflict(Map<String,Object> container, List<String> path, String key, Object current, Object previous) {
boolean handleAsList = isList(container, path, key, current, previous);
if (handleAsList) {
container.put(key, makeList(current, previous) );
return;
}
if (previous instanceof Map && current instanceof Map) {
Map<String,Object> prevMap = (Map<String,Object>)previous;
Map<String,Object> currMap = (Map<String,Object>)current;
if (prevMap.size() == 0) return;
mergeMap(prevMap, currMap, path);
container.put(key, prevMap);
return;
}
// if we aren't handling as a list, and we aren't handling as a map, then just overwrite (i.e. nothing else to do)
return;
}
// merges srcMap onto targetMap (i.e. changes targetMap but not srcMap)
public void mergeMap(Map<String,Object> targetMap, Map<String,Object> srcMap, List<String> path) {
if (srcMap.size() == 0) return;
// to keep ordering correct, start with prevMap and merge in currMap
for (Map.Entry<String,Object> srcEntry : srcMap.entrySet()) {
String subKey = srcEntry.getKey();
Object subVal = srcEntry.getValue();
Object subPrev = targetMap.put(subKey, subVal);
if (subPrev != null) {
// recurse
path.add(subKey);
handleConflict(targetMap, path, subKey, subVal, subPrev);
path.remove(path.size()-1);
}
}
}
protected Object makeList(Object current, Object previous) {
ArrayList lst = new ArrayList();
append(lst, previous); // make the original value(s) come first
append(lst, current);
return lst;
}
protected void append(List lst, Object current) {
if (current instanceof Collection) {
lst.addAll((Collection)current);
} else {
lst.add(current);
}
}
}
public static void mergeObjects(Map<String,Object> top, List<String> path, Object val, ConflictHandler handler) {
Map<String,Object> outer = top;
for (int i=0; i<path.size()-1; i++) {
Map<String,Object> sub = (Map<String,Object>)outer.get(path.get(i));
if (sub == null) {
sub = new LinkedHashMap<String,Object>();
outer.put(path.get(i), sub);
}
outer = sub;
}
String key = path.size() > 0 ? path.get(path.size()-1) : null;
if (key != null) {
Object existingVal = outer.put(key, val);
if (existingVal != null) {
// OK, now we need to merge values
handler.handleConflict(outer, path, key, val, existingVal);
}
} else {
// merging at top level...
Map<String,Object> newMap = (Map<String,Object>)val;
handler.mergeMap(outer, newMap, path);
}
}
}

View File

@ -0,0 +1,319 @@
package org.apache.solr.request.json;
/*
* 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 org.apache.commons.io.IOUtils;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.MultiMapSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ContentStream;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.core.SolrCore;
import org.apache.solr.handler.component.SearchHandler;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrRequestHandler;
import org.apache.solr.request.macro.MacroExpander;
import org.noggit.JSONParser;
import org.noggit.ObjectBuilder;
import java.io.IOException;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
public class RequestUtil {
/**
* Set default-ish params on a SolrQueryRequest as well as do standard macro processing and JSON request parsing.
*
* @param handler The search handler this is for (may be null if you don't want this method touching the content streams)
* @param req The request whose params we are interested in
* @param defaults values to be used if no values are specified in the request params
* @param appends values to be appended to those from the request (or defaults) when dealing with multi-val params, or treated as another layer of defaults for singl-val params.
* @param invariants values which will be used instead of any request, or default values, regardless of context.
*/
public static void processParams(SolrRequestHandler handler, SolrQueryRequest req, SolrParams defaults,
SolrParams appends, SolrParams invariants) {
boolean searchHandler = handler instanceof SearchHandler;
SolrParams params = req.getParams();
// Handle JSON stream for search requests
if (searchHandler && req.getContentStreams() != null) {
Map<String,String[]> map = MultiMapSolrParams.asMultiMap(params, false);
if (!(params instanceof MultiMapSolrParams || params instanceof ModifiableSolrParams)) {
// need to set params on request since we weren't able to access the original map
params = new MultiMapSolrParams(map);
req.setParams(params);
}
String[] jsonFromParams = map.remove("json"); // params from the query string should come after (and hence override) JSON content streams
for (ContentStream cs : req.getContentStreams()) {
String contentType = cs.getContentType();
if (contentType==null || !contentType.contains("/json")) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Bad contentType for search handler :" + contentType + " request="+req);
}
try {
String jsonString = IOUtils.toString( cs.getReader() );
MultiMapSolrParams.addParam("json", jsonString, map);
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Exception reading content stream for request:"+req, e);
}
}
// append existing "json" params
if (jsonFromParams != null) {
for (String json : jsonFromParams) {
MultiMapSolrParams.addParam("json", json, map);
}
}
}
String[] jsonS = params.getParams("json");
boolean hasAdditions = defaults != null || invariants != null || appends != null || jsonS != null;
// short circuit processing
if (!hasAdditions && !params.getBool("expandMacros", true)) {
return; // nothing to do...
}
boolean isShard = params.getBool("isShard", false);
Map<String, String[]> newMap = MultiMapSolrParams.asMultiMap(params, hasAdditions);
// see if the json has a "params" section
// TODO: we should currently *not* do this if this is a leaf of a distributed search since it could overwrite parameters set by the top-level
// The parameters we extract will be propagated anyway.
if (jsonS != null && !isShard) {
for (String json : jsonS) {
getParamsFromJSON(newMap, json);
}
}
// first populate defaults, etc..
if (defaults != null) {
Map<String, String[]> defaultsMap = MultiMapSolrParams.asMultiMap(defaults);
for (Map.Entry<String, String[]> entry : defaultsMap.entrySet()) {
String key = entry.getKey();
if (!newMap.containsKey(key)) {
newMap.put(key, entry.getValue());
}
}
}
if (appends != null) {
Map<String, String[]> appendsMap = MultiMapSolrParams.asMultiMap(appends);
for (Map.Entry<String, String[]> entry : appendsMap.entrySet()) {
String key = entry.getKey();
String[] arr = newMap.get(key);
if (arr == null) {
newMap.put(key, entry.getValue());
} else {
String[] appendArr = entry.getValue();
String[] newArr = new String[arr.length + appendArr.length];
System.arraycopy(arr, 0, newArr, 0, arr.length);
System.arraycopy(appendArr, 0, newArr, arr.length, appendArr.length);
newMap.put(key, newArr);
}
}
}
if (invariants != null) {
newMap.putAll( MultiMapSolrParams.asMultiMap(invariants) );
}
String[] doMacrosStr = newMap.get("expandMacros");
boolean doMacros = true;
if (doMacrosStr != null) {
doMacros = "true".equals(doMacrosStr[0]);
}
if (doMacros) {
newMap = MacroExpander.expand(newMap);
}
// Set these params as soon as possible so if there is an error processing later, things like
// "wt=json" will take effect from the defaults.
SolrParams newParams = new MultiMapSolrParams(newMap); // newMap may still change below, but that should be OK
req.setParams(newParams);
// Skip the rest of the processing (including json processing for now) if this isn't a search handler.
// For example json.command started to be used in SOLR-6294, and that caused errors here.
if (!searchHandler) return;
Map<String, Object> json = null;
// Handle JSON body first, so query params will always overlay on that
jsonS = newMap.get("json");
if (jsonS != null) {
if (json == null) {
json = new LinkedHashMap<>();
}
mergeJSON(json, "json", jsonS, new ObjectUtil.ConflictHandler());
}
for (String key : newMap.keySet()) {
// json.nl, json.wrf are existing query parameters
if (key.startsWith("json.") && !("json.nl".equals(key) || "json.wrf".equals(key))) {
if (json == null) {
json = new LinkedHashMap<>();
}
mergeJSON(json, key, newMap.get(key), new ObjectUtil.ConflictHandler());
}
}
// implement compat for existing components...
if (json != null && !isShard) {
for (Map.Entry<String,Object> entry : json.entrySet()) {
String key = entry.getKey();
String out = null;
boolean arr = false;
if ("query".equals(key)) {
out = "q";
} else if ("filter".equals(key)) {
out = "fq";
arr = true;
} else if ("fields".equals(key)) {
out = "fl";
arr = true;
} else if ("offset".equals(key)) {
out = "start";
} else if ("limit".equals(key)) {
out = "rows";
} else if ("sort".equals(key)) {
out = "sort";
} else if ("params".equals(key) || "facet".equals(key) ) {
// handled elsewhere
continue;
} else {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown top-level key in JSON request : " + key);
}
Object val = entry.getValue();
if (arr) {
String[] existing = newMap.get(out);
List lst = val instanceof List ? (List)val : null;
int existingSize = existing==null ? 0 : existing.length;
int jsonSize = lst==null ? 1 : lst.size();
String[] newval = new String[ existingSize + jsonSize ];
for (int i=0; i<existingSize; i++) {
newval[i] = existing[i];
}
if (lst != null) {
for (int i = 0; i < jsonSize; i++) {
Object v = lst.get(i);
newval[existingSize + i] = v.toString();
}
} else {
newval[newval.length-1] = val.toString();
}
newMap.put(out, newval);
} else {
newMap.put(out, new String[]{val.toString()});
}
}
}
if (json != null) {
req.setJSON(json);
}
}
// queryParamName is something like json.facet or json.query, or just json...
private static void mergeJSON(Map<String,Object> json, String queryParamName, String[] vals, ObjectUtil.ConflictHandler handler) {
try {
List<String> path = StrUtils.splitSmart(queryParamName, ".", true);
path = path.subList(1, path.size());
for (String jsonStr : vals) {
Object o = ObjectBuilder.fromJSON(jsonStr);
ObjectUtil.mergeObjects(json, path, o, handler);
}
} catch (IOException e) {
// impossible
}
}
private static void getParamsFromJSON(Map<String, String[]> params, String json) {
if (json.indexOf("params") < 0) {
return;
}
JSONParser parser = new JSONParser(json);
try {
JSONUtil.expect(parser, JSONParser.OBJECT_START);
boolean found = JSONUtil.advanceToMapKey(parser, "params", false);
if (!found) {
return;
}
parser.nextEvent(); // advance to the value
Object o = ObjectBuilder.getVal(parser);
if (!(o instanceof Map)) return;
Map<String,Object> map = (Map<String,Object>)o;
// To make consistent with json.param handling, we should make query params come after json params (i.e. query params should
// appear to overwrite json params.
// Solr params are based on String though, so we need to convert
for (Map.Entry<String, Object> entry : map.entrySet()) {
String key = entry.getKey();
Object val = entry.getValue();
if (params.get(key) != null) {
continue;
}
if (val == null) {
params.remove(key);
} else if (val instanceof List) {
List lst = (List) val;
String[] vals = new String[lst.size()];
for (int i = 0; i < vals.length; i++) {
vals[i] = lst.get(i).toString();
}
params.put(key, vals);
} else {
params.put(key, new String[]{val.toString()});
}
}
} catch (Exception e) {
// ignore parse exceptions at this stage, they may be caused by incomplete macro expansions
return;
}
}
}

View File

@ -0,0 +1,27 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
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.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
<p>
JSON related classes
</p>
</body>
</html>

View File

@ -0,0 +1,183 @@
package org.apache.solr.request.macro;
/*
* 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 org.apache.solr.common.SolrException;
import org.apache.solr.search.QueryParsing;
import org.apache.solr.search.SyntaxError;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class MacroExpander {
public static final String MACRO_START = "${";
private static final int MAX_LEVELS = 25;
private Map<String,String[]> orig;
private Map<String,String[]> expanded;
private String macroStart = MACRO_START;
private char escape = '\\';
private int level;
public MacroExpander(Map<String,String[]> orig) {
this.orig = orig;
}
public static Map<String,String[]> expand(Map<String,String[]> params) {
MacroExpander mc = new MacroExpander(params);
mc.expand();
return mc.expanded;
}
public boolean expand() {
this.expanded = new HashMap<>(orig.size());
boolean changed = false;
for (Map.Entry<String,String[]> entry : orig.entrySet()) {
String k = entry.getKey();
String newK = expand(k);
String[] values = entry.getValue();
List<String> newValues = null;
for (String v : values) {
String newV = expand(v);
if (newV != v) {
if (newValues == null) {
newValues = new ArrayList<>(values.length);
for (String vv : values) {
if (vv == v) break;
newValues.add(vv);
}
}
newValues.add(newV);
}
}
if (newValues != null) {
values = newValues.toArray(new String[newValues.size()]);
changed = true;
}
if (k != newK) {
changed = true;
}
expanded.put( newK, values );
}
return changed;
}
public String expand(String val) {
level++;
try {
if (level >= MAX_LEVELS) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Request template exceeded max nesting of " + MAX_LEVELS + " expanding '"+val+"'");
}
return _expand(val);
} finally {
level--;
}
}
private String _expand(String val) {
// quickest short circuit
int idx = val.indexOf(macroStart.charAt(0));
if (idx < 0) return val;
int start = 0; // start of the unprocessed part of the string
int end = 0;
StringBuilder sb = null;
for (;;) {
idx = val.indexOf(macroStart, idx);
int matchedStart = idx;
// check if escaped
if (idx > 0) {
// check if escaped...
// TODO: what if you *want* to actually have a backslash... perhaps that's when we allow changing
// of the escape character?
char ch = val.charAt(idx-1);
if (ch == escape) {
idx += macroStart.length();
continue;
}
}
else if (idx < 0) {
if (sb == null) return val;
sb.append(val.substring(start));
return sb.toString();
}
// found unescaped "${"
idx += macroStart.length();
int rbrace = val.indexOf('}', idx);
if (rbrace == -1) {
// no matching close brace...
continue;
}
if (sb == null) {
sb = new StringBuilder(val.length()*2);
}
if (matchedStart > 0) {
sb.append(val.substring(start, matchedStart));
}
// update "start" to be at the end of ${...}
start = rbrace + 1;
// String inbetween = val.substring(idx, rbrace);
QueryParsing.StrParser parser = new QueryParsing.StrParser(val, idx, rbrace);
try {
String paramName = parser.getId();
String defVal = null;
boolean hasDefault = parser.opt(':');
if (hasDefault) {
defVal = val.substring(parser.pos, rbrace);
}
// in the event that expansions become context dependent... consult original?
String[] replacementList = orig.get(paramName);
// TODO - handle a list somehow...
String replacement = replacementList!=null ? replacementList[0] : defVal;
if (replacement != null) {
String expandedReplacement = expand(replacement);
sb.append(expandedReplacement);
}
} catch (SyntaxError syntaxError) {
// append the part we would have skipped
sb.append( val.substring(matchedStart, start) );
continue;
}
}
}
}

View File

@ -0,0 +1,27 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
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.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
<p>
Parameter substitution / macro expansion.
</p>
</body>
</html>

View File

@ -17,6 +17,8 @@
package org.apache.solr.search; package org.apache.solr.search;
import java.io.Closeable;
import org.apache.lucene.search.Filter; import org.apache.lucene.search.Filter;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
@ -30,7 +32,7 @@ import org.apache.solr.common.SolrException;
* *
* @since solr 0.9 * @since solr 0.9
*/ */
public interface DocSet /* extends Collection<Integer> */ { public interface DocSet extends Closeable /* extends Collection<Integer> */ {
/** /**
* Adds the specified document if it is not currently in the DocSet * Adds the specified document if it is not currently in the DocSet

View File

@ -17,6 +17,8 @@
package org.apache.solr.search; package org.apache.solr.search;
import java.io.IOException;
import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.BitsFilteredDocIdSet; import org.apache.lucene.search.BitsFilteredDocIdSet;
@ -239,4 +241,9 @@ abstract class DocSetBase implements DocSet {
} }
} }
/** FUTURE: for off-heap */
@Override
public void close() throws IOException {
}
} }

View File

@ -23,6 +23,7 @@ import org.apache.lucene.search.Query;
import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.params.SolrParams;
import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.schema.SchemaField; import org.apache.solr.schema.SchemaField;
import org.apache.solr.search.facet.AggValueSource;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
@ -30,6 +31,10 @@ import java.util.List;
public class FunctionQParser extends QParser { public class FunctionQParser extends QParser {
public static final int FLAG_CONSUME_DELIMITER = 0x01; // consume delimiter after parsing arg
public static final int FLAG_IS_AGG = 0x02;
public static final int FLAG_DEFAULT = FLAG_CONSUME_DELIMITER;
/** @lucene.internal */ /** @lucene.internal */
public QueryParsing.StrParser sp; public QueryParsing.StrParser sp;
boolean parseMultipleSources = true; boolean parseMultipleSources = true;
@ -37,6 +42,15 @@ public class FunctionQParser extends QParser {
public FunctionQParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest req) { public FunctionQParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest req) {
super(qstr, localParams, params, req); super(qstr, localParams, params, req);
setString(qstr);
}
@Override
public void setString(String s) {
super.setString(s);
if (s != null) {
sp = new QueryParsing.StrParser( s );
}
} }
public void setParseMultipleSources(boolean parseMultipleSources) { public void setParseMultipleSources(boolean parseMultipleSources) {
@ -59,13 +73,11 @@ public class FunctionQParser extends QParser {
@Override @Override
public Query parse() throws SyntaxError { public Query parse() throws SyntaxError {
sp = new QueryParsing.StrParser(getString());
ValueSource vs = null; ValueSource vs = null;
List<ValueSource> lst = null; List<ValueSource> lst = null;
for(;;) { for(;;) {
ValueSource valsource = parseValueSource(false); ValueSource valsource = parseValueSource(FLAG_DEFAULT & ~FLAG_CONSUME_DELIMITER);
sp.eatws(); sp.eatws();
if (!parseMultipleSources) { if (!parseMultipleSources) {
vs = valsource; vs = valsource;
@ -210,7 +222,7 @@ public class FunctionQParser extends QParser {
public List<ValueSource> parseValueSourceList() throws SyntaxError { public List<ValueSource> parseValueSourceList() throws SyntaxError {
List<ValueSource> sources = new ArrayList<>(3); List<ValueSource> sources = new ArrayList<>(3);
while (hasMoreArguments()) { while (hasMoreArguments()) {
sources.add(parseValueSource(true)); sources.add(parseValueSource(FLAG_DEFAULT | FLAG_CONSUME_DELIMITER));
} }
return sources; return sources;
} }
@ -220,7 +232,7 @@ public class FunctionQParser extends QParser {
*/ */
public ValueSource parseValueSource() throws SyntaxError { public ValueSource parseValueSource() throws SyntaxError {
/* consume the delimiter afterward for an external call to parseValueSource */ /* consume the delimiter afterward for an external call to parseValueSource */
return parseValueSource(true); return parseValueSource(FLAG_DEFAULT | FLAG_CONSUME_DELIMITER);
} }
/* /*
@ -274,7 +286,11 @@ public class FunctionQParser extends QParser {
* *
* @param doConsumeDelimiter whether to consume a delimiter following the ValueSource * @param doConsumeDelimiter whether to consume a delimiter following the ValueSource
*/ */
protected ValueSource parseValueSource(boolean doConsumeDelimiter) throws SyntaxError { protected ValueSource parseValueSource(boolean doConsumeDelimiter) throws SyntaxError {
return parseValueSource( doConsumeDelimiter ? (FLAG_DEFAULT | FLAG_CONSUME_DELIMITER) : (FLAG_DEFAULT & ~FLAG_CONSUME_DELIMITER) );
}
protected ValueSource parseValueSource(int flags) throws SyntaxError {
ValueSource valueSource; ValueSource valueSource;
int ch = sp.peek(); int ch = sp.peek();
@ -365,12 +381,57 @@ public class FunctionQParser extends QParser {
} }
if (doConsumeDelimiter) if ((flags & FLAG_CONSUME_DELIMITER) != 0) {
consumeArgumentDelimiter(); consumeArgumentDelimiter();
}
return valueSource; return valueSource;
} }
/** @lucene.experimental */
public AggValueSource parseAgg(int flags) throws SyntaxError {
String id = sp.getId();
AggValueSource vs = null;
boolean hasParen = false;
if ("agg".equals(id)) {
hasParen = sp.opt("(");
vs = parseAgg(flags | FLAG_IS_AGG);
} else {
// parse as an aggregation...
if (!id.startsWith("agg_")) {
id = "agg_" + id;
}
hasParen = sp.opt("(");
ValueSourceParser argParser = req.getCore().getValueSourceParser(id);
argParser = req.getCore().getValueSourceParser(id);
if (argParser == null) {
throw new SyntaxError("Unknown aggregation " + id + " in (" + sp + ")");
}
ValueSource vv = argParser.parse(this);
if (!(vv instanceof AggValueSource)) {
if (argParser == null) {
throw new SyntaxError("Expected aggregation from " + id + " but got (" + vv + ") in (" + sp + ")");
}
}
vs = (AggValueSource) vv;
}
if (hasParen) {
sp.expect(")");
}
if ((flags & FLAG_CONSUME_DELIMITER) != 0) {
consumeArgumentDelimiter();
}
return vs;
}
/** /**
* Consume an argument delimiter (a comma) from the token stream. * Consume an argument delimiter (a comma) from the token stream.
* Only consumes if more arguments should exist (no ending parens or end of string). * Only consumes if more arguments should exist (no ending parens or end of string).

View File

@ -0,0 +1,104 @@
package org.apache.solr.search;
/*
* 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.IdentityHashMap;
import org.apache.lucene.search.IndexSearcher;
import org.apache.solr.common.SolrException;
import org.apache.solr.core.SolrCore;
import org.apache.solr.request.SolrRequestInfo;
/**
* Bridge between old style context and a real class
* @lucene.experimental
*/
public class QueryContext extends IdentityHashMap implements Closeable {
// private IdentityHashMap map; // we are the map for now (for compat w/ ValueSource)
private final SolrIndexSearcher searcher;
private final IndexSearcher indexSearcher;
private IdentityHashMap<Closeable,String> closeHooks;
// migrated from ValueSource
public static QueryContext newContext(IndexSearcher searcher) {
QueryContext context = new QueryContext(searcher);
return context;
}
public QueryContext(IndexSearcher searcher) {
this.searcher = searcher instanceof SolrIndexSearcher ? (SolrIndexSearcher)searcher : null;
indexSearcher = searcher;
this.put("searcher", searcher); // see ValueSource.newContext()
}
public SolrIndexSearcher searcher() {
return searcher;
}
public IndexSearcher indexSearcher() {
return indexSearcher;
}
/*** implementations obtained via inheritance
public Object get(Object key) {
return map.get(key);
}
public Object put(Object key, Object val) {
if (map == null) {
map = new IdentityHashMap();
}
return map.put(key, val);
}
***/
public void addCloseHook(Closeable closeable) {
if (closeHooks == null) {
closeHooks = new IdentityHashMap<Closeable, String>();
// for now, defer closing until the end of the request
SolrRequestInfo.getRequestInfo().addCloseHook(this);
}
closeHooks.put(closeable, "");
}
public boolean removeCloseHook(Closeable closeable) {
return closeHooks.remove(closeable) != null;
}
/** Don't call close explicitly! This will be automatically closed at the end of the request */
@Override
public void close() throws IOException {
if (closeHooks != null) {
for (Closeable hook : closeHooks.keySet()) {
try {
hook.close();
} catch (Exception e) {
SolrException.log(SolrCore.log, "Exception during close hook", e);
}
}
}
closeHooks = null;
// map = null;
}
}

View File

@ -593,9 +593,9 @@ public class QueryParsing {
* <b>Note: This API is experimental and may change in non backward-compatible ways in the future</b> * <b>Note: This API is experimental and may change in non backward-compatible ways in the future</b>
*/ */
public static class StrParser { public static class StrParser {
String val; public String val;
int pos; public int pos;
int end; public int end;
public StrParser(String val) { public StrParser(String val) {
this(val, 0, val.length()); this(val, 0, val.length());
@ -607,19 +607,19 @@ public class QueryParsing {
this.end = end; this.end = end;
} }
void eatws() { public void eatws() {
while (pos < end && Character.isWhitespace(val.charAt(pos))) pos++; while (pos < end && Character.isWhitespace(val.charAt(pos))) pos++;
} }
char ch() { public char ch() {
return pos < end ? val.charAt(pos) : 0; return pos < end ? val.charAt(pos) : 0;
} }
void skip(int nChars) { public void skip(int nChars) {
pos = Math.max(pos + nChars, end); pos = Math.max(pos + nChars, end);
} }
boolean opt(String s) { public boolean opt(String s) {
eatws(); eatws();
int slen = s.length(); int slen = s.length();
if (val.regionMatches(pos, s, 0, slen)) { if (val.regionMatches(pos, s, 0, slen)) {
@ -629,7 +629,7 @@ public class QueryParsing {
return false; return false;
} }
boolean opt(char ch) { public boolean opt(char ch) {
eatws(); eatws();
if (pos < end && val.charAt(pos) == ch) { if (pos < end && val.charAt(pos) == ch) {
pos++; pos++;
@ -639,7 +639,7 @@ public class QueryParsing {
} }
void expect(String s) throws SyntaxError { public void expect(String s) throws SyntaxError {
eatws(); eatws();
int slen = s.length(); int slen = s.length();
if (val.regionMatches(pos, s, 0, slen)) { if (val.regionMatches(pos, s, 0, slen)) {
@ -649,7 +649,7 @@ public class QueryParsing {
} }
} }
float getFloat() { public float getFloat() {
eatws(); eatws();
char[] arr = new char[end - pos]; char[] arr = new char[end - pos];
int i; int i;
@ -669,7 +669,7 @@ public class QueryParsing {
return Float.parseFloat(new String(arr, 0, i)); return Float.parseFloat(new String(arr, 0, i));
} }
Number getNumber() { public Number getNumber() {
eatws(); eatws();
int start = pos; int start = pos;
boolean flt = false; boolean flt = false;
@ -694,7 +694,7 @@ public class QueryParsing {
} }
} }
double getDouble() { public double getDouble() {
eatws(); eatws();
char[] arr = new char[end - pos]; char[] arr = new char[end - pos];
int i; int i;
@ -714,7 +714,7 @@ public class QueryParsing {
return Double.parseDouble(new String(arr, 0, i)); return Double.parseDouble(new String(arr, 0, i));
} }
int getInt() { public int getInt() {
eatws(); eatws();
char[] arr = new char[end - pos]; char[] arr = new char[end - pos];
int i; int i;
@ -734,11 +734,11 @@ public class QueryParsing {
} }
String getId() throws SyntaxError { public String getId() throws SyntaxError {
return getId("Expected identifier"); return getId("Expected identifier");
} }
String getId(String errMessage) throws SyntaxError { public String getId(String errMessage) throws SyntaxError {
eatws(); eatws();
int id_start = pos; int id_start = pos;
char ch; char ch;
@ -787,7 +787,7 @@ public class QueryParsing {
* Skips leading whitespace and returns whatever sequence of non * Skips leading whitespace and returns whatever sequence of non
* whitespace it can find (or hte empty string) * whitespace it can find (or hte empty string)
*/ */
String getSimpleString() { public String getSimpleString() {
eatws(); eatws();
int startPos = pos; int startPos = pos;
char ch; char ch;
@ -804,7 +804,7 @@ public class QueryParsing {
* sort direction. (True is desc, False is asc). * sort direction. (True is desc, False is asc).
* Position is advanced to after the comma (or end) when result is non null * Position is advanced to after the comma (or end) when result is non null
*/ */
Boolean getSortDirection() throws SyntaxError { public Boolean getSortDirection() throws SyntaxError {
final int startPos = pos; final int startPos = pos;
final String order = getId(null); final String order = getId(null);
@ -835,7 +835,7 @@ public class QueryParsing {
} }
// return null if not a string // return null if not a string
String getQuotedString() throws SyntaxError { public String getQuotedString() throws SyntaxError {
eatws(); eatws();
char delim = peekChar(); char delim = peekChar();
if (!(delim == '\"' || delim == '\'')) { if (!(delim == '\"' || delim == '\'')) {
@ -888,13 +888,13 @@ public class QueryParsing {
} }
// next non-whitespace char // next non-whitespace char
char peek() { public char peek() {
eatws(); eatws();
return pos < end ? val.charAt(pos) : 0; return pos < end ? val.charAt(pos) : 0;
} }
// next char // next char
char peekChar() { public char peekChar() {
return pos < end ? val.charAt(pos) : 0; return pos < end ? val.charAt(pos) : 0;
} }

View File

@ -80,7 +80,7 @@ import org.apache.solr.core.SolrInfoMBean;
import org.apache.solr.request.LocalSolrQueryRequest; import org.apache.solr.request.LocalSolrQueryRequest;
import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrRequestInfo; import org.apache.solr.request.SolrRequestInfo;
import org.apache.solr.request.UnInvertedField; import org.apache.solr.search.facet.UnInvertedField;
import org.apache.solr.response.SolrQueryResponse; import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.schema.IndexSchema; import org.apache.solr.schema.IndexSchema;
import org.apache.solr.schema.SchemaField; import org.apache.solr.schema.SchemaField;
@ -775,7 +775,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
//////////////////////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////////////////////
/** expert: internal API, subject to change */ /** expert: internal API, subject to change */
public SolrCache<String,UnInvertedField> getFieldValueCache() { public SolrCache<String, org.apache.solr.search.facet.UnInvertedField> getFieldValueCache() {
return fieldValueCache; return fieldValueCache;
} }

View File

@ -151,7 +151,7 @@ public class SolrReturnFields extends ReturnFields {
} }
// like getId, but also accepts dashes for legacy fields // like getId, but also accepts dashes for legacy fields
String getFieldName(QueryParsing.StrParser sp) { public static String getFieldName(QueryParsing.StrParser sp) {
sp.eatws(); sp.eatws();
int id_start = sp.pos; int id_start = sp.pos;
char ch; char ch;

View File

@ -40,6 +40,14 @@ import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.NamedList;
import org.apache.solr.request.SolrRequestInfo; import org.apache.solr.request.SolrRequestInfo;
import org.apache.solr.schema.*; import org.apache.solr.schema.*;
import org.apache.solr.search.facet.AggValueSource;
import org.apache.solr.search.facet.AvgAgg;
import org.apache.solr.search.facet.CountAgg;
import org.apache.solr.search.facet.MaxAgg;
import org.apache.solr.search.facet.MinAgg;
import org.apache.solr.search.facet.SumAgg;
import org.apache.solr.search.facet.SumsqAgg;
import org.apache.solr.search.facet.UniqueAgg;
import org.apache.solr.search.function.CollapseScoreFunction; import org.apache.solr.search.function.CollapseScoreFunction;
import org.apache.solr.search.function.OrdFieldSource; import org.apache.solr.search.function.OrdFieldSource;
import org.apache.solr.search.function.ReverseOrdFieldSource; import org.apache.solr.search.function.ReverseOrdFieldSource;
@ -787,8 +795,88 @@ public abstract class ValueSourceParser implements NamedListInitializedPlugin {
} }
}); });
addParser("agg", new ValueSourceParser() {
@Override
public AggValueSource parse(FunctionQParser fp) throws SyntaxError {
return fp.parseAgg(FunctionQParser.FLAG_DEFAULT);
}
});
addParser("agg_count", new ValueSourceParser() {
@Override
public ValueSource parse(FunctionQParser fp) throws SyntaxError {
return new CountAgg();
}
});
addParser("agg_unique", new ValueSourceParser() {
@Override
public ValueSource parse(FunctionQParser fp) throws SyntaxError {
return new UniqueAgg(fp.parseArg());
}
});
addParser("agg_sum", new ValueSourceParser() {
@Override
public ValueSource parse(FunctionQParser fp) throws SyntaxError {
return new SumAgg(fp.parseValueSource());
}
});
addParser("agg_avg", new ValueSourceParser() {
@Override
public ValueSource parse(FunctionQParser fp) throws SyntaxError {
return new AvgAgg(fp.parseValueSource());
}
});
addParser("agg_sumsq", new ValueSourceParser() {
@Override
public ValueSource parse(FunctionQParser fp) throws SyntaxError {
return new SumsqAgg(fp.parseValueSource());
}
});
/***
addParser("agg_stdev", new ValueSourceParser() {
@Override
public ValueSource parse(FunctionQParser fp) throws SyntaxError {
return null;
}
});
addParser("agg_multistat", new ValueSourceParser() {
@Override
public ValueSource parse(FunctionQParser fp) throws SyntaxError {
return null;
}
});
***/
addParser("agg_min", new ValueSourceParser() {
@Override
public ValueSource parse(FunctionQParser fp) throws SyntaxError {
return new MinAgg(fp.parseValueSource());
}
});
addParser("agg_max", new ValueSourceParser() {
@Override
public ValueSource parse(FunctionQParser fp) throws SyntaxError {
return new MaxAgg(fp.parseValueSource());
}
});
} }
///////////////////////////////////////////////////////////////////////////////
///////////////////////////////////////////////////////////////////////////////
///////////////////////////////////////////////////////////////////////////////
private static TInfo parseTerm(FunctionQParser fp) throws SyntaxError { private static TInfo parseTerm(FunctionQParser fp) throws SyntaxError {
TInfo tinfo = new TInfo(); TInfo tinfo = new TInfo();

View File

@ -0,0 +1,62 @@
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.Map;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.solr.search.QueryContext;
public abstract class AggValueSource extends ValueSource {
protected String name;
public AggValueSource(String name) {
this.name = name;
}
public String name() {
return this.name;
}
public ValueSource[] getChildren() {
return null;
}
@Override
public boolean equals(Object o) {
return this.getClass() == o.getClass() && name.equals(((AggValueSource) o).name);
}
@Override
public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
// FUTURE
throw new UnsupportedOperationException("NOT IMPLEMENTED " + name + " " + this);
}
// TODO: make abstract
public SlotAcc createSlotAcc(FacetContext fcontext, int numDocs, int numSlots) throws IOException {
throw new UnsupportedOperationException("NOT IMPLEMENTED " + name + " " + this);
}
public abstract FacetMerger createFacetMerger(Object prototype);
}

View File

@ -0,0 +1,55 @@
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.List;
import org.apache.lucene.queries.function.ValueSource;
public class AvgAgg extends SimpleAggValueSource {
public AvgAgg(ValueSource vs) {
super("avg", vs);
}
@Override
public SlotAcc createSlotAcc(FacetContext fcontext, int numDocs, int numSlots) throws IOException {
return new AvgSlotAcc(getArg(), fcontext, numSlots);
}
@Override
public FacetMerger createFacetMerger(Object prototype) {
return new FacetMerger() {
long num;
double sum;
@Override
public void merge(Object facetResult) {
List<Number> numberList = (List<Number>)facetResult;
num += numberList.get(0).longValue();
sum += numberList.get(1).doubleValue();
}
@Override
public Object getMergedResult() {
return num==0 ? 0.0d : sum/num;
}
};
}
}

View File

@ -0,0 +1,36 @@
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;
public class CountAgg extends SimpleAggValueSource {
public CountAgg() {
super("count", null);
}
@Override
public SlotAcc createSlotAcc(FacetContext fcontext, int numDocs, int numSlots) throws IOException {
return new CountSlotAcc(fcontext, numSlots);
}
@Override
public FacetMerger createFacetMerger(Object prototype) {
return new FacetLongMerger();
}
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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);
}
}

View File

@ -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());
}
}
}

View File

@ -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;
}
}

View File

@ -0,0 +1,66 @@
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.index.SortedDocValues;
import org.apache.lucene.util.BytesRef;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.search.QParser;
import org.apache.solr.search.QueryContext;
import org.apache.solr.search.SolrIndexSearcher;
/** @lucene.internal
* Porting helper... may be removed if it offers no value in the future.
*/
public class FieldUtil {
/** Simpler method that creates a request context and looks up the field for you */
public static SortedDocValues getSortedDocValues(SolrIndexSearcher searcher, String field) throws IOException {
SchemaField sf = searcher.getSchema().getField(field);
QueryContext qContext = QueryContext.newContext(searcher);
return getSortedDocValues( qContext, sf, null );
}
public static SortedDocValues getSortedDocValues(QueryContext context, SchemaField field, QParser qparser) throws IOException {
SortedDocValues si = context.searcher().getLeafReader().getSortedDocValues( field.getName() );
// if (!field.hasDocValues() && (field.getType() instanceof StrField || field.getType() instanceof TextField)) {
// }
return si == null ? EMPTY_SortedDocValues : si;
}
private static SortedDocValues EMPTY_SortedDocValues = new SortedDocValues() {
@Override
public int getOrd(int docID) {
return -1;
}
@Override
public BytesRef lookupOrd(int ord) {
return null;
}
@Override
public int getValueCount() {
return 0;
}
};
}

View File

@ -0,0 +1,318 @@
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.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.FacetParams;
import org.apache.solr.common.params.RequiredSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.core.SolrCore;
import org.apache.solr.search.QueryParsing;
import org.apache.solr.search.SolrReturnFields;
import org.apache.solr.search.SyntaxError;
public class LegacyFacet {
private SolrParams params;
private Map<String,Object> json;
private Map<String,Object> currentCommand = null; // always points to the current facet command
private Map<String,Object> currentSubs; // always points to the current facet:{} block
String facetValue;
String key;
SolrParams localParams;
SolrParams orig;
SolrParams required;
Map<String, List<Subfacet>> subFacets; // only parsed once
public LegacyFacet(SolrParams params) {
this.params = params;
this.orig = params;
this.json = new LinkedHashMap<>();
this.currentSubs = json;
}
Map<String,Object> getLegacy() {
subFacets = parseSubFacets(params);
String[] queries = params.getParams(FacetParams.FACET_QUERY);
if (queries != null) {
for (String q : queries) {
addQueryFacet(q);
}
}
String[] fields = params.getParams(FacetParams.FACET_FIELD);
if (fields != null) {
for (String field : fields) {
addFieldFacet(field);
}
}
String[] ranges = params.getParams(FacetParams.FACET_RANGE);
if (ranges != null) {
for (String range : ranges) {
addRangeFacet(range);
}
}
// SolrCore.log.error("###################### JSON FACET:" + json);
return json;
}
protected static class Subfacet {
public String parentKey;
public String type; // query, range, field
public String value; // the actual field or the query, including possible local params
}
protected static Map<String, List<Subfacet>> parseSubFacets(SolrParams params) {
Map<String,List<Subfacet>> map = new HashMap<>();
Iterator<String> iter = params.getParameterNamesIterator();
String SUBFACET="subfacet.";
while (iter.hasNext()) {
String key = iter.next();
if (key.startsWith(SUBFACET)) {
List<String> parts = StrUtils.splitSmart(key, '.');
if (parts.size() != 3) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "expected subfacet parameter name of the form subfacet.mykey.field, got:" + key);
}
Subfacet sub = new Subfacet();
sub.parentKey = parts.get(1);
sub.type = parts.get(2);
sub.value = params.get(key);
List<Subfacet> subs = map.get(sub.parentKey);
if (subs == null) {
subs = new ArrayList<>(1);
}
subs.add(sub);
map.put(sub.parentKey, subs);
}
}
return map;
}
protected void addQueryFacet(String q) {
parseParams(FacetParams.FACET_QUERY, q);
Map<String,Object> cmd = new HashMap<String,Object>(2);
Map<String,Object> type = new HashMap<String,Object>(1);
type.put("query", cmd);
cmd.put("q", q);
addSub(key, type);
handleSubs(cmd);
}
protected void addRangeFacet(String field) {
parseParams(FacetParams.FACET_RANGE, field);
Map<String,Object> cmd = new HashMap<String,Object>(5);
Map<String,Object> type = new HashMap<String,Object>(1);
type.put("range", cmd);
String f = key;
cmd.put("field", facetValue);
cmd.put("start", required.getFieldParam(f,FacetParams.FACET_RANGE_START));
cmd.put("end", required.getFieldParam(f,FacetParams.FACET_RANGE_END));
cmd.put("gap", required.getFieldParam(f, FacetParams.FACET_RANGE_GAP));
String[] p = params.getFieldParams(f, FacetParams.FACET_RANGE_OTHER);
if (p != null) cmd.put("other", p.length==1 ? p[0] : Arrays.asList(p));
p = params.getFieldParams(f, FacetParams.FACET_RANGE_INCLUDE);
if (p != null) cmd.put("include", p.length==1 ? p[0] : Arrays.asList(p));
final int mincount = params.getFieldInt(f,FacetParams.FACET_MINCOUNT, 0);
cmd.put("mincount", mincount);
boolean hardend = params.getFieldBool(f,FacetParams.FACET_RANGE_HARD_END,false);
if (hardend) cmd.put("hardend", hardend);
addSub(key, type);
handleSubs(cmd);
}
protected void addFieldFacet(String field) {
parseParams(FacetParams.FACET_FIELD, field);
String f = key; // the parameter to use for per-field parameters... f.key.facet.limit=10
int offset = params.getFieldInt(f, FacetParams.FACET_OFFSET, 0);
int limit = params.getFieldInt(f, FacetParams.FACET_LIMIT, 10);
int mincount = params.getFieldInt(f, FacetParams.FACET_MINCOUNT, 1);
boolean missing = params.getFieldBool(f, FacetParams.FACET_MISSING, false);
// default to sorting if there is a limit.
String sort = params.getFieldParam(f, FacetParams.FACET_SORT, limit>0 ? FacetParams.FACET_SORT_COUNT : FacetParams.FACET_SORT_INDEX);
String prefix = params.getFieldParam(f, FacetParams.FACET_PREFIX);
Map<String,Object> cmd = new HashMap<String,Object>();
cmd.put("field", facetValue);
if (offset != 0) cmd.put("offset", offset);
if (limit != 10) cmd.put("limit", limit);
if (mincount != 1) cmd.put("mincount", mincount);
if (missing) cmd.put("missing", missing);
if (prefix != null) cmd.put("prefix", prefix);
if (sort.equals("count")) {
// our default
} else if (sort.equals("index")) {
cmd.put("sort", "index asc");
} else {
cmd.put("sort", sort); // can be sort by one of our stats
}
Map<String,Object> type = new HashMap<>(1);
type.put("terms", cmd);
addSub(key, type);
handleSubs(cmd);
}
private void handleSubs(Map<String,Object> cmd) {
Map<String,Object> savedCmd = currentCommand;
Map<String,Object> savedSubs = currentSubs;
try {
currentCommand = cmd;
currentSubs = null;
// parse stats for this facet
String[] stats = params.getFieldParams(key, "facet.stat");
if (stats != null) {
for (String stat : stats) {
addStat(stat);
}
}
List<Subfacet> subs = subFacets.get(key);
if (subs != null) {
for (Subfacet subfacet : subs) {
if ("field".equals(subfacet.type)) {
addFieldFacet(subfacet.value);
} else if ("query".equals(subfacet.type)) {
addQueryFacet(subfacet.value);
} else if ("range".equals(subfacet.type)) {
addQueryFacet(subfacet.value);
}
}
}
} finally {
currentCommand = savedCmd;
currentSubs = savedSubs;
}
}
private void addStat(String val) {
QueryParsing.StrParser sp = new QueryParsing.StrParser(val);
int start = 0;
sp.eatws();
if (sp.pos >= sp.end) addStat(val, val);
// try key:func() format
String key = null;
String funcStr = val;
if (key == null) {
key = SolrReturnFields.getFieldName(sp);
if (key != null && sp.opt(':')) {
// OK, we got the key
funcStr = val.substring(sp.pos);
} else {
// an invalid key... it must not be present.
sp.pos = start;
key = null;
}
}
if (key == null) {
key = funcStr; // not really ideal
}
addStat(key, funcStr);
}
private void addStat(String key, String val) {
if ("count".equals(val) || "count()".equals(val)) return; // we no longer have a count function, we always return the count
getCurrentSubs().put(key, val);
}
private void addSub(String key, Map<String,Object> sub) {
getCurrentSubs().put(key, sub);
}
private Map<String,Object> getCurrentSubs() {
if (currentSubs == null) {
currentSubs = new LinkedHashMap();
currentCommand.put("facet", currentSubs);
}
return currentSubs;
}
protected void parseParams(String type, String param) {
facetValue = param;
key = param;
try {
localParams = QueryParsing.getLocalParams(param, orig);
if (localParams == null) {
params = orig;
required = new RequiredSolrParams(params);
// setupStats();
return;
}
params = SolrParams.wrapDefaults(localParams, orig);
required = new RequiredSolrParams(params);
// remove local params unless it's a query
if (type != FacetParams.FACET_QUERY) {
facetValue = localParams.get(CommonParams.VALUE);
}
// reset set the default key now that localParams have been removed
key = facetValue;
// allow explicit set of the key
key = localParams.get(CommonParams.OUTPUT_KEY, key);
// setupStats();
} catch (SyntaxError e) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
}
}
}

View File

@ -0,0 +1,56 @@
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.queries.function.ValueSource;
public class MaxAgg extends SimpleAggValueSource {
public MaxAgg(ValueSource vs) {
super("max", vs);
}
@Override
public SlotAcc createSlotAcc(FacetContext fcontext, int numDocs, int numSlots) throws IOException {
return new MaxSlotAcc(getArg(), fcontext, numSlots);
}
@Override
public FacetMerger createFacetMerger(Object prototype) {
return new FacetMerger() {
double val = Double.NaN;
@Override
public void merge(Object facetResult) {
double result = ((Number)facetResult).doubleValue();
if (result > val || Double.isNaN(val)) {
val = result;
}
}
@Override
public Object getMergedResult() {
return val;
}
};
}
}

View File

@ -0,0 +1,53 @@
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.queries.function.ValueSource;
public class MinAgg extends SimpleAggValueSource {
public MinAgg(ValueSource vs) {
super("min", vs);
}
@Override
public SlotAcc createSlotAcc(FacetContext fcontext, int numDocs, int numSlots) throws IOException {
return new MinSlotAcc(getArg(), fcontext, numSlots);
}
@Override
public FacetMerger createFacetMerger(Object prototype) {
return new FacetMerger() {
double val = Double.NaN;
@Override
public void merge(Object facetResult) {
double result = ((Number)facetResult).doubleValue();
if (result < val || Double.isNaN(val)) {
val = result;
}
}
@Override
public Object getMergedResult() {
return val;
}
};
}
}

View File

@ -0,0 +1,66 @@
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 org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.solr.search.QueryContext;
import java.io.IOException;
import java.util.Map;
public abstract class SimpleAggValueSource extends AggValueSource {
ValueSource arg;
public SimpleAggValueSource(String name, ValueSource vs) {
super(name);
this.arg = vs;
}
public ValueSource getArg() {
return arg;
}
@Override
public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public boolean equals(Object o) {
if (!super.equals(o)) return false;
ValueSource otherArg = ((SimpleAggValueSource)o).arg;
if (arg == otherArg) return true;
return (arg != null && arg.equals(otherArg));
}
@Override
public int hashCode() {
return getClass().hashCode() + (arg == null ? 0 : arg.hashCode());
}
@Override
public String description() {
return name() + "(" + (arg==null ? "" : arg.description()) + ")";
}
}

View File

@ -0,0 +1,532 @@
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 org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.util.BitSetIterator;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.search.SolrIndexSearcher;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
public abstract class SlotAcc implements Closeable {
String key; // todo...
protected final FacetContext fcontext;
public SlotAcc(FacetContext fcontext) {
this.fcontext = fcontext;
}
public void setNextReader(LeafReaderContext readerContext) throws IOException {
}
public abstract void collect(int doc, int slot) throws IOException;
public abstract int compare(int slotA, int slotB);
public abstract Object getValue(int slotNum) throws IOException;
public void setValues(SimpleOrderedMap<Object> bucket, int slotNum) throws IOException {
if (key == null) return;
bucket.add(key, getValue(slotNum));
}
public abstract void reset();
@Override
public void close() throws IOException {
}
}
// TODO: we should really have a decoupled value provider...
// This would enhance reuse and also prevent multiple lookups of same value across diff stats
abstract class FuncSlotAcc extends SlotAcc {
protected final ValueSource valueSource;
protected FunctionValues values;
public FuncSlotAcc(ValueSource values, FacetContext fcontext, int numSlots) {
super(fcontext);
this.valueSource = values;
}
@Override
public void setNextReader(LeafReaderContext readerContext) throws IOException {
values = valueSource.getValues(fcontext.qcontext, readerContext);
}
}
// have a version that counts the number of times a Slot has been hit? (for avg... what else?)
// TODO: make more sense to have func as the base class rather than double?
// double-slot-func -> func-slot -> slot -> acc
// double-slot-func -> double-slot -> slot -> acc
abstract class DoubleFuncSlotAcc extends FuncSlotAcc {
double[] result; // TODO: use DoubleArray
double initialValue;
public DoubleFuncSlotAcc(ValueSource values, FacetContext fcontext, int numSlots) {
this(values, fcontext, numSlots, 0);
}
public DoubleFuncSlotAcc(ValueSource values, FacetContext fcontext, int numSlots, double initialValue) {
super(values, fcontext, numSlots);
this.initialValue = initialValue;
result = new double[numSlots];
if (initialValue != 0) {
reset();
}
}
@Override
public int compare(int slotA, int slotB) {
return Double.compare(result[slotA], result[slotB]);
}
@Override
public Object getValue(int slot) {
return result[slot];
}
@Override
public void reset() {
for (int i=0; i<result.length; i++) {
result[i] = initialValue;
}
}
}
abstract class IntSlotAcc extends SlotAcc {
int[] result; // use LongArray32
int initialValue;
public IntSlotAcc(FacetContext fcontext, int numSlots, int initialValue) {
super(fcontext);
this.initialValue = initialValue;
result = new int[numSlots];
if (initialValue != 0) {
reset();
}
}
@Override
public int compare(int slotA, int slotB) {
return Integer.compare(result[slotA], result[slotB]);
}
@Override
public Object getValue(int slot) {
return result[slot];
}
@Override
public void reset() {
for (int i=0; i<result.length; i++) {
result[i] = initialValue;
}
}
}
class SumSlotAcc extends DoubleFuncSlotAcc {
public SumSlotAcc(ValueSource values, FacetContext fcontext, int numSlots) {
super(values, fcontext, numSlots);
}
public void collect(int doc, int slotNum) {
double val = values.doubleVal(doc); // todo: worth trying to share this value across multiple stats that need it?
result[slotNum] += val;
}
}
class SumsqSlotAcc extends DoubleFuncSlotAcc {
public SumsqSlotAcc(ValueSource values, FacetContext fcontext, int numSlots) {
super(values, fcontext, numSlots);
}
@Override
public void collect(int doc, int slotNum) {
double val = values.doubleVal(doc);
val = val * val;
result[slotNum] += val;
}
}
class MinSlotAcc extends DoubleFuncSlotAcc {
public MinSlotAcc(ValueSource values, FacetContext fcontext, int numSlots) {
super(values, fcontext, numSlots, Double.NaN);
}
@Override
public void collect(int doc, int slotNum) {
double val = values.doubleVal(doc);
if (val == 0 && !values.exists(doc)) return; // depend on fact that non existing values return 0 for func query
double currMin = result[slotNum];
if (!(val >= currMin)) { // val>=currMin will be false for staring value: val>=NaN
result[slotNum] = val;
}
}
}
class MaxSlotAcc extends DoubleFuncSlotAcc {
public MaxSlotAcc(ValueSource values, FacetContext fcontext, int numSlots) {
super(values, fcontext, numSlots, Double.NaN);
}
@Override
public void collect(int doc, int slotNum) {
double val = values.doubleVal(doc);
if (val == 0 && !values.exists(doc)) return; // depend on fact that non existing values return 0 for func query
double currMax = result[slotNum];
if (!(val <= currMax)) { // reversed order to handle NaN
result[slotNum] = val;
}
}
}
class AvgSlotAcc extends DoubleFuncSlotAcc {
int[] counts;
public AvgSlotAcc(ValueSource values, FacetContext fcontext, int numSlots) {
super(values, fcontext, numSlots);
counts = new int[numSlots];
}
@Override
public void reset() {
super.reset();
for (int i=0; i<counts.length; i++) {
counts[i] = 0;
}
}
@Override
public void collect(int doc, int slotNum) {
double val = values.doubleVal(doc); // todo: worth trying to share this value across multiple stats that need it?
result[slotNum] += val;
counts[slotNum] += 1;
}
private double avg(double tot, int count) {
return count==0 ? 0 : tot/count; // returns 0 instead of NaN.. todo - make configurable? if NaN, we need to handle comparisons though...
}
private double avg(int slot) {
return avg(result[slot], counts[slot]); // calc once and cache in result?
}
@Override
public int compare(int slotA, int slotB) {
return Double.compare(avg(slotA), avg(slotB));
}
@Override
public Object getValue(int slot) {
if (fcontext.isShard()) {
ArrayList lst = new ArrayList(2);
lst.add( counts[slot] );
lst.add( result[slot] );
return lst;
} else {
return avg(slot);
}
}
}
class CountSlotAcc extends IntSlotAcc {
public CountSlotAcc(FacetContext fcontext, int numSlots) {
super(fcontext, numSlots, 0);
}
@Override
public void collect(int doc, int slotNum) { // TODO: count arrays can use fewer bytes based on the number of docs in the base set (that's the upper bound for single valued) - look at ttf?
result[slotNum] = result[slotNum] + 1;
}
public void incrementCount(int slot, int count) {
result[slot] += count;
}
public int getCount(int slot) {
return result[slot];
}
@Override
public void reset() {
super.reset();
}
}
class SortSlotAcc extends SlotAcc {
public SortSlotAcc(FacetContext fcontext) {
super(fcontext);
}
@Override
public void collect(int doc, int slot) throws IOException {
// no-op
}
public int compare(int slotA, int slotB) {
return slotA - slotB;
}
@Override
public Object getValue(int slotNum) {
return slotNum;
}
@Override
public void reset() {
// no-op
}
}
abstract class UniqueSlotAcc extends SlotAcc {
SchemaField field;
FixedBitSet[] arr;
int currentDocBase;
int[] counts; // populated with the cardinality once
int nTerms;
public UniqueSlotAcc(FacetContext fcontext, String field, int numSlots) throws IOException {
super(fcontext);
arr = new FixedBitSet[numSlots];
this.field = fcontext.searcher.getSchema().getField(field);
}
@Override
public void reset() {
counts = null;
for (FixedBitSet bits : arr) {
if (bits == null) continue;
bits.clear(0, bits.length());
}
}
@Override
public void setNextReader(LeafReaderContext readerContext) throws IOException {
currentDocBase = readerContext.docBase;
}
@Override
public Object getValue(int slot) throws IOException {
if (fcontext.isShard()) {
return getShardValue(slot);
}
if (counts != null) { // will only be pre-populated if this was used for sorting.
return counts[slot];
}
FixedBitSet bs = arr[slot];
return bs==null ? 0 : bs.cardinality();
}
public Object getShardValue(int slot) throws IOException {
FixedBitSet ords = arr[slot];
int unique;
if (counts != null) {
unique = counts[slot];
} else {
unique = ords==null ? 0 : ords.cardinality();
}
SimpleOrderedMap map = new SimpleOrderedMap();
map.add("unique", unique);
map.add("nTerms", nTerms);
int maxExplicit=100;
// TODO: make configurable
// TODO: share values across buckets
if (unique <= maxExplicit) {
List lst = new ArrayList( Math.min(unique, maxExplicit) );
if (ords != null) {
for (int ord=-1;;) {
if (++ord >= unique) break;
ord = ords.nextSetBit(ord);
if (ord == DocIdSetIterator.NO_MORE_DOCS) break;
BytesRef val = lookupOrd(ord);
Object o = field.getType().toObject(field, val);
lst.add(o);
}
}
map.add("vals", lst);
}
return map;
}
protected abstract BytesRef lookupOrd(int ord) throws IOException;
// we only calculate all the counts when sorting by count
public void calcCounts() {
counts = new int[arr.length];
for (int i=0; i<arr.length; i++) {
FixedBitSet bs = arr[i];
counts[i] = bs == null ? 0 : bs.cardinality();
}
}
@Override
public int compare(int slotA, int slotB) {
if (counts == null) { // TODO: a more efficient way to do this? prepareSort?
calcCounts();
}
return counts[slotA] - counts[slotB];
}
}
class UniqueSinglevaluedSlotAcc extends UniqueSlotAcc {
SortedDocValues si;
public UniqueSinglevaluedSlotAcc(FacetContext fcontext, String field, int numSlots) throws IOException {
super(fcontext, field, numSlots);
SolrIndexSearcher searcher = fcontext.qcontext.searcher();
si = FieldUtil.getSortedDocValues(fcontext.qcontext, searcher.getSchema().getField(field), null);
nTerms = si.getValueCount();
}
@Override
protected BytesRef lookupOrd(int ord) {
return si.lookupOrd(ord);
}
@Override
public void collect(int doc, int slotNum) {
int ord = si.getOrd(doc + currentDocBase);
if (ord < 0) return; // -1 means missing
FixedBitSet bits = arr[slotNum];
if (bits == null) {
bits = new FixedBitSet(nTerms);
arr[slotNum] = bits;
}
bits.set(ord);
}
}
class UniqueMultivaluedSlotAcc extends UniqueSlotAcc implements UnInvertedField.Callback {
private UnInvertedField uif;
private UnInvertedField.DocToTerm docToTerm;
public UniqueMultivaluedSlotAcc(FacetContext fcontext, String field, int numSlots) throws IOException {
super(fcontext, field, numSlots);
SolrIndexSearcher searcher = fcontext.qcontext.searcher();
uif = UnInvertedField.getUnInvertedField(field, searcher);
docToTerm = uif.new DocToTerm();
fcontext.qcontext.addCloseHook(this); // TODO: find way to close accumulators instead of using close hook?
nTerms = uif.numTerms();
}
@Override
public Object getShardValue(int slot) throws IOException {
FixedBitSet ords = arr[slot];
int unique;
if (counts != null) {
unique = counts[slot];
} else {
unique = ords == null ? 0 : ords.cardinality();
}
SimpleOrderedMap map = new SimpleOrderedMap();
map.add("unique", unique);
map.add("nTerms", nTerms);
int maxExplicit=100;
// TODO: make configurable
// TODO: share values across buckets
if (unique <= maxExplicit) {
List lst = new ArrayList( Math.min(unique, maxExplicit) );
if (ords != null) {
for (int ord=-1;;) {
if (++ord >= unique) break;
ord = ords.nextSetBit(ord);
if (ord == DocIdSetIterator.NO_MORE_DOCS) break;
BytesRef val = docToTerm.lookupOrd(ord);
Object o = field.getType().toObject(field, val);
lst.add(o);
}
}
map.add("vals", lst);
}
return map;
}
@Override
protected BytesRef lookupOrd(int ord) throws IOException {
return docToTerm.lookupOrd(ord);
}
private FixedBitSet bits; // bits for the current slot, only set for the callback
@Override
public void call(int termNum) {
bits.set(termNum);
}
@Override
public void collect(int doc, int slotNum) throws IOException {
bits = arr[slotNum];
if (bits == null) {
bits = new FixedBitSet(nTerms);
arr[slotNum] = bits;
}
docToTerm.getTerms(doc + currentDocBase, this); // this will call back to our Callback.call(int termNum)
}
@Override
public void close() throws IOException {
if (docToTerm != null) {
docToTerm.close();
docToTerm = null;
}
}
}

View File

@ -0,0 +1,58 @@
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 org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.solr.search.QueryContext;
import java.io.IOException;
public abstract class StrAggValueSource extends AggValueSource {
protected String arg;
public StrAggValueSource(String name, String arg) {
super(name);
this.arg = arg;
}
public String getArg() {
return arg;
}
@Override
public boolean equals(Object o) {
if (!super.equals(o)) return false;
String otherArg = ((StrAggValueSource)o).arg;
if (arg == otherArg) return true;
return (arg != null && arg.equals(otherArg));
}
@Override
public int hashCode() {
return getClass().hashCode() + (arg == null ? 0 : arg.hashCode());
}
@Override
public String description() {
return name() + "(" + arg + ")";
}
}

View File

@ -0,0 +1,40 @@
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.queries.function.ValueSource;
public class SumAgg extends SimpleAggValueSource {
public SumAgg(ValueSource vs) {
super("sum", vs);
}
@Override
public SlotAcc createSlotAcc(FacetContext fcontext, int numDocs, int numSlots) throws IOException {
return new SumSlotAcc(getArg(), fcontext, numSlots);
}
@Override
public FacetMerger createFacetMerger(Object prototype) {
return new FacetDoubleMerger();
}
}

View File

@ -0,0 +1,38 @@
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.queries.function.ValueSource;
public class SumsqAgg extends SimpleAggValueSource {
public SumsqAgg(ValueSource vs) {
super("sumsq", vs);
}
@Override
public SlotAcc createSlotAcc(FacetContext fcontext, int numDocs, int numSlots) throws IOException {
return new SumsqSlotAcc(getArg(), fcontext, numSlots);
}
@Override
public FacetMerger createFacetMerger(Object prototype) {
return new FacetDoubleMerger();
}
}

View File

@ -0,0 +1,763 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.search.facet;
import java.io.Closeable;
import java.io.IOException;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TermRangeQuery;
import org.apache.lucene.uninverting.DocTermOrds;
import org.apache.lucene.uninverting.UninvertingReader;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.CharsRefBuilder;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.UnicodeUtil;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.FacetParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.core.SolrCore;
import org.apache.solr.handler.component.FieldFacetStats;
import org.apache.solr.handler.component.StatsField;
import org.apache.solr.handler.component.StatsValues;
import org.apache.solr.handler.component.StatsValuesFactory;
import org.apache.solr.request.SimpleFacets;
import org.apache.solr.schema.FieldType;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.schema.TrieField;
import org.apache.solr.search.BitDocSet;
import org.apache.solr.search.DocIterator;
import org.apache.solr.search.DocSet;
import org.apache.solr.search.SolrCache;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.util.LongPriorityQueue;
import org.apache.solr.util.PrimUtils;
/**
*
* Final form of the un-inverted field:
* Each document points to a list of term numbers that are contained in that document.
*
* Term numbers are in sorted order, and are encoded as variable-length deltas from the
* previous term number. Real term numbers start at 2 since 0 and 1 are reserved. A
* term number of 0 signals the end of the termNumber list.
*
* There is a single int[maxDoc()] which either contains a pointer into a byte[] for
* the termNumber lists, or directly contains the termNumber list if it fits in the 4
* bytes of an integer. If the first byte in the integer is 1, the next 3 bytes
* are a pointer into a byte[] where the termNumber list starts.
*
* There are actually 256 byte arrays, to compensate for the fact that the pointers
* into the byte arrays are only 3 bytes long. The correct byte array for a document
* is a function of its id.
*
* To save space and speed up faceting, any term that matches enough documents will
* not be un-inverted... it will be skipped while building the un-inverted field structure,
* and will use a set intersection method during faceting.
*
* To further save memory, the terms (the actual string values) are not all stored in
* memory, but a TermIndex is used to convert term numbers to term values only
* for the terms needed after faceting has completed. Only every 128th term value
* is stored, along with its corresponding term number, and this is used as an
* index to find the closest term and iterate until the desired number is hit (very
* much like Lucene's own internal term index).
*
*/
public class UnInvertedField extends DocTermOrds {
private static int TNUM_OFFSET=2;
static class TopTerm {
BytesRef term;
int termNum;
long memSize() {
return 8 + // obj header
8 + 8 +term.length + //term
4; // int
}
}
long memsz;
final AtomicLong use = new AtomicLong(); // number of uses
int[] maxTermCounts = new int[1024];
final Map<Integer,TopTerm> bigTerms = new LinkedHashMap<>();
private SolrIndexSearcher.DocsEnumState deState;
private final SolrIndexSearcher searcher;
private static UnInvertedField uifPlaceholder = new UnInvertedField();
private UnInvertedField() { // Dummy for synchronization.
super("fake", 0, 0); // cheapest initialization I can find.
searcher = null;
}
@Override
protected void visitTerm(TermsEnum te, int termNum) throws IOException {
if (termNum >= maxTermCounts.length) {
// resize by doubling - for very large number of unique terms, expanding
// by 4K and resultant GC will dominate uninvert times. Resize at end if material
int[] newMaxTermCounts = new int[maxTermCounts.length*2];
System.arraycopy(maxTermCounts, 0, newMaxTermCounts, 0, termNum);
maxTermCounts = newMaxTermCounts;
}
final BytesRef term = te.term();
if (te.docFreq() > maxTermDocFreq) {
TopTerm topTerm = new TopTerm();
topTerm.term = BytesRef.deepCopyOf(term);
topTerm.termNum = termNum;
bigTerms.put(topTerm.termNum, topTerm);
if (deState == null) {
deState = new SolrIndexSearcher.DocsEnumState();
deState.fieldName = field;
deState.liveDocs = searcher.getLeafReader().getLiveDocs();
deState.termsEnum = te; // TODO: check for MultiTermsEnum in SolrIndexSearcher could now fail?
deState.postingsEnum = postingsEnum;
deState.minSetSizeCached = maxTermDocFreq;
}
postingsEnum = deState.postingsEnum;
DocSet set = searcher.getDocSet(deState);
maxTermCounts[termNum] = set.size();
}
}
@Override
protected void setActualDocFreq(int termNum, int docFreq) {
maxTermCounts[termNum] = docFreq;
}
public long memSize() {
// can cache the mem size since it shouldn't change
if (memsz!=0) return memsz;
long sz = super.ramBytesUsed();
sz += 8*8 + 32; // local fields
sz += bigTerms.size() * 64;
for (TopTerm tt : bigTerms.values()) {
sz += tt.memSize();
}
if (maxTermCounts != null)
sz += maxTermCounts.length * 4;
if (indexedTermsArray != null) {
// assume 8 byte references?
sz += 8+8+8+8+(indexedTermsArray.length<<3)+sizeOfIndexedStrings;
}
memsz = sz;
return sz;
}
public UnInvertedField(String field, SolrIndexSearcher searcher) throws IOException {
super(field,
// threshold, over which we use set intersections instead of counting
// to (1) save memory, and (2) speed up faceting.
// Add 2 for testing purposes so that there will always be some terms under
// the threshold even when the index is very
// small.
searcher.maxDoc()/20 + 2,
DEFAULT_INDEX_INTERVAL_BITS);
//System.out.println("maxTermDocFreq=" + maxTermDocFreq + " maxDoc=" + searcher.maxDoc());
final String prefix = TrieField.getMainValuePrefix(searcher.getSchema().getFieldType(field));
this.searcher = searcher;
try {
// TODO: it's wasteful to create one of these each time
// but DocTermOrds will throw an exception if it thinks the field has doc values (which is faked by UnInvertingReader)
LeafReader r = SlowCompositeReaderWrapper.wrap(searcher.getRawReader());
uninvert(r, r.getLiveDocs(), prefix == null ? null : new BytesRef(prefix));
} catch (IllegalStateException ise) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, ise);
}
if (tnums != null) {
for(byte[] target : tnums) {
if (target != null && target.length > (1<<24)*.9) {
SolrCore.log.warn("Approaching too many values for UnInvertedField faceting on field '"+field+"' : bucket size=" + target.length);
}
}
}
// free space if outrageously wasteful (tradeoff memory/cpu)
if ((maxTermCounts.length - numTermsInField) > 1024) { // too much waste!
int[] newMaxTermCounts = new int[numTermsInField];
System.arraycopy(maxTermCounts, 0, newMaxTermCounts, 0, numTermsInField);
maxTermCounts = newMaxTermCounts;
}
SolrCore.log.info("UnInverted multi-valued field " + toString());
//System.out.println("CREATED: " + toString() + " ti.index=" + ti.index);
}
public int getNumTerms() {
return numTermsInField;
}
public class DocToTerm implements Closeable {
private final DocSet[] bigTermSets;
private final int[] bigTermNums;
private TermsEnum te;
public DocToTerm() throws IOException {
bigTermSets = new DocSet[bigTerms.size()];
bigTermNums = new int[bigTerms.size()];
int i=0;
for (TopTerm tt : bigTerms.values()) {
bigTermSets[i] = searcher.getDocSet(new TermQuery(new Term(field, tt.term)));
bigTermNums[i] = tt.termNum;
i++;
}
}
public BytesRef lookupOrd(int ord) throws IOException {
return getTermValue( getTermsEnum() , ord );
}
public TermsEnum getTermsEnum() throws IOException {
if (te == null) {
te = getOrdTermsEnum(searcher.getLeafReader());
}
return te;
}
public void getTerms(int doc, Callback target) throws IOException {
if (bigTermSets != null) {
for (int i=0; i<bigTermSets.length; i++) {
if (bigTermSets[i].exists(doc)) {
target.call( bigTermNums[i] );
}
}
}
getNonBigTerms(doc, target);
}
public void getNonBigTerms(int doc, Callback target) {
if (termInstances > 0) {
int code = index[doc];
if ((code & 0xff)==1) {
int pos = code>>>8;
int whichArray = (doc >>> 16) & 0xff;
byte[] arr = tnums[whichArray];
int tnum = 0;
for(;;) {
int delta = 0;
for(;;) {
byte b = arr[pos++];
delta = (delta << 7) | (b & 0x7f);
if ((b & 0x80) == 0) break;
}
if (delta == 0) break;
tnum += delta - TNUM_OFFSET;
target.call(tnum);
}
} else {
int tnum = 0;
int delta = 0;
for (;;) {
delta = (delta << 7) | (code & 0x7f);
if ((code & 0x80)==0) {
if (delta==0) break;
tnum += delta - TNUM_OFFSET;
target.call(tnum);
delta = 0;
}
code >>>= 8;
}
}
}
}
@Override
public void close() throws IOException {
for (DocSet set : bigTermSets) {
// set.decref(); // OFF-HEAP
}
}
}
public interface Callback {
public void call(int termNum);
}
public NamedList<Integer> getCounts(SolrIndexSearcher searcher, DocSet baseDocs, int offset, int limit, Integer mincount, boolean missing, String sort, String prefix) throws IOException {
use.incrementAndGet();
FieldType ft = searcher.getSchema().getFieldType(field);
NamedList<Integer> res = new NamedList<>(); // order is important
DocSet docs = baseDocs;
int baseSize = docs.size();
int maxDoc = searcher.maxDoc();
//System.out.println("GET COUNTS field=" + field + " baseSize=" + baseSize + " minCount=" + mincount + " maxDoc=" + maxDoc + " numTermsInField=" + numTermsInField);
if (baseSize >= mincount) {
final int[] index = this.index;
// tricky: we add more more element than we need because we will reuse this array later
// for ordering term ords before converting to term labels.
final int[] counts = new int[numTermsInField + 1];
//
// If there is prefix, find its start and end term numbers
//
int startTerm = 0;
int endTerm = numTermsInField; // one past the end
TermsEnum te = getOrdTermsEnum(searcher.getLeafReader());
if (te != null && prefix != null && prefix.length() > 0) {
final BytesRefBuilder prefixBr = new BytesRefBuilder();
prefixBr.copyChars(prefix);
if (te.seekCeil(prefixBr.get()) == TermsEnum.SeekStatus.END) {
startTerm = numTermsInField;
} else {
startTerm = (int) te.ord();
}
prefixBr.append(UnicodeUtil.BIG_TERM);
if (te.seekCeil(prefixBr.get()) == TermsEnum.SeekStatus.END) {
endTerm = numTermsInField;
} else {
endTerm = (int) te.ord();
}
}
/***********
// Alternative 2: get the docSet of the prefix (could take a while) and
// then do the intersection with the baseDocSet first.
if (prefix != null && prefix.length() > 0) {
docs = searcher.getDocSet(new ConstantScorePrefixQuery(new Term(field, ft.toInternal(prefix))), docs);
// The issue with this method are problems of returning 0 counts for terms w/o
// the prefix. We can't just filter out those terms later because it may
// mean that we didn't collect enough terms in the queue (in the sorted case).
}
***********/
boolean doNegative = baseSize > maxDoc >> 1 && termInstances > 0
&& startTerm==0 && endTerm==numTermsInField
&& docs instanceof BitDocSet;
if (doNegative) {
FixedBitSet bs = ((BitDocSet)docs).getBits().clone();
bs.flip(0, maxDoc);
// TODO: when iterator across negative elements is available, use that
// instead of creating a new bitset and inverting.
docs = new BitDocSet(bs, maxDoc - baseSize);
// simply negating will mean that we have deleted docs in the set.
// that should be OK, as their entries in our table should be empty.
//System.out.println(" NEG");
}
// For the biggest terms, do straight set intersections
for (TopTerm tt : bigTerms.values()) {
//System.out.println(" do big termNum=" + tt.termNum + " term=" + tt.term.utf8ToString());
// TODO: counts could be deferred if sorted==false
if (tt.termNum >= startTerm && tt.termNum < endTerm) {
counts[tt.termNum] = searcher.numDocs(new TermQuery(new Term(field, tt.term)), docs);
//System.out.println(" count=" + counts[tt.termNum]);
} else {
//System.out.println("SKIP term=" + tt.termNum);
}
}
// TODO: we could short-circuit counting altogether for sorted faceting
// where we already have enough terms from the bigTerms
// TODO: we could shrink the size of the collection array, and
// additionally break when the termNumber got above endTerm, but
// it would require two extra conditionals in the inner loop (although
// they would be predictable for the non-prefix case).
// Perhaps a different copy of the code would be warranted.
if (termInstances > 0) {
DocIterator iter = docs.iterator();
while (iter.hasNext()) {
int doc = iter.nextDoc();
//System.out.println("iter doc=" + doc);
int code = index[doc];
if ((code & 0xff)==1) {
//System.out.println(" ptr");
int pos = code>>>8;
int whichArray = (doc >>> 16) & 0xff;
byte[] arr = tnums[whichArray];
int tnum = 0;
for(;;) {
int delta = 0;
for(;;) {
byte b = arr[pos++];
delta = (delta << 7) | (b & 0x7f);
if ((b & 0x80) == 0) break;
}
if (delta == 0) break;
tnum += delta - TNUM_OFFSET;
//System.out.println(" tnum=" + tnum);
counts[tnum]++;
}
} else {
//System.out.println(" inlined");
int tnum = 0;
int delta = 0;
for (;;) {
delta = (delta << 7) | (code & 0x7f);
if ((code & 0x80)==0) {
if (delta==0) break;
tnum += delta - TNUM_OFFSET;
//System.out.println(" tnum=" + tnum);
counts[tnum]++;
delta = 0;
}
code >>>= 8;
}
}
}
}
final CharsRefBuilder charsRef = new CharsRefBuilder();
int off=offset;
int lim=limit>=0 ? limit : Integer.MAX_VALUE;
if (sort.equals(FacetParams.FACET_SORT_COUNT) || sort.equals(FacetParams.FACET_SORT_COUNT_LEGACY)) {
int maxsize = limit>0 ? offset+limit : Integer.MAX_VALUE-1;
maxsize = Math.min(maxsize, numTermsInField);
LongPriorityQueue queue = new LongPriorityQueue(Math.min(maxsize,1000), maxsize, Long.MIN_VALUE);
int min=mincount-1; // the smallest value in the top 'N' values
//System.out.println("START=" + startTerm + " END=" + endTerm);
for (int i=startTerm; i<endTerm; i++) {
int c = doNegative ? maxTermCounts[i] - counts[i] : counts[i];
if (c>min) {
// NOTE: we use c>min rather than c>=min as an optimization because we are going in
// index order, so we already know that the keys are ordered. This can be very
// important if a lot of the counts are repeated (like zero counts would be).
// smaller term numbers sort higher, so subtract the term number instead
long pair = (((long)c)<<32) + (Integer.MAX_VALUE - i);
boolean displaced = queue.insert(pair);
if (displaced) min=(int)(queue.top() >>> 32);
}
}
// now select the right page from the results
// 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;
// the start and end indexes of our list "sorted" (starting with the highest value)
int sortedIdxStart = queue.size() - (collectCount - 1);
int sortedIdxEnd = queue.size() + 1;
final long[] sorted = queue.sort(collectCount);
final int[] indirect = counts; // reuse the counts array for the index into the tnums array
assert indirect.length >= sortedIdxEnd;
for (int i=sortedIdxStart; i<sortedIdxEnd; i++) {
long pair = sorted[i];
int c = (int)(pair >>> 32);
int tnum = Integer.MAX_VALUE - (int)pair;
indirect[i] = i; // store the index for indirect sorting
sorted[i] = tnum; // reuse the "sorted" array to store the term numbers for indirect sorting
// add a null label for now... we'll fill it in later.
res.add(null, c);
}
// now sort the indexes by the term numbers
PrimUtils.sort(sortedIdxStart, sortedIdxEnd, indirect, new PrimUtils.IntComparator() {
@Override
public int compare(int a, int b) {
return (int)sorted[a] - (int)sorted[b];
}
@Override
public boolean lessThan(int a, int b) {
return sorted[a] < sorted[b];
}
@Override
public boolean equals(int a, int b) {
return sorted[a] == sorted[b];
}
});
// convert the term numbers to term values and set
// as the label
//System.out.println("sortStart=" + sortedIdxStart + " end=" + sortedIdxEnd);
for (int i=sortedIdxStart; i<sortedIdxEnd; i++) {
int idx = indirect[i];
int tnum = (int)sorted[idx];
final String label = getReadableValue(getTermValue(te, tnum), ft, charsRef);
//System.out.println(" label=" + label);
res.setName(idx - sortedIdxStart, label);
}
} else {
// add results in index order
int i=startTerm;
if (mincount<=0) {
// if mincount<=0, then we won't discard any terms and we know exactly
// where to start.
i=startTerm+off;
off=0;
}
for (; i<endTerm; i++) {
int c = doNegative ? maxTermCounts[i] - counts[i] : counts[i];
if (c<mincount || --off>=0) continue;
if (--lim<0) break;
final String label = getReadableValue(getTermValue(te, i), ft, charsRef);
res.add(label, c);
}
}
}
if (missing) {
// TODO: a faster solution for this?
res.add(null, SimpleFacets.getFieldMissingCount(searcher, baseDocs, field));
}
//System.out.println(" res=" + res);
return res;
}
// called from FieldFacetProcessor
// TODO: do a callback version that can be specialized!
public void collectDocs(FacetFieldProcessorUIF processor) throws IOException {
use.incrementAndGet();
DocSet docs = processor.fcontext.base;
int startTermIndex = processor.startTermIndex;
int endTermIndex = processor.endTermIndex;
int nTerms = processor.nTerms;
int uniqueTerms = 0;
for (TopTerm tt : bigTerms.values()) {
if (tt.termNum >= startTermIndex && tt.termNum < endTermIndex) {
// handle the biggest terms
try ( DocSet intersection = searcher.getDocSet(new TermQuery(new Term(field, tt.term)), docs); )
{
int collected = processor.collect(intersection, tt.termNum - startTermIndex);
processor.countAcc.incrementCount(tt.termNum - startTermIndex, collected);
if (collected > 0) {
uniqueTerms++;
}
}
}
}
if (termInstances > 0) {
final List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
final Iterator<LeafReaderContext> ctxIt = leaves.iterator();
LeafReaderContext ctx = null;
int segBase = 0;
int segMax;
int adjustedMax = 0;
// TODO: handle facet.prefix here!!!
DocIterator iter = docs.iterator();
while (iter.hasNext()) {
int doc = iter.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;
processor.setNextReader(ctx);
}
int segDoc = doc - segBase;
int code = index[doc];
if ((code & 0xff)==1) {
int pos = code>>>8;
int whichArray = (doc >>> 16) & 0xff;
byte[] arr = tnums[whichArray];
int tnum = 0;
for(;;) {
int delta = 0;
for(;;) {
byte b = arr[pos++];
delta = (delta << 7) | (b & 0x7f);
if ((b & 0x80) == 0) break;
}
if (delta == 0) break;
tnum += delta - TNUM_OFFSET;
int arrIdx = tnum - startTermIndex;
if (arrIdx < 0) continue;
if (arrIdx >= nTerms) break;
processor.countAcc.incrementCount(arrIdx, 1);
processor.collect(segDoc, arrIdx);
if (processor.allBucketsSlot >= 0) {
processor.countAcc.incrementCount(processor.allBucketsSlot, 1);
processor.collect(segDoc, processor.allBucketsSlot);
}
}
} else {
int tnum = 0;
int delta = 0;
for (;;) {
delta = (delta << 7) | (code & 0x7f);
if ((code & 0x80)==0) {
if (delta==0) break;
tnum += delta - TNUM_OFFSET;
int arrIdx = tnum - startTermIndex;
if (arrIdx < 0) continue;
if (arrIdx >= nTerms) break;
processor.countAcc.incrementCount(arrIdx, 1);
processor.collect(segDoc, arrIdx);
if (processor.allBucketsSlot >= 0) {
processor.countAcc.incrementCount(processor.allBucketsSlot, 1);
processor.collect(segDoc, processor.allBucketsSlot);
}
delta = 0;
}
code >>>= 8;
}
}
}
}
}
String getReadableValue(BytesRef termval, FieldType ft, CharsRefBuilder charsRef) {
return ft.indexedToReadable(termval, charsRef).toString();
}
/** may return a reused BytesRef */
BytesRef getTermValue(TermsEnum te, int termNum) throws IOException {
//System.out.println("getTermValue termNum=" + termNum + " this=" + this + " numTerms=" + numTermsInField);
if (bigTerms.size() > 0) {
// see if the term is one of our big terms.
TopTerm tt = bigTerms.get(termNum);
if (tt != null) {
//System.out.println(" return big " + tt.term);
return tt.term;
}
}
return lookupTerm(te, termNum);
}
@Override
public String toString() {
final long indexSize = indexedTermsArray == null ? 0 : (8+8+8+8+(indexedTermsArray.length<<3)+sizeOfIndexedStrings); // assume 8 byte references?
return "{field=" + field
+ ",memSize="+memSize()
+ ",tindexSize="+indexSize
+ ",time="+total_time
+ ",phase1="+phase1_time
+ ",nTerms="+numTermsInField
+ ",bigTerms="+bigTerms.size()
+ ",termInstances="+termInstances
+ ",uses="+use.get()
+ "}";
}
//////////////////////////////////////////////////////////////////
//////////////////////////// caching /////////////////////////////
//////////////////////////////////////////////////////////////////
public static UnInvertedField getUnInvertedField(String field, SolrIndexSearcher searcher) throws IOException {
SolrCache<String,UnInvertedField> cache = searcher.getFieldValueCache();
if (cache == null) {
return new UnInvertedField(field, searcher);
}
UnInvertedField uif = null;
Boolean doWait = false;
synchronized (cache) {
uif = cache.get(field);
if (uif == null) {
/**
* We use this place holder object to pull the UninvertedField construction out of the sync
* so that if many fields are accessed in a short time, the UninvertedField can be
* built for these fields in parallel rather than sequentially.
*/
cache.put(field, uifPlaceholder);
} else {
if (uif != uifPlaceholder) {
return uif;
}
doWait = true; // Someone else has put the place holder in, wait for that to complete.
}
}
while (doWait) {
try {
synchronized (cache) {
uif = cache.get(field); // Should at least return the placeholder, NPE if not is OK.
if (uif != uifPlaceholder) { // OK, another thread put this in the cache we should be good.
return uif;
}
cache.wait();
}
} catch (InterruptedException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Thread interrupted in getUninvertedField.");
}
}
uif = new UnInvertedField(field, searcher);
synchronized (cache) {
cache.put(field, uif); // Note, this cleverly replaces the placeholder.
cache.notifyAll();
}
return uif;
}
}

View File

@ -0,0 +1,82 @@
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.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.solr.common.util.SimpleOrderedMap;
public class UniqueAgg extends StrAggValueSource {
public static String UNIQUE = "unique";
// internal constants used for aggregating values from multiple shards
static String VALS = "vals";
public UniqueAgg(String field) {
super(UNIQUE, field);
}
@Override
public SlotAcc createSlotAcc(FacetContext fcontext, int numDocs, int numSlots) throws IOException {
if (fcontext.qcontext.searcher().getSchema().getField(getArg()).multiValued())
return new UniqueMultivaluedSlotAcc(fcontext, getArg(), numSlots);
else
return new UniqueSinglevaluedSlotAcc(fcontext, getArg(), numSlots);
}
@Override
public FacetMerger createFacetMerger(Object prototype) {
return new FacetMerger() {
long sumUnique;
Set<Object> values;
int shardsMissing;
long shardsMissingSum;
long shardsMissingMax;
@Override
public void merge(Object facetResult) {
SimpleOrderedMap map = (SimpleOrderedMap)facetResult;
long unique = ((Number)map.get("unique")).longValue();
sumUnique += unique;
List vals = (List)map.get("vals");
if (vals != null) {
if (values == null) {
values = new HashSet<>(vals.size()*4);
}
values.addAll(vals);
} else {
shardsMissing++;
shardsMissingSum += unique;
shardsMissingMax = Math.max(shardsMissingMax, unique);
}
// TODO: somehow get & use the count in the bucket?
}
@Override
public Object getMergedResult() {
long exactCount = values == null ? 0 : values.size();
return exactCount + shardsMissingSum;
}
};
}
}

View File

@ -0,0 +1,28 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
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.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
<p>
APIs and classes for the JSON Facet API.
This is currently experimental!
</p>
</body>
</html>

View File

@ -58,6 +58,8 @@ import org.apache.solr.handler.component.ShardRequest;
import org.apache.solr.highlight.SolrHighlighter; import org.apache.solr.highlight.SolrHighlighter;
import org.apache.solr.parser.QueryParser; import org.apache.solr.parser.QueryParser;
import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrRequestHandler;
import org.apache.solr.request.json.RequestUtil;
import org.apache.solr.response.SolrQueryResponse; import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.schema.IndexSchema; import org.apache.solr.schema.IndexSchema;
import org.apache.solr.schema.SchemaField; import org.apache.solr.schema.SchemaField;
@ -88,7 +90,7 @@ import org.apache.solr.search.SyntaxError;
* default parameter settings. * default parameter settings.
*/ */
public class SolrPluginUtils { public class SolrPluginUtils {
/** /**
* Map containing all the possible purposes codes of a request as key and * Map containing all the possible purposes codes of a request as key and
@ -130,7 +132,11 @@ public class SolrPluginUtils {
* @param appends values to be appended to those from the request (or defaults) when dealing with multi-val params, or treated as another layer of defaults for singl-val params. * @param appends values to be appended to those from the request (or defaults) when dealing with multi-val params, or treated as another layer of defaults for singl-val params.
* @param invariants values which will be used instead of any request, or default values, regardless of context. * @param invariants values which will be used instead of any request, or default values, regardless of context.
*/ */
public static void setDefaults(SolrQueryRequest req, SolrParams defaults, public static void setDefaults(SolrQueryRequest req, SolrParams defaults, SolrParams appends, SolrParams invariants) {
setDefaults(null, req, defaults, appends, invariants);
}
public static void setDefaults(SolrRequestHandler handler, SolrQueryRequest req, SolrParams defaults,
SolrParams appends, SolrParams invariants) { SolrParams appends, SolrParams invariants) {
List<String> paramNames = null; List<String> paramNames = null;
@ -156,12 +162,7 @@ public class SolrPluginUtils {
} }
} }
SolrParams p = req.getParams(); RequestUtil.processParams(handler, req, defaults, appends, invariants);
p = SolrParams.wrapDefaults(p, defaults);
p = SolrParams.wrapAppended(p, appends);
p = SolrParams.wrapDefaults(invariants, p);
req.setParams(p);
} }

View File

@ -0,0 +1,751 @@
<?xml version="1.0" encoding="UTF-8" ?>
<!--
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.
-->
<!--
This is the Solr schema file. This file should be named "schema.xml" and
should be in the conf directory under the solr home
(i.e. ./solr/conf/schema.xml by default)
or located where the classloader for the Solr webapp can find it.
This example schema is the recommended starting point for users.
It should be kept correct and concise, usable out-of-the-box.
For more information, on how to customize this file, please see
http://wiki.apache.org/solr/SchemaXml
PERFORMANCE NOTE: this schema includes many optional features and should not
be used for benchmarking. To improve performance one could
- set stored="false" for all fields possible (esp large fields) when you
only need to search on the field but don't need to return the original
value.
- set indexed="false" if you don't need to search on the field, but only
return the field as a result of searching on other indexed fields.
- remove all unneeded copyField statements
- for best index size and searching performance, set "index" to false
for all general text fields, use copyField to copy them to the
catchall "text" field, and use that for searching.
- For maximum indexing performance, use the StreamingUpdateSolrServer
java client.
- Remember to run the JVM in server mode, and use a higher logging level
that avoids logging every request
-->
<schema name="example" version="1.5">
<!-- attribute "name" is the name of this schema and is only used for display purposes.
version="x.y" is Solr's version number for the schema syntax and
semantics. It should not normally be changed by applications.
1.0: multiValued attribute did not exist, all fields are multiValued
by nature
1.1: multiValued attribute introduced, false by default
1.2: omitTermFreqAndPositions attribute introduced, true by default
except for text fields.
1.3: removed optional field compress feature
1.4: autoGeneratePhraseQueries attribute introduced to drive QueryParser
behavior when a single string produces multiple tokens. Defaults
to off for version >= 1.4
1.5: omitNorms defaults to true for primitive field types
(int, float, boolean, string...)
-->
<fields>
<!-- Valid attributes for fields:
name: mandatory - the name for the field
type: mandatory - the name of a field type from the
<types> fieldType section
indexed: true if this field should be indexed (searchable or sortable)
stored: true if this field should be retrievable
docValues: true if this field should have doc values. Doc values are
useful for faceting, grouping, sorting and function queries. Although not
required, doc values will make the index faster to load, more
NRT-friendly and more memory-efficient. They however come with some
limitations: they are currently only supported by StrField, UUIDField
and all Trie*Fields, and depending on the field type, they might
require the field to be single-valued, be required or have a default
value (check the documentation of the field type you're interested in
for more information)
multiValued: true if this field may contain multiple values per document
omitNorms: (expert) set to true to omit the norms associated with
this field (this disables length normalization and index-time
boosting for the field, and saves some memory). Only full-text
fields or fields that need an index-time boost need norms.
Norms are omitted for primitive (non-analyzed) types by default.
termVectors: [false] set to true to store the term vector for a
given field.
When using MoreLikeThis, fields used for similarity should be
stored for best performance.
termPositions: Store position information with the term vector.
This will increase storage costs.
termOffsets: Store offset information with the term vector. This
will increase storage costs.
required: The field is required. It will throw an error if the
value does not exist
default: a value that should be used if no value is specified
when adding a document.
-->
<!-- field names should consist of alphanumeric or underscore characters only and
not start with a digit. This is not currently strictly enforced,
but other field names will not have first class support from all components
and back compatibility is not guaranteed. Names with both leading and
trailing underscores (e.g. _version_) are reserved.
-->
<!-- If you remove this field, you must _also_ disable the update log in solrconfig.xml
or Solr won't start. _version_ and update log are required for SolrCloud
-->
<field name="_version_" type="long" indexed="true" stored="true"/>
<!-- points to the root document of a block of nested documents. Required for nested
document support, may be removed otherwise
-->
<field name="_root_" type="string" indexed="true" stored="false"/>
<!-- Only remove the "id" field if you have a very good reason to. While not strictly
required, it is highly recommended. A <uniqueKey> is present in almost all Solr
installations. See the <uniqueKey> declaration below where <uniqueKey> is set to "id".
-->
<field name="id" type="string" indexed="true" stored="true" required="true" multiValued="false" />
<field name="sku" type="text_en_splitting_tight" indexed="true" stored="true" omitNorms="true"/>
<field name="name" type="text_general" indexed="true" stored="true"/>
<field name="manu" type="text_general" indexed="true" stored="true" omitNorms="true"/>
<field name="cat" type="string" indexed="true" stored="true" multiValued="true"/>
<field name="features" type="text_general" indexed="true" stored="true" multiValued="true"/>
<field name="includes" type="text_general" indexed="true" stored="true" termVectors="true" termPositions="true" termOffsets="true" />
<field name="weight" type="float" indexed="true" stored="true"/>
<field name="price" type="float" indexed="true" stored="true"/>
<field name="popularity" type="int" indexed="true" stored="true" />
<field name="inStock" type="boolean" indexed="true" stored="true" />
<field name="store" type="location" indexed="true" stored="true"/>
<!-- Common metadata fields, named specifically to match up with
SolrCell metadata when parsing rich documents such as Word, PDF.
Some fields are multiValued only because Tika currently may return
multiple values for them. Some metadata is parsed from the documents,
but there are some which come from the client context:
"content_type": From the HTTP headers of incoming stream
"resourcename": From SolrCell request param resource.name
-->
<field name="title" type="text_general" indexed="true" stored="true" multiValued="true"/>
<field name="subject" type="text_general" indexed="true" stored="true"/>
<field name="description" type="text_general" indexed="true" stored="true"/>
<field name="comments" type="text_general" indexed="true" stored="true"/>
<field name="author" type="text_general" indexed="true" stored="true"/>
<field name="keywords" type="text_general" indexed="true" stored="true"/>
<field name="category" type="text_general" indexed="true" stored="true"/>
<field name="resourcename" type="text_general" indexed="true" stored="true"/>
<field name="url" type="text_general" indexed="true" stored="true"/>
<field name="content_type" type="string" indexed="true" stored="true" multiValued="true"/>
<field name="last_modified" type="date" indexed="true" stored="true"/>
<field name="links" type="string" indexed="true" stored="true" multiValued="true"/>
<!-- Main body of document extracted by SolrCell.
NOTE: This field is not indexed by default, since it is also copied to "text"
using copyField below. This is to save space. Use this field for returning and
highlighting document content. Use the "text" field to search the content. -->
<field name="content" type="text_general" indexed="false" stored="true" multiValued="true"/>
<!-- catchall field, containing all other searchable text fields (implemented
via copyField further on in this schema -->
<field name="text" type="text_general" indexed="true" stored="false" multiValued="true"/>
<!-- catchall text field that indexes tokens both normally and in reverse for efficient
leading wildcard queries. -->
<field name="text_rev" type="text_general_rev" indexed="true" stored="false" multiValued="true"/>
<!-- non-tokenized version of manufacturer to make it easier to sort or group
results by manufacturer. copied from "manu" via copyField -->
<field name="manu_exact" type="string" indexed="true" stored="false"/>
<field name="payloads" type="payloads" indexed="true" stored="true"/>
<!--
Some fields such as popularity and manu_exact could be modified to
leverage doc values:
<field name="popularity" type="int" indexed="true" stored="true" docValues="true" />
<field name="manu_exact" type="string" indexed="false" stored="false" docValues="true" />
<field name="cat" type="string" indexed="true" stored="true" docValues="true" multiValued="true"/>
Although it would make indexing slightly slower and the index bigger, it
would also make the index faster to load, more memory-efficient and more
NRT-friendly.
-->
<!-- Dynamic field definitions allow using convention over configuration
for fields via the specification of patterns to match field names.
EXAMPLE: name="*_i" will match any field ending in _i (like myid_i, z_i)
RESTRICTION: the glob-like pattern in the name attribute must have
a "*" only at the start or the end. -->
<dynamicField name="*_i" type="int" indexed="true" stored="true"/>
<dynamicField name="*_is" type="int" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_id" type="int" indexed="true" stored="true" docValues="true" />
<dynamicField name="*_ids" type="int" indexed="true" stored="true" multiValued="true" docValues="true" />
<dynamicField name="*_s" type="string" indexed="true" stored="true" />
<dynamicField name="*_s1" type="string" indexed="true" stored="true" />
<dynamicField name="*_ss" type="string" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_sd" type="string" indexed="true" stored="true" docValues="true" />
<dynamicField name="*_sds" type="string" indexed="true" stored="true" multiValued="true" docValues="true" />
<dynamicField name="*_l" type="long" indexed="true" stored="true"/>
<dynamicField name="*_ls" type="long" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_ld" type="long" indexed="true" stored="true" docValues="true" />
<dynamicField name="*_lds" type="long" indexed="true" stored="true" multiValued="true" docValues="true" />
<dynamicField name="*_f" type="float" indexed="true" stored="true"/>
<dynamicField name="*_fs" type="float" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_fd" type="float" indexed="true" stored="true" docValues="true" />
<dynamicField name="*_fds" type="float" indexed="true" stored="true" multiValued="true" docValues="true" />
<dynamicField name="*_d" type="double" indexed="true" stored="true"/>
<dynamicField name="*_ds" type="double" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_dd" type="double" indexed="true" stored="true" docValues="true" />
<dynamicField name="*_dds" type="double" indexed="true" stored="true" multiValued="true" docValues="true" />
<dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>
<dynamicField name="*_bs" type="boolean" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_t" type="text_general" indexed="true" stored="true"/>
<dynamicField name="*_txt" type="text_general" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_en" type="text_en" indexed="true" stored="true" multiValued="true"/>
<!-- Type used to index the lat and lon components for the "location" FieldType -->
<dynamicField name="*_coordinate" type="tdouble" indexed="true" stored="false" />
<dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
<dynamicField name="*_dts" type="date" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_p" type="location" indexed="true" stored="true"/>
<!-- some trie-coded dynamic fields for faster range queries -->
<dynamicField name="*_ti" type="tint" indexed="true" stored="true"/>
<dynamicField name="*_tl" type="tlong" indexed="true" stored="true"/>
<dynamicField name="*_tf" type="tfloat" indexed="true" stored="true"/>
<dynamicField name="*_td" type="tdouble" indexed="true" stored="true"/>
<dynamicField name="*_tdt" type="tdate" indexed="true" stored="true"/>
<dynamicField name="*_c" type="currency" indexed="true" stored="true"/>
<dynamicField name="ignored_*" type="ignored" multiValued="true"/>
<dynamicField name="attr_*" type="text_general" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="random_*" type="random" />
<!-- uncomment the following to ignore any fields that don't already match an existing
field name or dynamic field, rather than reporting them as an error.
alternately, change the type="ignored" to some other type e.g. "text" if you want
unknown fields indexed and/or stored by default -->
<!--dynamicField name="*" type="ignored" multiValued="true" /-->
<!-- needed by dedup config -->
<dynamicField name="*_sS" type="string" indexed="false" stored="true"/>
<field name="signatureField" type="string" indexed="true" stored="false"/>
</fields>
<!-- Field to use to determine and enforce document uniqueness.
Unless this field is marked with required="false", it will be a required field
-->
<uniqueKey>id</uniqueKey>
<!-- DEPRECATED: The defaultSearchField is consulted by various query parsers when
parsing a query string that isn't explicit about the field. Machine (non-user)
generated queries are best made explicit, or they can use the "df" request parameter
which takes precedence over this.
Note: Un-commenting defaultSearchField will be insufficient if your request handler
in solrconfig.xml defines "df", which takes precedence. That would need to be removed.
<defaultSearchField>text</defaultSearchField> -->
<!-- DEPRECATED: The defaultOperator (AND|OR) is consulted by various query parsers
when parsing a query string to determine if a clause of the query should be marked as
required or optional, assuming the clause isn't already marked by some operator.
The default is OR, which is generally assumed so it is not a good idea to change it
globally here. The "q.op" request parameter takes precedence over this.
<solrQueryParser defaultOperator="OR"/> -->
<!-- copyField commands copy one field to another at the time a document
is added to the index. It's used either to index the same field differently,
or to add multiple fields to the same field for easier/faster searching. -->
<copyField source="cat" dest="text"/>
<copyField source="name" dest="text"/>
<copyField source="manu" dest="text"/>
<copyField source="features" dest="text"/>
<copyField source="includes" dest="text"/>
<copyField source="manu" dest="manu_exact"/>
<!-- Copy the price into a currency enabled field (default USD) -->
<copyField source="price" dest="price_c"/>
<!-- Text fields from SolrCell to search by default in our catch-all field -->
<copyField source="title" dest="text"/>
<copyField source="author" dest="text"/>
<copyField source="description" dest="text"/>
<copyField source="keywords" dest="text"/>
<copyField source="content" dest="text"/>
<copyField source="content_type" dest="text"/>
<copyField source="resourcename" dest="text"/>
<copyField source="url" dest="text"/>
<!-- Create a string version of author for faceting -->
<copyField source="author" dest="author_s"/>
<!-- Above, multiple source fields are copied to the [text] field.
Another way to map multiple source fields to the same
destination field is to use the dynamic field syntax.
copyField also supports a maxChars to copy setting. -->
<!-- <copyField source="*_t" dest="text" maxChars="3000"/> -->
<!-- copy name to alphaNameSort, a field designed for sorting by name -->
<!-- <copyField source="name" dest="alphaNameSort"/> -->
<types>
<!-- field type definitions. The "name" attribute is
just a label to be used by field definitions. The "class"
attribute and any other attributes determine the real
behavior of the fieldType.
Class names starting with "solr" refer to java classes in a
standard package such as org.apache.solr.analysis
-->
<!-- The StrField type is not analyzed, but indexed/stored verbatim.
It supports doc values but in that case the field needs to be
single-valued and either required or have a default value.
-->
<fieldType name="string" class="solr.StrField" sortMissingLast="true" />
<!-- boolean type: "true" or "false" -->
<fieldType name="boolean" class="solr.BoolField" sortMissingLast="true"/>
<!-- sortMissingLast and sortMissingFirst attributes are optional attributes are
currently supported on types that are sorted internally as strings
and on numeric types.
This includes "string","boolean", and, as of 3.5 (and 4.x),
int, float, long, date, double, including the "Trie" variants.
- If sortMissingLast="true", then a sort on this field will cause documents
without the field to come after documents with the field,
regardless of the requested sort order (asc or desc).
- If sortMissingFirst="true", then a sort on this field will cause documents
without the field to come before documents with the field,
regardless of the requested sort order.
- If sortMissingLast="false" and sortMissingFirst="false" (the default),
then default lucene sorting will be used which places docs without the
field first in an ascending sort and last in a descending sort.
-->
<!--
Default numeric field types. For faster range queries, consider the tint/tfloat/tlong/tdouble types.
These fields support doc values, but they require the field to be
single-valued and either be required or have a default value.
-->
<fieldType name="int" class="solr.TrieIntField" precisionStep="0" positionIncrementGap="0"/>
<fieldType name="float" class="solr.TrieFloatField" precisionStep="0" positionIncrementGap="0"/>
<fieldType name="long" class="solr.TrieLongField" precisionStep="0" positionIncrementGap="0"/>
<fieldType name="double" class="solr.TrieDoubleField" precisionStep="0" positionIncrementGap="0"/>
<!--
Numeric field types that index each value at various levels of precision
to accelerate range queries when the number of values between the range
endpoints is large. See the javadoc for NumericRangeQuery for internal
implementation details.
Smaller precisionStep values (specified in bits) will lead to more tokens
indexed per value, slightly larger index size, and faster range queries.
A precisionStep of 0 disables indexing at different precision levels.
-->
<fieldType name="tint" class="solr.TrieIntField" precisionStep="8" positionIncrementGap="0"/>
<fieldType name="tfloat" class="solr.TrieFloatField" precisionStep="8" positionIncrementGap="0"/>
<fieldType name="tlong" class="solr.TrieLongField" precisionStep="8" positionIncrementGap="0"/>
<fieldType name="tdouble" class="solr.TrieDoubleField" precisionStep="8" positionIncrementGap="0"/>
<!-- The format for this date field is of the form 1995-12-31T23:59:59Z, and
is a more restricted form of the canonical representation of dateTime
http://www.w3.org/TR/xmlschema-2/#dateTime
The trailing "Z" designates UTC time and is mandatory.
Optional fractional seconds are allowed: 1995-12-31T23:59:59.999Z
All other components are mandatory.
Expressions can also be used to denote calculations that should be
performed relative to "NOW" to determine the value, ie...
NOW/HOUR
... Round to the start of the current hour
NOW-1DAY
... Exactly 1 day prior to now
NOW/DAY+6MONTHS+3DAYS
... 6 months and 3 days in the future from the start of
the current day
Consult the DateField javadocs for more information.
Note: For faster range queries, consider the tdate type
-->
<fieldType name="date" class="solr.TrieDateField" precisionStep="0" positionIncrementGap="0"/>
<!-- A Trie based date field for faster date range queries and date faceting. -->
<fieldType name="tdate" class="solr.TrieDateField" precisionStep="6" positionIncrementGap="0"/>
<!--Binary data type. The data should be sent/retrieved in as Base64 encoded Strings -->
<fieldtype name="binary" class="solr.BinaryField"/>
<!-- The "RandomSortField" is not used to store or search any
data. You can declare fields of this type it in your schema
to generate pseudo-random orderings of your docs for sorting
or function purposes. The ordering is generated based on the field
name and the version of the index. As long as the index version
remains unchanged, and the same field name is reused,
the ordering of the docs will be consistent.
If you want different psuedo-random orderings of documents,
for the same version of the index, use a dynamicField and
change the field name in the request.
-->
<fieldType name="random" class="solr.RandomSortField" indexed="true" />
<!-- solr.TextField allows the specification of custom text analyzers
specified as a tokenizer and a list of token filters. Different
analyzers may be specified for indexing and querying.
The optional positionIncrementGap puts space between multiple fields of
this type on the same document, with the purpose of preventing false phrase
matching across fields.
For more info on customizing your analyzer chain, please see
http://wiki.apache.org/solr/AnalyzersTokenizersTokenFilters
-->
<!-- One can also specify an existing Analyzer class that has a
default constructor via the class attribute on the analyzer element.
Example:
<fieldType name="text_greek" class="solr.TextField">
<analyzer class="org.apache.lucene.analysis.el.GreekAnalyzer"/>
</fieldType>
-->
<!-- A text field that only splits on whitespace for exact matching of words -->
<fieldType name="text_ws" class="solr.TextField" positionIncrementGap="100">
<analyzer>
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
</analyzer>
</fieldType>
<!-- A general text field that has reasonable, generic
cross-language defaults: it tokenizes with StandardTokenizer,
removes stop words from case-insensitive "stopwords.txt"
(empty by default), and down cases. At query time only, it
also applies synonyms. -->
<fieldType name="text_general" class="solr.TextField" positionIncrementGap="100">
<analyzer type="index">
<tokenizer class="solr.StandardTokenizerFactory"/>
<filter class="solr.StopFilterFactory" ignoreCase="true" words="stopwords.txt" />
<!-- in this example, we will only use synonyms at query time
<filter class="solr.SynonymFilterFactory" synonyms="index_synonyms.txt" ignoreCase="true" expand="false"/>
-->
<filter class="solr.LowerCaseFilterFactory"/>
</analyzer>
<analyzer type="query">
<tokenizer class="solr.StandardTokenizerFactory"/>
<filter class="solr.StopFilterFactory" ignoreCase="true" words="stopwords.txt" />
<filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt" ignoreCase="true" expand="true"/>
<filter class="solr.LowerCaseFilterFactory"/>
</analyzer>
</fieldType>
<!-- A text field with defaults appropriate for English: it
tokenizes with StandardTokenizer, removes English stop words
(stopwords.txt), down cases, protects words from protwords.txt, and
finally applies Porter's stemming. The query time analyzer
also applies synonyms from synonyms.txt. -->
<fieldType name="text_en" class="solr.TextField" positionIncrementGap="100">
<analyzer type="index">
<tokenizer class="solr.StandardTokenizerFactory"/>
<!-- in this example, we will only use synonyms at query time
<filter class="solr.SynonymFilterFactory" synonyms="index_synonyms.txt" ignoreCase="true" expand="false"/>
-->
<!-- Case insensitive stop word removal.
-->
<filter class="solr.StopFilterFactory"
ignoreCase="true"
words="stopwords.txt"
/>
<filter class="solr.LowerCaseFilterFactory"/>
<filter class="solr.EnglishPossessiveFilterFactory"/>
<filter class="solr.KeywordMarkerFilterFactory" protected="protwords.txt"/>
<!-- Optionally you may want to use this less aggressive stemmer instead of PorterStemFilterFactory:
<filter class="solr.EnglishMinimalStemFilterFactory"/>
-->
<filter class="solr.PorterStemFilterFactory"/>
</analyzer>
<analyzer type="query">
<tokenizer class="solr.StandardTokenizerFactory"/>
<filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt" ignoreCase="true" expand="true"/>
<filter class="solr.StopFilterFactory"
ignoreCase="true"
words="stopwords.txt"
/>
<filter class="solr.LowerCaseFilterFactory"/>
<filter class="solr.EnglishPossessiveFilterFactory"/>
<filter class="solr.KeywordMarkerFilterFactory" protected="protwords.txt"/>
<!-- Optionally you may want to use this less aggressive stemmer instead of PorterStemFilterFactory:
<filter class="solr.EnglishMinimalStemFilterFactory"/>
-->
<filter class="solr.PorterStemFilterFactory"/>
</analyzer>
</fieldType>
<!-- A text field with defaults appropriate for English, plus
aggressive word-splitting and autophrase features enabled.
This field is just like text_en, except it adds
WordDelimiterFilter to enable splitting and matching of
words on case-change, alpha numeric boundaries, and
non-alphanumeric chars. This means certain compound word
cases will work, for example query "wi fi" will match
document "WiFi" or "wi-fi".
-->
<fieldType name="text_en_splitting" class="solr.TextField" positionIncrementGap="100" autoGeneratePhraseQueries="true">
<analyzer type="index">
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
<!-- in this example, we will only use synonyms at query time
<filter class="solr.SynonymFilterFactory" synonyms="index_synonyms.txt" ignoreCase="true" expand="false"/>
-->
<!-- Case insensitive stop word removal.
-->
<filter class="solr.StopFilterFactory"
ignoreCase="true"
words="stopwords.txt"
/>
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0" splitOnCaseChange="1"/>
<filter class="solr.LowerCaseFilterFactory"/>
<filter class="solr.KeywordMarkerFilterFactory" protected="protwords.txt"/>
<filter class="solr.PorterStemFilterFactory"/>
</analyzer>
<analyzer type="query">
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
<filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt" ignoreCase="true" expand="true"/>
<filter class="solr.StopFilterFactory"
ignoreCase="true"
words="stopwords.txt"
/>
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="1"/>
<filter class="solr.LowerCaseFilterFactory"/>
<filter class="solr.KeywordMarkerFilterFactory" protected="protwords.txt"/>
<filter class="solr.PorterStemFilterFactory"/>
</analyzer>
</fieldType>
<!-- Less flexible matching, but less false matches. Probably not ideal for product names,
but may be good for SKUs. Can insert dashes in the wrong place and still match. -->
<fieldType name="text_en_splitting_tight" class="solr.TextField" positionIncrementGap="100" autoGeneratePhraseQueries="true">
<analyzer>
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
<filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt" ignoreCase="true" expand="false"/>
<filter class="solr.StopFilterFactory" ignoreCase="true" words="stopwords.txt"/>
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="0" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
<filter class="solr.LowerCaseFilterFactory"/>
<filter class="solr.KeywordMarkerFilterFactory" protected="protwords.txt"/>
<filter class="solr.EnglishMinimalStemFilterFactory"/>
<!-- this filter can remove any duplicate tokens that appear at the same position - sometimes
possible with WordDelimiterFilter in conjuncton with stemming. -->
<filter class="solr.RemoveDuplicatesTokenFilterFactory"/>
</analyzer>
</fieldType>
<!-- Just like text_general except it reverses the characters of
each token, to enable more efficient leading wildcard queries. -->
<fieldType name="text_general_rev" class="solr.TextField" positionIncrementGap="100">
<analyzer type="index">
<tokenizer class="solr.StandardTokenizerFactory"/>
<filter class="solr.StopFilterFactory" ignoreCase="true" words="stopwords.txt" />
<filter class="solr.LowerCaseFilterFactory"/>
<filter class="solr.ReversedWildcardFilterFactory" withOriginal="true"
maxPosAsterisk="3" maxPosQuestion="2" maxFractionAsterisk="0.33"/>
</analyzer>
<analyzer type="query">
<tokenizer class="solr.StandardTokenizerFactory"/>
<filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt" ignoreCase="true" expand="true"/>
<filter class="solr.StopFilterFactory" ignoreCase="true" words="stopwords.txt" />
<filter class="solr.LowerCaseFilterFactory"/>
</analyzer>
</fieldType>
<!-- charFilter + WhitespaceTokenizer -->
<!--
<fieldType name="text_char_norm" class="solr.TextField" positionIncrementGap="100" >
<analyzer>
<charFilter class="solr.MappingCharFilterFactory" mapping="mapping-ISOLatin1Accent.txt"/>
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
</analyzer>
</fieldType>
-->
<!-- This is an example of using the KeywordTokenizer along
With various TokenFilterFactories to produce a sortable field
that does not include some properties of the source text
-->
<fieldType name="alphaOnlySort" class="solr.TextField" sortMissingLast="true" omitNorms="true">
<analyzer>
<!-- KeywordTokenizer does no actual tokenizing, so the entire
input string is preserved as a single token
-->
<tokenizer class="solr.KeywordTokenizerFactory"/>
<!-- The LowerCase TokenFilter does what you expect, which can be
when you want your sorting to be case insensitive
-->
<filter class="solr.LowerCaseFilterFactory" />
<!-- The TrimFilter removes any leading or trailing whitespace -->
<filter class="solr.TrimFilterFactory" />
<!-- The PatternReplaceFilter gives you the flexibility to use
Java Regular expression to replace any sequence of characters
matching a pattern with an arbitrary replacement string,
which may include back references to portions of the original
string matched by the pattern.
See the Java Regular Expression documentation for more
information on pattern and replacement string syntax.
http://docs.oracle.com/javase/7/docs/api/java/util/regex/package-summary.html
-->
<filter class="solr.PatternReplaceFilterFactory"
pattern="([^a-z])" replacement="" replace="all"
/>
</analyzer>
</fieldType>
<fieldtype name="phonetic" stored="false" indexed="true" class="solr.TextField" >
<analyzer>
<tokenizer class="solr.StandardTokenizerFactory"/>
<filter class="solr.DoubleMetaphoneFilterFactory" inject="false"/>
</analyzer>
</fieldtype>
<fieldtype name="payloads" stored="false" indexed="true" class="solr.TextField" >
<analyzer>
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
<!--
The DelimitedPayloadTokenFilter can put payloads on tokens... for example,
a token of "foo|1.4" would be indexed as "foo" with a payload of 1.4f
Attributes of the DelimitedPayloadTokenFilterFactory :
"delimiter" - a one character delimiter. Default is | (pipe)
"encoder" - how to encode the following value into a playload
float -> org.apache.lucene.analysis.payloads.FloatEncoder,
integer -> o.a.l.a.p.IntegerEncoder
identity -> o.a.l.a.p.IdentityEncoder
Fully Qualified class name implementing PayloadEncoder, Encoder must have a no arg constructor.
-->
<filter class="solr.DelimitedPayloadTokenFilterFactory" encoder="float"/>
</analyzer>
</fieldtype>
<!-- lowercases the entire field value, keeping it as a single token. -->
<fieldType name="lowercase" class="solr.TextField" positionIncrementGap="100">
<analyzer>
<tokenizer class="solr.KeywordTokenizerFactory"/>
<filter class="solr.LowerCaseFilterFactory" />
</analyzer>
</fieldType>
<!--
Example of using PathHierarchyTokenizerFactory at index time, so
queries for paths match documents at that path, or in descendent paths
-->
<fieldType name="descendent_path" class="solr.TextField">
<analyzer type="index">
<tokenizer class="solr.PathHierarchyTokenizerFactory" delimiter="/" />
</analyzer>
<analyzer type="query">
<tokenizer class="solr.KeywordTokenizerFactory" />
</analyzer>
</fieldType>
<!--
Example of using PathHierarchyTokenizerFactory at query time, so
queries for paths match documents at that path, or in ancestor paths
-->
<fieldType name="ancestor_path" class="solr.TextField">
<analyzer type="index">
<tokenizer class="solr.KeywordTokenizerFactory" />
</analyzer>
<analyzer type="query">
<tokenizer class="solr.PathHierarchyTokenizerFactory" delimiter="/" />
</analyzer>
</fieldType>
<!-- since fields of this type are by default not stored or indexed,
any data added to them will be ignored outright. -->
<fieldtype name="ignored" stored="false" indexed="false" multiValued="true" class="solr.StrField" />
<!-- This point type indexes the coordinates as separate fields (subFields)
If subFieldType is defined, it references a type, and a dynamic field
definition is created matching *___<typename>. Alternately, if
subFieldSuffix is defined, that is used to create the subFields.
Example: if subFieldType="double", then the coordinates would be
indexed in fields myloc_0___double,myloc_1___double.
Example: if subFieldSuffix="_d" then the coordinates would be indexed
in fields myloc_0_d,myloc_1_d
The subFields are an implementation detail of the fieldType, and end
users normally should not need to know about them.
-->
<fieldType name="point" class="solr.PointType" dimension="2" subFieldSuffix="_d"/>
<!-- A specialized field for geospatial search. If indexed, this fieldType must not be multivalued. -->
<fieldType name="location" class="solr.LatLonType" subFieldSuffix="_coordinate"/>
<!-- An alternative geospatial field type new to Solr 4. It supports multiValued and polygon shapes.
For more information about this and other Spatial fields new to Solr 4, see:
http://wiki.apache.org/solr/SolrAdaptersForLuceneSpatial4
-->
<fieldType name="location_rpt" class="solr.SpatialRecursivePrefixTreeFieldType"
geo="true" distErrPct="0.025" maxDistErr="0.000009" units="degrees" />
<!-- Money/currency field type. See http://wiki.apache.org/solr/MoneyFieldType
Parameters:
defaultCurrency: Specifies the default currency if none specified. Defaults to "USD"
precisionStep: Specifies the precisionStep for the TrieLong field used for the amount
providerClass: Lets you plug in other exchange provider backend:
solr.FileExchangeRateProvider is the default and takes one parameter:
currencyConfig: name of an xml file holding exchange rates
solr.OpenExchangeRatesOrgProvider uses rates from openexchangerates.org:
ratesFileLocation: URL or path to rates JSON file (default latest.json on the web)
refreshInterval: Number of minutes between each rates fetch (default: 1440, min: 60)
-->
<fieldType name="currency" class="solr.CurrencyField" precisionStep="8" defaultCurrency="USD" currencyConfig="currency.xml" />
<!-- some examples for different languages (generally ordered by ISO code) -->
<!-- REMOVED. these reference things not in the test config, like lang/stopwords_en.txt -->
</types>
<!-- Similarity is the scoring routine for each document vs. a query.
A custom Similarity or SimilarityFactory may be specified here, but
the default is fine for most applications.
For more info: http://wiki.apache.org/solr/SchemaXml#Similarity
-->
<!--
<similarity class="com.example.solr.CustomSimilarityFactory">
<str name="paramkey">param value</str>
</similarity>
-->
</schema>

View File

@ -945,5 +945,19 @@ public class QueryEqualityTest extends SolrTestCaseJ4 {
} }
public void testAggs() throws Exception {
assertFuncEquals("agg(avg(foo_i))", "agg(avg(foo_i))");
assertFuncEquals("agg(avg(foo_i))", "agg_avg(foo_i)");
assertFuncEquals("agg_min(foo_i)", "agg(min(foo_i))");
assertFuncEquals("agg_max(foo_i)", "agg(max(foo_i))");
assertFuncEquals("agg_avg(foo_i)", "agg_avg(foo_i)");
assertFuncEquals("agg_sum(foo_i)", "agg_sum(foo_i)");
assertFuncEquals("agg_count()", "agg_count()");
assertFuncEquals("agg_unique(foo_i)", "agg_unique(foo_i)");
assertFuncEquals("agg_sumsq(foo_i)", "agg_sumsq(foo_i)");
// assertFuncEquals("agg_stdev(foo_i)", "agg_stdev(foo_i)");
// assertFuncEquals("agg_multistat(foo_i)", "agg_multistat(foo_i)");
}
} }

View File

@ -0,0 +1,673 @@
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.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Random;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.JSONTestUtil;
import org.apache.solr.SolrTestCaseHS;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.request.macro.MacroExpander;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@LuceneTestCase.SuppressCodecs({"Lucene3x","Lucene40","Lucene41","Lucene42","Lucene45","Appending"})
public class TestJsonFacets extends SolrTestCaseHS {
private static SolrInstances servers; // for distributed testing
@BeforeClass
public static void beforeTests() throws Exception {
JSONTestUtil.failRepeatedKeys = true;
initCore("solrconfig-tlog.xml","schema_latest.xml");
}
public static void initServers() throws Exception {
if (servers == null) {
servers = new SolrInstances(3, "solrconfig-tlog.xml", "schema_latest.xml");
}
}
@AfterClass
public static void afterTests() throws Exception {
JSONTestUtil.failRepeatedKeys = false;
if (servers != null) {
servers.stop();
}
}
// attempt to reproduce https://github.com/Heliosearch/heliosearch/issues/33
@Test
public void testComplex() throws Exception {
Random r = random();
Client client = Client.localClient;
double price_low = 11000;
double price_high = 100000;
ModifiableSolrParams p = params("make_s","make_s", "model_s","model_s", "price_low",Double.toString(price_low), "price_high",Double.toString(price_high));
MacroExpander m = new MacroExpander( p.getMap() );
String make_s = m.expand("${make_s}");
String model_s = m.expand("${model_s}");
client.deleteByQuery("*:*", null);
int nDocs = 99;
String[] makes = {"honda", "toyota", "ford", null};
Double[] prices = {10000.0, 30000.0, 50000.0, 0.0, null};
String[] honda_models = {"accord", "civic", "fit", "pilot", null}; // make sure this is alphabetized to match tiebreaks in index
String[] other_models = {"a", "b", "c", "x", "y", "z", null};
int nHonda = 0;
final int[] honda_model_counts = new int[honda_models.length];
for (int i=0; i<nDocs; i++) {
SolrInputDocument doc = sdoc("id", Integer.toString(i));
Double price = rand(prices);
if (price != null) {
doc.addField("cost_f", price);
}
boolean matches_price = price!=null && price >= price_low && price <= price_high;
String make = rand(makes);
if (make != null) {
doc.addField(make_s, make);
}
if ("honda".equals(make)) {
int modelNum = r.nextInt(honda_models.length);
String model = honda_models[modelNum];
if (model != null) {
doc.addField(model_s, model);
}
if (matches_price) {
nHonda++;
honda_model_counts[modelNum]++;
}
} else if (make == null) {
doc.addField(model_s, rand(honda_models)); // add some docs w/ model but w/o make
} else {
// other makes
doc.addField(model_s, rand(other_models)); // add some docs w/ model but w/o make
}
client.add(doc, null);
if (r.nextInt(10) == 0) {
client.add(doc, null); // dup, causing a delete
}
if (r.nextInt(20) == 0) {
client.commit(); // force new seg
}
}
client.commit();
// now figure out top counts
List<Integer> idx = new ArrayList<>();
for (int i=0; i<honda_model_counts.length-1; i++) {
idx.add(i);
}
Collections.sort(idx, new Comparator<Integer>() {
@Override
public int compare(Integer o1, Integer o2) {
int cmp = honda_model_counts[o2] - honda_model_counts[o1];
return cmp == 0 ? o1 - o2 : cmp;
}
});
// straight query facets
client.testJQ(params(p, "q", "*:*", "rows","0", "fq","+${make_s}:honda +cost_f:[${price_low} TO ${price_high}]"
, "json.facet", "{makes:{terms:{field:${make_s}, facet:{models:{terms:{field:${model_s}, limit:2, mincount:0}}}}}}}"
, "facet","true", "facet.pivot","make_s,model_s", "facet.limit", "2"
)
, "facets=={count:" + nHonda + ", makes:{buckets:[{val:honda, count:" + nHonda + ", models:{buckets:["
+ "{val:" + honda_models[idx.get(0)] + ", count:" + honda_model_counts[idx.get(0)] + "},"
+ "{val:" + honda_models[idx.get(1)] + ", count:" + honda_model_counts[idx.get(1)] + "}]}"
+ "}]}}"
);
}
public void testStatsSimple() throws Exception {
assertU(delQ("*:*"));
assertU(add(doc("id", "1", "cat_s", "A", "where_s", "NY", "num_d", "4", "num_i", "2", "val_b", "true", "sparse_s","one")));
assertU(add(doc("id", "2", "cat_s", "B", "where_s", "NJ", "num_d", "-9", "num_i", "-5", "val_b", "false")));
assertU(add(doc("id", "3")));
assertU(commit());
assertU(add(doc("id", "4", "cat_s", "A", "where_s", "NJ", "num_d", "2", "num_i", "3")));
assertU(add(doc("id", "5", "cat_s", "B", "where_s", "NJ", "num_d", "11", "num_i", "7", "sparse_s","two")));
assertU(commit());
assertU(add(doc("id", "6", "cat_s", "B", "where_s", "NY", "num_d", "-5", "num_i", "-5")));
assertU(commit());
// test multiple json.facet commands
assertJQ(req("q", "*:*", "rows", "0"
, "json.facet", "{x:'sum(num_d)'}"
, "json.facet", "{y:'min(num_d)'}"
)
, "facets=={count:6 , x:3.0, y:-9.0 }"
);
// test streaming
assertJQ(req("q", "*:*", "rows", "0"
, "json.facet", "{ cat:{terms:{field:'cat_s', method:stream }}" +
", cat2:{terms:{field:'cat_s', method:stream, sort:'index asc' }}" + // default sort
", cat3:{terms:{field:'cat_s', method:stream, mincount:3 }}" + // mincount
", cat4:{terms:{field:'cat_s', method:stream, prefix:B }}" + // prefix
", cat5:{terms:{field:'cat_s', method:stream, offset:1 }}" + // offset
" }"
)
, "facets=={count:6 " +
", cat :{buckets:[{val:A, count:2},{val:B, count:3}]}" +
", cat2:{buckets:[{val:A, count:2},{val:B, count:3}]}" +
", cat3:{buckets:[{val:B, count:3}]}" +
", cat4:{buckets:[{val:B, count:3}]}" +
", cat5:{buckets:[{val:B, count:3}]}" +
" }"
);
// test nested streaming under non-streaming
assertJQ(req("q", "*:*", "rows", "0"
, "json.facet", "{ cat:{terms:{field:'cat_s', sort:'index asc', facet:{where:{terms:{field:where_s,method:stream}}} }}}"
)
, "facets=={count:6 " +
", cat :{buckets:[{val:A, count:2, where:{buckets:[{val:NJ,count:1},{val:NY,count:1}]} },{val:B, count:3, where:{buckets:[{val:NJ,count:2},{val:NY,count:1}]} }]}"
+ "}"
);
// test nested streaming under streaming
assertJQ(req("q", "*:*", "rows", "0"
, "json.facet", "{ cat:{terms:{field:'cat_s', method:stream, facet:{where:{terms:{field:where_s,method:stream}}} }}}"
)
, "facets=={count:6 " +
", cat :{buckets:[{val:A, count:2, where:{buckets:[{val:NJ,count:1},{val:NY,count:1}]} },{val:B, count:3, where:{buckets:[{val:NJ,count:2},{val:NY,count:1}]} }]}"
+ "}"
);
// test nested streaming with stats under streaming
assertJQ(req("q", "*:*", "rows", "0"
, "json.facet", "{ cat:{terms:{field:'cat_s', method:stream, facet:{ where:{terms:{field:where_s,method:stream, facet:{x:'max(num_d)'} }}} }}}"
)
, "facets=={count:6 " +
", cat :{buckets:[{val:A, count:2, where:{buckets:[{val:NJ,count:1,x:2.0},{val:NY,count:1,x:4.0}]} },{val:B, count:3, where:{buckets:[{val:NJ,count:2,x:11.0},{val:NY,count:1,x:-5.0}]} }]}"
+ "}"
);
// test nested streaming with stats under streaming with stats
assertJQ(req("q", "*:*", "rows", "0",
"facet","true"
, "json.facet", "{ cat:{terms:{field:'cat_s', method:stream, facet:{ y:'min(num_d)', where:{terms:{field:where_s,method:stream, facet:{x:'max(num_d)'} }}} }}}"
)
, "facets=={count:6 " +
", cat :{buckets:[{val:A, count:2, y:2.0, where:{buckets:[{val:NJ,count:1,x:2.0},{val:NY,count:1,x:4.0}]} },{val:B, count:3, y:-9.0, where:{buckets:[{val:NJ,count:2,x:11.0},{val:NY,count:1,x:-5.0}]} }]}"
+ "}"
);
assertJQ(req("q", "*:*", "fq","cat_s:A")
, "response/numFound==2"
);
}
@Test
public void testStats() throws Exception {
// single valued strings
doStats(Client.localClient, params());
}
public void doStats(Client client, ModifiableSolrParams p) throws Exception {
// single valued strings
doStatsTemplated(client, params(p, "rows","0", "noexist","noexist_s", "cat_s","cat_s", "where_s","where_s", "num_d","num_d", "num_i","num_i", "super_s","super_s", "val_b","val_b", "sparse_s","sparse_s" ,"multi_ss","multi_ss") );
// multi-valued strings
doStatsTemplated(client, params(p, "facet","true", "rows","0", "noexist","noexist_ss", "cat_s","cat_ss", "where_s","where_ss", "num_d","num_d", "num_i","num_i", "super_s","super_ss", "val_b","val_b", "sparse_s","sparse_ss", "multi_ss","multi_ss") );
}
public static void doStatsTemplated(Client client, ModifiableSolrParams p) throws Exception {
MacroExpander m = new MacroExpander( p.getMap() );
String cat_s = m.expand("${cat_s}");
String where_s = m.expand("${where_s}");
String num_d = m.expand("${num_d}");
String num_i = m.expand("${num_i}");
String val_b = m.expand("${val_b}");
String super_s = m.expand("${super_s}");
String sparse_s = m.expand("${sparse_s}");
String multi_ss = m.expand("${multi_ss}");
client.deleteByQuery("*:*", null);
client.add(sdoc("id", "1", cat_s, "A", where_s, "NY", num_d, "4", num_i, "2", super_s, "zodiac", val_b, "true", sparse_s, "one"), null);
client.add(sdoc("id", "2", cat_s, "B", where_s, "NJ", num_d, "-9", num_i, "-5", super_s,"superman", val_b, "false" , multi_ss,"a", "multi_ss","b" ), null);
client.add(sdoc("id", "3"), null);
client.commit();
client.add(sdoc("id", "4", cat_s, "A", where_s, "NJ", num_d, "2", num_i, "3", super_s,"spiderman" , multi_ss, "b"), null);
client.add(sdoc("id", "5", cat_s, "B", where_s, "NJ", num_d, "11", num_i, "7", super_s,"batman" ,sparse_s,"two", multi_ss, "a"), null);
client.commit();
client.add(sdoc("id", "6", cat_s, "B", where_s, "NY", num_d, "-5", num_i, "-5", super_s,"hulk" , multi_ss, "b", multi_ss, "a" ), null);
client.commit();
// straight query facets
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{catA:{query:{q:'${cat_s}:A'}}, catA2:{query:{query:'${cat_s}:A'}}, catA3:{query:'${cat_s}:A'} }"
)
, "facets=={ 'count':6, 'catA':{ 'count':2}, 'catA2':{ 'count':2}, 'catA3':{ 'count':2}}"
);
// nested query facets
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{ catB:{query:{q:'${cat_s}:B', facet:{nj:{query:'${where_s}:NJ'}, ny:{query:'${where_s}:NY'}} }}}"
)
, "facets=={ 'count':6, 'catB':{'count':3, 'nj':{'count':2}, 'ny':{'count':1}}}"
);
// nested query facets on subset
client.testJQ(params(p, "q", "id:(2 3)"
, "json.facet", "{ catB:{query:{q:'${cat_s}:B', facet:{nj:{query:'${where_s}:NJ'}, ny:{query:'${where_s}:NY'}} }}}"
)
, "facets=={ 'count':2, 'catB':{'count':1, 'nj':{'count':1}, 'ny':{'count':0}}}"
);
// nested query facets with stats
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{ catB:{query:{q:'${cat_s}:B', facet:{nj:{query:{q:'${where_s}:NJ'}}, ny:{query:'${where_s}:NY'}} }}}"
)
, "facets=={ 'count':6, 'catB':{'count':3, 'nj':{'count':2}, 'ny':{'count':1}}}"
);
// field/terms facet
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{c1:{field:'${cat_s}'}, c2:{field:{field:'${cat_s}'}}, c3:{terms:{field:'${cat_s}'}} }"
)
, "facets=={ 'count':6, " +
"'c1':{ 'buckets':[{ 'val':'B', 'count':3}, { 'val':'A', 'count':2}]}, " +
"'c2':{ 'buckets':[{ 'val':'B', 'count':3}, { 'val':'A', 'count':2}]}, " +
"'c3':{ 'buckets':[{ 'val':'B', 'count':3}, { 'val':'A', 'count':2}]}} "
);
// test mincount
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{f1:{terms:{field:'${cat_s}', mincount:3}}}"
)
, "facets=={ 'count':6, " +
"'f1':{ 'buckets':[{ 'val':'B', 'count':3}]} } "
);
// test default mincount of 1
client.testJQ(params(p, "q", "id:1"
, "json.facet", "{f1:{terms:'${cat_s}'}}"
)
, "facets=={ 'count':1, " +
"'f1':{ 'buckets':[{ 'val':'A', 'count':1}]} } "
);
// test mincount of 0 - need processEmpty for distrib to match up
client.testJQ(params(p, "q", "id:1"
, "json.facet", "{processEmpty:true, f1:{terms:{field:'${cat_s}', mincount:0}}}"
)
, "facets=={ 'count':1, " +
"'f1':{ 'buckets':[{ 'val':'A', 'count':1}, { 'val':'B', 'count':0}]} } "
);
// test mincount of 0 with stats, need processEmpty for distrib to match up
client.testJQ(params(p, "q", "id:1"
, "json.facet", "{processEmpty:true, f1:{terms:{field:'${cat_s}', mincount:0, allBuckets:true, facet:{n1:'sum(${num_d})'} }}}"
)
, "facets=={ 'count':1, " +
"'f1':{ allBuckets:{ 'count':1, n1:4.0}, 'buckets':[{ 'val':'A', 'count':1, n1:4.0}, { 'val':'B', 'count':0 /*, n1:0.0 */ }]} } "
);
// test sorting by stat
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{f1:{terms:{field:'${cat_s}', sort:'n1 desc', facet:{n1:'sum(${num_d})'} }}" +
" , f2:{terms:{field:'${cat_s}', sort:'n1 asc', facet:{n1:'sum(${num_d})'} }} }"
)
, "facets=={ 'count':6, " +
" f1:{ 'buckets':[{ val:'A', count:2, n1:6.0 }, { val:'B', count:3, n1:-3.0}]}" +
", f2:{ 'buckets':[{ val:'B', count:3, n1:-3.0}, { val:'A', count:2, n1:6.0 }]} }"
);
// test sorting by count/index order
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{f1:{terms:{field:'${cat_s}', sort:'count desc' } }" +
" , f2:{terms:{field:'${cat_s}', sort:'count asc' } }" +
" , f3:{terms:{field:'${cat_s}', sort:'index asc' } }" +
" , f4:{terms:{field:'${cat_s}', sort:'index desc' } }" +
"}"
)
, "facets=={ count:6 " +
" ,f1:{buckets:[ {val:B,count:3}, {val:A,count:2} ] }" +
" ,f2:{buckets:[ {val:A,count:2}, {val:B,count:3} ] }" +
" ,f3:{buckets:[ {val:A,count:2}, {val:B,count:3} ] }" +
" ,f4:{buckets:[ {val:B,count:3}, {val:A,count:2} ] }" +
"}"
);
// test tiebreaks when sorting by count
client.testJQ(params(p, "q", "id:1 id:6"
, "json.facet", "{f1:{terms:{field:'${cat_s}', sort:'count desc' } }" +
" , f2:{terms:{field:'${cat_s}', sort:'count asc' } }" +
"}"
)
, "facets=={ count:2 " +
" ,f1:{buckets:[ {val:A,count:1}, {val:B,count:1} ] }" +
" ,f2:{buckets:[ {val:A,count:1}, {val:B,count:1} ] }" +
"}"
);
// terms facet with nested query facet
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{cat:{terms:{field:'${cat_s}', facet:{nj:{query:'${where_s}:NJ'}} } }} }"
)
, "facets=={ 'count':6, " +
"'cat':{ 'buckets':[{ 'val':'B', 'count':3, 'nj':{ 'count':2}}, { 'val':'A', 'count':2, 'nj':{ 'count':1}}]} }"
);
// terms facet with nested query facet on subset
client.testJQ(params(p, "q", "id:(2 5 4)"
, "json.facet", "{cat:{terms:{field:'${cat_s}', facet:{nj:{query:'${where_s}:NJ'}} } }} }"
)
, "facets=={ 'count':3, " +
"'cat':{ 'buckets':[{ 'val':'B', 'count':2, 'nj':{ 'count':2}}, { 'val':'A', 'count':1, 'nj':{ 'count':1}}]} }"
);
// test prefix
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{f1:{terms:{field:${super_s}, prefix:s, mincount:0 }}}" // even with mincount=0, we should only see buckets with the prefix
)
, "facets=={ 'count':6, " +
"'f1':{ 'buckets':[{val:spiderman, count:1}, {val:superman, count:1}]} } "
);
// test prefix that doesn't exist
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{f1:{terms:{field:${super_s}, prefix:ttt, mincount:0 }}}"
)
, "facets=={ 'count':6, " +
"'f1':{ 'buckets':[]} } "
);
// test prefix that doesn't exist at start
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{f1:{terms:{field:${super_s}, prefix:aaaaaa, mincount:0 }}}"
)
, "facets=={ 'count':6, " +
"'f1':{ 'buckets':[]} } "
);
// test prefix that doesn't exist at end
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{f1:{terms:{field:${super_s}, prefix:zzzzzz, mincount:0 }}}"
)
, "facets=={ 'count':6, " +
"'f1':{ 'buckets':[]} } "
);
//
// missing
//
// test missing w/ non-existent field
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{f1:{terms:{field:${noexist}, missing:true}}}"
)
, "facets=={ 'count':6, " +
"'f1':{ 'buckets':[], missing:{count:6} } } "
);
// test missing
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{f1:{terms:{field:${sparse_s}, missing:true }}}"
)
, "facets=={ 'count':6, " +
"'f1':{ 'buckets':[{val:one, count:1}, {val:two, count:1}], missing:{count:4} } } "
);
// test missing with stats
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{f1:{terms:{field:${sparse_s}, missing:true, facet:{x:'sum(num_d)'} }}}"
)
, "facets=={ 'count':6, " +
"'f1':{ 'buckets':[{val:one, count:1, x:4.0}, {val:two, count:1, x:11.0}], missing:{count:4, x:-12.0} } } "
);
// test that the missing bucket is not affected by any prefix
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{f1:{terms:{field:${sparse_s}, missing:true, prefix:on, facet:{x:'sum(num_d)'} }}}"
)
, "facets=={ 'count':6, " +
"'f1':{ 'buckets':[{val:one, count:1, x:4.0}], missing:{count:4, x:-12.0} } } "
);
// test missing with prefix that doesn't exist
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{f1:{terms:{field:${sparse_s}, missing:true, prefix:ppp, facet:{x:'sum(num_d)'} }}}"
)
, "facets=={ 'count':6, " +
"'f1':{ 'buckets':[], missing:{count:4, x:-12.0} } } "
);
// test numBuckets
client.testJQ(params(p, "q", "*:*", "rows", "0", "facet", "true"
, "json.facet", "{f1:{terms:{field:${cat_s}, numBuckets:true, limit:1}}}" // TODO: limit:0 produced an error
)
, "facets=={ 'count':6, " +
"'f1':{ numBuckets:2, buckets:[{val:B, count:3}]} } "
);
// prefix should lower numBuckets
client.testJQ(params(p, "q", "*:*", "rows", "0", "facet", "true"
, "json.facet", "{f1:{terms:{field:${cat_s}, numBuckets:true, prefix:B}}}"
)
, "facets=={ 'count':6, " +
"'f1':{ numBuckets:1, buckets:[{val:B, count:3}]} } "
);
// mincount should lower numBuckets
client.testJQ(params(p, "q", "*:*", "rows", "0", "facet", "true"
, "json.facet", "{f1:{terms:{field:${cat_s}, numBuckets:true, mincount:3}}}"
)
, "facets=={ 'count':6, " +
"'f1':{ numBuckets:1, buckets:[{val:B, count:3}]} } "
);
// basic range facet
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{f:{range:{field:${num_d}, start:-5, end:10, gap:5}}}"
)
, "facets=={count:6, f:{buckets:[ {val:-5.0,count:1}, {val:0.0,count:2}, {val:5.0,count:0} ] } }"
);
// basic range facet with "include" params
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{f:{range:{field:${num_d}, start:-5, end:10, gap:5, include:upper}}}"
)
, "facets=={count:6, f:{buckets:[ {val:-5.0,count:0}, {val:0.0,count:2}, {val:5.0,count:0} ] } }"
);
// range facet with sub facets and stats
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{f:{range:{field:${num_d}, start:-5, end:10, gap:5, facet:{ x:'sum(${num_i})', ny:{query:'${where_s}:NY'}} }}}"
)
, "facets=={count:6, f:{buckets:[ {val:-5.0,count:1,x:-5.0,ny:{count:1}}, {val:0.0,count:2,x:5.0,ny:{count:1}}, {val:5.0,count:0 /* ,x:0.0,ny:{count:0} */ } ] } }"
);
// range facet with sub facets and stats, with "other:all"
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{f:{range:{field:${num_d}, start:-5, end:10, gap:5, other:all, facet:{ x:'sum(${num_i})', ny:{query:'${where_s}:NY'}} }}}"
)
, "facets=={count:6, f:{buckets:[ {val:-5.0,count:1,x:-5.0,ny:{count:1}}, {val:0.0,count:2,x:5.0,ny:{count:1}}, {val:5.0,count:0 /* ,x:0.0,ny:{count:0} */} ]" +
",before: {count:1,x:-5.0,ny:{count:0}}" +
",after: {count:1,x:7.0, ny:{count:0}}" +
",between:{count:3,x:0.0, ny:{count:2}}" +
" } }"
);
// range facet with sub facets and stats, with "other:all", on subset
client.testJQ(params(p, "q", "id:(3 4 6)"
, "json.facet", "{f:{range:{field:${num_d}, start:-5, end:10, gap:5, other:all, facet:{ x:'sum(${num_i})', ny:{query:'${where_s}:NY'}} }}}"
)
, "facets=={count:3, f:{buckets:[ {val:-5.0,count:1,x:-5.0,ny:{count:1}}, {val:0.0,count:1,x:3.0,ny:{count:0}}, {val:5.0,count:0 /* ,x:0.0,ny:{count:0} */} ]" +
",before: {count:0 /* ,x:0.0,ny:{count:0} */ }" +
",after: {count:0 /* ,x:0.0,ny:{count:0} */}" +
",between:{count:2,x:-2.0, ny:{count:1}}" +
" } }"
);
// stats at top level
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{ sum1:'sum(${num_d})', sumsq1:'sumsq(${num_d})', avg1:'avg(${num_d})', min1:'min(${num_d})', max1:'max(${num_d})', numwhere:'unique(${where_s})' }"
)
, "facets=={ 'count':6, " +
"sum1:3.0, sumsq1:247.0, avg1:0.5, min1:-9.0, max1:11.0, numwhere:2 }"
);
// stats at top level, no matches
client.testJQ(params(p, "q", "id:DOESNOTEXIST"
, "json.facet", "{ sum1:'sum(${num_d})', sumsq1:'sumsq(${num_d})', avg1:'avg(${num_d})', min1:'min(${num_d})', max1:'max(${num_d})', numwhere:'unique(${where_s})' }"
)
, "facets=={count:0 " +
"/* ,sum1:0.0, sumsq1:0.0, avg1:0.0, min1:'NaN', max1:'NaN', numwhere:0 */ }"
);
//
// tests on a multi-valued field with actual multiple values, just to ensure that we are
// using a multi-valued method for the rest of the tests when appropriate.
//
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{cat:{terms:{field:'${multi_ss}', facet:{nj:{query:'${where_s}:NJ'}} } }} }"
)
, "facets=={ 'count':6, " +
"'cat':{ 'buckets':[{ 'val':'a', 'count':3, 'nj':{ 'count':2}}, { 'val':'b', 'count':3, 'nj':{ 'count':2}}]} }"
);
// test unique on multi-valued field
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{x:'unique(${multi_ss})', y:{query:{q:'id:2', facet:{x:'unique(${multi_ss})'} }} }"
)
, "facets=={ 'count':6, " +
"x:2," +
"y:{count:1, x:2}" + // single document should yield 2 unique values
" }"
);
//////////////////////////////////////////////////////////////////////////////////////////////////////////
// test converting legacy facets
// test mincount
client.testJQ(params(p, "q", "*:*"
// , "json.facet", "{f1:{terms:{field:'${cat_s}', mincount:3}}}"
, "facet","true", "facet.version", "2", "facet.field","{!key=f1}${cat_s}", "facet.mincount","3"
)
, "facets=={ 'count':6, " +
"'f1':{ 'buckets':[{ 'val':'B', 'count':3}]} } "
);
// test prefix
client.testJQ(params(p, "q", "*:*"
// , "json.facet", "{f1:{terms:{field:${super_s}, prefix:s, mincount:0 }}}" // even with mincount=0, we should only see buckets with the prefix
, "facet","true", "facet.version", "2", "facet.field","{!key=f1}${super_s}", "facet.prefix","s", "facet.mincount","0"
)
, "facets=={ 'count':6, " +
"'f1':{ 'buckets':[{val:spiderman, count:1}, {val:superman, count:1}]} } "
);
// range facet with sub facets and stats
client.testJQ(params(p, "q", "*:*"
// , "json.facet", "{f:{range:{field:${num_d}, start:-5, end:10, gap:5, facet:{ x:'sum(${num_i})', ny:{query:'${where_s}:NY'}} }}}"
, "facet","true", "facet.version", "2", "facet.range","{!key=f}${num_d}", "facet.range.start","-5", "facet.range.end","10", "facet.range.gap","5"
, "f.f.facet.stat","x:sum(${num_i})", "subfacet.f.query","{!key=ny}${where_s}:NY"
)
, "facets=={count:6, f:{buckets:[ {val:-5.0,count:1,x:-5.0,ny:{count:1}}, {val:0.0,count:2,x:5.0,ny:{count:1}}, {val:5.0,count:0 /* ,x:0.0,ny:{count:0} */ } ] } }"
);
// test sorting by stat
client.testJQ(params(p, "q", "*:*"
// , "json.facet", "{f1:{terms:{field:'${cat_s}', sort:'n1 desc', facet:{n1:'sum(${num_d})'} }}" +
// " , f2:{terms:{field:'${cat_s}', sort:'n1 asc', facet:{n1:'sum(${num_d})'} }} }"
, "facet","true", "facet.version", "2", "facet.field","{!key=f1}${cat_s}", "f.f1.facet.sort","n1 desc", "facet.stat","n1:sum(num_d)"
, "facet.field","{!key=f2}${cat_s}", "f.f1.facet.sort","n1 asc"
)
, "facets=={ 'count':6, " +
" f1:{ 'buckets':[{ val:'A', count:2, n1:6.0 }, { val:'B', count:3, n1:-3.0}]}" +
", f2:{ 'buckets':[{ val:'B', count:3, n1:-3.0}, { val:'A', count:2, n1:6.0 }]} }"
);
// range facet with sub facets and stats, with "other:all", on subset
client.testJQ(params(p, "q", "id:(3 4 6)"
//, "json.facet", "{f:{range:{field:${num_d}, start:-5, end:10, gap:5, other:all, facet:{ x:'sum(${num_i})', ny:{query:'${where_s}:NY'}} }}}"
, "facet","true", "facet.version", "2", "facet.range","{!key=f}${num_d}", "facet.range.start","-5", "facet.range.end","10", "facet.range.gap","5"
, "f.f.facet.stat","x:sum(${num_i})", "subfacet.f.query","{!key=ny}${where_s}:NY", "facet.range.other","all"
)
, "facets=={count:3, f:{buckets:[ {val:-5.0,count:1,x:-5.0,ny:{count:1}}, {val:0.0,count:1,x:3.0,ny:{count:0}}, {val:5.0,count:0 /* ,x:0.0,ny:{count:0} */} ]" +
",before: {count:0 /* ,x:0.0,ny:{count:0} */ }" +
",after: {count:0 /* ,x:0.0,ny:{count:0} */}" +
",between:{count:2,x:-2.0, ny:{count:1}}" +
" } }"
);
// TODO:
// numdocs('query') stat (don't make a bucket... just a count)
// missing(field)
// make missing configurable in min, max, etc
// exclusions
// zeroes
// instead of json.facet make it facet?
}
@Test
public void testDistrib() throws Exception {
initServers();
Client client = servers.getClient( random().nextInt() );
client.queryDefaults().set( "shards", servers.getShards() );
doStats( client, params() );
}
}

View File

@ -57,6 +57,11 @@ public class ModifiableSolrParams extends SolrParams
} }
} }
public Map<String,String[]> getMap() {
return vals;
}
//---------------------------------------------------------------- //----------------------------------------------------------------
//---------------------------------------------------------------- //----------------------------------------------------------------

View File

@ -19,6 +19,7 @@ package org.apache.solr.common.params;
import org.apache.solr.common.util.StrUtils; import org.apache.solr.common.util.StrUtils;
import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.Map; import java.util.Map;
import java.io.IOException; import java.io.IOException;
@ -30,18 +31,33 @@ public class MultiMapSolrParams extends SolrParams {
protected final Map<String,String[]> map; protected final Map<String,String[]> map;
public static void addParam(String name, String val, Map<String,String[]> map) { public static void addParam(String name, String val, Map<String,String[]> map) {
String[] arr = map.get(name); String[] arr = map.get(name);
if (arr ==null) { if (arr == null) {
arr =new String[]{val}; arr = new String[]{val};
} else { } else {
String[] newarr = new String[arr.length+1]; String[] newarr = new String[arr.length+1];
System.arraycopy(arr,0,newarr,0,arr.length); System.arraycopy(arr, 0, newarr, 0, arr.length);
newarr[arr.length]=val; newarr[arr.length] = val;
arr =newarr; arr = newarr;
} }
map.put(name, arr); map.put(name, arr);
} }
public static void addParam(String name, String[] vals, Map<String,String[]> map) {
String[] arr = map.put(name, vals);
if (arr == null) {
return;
}
String[] newarr = new String[arr.length+vals.length];
System.arraycopy(arr, 0, newarr, 0, arr.length);
System.arraycopy(vals, 0, newarr, arr.length, vals.length);
arr = newarr;
map.put(name, arr);
}
public MultiMapSolrParams(Map<String,String[]> map) { public MultiMapSolrParams(Map<String,String[]> map) {
this.map = map; this.map = map;
} }
@ -88,5 +104,35 @@ public class MultiMapSolrParams extends SolrParams {
return sb.toString(); return sb.toString();
} }
/** Returns a MultiMap view of the SolrParams as efficiently as possible. The returned map may or may not be a backing implementation. */
public static Map<String,String[]> asMultiMap(SolrParams params) {
return asMultiMap(params, false);
}
/** Returns a MultiMap view of the SolrParams. A new map will be created if newCopy==true */
public static Map<String,String[]> asMultiMap(SolrParams params, boolean newCopy) {
if (params instanceof MultiMapSolrParams) {
Map<String,String[]> map = ((MultiMapSolrParams)params).getMap();
if (newCopy) {
return new HashMap<>(map);
}
return map;
} else if (params instanceof ModifiableSolrParams) {
Map<String,String[]> map = ((ModifiableSolrParams)params).getMap();
if (newCopy) {
return new HashMap<>(map);
}
return map;
} else {
Map<String,String[]> map = new HashMap<>();
Iterator<String> iterator = params.getParameterNamesIterator();
while (iterator.hasNext()) {
String name = iterator.next();
map.put(name, params.getParams(name));
}
return map;
}
}
} }

View File

@ -320,13 +320,8 @@ public abstract class SolrParams implements Serializable {
/** Create SolrParams from NamedList. */ /** Create SolrParams from NamedList. */
public static SolrParams toSolrParams(NamedList params) { public static SolrParams toSolrParams(NamedList params) {
// if no keys are repeated use the faster MapSolrParams // always use MultiMap for easier processing further down the chain
HashMap<String,String> map = new HashMap<>(); return new MultiMapSolrParams(toMultiMap(params));
for (int i=0; i<params.size(); i++) {
String prev = map.put(params.getName(i), params.getVal(i).toString());
if (prev!=null) return new MultiMapSolrParams(toMultiMap(params));
}
return new MapSolrParams(map);
} }
/** Create filtered SolrParams. */ /** Create filtered SolrParams. */

View File

@ -65,6 +65,12 @@ public class NamedList<T> implements Cloneable, Serializable, Iterable<Map.Entry
nvPairs = new ArrayList<>(); nvPairs = new ArrayList<>();
} }
public NamedList(int sz) {
nvPairs = new ArrayList<>(sz<<1);
}
/** /**
* Creates a NamedList instance containing the "name,value" pairs contained in the * Creates a NamedList instance containing the "name,value" pairs contained in the
* Entry[]. * Entry[].

View File

@ -43,6 +43,10 @@ public class SimpleOrderedMap<T> extends NamedList<T> {
super(); super();
} }
public SimpleOrderedMap(int sz) {
super(sz);
}
/** /**
* Creates an instance backed by an explicitly specified list of * Creates an instance backed by an explicitly specified list of
* pairwise names/values. * pairwise names/values.

View File

@ -0,0 +1,549 @@
package org.apache.solr;
/*
* 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 com.google.common.base.Charsets;
import org.apache.commons.io.FileUtils;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.impl.NoOpResponseParser;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.client.solrj.response.UpdateResponse;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.schema.IndexSchema;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.servlet.DirectSolrConnection;
import org.noggit.JSONUtil;
import org.noggit.ObjectBuilder;
import org.slf4j.Logger;
import java.io.File;
import java.io.IOException;
import java.io.OutputStreamWriter;
import java.io.Writer;
import java.nio.file.Files;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Random;
import java.util.Set;
@SolrTestCaseJ4.SuppressSSL
@LuceneTestCase.SuppressCodecs({"Lucene3x","Lucene40","Lucene41","Lucene42","Lucene45","Appending","Asserting"})
public class SolrTestCaseHS extends SolrTestCaseJ4 {
@SafeVarargs
public static <T> Set<T> set(T... a) {
LinkedHashSet<T> s = new LinkedHashSet<>();
for (T t : a) {
s.add(t);
}
return s;
}
public static <T> T rand(T... vals) {
return vals[ random().nextInt(vals.length) ];
}
public static ModifiableSolrParams params(SolrParams params, String... moreParams) {
ModifiableSolrParams msp = new ModifiableSolrParams(params);
for (int i=0; i<moreParams.length; i+=2) {
msp.add(moreParams[i], moreParams[i+1]);
}
return msp;
}
public static Map<String,Object> toObject(Doc doc, IndexSchema schema, Collection<String> fieldNames) {
Map<String,Object> result = new HashMap<>();
for (Fld fld : doc.fields) {
if (fieldNames != null && !fieldNames.contains(fld.ftype.fname)) continue;
SchemaField sf = schema.getField(fld.ftype.fname);
if (!sf.multiValued()) {
result.put(fld.ftype.fname, fld.vals.get(0));
} else {
result.put(fld.ftype.fname, fld.vals);
}
}
return result;
}
public static Object createDocObjects(Map<Comparable, Doc> fullModel, Comparator sort, int rows, Collection<String> fieldNames) {
List<Doc> docList = new ArrayList<>(fullModel.values());
Collections.sort(docList, sort);
List sortedDocs = new ArrayList(rows);
for (Doc doc : docList) {
if (sortedDocs.size() >= rows) break;
Map<String,Object> odoc = toObject(doc, h.getCore().getLatestSchema(), fieldNames);
sortedDocs.add(toObject(doc, h.getCore().getLatestSchema(), fieldNames));
}
return sortedDocs;
}
public static void compare(SolrQueryRequest req, String path, Object model, Map<Comparable, Doc> fullModel) throws Exception {
String strResponse = h.query(req);
Object realResponse = ObjectBuilder.fromJSON(strResponse);
String err = JSONTestUtil.matchObj(path, realResponse, model);
if (err != null) {
log.error("RESPONSE MISMATCH: " + err
+ "\n\trequest="+req
+ "\n\tresult="+strResponse
+ "\n\texpected="+ JSONUtil.toJSON(model)
+ "\n\tmodel="+ fullModel
);
// re-execute the request... good for putting a breakpoint here for debugging
String rsp = h.query(req);
fail(err);
}
}
/** Pass "null" for the client to query the local server */
public static void assertJQ(SolrClient client, SolrParams args, String... tests) throws Exception {
String resp;
resp = getJSON(client, args);
matchJSON(resp, tests);
}
public static void matchJSON(String response, String... tests) throws Exception {
boolean failed = false;
for (String test : tests) {
if (test == null || test.length()==0) continue;
try {
failed = true;
String err = JSONTestUtil.match(response, test, JSONTestUtil.DEFAULT_DELTA);
failed = false;
if (err != null) {
log.error("query failed JSON validation. error=" + err +
"\n expected =" + test +
"\n response = " + response
);
throw new RuntimeException(err);
}
} finally {
if (failed) {
log.error("JSON query validation threw an exception." +
"\n expected =" + test +
"\n response = " + response
);
}
}
}
}
/***
public static void clearNCache() {
SolrQueryRequest req = req();
req.getSearcher().getnCache().clear(); // OFF-HEAP
req.close();
}***/
public static void clearQueryCache() {
SolrQueryRequest req = req();
req.getSearcher();
req.close();
}
public static String getQueryResponse(SolrClient client, String wt, SolrParams params) throws Exception {
if (client == null) {
return getQueryResponse(wt, params);
}
ModifiableSolrParams p = new ModifiableSolrParams(params);
p.set("wt", wt);
String path = p.get("qt");
p.remove("qt");
p.set("indent","true");
QueryRequest query = new QueryRequest( p );
if (path != null) {
query.setPath(path);
}
query.setResponseParser(new NoOpResponseParser(wt));
NamedList<Object> rsp = client.request(query);
String raw = (String)rsp.get("response");
return raw;
}
public static String getQueryResponse(String wt, SolrParams params) throws Exception {
ModifiableSolrParams p = new ModifiableSolrParams(params);
p.set("wt", wt);
String path = p.get("qt");
p.remove("qt");
p.set("indent","true");
DirectSolrConnection connection = new DirectSolrConnection(h.getCore());
String raw = connection.request(path, p, null);
return raw;
}
public static String getJSON(SolrClient client, SolrParams params) throws Exception {
return getQueryResponse(client, "json", params);
}
/** Adds a document using the specific client, or to the local test core if null.
* Returns the version. TODO: work in progress... version not always returned. */
public static Long add(SolrClient client, SolrInputDocument sdoc, ModifiableSolrParams params) throws Exception {
if (client == null) {
Long version = addAndGetVersion( sdoc, params );
return version;
} else {
UpdateRequest updateRequest = new UpdateRequest();
if (params != null) {
updateRequest.setParams(params);
}
updateRequest.add( sdoc );
UpdateResponse rsp = updateRequest.process( client );
// TODO - return version
return null;
}
}
public static class Client {
ClientProvider provider;
ModifiableSolrParams queryDefaults;
public static Client localClient = new Client(null, 1);
public static Client localClient() {
return new Client(null, 1);
}
public Client(List<SolrClient> clients, int seed) {
if (clients != null) {
provider = new ClientProvider(clients, seed);
}
}
public static int hash(int x) {
// from Thomas Mueller
x = ((x >>> 16) ^ x) * 0x45d9f3b;
x = ((x >>> 16) ^ x) * 0x45d9f3b;
x = ((x >>> 16) ^ x);
return x;
}
public ModifiableSolrParams queryDefaults() {
if (queryDefaults == null) {
queryDefaults = new ModifiableSolrParams();
}
return queryDefaults;
}
public boolean local() {
return provider == null;
}
public void testJQ(SolrParams args, String... tests) throws Exception {
if (queryDefaults != null) {
ModifiableSolrParams newParams = params(queryDefaults);
newParams.add(args);
args = newParams;
}
SolrClient client = provider==null ? null : provider.client(null, args);
SolrTestCaseHS.assertJQ(client, args, tests);
}
public Long add(SolrInputDocument sdoc, ModifiableSolrParams params) throws Exception {
SolrClient client = provider==null ? null : provider.client(sdoc, params);
return SolrTestCaseHS.add(client, sdoc, params);
}
public void commit() throws IOException, SolrServerException {
if (local()) {
assertU(SolrTestCaseJ4.commit());
return;
}
for (SolrClient client : provider.all()) {
client.commit();
}
}
public void deleteByQuery(String query, ModifiableSolrParams params) throws IOException, SolrServerException {
if (local()) {
assertU(delQ(query)); // todo - handle extra params
return;
}
for (SolrClient client : provider.all()) {
client.deleteByQuery(query); // todo - handle extra params
}
}
}
public static class ClientProvider {
public static String idField = "id";
List<SolrClient> clients;
Random r;
int hashSeed;
// thisIsIgnored needed because we need a diff signature
public ClientProvider(List<SolrClient> clients, int seed) {
this.hashSeed = Client.hash(seed);
this.clients = clients;
r = new Random(seed);
}
public SolrClient client(SolrInputDocument sdoc, SolrParams params) {
String idStr = null;
if (sdoc != null) {
idStr = sdoc.getFieldValue(idField).toString();
} else if (params!=null) {
idStr = params.get(idField);
}
int hash;
if (idStr != null) {
// make the client chosen the same for a duplicate ID
hash = idStr.hashCode() ^ hashSeed;
} else {
hash = r.nextInt();
}
return clients.get( (hash & Integer.MAX_VALUE) % clients.size() );
}
public List<SolrClient> all() {
return clients;
}
}
//
// Helper to run an internal Jetty instance.
// Example:
// SolrInstance s1 = new SolrInstance(createTempDir("s1"), "solrconfig-tlog.xml", "schema_latest.xml");
// s1.start();
// SolrClient c1 = s1.getSolrJ();
// assertJQ(c1, params("q", "*:*"), "/response/numFound==3");
// String json = getJSON(c1, params("q","id:1"));
// s1.stop();
//
// To manage multiple servers, see SolrInstances
//
public static class SolrInstance {
private static Logger log = SolrTestCaseJ4.log;
private String collection = "collection1";
private int port = 0;
private String solrconfigFile;
private String schemaFile;
private File baseDir;
private JettySolrRunner jetty;
private SolrClient solrj;
private boolean homeCreated = false;
public SolrInstance(File homeDir, String solrconfigFile, String schemaFile) {
this.baseDir = homeDir;
this.solrconfigFile = solrconfigFile;
this.schemaFile = schemaFile;
}
public String getBaseDir() {
return baseDir.toString();
}
public String getBaseURL() {
return (SolrTestCaseJ4.isSSLMode() ? "https" : "http") + "://127.0.0.1:" + port + "/solr";
}
public String getCollectionURL() {
return getBaseURL() + "/" + collection;
}
/** string appropriate for passing in shards param (i.e. missing http://) */
public String getShardURL() {
return "127.0.0.1:" + port + "/solr" + "/" + collection;
}
public SolrClient getSolrJ() {
if (solrj == null) {
solrj = new HttpSolrClient(getCollectionURL());
}
return solrj;
}
/** If it needs to change */
public void setPort(int port) {
this.port = port;
}
public void createHome() throws Exception {
homeCreated=true;
SolrTestCaseJ4.copySolrHomeToTemp(baseDir, collection);
copyConfFile(baseDir, collection, solrconfigFile);
copyConfFile(baseDir, collection, schemaFile);
File collDir = new File(baseDir, collection);
try (Writer w = new OutputStreamWriter(Files.newOutputStream(collDir.toPath().resolve("core.properties")), Charsets.UTF_8)) {
Properties coreProps = new Properties();
coreProps.put("name", "collection1");
coreProps.put("config", solrconfigFile);
coreProps.put("schema", schemaFile);
coreProps.store(w, "");
}
}
public void start() throws Exception {
if (!homeCreated) {
createHome();
}
if (jetty == null) {
jetty = new JettySolrRunner(baseDir.getAbsolutePath(), "/solr", port, solrconfigFile, schemaFile, true, null, null, null);
}
// silly stuff included from solrconfig.snippet.randomindexconfig.xml
System.setProperty("solr.tests.maxBufferedDocs", String.valueOf(100000));
jetty.start();
port = jetty.getLocalPort();
log.info("===> Started solr server port=" + port + " home="+getBaseDir());
}
public void stop() throws Exception {
jetty.stop();
if (solrj != null) solrj.close();
}
public void tearDown() throws Exception {
IOUtils.deleteFilesIfExist(baseDir.toPath());
}
private static void copyConfFile(File dstRoot, String destCollection, String file) throws Exception {
File subHome = new File(dstRoot, destCollection + File.separator + "conf");
String top = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
FileUtils.copyFile(new File(top, file), new File(subHome, file));
}
public void copyConfigFile(File dstRoot, String destCollection, String file) throws Exception {
if (!homeCreated) {
createHome();
}
File subHome = new File(dstRoot, destCollection + File.separator + "conf");
String top = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
FileUtils.copyFile(new File(top, file), new File(subHome, file));
}
}
// Manages a number of Solr servers and provides a Client to partition documents and randomly assign query requests.
// Example:
// SolrInstances servers = new SolrInstances(3, "solrconfig-tlog.xml","schema_latest.xml");
// Client = servers.getClient(0);
// client.add(sdoc("id", "3"), null);
// client.commit();
// client.testJQ(params("q", "*:*"), "/response/numFound==3")
// servers.stop();
//
public static class SolrInstances {
public List<SolrInstance> slist;
public Client client;
public SolrInstances(int numServers, String solrconfig, String schema) throws Exception {
slist = new ArrayList<>(numServers);
for (int i=0; i<numServers; i++) {
SolrInstance instance = new SolrInstance(createTempDir("s"+ i).toFile(), solrconfig, schema);
slist.add(instance);
instance.start();
}
}
public void stop() throws Exception {
for (SolrInstance instance : slist) {
instance.stop();
}
}
// For params.set("shards", getShards())
public String getShards() {
return getShardsParam(slist);
}
public List<SolrClient> getSolrJs() {
List<SolrClient> solrjs = new ArrayList<>(slist.size());
for (SolrInstance instance : slist) {
solrjs.add( instance.getSolrJ() );
}
return solrjs;
}
public Client getClient(int seed) {
if (client == null) {
client = new Client(getSolrJs(), seed);
}
return client;
}
public static String getShardsParam(List<SolrInstance> instances) {
StringBuilder sb = new StringBuilder();
boolean first = true;
for (SolrInstance instance : instances) {
if (first) {
first = false;
} else {
sb.append(',');
}
sb.append( instance.getShardURL() );
}
return sb.toString();
}
}
}