You are viewing a plain text version of this content. The canonical link for it is here.
Posted to solr-commits@lucene.apache.org by yo...@apache.org on 2008/07/06 03:16:13 UTC

svn commit: r674249 - in /lucene/solr/trunk: ./ client/java/solrj/src/org/apache/solr/client/solrj/ client/java/solrj/test/org/apache/solr/client/solrj/ src/java/org/apache/solr/common/params/ src/java/org/apache/solr/handler/ src/java/org/apache/solr/...

Author: yonik
Date: Sat Jul  5 18:16:12 2008
New Revision: 674249

URL: http://svn.apache.org/viewvc?rev=674249&view=rev
Log:
SOLR-502: Add search timeout support

Modified:
    lucene/solr/trunk/CHANGES.txt
    lucene/solr/trunk/client/java/solrj/src/org/apache/solr/client/solrj/SolrQuery.java
    lucene/solr/trunk/client/java/solrj/test/org/apache/solr/client/solrj/SolrQueryTest.java
    lucene/solr/trunk/src/java/org/apache/solr/common/params/CommonParams.java
    lucene/solr/trunk/src/java/org/apache/solr/handler/RequestHandlerBase.java
    lucene/solr/trunk/src/java/org/apache/solr/handler/component/QueryComponent.java
    lucene/solr/trunk/src/java/org/apache/solr/handler/component/ResponseBuilder.java
    lucene/solr/trunk/src/java/org/apache/solr/search/SolrIndexSearcher.java

Modified: lucene/solr/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/CHANGES.txt?rev=674249&r1=674248&r2=674249&view=diff
==============================================================================
--- lucene/solr/trunk/CHANGES.txt (original)
+++ lucene/solr/trunk/CHANGES.txt Sat Jul  5 18:16:12 2008
@@ -305,6 +305,8 @@
 
 57. SOLR-14: Add preserveOriginal flag to WordDelimiterFilter
     (Geoffrey Young, Trey Hyde, Ankur Madnani, yonik)
+
+58. SOLR-502: Add search timeout support. (Sean Timm via yonik)
     
 Changes in runtime behavior
  1. SOLR-559: use Lucene updateDocument, deleteDocuments methods.  This

Modified: lucene/solr/trunk/client/java/solrj/src/org/apache/solr/client/solrj/SolrQuery.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/client/java/solrj/src/org/apache/solr/client/solrj/SolrQuery.java?rev=674249&r1=674248&r2=674249&view=diff
==============================================================================
--- lucene/solr/trunk/client/java/solrj/src/org/apache/solr/client/solrj/SolrQuery.java (original)
+++ lucene/solr/trunk/client/java/solrj/src/org/apache/solr/client/solrj/SolrQuery.java Sat Jul  5 18:16:12 2008
@@ -464,6 +464,30 @@
     return q;
   }
   
+  /**
+  * Set the maximum time allowed for this query. If the query takes more time
+  * than the specified milliseconds, a timeout occurs and partial (or no)
+  * results may be returned.
+  * 
+  * If given Long is null, then this parameter is removed from the request
+  * 
+  *@param milliseconds the time in milliseconds allowed for this query
+  */
+  public void setTimeAllowed(Integer milliseconds) {
+    if (milliseconds == null) {
+      this.remove(CommonParams.TIME_ALLOWED);
+    } else {
+      this.set(CommonParams.TIME_ALLOWED, milliseconds);
+    }
+  }
+  
+  /**
+  * Get the maximum time allowed for this query.
+  */
+  public Integer getTimeAllowed() {
+    return this.getInt(CommonParams.TIME_ALLOWED);
+  }
+
   ///////////////////////
   //  Utility functions
   ///////////////////////

Modified: lucene/solr/trunk/client/java/solrj/test/org/apache/solr/client/solrj/SolrQueryTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/client/java/solrj/test/org/apache/solr/client/solrj/SolrQueryTest.java?rev=674249&r1=674248&r2=674249&view=diff
==============================================================================
--- lucene/solr/trunk/client/java/solrj/test/org/apache/solr/client/solrj/SolrQueryTest.java (original)
+++ lucene/solr/trunk/client/java/solrj/test/org/apache/solr/client/solrj/SolrQueryTest.java Sat Jul  5 18:16:12 2008
@@ -79,6 +79,13 @@
     
     // check to see that the removes are properly clearing the cgi params
     Assert.assertEquals(q.toString(), "q=dog");
+
+    //Add time allowed param
+    q.setTimeAllowed(1000);
+    Assert.assertEquals((Integer)1000, q.getTimeAllowed() );
+    //Adding a null should remove it
+    q.setTimeAllowed(null);
+    Assert.assertEquals(null, q.getTimeAllowed() ); 
     
     System.out.println(q);
   }

Modified: lucene/solr/trunk/src/java/org/apache/solr/common/params/CommonParams.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/common/params/CommonParams.java?rev=674249&r1=674248&r2=674249&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/common/params/CommonParams.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/common/params/CommonParams.java Sat Jul  5 18:16:12 2008
@@ -78,7 +78,11 @@
    * will be used for all of them.  
    */
   public static final String STREAM_CONTENTTYPE = "stream.contentType";
-    
+  
+  /**
+   * Timeout value in milliseconds.  If not set, or the value is <= 0, there is no timeout.
+   */
+  public static final String TIME_ALLOWED = "timeAllowed";
   
   /** 'true' if the header should include the handler name */
   public static final String HEADER_ECHO_HANDLER = "echoHandler";

