diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 33e32615597..b27fce27059 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -653,6 +653,14 @@ Changes in backwards compatibility policy
* 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
diff --git a/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java b/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java
index 63ad23d9d6c..404e6a7bac1 100644
--- a/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java
+++ b/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java
@@ -20,6 +20,7 @@ package org.apache.lucene.search;
import java.io.IOException;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.util.Counter;
import org.apache.lucene.util.ThreadInterruptedException;
/**
@@ -30,69 +31,9 @@ import org.apache.lucene.util.ThreadInterruptedException;
*/
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,19 +58,12 @@ public class TimeLimitingCollector extends Collector {
}
}
- // 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;
/**
@@ -137,38 +71,45 @@ public class TimeLimitingCollector extends Collector {
* @param collector the wrapped {@link Collector}
* @param timeAllowed 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;
- }
-
+
/**
- * 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.
- *
Note that:
- *
+ * Example usage: + *
+ * Counter clock = ...; + * long baseline = clock.get(); + * // ... prepare search + * TimeLimitingCollector collector = new TimeLimitingCollector(c, clock, numTicks); + * collector.setBaseline(baseline); + * indexSearcher.search(query, collector); + *+ * + * @see #setBaseline() + * @param clockTime */ - 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(long clockTime) { + t0 = clockTime; + timeout = t0 + ticksAllowed; } - + + /** + * Syntactic sugar for {@link #setBaseline(long)} using {@link Counter#get()} + * on the clock passed to the construcutor. + */ + 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 +140,7 @@ public class TimeLimitingCollector extends Collector { */ @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 +157,9 @@ public class TimeLimitingCollector extends Collector { public void setNextReader(AtomicReaderContext context) throws IOException { collector.setNextReader(context); this.docBase = context.docBase; + if (Long.MIN_VALUE == t0) { + setBaseline(); + } } @Override @@ -228,4 +172,131 @@ public class TimeLimitingCollector extends Collector { return collector.acceptsDocsOutOfOrder(); } + + /** + * Returns the global TimerThreads {@link Counter} + *
+ * 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()}. + *
+ * @return the global TimerThreads {@link Counter} + * @lucene.experimental + */ + public static Counter getGlobalCounter() { + return TimerThreadHolder.THREAD.counter; + } + + /** + * Returns the global {@link TimerThread}. + *+ * 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()}. + *
+ * + * @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. + *