Merge branch 'master' into jira/solr-13350
diff --git a/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java b/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java
index 82251e4..56965e6 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java
@@ -20,6 +20,7 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collections;
import java.util.List;
import org.apache.lucene.index.LeafReaderContext;
@@ -115,6 +116,11 @@
return scoreMode;
}
+ // nocommit: need to raise a LUCENE jira for this?
+ public List<Collector> getCollectors() {
+ return Collections.unmodifiableList(Arrays.asList(collectors));
+ }
+
@Override
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
final List<LeafCollector> leafCollectors = new ArrayList<>();
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 03592c3..0d525b3 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -178,6 +178,8 @@
private volatile ExecutorService coreContainerWorkExecutor = ExecutorUtil.newMDCAwareCachedThreadPool(
new DefaultSolrThreadFactory("coreContainerWorkExecutor"));
+ final private ExecutorService collectorExecutor;
+
private final OrderedExecutor replayUpdatesExecutor;
protected volatile LogWatcher logging = null;
@@ -340,6 +342,8 @@
ExecutorUtil.newMDCAwareCachedThreadPool(
cfg.getReplayUpdatesThreads(),
new DefaultSolrThreadFactory("replayUpdatesExecutor")));
+ this.collectorExecutor = ExecutorUtil.newMDCAwareCachedThreadPool(cfg.getCollectorsPoolSize(),
+ new DefaultSolrThreadFactory("searcherCollectorExecutor"));
}
private synchronized void initializeAuthorizationPlugin(Map<String, Object> authorizationConf) {
@@ -531,6 +535,8 @@
cfg = null;
containerProperties = null;
replayUpdatesExecutor = null;
+
+ this.collectorExecutor = ExecutorUtil.newMDCAwareCachedThreadPool(new DefaultSolrThreadFactory("searcherCollectorExecutor"));
}
public static CoreContainer createAndLoad(Path solrHome) {
@@ -908,6 +914,7 @@
}
log.info("Shutting down CoreContainer instance=" + System.identityHashCode(this));
+ ExecutorUtil.shutdownAndAwaitTermination(collectorExecutor);
ExecutorUtil.shutdownAndAwaitTermination(coreContainerAsyncTaskExecutor);
ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("closeThreadPool"));
@@ -2004,6 +2011,10 @@
public void runAsync(Runnable r) {
coreContainerAsyncTaskExecutor.submit(r);
}
+
+ public ExecutorService getCollectorExecutor() {
+ return collectorExecutor;
+ }
}
class CloserThread extends Thread {
diff --git a/solr/core/src/java/org/apache/solr/core/NodeConfig.java b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
index 150b008..13fcac8 100644
--- a/solr/core/src/java/org/apache/solr/core/NodeConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
@@ -63,6 +63,8 @@
private final Integer coreLoadThreads;
private final int replayUpdatesThreads;
+
+ private final int collectorsPoolSize;
@Deprecated
// This should be part of the transientCacheConfig, remove in 7.0
@@ -86,7 +88,7 @@
String coreAdminHandlerClass, String collectionsAdminHandlerClass,
String healthCheckHandlerClass, String infoHandlerClass, String configSetsHandlerClass,
LogWatcherConfig logWatcherConfig, CloudConfig cloudConfig, Integer coreLoadThreads, int replayUpdatesThreads,
- int transientCacheSize, boolean useSchemaCache, String managementPath, SolrResourceLoader loader,
+ int collectorsPoolSize, int transientCacheSize, boolean useSchemaCache, String managementPath, SolrResourceLoader loader,
Properties solrProperties, PluginInfo[] backupRepositoryPlugins,
MetricsConfig metricsConfig, PluginInfo transientCacheConfig, PluginInfo tracerConfig) {
this.nodeName = nodeName;
@@ -106,6 +108,7 @@
this.cloudConfig = cloudConfig;
this.coreLoadThreads = coreLoadThreads;
this.replayUpdatesThreads = replayUpdatesThreads;
+ this.collectorsPoolSize = collectorsPoolSize;
this.transientCacheSize = transientCacheSize;
this.useSchemaCache = useSchemaCache;
this.managementPath = managementPath;
@@ -161,6 +164,10 @@
return replayUpdatesThreads;
}
+ public int getCollectorsPoolSize() {
+ return collectorsPoolSize;
+ }
+
public String getSharedLibDirectory() {
return sharedLibDirectory;
}
@@ -252,6 +259,7 @@
private CloudConfig cloudConfig;
private int coreLoadThreads = DEFAULT_CORE_LOAD_THREADS;
private int replayUpdatesThreads = Runtime.getRuntime().availableProcessors();
+ private int collectorsPoolSize = DEFAULT_COLLECTORS_POOL_SIZE;
@Deprecated
//Remove in 7.0 and put it all in the transientCache element in solrconfig.xml
private int transientCacheSize = DEFAULT_TRANSIENT_CACHE_SIZE;
@@ -267,6 +275,7 @@
private final String nodeName;
public static final int DEFAULT_CORE_LOAD_THREADS = 3;
+ public static final int DEFAULT_COLLECTORS_POOL_SIZE = 32768;
//No:of core load threads in cloud mode is set to a default of 8
public static final int DEFAULT_CORE_LOAD_THREADS_IN_CLOUD = 8;
@@ -382,6 +391,11 @@
return this;
}
+ public NodeConfigBuilder setCollectorsPoolSize(int collectorsPoolSize) {
+ this.collectorsPoolSize = collectorsPoolSize;
+ return this;
+ }
+
// Remove in Solr 7.0
@Deprecated
public NodeConfigBuilder setTransientCacheSize(int transientCacheSize) {
@@ -428,7 +442,7 @@
return new NodeConfig(nodeName, coreRootDirectory, solrDataHome, booleanQueryMaxClauseCount,
configSetBaseDirectory, sharedLibDirectory, shardHandlerFactoryConfig,
updateShardHandlerConfig, coreAdminHandlerClass, collectionsAdminHandlerClass, healthCheckHandlerClass, infoHandlerClass, configSetsHandlerClass,
- logWatcherConfig, cloudConfig, coreLoadThreads, replayUpdatesThreads, transientCacheSize, useSchemaCache, managementPath, loader, solrProperties,
+ logWatcherConfig, cloudConfig, coreLoadThreads, replayUpdatesThreads, collectorsPoolSize, transientCacheSize, useSchemaCache, managementPath, loader, solrProperties,
backupRepositoryPlugins, metricsConfig, transientCacheConfig, tracerConfig);
}
}
diff --git a/solr/core/src/java/org/apache/solr/core/SolrConfig.java b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
index 0c494a5..9ddcaab 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
@@ -253,7 +253,7 @@
// filtOptEnabled = getBool("query/boolTofilterOptimizer/@enabled", false);
// filtOptCacheSize = getInt("query/boolTofilterOptimizer/@cacheSize",32);
// filtOptThreshold = getFloat("query/boolTofilterOptimizer/@threshold",.05f);
-
+ collectorPoolSize = getInt("query/collectorPoolSize", 4096);
useFilterForSortedQuery = getBool("query/useFilterForSortedQuery", false);
queryResultWindowSize = Math.max(1, getInt("query/queryResultWindowSize", 1));
queryResultMaxDocsCached = getInt("query/queryResultMaxDocsCached", Integer.MAX_VALUE);
@@ -558,7 +558,8 @@
public final int queryResultWindowSize;
public final int queryResultMaxDocsCached;
public final boolean enableLazyFieldLoading;
-
+ public final int collectorPoolSize;
+
public final boolean useRangeVersionsForPeerSync;
// DocSet
diff --git a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
index 3be075f..e314199 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
@@ -279,6 +279,9 @@
case "replayUpdatesThreads":
builder.setReplayUpdatesThreads(parseInt(name, value));
break;
+ case "collectorsPoolSize":
+ builder.setCollectorsPoolSize(parseInt(name, value));
+ break;
case "transientCacheSize":
builder.setTransientCacheSize(parseInt(name, value));
break;
diff --git a/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java b/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
index 7ebe7d1..9a34014 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
@@ -911,7 +911,7 @@
numFound += docs.getNumFound();
NamedList sortFieldValues = (NamedList)(srsp.getSolrResponse().getResponse().get("sort_values"));
- if (sortFieldValues.size()==0 && // we bypass merging this response only if it's partial itself
+ if (sortFieldValues == null || sortFieldValues.size()==0 && // we bypass merging this response only if it's partial itself
thisResponseIsPartial) { // but not the previous one!!
continue; //fsv timeout yields empty sort_vlaues
}
diff --git a/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java b/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
index 859c55a..2a466dd 100644
--- a/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
+++ b/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
@@ -26,6 +26,7 @@
import java.util.TimeZone;
import java.util.concurrent.atomic.AtomicReference;
+import org.apache.solr.common.Callable;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.util.ExecutorUtil;
@@ -47,6 +48,8 @@
protected TimeZone tz;
protected ResponseBuilder rb;
protected List<Closeable> closeHooks;
+ protected List<Callable> initHooks;
+ protected Object initData; // Any additional auxiliary data that needs to be stored
protected SolrDispatchFilter.Action action;
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -67,6 +70,20 @@
threadLocal.set(info);
}
+ public static void init() {
+ SolrRequestInfo info = threadLocal.get();
+ if (info != null && info.initHooks != null) {
+ for (Callable hook : info.initHooks) {
+ try {
+ hook.call(null);
+ } catch (Exception e) {
+ SolrException.log(log, "Exception during init hook", e);
+ }
+ }
+ }
+ }
+
+
public static void clearRequestInfo() {
try {
SolrRequestInfo info = threadLocal.get();
@@ -151,6 +168,15 @@
this.rb = rb;
}
+ public void addInitHook(Callable hook) {
+ // is this better here, or on SolrQueryRequest?
+ synchronized (this) {
+ if (initHooks == null) {
+ initHooks = new LinkedList<>();
+ }
+ initHooks.add(hook);
+ }
+ }
public void addCloseHook(Closeable hook) {
// is this better here, or on SolrQueryRequest?
synchronized (this) {
@@ -184,6 +210,7 @@
ctx.set(null);
SolrRequestInfo.setRequestInfo(me);
}
+ SolrRequestInfo.init();
}
@Override
@@ -192,4 +219,14 @@
}
};
}
+
+ public Object getInitData() {
+ return initData;
+ }
+
+ public void setInitData(Object initData) {
+ synchronized (this) {
+ this.initData = initData;
+ }
+ }
}
diff --git a/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
index c6fb0de..138c56f 100644
--- a/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
@@ -18,6 +18,7 @@
import java.io.Closeable;
import java.io.IOException;
+import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
@@ -44,6 +45,7 @@
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.StringHelper;
+import org.apache.solr.common.Callable;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.SimpleOrderedMap;
@@ -59,6 +61,8 @@
import org.apache.solr.search.join.ScoreJoinQParserPlugin;
import org.apache.solr.util.RTimer;
import org.apache.solr.util.RefCounted;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class JoinQParserPlugin extends QParserPlugin {
public static final String NAME = "join";
@@ -124,6 +128,8 @@
};
}
+ public static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
/**
* A helper method for other plugins to create (non-scoring) JoinQueries wrapped around arbitrary queries against the same core.
*
@@ -176,23 +182,21 @@
}
private class JoinQueryWeight extends ConstantScoreWeight {
- SolrIndexSearcher fromSearcher;
- RefCounted<SolrIndexSearcher> fromRef;
SolrIndexSearcher toSearcher;
ResponseBuilder rb;
ScoreMode scoreMode;
+ final boolean isSameCoreJoin;
public JoinQueryWeight(SolrIndexSearcher searcher, ScoreMode scoreMode, float boost) {
super(JoinQuery.this, boost);
this.scoreMode = scoreMode;
- this.fromSearcher = searcher;
SolrRequestInfo info = SolrRequestInfo.getRequestInfo();
if (info != null) {
rb = info.getResponseBuilder();
}
if (fromIndex == null) {
- this.fromSearcher = searcher;
+ isSameCoreJoin = true;
} else {
if (info == null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Cross-core join must have SolrRequestInfo");
@@ -200,42 +204,42 @@
CoreContainer container = searcher.getCore().getCoreContainer();
final SolrCore fromCore = container.getCore(fromIndex);
+ try {
+ if (fromCore == null) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Cross-core join: no such core " + fromIndex);
+ }
- if (fromCore == null) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Cross-core join: no such core " + fromIndex);
+ if (info.getReq().getCore() == fromCore) {
+ isSameCoreJoin = true;
+ } else {
+ isSameCoreJoin = false;
+ }
+ } finally {
+ fromCore.close();
}
+ info.addInitHook(new Callable<Object>() {
- if (info.getReq().getCore() == fromCore) {
- // if this is the same core, use the searcher passed in... otherwise we could be warming and
- // get an older searcher from the core.
- fromSearcher = searcher;
- } else {
- // This could block if there is a static warming query with a join in it, and if useColdSearcher is true.
- // Deadlock could result if two cores both had useColdSearcher and had joins that used eachother.
- // This would be very predictable though (should happen every time if misconfigured)
- fromRef = fromCore.getSearcher(false, true, null);
+ @Override
+ public void call(Object data) {
+ if (!isSameCoreJoin ) {
+ info.setInitData(container.getCore(fromIndex));
+ }
+ }
+ });
- // be careful not to do anything with this searcher that requires the thread local
- // SolrRequestInfo in a manner that requires the core in the request to match
- fromSearcher = fromRef.get();
- }
-
- if (fromRef != null) {
- final RefCounted<SolrIndexSearcher> ref = fromRef;
+ JoinQParserPlugin.log.info("Adding hook...");
info.addCloseHook(new Closeable() {
@Override
public void close() {
- ref.decref();
+ synchronized (info) {
+ SolrCore fromCore = (SolrCore) info.getInitData();
+ if (fromCore != null) {
+ fromCore.close();
+ info.setInitData(null); // unset
+ }
+ }
}
});
- }
-
- info.addCloseHook(new Closeable() {
- @Override
- public void close() {
- fromCore.close();
- }
- });
}
this.toSearcher = searcher;
@@ -309,248 +313,301 @@
public DocSet getDocSet() throws IOException {
- SchemaField fromSchemaField = fromSearcher.getSchema().getField(fromField);
- SchemaField toSchemaField = toSearcher.getSchema().getField(toField);
-
- boolean usePoints = false;
- if (toSchemaField.getType().isPointField()) {
- if (!fromSchemaField.hasDocValues()) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "join from field " + fromSchemaField + " should have docValues to join with points field " + toSchemaField);
+ RefCounted<SolrIndexSearcher> fromSearcherRef = null;
+ SolrIndexSearcher fromSearcher;
+ SolrCore fromCore = null;
+ boolean openedFromCoreHere = false;
+ if (isSameCoreJoin) {
+ // if this is the same core, use the searcher passed in... otherwise we could be warming and
+ // get an older searcher from the core.
+ fromSearcher = toSearcher;
+ } else {
+ fromCore = (SolrCore) SolrRequestInfo.getRequestInfo().getInitData();
+ if (fromCore == null) {
+ fromCore = toSearcher.getCore().getCoreContainer().getCore(fromIndex);
+ openedFromCoreHere = true;
}
- usePoints = true;
+ fromSearcherRef = fromCore.getSearcher();
+ fromSearcher = fromSearcherRef.get();
}
-
- if (!usePoints) {
- return getDocSetEnumerate();
+ try {
+ SchemaField fromSchemaField = fromSearcher.getSchema().getField(fromField);
+ SchemaField toSchemaField = toSearcher.getSchema().getField(toField);
+
+ boolean usePoints = false;
+ if (toSchemaField.getType().isPointField()) {
+ if (!fromSchemaField.hasDocValues()) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "join from field " + fromSchemaField + " should have docValues to join with points field " + toSchemaField);
+ }
+ usePoints = true;
+ }
+
+ if (!usePoints) {
+ return getDocSetEnumerate();
+ }
+
+ // point fields
+ GraphPointsCollector collector = new GraphPointsCollector(fromSchemaField, null, null);
+ fromSearcher.search(q, collector);
+ Query resultQ = collector.getResultQuery(toSchemaField, false);
+ // don't cache the resulting docSet... the query may be very large. Better to cache the results of the join query itself
+ DocSet result = resultQ==null ? DocSet.EMPTY : toSearcher.getDocSetNC(resultQ, null);
+ return result;
+ } finally {
+ if (fromSearcherRef != null) {
+ fromSearcherRef.decref();
+ }
+ if (fromCore != null && openedFromCoreHere) {
+ fromCore.close();
+ }
}
-
- // point fields
- GraphPointsCollector collector = new GraphPointsCollector(fromSchemaField, null, null);
- fromSearcher.search(q, collector);
- Query resultQ = collector.getResultQuery(toSchemaField, false);
- // don't cache the resulting docSet... the query may be very large. Better to cache the results of the join query itself
- DocSet result = resultQ==null ? DocSet.EMPTY : toSearcher.getDocSetNC(resultQ, null);
- return result;
}
public DocSet getDocSetEnumerate() throws IOException {
- FixedBitSet resultBits = null;
-
- // minimum docFreq to use the cache
- int minDocFreqFrom = Math.max(5, fromSearcher.maxDoc() >> 13);
- int minDocFreqTo = Math.max(5, toSearcher.maxDoc() >> 13);
-
- // use a smaller size than normal since we will need to sort and dedup the results
- int maxSortedIntSize = Math.max(10, toSearcher.maxDoc() >> 10);
-
- DocSet fromSet = fromSearcher.getDocSet(q);
- fromSetSize = fromSet.size();
-
- List<DocSet> resultList = new ArrayList<>(10);
-
- // make sure we have a set that is fast for random access, if we will use it for that
- DocSet fastForRandomSet = fromSet;
- if (minDocFreqFrom>0 && fromSet instanceof SortedIntDocSet) {
- SortedIntDocSet sset = (SortedIntDocSet)fromSet;
- fastForRandomSet = new HashDocSet(sset.getDocs(), 0, sset.size());
- }
-
-
- LeafReader fromReader = fromSearcher.getSlowAtomicReader();
- LeafReader toReader = fromSearcher==toSearcher ? fromReader : toSearcher.getSlowAtomicReader();
- Terms terms = fromReader.terms(fromField);
- Terms toTerms = toReader.terms(toField);
- if (terms == null || toTerms==null) return DocSet.EMPTY;
- String prefixStr = TrieField.getMainValuePrefix(fromSearcher.getSchema().getFieldType(fromField));
- BytesRef prefix = prefixStr == null ? null : new BytesRef(prefixStr);
-
- BytesRef term = null;
- TermsEnum termsEnum = terms.iterator();
- TermsEnum toTermsEnum = toTerms.iterator();
- SolrIndexSearcher.DocsEnumState fromDeState = null;
- SolrIndexSearcher.DocsEnumState toDeState = null;
-
- if (prefix == null) {
- term = termsEnum.next();
+ RefCounted<SolrIndexSearcher> fromSearcherRef = null;
+ SolrIndexSearcher fromSearcher;
+ SolrCore fromCore = null;
+ boolean openedFromCoreHere = false;
+ if (isSameCoreJoin) {
+ fromSearcher = toSearcher;
} else {
- if (termsEnum.seekCeil(prefix) != TermsEnum.SeekStatus.END) {
- term = termsEnum.term();
+ fromCore = (SolrCore) SolrRequestInfo.getRequestInfo().getInitData();
+ if (fromCore == null) {
+ fromCore = toSearcher.getCore().getCoreContainer().getCore(fromIndex);
+ openedFromCoreHere = true;
}
+ fromSearcherRef = fromCore.getSearcher();
+ fromSearcher = fromSearcherRef.get();
}
+ try {
- Bits fromLiveDocs = fromSearcher.getLiveDocsBits();
- Bits toLiveDocs = fromSearcher == toSearcher ? fromLiveDocs : toSearcher.getLiveDocsBits();
-
- fromDeState = new SolrIndexSearcher.DocsEnumState();
- fromDeState.fieldName = fromField;
- fromDeState.liveDocs = fromLiveDocs;
- fromDeState.termsEnum = termsEnum;
- fromDeState.postingsEnum = null;
- fromDeState.minSetSizeCached = minDocFreqFrom;
-
- toDeState = new SolrIndexSearcher.DocsEnumState();
- toDeState.fieldName = toField;
- toDeState.liveDocs = toLiveDocs;
- toDeState.termsEnum = toTermsEnum;
- toDeState.postingsEnum = null;
- toDeState.minSetSizeCached = minDocFreqTo;
-
- while (term != null) {
- if (prefix != null && !StringHelper.startsWith(term, prefix))
- break;
-
- fromTermCount++;
-
- boolean intersects = false;
- int freq = termsEnum.docFreq();
- fromTermTotalDf++;
-
- if (freq < minDocFreqFrom) {
- fromTermDirectCount++;
- // OK to skip liveDocs, since we check for intersection with docs matching query
- fromDeState.postingsEnum = fromDeState.termsEnum.postings(fromDeState.postingsEnum, PostingsEnum.NONE);
- PostingsEnum postingsEnum = fromDeState.postingsEnum;
-
- if (postingsEnum instanceof MultiPostingsEnum) {
- MultiPostingsEnum.EnumWithSlice[] subs = ((MultiPostingsEnum) postingsEnum).getSubs();
- int numSubs = ((MultiPostingsEnum) postingsEnum).getNumSubs();
- outer: for (int subindex = 0; subindex<numSubs; subindex++) {
- MultiPostingsEnum.EnumWithSlice sub = subs[subindex];
- if (sub.postingsEnum == null) continue;
- int base = sub.slice.start;
+ FixedBitSet resultBits = null;
+
+ // minimum docFreq to use the cache
+ int minDocFreqFrom = Math.max(5, fromSearcher.maxDoc() >> 13);
+ int minDocFreqTo = Math.max(5, toSearcher.maxDoc() >> 13);
+
+ // use a smaller size than normal since we will need to sort and dedup the results
+ int maxSortedIntSize = Math.max(10, toSearcher.maxDoc() >> 10);
+
+ DocSet fromSet = fromSearcher.getDocSet(q);
+ fromSetSize = fromSet.size();
+
+ List<DocSet> resultList = new ArrayList<>(10);
+
+ // make sure we have a set that is fast for random access, if we will use it for that
+ DocSet fastForRandomSet = fromSet;
+ if (minDocFreqFrom>0 && fromSet instanceof SortedIntDocSet) {
+ SortedIntDocSet sset = (SortedIntDocSet)fromSet;
+ fastForRandomSet = new HashDocSet(sset.getDocs(), 0, sset.size());
+ }
+
+
+ LeafReader fromReader = fromSearcher.getSlowAtomicReader();
+ LeafReader toReader = fromSearcher==toSearcher ? fromReader : toSearcher.getSlowAtomicReader();
+ Terms terms = fromReader.terms(fromField);
+ Terms toTerms = toReader.terms(toField);
+ if (terms == null || toTerms==null) return DocSet.EMPTY;
+ String prefixStr = TrieField.getMainValuePrefix(fromSearcher.getSchema().getFieldType(fromField));
+ BytesRef prefix = prefixStr == null ? null : new BytesRef(prefixStr);
+
+ BytesRef term = null;
+ TermsEnum termsEnum = terms.iterator();
+ TermsEnum toTermsEnum = toTerms.iterator();
+ SolrIndexSearcher.DocsEnumState fromDeState = null;
+ SolrIndexSearcher.DocsEnumState toDeState = null;
+
+ if (prefix == null) {
+ term = termsEnum.next();
+ } else {
+ if (termsEnum.seekCeil(prefix) != TermsEnum.SeekStatus.END) {
+ term = termsEnum.term();
+ }
+ }
+
+ Bits fromLiveDocs = fromSearcher.getLiveDocsBits();
+ Bits toLiveDocs = fromSearcher == toSearcher ? fromLiveDocs : toSearcher.getLiveDocsBits();
+
+ fromDeState = new SolrIndexSearcher.DocsEnumState();
+ fromDeState.fieldName = fromField;
+ fromDeState.liveDocs = fromLiveDocs;
+ fromDeState.termsEnum = termsEnum;
+ fromDeState.postingsEnum = null;
+ fromDeState.minSetSizeCached = minDocFreqFrom;
+
+ toDeState = new SolrIndexSearcher.DocsEnumState();
+ toDeState.fieldName = toField;
+ toDeState.liveDocs = toLiveDocs;
+ toDeState.termsEnum = toTermsEnum;
+ toDeState.postingsEnum = null;
+ toDeState.minSetSizeCached = minDocFreqTo;
+
+ while (term != null) {
+ if (prefix != null && !StringHelper.startsWith(term, prefix))
+ break;
+
+ fromTermCount++;
+
+ boolean intersects = false;
+ int freq = termsEnum.docFreq();
+ fromTermTotalDf++;
+
+ if (freq < minDocFreqFrom) {
+ fromTermDirectCount++;
+ // OK to skip liveDocs, since we check for intersection with docs matching query
+ fromDeState.postingsEnum = fromDeState.termsEnum.postings(fromDeState.postingsEnum, PostingsEnum.NONE);
+ PostingsEnum postingsEnum = fromDeState.postingsEnum;
+
+ if (postingsEnum instanceof MultiPostingsEnum) {
+ MultiPostingsEnum.EnumWithSlice[] subs = ((MultiPostingsEnum) postingsEnum).getSubs();
+ int numSubs = ((MultiPostingsEnum) postingsEnum).getNumSubs();
+ outer: for (int subindex = 0; subindex<numSubs; subindex++) {
+ MultiPostingsEnum.EnumWithSlice sub = subs[subindex];
+ if (sub.postingsEnum == null) continue;
+ int base = sub.slice.start;
+ int docid;
+ while ((docid = sub.postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+ if (fastForRandomSet.exists(docid+base)) {
+ intersects = true;
+ break outer;
+ }
+ }
+ }
+ } else {
int docid;
- while ((docid = sub.postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
- if (fastForRandomSet.exists(docid+base)) {
+ while ((docid = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+ if (fastForRandomSet.exists(docid)) {
intersects = true;
- break outer;
+ break;
}
}
}
} else {
- int docid;
- while ((docid = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
- if (fastForRandomSet.exists(docid)) {
- intersects = true;
- break;
- }
- }
+ // use the filter cache
+ DocSet fromTermSet = fromSearcher.getDocSet(fromDeState);
+ intersects = fromSet.intersects(fromTermSet);
}
- } else {
- // use the filter cache
- DocSet fromTermSet = fromSearcher.getDocSet(fromDeState);
- intersects = fromSet.intersects(fromTermSet);
- }
-
- if (intersects) {
- fromTermHits++;
- fromTermHitsTotalDf++;
- TermsEnum.SeekStatus status = toTermsEnum.seekCeil(term);
- if (status == TermsEnum.SeekStatus.END) break;
- if (status == TermsEnum.SeekStatus.FOUND) {
- toTermHits++;
- int df = toTermsEnum.docFreq();
- toTermHitsTotalDf += df;
- if (resultBits==null && df + resultListDocs > maxSortedIntSize && resultList.size() > 0) {
- resultBits = new FixedBitSet(toSearcher.maxDoc());
- }
-
- // if we don't have a bitset yet, or if the resulting set will be too large
- // use the filterCache to get a DocSet
- if (toTermsEnum.docFreq() >= minDocFreqTo || resultBits == null) {
- // use filter cache
- DocSet toTermSet = toSearcher.getDocSet(toDeState);
- resultListDocs += toTermSet.size();
- if (resultBits != null) {
- toTermSet.addAllTo(new BitDocSet(resultBits));
- } else {
- if (toTermSet instanceof BitDocSet) {
- resultBits = ((BitDocSet)toTermSet).bits.clone();
- } else {
- resultList.add(toTermSet);
- }
+
+ if (intersects) {
+ fromTermHits++;
+ fromTermHitsTotalDf++;
+ TermsEnum.SeekStatus status = toTermsEnum.seekCeil(term);
+ if (status == TermsEnum.SeekStatus.END) break;
+ if (status == TermsEnum.SeekStatus.FOUND) {
+ toTermHits++;
+ int df = toTermsEnum.docFreq();
+ toTermHitsTotalDf += df;
+ if (resultBits==null && df + resultListDocs > maxSortedIntSize && resultList.size() > 0) {
+ resultBits = new FixedBitSet(toSearcher.maxDoc());
}
- } else {
- toTermDirectCount++;
-
- // need to use liveDocs here so we don't map to any deleted ones
- toDeState.postingsEnum = toDeState.termsEnum.postings(toDeState.postingsEnum, PostingsEnum.NONE);
- toDeState.postingsEnum = BitsFilteredPostingsEnum.wrap(toDeState.postingsEnum, toDeState.liveDocs);
- PostingsEnum postingsEnum = toDeState.postingsEnum;
-
- 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;
- while ((docid = sub.postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
- resultListDocs++;
- resultBits.set(docid + base);
+
+ // if we don't have a bitset yet, or if the resulting set will be too large
+ // use the filterCache to get a DocSet
+ if (toTermsEnum.docFreq() >= minDocFreqTo || resultBits == null) {
+ // use filter cache
+ DocSet toTermSet = toSearcher.getDocSet(toDeState);
+ resultListDocs += toTermSet.size();
+ if (resultBits != null) {
+ toTermSet.addAllTo(new BitDocSet(resultBits));
+ } else {
+ if (toTermSet instanceof BitDocSet) {
+ resultBits = ((BitDocSet)toTermSet).bits.clone();
+ } else {
+ resultList.add(toTermSet);
}
}
} else {
- int docid;
- while ((docid = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
- resultListDocs++;
- resultBits.set(docid);
+ toTermDirectCount++;
+
+ // need to use liveDocs here so we don't map to any deleted ones
+ toDeState.postingsEnum = toDeState.termsEnum.postings(toDeState.postingsEnum, PostingsEnum.NONE);
+ toDeState.postingsEnum = BitsFilteredPostingsEnum.wrap(toDeState.postingsEnum, toDeState.liveDocs);
+ PostingsEnum postingsEnum = toDeState.postingsEnum;
+
+ 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;
+ while ((docid = sub.postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+ resultListDocs++;
+ resultBits.set(docid + base);
+ }
+ }
+ } else {
+ int docid;
+ while ((docid = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+ resultListDocs++;
+ resultBits.set(docid);
+ }
}
}
+
}
-
}
+
+ term = termsEnum.next();
}
-
- term = termsEnum.next();
- }
-
- smallSetsDeferred = resultList.size();
-
- if (resultBits != null) {
- BitDocSet bitSet = new BitDocSet(resultBits);
+
+ smallSetsDeferred = resultList.size();
+
+ if (resultBits != null) {
+ BitDocSet bitSet = new BitDocSet(resultBits);
+ for (DocSet set : resultList) {
+ set.addAllTo(bitSet);
+ }
+ return bitSet;
+ }
+
+ if (resultList.size()==0) {
+ return DocSet.EMPTY;
+ }
+
+ if (resultList.size() == 1) {
+ return resultList.get(0);
+ }
+
+ int sz = 0;
+
+ for (DocSet set : resultList)
+ sz += set.size();
+
+ int[] docs = new int[sz];
+ int pos = 0;
for (DocSet set : resultList) {
- set.addAllTo(bitSet);
+ System.arraycopy(((SortedIntDocSet)set).getDocs(), 0, docs, pos, set.size());
+ pos += set.size();
}
- return bitSet;
+ Arrays.sort(docs);
+ int[] dedup = new int[sz];
+ pos = 0;
+ int last = -1;
+ for (int doc : docs) {
+ if (doc != last)
+ dedup[pos++] = doc;
+ last = doc;
+ }
+
+ if (pos != dedup.length) {
+ dedup = Arrays.copyOf(dedup, pos);
+ }
+
+ return new SortedIntDocSet(dedup, dedup.length);
+
+ } finally {
+ if (fromSearcherRef != null) {
+ fromSearcherRef.decref();
+ }
+ if (fromCore != null && openedFromCoreHere) {
+ fromCore.close();
+ }
}
- if (resultList.size()==0) {
- return DocSet.EMPTY;
- }
-
- if (resultList.size() == 1) {
- return resultList.get(0);
- }
-
- int sz = 0;
-
- for (DocSet set : resultList)
- sz += set.size();
-
- int[] docs = new int[sz];
- int pos = 0;
- for (DocSet set : resultList) {
- System.arraycopy(((SortedIntDocSet)set).getDocs(), 0, docs, pos, set.size());
- pos += set.size();
- }
- Arrays.sort(docs);
- int[] dedup = new int[sz];
- pos = 0;
- int last = -1;
- for (int doc : docs) {
- if (doc != last)
- dedup[pos++] = doc;
- last = doc;
- }
-
- if (pos != dedup.length) {
- dedup = Arrays.copyOf(dedup, pos);
- }
-
- return new SortedIntDocSet(dedup, dedup.length);
}
}
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index 7d33a19..6de5bd5 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -31,12 +31,11 @@
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
-import com.codahale.metrics.MetricRegistry;
-import com.google.common.collect.Iterables;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.ExitableDirectoryReader;
@@ -50,8 +49,41 @@
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.*;
+import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.CollectionStatistics;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.CollectorManager;
+import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.FieldDoc;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.MultiCollector;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.Scorable;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TermStatistics;
+import org.apache.lucene.search.TimeLimitingCollector;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TopDocsCollector;
+import org.apache.lucene.search.TopFieldCollector;
+import org.apache.lucene.search.TopFieldDocs;
+import org.apache.lucene.search.TopScoreDocCollector;
+import org.apache.lucene.search.TotalHitCountCollector;
+import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.search.Weight;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@@ -84,6 +116,9 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.codahale.metrics.MetricRegistry;
+import com.google.common.collect.Iterables;
+
/**
* SolrIndexSearcher adds schema awareness and caching functionality over {@link IndexSearcher}.
*
@@ -230,10 +265,8 @@
}
public SolrIndexSearcher(SolrCore core, String path, IndexSchema schema, String name, DirectoryReader r,
- boolean closeReader, boolean enableCache, boolean reserveDirectory, DirectoryFactory directoryFactory)
- throws IOException {
- super(wrapReader(core, r));
-
+ boolean closeReader, boolean enableCache, boolean reserveDirectory, DirectoryFactory directoryFactory) throws IOException {
+ super(wrapReader(core, r), core.getCoreContainer().getCollectorExecutor());
this.path = path;
this.directoryFactory = directoryFactory;
this.reader = (DirectoryReader) super.readerContext.reader();
@@ -485,7 +518,7 @@
if (releaseDirectory) {
directoryFactory.release(getIndexReader().directory());
}
-
+
// do this at the end so it only gets done if there are no exceptions
numCloses.incrementAndGet();
assert ObjectReleaseTracker.release(this);
@@ -1484,7 +1517,7 @@
// if we have a cursor, then we have a sort that at minimum involves uniqueKey..
// so we must have a TopFieldDocs containing FieldDoc[]
- assert topDocs instanceof TopFieldDocs : "TopFieldDocs cursor constraint violated";
+ assert topDocs instanceof TopFieldDocs : "TopFieldDocs cursor constraint violated, the actual type is: "+topDocs.getClass();
final TopFieldDocs topFieldDocs = (TopFieldDocs) topDocs;
final ScoreDoc[] scoreDocs = topFieldDocs.scoreDocs;
@@ -1593,6 +1626,7 @@
};
}
+ log.info("calling from 1"); // nocommit
buildAndRunCollectorChain(qr, query, collector, cmd, pf.postFilter);
nDocsReturned = 0;
@@ -1603,24 +1637,39 @@
// no docs on this page, so cursor doesn't change
qr.setNextCursorMark(cmd.getCursorMark());
} else {
- final TopDocsCollector topCollector = buildTopDocsCollector(len, cmd);
- MaxScoreCollector maxScoreCollector = null;
- Collector collector = topCollector;
- if ((cmd.getFlags() & GET_SCORES) != 0) {
- maxScoreCollector = new MaxScoreCollector();
- collector = MultiCollector.wrap(topCollector, maxScoreCollector);
- }
- buildAndRunCollectorChain(qr, query, collector, cmd, pf.postFilter);
+ TopDocs topDocs;
+ log.info("calling from 2, query: "+query.getClass()); // nocommit
+ if (pf.postFilter != null || cmd.getSegmentTerminateEarly() || cmd.getTimeAllowed() > 0
+ || query instanceof RankQuery) {
+ log.debug("skipping collector manager");
+ final TopDocsCollector topCollector = buildTopDocsCollector(len, cmd);
+ MaxScoreCollector maxScoreCollector = null;
+ Collector collector = topCollector;
+ if ((cmd.getFlags() & GET_SCORES) != 0) {
+ maxScoreCollector = new MaxScoreCollector();
+ collector = MultiCollector.wrap(topCollector, maxScoreCollector);
+ }
+ buildAndRunCollectorChain(qr, query, collector, cmd, pf.postFilter);
+ totalHits = topCollector.getTotalHits();
+ topDocs = topCollector.topDocs(0, len);
- totalHits = topCollector.getTotalHits();
- TopDocs topDocs = topCollector.topDocs(0, len);
+ maxScore = totalHits > 0 ? (maxScoreCollector == null ? Float.NaN : maxScoreCollector.getMaxScore()) : 0.0f;
+ nDocsReturned = topDocs.scoreDocs.length;
+ } else {
+ log.debug("using collectormanager");
+ CollectorManagerResult result = searchCollectorManagers(len, cmd, query, true, true, false); // nocommit: need docset should be false
+ totalHits = result.totalHits;
+
+ maxScore = result.maxScore;
+ nDocsReturned = result.topDocs.scoreDocs.length;
+ topDocs = result.topDocs;
+ }
+
if (cmd.getSort() != null && query instanceof RankQuery == false && (cmd.getFlags() & GET_SCORES) != 0) {
TopFieldCollector.populateScores(topDocs.scoreDocs, this, query);
}
populateNextCursorMarkFromTopDocs(qr, cmd, topDocs);
- maxScore = totalHits > 0 ? (maxScoreCollector == null ? Float.NaN : maxScoreCollector.getMaxScore()) : 0.0f;
- nDocsReturned = topDocs.scoreDocs.length;
ids = new int[nDocsReturned];
scores = (cmd.getFlags() & GET_SCORES) != 0 ? new float[nDocsReturned] : null;
for (int i = 0; i < nDocsReturned; i++) {
@@ -1634,6 +1683,83 @@
if (sliceLen < 0) sliceLen = 0;
qr.setDocList(new DocSlice(0, sliceLen, ids, scores, totalHits, maxScore));
}
+
+ CollectorManagerResult searchCollectorManagers(int len, QueryCommand cmd, Query query,
+ boolean needTopDocs, boolean needMaxScore, boolean needDocSet) throws IOException {
+ CollectorManager<MultiCollector, CollectorManagerResult> manager = new CollectorManager<MultiCollector, CollectorManagerResult>() {
+ @Override
+ public MultiCollector newCollector() throws IOException {
+ // nocommit: Here, creating a MultiCollector for every segment (correctness > speed).
+ // Need to explore sharing a single MultiCollector with every segment. Are these
+ // sub-collectors thread-safe? DocSetCollector seems like not thread-safe, does someone know?
+ Collection<Collector> collectors = new ArrayList<Collector>();
+ if (needTopDocs) collectors.add(buildTopDocsCollector(len, cmd));
+ if (needMaxScore) collectors.add(new MaxScoreCollector());
+ if (needDocSet) collectors.add(new DocSetCollector(maxDoc()));
+ return (MultiCollector) MultiCollector.wrap(collectors);
+ }
+
+ @Override
+ public CollectorManagerResult reduce(Collection<MultiCollector> multiCollectors) throws IOException {
+ final TopDocs[] topDocs = new TopDocs[multiCollectors.size()];
+ float maxScore = 0.0f;
+ DocSet docSet = new BitDocSet(new FixedBitSet(maxDoc())); // TODO: if docset is not needed, avoid this initialization
+ int i = 0;
+ for (MultiCollector multiCollector: multiCollectors) {
+ int c = 0;
+ List<Collector> subCollectors = multiCollector.getCollectors();
+ TopDocsCollector topDocsCollector = needTopDocs? ((TopDocsCollector) subCollectors.get(c++)): null;
+ MaxScoreCollector maxScoreCollector = needMaxScore? ((MaxScoreCollector) subCollectors.get(c++)): null;
+ DocSetCollector docSetCollector = needDocSet? ((DocSetCollector) subCollectors.get(c++)): null;
+
+ if (needTopDocs) topDocs[i++] = (topDocsCollector instanceof TopFieldCollector)? ((TopFieldCollector)topDocsCollector).topDocs(0, len): topDocsCollector.topDocs(0, len);
+ if (needMaxScore)
+ if (!Float.isNaN(maxScoreCollector.getMaxScore()))
+ maxScore = Math.max(maxScore, maxScoreCollector.getMaxScore());
+ if (needDocSet) docSet = docSet.union(docSetCollector.getDocSet());
+ }
+ TopDocs mergedTopDocs;
+ if (topDocs != null && topDocs.length>0 && topDocs[0] instanceof TopFieldDocs) {
+ TopFieldDocs[] topFieldDocs = Arrays.copyOf(topDocs, topDocs.length, TopFieldDocs[].class);
+ mergedTopDocs = TopFieldDocs.merge(weightSort(cmd.getSort()), len, topFieldDocs);
+ } else {
+ mergedTopDocs = needTopDocs? TopDocs.merge(0, len, topDocs): null;
+ }
+ int totalHits = needTopDocs? (int)mergedTopDocs.totalHits.value: -1;
+ maxScore = totalHits > 0 ? maxScore : 0.0f;
+ return new CollectorManagerResult(mergedTopDocs, docSet, maxScore, totalHits);
+ }
+
+ };
+
+ CollectorManagerResult ret;
+ try {
+ ret = super.search(query, manager);
+ } catch (Exception ex) {
+ if (ex instanceof RuntimeException &&
+ ex.getCause() != null & ex.getCause() instanceof ExecutionException
+ && ex.getCause().getCause() != null && ex.getCause().getCause() instanceof RuntimeException) {
+ throw (RuntimeException)ex.getCause().getCause();
+ } else {
+ throw ex;
+ }
+ }
+ return ret;
+ }
+
+ class CollectorManagerResult {
+ TopDocs topDocs;
+ DocSet docSet;
+ float maxScore;
+ int totalHits;
+
+ public CollectorManagerResult(TopDocs topDocs, DocSet docSet, float maxScore, int totalHits) {
+ this.topDocs = topDocs;
+ this.docSet = docSet;
+ this.maxScore = maxScore;
+ this.totalHits = totalHits;
+ }
+ }
// any DocSet returned is for the query only, without any filtering... that way it may
// be cached if desired.
@@ -1691,7 +1817,7 @@
collector = MultiCollector.wrap(setCollector, topScoreCollector);
}
-
+ log.info("calling from 3"); // nocommit
buildAndRunCollectorChain(qr, query, collector, cmd, pf.postFilter);
set = DocSetUtil.getDocSet(setCollector, this);
@@ -1704,33 +1830,46 @@
// no docs on this page, so cursor doesn't change
qr.setNextCursorMark(cmd.getCursorMark());
} else {
+ TopDocs topDocs;
+ if (pf.postFilter != null || cmd.getSegmentTerminateEarly() || cmd.getTimeAllowed() > 0 || query instanceof RankQuery) {
- final TopDocsCollector topCollector = buildTopDocsCollector(len, cmd);
- DocSetCollector setCollector = new DocSetCollector(maxDoc);
- MaxScoreCollector maxScoreCollector = null;
- List<Collector> collectors = new ArrayList<>(Arrays.asList(topCollector, setCollector));
+ final TopDocsCollector topCollector = buildTopDocsCollector(len, cmd);
+ DocSetCollector setCollector = new DocSetCollector(maxDoc);
+ MaxScoreCollector maxScoreCollector = null;
+ List<Collector> collectors = new ArrayList<>(Arrays.asList(topCollector, setCollector));
- if ((cmd.getFlags() & GET_SCORES) != 0) {
- maxScoreCollector = new MaxScoreCollector();
- collectors.add(maxScoreCollector);
+ if ((cmd.getFlags() & GET_SCORES) != 0) {
+ maxScoreCollector = new MaxScoreCollector();
+ collectors.add(maxScoreCollector);
+ }
+
+ Collector collector = MultiCollector.wrap(collectors);
+
+ log.info("calling from 4"); // nocommit
+ buildAndRunCollectorChain(qr, query, collector, cmd, pf.postFilter);
+
+ set = DocSetUtil.getDocSet(setCollector, this);
+ totalHits = topCollector.getTotalHits();
+ assert (totalHits == set.size()) || qr.isPartialResults();
+ topDocs = topCollector.topDocs(0, len);
+ maxScore = totalHits > 0 ? (maxScoreCollector == null ? Float.NaN : maxScoreCollector.getMaxScore()) : 0.0f;
+ nDocsReturned = topDocs.scoreDocs.length;
+
+ } else {
+ log.debug("using collectormanager");
+ CollectorManagerResult result = searchCollectorManagers(len, cmd, query, true, true, true);
+ set = result.docSet;
+ totalHits = result.totalHits;
+ assert (totalHits == set.size()) || qr.isPartialResults();
+ topDocs = result.topDocs;
+ maxScore = result.maxScore;
+ nDocsReturned = result.topDocs.scoreDocs.length;
}
- Collector collector = MultiCollector.wrap(collectors);
-
- buildAndRunCollectorChain(qr, query, collector, cmd, pf.postFilter);
-
- set = DocSetUtil.getDocSet(setCollector, this);
-
- totalHits = topCollector.getTotalHits();
- assert (totalHits == set.size()) || qr.isPartialResults();
-
- TopDocs topDocs = topCollector.topDocs(0, len);
if (cmd.getSort() != null && query instanceof RankQuery == false && (cmd.getFlags() & GET_SCORES) != 0) {
TopFieldCollector.populateScores(topDocs.scoreDocs, this, query);
}
populateNextCursorMarkFromTopDocs(qr, cmd, topDocs);
- maxScore = totalHits > 0 ? (maxScoreCollector == null ? Float.NaN : maxScoreCollector.getMaxScore()) : 0.0f;
- nDocsReturned = topDocs.scoreDocs.length;
ids = new int[nDocsReturned];
scores = (cmd.getFlags() & GET_SCORES) != 0 ? new float[nDocsReturned] : null;
diff --git a/solr/core/src/test/org/apache/solr/schema/TestOmitPositions.java b/solr/core/src/test/org/apache/solr/schema/TestOmitPositions.java
index 0fcc9fe..c672d68 100644
--- a/solr/core/src/test/org/apache/solr/schema/TestOmitPositions.java
+++ b/solr/core/src/test/org/apache/solr/schema/TestOmitPositions.java
@@ -51,7 +51,8 @@
"//*[@numFound='0']"
);
} catch (Exception expected) {
- assertTrue(expected.getCause() instanceof IllegalStateException);
+ assertTrue(expected.getCause() instanceof IllegalStateException ||
+ expected.getCause().getCause().getCause() instanceof IllegalStateException);
// in lucene 4.0, queries don't silently fail
}
resetExceptionIgnores();
diff --git a/solr/core/src/test/org/apache/solr/search/TestFiltering.java b/solr/core/src/test/org/apache/solr/search/TestFiltering.java
index b2ad9cf..4114595 100644
--- a/solr/core/src/test/org/apache/solr/search/TestFiltering.java
+++ b/solr/core/src/test/org/apache/solr/search/TestFiltering.java
@@ -77,7 +77,7 @@
if (live == null) {
live = searcher.getLiveDocSet();
}
- assertTrue( set == live);
+ assertTrue( set.equals(live));
QueryCommand cmd = new QueryCommand();
cmd.setQuery( QParser.getParser(qstr, null, req).getQuery() );
@@ -86,14 +86,14 @@
QueryResult res = new QueryResult();
searcher.search(res, cmd);
set = res.getDocSet();
- assertTrue( set == live );
+ assertTrue( set.equals(live) );
cmd.setQuery( QParser.getParser(qstr + " OR id:0", null, req).getQuery() );
cmd.setFilterList( QParser.getParser(qstr + " OR id:1", null, req).getQuery() );
res = new QueryResult();
searcher.search(res, cmd);
set = res.getDocSet();
- assertTrue( set == live );
+ assertTrue( set.equals(live) );
}
} finally {