Modified: lucene/solr/trunk/src/java/org/apache/solr/handler/RequestHandlerBase.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/RequestHandlerBase.java?rev=674249&r1=674248&r2=674249&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/RequestHandlerBase.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/RequestHandlerBase.java Sat Jul  5 18:16:12 2008
@@ -18,6 +18,7 @@
 package org.apache.solr.handler;
 
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
@@ -26,6 +27,7 @@
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrQueryResponse;
 import org.apache.solr.request.SolrRequestHandler;
+import org.apache.solr.search.DocSet;
 import org.apache.solr.util.SolrPluginUtils;
 import org.apache.lucene.queryParser.ParseException;
 
@@ -41,6 +43,7 @@
   // acceptable every million requests or so?
   volatile long numRequests;
   volatile long numErrors;
+  volatile long numTimeouts;
   protected NamedList initArgs = null;
   protected SolrParams defaults;
   protected SolrParams appends;
@@ -123,6 +126,12 @@
     try {
       U.setDefaults(req,defaults,appends,invariants);
       handleRequestBody( req, rsp );
+      // count timeouts
+      boolean timedOut = (Boolean)rsp.getResponseHeader().get("partialResults") == null ? false : (Boolean)rsp.getResponseHeader().get("partialResults");
+      if( timedOut ) {
+        numTimeouts++;
+        rsp.setHttpCaching(false);
+      }
     } catch (Exception e) {
       SolrException.log(SolrCore.log,e);
       if (e instanceof ParseException) {
@@ -158,6 +167,7 @@
     NamedList lst = new SimpleOrderedMap();
     lst.add("requests", numRequests);
     lst.add("errors", numErrors);
+    lst.add("timeouts", numTimeouts);
     lst.add("avgTimePerRequest", (float) totalTime / (float) this.numRequests);
     lst.add("avgRequestsPerSecond", (float) numRequests*1000 / (float)(System.currentTimeMillis()-handlerStart));   
     return lst;

Modified: lucene/solr/trunk/src/java/org/apache/solr/handler/component/QueryComponent.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/component/QueryComponent.java?rev=674249&r1=674248&r2=674249&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/component/QueryComponent.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/component/QueryComponent.java Sat Jul  5 18:16:12 2008
@@ -61,7 +61,7 @@
     SolrQueryRequest req = rb.req;
     SolrQueryResponse rsp = rb.rsp;
     SolrParams params = req.getParams();
-    
+
     // Set field flags
     String fl = params.get(CommonParams.FL);
     int fieldFlags = 0;
@@ -120,6 +120,9 @@
     SolrIndexSearcher searcher = req.getSearcher();
     SolrParams params = req.getParams();
 
+    // -1 as flag if not set.
+    long timeAllowed = (long)params.getInt( CommonParams.TIME_ALLOWED, -1 );
+
     // Optional: This could also be implemented by the top-level searcher sending
     // a filter that lists the ids... that would be transparent to
     // the request handler, but would be more expensive (and would preserve score
@@ -151,20 +154,11 @@
       return;
     }
 
-    if( rb.isNeedDocSet() ) {
-      rb.setResults( searcher.getDocListAndSet(
-          rb.getQuery(), rb.getFilters(), rb.getSortSpec().getSort(),
-          rb.getSortSpec().getOffset(), rb.getSortSpec().getCount(),
-          rb.getFieldFlags() ) );
-    }
-    else {
-      DocListAndSet results = new DocListAndSet();
-      results.docList = searcher.getDocList(
-          rb.getQuery(), rb.getFilters(), rb.getSortSpec().getSort(),
-          rb.getSortSpec().getOffset(), rb.getSortSpec().getCount(),
-          rb.getFieldFlags() );
-      rb.setResults( results );
-    }
+    SolrIndexSearcher.QueryCommand cmd = rb.getQueryCommand();
+    cmd.setTimeAllowed(timeAllowed);
+    SolrIndexSearcher.QueryResult result = new SolrIndexSearcher.QueryResult();
+    searcher.search(result,cmd);
+    rb.setResult( result );
 
     rsp.add("response",rb.getResults().docList);
     rsp.getToLog().add("hits", rb.getResults().docList.size());
@@ -432,7 +426,6 @@
       if (maxScore!=null) responseDocs.setMaxScore(maxScore);
       responseDocs.setNumFound(numFound);
       responseDocs.setStart(ss.getOffset());
-
       // size appropriately
       for (int i=0; i<resultSize; i++) responseDocs.add(null);
 
@@ -504,7 +497,6 @@
     // could/should bypass middlemen (like retrieving stored fields)
     // TODO: merge fsv to if requested
 
-
     if ((sreq.purpose & ShardRequest.PURPOSE_GET_FIELDS) != 0) {
       boolean returnScores = (rb.getFieldFlags() & SolrIndexSearcher.GET_SCORES) != 0;
 

Modified: lucene/solr/trunk/src/java/org/apache/solr/handler/component/ResponseBuilder.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/component/ResponseBuilder.java?rev=674249&r1=674248&r2=674249&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/component/ResponseBuilder.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/component/ResponseBuilder.java Sat Jul  5 18:16:12 2008
@@ -27,6 +27,7 @@
 import org.apache.solr.search.DocListAndSet;
 import org.apache.solr.search.QParser;
 import org.apache.solr.search.SortSpec;
+import org.apache.solr.search.SolrIndexSearcher;
 
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -261,4 +262,29 @@
     }
   }
 
