You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by an...@apache.org on 2014/09/25 21:42:08 UTC

svn commit: r1627622 - in /lucene/dev/trunk: lucene/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/src/java/org/apache/lucene/search/ lucene/core/src/test/org/apache/lucene/index/ lucene/core/src/test/org/apache/lucene/search/ solr/ solr/co...

Author: anshum
Date: Thu Sep 25 19:42:07 2014
New Revision: 1627622

URL: http://svn.apache.org/r1627622
Log:
SOLR-5986: Add an ExitableDirectoryReader in Lucene and use that to support exiting of long running queries in Solr.

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ExitableDirectoryReader.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/QueryTimeout.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/QueryTimeoutImpl.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestExitableDirectoryReader.java   (with props)
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrQueryTimeoutImpl.java   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CloudExitableDirectoryReaderTest.java   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/ExitableDirectoryReaderTest.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/MoreLikeThisHandler.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestDistributedSearch.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/SolrParams.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1627622&r1=1627621&r2=1627622&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Thu Sep 25 19:42:07 2014
@@ -66,6 +66,10 @@ New Features
 
 * LUCENE-5949: Add Accountable.getChildResources(). (Robert Muir)
 
+* SOLR-5986: Added ExitableDirectoryReader that extends FilterDirectoryReader and enables
+  exiting requests that take too long to enumerate over terms. (Anshum Gupta, Steve Rowe,
+  Robert Muir)
+
 API Changes
 
 * LUCENE-5900: Deprecated more constructors taking Version in *InfixSuggester and

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ExitableDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ExitableDirectoryReader.java?rev=1627622&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ExitableDirectoryReader.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ExitableDirectoryReader.java Thu Sep 25 19:42:07 2014
@@ -0,0 +1,217 @@
+package org.apache.lucene.index;
+
+/*
+ * 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.FilterLeafReader.FilterFields;
+import org.apache.lucene.index.FilterLeafReader.FilterTerms;
+import org.apache.lucene.index.FilterLeafReader.FilterTermsEnum;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+
+import java.io.IOException;
+
+
+/**
+ * The {@link ExitableDirectoryReader} wraps a real index {@link DirectoryReader} and
+ * allows for a {@link QueryTimeout} implementation object to be checked periodically
+ * to see if the thread should exit or not.  If {@link QueryTimeout#shouldExit()}
+ * returns true, an {@link ExitingReaderException} is thrown.
+ */
+public class ExitableDirectoryReader extends FilterDirectoryReader {
+  
+  private QueryTimeout queryTimeout;
+
+  /**
+   * Exception that is thrown to prematurely terminate a term enumeration.
+   */
+  @SuppressWarnings("serial")
+  public static class ExitingReaderException extends RuntimeException {
+
+    /** Constructor **/
+    ExitingReaderException(String msg) {
+      super(msg);
+    }
+  }
+
+  /**
+   * Wrapper class for a SubReaderWrapper that is used by the ExitableDirectoryReader.
+   */
+  public static class ExitableSubReaderWrapper extends SubReaderWrapper {
+    private QueryTimeout queryTimeout;
+
+    /** Constructor **/
+    public ExitableSubReaderWrapper(QueryTimeout queryTimeout) {
+      this.queryTimeout = queryTimeout;
+    }
+
+    @Override
+    public LeafReader wrap(LeafReader reader) {
+      return new ExitableFilterAtomicReader(reader, queryTimeout);
+    }
+  }
+
+  /**
+   * Wrapper class for another FilterAtomicReader. This is used by ExitableSubReaderWrapper.
+   */
+  public static class ExitableFilterAtomicReader extends FilterLeafReader {
+
+    private QueryTimeout queryTimeout;
+    
+    /** Constructor **/
+    public ExitableFilterAtomicReader(LeafReader in, QueryTimeout queryTimeout) {
+      super(in);
+      this.queryTimeout = queryTimeout;
+    }
+
+    @Override
+    public Fields fields() throws IOException {
+      Fields fields = super.fields();
+      if (fields == null) {
+        return null;
+      }
+      return new ExitableFields(fields, queryTimeout);
+    }
+    
+    @Override
+    public Object getCoreCacheKey() {
+      return in.getCoreCacheKey();  
+    }
+    
+    @Override
+    public Object getCombinedCoreAndDeletesKey() {
+      return in.getCombinedCoreAndDeletesKey();
+    }
+    
+  }
+
+  /**
+   * Wrapper class for another Fields implementation that is used by the ExitableFilterAtomicReader.
+   */
+  public static class ExitableFields extends FilterFields {
+    
+    private QueryTimeout queryTimeout;
+
+    /** Constructor **/
+    public ExitableFields(Fields fields, QueryTimeout queryTimeout) {
+      super(fields);
+      this.queryTimeout = queryTimeout;
+    }
+
+    @Override
+    public Terms terms(String field) throws IOException {
+      Terms terms = in.terms(field);
+      if (terms == null) {
+        return null;
+      }
+      return new ExitableTerms(terms, queryTimeout);
+    }
+  }
+
+  /**
+   * Wrapper class for another Terms implementation that is used by ExitableFields.
+   */
+  public static class ExitableTerms extends FilterTerms {
+
+    private QueryTimeout queryTimeout;
+    
+    /** Constructor **/
+    public ExitableTerms(Terms terms, QueryTimeout queryTimeout) {
+      super(terms);
+      this.queryTimeout = queryTimeout;
+    }
+
+    @Override
+    public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
+      return new ExitableTermsEnum(in.intersect(compiled, startTerm), queryTimeout);
+    }
+
+    @Override
+    public TermsEnum iterator(TermsEnum reuse) throws IOException {
+      return new ExitableTermsEnum(in.iterator(reuse), queryTimeout);
+    }
+  }
+
+  /**
+   * Wrapper class for TermsEnum that is used by ExitableTerms for implementing an
+   * exitable enumeration of terms.
+   */
+  public static class ExitableTermsEnum extends FilterTermsEnum {
+
+    private QueryTimeout queryTimeout;
+    
+    /** Constructor **/
+    public ExitableTermsEnum(TermsEnum termsEnum, QueryTimeout queryTimeout) {
+      super(termsEnum);
+      this.queryTimeout = queryTimeout;
+      checkAndThrow();
+    }
+
+    /**
+     * Throws {@link ExitingReaderException} if {@link QueryTimeout#shouldExit()} returns true,
+     * or if {@link Thread#interrupted()} returns true.
+     */
+    private void checkAndThrow() {
+      if (queryTimeout.shouldExit()) {
+        throw new ExitingReaderException("The request took too long to iterate over terms. Timeout: " 
+            + queryTimeout.toString()
+            + ", TermsEnum=" + in
+        );
+      } else if (Thread.interrupted()) {
+        throw new ExitingReaderException("Interrupted while iterating over terms. TermsEnum=" + in);
+      }
+    }
+
+    @Override
+    public BytesRef next() throws IOException {
+      // Before every iteration, check if the iteration should exit
+      checkAndThrow();
+      return in.next();
+    }
+  }
+
+  /**
+   * Constructor
+   * @param in DirectoryReader that this ExitableDirectoryReader wraps around to make it Exitable.
+   * @param queryTimeout The object to periodically check if the query should time out.
+   */
+  public ExitableDirectoryReader(DirectoryReader in, QueryTimeout queryTimeout) {
+    super(in, new ExitableSubReaderWrapper(queryTimeout));
+    this.queryTimeout = queryTimeout;
+  }
+
+  @Override
+  protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
+    return new ExitableDirectoryReader(in, queryTimeout);
+  }
+
+  /**
+   * Wraps a provided DirectoryReader. Note that for convenience, the returned reader
+   * can be used normally (e.g. passed to {@link DirectoryReader#openIfChanged(DirectoryReader)})
+   * and so on.
+   */
+  public static DirectoryReader wrap(DirectoryReader in, QueryTimeout queryTimeout) {
+    return new ExitableDirectoryReader(in, queryTimeout);
+  }
+
+  @Override
+  public String toString() {
+    return "ExitableDirectoryReader(" + in.toString() + ")";
+  }
+}
+
+

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/QueryTimeout.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/QueryTimeout.java?rev=1627622&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/QueryTimeout.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/QueryTimeout.java Thu Sep 25 19:42:07 2014
@@ -0,0 +1,33 @@
+package org.apache.lucene.index;
+
+/*
+ * 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.
+ */
+
+/**
+ * Base for query timeout implementations, which will provide a {@code shouldExit()} method,
+ * used with {@link ExitableDirectoryReader}.
+ */
+public interface QueryTimeout {
+  
+  /**
+   * Called from {@link ExitableDirectoryReader.ExitableTermsEnum#next()} 
+   * to determine whether to stop processing a query.
+   */
+  public abstract boolean shouldExit();
+
+}
+

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/QueryTimeoutImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/QueryTimeoutImpl.java?rev=1627622&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/QueryTimeoutImpl.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/QueryTimeoutImpl.java Thu Sep 25 19:42:07 2014
@@ -0,0 +1,80 @@
+package org.apache.lucene.index;
+
+/*
+ * 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.concurrent.TimeUnit;
+
+import static java.lang.System.nanoTime;
+
+/**
+ * An implementation of {@link QueryTimeout} that can be used by
+ * the {@link ExitableDirectoryReader} class to time out and exit out
+ * when a query takes a long time to rewrite.
+ */
+public class QueryTimeoutImpl implements QueryTimeout {
+
+  /**
+   * The local variable to store the time beyond which, the processing should exit.
+   */
+  private Long timeoutAt;
+
+  /** 
+   * Sets the time at which to time out by adding the given timeAllowed to the current time.
+   * 
+   * @param timeAllowed Number of milliseconds after which to time out. Use {@code Long.MAX_VALUE}
+   *                    to effectively never time out.
+   */                    
+  public QueryTimeoutImpl(long timeAllowed) {
+    if (timeAllowed < 0L) {
+      timeAllowed = Long.MAX_VALUE;
+    }
+    timeoutAt = nanoTime() + TimeUnit.NANOSECONDS.convert(timeAllowed, TimeUnit.MILLISECONDS);
+  }
+
+  /**
+   * Returns time at which to time out, in nanoseconds relative to the (JVM-specific)
+   * epoch for {@link System#nanoTime()}, to compare with the value returned by
+   * {@code nanoTime()}.
+   */
+  public Long getTimeoutAt() {
+    return timeoutAt;
+  }
+
+  /**
+   * Return true if {@link #reset()} has not been called
+   * and the elapsed time has exceeded the time allowed.
+   */
+  @Override
+  public boolean shouldExit() {
+    return timeoutAt != null && nanoTime() - timeoutAt > 0;
+  }
+
+  /**
+   * Reset the timeout value.
+   */
+  public void reset() {
+    timeoutAt = null;
+  }
+  
+  @Override
+  public String toString() {
+    return "timeoutAt: " + timeoutAt + " (System.nanoTime(): " + nanoTime() + ")";
+  }
+}
+
+

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java?rev=1627622&r1=1627621&r2=1627622&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java Thu Sep 25 19:42:07 2014
@@ -39,7 +39,7 @@ public class TimeLimitingCollector imple
     private long timeElapsed;
     private int lastDocCollected;
     private TimeExceededException(long timeAllowed, long timeElapsed, int lastDocCollected) {
-      super("Elapsed time: " + timeElapsed + "Exceeded allowed search time: " + timeAllowed + " ms.");
+      super("Elapsed time: " + timeElapsed + ".  Exceeded allowed search time: " + timeAllowed + " ms.");
       this.timeAllowed = timeAllowed;
       this.timeElapsed = timeElapsed;
       this.lastDocCollected = lastDocCollected;
@@ -142,7 +142,7 @@ public class TimeLimitingCollector imple
       @Override
       public void collect(int doc) throws IOException {
         final long time = clock.get();
-        if (timeout < time) {
+        if (time - timeout > 0L) {
           if (greedy) {
             //System.out.println(this+"  greedy: before failing, collecting doc: "+(docBase + doc)+"  "+(time-t0));
             in.collect(doc);

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestExitableDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestExitableDirectoryReader.java?rev=1627622&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestExitableDirectoryReader.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestExitableDirectoryReader.java Thu Sep 25 19:42:07 2014
@@ -0,0 +1,170 @@
+package org.apache.lucene.index;
+
+/*
+ * 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.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.ExitableDirectoryReader.ExitingReaderException;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.PrefixQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+
+import java.io.IOException;
+
+/**
+ * Test that uses a default/lucene Implementation of {@link QueryTimeout}
+ * to exit out long running queries that take too long to iterate over Terms.
+ */
+public class TestExitableDirectoryReader extends LuceneTestCase {
+  private static class TestReader extends FilterLeafReader {
+
+    private static class TestFields extends FilterFields {
+      TestFields(Fields in) {
+        super(in);
+      }
+
+      @Override
+      public Terms terms(String field) throws IOException {
+        return new TestTerms(super.terms(field));
+      }
+    }
+
+    private static class TestTerms extends FilterTerms {
+      TestTerms(Terms in) {
+        super(in);
+      }
+
+      @Override
+      public TermsEnum iterator(TermsEnum reuse) throws IOException {
+        return new TestTermsEnum(super.iterator(reuse));
+      }
+    }
+
+    private static class TestTermsEnum extends FilterTermsEnum {
+      public TestTermsEnum(TermsEnum in) {
+        super(in);
+      }
+
+      /**
+       * Sleep between iterations to timeout things.
+       */
+      @Override
+      public BytesRef next() throws IOException {
+        try {
+          // Sleep for 1s before each .next() call.
+          Thread.sleep(1000);
+        } catch (InterruptedException e) {
+        }
+        return in.next();
+      }
+    }
+
+    public TestReader(IndexReader reader) throws IOException {
+      super(SlowCompositeReaderWrapper.wrap(reader));
+    }
+
+    @Override
+    public Fields fields() throws IOException {
+      return new TestFields(super.fields());
+    }
+  }
+
+  /**
+   * Tests timing out of TermsEnum iterations
+   * @throws Exception on error
+   */
+  public void testExitableFilterIndexReader() throws Exception {
+    Directory directory = newDirectory();
+    IndexWriter writer = new IndexWriter(directory, newIndexWriterConfig(new MockAnalyzer(random())));
+
+    Document d1 = new Document();
+    d1.add(newTextField("default", "one two", Field.Store.YES));
+    writer.addDocument(d1);
+
+    Document d2 = new Document();
+    d2.add(newTextField("default", "one three", Field.Store.YES));
+    writer.addDocument(d2);
+
+    Document d3 = new Document();
+    d3.add(newTextField("default", "ones two four", Field.Store.YES));
+    writer.addDocument(d3);
+
+    writer.commit();
+    writer.close();
+
+    DirectoryReader directoryReader;
+    DirectoryReader exitableDirectoryReader;
+    IndexReader reader;
+    IndexSearcher searcher;
+
+    Query query = new PrefixQuery(new Term("default", "o"));
+
+    // Set a fairly high timeout value (10 seconds) and expect the query to complete in that time frame.
+    // Not checking the validity of the result, all we are bothered about in this test is the timing out.
+    directoryReader = DirectoryReader.open(directory);
+    exitableDirectoryReader = new ExitableDirectoryReader(directoryReader, new QueryTimeoutImpl(10000));
+    reader = new TestReader(exitableDirectoryReader);
+    searcher = new IndexSearcher(reader);
+    searcher.search(query, 10);
+    reader.close();
+    exitableDirectoryReader.close();
+
+
+    // Set a really low timeout value (1 millisecond) and expect an Exception
+    directoryReader = DirectoryReader.open(directory);
+    exitableDirectoryReader = new ExitableDirectoryReader(directoryReader, new QueryTimeoutImpl(1));
+    reader = new TestReader(exitableDirectoryReader);
+    searcher = new IndexSearcher(reader);
+    try {
+      searcher.search(query, 10);
+      fail("This query should have led to an ExitingReaderException!");
+    } catch (ExitingReaderException ex) {
+      // Do nothing, we expect this!
+    } finally {
+      reader.close();
+      exitableDirectoryReader.close();
+    }
+   
+    // Set maximum time out and expect the query to complete. 
+    // Not checking the validity of the result, all we are bothered about in this test is the timing out.
+    directoryReader = DirectoryReader.open(directory);
+    exitableDirectoryReader = new ExitableDirectoryReader(directoryReader, new QueryTimeoutImpl(Long.MAX_VALUE));
+    reader = new TestReader(exitableDirectoryReader);
+    searcher = new IndexSearcher(reader);
+    searcher.search(query, 10);
+    reader.close();
+    exitableDirectoryReader.close();
+
+    // Set a negative time allowed and expect the query to complete (should disable timeouts)
+    // Not checking the validity of the result, all we are bothered about in this test is the timing out.
+    directoryReader = DirectoryReader.open(directory);
+    exitableDirectoryReader = new ExitableDirectoryReader(directoryReader, new QueryTimeoutImpl(-189034L));
+    reader = new TestReader(exitableDirectoryReader);
+    searcher = new IndexSearcher(reader);
+    searcher.search(query, 10);
+    reader.close();
+    exitableDirectoryReader.close();
+
+    directory.close();
+  }
+}
+

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java?rev=1627622&r1=1627621&r2=1627622&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java Thu Sep 25 19:42:07 2014
@@ -33,6 +33,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.ThreadInterruptedException;
 
 /**
@@ -143,7 +144,8 @@ public class TestTimeLimitingCollector e
       
       myHc = new MyHitCollector();
       long oneHour = 3600000;
-      Collector tlCollector = createTimedCollector(myHc, oneHour, false);
+      long duration = TestUtil.nextLong(random(), oneHour, Long.MAX_VALUE); 
+      Collector tlCollector = createTimedCollector(myHc, duration, false);
       search(tlCollector);
       totalTLCResults = myHc.hitCount();
     } catch (Exception e) {

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1627622&r1=1627621&r2=1627622&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Thu Sep 25 19:42:07 2014
@@ -140,6 +140,9 @@ New Features
 
 * SOLR-6543: Give HttpSolrServer the ability to send PUT requests (Gregory Chanan)
 
+* SOLR-5986: Don't allow runaway queries from harming Solr cluster health or search 
+  performance (Anshum Gupta, Steve Rowe, Robert Muir)
+
 Bug Fixes
 ----------------------
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/MoreLikeThisHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/MoreLikeThisHandler.java?rev=1627622&r1=1627621&r2=1627622&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/MoreLikeThisHandler.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/MoreLikeThisHandler.java Thu Sep 25 19:42:07 2014
@@ -17,29 +17,21 @@
 
 package org.apache.solr.handler;
 
-import java.io.IOException;
-import java.io.Reader;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Comparator;
-
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Pattern;
-
+import org.apache.lucene.index.ExitableDirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.search.*;
 import org.apache.lucene.queries.mlt.MoreLikeThis;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
 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.MoreLikeThisParams;
-import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.MoreLikeThisParams.TermStyle;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
@@ -49,10 +41,31 @@ import org.apache.solr.request.SolrQuery
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
-import org.apache.solr.search.*;
-
+import org.apache.solr.search.DocIterator;
+import org.apache.solr.search.DocList;
+import org.apache.solr.search.DocListAndSet;
+import org.apache.solr.search.QParser;
+import org.apache.solr.search.QParserPlugin;
+import org.apache.solr.search.QueryParsing;
+import org.apache.solr.search.ReturnFields;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.search.SolrQueryTimeoutImpl;
+import org.apache.solr.search.SolrReturnFields;
+import org.apache.solr.search.SortSpec;
+import org.apache.solr.search.SyntaxError;
 import org.apache.solr.util.SolrPluginUtils;
 
+import java.io.IOException;
+import java.io.Reader;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
 /**
  * Solr MoreLikeThis --
  * 
@@ -75,180 +88,190 @@ public class MoreLikeThisHandler extends
   {
     SolrParams params = req.getParams();
 
-    // Set field flags
-    ReturnFields returnFields = new SolrReturnFields( req );
-    rsp.setReturnFields( returnFields );
-    int flags = 0;
-    if (returnFields.wantsScore()) {
-      flags |= SolrIndexSearcher.GET_SCORES;
+    long timeAllowed = (long)params.getInt( CommonParams.TIME_ALLOWED, -1 );
+    if(timeAllowed > 0) {
+      SolrQueryTimeoutImpl.set(timeAllowed);
     }
+      try {
 
-    String defType = params.get(QueryParsing.DEFTYPE, QParserPlugin.DEFAULT_QTYPE);
-    String q = params.get( CommonParams.Q );
-    Query query = null;
-    SortSpec sortSpec = null;
-    List<Query> filters = null;
-
-    try {
-      if (q != null) {
-        QParser parser = QParser.getParser(q, defType, req);
-        query = parser.getQuery();
-        sortSpec = parser.getSort(true);
-      }
+        // Set field flags
+        ReturnFields returnFields = new SolrReturnFields(req);
+        rsp.setReturnFields(returnFields);
+        int flags = 0;
+        if (returnFields.wantsScore()) {
+          flags |= SolrIndexSearcher.GET_SCORES;
+        }
+
+        String defType = params.get(QueryParsing.DEFTYPE, QParserPlugin.DEFAULT_QTYPE);
+        String q = params.get(CommonParams.Q);
+        Query query = null;
+        SortSpec sortSpec = null;
+        List<Query> filters = null;
+
+        try {
+          if (q != null) {
+            QParser parser = QParser.getParser(q, defType, req);
+            query = parser.getQuery();
+            sortSpec = parser.getSort(true);
+          }
 
-      String[] fqs = req.getParams().getParams(CommonParams.FQ);
-      if (fqs!=null && fqs.length!=0) {
-          filters = new ArrayList<>();
-        for (String fq : fqs) {
-          if (fq != null && fq.trim().length()!=0) {
-            QParser fqp = QParser.getParser(fq, null, req);
-            filters.add(fqp.getQuery());
+          String[] fqs = req.getParams().getParams(CommonParams.FQ);
+          if (fqs != null && fqs.length != 0) {
+            filters = new ArrayList<>();
+            for (String fq : fqs) {
+              if (fq != null && fq.trim().length() != 0) {
+                QParser fqp = QParser.getParser(fq, null, req);
+                filters.add(fqp.getQuery());
+              }
+            }
           }
+        } catch (SyntaxError e) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
         }
-      }
-    } catch (SyntaxError e) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
-    }
 
-    SolrIndexSearcher searcher = req.getSearcher();
+        SolrIndexSearcher searcher = req.getSearcher();
 
-    MoreLikeThisHelper mlt = new MoreLikeThisHelper( params, searcher );
+        MoreLikeThisHelper mlt = new MoreLikeThisHelper(params, searcher);
 
-    // Hold on to the interesting terms if relevant
-    TermStyle termStyle = TermStyle.get( params.get( MoreLikeThisParams.INTERESTING_TERMS ) );
-    List<InterestingTerm> interesting = (termStyle == TermStyle.NONE )
-      ? null : new ArrayList<InterestingTerm>( mlt.mlt.getMaxQueryTerms() );
-    
-    DocListAndSet mltDocs = null;
-
-    // Parse Required Params
-    // This will either have a single Reader or valid query
-    Reader reader = null;
-    try {
-      if (q == null || q.trim().length() < 1) {
-        Iterable<ContentStream> streams = req.getContentStreams();
-        if (streams != null) {
-          Iterator<ContentStream> iter = streams.iterator();
-          if (iter.hasNext()) {
-            reader = iter.next().getReader();
+        // Hold on to the interesting terms if relevant
+        TermStyle termStyle = TermStyle.get(params.get(MoreLikeThisParams.INTERESTING_TERMS));
+        List<InterestingTerm> interesting = (termStyle == TermStyle.NONE)
+            ? null : new ArrayList<InterestingTerm>(mlt.mlt.getMaxQueryTerms());
+
+        DocListAndSet mltDocs = null;
+
+        // Parse Required Params
+        // This will either have a single Reader or valid query
+        Reader reader = null;
+        try {
+          if (q == null || q.trim().length() < 1) {
+            Iterable<ContentStream> streams = req.getContentStreams();
+            if (streams != null) {
+              Iterator<ContentStream> iter = streams.iterator();
+              if (iter.hasNext()) {
+                reader = iter.next().getReader();
+              }
+              if (iter.hasNext()) {
+                throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                    "MoreLikeThis does not support multiple ContentStreams");
+              }
+            }
           }
-          if (iter.hasNext()) {
+
+          int start = params.getInt(CommonParams.START, 0);
+          int rows = params.getInt(CommonParams.ROWS, 10);
+
+          // Find documents MoreLikeThis - either with a reader or a query
+          // --------------------------------------------------------------------------------
+          if (reader != null) {
+            mltDocs = mlt.getMoreLikeThis(reader, start, rows, filters,
+                interesting, flags);
+          } else if (q != null) {
+            // Matching options
+            boolean includeMatch = params.getBool(MoreLikeThisParams.MATCH_INCLUDE,
+                true);
+            int matchOffset = params.getInt(MoreLikeThisParams.MATCH_OFFSET, 0);
+            // Find the base match
+            DocList match = searcher.getDocList(query, null, null, matchOffset, 1,
+                flags); // only get the first one...
+            if (includeMatch) {
+              rsp.add("match", match);
+            }
+
+            // This is an iterator, but we only handle the first match
+            DocIterator iterator = match.iterator();
+            if (iterator.hasNext()) {
+              // do a MoreLikeThis query for each document in results
+              int id = iterator.nextDoc();
+              mltDocs = mlt.getMoreLikeThis(id, start, rows, filters, interesting,
+                  flags);
+            }
+          } else {
             throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                "MoreLikeThis does not support multiple ContentStreams");
+                "MoreLikeThis requires either a query (?q=) or text to find similar documents.");
           }
-        }
-      }
-
-      int start = params.getInt(CommonParams.START, 0);
-      int rows = params.getInt(CommonParams.ROWS, 10);
 
-      // Find documents MoreLikeThis - either with a reader or a query
-      // --------------------------------------------------------------------------------
-      if (reader != null) {
-        mltDocs = mlt.getMoreLikeThis(reader, start, rows, filters,
-            interesting, flags);
-      } else if (q != null) {
-        // Matching options
-        boolean includeMatch = params.getBool(MoreLikeThisParams.MATCH_INCLUDE,
-            true);
-        int matchOffset = params.getInt(MoreLikeThisParams.MATCH_OFFSET, 0);
-        // Find the base match
-        DocList match = searcher.getDocList(query, null, null, matchOffset, 1,
-            flags); // only get the first one...
-        if (includeMatch) {
-          rsp.add("match", match);
-        }
-
-        // This is an iterator, but we only handle the first match
-        DocIterator iterator = match.iterator();
-        if (iterator.hasNext()) {
-          // do a MoreLikeThis query for each document in results
-          int id = iterator.nextDoc();
-          mltDocs = mlt.getMoreLikeThis(id, start, rows, filters, interesting,
-              flags);
+        } finally {
+          if (reader != null) {
+            reader.close();
+          }
         }
-      } else {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-            "MoreLikeThis requires either a query (?q=) or text to find similar documents.");
-      }
 
-    } finally {
-      if (reader != null) {
-        reader.close();
-      }
-    }
-    
-    if( mltDocs == null ) {
-      mltDocs = new DocListAndSet(); // avoid NPE
-    }
-    rsp.add( "response", mltDocs.docList );
-    
-  
-    if( interesting != null ) {
-      if( termStyle == TermStyle.DETAILS ) {
-        NamedList<Float> it = new NamedList<>();
-        for( InterestingTerm t : interesting ) {
-          it.add( t.term.toString(), t.boost );
+        if (mltDocs == null) {
+          mltDocs = new DocListAndSet(); // avoid NPE
         }
-        rsp.add( "interestingTerms", it );
-      }
-      else {
-        List<String> it = new ArrayList<>( interesting.size() );
-        for( InterestingTerm t : interesting ) {
-          it.add( t.term.text());
+        rsp.add("response", mltDocs.docList);
+
+
+        if (interesting != null) {
+          if (termStyle == TermStyle.DETAILS) {
+            NamedList<Float> it = new NamedList<>();
+            for (InterestingTerm t : interesting) {
+              it.add(t.term.toString(), t.boost);
+            }
+            rsp.add("interestingTerms", it);
+          } else {
+            List<String> it = new ArrayList<>(interesting.size());
+            for (InterestingTerm t : interesting) {
+              it.add(t.term.text());
+            }
+            rsp.add("interestingTerms", it);
+          }
         }
-        rsp.add( "interestingTerms", it );
-      }
-    }
-    
-    // maybe facet the results
-    if (params.getBool(FacetParams.FACET,false)) {
-      if( mltDocs.docSet == null ) {
-        rsp.add( "facet_counts", null );
-      }
-      else {
-        SimpleFacets f = new SimpleFacets(req, mltDocs.docSet, params );
-        rsp.add( "facet_counts", f.getFacetCounts() );
-      }
-    }
-    boolean dbg = req.getParams().getBool(CommonParams.DEBUG_QUERY, false);
 
-    boolean dbgQuery = false, dbgResults = false;
-    if (dbg == false){//if it's true, we are doing everything anyway.
-      String[] dbgParams = req.getParams().getParams(CommonParams.DEBUG);
-      if (dbgParams != null) {
-        for (int i = 0; i < dbgParams.length; i++) {
-          if (dbgParams[i].equals(CommonParams.QUERY)){
-            dbgQuery = true;
-          } else if (dbgParams[i].equals(CommonParams.RESULTS)){
-            dbgResults = true;
+        // maybe facet the results
+        if (params.getBool(FacetParams.FACET, false)) {
+          if (mltDocs.docSet == null) {
+            rsp.add("facet_counts", null);
+          } else {
+            SimpleFacets f = new SimpleFacets(req, mltDocs.docSet, params);
+            rsp.add("facet_counts", f.getFacetCounts());
           }
         }
-      }
-    } else {
-      dbgQuery = true;
-      dbgResults = true;
-    }
-    // Copied from StandardRequestHandler... perhaps it should be added to doStandardDebug?
-    if (dbg == true) {
-      try {
-        NamedList<Object> dbgInfo = SolrPluginUtils.doStandardDebug(req, q, mlt.getRawMLTQuery(), mltDocs.docList, dbgQuery, dbgResults);
-        if (null != dbgInfo) {
-          if (null != filters) {
-            dbgInfo.add("filter_queries",req.getParams().getParams(CommonParams.FQ));
-            List<String> fqs = new ArrayList<>(filters.size());
-            for (Query fq : filters) {
-              fqs.add(QueryParsing.toString(fq, req.getSchema()));
+        boolean dbg = req.getParams().getBool(CommonParams.DEBUG_QUERY, false);
+
+        boolean dbgQuery = false, dbgResults = false;
+        if (dbg == false) {//if it's true, we are doing everything anyway.
+          String[] dbgParams = req.getParams().getParams(CommonParams.DEBUG);
+          if (dbgParams != null) {
+            for (int i = 0; i < dbgParams.length; i++) {
+              if (dbgParams[i].equals(CommonParams.QUERY)) {
+                dbgQuery = true;
+              } else if (dbgParams[i].equals(CommonParams.RESULTS)) {
+                dbgResults = true;
+              }
             }
-            dbgInfo.add("parsed_filter_queries",fqs);
           }
-          rsp.add("debug", dbgInfo);
+        } else {
+          dbgQuery = true;
+          dbgResults = true;
+        }
+        // Copied from StandardRequestHandler... perhaps it should be added to doStandardDebug?
+        if (dbg == true) {
+          try {
+            NamedList<Object> dbgInfo = SolrPluginUtils.doStandardDebug(req, q, mlt.getRawMLTQuery(), mltDocs.docList, dbgQuery, dbgResults);
+            if (null != dbgInfo) {
+              if (null != filters) {
+                dbgInfo.add("filter_queries", req.getParams().getParams(CommonParams.FQ));
+                List<String> fqs = new ArrayList<>(filters.size());
+                for (Query fq : filters) {
+                  fqs.add(QueryParsing.toString(fq, req.getSchema()));
+                }
+                dbgInfo.add("parsed_filter_queries", fqs);
+              }
+              rsp.add("debug", dbgInfo);
+            }
+          } catch (Exception e) {
+            SolrException.log(SolrCore.log, "Exception during debug", e);
+            rsp.add("exception_during_debug", SolrException.toStr(e));
+          }
         }
-      } catch (Exception e) {
-        SolrException.log(SolrCore.log, "Exception during debug", e);
-        rsp.add("exception_during_debug", SolrException.toStr(e));
+      } catch (ExitableDirectoryReader.ExitingReaderException ex) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            "MLTHandler Request took too long during query expansion. Terminating request.");
+      } finally {
+        SolrQueryTimeoutImpl.reset();
       }
-    }
   }
   
   public static class InterestingTerm

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java?rev=1627622&r1=1627621&r2=1627622&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java Thu Sep 25 19:42:07 2014
@@ -285,7 +285,7 @@ public class QueryComponent extends Sear
     }
 
     // -1 as flag if not set.
-    long timeAllowed = (long)params.getInt( CommonParams.TIME_ALLOWED, -1 );
+    long timeAllowed = params.getLong(CommonParams.TIME_ALLOWED, -1L);
     if (null != rb.getCursorMark() && 0 < timeAllowed) {
       // fundementally incompatible
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Can not search using both " +

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java?rev=1627622&r1=1627621&r2=1627622&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java Thu Sep 25 19:42:07 2014
@@ -17,19 +17,13 @@
 
 package org.apache.solr.handler.component;
 
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.LinkedList;
-import java.util.List;
-
+import org.apache.lucene.index.ExitableDirectoryReader;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.ShardParams;
-import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.core.CloseHook;
@@ -38,6 +32,7 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.search.SolrQueryTimeoutImpl;
 import org.apache.solr.util.RTimer;
 import org.apache.solr.util.SolrPluginUtils;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
@@ -45,6 +40,12 @@ import org.apache.solr.util.plugin.SolrC
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
 
 /**
  *
@@ -212,30 +213,42 @@ public class SearchHandler extends Reque
     if (!rb.isDistrib) {
       // a normal non-distributed request
 
-      // The semantics of debugging vs not debugging are different enough that
-      // it makes sense to have two control loops
-      if(!rb.isDebug()) {
-        // Process
-        for( SearchComponent c : components ) {
-          c.process(rb);
-        }
-      }
-      else {
-        // Process
-        RTimer subt = timer.sub( "process" );
-        for( SearchComponent c : components ) {
-          rb.setTimer( subt.sub( c.getName() ) );
-          c.process(rb);
-          rb.getTimer().stop();
+      long timeAllowed = req.getParams().getLong(CommonParams.TIME_ALLOWED, -1L);
+      if (timeAllowed > 0L) {
+        SolrQueryTimeoutImpl.set(timeAllowed);
+      }
+      try {
+        // The semantics of debugging vs not debugging are different enough that
+        // it makes sense to have two control loops
+        if(!rb.isDebug()) {
+          // Process
+          for( SearchComponent c : components ) {
+            c.process(rb);
+          }
         }
-        subt.stop();
-        timer.stop();
+        else {
+          // Process
+          RTimer subt = timer.sub( "process" );
+          for( SearchComponent c : components ) {
+            rb.setTimer( subt.sub( c.getName() ) );
+            c.process(rb);
+            rb.getTimer().stop();
+          }
+          subt.stop();
+          timer.stop();
 
-        // add the timing info
-        if (rb.isDebugTimings()) {
-          rb.addDebugInfo("timing", timer.asNamedList() );
+          // add the timing info
+          if (rb.isDebugTimings()) {
+            rb.addDebugInfo("timing", timer.asNamedList() );
+          }
         }
-      }      
+      } catch (ExitableDirectoryReader.ExitingReaderException ex) {
+        log.warn( "Query: " + req.getParamString() + "; " + ex.getMessage());
+        throw new SolrException(ErrorCode.BAD_REQUEST,
+            "Request took too long during query expansion. Terminating request.");
+      } finally {
+        SolrQueryTimeoutImpl.reset();
+      }
     } else {
       // a distributed request
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=1627622&r1=1627621&r2=1627622&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java Thu Sep 25 19:42:07 2014
@@ -48,6 +48,7 @@ import org.apache.lucene.index.LeafReade
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.ExitableDirectoryReader;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
@@ -193,7 +194,9 @@ public class SolrIndexSearcher extends I
   // this reader supports reopen
   private static DirectoryReader wrapReader(SolrCore core, DirectoryReader reader) {
     assert reader != null;
-    return UninvertingReader.wrap(reader, core.getLatestSchema().getUninversionMap(reader));
+    return ExitableDirectoryReader.wrap
+        (UninvertingReader.wrap(reader, core.getLatestSchema().getUninversionMap(reader)), 
+         SolrQueryTimeoutImpl.getInstance());
   }
 
   /**

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrQueryTimeoutImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrQueryTimeoutImpl.java?rev=1627622&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrQueryTimeoutImpl.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrQueryTimeoutImpl.java Thu Sep 25 19:42:07 2014
@@ -0,0 +1,86 @@
+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 org.apache.lucene.index.QueryTimeout;
+
+import java.util.concurrent.TimeUnit;
+
+import static java.lang.System.nanoTime;
+
+/**
+ * Implementation of {@link QueryTimeout} that is used by Solr. 
+ * It uses a ThreadLocal variable to track the timeoutAt value
+ * for each request thread.
+ */
+public class SolrQueryTimeoutImpl implements QueryTimeout {
+  /**
+   * The ThreadLocal variable to store the time beyond which, the processing should exit.
+   */
+  public static ThreadLocal<Long> timeoutAt = new ThreadLocal<Long>() {
+    @Override
+    protected Long initialValue() {
+      return nanoTime() + Long.MAX_VALUE;
+    }
+  };
+
+  private SolrQueryTimeoutImpl() { }
+  private static SolrQueryTimeoutImpl instance = new SolrQueryTimeoutImpl();
+
+  /** Return singleton instance */
+  public static SolrQueryTimeoutImpl getInstance() { 
+    return instance; 
+  }
+
+  /**
+   * Get the current value of timeoutAt.
+   */
+  public static Long get() {
+    return timeoutAt.get();
+  }
+
+  /**
+   * Return true if a timeoutAt value is set and the current time has exceeded the set timeOut.
+   */
+  @Override
+  public boolean shouldExit() {
+    return get() - nanoTime() < 0L;
+  }
+
+  /**
+   * Method to set the time at which the timeOut should happen.
+   * @param timeAllowed set the time at which this thread should timeout.
+   */
+  public static void set(Long timeAllowed) {
+    long time = nanoTime() + TimeUnit.NANOSECONDS.convert(timeAllowed, TimeUnit.MILLISECONDS);
+    timeoutAt.set(time);
+  }
+
+  /**
+   * Cleanup the ThreadLocal timeout value.
+   */
+  public static void reset() {
+    timeoutAt.remove();
+  }
+
+  @Override
+  public String toString() {
+    return "timeoutAt: " + get() + " (System.nanoTime(): " + nanoTime() + ")";
+  }
+}
+

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestDistributedSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestDistributedSearch.java?rev=1627622&r1=1627621&r2=1627622&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestDistributedSearch.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestDistributedSearch.java Thu Sep 25 19:42:07 2014
@@ -492,6 +492,10 @@ public class TestDistributedSearch exten
           ShardParams.SHARDS_TOLERANT, "true");
 
       // test group query
+      // TODO: Remove this? This doesn't make any real sense now that timeAllowed might trigger early
+      //       termination of the request during Terms enumeration/Query expansion.
+      //       During such an exit, partial results isn't supported as it wouldn't make any sense.
+      // Increasing the timeAllowed from 1 to 100 for now.
       queryPartialResults(upShards, upClients,
           "q", "*:*",
           "rows", 100,
@@ -500,7 +504,7 @@ public class TestDistributedSearch exten
           "group.query", t1 + ":kings OR " + t1 + ":eggs",
           "group.limit", 10,
           "sort", i1 + " asc, id asc",
-          CommonParams.TIME_ALLOWED, 1,
+          CommonParams.TIME_ALLOWED, 100,
           ShardParams.SHARDS_INFO, "true",
           ShardParams.SHARDS_TOLERANT, "true");
 

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestGroupingSearch.java?rev=1627622&r1=1627621&r2=1627622&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestGroupingSearch.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestGroupingSearch.java Thu Sep 25 19:42:07 2014
@@ -278,7 +278,9 @@ public class TestGroupingSearch extends 
     assertU(commit());
 
     // Just checking if no errors occur
-    assertJQ(req("q", "*:*", "group", "true", "group.query", "id:1", "group.query", "id:2", "timeAllowed", "1"));
+    // TODO: Check if this makes any sense now that timeAllowed is also used during Terms enumeration.
+    //       The query can potentially timeout iterating over terms if this is set to too low.
+    assertJQ(req("q", "*:*", "group", "true", "group.query", "id:1", "group.query", "id:2", "timeAllowed", "100"));
   }
 
   @Test

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CloudExitableDirectoryReaderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CloudExitableDirectoryReaderTest.java?rev=1627622&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CloudExitableDirectoryReaderTest.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CloudExitableDirectoryReaderTest.java Thu Sep 25 19:42:07 2014
@@ -0,0 +1,128 @@
+package org.apache.solr.cloud;
+
+/*
+* 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.util.TestUtil;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.SolrException.ErrorCode;
+
+/**
+* Distributed test for {@link org.apache.lucene.index.ExitableDirectoryReader} 
+*/
+public class CloudExitableDirectoryReaderTest extends AbstractFullDistribZkTestBase {
+  public static Logger log = LoggerFactory.getLogger(CloudExitableDirectoryReaderTest.class);
+  private static final int NUM_DOCS_PER_TYPE = 400;
+
+  public CloudExitableDirectoryReaderTest() {
+    configString = "solrconfig-tlog.xml";
+    schemaString = "schema.xml";
+  }
+
+  @Override
+  protected String getCloudSolrConfig() {
+    return configString;
+  }
+
+  @Override
+  public void doTest() throws Exception {
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+    waitForRecoveriesToFinish(false);
+    indexDocs();
+    doTimeoutTests();
+  }
+
+  public void indexDocs() throws Exception {
+    int counter = 1;
+
+    for(; (counter % NUM_DOCS_PER_TYPE) != 0; counter++ )
+      indexDoc(sdoc("id", Integer.toString(counter), "name", "a" + counter));
+
+    counter++;
+    for(; (counter % NUM_DOCS_PER_TYPE) != 0; counter++ )
+      indexDoc(sdoc("id", Integer.toString(counter), "name", "b" + counter));
+
+    counter++;
+    for(; counter % NUM_DOCS_PER_TYPE != 0; counter++ )
+      indexDoc(sdoc("id", Integer.toString(counter), "name", "dummy term doc" + counter));
+
+    commit();
+  }
+
+  public void doTimeoutTests() throws Exception {
+    assertFail(params("q", "name:a*", "timeAllowed", "1"));
+
+    long oneSecond = 1000L; // query rewriting for NUM_DOCS_PER_TYPE terms should take less time than this
+    Long timeAllowed = TestUtil.nextLong(random(), oneSecond, Long.MAX_VALUE);
+    assertSuccess(params("q", "name:a*", "timeAllowed",timeAllowed.toString()));
+
+    assertFail(params("q", "name:a*", "timeAllowed", "1"));
+
+    timeAllowed = TestUtil.nextLong(random(), oneSecond, Long.MAX_VALUE);
+    assertSuccess(params("q", "name:b*", "timeAllowed",timeAllowed.toString()));
+
+    timeAllowed = TestUtil.nextLong(random(), Long.MIN_VALUE, -1L);  // negative timeAllowed should disable timeouts
+    assertSuccess(params("q", "name:b*", "timeAllowed",timeAllowed.toString()));
+
+    assertSuccess(params("q","name:b*")); // no time limitation
+  }
+
+  /**
+   * execute a request, verify that we get an expected error
+   */
+  public void assertFail(ModifiableSolrParams p) throws Exception {
+    String timeoutMessage = "Request took too long during query expansion. Terminating request.";
+
+    try {
+      ignoreException(timeoutMessage);
+      queryServer(p);
+      fail("no exception matching expected: " + ErrorCode.BAD_REQUEST.code + ": " + timeoutMessage);
+    } catch (SolrServerException e) {
+      assertTrue("Exception " + e.getCause() + " is not a SolrException:\n" + prettyStackTrace(e.getCause()),
+          e.getCause() instanceof SolrException);
+      assertEquals(ErrorCode.BAD_REQUEST.code, ((SolrException)e.getCause()).code());
+      assertTrue("Expected error message substr not found: " + timeoutMessage + " <!< " + e.getMessage(),
+                 e.getMessage().contains(timeoutMessage));
+    } finally {
+      unIgnoreException(timeoutMessage);
+    }
+  }
+  
+  public void assertSuccess(ModifiableSolrParams p) throws Exception {
+    QueryResponse response = queryServer(p);
+    assertEquals("Wrong #docs in response", NUM_DOCS_PER_TYPE - 1, response.getResults().getNumFound());
+  }
+
+  public String prettyStackTrace(Throwable t) {
+    StringBuilder builder = new StringBuilder();
+    for (StackTraceElement elem : t.getStackTrace()) {
+      builder.append("    at ");
+      builder.append(elem.toString());
+      builder.append('\n');
+    }
+    return builder.toString();
+  }
+}
+

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/ExitableDirectoryReaderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/ExitableDirectoryReaderTest.java?rev=1627622&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/ExitableDirectoryReaderTest.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/ExitableDirectoryReaderTest.java Thu Sep 25 19:42:07 2014
@@ -0,0 +1,95 @@
+package org.apache.solr.core;
+
+/*
+ * 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.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test that checks that long running queries are exited by Solr using the
+ * SolrQueryTimeoutImpl implementation.
+ */
+public class ExitableDirectoryReaderTest extends SolrTestCaseJ4 {
+  
+  static int NUM_DOCS_PER_TYPE = 400;
+  static final String assertionString = "//result[@numFound='"+ (NUM_DOCS_PER_TYPE - 1) + "']";
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    System.setProperty("enable.update.log", "false"); // schema12 doesn't support _version_
+    initCore("solrconfig-nocache.xml", "schema12.xml");
+    createIndex();
+  }
+
+  public static void createIndex() {
+    int counter = 1;
+    
+    for(; (counter % NUM_DOCS_PER_TYPE) != 0; counter++ )
+      assertU(adoc("id", Integer.toString(counter), "name", "a" + counter));
+
+    counter++;
+    for(; (counter % NUM_DOCS_PER_TYPE) != 0; counter++ )
+      assertU(adoc("id", Integer.toString(counter), "name", "b" + counter));
+
+    counter++;
+    for(; counter % NUM_DOCS_PER_TYPE != 0; counter++ )
+      assertU(adoc("id", Integer.toString(counter), "name", "dummy term doc" + counter));
+
+    assertU(commit());
+  }
+
+  @Test
+  public void testPrefixQuery() {
+    assertQEx("", req("q","name:a*", "indent","true","timeAllowed","1")
+        , SolrException.ErrorCode.BAD_REQUEST
+    );
+
+    assertQ(req("q","name:a*", "indent","true", "timeAllowed","1000"), assertionString);
+
+    assertQEx("", req("q","name:a*", "indent","true", "timeAllowed","1")
+        , SolrException.ErrorCode.BAD_REQUEST
+    );
+
+    assertQ(req("q","name:b*", "indent","true", "timeAllowed","1000"), assertionString);
+
+    assertQ(req("q","name:b*", "indent","true", "timeAllowed",Long.toString(Long.MAX_VALUE)), assertionString);
+
+    assertQ(req("q","name:b*", "indent","true", "timeAllowed","-7")); // negative timeAllowed should disable timeouts
+
+    assertQ(req("q","name:b*", "indent","true"));
+  }
+  
+  @Test
+  public void testQueriesOnDocsWithMultipleTerms() {
+    assertQ(req("q","name:dummy", "indent","true", "timeAllowed","1000"), assertionString);
+
+    // This should pass even though this may take more than the 'timeAllowed' time, it doesn't take long
+    // to iterate over 1 term (dummy).
+    assertQ(req("q","name:dummy", "indent","true", "timeAllowed","1000"), assertionString);
+
+    assertQEx("", req("q","name:doc*", "indent","true", "timeAllowed","1")
+        , SolrException.ErrorCode.BAD_REQUEST
+    );
+
+  }
+}
+
+

Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/SolrParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/SolrParams.java?rev=1627622&r1=1627621&r2=1627622&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/SolrParams.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/SolrParams.java Thu Sep 25 19:42:07 2014
@@ -142,7 +142,28 @@ public abstract class SolrParams impleme
       throw new SolrException( SolrException.ErrorCode.BAD_REQUEST, ex.getMessage(), ex );
     }
   }
-  
+
+  /** Returns the Long value of the param, or null if not set */
+  public Long getLong(String param) {
+    String val = get(param);
+    try {
+      return val == null ? null : Long.valueOf(val);
+    } catch (Exception ex) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, ex.getMessage(), ex);
+    }
+  }
+
+  /** Returns the long value of the param, or def if not set */
+  public long getLong(String param, long def) {
+    String val = get(param);
+    try {
+      return val == null ? def : Long.parseLong(val);
+    } catch (Exception ex) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, ex.getMessage(), ex);
+    }
+  }
+
+
   /**
    * @return The int value of the field param, or the value for param 
    * or <code>null</code> if neither is set.