lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From sim...@apache.org
Subject svn commit: r1201291 - in /lucene/dev/branches/branch_3x: ./ lucene/ lucene/backwards/src/test/ lucene/backwards/src/test/org/apache/lucene/search/ lucene/src/java/org/apache/lucene/search/ lucene/src/java/org/apache/lucene/util/ lucene/src/test/org/ap...
Date Sat, 12 Nov 2011 18:02:04 GMT
Author: simonw
Date: Sat Nov 12 18:02:03 2011
New Revision: 1201291

URL: http://svn.apache.org/viewvc?rev=1201291&view=rev
Log:
LUCENE-2822: prevent TimeLimitedCollector from starting a statically referenced thread

Added:
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/Counter.java
      - copied, changed from r1163576, lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/Counter.java
Removed:
    lucene/dev/branches/branch_3x/lucene/backwards/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java
Modified:
    lucene/dev/branches/branch_3x/   (props changed)
    lucene/dev/branches/branch_3x/lucene/   (props changed)
    lucene/dev/branches/branch_3x/lucene/CHANGES.txt
    lucene/dev/branches/branch_3x/lucene/backwards/src/test/   (props changed)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java
    lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java
    lucene/dev/branches/branch_3x/solr/   (props changed)
    lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java

Modified: lucene/dev/branches/branch_3x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/CHANGES.txt?rev=1201291&r1=1201290&r2=1201291&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_3x/lucene/CHANGES.txt Sat Nov 12 18:02:03 2011
@@ -27,6 +27,14 @@ Changes in backwards compatibility polic
 * LUCENE-3541: Remove IndexInput's protected copyBuf. If you want to
   keep a buffer in your IndexInput, do this yourself in your implementation, 
   and be sure to do the right thing on clone()!  (Robert Muir)