+  /**
+   * Creates a SolrIndexSearcher.QueryCommand from this
+   * ResponseBuilder.  TimeAllowed is left unset.
+   */
+  public SolrIndexSearcher.QueryCommand getQueryCommand() {
+    SolrIndexSearcher.QueryCommand cmd = new SolrIndexSearcher.QueryCommand();
+    cmd.setQuery( getQuery() )
+      .setFilterList( getFilters() )
+      .setSort( getSortSpec().getSort() )
+      .setOffset( getSortSpec().getOffset() )
+      .setLen( getSortSpec().getCount() )
+      .setFlags( getFieldFlags() )
+      .setNeedDocSet( isNeedDocSet() );
+    return cmd;
+  }
+
+  /**
+   * Sets results from a SolrIndexSearcher.QueryResult.
+   */
+  public void setResult( SolrIndexSearcher.QueryResult result ) {
+    setResults( result.getDocListAndSet() );
+    if( result.isPartialResults() ) {
+      rsp.getResponseHeader().add( "partialResults", Boolean.TRUE );
+    }
+  }
 }

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=674249&r1=674248&r2=674249&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/SolrIndexSearcher.java Sat Jul  5 18:16:12 2008
@@ -248,10 +248,16 @@
                     nDocs = Math.min(oldnDocs,40);
                   }
 
-                  DocListAndSet ret = new DocListAndSet();
                   int flags=NO_CHECK_QCACHE | key.nc_flags;
-
-                  newSearcher.getDocListC(ret, key.query, key.filters, null, key.sort, 0, nDocs, flags);
+                  QueryCommand qc = new QueryCommand();
+                  qc.setQuery(key.query)
+                    .setFilterList(key.filters)
+                    .setSort(key.sort)
+                    .setLen(nDocs)
+                    .setSupersetMaxDoc(nDocs)
+                    .setFlags(flags);
+                  QueryResult qr = new QueryResult();
+                  newSearcher.getDocListC(qr,qc);
                   return true;
                 }
               }
@@ -259,6 +265,11 @@
     }
   }
 