+  
+* LUCENE-2822: TimeLimitingCollector now expects a counter clock instead of
+  relying on a private daemon thread. The global time limiting clock thread
+  has been exposed and is now lazily loaded and fully optional.
+  TimeLimitingCollector now supports setting clock baseline manually to include
+  prelude of a search. Previous versions set the baseline on construction time, 
+  now baseline is set once the first IndexReader is passed to the collector
+  unless set before. (Simon Willnauer)
 
 Changes in runtime behavior
 

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java?rev=1201291&r1=1201290&r2=1201291&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java
(original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java
Sat Nov 12 18:02:03 2011
@@ -20,6 +20,7 @@ package org.apache.lucene.search;
 import java.io.IOException;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.ThreadInterruptedException;
 
 /**
@@ -30,69 +31,9 @@ import org.apache.lucene.util.ThreadInte
  */
 public class TimeLimitingCollector extends Collector {
 
-  /** 
-   * Default timer resolution.
-   * @see #setResolution(long) 
-   */
-  public static final int DEFAULT_RESOLUTION = 20;
-
-  /**
-   * Default for {@link #isGreedy()}.
-   * @see #isGreedy()
-   */
-  public boolean DEFAULT_GREEDY = false; 
-
-  private static long resolution = DEFAULT_RESOLUTION;
-  
-  private boolean greedy = DEFAULT_GREEDY ;
-
-  private static final class TimerThread extends Thread  {
-
-    // NOTE: we can avoid explicit synchronization here for several reasons:
-    // * updates to volatile long variables are atomic
-    // * only single thread modifies this value
-    // * use of volatile keyword ensures that it does not reside in
-    //   a register, but in main memory (so that changes are visible to
-    //   other threads).
-    // * visibility of changes does not need to be instantaneous, we can
-    //   afford losing a tick or two.
-    //
-    // See section 17 of the Java Language Specification for details.
-    private volatile long time = 0;
-
-    /**
-     * TimerThread provides a pseudo-clock service to all searching
-     * threads, so that they can count elapsed time with less overhead
-     * than repeatedly calling System.currentTimeMillis.  A single
-     * thread should be created to be used for all searches.
-     */
-    private TimerThread() {
-      super("TimeLimitedCollector timer thread");
-      this.setDaemon( true );
-    }
-
-    @Override
-    public void run() {
-      while (true) {
-        // TODO: Use System.nanoTime() when Lucene moves to Java SE 5.
-        time += resolution;
-        try {
-          Thread.sleep( resolution );
-        } catch (InterruptedException ie) {
-          throw new ThreadInterruptedException(ie);
-        }
-      }
-    }
-
-    /**
-     * Get the timer value in milliseconds.
-     */
-    public long getMilliseconds() {
-      return time;
-    }
-  }
 
   /** Thrown when elapsed search time exceeds allowed search time. */
+  @SuppressWarnings("serial")
   public static class TimeExceededException extends RuntimeException {
     private long timeAllowed;
     private long timeElapsed;
@@ -117,58 +58,59 @@ public class TimeLimitingCollector exten
     }
   }
 
-  // Declare and initialize a single static timer thread to be used by
-  // all TimeLimitedCollector instances.  The JVM assures that
-  // this only happens once.
-  private final static TimerThread TIMER_THREAD = new TimerThread();
-  
-  static  {
-    TIMER_THREAD.start();
-  }
-
-  private final long t0;
-  private final long timeout;
+  private long t0 = Long.MIN_VALUE;
+  private long timeout = Long.MIN_VALUE;
   private final Collector collector;
-  
+  private final Counter clock;
+  private final long ticksAllowed;
+  private boolean greedy = false;
   private int docBase;
 
   /**
    * Create a TimeLimitedCollector wrapper over another {@link Collector} with a specified
timeout.
    * @param collector the wrapped {@link Collector}
-   * @param timeAllowed max time allowed for collecting hits after which {@link TimeExceededException}
is thrown
+   * @param clock the timer clock
+   * @param ticksAllowed max time allowed for collecting
+   * hits after which {@link TimeExceededException} is thrown
    */
-  public TimeLimitingCollector(final Collector collector, final long timeAllowed ) {
+  public TimeLimitingCollector(final Collector collector, Counter clock, final long ticksAllowed
) {
     this.collector = collector;
-    t0 = TIMER_THREAD.getMilliseconds();
-    this.timeout = t0 + timeAllowed;
+    this.clock = clock;
+    this.ticksAllowed = ticksAllowed;
   }
-
-  /** 
-   * Return the timer resolution.
-   * @see #setResolution(long)
-   */
-  public static long getResolution() {
-    return resolution;
+  
+  /**
+   * Sets the baseline for this collector. By default the collectors baseline is 
+   * initialized once the first reader is passed to the collector. 
+   * To include operations executed in prior to the actual document collection
+   * set the baseline through this method in your prelude.
+   * <p>
+   * Example usage:
+   * <pre>
+   *   Counter clock = ...;
+   *   long baseline = clock.get();
+   *   // ... prepare search
+   *   TimeLimitingCollector collector = new TimeLimitingCollector(c, clock, numTicks);
+   *   collector.setBaseline(baseline);
+   *   indexSearcher.search(query, collector);
+   * </pre>
+   * </p>
+   * @see #setBaseline() 
+   * @param clockTime
+   */
+  public void setBaseline(long clockTime) {
+    t0 = clockTime;
+    timeout = t0 + ticksAllowed;
   }
-
+  
   /**
-   * Set the timer resolution.
-   * The default timer resolution is 20 milliseconds. 
-   * This means that a search required to take no longer than 
-   * 800 milliseconds may be stopped after 780 to 820 milliseconds.
-   * <br>Note that: 
-   * <ul>
-   * <li>Finer (smaller) resolution is more accurate but less efficient.</li>
-   * <li>Setting resolution to less than 5 milliseconds will be silently modified to
5 milliseconds.</li>
-   * <li>Setting resolution smaller than current resolution might take effect only
after current 
-   * resolution. (Assume current resolution of 20 milliseconds is modified to 5 milliseconds,

-   * then it can take up to 20 milliseconds for the change to have effect.</li>
-   * </ul>      
+   * Syntactic sugar for {@link #setBaseline(long)} using {@link Counter#get()}
+   * on the clock passed to the construcutor.
    */
-  public static void setResolution(long newResolution) {
-    resolution = Math.max(newResolution,5); // 5 milliseconds is about the minimum reasonable
time for a Object.wait(long) call.
+  public void setBaseline() {
+    setBaseline(clock.get());
   }
-
+  
   /**
    * Checks if this time limited collector is greedy in collecting the last hit.
    * A non greedy collector, upon a timeout, would throw a {@link TimeExceededException}

@@ -199,7 +141,7 @@ public class TimeLimitingCollector exten
    */
   @Override
   public void collect(final int doc) throws IOException {
-    long time = TIMER_THREAD.getMilliseconds();
+    final long time = clock.get();
     if (timeout < time) {
       if (greedy) {
         //System.out.println(this+"  greedy: before failing, collecting doc: "+(docBase +
doc)+"  "+(time-t0));
@@ -216,6 +158,9 @@ public class TimeLimitingCollector exten
   public void setNextReader(IndexReader reader, int base) throws IOException {
     collector.setNextReader(reader, base);
     this.docBase = base;
+    if (Long.MIN_VALUE == t0) {
+      setBaseline();
+    }
   }
   
   @Override
@@ -228,4 +173,131 @@ public class TimeLimitingCollector exten
     return collector.acceptsDocsOutOfOrder();
   }
 
+
+  /**
+   * Returns the global TimerThreads {@link Counter}
+   * <p>
+   * Invoking this creates may create a new instance of {@link TimerThread} iff
+   * the global {@link TimerThread} has never been accessed before. The thread
+   * returned from this method is started on creation and will be alive unless
+   * you stop the {@link TimerThread} via {@link TimerThread#stopTimer()}.
+   * </p>
+   * @return the global TimerThreads {@link Counter}
+   * @lucene.experimental
+   */
+  public static Counter getGlobalCounter() {
+    return TimerThreadHolder.THREAD.counter;
+  }
+  
+  /**
+   * Returns the global {@link TimerThread}.
+   * <p>
+   * Invoking this creates may create a new instance of {@link TimerThread} iff
+   * the global {@link TimerThread} has never been accessed before. The thread
+   * returned from this method is started on creation and will be alive unless
+   * you stop the {@link TimerThread} via {@link TimerThread#stopTimer()}.
+   * </p>
+   * 
+   * @return the global {@link TimerThread}
+   * @lucene.experimental
+   */
+  public static TimerThread getGlobalTimerThread() {
+    return TimerThreadHolder.THREAD;
+  }
+  
+  private static final class TimerThreadHolder {
+    static final TimerThread THREAD;
+    static {
+      THREAD = new TimerThread(Counter.newCounter(true));
+      THREAD.start();
+    }
+  }
+
+  /**
+   * @lucene.experimental
+   */
+  public static final class TimerThread extends Thread  {
+    
+    public static final String THREAD_NAME = "TimeLimitedCollector timer thread";
+    public static final int DEFAULT_RESOLUTION = 20;
+    // NOTE: we can avoid explicit synchronization here for several reasons:
+    // * updates to volatile long variables are atomic
+    // * only single thread modifies this value
+    // * use of volatile keyword ensures that it does not reside in
+    //   a register, but in main memory (so that changes are visible to
+    //   other threads).
+    // * visibility of changes does not need to be instantaneous, we can
+    //   afford losing a tick or two.
+    //
+    // See section 17 of the Java Language Specification for details.
+    private volatile long time = 0;
+    private volatile boolean stop = false;
+    private volatile long resolution;
+    final Counter counter;
+    
+    public TimerThread(long resolution, Counter counter) {
+      super(THREAD_NAME);
+      this.resolution = resolution;
+      this.counter = counter;
+      this.setDaemon(true);
+    }
+    
+    public TimerThread(Counter counter) {
+      this(DEFAULT_RESOLUTION, counter);
+    }
+
+    @Override
+    public void run() {
+      while (!stop) {
+        // TODO: Use System.nanoTime() when Lucene moves to Java SE 5.
+        counter.addAndGet(resolution);
+        try {
+          Thread.sleep( resolution );
+        } catch (InterruptedException ie) {
+          throw new ThreadInterruptedException(ie);
+        }
+      }
+    }
+
+    /**
+     * Get the timer value in milliseconds.
+     */
+    public long getMilliseconds() {
+      return time;
+    }
+    
+    /**
+     * Stops the timer thread 
+     */
+    public void stopTimer() {
+      stop = true;
+    }
+    
+    /** 
+     * Return the timer resolution.
+     * @see #setResolution(long)
+     */
+    public long getResolution() {
+      return resolution;
+    }
+    
+    /**
+     * Set the timer resolution.
+     * The default timer resolution is 20 milliseconds. 
+     * This means that a search required to take no longer than 
+     * 800 milliseconds may be stopped after 780 to 820 milliseconds.
+     * <br>Note that: 
+     * <ul>
+     * <li>Finer (smaller) resolution is more accurate but less efficient.</li>
+     * <li>Setting resolution to less than 5 milliseconds will be silently modified
to 5 milliseconds.</li>
+     * <li>Setting resolution smaller than current resolution might take effect only
after current 
+     * resolution. (Assume current resolution of 20 milliseconds is modified to 5 milliseconds,

+     * then it can take up to 20 milliseconds for the change to have effect.</li>
+     * </ul>      
+     */
+    public void setResolution(long resolution) {
+      this.resolution = Math.max(resolution, 5); // 5 milliseconds is about the minimum reasonable
time for a Object.wait(long) call.
+    }
+  }
+  
 }

Copied: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/Counter.java
(from r1163576, lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/Counter.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/Counter.java?p2=lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/Counter.java&p1=lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/Counter.java&r1=1163576&r2=1201291&rev=1201291&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/Counter.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/Counter.java Sat
Nov 12 18:02:03 2011
@@ -77,7 +77,7 @@ public abstract class Counter {
   }
 
   private final static class AtomicCounter extends Counter {
-    private AtomicLong count;
+    private final AtomicLong count = new AtomicLong();
 
     @Override
     public long addAndGet(long delta) {

Modified: lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java?rev=1201291&r1=1201290&r2=1201291&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java
(original)
+++ lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java
Sat Nov 12 18:02:03 2011
@@ -27,7 +27,9 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.queryParser.QueryParser;
 import org.apache.lucene.search.TimeLimitingCollector.TimeExceededException;
+import org.apache.lucene.search.TimeLimitingCollector.TimerThread;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.ThreadInterruptedException;
 
@@ -54,6 +56,8 @@ public class TestTimeLimitingCollector e
 
   private final String FIELD_NAME = "body";
   private Query query;
+  private Counter counter;
+  private TimerThread counterThread;
 
   /**
    * initializes searcher with a document set
@@ -61,6 +65,9 @@ public class TestTimeLimitingCollector e
   @Override
   public void setUp() throws Exception {
     super.setUp();
+    counter = Counter.newCounter(true);
+    counterThread = new TimerThread(counter);
+    counterThread.start();
     final String docText[] = {
         "docThatNeverMatchesSoWeCanRequireLastDocCollectedToBeGreaterThanZero",
         "one blah three",
@@ -91,7 +98,6 @@ public class TestTimeLimitingCollector e
     
     // warm the searcher
     searcher.search(query, null, 1000);
-
   }
 
   @Override
@@ -99,6 +105,8 @@ public class TestTimeLimitingCollector e
     searcher.close();
     reader.close();
     directory.close();
+    counterThread.stopTimer();
+    counterThread.join();
     super.tearDown();
   }
 
@@ -140,7 +148,7 @@ public class TestTimeLimitingCollector e
   }
 
   private Collector createTimedCollector(MyHitCollector hc, long timeAllowed, boolean greedy)
{
-    TimeLimitingCollector res = new TimeLimitingCollector(hc, timeAllowed);
+    TimeLimitingCollector res = new TimeLimitingCollector(hc, counter, timeAllowed);
     res.setGreedy(greedy); // set to true to make sure at least one doc is collected.
     return res;
   }
@@ -192,8 +200,8 @@ public class TestTimeLimitingCollector e
     // verify that elapsed time at exception is within valid limits
     assertEquals( timoutException.getTimeAllowed(), TIME_ALLOWED);
     // a) Not too early
-    assertTrue ( "elapsed="+timoutException.getTimeElapsed()+" <= (allowed-resolution)="+(TIME_ALLOWED-TimeLimitingCollector.getResolution()),
-        timoutException.getTimeElapsed() > TIME_ALLOWED-TimeLimitingCollector.getResolution());
+    assertTrue ( "elapsed="+timoutException.getTimeElapsed()+" <= (allowed-resolution)="+(TIME_ALLOWED-counterThread.getResolution()),
+        timoutException.getTimeElapsed() > TIME_ALLOWED-counterThread.getResolution());
     // b) Not too late.
     //    This part is problematic in a busy test system, so we just print a warning.
     //    We already verified that a timeout occurred, we just can't be picky about how long
it took.
@@ -208,7 +216,7 @@ public class TestTimeLimitingCollector e
   }
 
   private long maxTime(boolean multiThreaded) {
-    long res = 2 * TimeLimitingCollector.getResolution() + TIME_ALLOWED + SLOW_DOWN; // some
slack for less noise in this test
+    long res = 2 * counterThread.getResolution() + TIME_ALLOWED + SLOW_DOWN; // some slack
for less noise in this test
     if (multiThreaded) {
       res *= MULTI_THREAD_SLACK; // larger slack  
     }
@@ -219,7 +227,7 @@ public class TestTimeLimitingCollector e
     String s =
       "( " +
       "2*resolution +  TIME_ALLOWED + SLOW_DOWN = " +
-      "2*" + TimeLimitingCollector.getResolution() + " + " + TIME_ALLOWED + " + " + SLOW_DOWN
+
+      "2*" + counterThread.getResolution() + " + " + TIME_ALLOWED + " + " + SLOW_DOWN +
       ")";
     if (multiThreaded) {
       s = MULTI_THREAD_SLACK + " * "+s;  
@@ -233,22 +241,22 @@ public class TestTimeLimitingCollector e
   public void testModifyResolution() {
     try {
       // increase and test
-      long resolution = 20 * TimeLimitingCollector.DEFAULT_RESOLUTION; //400
-      TimeLimitingCollector.setResolution(resolution);
-      assertEquals(resolution, TimeLimitingCollector.getResolution());
+      long resolution = 20 * TimerThread.DEFAULT_RESOLUTION; //400
+      counterThread.setResolution(resolution);
+      assertEquals(resolution, counterThread.getResolution());
       doTestTimeout(false,true);
       // decrease much and test
       resolution = 5;
-      TimeLimitingCollector.setResolution(resolution);
-      assertEquals(resolution, TimeLimitingCollector.getResolution());
+      counterThread.setResolution(resolution);
+      assertEquals(resolution, counterThread.getResolution());
       doTestTimeout(false,true);
       // return to default and test
-      resolution = TimeLimitingCollector.DEFAULT_RESOLUTION;
-      TimeLimitingCollector.setResolution(resolution);
-      assertEquals(resolution, TimeLimitingCollector.getResolution());
+      resolution = TimerThread.DEFAULT_RESOLUTION;
+      counterThread.setResolution(resolution);
+      assertEquals(resolution, counterThread.getResolution());
       doTestTimeout(false,true);
     } finally {
-      TimeLimitingCollector.setResolution(TimeLimitingCollector.DEFAULT_RESOLUTION);
+      counterThread.setResolution(TimerThread.DEFAULT_RESOLUTION);
     }
   }
   

Modified: lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=1201291&r1=1201290&r2=1201291&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
(original)
+++ lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
Sat Nov 12 18:02:03 2011
@@ -1243,7 +1243,7 @@ public class SolrIndexSearcher extends I
       }
       
       if( timeAllowed > 0 ) {
-        collector = new TimeLimitingCollector(collector, timeAllowed);
+        collector = new TimeLimitingCollector(collector, TimeLimitingCollector.getGlobalCounter(),
timeAllowed);
       }
       if (pf.postFilter != null) {
         pf.postFilter.setLastDelegate(collector);
@@ -1272,7 +1272,7 @@ public class SolrIndexSearcher extends I
       }
       Collector collector = topCollector;
       if( timeAllowed > 0 ) {
-        collector = new TimeLimitingCollector(collector, timeAllowed);
+        collector = new TimeLimitingCollector(collector, TimeLimitingCollector.getGlobalCounter(),
timeAllowed);
       }
       if (pf.postFilter != null) {
         pf.postFilter.setLastDelegate(collector);
@@ -1362,7 +1362,7 @@ public class SolrIndexSearcher extends I
        }
 
        if( timeAllowed > 0 ) {
-         collector = new TimeLimitingCollector(collector, timeAllowed);
+         collector = new TimeLimitingCollector(collector, TimeLimitingCollector.getGlobalCounter(),
timeAllowed);
        }
       if (pf.postFilter != null) {
         pf.postFilter.setLastDelegate(collector);
@@ -1398,7 +1398,7 @@ public class SolrIndexSearcher extends I
       Collector collector = setCollector;
 
       if( timeAllowed > 0 ) {
-        collector = new TimeLimitingCollector(collector, timeAllowed );
+        collector = new TimeLimitingCollector(collector, TimeLimitingCollector.getGlobalCounter(),
timeAllowed );
       }
       if (pf.postFilter != null) {
         pf.postFilter.setLastDelegate(collector);



Mime
View raw message