+  public QueryResult search(QueryResult qr, QueryCommand cmd) throws IOException {
+    getDocListC(qr,cmd);
+    return qr;
+  }
+
   public Hits search(Query query, Filter filter, Sort sort) throws IOException {
     // todo - when Solr starts accepting filters, need to
     // change this conditional check (filter!=null) and create a new filter
@@ -669,12 +680,15 @@
    * @throws IOException
    */
   public DocList getDocList(Query query, Query filter, Sort lsort, int offset, int len) throws IOException {
-    List<Query> filterList = null;
-    if (filter != null) {
-      filterList = new ArrayList<Query>(1);
-      filterList.add(filter);
-    }
-    return getDocList(query, filterList, lsort, offset, len, 0);
+    QueryCommand qc = new QueryCommand();
+    qc.setQuery(query)
+      .setFilterList(filter)
+      .setSort(lsort)
+      .setOffset(offset)
+      .setLen(len);
+    QueryResult qr = new QueryResult();
+    search(qr,qc);
+    return qr.getDocList();
   }
 
 
@@ -696,59 +710,69 @@
    * @throws IOException
    */
   public DocList getDocList(Query query, List<Query> filterList, Sort lsort, int offset, int len, int flags) throws IOException {
-    DocListAndSet answer = new DocListAndSet();
-    getDocListC(answer,query,filterList,null,lsort,offset,len,flags);
-    return answer.docList;
+    QueryCommand qc = new QueryCommand();
+    qc.setQuery(query)
+      .setFilterList(filterList)
+      .setSort(lsort)
+      .setOffset(offset)
+      .setLen(len)
+      .setFlags(flags);
+    QueryResult qr = new QueryResult();
+    search(qr,qc);
+    return qr.getDocList();
   }
 
-
   private static final int NO_CHECK_QCACHE       = 0x80000000;
   private static final int GET_DOCSET            = 0x40000000;
   private static final int NO_CHECK_FILTERCACHE  = 0x20000000;
 
   public static final int GET_SCORES             =       0x01;
 
-  /** getDocList version that uses+populates query and filter caches.
-   * This should only be called using either filterList or filter, but not both.
+  /**
+   * getDocList version that uses+populates query and filter caches.
+   * In the event of a timeout, the cache is not populated.
    */
-  private void getDocListC(DocListAndSet out, Query query, List<Query> filterList, DocSet filter, Sort lsort, int offset, int len, int flags) throws IOException {
+  private void getDocListC(QueryResult qr, QueryCommand cmd) throws IOException {
+    // old parameters: DocListAndSet out, Query query, List<Query> filterList, DocSet filter, Sort lsort, int offset, int len, int flags, long timeAllowed, NamedList<Object> responseHeader
+    DocListAndSet out = new DocListAndSet();
+    qr.setDocListAndSet(out);
     QueryResultKey key=null;
-    int maxDocRequested = offset + len;
+    int maxDocRequested = cmd.getOffset() + cmd.getLen();
     // check for overflow, and check for # docs in index
     if (maxDocRequested < 0 || maxDocRequested > maxDoc()) maxDocRequested = maxDoc();
     int supersetMaxDoc= maxDocRequested;
     DocList superset;
 
-
     // we can try and look up the complete query in the cache.
     // we can't do that if filter!=null though (we don't want to
     // do hashCode() and equals() for a big DocSet).
-    if (queryResultCache != null && filter==null) {
+    if (queryResultCache != null && cmd.getFilter()==null) {
         // all of the current flags can be reused during warming,
         // so set all of them on the cache key.
-        key = new QueryResultKey(query, filterList, lsort, flags);
-        if ((flags & NO_CHECK_QCACHE)==0) {
+        key = new QueryResultKey(cmd.getQuery(), cmd.getFilterList(), cmd.getSort(), cmd.getFlags());
+        if ((cmd.getFlags() & NO_CHECK_QCACHE)==0) {
           superset = (DocList)queryResultCache.get(key);
 
           if (superset != null) {
             // check that the cache entry has scores recorded if we need them
-            if ((flags & GET_SCORES)==0 || superset.hasScores()) {
+            if ((cmd.getFlags() & GET_SCORES)==0 || superset.hasScores()) {
               // NOTE: subset() returns null if the DocList has fewer docs than
               // requested
-              out.docList = superset.subset(offset,len);
+              out.docList = superset.subset(cmd.getOffset(),cmd.getLen());
             }
           }
           if (out.docList != null) {
             // found the docList in the cache... now check if we need the docset too.
             // OPT: possible future optimization - if the doclist contains all the matches,
             // use it to make the docset instead of rerunning the query.
-            if (out.docSet==null && ((flags & GET_DOCSET)!=0) ) {
-              if (filterList==null) {
-                out.docSet = getDocSet(query);
+            if (out.docSet==null && ((cmd.getFlags() & GET_DOCSET)!=0) ) {
+              if (cmd.getFilterList()==null) {
+                out.docSet = getDocSet(cmd.getQuery());
               } else {
-                List<Query> newList = new ArrayList<Query>(filterList.size()+1);
-                newList.add(query);
-                newList.addAll(filterList);
+                List<Query> newList = new ArrayList<Query>(cmd.getFilterList()
+.size()+1);
+                newList.add(cmd.getQuery());
+                newList.addAll(cmd.getFilterList());
                 out.docSet = getDocSet(newList);
               }
             }
@@ -778,9 +802,9 @@
 
     // check if we should try and use the filter cache
     boolean useFilterCache=false;
-    if ((flags & (GET_SCORES|NO_CHECK_FILTERCACHE))==0 && useFilterForSortedQuery && lsort != null && filterCache != null) {
+    if ((cmd.getFlags() & (GET_SCORES|NO_CHECK_FILTERCACHE))==0 && useFilterForSortedQuery && cmd.getSort() != null && filterCache != null) {
       useFilterCache=true;
-      SortField[] sfields = lsort.getSort();
+      SortField[] sfields = cmd.getSort().getSort();
       for (SortField sf : sfields) {
         if (sf.getType() == SortField.SCORE) {
           useFilterCache=false;
@@ -794,41 +818,46 @@
       // for large filters that match few documents, this may be
       // slower than simply re-executing the query.
       if (out.docSet == null) {
-        out.docSet = getDocSet(query,filter);
-        DocSet bigFilt = getDocSet(filterList);
+        out.docSet = getDocSet(cmd.getQuery(),cmd.getFilter());
+        DocSet bigFilt = getDocSet(cmd.getFilterList());
         if (bigFilt != null) out.docSet = out.docSet.intersection(bigFilt);
       }
       // todo: there could be a sortDocSet that could take a list of
       // the filters instead of anding them first...
       // perhaps there should be a multi-docset-iterator
-      superset = sortDocSet(out.docSet,lsort,supersetMaxDoc);
-      out.docList = superset.subset(offset,len);
+      superset = sortDocSet(out.docSet,cmd.getSort(),supersetMaxDoc);
+      out.docList = superset.subset(cmd.getOffset(),cmd.getLen());
     } else {
       // do it the normal way...
-      DocSet theFilt = filter!=null ? filter : getDocSet(filterList);
-
-      if ((flags & GET_DOCSET)!=0) {
-        DocSet qDocSet = getDocListAndSetNC(out,query,theFilt,lsort,0,supersetMaxDoc,flags);
+      cmd.setSupersetMaxDoc(supersetMaxDoc);
+      if ((cmd.getFlags() & GET_DOCSET)!=0) {
+        DocSet qDocSet = getDocListAndSetNC(qr,cmd);
         // cache the docSet matching the query w/o filtering
-        if (filterCache!=null) filterCache.put(query,qDocSet);
+        if (filterCache!=null && !qr.isPartialResults()) filterCache.put(cmd.getQuery(),qDocSet);
       } else {
-        out.docList = getDocListNC(query,theFilt,lsort,0,supersetMaxDoc,flags);
+        getDocListNC(qr,cmd);
+        //Parameters: cmd.getQuery(),theFilt,cmd.getSort(),0,supersetMaxDoc,cmd.getFlags(),cmd.getTimeAllowed(),responseHeader);
       }
       superset = out.docList;
-      out.docList = superset.subset(offset,len);
+      out.docList = superset.subset(cmd.getOffset(),cmd.getLen());
     }
 
     // lastly, put the superset in the cache if the size is less than or equal
     // to queryResultMaxDocsCached
-    if (key != null && superset.size() <= queryResultMaxDocsCached) {
+    if (key != null && superset.size() <= queryResultMaxDocsCached && !qr.isPartialResults()) {
       queryResultCache.put(key, superset);
     }
   }
 
 
 
-  private DocList getDocListNC(Query query, DocSet filter, Sort lsort, int offset, int len, int flags) throws IOException {
-    int last = offset+len;
+  private void getDocListNC(QueryResult qr,QueryCommand cmd) throws IOException {
+    //Parameters: cmd.getQuery(),theFilt,cmd.getSort(),0,supersetMaxDoc,cmd.getFlags(),cmd.getTimeAllowed(),responseHeader);
+    //Query query, DocSet filter, Sort lsort, int offset, int len, int flags, long timeAllowed, NamedList<Object> responseHeader
+    DocSet filter = cmd.getFilter()!=null ? cmd.getFilter() : getDocSet(cmd.getFilterList());
+    final long timeAllowed = cmd.getTimeAllowed();
+    int len = cmd.getSupersetMaxDoc();
+    int last = len;
     if (last < 0 || last > maxDoc()) last=maxDoc();
     final int lastDocRequested = last;
     int nDocsReturned;
@@ -837,7 +866,7 @@
     int[] ids;
     float[] scores;
 
-    query = QueryUtils.makeQueryable(query);
+    Query query = QueryUtils.makeQueryable(cmd.getQuery());
 
     // handle zero case...
     if (lastDocRequested<=0) {
@@ -845,44 +874,62 @@
       final float[] topscore = new float[] { Float.NEGATIVE_INFINITY };
       final int[] numHits = new int[1];
 
-      searcher.search(query, new HitCollector() {
+      HitCollector hc = new HitCollector() {
         public void collect(int doc, float score) {
           if (filt!=null && !filt.exists(doc)) return;
           numHits[0]++;
           if (score > topscore[0]) topscore[0]=score;
         }
+      };
+      if( timeAllowed > 0 ) {
+        hc = new TimeLimitedCollector( hc, timeAllowed );
+      }
+      try {
+        searcher.search(query, hc );
+      }
+      catch( TimeLimitedCollector.TimeExceededException x ) {
+        log.warning( "Query: " + query + "; " + x.getMessage() );
+        qr.setPartialResults(true);
       }
-      );
 
       nDocsReturned=0;
       ids = new int[nDocsReturned];
       scores = new float[nDocsReturned];
       totalHits = numHits[0];
       maxScore = totalHits>0 ? topscore[0] : 0.0f;
-    } else if (lsort != null) {
+    } else if (cmd.getSort() != null) {
       // can't use TopDocs if there is a sort since it
       // will do automatic score normalization.
       // NOTE: this changed late in Lucene 1.9
 
       final DocSet filt = filter;
       final int[] numHits = new int[1];
-      final FieldSortedHitQueue hq = new FieldSortedHitQueue(reader, lsort.getSort(), offset+len);
+      final FieldSortedHitQueue hq = new FieldSortedHitQueue(reader, cmd.getSort().getSort(), len);
 
-      searcher.search(query, new HitCollector() {
+      HitCollector hc = new HitCollector() {
         public void collect(int doc, float score) {
           if (filt!=null && !filt.exists(doc)) return;
           numHits[0]++;
           hq.insert(new FieldDoc(doc, score));
         }
+      };
+      if( timeAllowed > 0 ) {
+        hc = new TimeLimitedCollector( hc, timeAllowed );
+      }
+      try {
+        searcher.search(query, hc );
+      }
+      catch( TimeLimitedCollector.TimeExceededException x ) {
+        log.warning( "Query: " + query + "; " + x.getMessage() );
+        qr.setPartialResults(true);
       }
-      );
 
       totalHits = numHits[0];
       maxScore = totalHits>0 ? hq.getMaxScore() : 0.0f;
 
       nDocsReturned = hq.size();
       ids = new int[nDocsReturned];
-      scores = (flags&GET_SCORES)!=0 ? new float[nDocsReturned] : null;
+      scores = (cmd.getFlags()&GET_SCORES)!=0 ? new float[nDocsReturned] : null;
       for (int i = nDocsReturned -1; i >= 0; i--) {
         FieldDoc fieldDoc = (FieldDoc)hq.pop();
         // fillFields is the point where score normalization happens
@@ -898,7 +945,7 @@
       final DocSet filt = filter;
       final ScorePriorityQueue hq = new ScorePriorityQueue(lastDocRequested);
       final int[] numHits = new int[1];
-      searcher.search(query, new HitCollector() {
+      HitCollector hc = new HitCollector() {
         float minScore=Float.NEGATIVE_INFINITY;  // minimum score in the priority queue
         public void collect(int doc, float score) {
           if (filt!=null && !filt.exists(doc)) return;
@@ -911,13 +958,22 @@
             minScore = ((ScoreDoc)hq.top()).score;
           }
         }
+      };
+      if( timeAllowed > 0 ) {
+        hc = new TimeLimitedCollector( hc, timeAllowed );
+      }
+      try {
+        searcher.search(query, hc );
+      }
+      catch( TimeLimitedCollector.TimeExceededException x ) {
+        log.warning( "Query: " + query + "; " + x.getMessage() );
+        qr.setPartialResults(true);
       }
-      );
 
       totalHits = numHits[0];
       nDocsReturned = hq.size();
       ids = new int[nDocsReturned];
-      scores = (flags&GET_SCORES)!=0 ? new float[nDocsReturned] : null;
+      scores = (cmd.getFlags()&GET_SCORES)!=0 ? new float[nDocsReturned] : null;
       ScoreDoc sdoc =null;
       for (int i = nDocsReturned -1; i >= 0; i--) {
         sdoc = (ScoreDoc)hq.pop();
@@ -928,9 +984,9 @@
     }
 
 
-    int sliceLen = Math.min(lastDocRequested,nDocsReturned) - offset;
+    int sliceLen = Math.min(lastDocRequested,nDocsReturned);
     if (sliceLen < 0) sliceLen=0;
-    return new DocSlice(offset,sliceLen,ids,scores,totalHits,maxScore);
+    qr.setDocList(new DocSlice(0,sliceLen,ids,scores,totalHits,maxScore));
 
 
 
@@ -987,8 +1043,10 @@
 
   // the DocSet returned is for the query only, without any filtering... that way it may
   // be cached if desired.
-  private DocSet getDocListAndSetNC(DocListAndSet out, Query query, DocSet filter, Sort lsort, int offset, int len, int flags) throws IOException {
-    int last = offset+len;
+  private DocSet getDocListAndSetNC(QueryResult qr,QueryCommand cmd) throws IOException {
+    int len = cmd.getSupersetMaxDoc();
+    DocSet filter = cmd.getFilter()!=null ? cmd.getFilter() : getDocSet(cmd.getFilterList());
+    int last = len;
     if (last < 0 || last > maxDoc()) last=maxDoc();
     final int lastDocRequested = last;
     int nDocsReturned;
@@ -997,8 +1055,9 @@
     int[] ids;
     float[] scores;
     final DocSetHitCollector setHC = new DocSetHitCollector(HASHSET_INVERSE_LOAD_FACTOR, HASHDOCSET_MAXSIZE, maxDoc());
+    final HitCollector hitCollector = ( cmd.getTimeAllowed() > 0 ) ? new TimeLimitedCollector( setHC, cmd.getTimeAllowed() ) : setHC;
 
-    query = QueryUtils.makeQueryable(query);
+    Query query = QueryUtils.makeQueryable(cmd.getQuery());
 
     // TODO: perhaps unify getDocListAndSetNC and getDocListNC without imposing a significant performance hit
 
@@ -1018,46 +1077,58 @@
       final float[] topscore = new float[] { Float.NEGATIVE_INFINITY };
       final int[] numHits = new int[1];
 
-      searcher.search(query, new HitCollector() {
-        public void collect(int doc, float score) {
-          setHC.collect(doc,score);
-          if (filt!=null && !filt.exists(doc)) return;
-          numHits[0]++;
-          if (score > topscore[0]) topscore[0]=score;
+      try {
+        searcher.search(query, new HitCollector() {
+          public void collect(int doc, float score) {
+            hitCollector.collect(doc,score);
+            if (filt!=null && !filt.exists(doc)) return;
+            numHits[0]++;
+            if (score > topscore[0]) topscore[0]=score;
+          }
         }
+        );
+      }
+      catch( TimeLimitedCollector.TimeExceededException x ) {
+        log.warning( "Query: " + query + "; " + x.getMessage() );
+        qr.setPartialResults(true);
       }
-      );
 
       nDocsReturned=0;
       ids = new int[nDocsReturned];
       scores = new float[nDocsReturned];
       totalHits = numHits[0];
       maxScore = totalHits>0 ? topscore[0] : 0.0f;
-    } else if (lsort != null) {
+    } else if (cmd.getSort() != null) {
       // can't use TopDocs if there is a sort since it
       // will do automatic score normalization.
       // NOTE: this changed late in Lucene 1.9
 
       final DocSet filt = filter;
       final int[] numHits = new int[1];
-      final FieldSortedHitQueue hq = new FieldSortedHitQueue(reader, lsort.getSort(), offset+len);
+      final FieldSortedHitQueue hq = new FieldSortedHitQueue(reader, cmd.getSort().getSort(), len);
 
-      searcher.search(query, new HitCollector() {
-        public void collect(int doc, float score) {
-          setHC.collect(doc,score);
-          if (filt!=null && !filt.exists(doc)) return;
-          numHits[0]++;
-          hq.insert(new FieldDoc(doc, score));
+      try {
+        searcher.search(query, new HitCollector() {
+          public void collect(int doc, float score) {
+            hitCollector.collect(doc,score);
+            if (filt!=null && !filt.exists(doc)) return;
+            numHits[0]++;
+            hq.insert(new FieldDoc(doc, score));
+          }
         }
+        );
+      }
+      catch( TimeLimitedCollector.TimeExceededException x ) {
+        log.warning( "Query: " + query + "; " + x.getMessage() );
+        qr.setPartialResults(true);
       }
-      );
 
       totalHits = numHits[0];
       maxScore = totalHits>0 ? hq.getMaxScore() : 0.0f;
 
       nDocsReturned = hq.size();
       ids = new int[nDocsReturned];
-      scores = (flags&GET_SCORES)!=0 ? new float[nDocsReturned] : null;
+      scores = (cmd.getFlags()&GET_SCORES)!=0 ? new float[nDocsReturned] : null;
       for (int i = nDocsReturned -1; i >= 0; i--) {
         FieldDoc fieldDoc = (FieldDoc)hq.pop();
         // fillFields is the point where score normalization happens
@@ -1073,25 +1144,31 @@
       final DocSet filt = filter;
       final ScorePriorityQueue hq = new ScorePriorityQueue(lastDocRequested);
       final int[] numHits = new int[1];
-      searcher.search(query, new HitCollector() {
-        float minScore=Float.NEGATIVE_INFINITY;  // minimum score in the priority queue
-        public void collect(int doc, float score) {
-          setHC.collect(doc,score);
-          if (filt!=null && !filt.exists(doc)) return;
-          if (numHits[0]++ < lastDocRequested || score >= minScore) {
-            // if docs are always delivered in order, we could use "score>minScore"
-            // but might BooleanScorer14 might still be used and deliver docs out-of-order?
-            hq.insert(new ScoreDoc(doc, score));
-            minScore = ((ScoreDoc)hq.top()).score;
+      try {
+        searcher.search(query, new HitCollector() {
+          float minScore=Float.NEGATIVE_INFINITY;  // minimum score in the priority queue
+          public void collect(int doc, float score) {
+            hitCollector.collect(doc,score);
+            if (filt!=null && !filt.exists(doc)) return;
+            if (numHits[0]++ < lastDocRequested || score >= minScore) {
+              // if docs are always delivered in order, we could use "score>minScore"
+              // but might BooleanScorer14 might still be used and deliver docs out-of-order?
+              hq.insert(new ScoreDoc(doc, score));
+              minScore = ((ScoreDoc)hq.top()).score;
+            }
           }
         }
+        );
+      }
+      catch( TimeLimitedCollector.TimeExceededException x ) {
+        log.warning( "Query: " + query + "; " + x.getMessage() );
+        qr.setPartialResults(true);
       }
-      );
 
       totalHits = numHits[0];
       nDocsReturned = hq.size();
       ids = new int[nDocsReturned];
-      scores = (flags&GET_SCORES)!=0 ? new float[nDocsReturned] : null;
+      scores = (cmd.getFlags()&GET_SCORES)!=0 ? new float[nDocsReturned] : null;
       ScoreDoc sdoc =null;
       for (int i = nDocsReturned -1; i >= 0; i--) {
         sdoc = (ScoreDoc)hq.pop();
@@ -1102,11 +1179,12 @@
     }
 
 
-    int sliceLen = Math.min(lastDocRequested,nDocsReturned) - offset;
+    int sliceLen = Math.min(lastDocRequested,nDocsReturned);
     if (sliceLen < 0) sliceLen=0;
-    out.docList = new DocSlice(offset,sliceLen,ids,scores,totalHits,maxScore);
+    
+    qr.setDocList(new DocSlice(0,sliceLen,ids,scores,totalHits,maxScore));
     DocSet qDocSet = setHC.getDocSet();
-    out.docSet = filter==null ? qDocSet : qDocSet.intersection(filter);
+    qr.setDocSet(filter==null ? qDocSet : qDocSet.intersection(filter));
     return qDocSet;
   }
 
@@ -1126,9 +1204,15 @@
    * @throws IOException
    */
   public DocList getDocList(Query query, DocSet filter, Sort lsort, int offset, int len) throws IOException {
-    DocListAndSet answer = new DocListAndSet();
-    getDocListC(answer,query,null,filter,lsort,offset,len,0);
-    return answer.docList;
+    QueryCommand qc = new QueryCommand();
+    qc.setQuery(query)
+      .setFilter(filter)
+      .setSort(lsort)
+      .setOffset(offset)
+      .setLen(len);
+    QueryResult qr = new QueryResult();
+    search(qr,qc);
+    return qr.getDocList();
   }
 
   /**
@@ -1152,9 +1236,16 @@
    * @throws IOException
    */
   public DocListAndSet getDocListAndSet(Query query, Query filter, Sort lsort, int offset, int len) throws IOException {
-    List<Query> filterList = buildQueryList(filter);
-    return getDocListAndSet(query, filterList, lsort, offset, len);
-
+    QueryCommand qc = new QueryCommand();
+    qc.setQuery(query)
+      .setFilterList(filter)
+      .setSort(lsort)
+      .setOffset(offset)
+      .setLen(len)
+      .setNeedDocSet(true);
+    QueryResult qr = new QueryResult();
+    search(qr,qc);
+    return qr.getDocListAndSet();
   }
 
   /**
@@ -1179,22 +1270,19 @@
    * @throws IOException
    */
   public DocListAndSet getDocListAndSet(Query query, Query filter, Sort lsort, int offset, int len, int flags) throws IOException {
-	List<Query> filterList = buildQueryList(filter);
-	return getDocListAndSet(query, filterList, lsort, offset, len, flags);
+    QueryCommand qc = new QueryCommand();
+    qc.setQuery(query)
+      .setFilterList(filter)
+      .setSort(lsort)
+      .setOffset(offset)
+      .setLen(len)
+      .setFlags(flags)
+      .setNeedDocSet(true);
+    QueryResult qr = new QueryResult();
+    search(qr,qc);
+    return qr.getDocListAndSet();
   }
   
-  /**
-   * A simple utility method for to build a filterList from a query
-   * @param filter
-   */
-  private List<Query> buildQueryList(Query filter) {
-	List<Query> filterList = null;
-	if (filter != null) {
-	  filterList = new ArrayList<Query>(2);
-	  filterList.add(filter);
-	}
-	return filterList;
-  }
 
   /**
    * Returns documents matching both <code>query</code> and the intersection 
@@ -1219,9 +1307,16 @@
    * @throws IOException
    */
   public DocListAndSet getDocListAndSet(Query query, List<Query> filterList, Sort lsort, int offset, int len) throws IOException {
-    DocListAndSet ret = new DocListAndSet();
-    getDocListC(ret,query,filterList,null,lsort,offset,len,GET_DOCSET);
-    return ret;
+    QueryCommand qc = new QueryCommand();
+    qc.setQuery(query)
+      .setFilterList(filterList)
+      .setSort(lsort)
+      .setOffset(offset)
+      .setLen(len)
+      .setNeedDocSet(true);
+    QueryResult qr = new QueryResult();
+    search(qr,qc);
+    return qr.getDocListAndSet();
   }
 
   /**
@@ -1248,9 +1343,17 @@
    * @throws IOException
    */
   public DocListAndSet getDocListAndSet(Query query, List<Query> filterList, Sort lsort, int offset, int len, int flags) throws IOException {
-	    DocListAndSet ret = new DocListAndSet();
-	    getDocListC(ret,query,filterList,null,lsort,offset,len, flags |= GET_DOCSET);
-	    return ret;
+    QueryCommand qc = new QueryCommand();
+    qc.setQuery(query)
+      .setFilterList(filterList)
+      .setSort(lsort)
+      .setOffset(offset)
+      .setLen(len)
+      .setFlags(flags)
+      .setNeedDocSet(true);
+    QueryResult qr = new QueryResult();
+    search(qr,qc);
+    return qr.getDocListAndSet();
   }
 
   /**
@@ -1269,9 +1372,16 @@
    * @throws IOException
    */
   public DocListAndSet getDocListAndSet(Query query, DocSet filter, Sort lsort, int offset, int len) throws IOException {
-    DocListAndSet ret = new DocListAndSet();
-    getDocListC(ret,query,null,filter,lsort,offset,len,GET_DOCSET);
-    return ret;
+    QueryCommand qc = new QueryCommand();
+    qc.setQuery(query)
+      .setFilter(filter)
+      .setSort(lsort)
+      .setOffset(offset)
+      .setLen(len)
+      .setNeedDocSet(true);
+    QueryResult qr = new QueryResult();
+    search(qr,qc);
+    return qr.getDocListAndSet();
   }
 
   /**
@@ -1296,10 +1406,18 @@
    * @throws IOException
    */
   public DocListAndSet getDocListAndSet(Query query, DocSet filter, Sort lsort, int offset, int len, int flags) throws IOException {
-	    DocListAndSet ret = new DocListAndSet();
-	    getDocListC(ret,query,null,filter,lsort,offset,len, flags |= GET_DOCSET);
-	    return ret;
-	  }
+    QueryCommand qc = new QueryCommand();
+    qc.setQuery(query)
+      .setFilter(filter)
+      .setSort(lsort)
+      .setOffset(offset)
+      .setLen(len)
+      .setFlags(flags)
+      .setNeedDocSet(true);
+    QueryResult qr = new QueryResult();
+    search(qr,qc);
+    return qr.getDocListAndSet();
+  }
 
   protected DocList sortDocSet(DocSet set, Sort sort, int nDocs) throws IOException {
     final FieldSortedHitQueue hq =
@@ -1473,9 +1591,154 @@
     if (registerTime!=0) lst.add("registeredAt", new Date(registerTime));
     return lst;
   }
-}
 
+  /**
+   * A query request command to avoid having to change the method signatures
+   * if we want to pass additional information to the searcher.
+   */
+  public static class QueryCommand {
+    private Query query;
+    private List<Query> filterList;
+    private DocSet filter;
+    private Sort sort;
+    private int offset;
+    private int len;
+    private int supersetMaxDoc;
+    private int flags;
+    private long timeAllowed = -1;
+    private boolean needDocSet;
+
+    public Query getQuery() { return query; }
+    public QueryCommand setQuery(Query query) {
+      this.query = query;
+      return this;
+    }
+    
+    public List<Query> getFilterList() { return filterList; }
+    /**
+     * @throws IllegalArgumentException if filter is not null.
+     */
+    public QueryCommand setFilterList(List<Query> filterList) {
+      if( filter != null ) {
+        throw new IllegalArgumentException( "Either filter or filterList may be set in the QueryCommand, but not both." );
+      }
+      this.filterList = filterList;
+      return this;
+    }
+    /**
+     * A simple setter to build a filterList from a query
+     * @throws IllegalArgumentException if filter is not null.
+     */
+    public QueryCommand setFilterList(Query f) {
+      if( filter != null ) {
+        throw new IllegalArgumentException( "Either filter or filterList may be set in the QueryCommand, but not both." );
+      }
+      filterList = null;
+      if (f != null) {
+        filterList = new ArrayList<Query>(2);
+        filterList.add(f);
+      }
+      return this;
+    }
+    
+    public DocSet getFilter() { return filter; }
+    /**
+     * @throws IllegalArgumentException if filterList is not null.
+     */
+    public QueryCommand setFilter(DocSet filter) {
+      if( filterList != null ) {
+        throw new IllegalArgumentException( "Either filter or filterList may be set in the QueryCommand, but not both." );
+      }
+      this.filter = filter;
+      return this;
+    }
+
+    public Sort getSort() { return sort; }
+    public QueryCommand setSort(Sort sort) {
+      this.sort = sort;
+      return this;
+    }
+    
+    public int getOffset() { return offset; }
+    public QueryCommand setOffset(int offset) {
+      this.offset = offset;
+      return this;
+    }
+    
+    public int getLen() { return len; }
+    public QueryCommand setLen(int len) {
+      this.len = len;
+      return this;
+    }
+    
+    public int getSupersetMaxDoc() { return supersetMaxDoc; }
+    public QueryCommand setSupersetMaxDoc(int supersetMaxDoc) {
+      this.supersetMaxDoc = supersetMaxDoc;
+      return this;
+    }
+
+    public int getFlags() {
+      return flags;
+    }
 
+    public QueryCommand replaceFlags(int flags) {
+      this.flags = flags;
+      return this;
+    }
+
+    public QueryCommand setFlags(int flags) {
+      this.flags |= flags;
+      return this;
+    }
+
+    public QueryCommand clearFlags(int flags) {
+      this.flags &= ~flags;
+      return this;
+    }
+
+    public long getTimeAllowed() { return timeAllowed; }
+    public QueryCommand setTimeAllowed(long timeAllowed) {
+      this.timeAllowed = timeAllowed;
+      return this;
+    }
+    
+    public boolean isNeedDocSet() { return (flags & GET_DOCSET) != 0; }
+    public QueryCommand setNeedDocSet(boolean needDocSet) {
+      return needDocSet ? setFlags(GET_DOCSET) : clearFlags(GET_DOCSET);
+    }
+  }
+
+  /**
+   * The result of a search.
+   */
+  public static class QueryResult {
+    private boolean partialResults;
+    private DocListAndSet docListAndSet;
+    
+    public DocList getDocList() { return docListAndSet.docList; }
+    public void setDocList(DocList list) {
+      if( docListAndSet == null ) {
+        docListAndSet = new DocListAndSet();
+      }
+      docListAndSet.docList = list;
+    }
+
+    public DocSet getDocSet() { return docListAndSet.docSet; }
+    public void setDocSet(DocSet set) {
+      if( docListAndSet == null ) {
+        docListAndSet = new DocListAndSet();
+      }
+      docListAndSet.docSet = set;
+    }
+
+    public boolean isPartialResults() { return partialResults; }
+    public void setPartialResults(boolean partialResults) { this.partialResults = partialResults; }
+
+    public void setDocListAndSet( DocListAndSet listSet ) { docListAndSet = listSet; }
+    public DocListAndSet getDocListAndSet() { return docListAndSet; }
+  }
+
+}
 
 // Lucene's HitQueue isn't public, so here is our own.
 final class ScorePriorityQueue extends PriorityQueue {
@@ -1492,4 +1755,3 @@
 }
 
 
-