diff --git a/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java b/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java index 63ad23d..5bc5304 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; /** @@ -42,11 +43,26 @@ public class TimeLimitingCollector extends Collector { */ public boolean DEFAULT_GREEDY = false; - private static long resolution = DEFAULT_RESOLUTION; - private boolean greedy = DEFAULT_GREEDY ; + + // nocommit - javadoc + public static Counter getDefaultCounter() { + return TimerThreadHolder.THREAD.counter; + } + + private static final class TimerThreadHolder { + static final TimerThread THREAD; + static { + THREAD = new TimerThread(Counter.newCounter(true)); + THREAD.setDaemon(true); + THREAD.start(); + } + } - private static final class TimerThread extends Thread { + /** + * @lucene.experimental + */ + public static final class TimerThread extends Thread { // NOTE: we can avoid explicit synchronization here for several reasons: // * updates to volatile long variables are atomic @@ -59,23 +75,26 @@ public class TimeLimitingCollector extends Collector { // // 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 ); + public static final String THREAD_NAME = "TimeLimitedCollector timer thread"; + 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; + } + + public TimerThread(Counter counter) { + this(DEFAULT_RESOLUTION, counter); } @Override public void run() { - while (true) { + while (!stop) { // TODO: Use System.nanoTime() when Lucene moves to Java SE 5. - time += resolution; + counter.addAndGet(resolution); try { Thread.sleep( resolution ); } catch (InterruptedException ie) { @@ -90,8 +109,22 @@ public class TimeLimitingCollector extends Collector { public long getMilliseconds() { return time; } - } + //nocommit - javadoc + public void stopTimer() { + stop = true; + } + //nocommit - javadoc + public long getResolution() { + return resolution; + } + + //nocommit - javadoc + 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. + } + } + /** Thrown when elapsed search time exceeds allowed search time. */ public static class TimeExceededException extends RuntimeException { private long timeAllowed; @@ -117,18 +150,10 @@ 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 final Collector collector; + private final Counter clock; private int docBase; @@ -137,18 +162,19 @@ 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; + t0 = clock.get(); + this.timeout = t0 + ticksAllowed; } - + /** * Return the timer resolution. * @see #setResolution(long) */ public static long getResolution() { - return resolution; + return TimerThreadHolder.THREAD.getResolution(); } /** @@ -166,7 +192,7 @@ public class TimeLimitingCollector extends Collector { * */ 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. + TimerThreadHolder.THREAD.setResolution(newResolution); } /** @@ -199,7 +225,7 @@ public class TimeLimitingCollector extends Collector { */ @Override public void collect(final int doc) throws IOException { - long time = TIMER_THREAD.getMilliseconds(); + long time = clock.get(); if (timeout < time) { if (greedy) { //System.out.println(this+" greedy: before failing, collecting doc: "+(docBase + doc)+" "+(time-t0)); diff --git a/lucene/src/java/org/apache/lucene/util/Counter.java b/lucene/src/java/org/apache/lucene/util/Counter.java index d605aef..b7642ac 100644 --- a/lucene/src/java/org/apache/lucene/util/Counter.java +++ b/lucene/src/java/org/apache/lucene/util/Counter.java @@ -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) { diff --git a/lucene/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java b/lucene/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java index 0a3b1a7..9506640 100644 --- a/lucene/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java +++ b/lucene/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java @@ -28,7 +28,9 @@ import org.apache.lucene.index.IndexReader.AtomicReaderContext; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.Term; 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; import org.junit.Ignore; @@ -38,7 +40,6 @@ import org.junit.Ignore; * correctness (regardless of timeout), (2) expected timeout behavior, * and (3) a sanity test with multiple searching threads. */ -@Ignore("broken: see https://issues.apache.org/jira/browse/LUCENE-2822") public class TestTimeLimitingCollector extends LuceneTestCase { private static final int SLOW_DOWN = 3; private static final long TIME_ALLOWED = 17 * SLOW_DOWN; // so searches can find about 17 docs. @@ -57,6 +58,8 @@ public class TestTimeLimitingCollector extends LuceneTestCase { private final String FIELD_NAME = "body"; private Query query; + private Counter counter; + private TimerThread counterThread; /** * initializes searcher with a document set @@ -64,6 +67,10 @@ public class TestTimeLimitingCollector extends LuceneTestCase { @Override public void setUp() throws Exception { super.setUp(); + counter = Counter.newCounter(true); + counterThread = new TimerThread(counter); + counterThread.setDaemon(true); + counterThread.start(); final String docText[] = { "docThatNeverMatchesSoWeCanRequireLastDocCollectedToBeGreaterThanZero", "one blah three", @@ -98,7 +105,6 @@ public class TestTimeLimitingCollector extends LuceneTestCase { // warm the searcher searcher.search(query, null, 1000); - } @Override @@ -106,6 +112,8 @@ public class TestTimeLimitingCollector extends LuceneTestCase { searcher.close(); reader.close(); directory.close(); + counterThread.stopTimer(); + counterThread.join(); super.tearDown(); } @@ -147,7 +155,7 @@ public class TestTimeLimitingCollector extends LuceneTestCase { } 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; } @@ -199,8 +207,8 @@ public class TestTimeLimitingCollector extends LuceneTestCase { // 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. @@ -215,7 +223,7 @@ public class TestTimeLimitingCollector extends LuceneTestCase { } 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 } @@ -226,7 +234,7 @@ public class TestTimeLimitingCollector extends LuceneTestCase { 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; @@ -241,21 +249,21 @@ public class TestTimeLimitingCollector extends LuceneTestCase { try { // increase and test long resolution = 20 * TimeLimitingCollector.DEFAULT_RESOLUTION; //400 - TimeLimitingCollector.setResolution(resolution); - assertEquals(resolution, TimeLimitingCollector.getResolution()); + 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()); + counterThread.setResolution(resolution); + assertEquals(resolution, counterThread.getResolution()); doTestTimeout(false,true); } finally { - TimeLimitingCollector.setResolution(TimeLimitingCollector.DEFAULT_RESOLUTION); + counterThread.setResolution(TimeLimitingCollector.DEFAULT_RESOLUTION); } } diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java index 3ce2a98..e034b82 100644 --- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java +++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java @@ -32,6 +32,7 @@ import org.apache.lucene.document.TextField; import org.apache.lucene.index.*; import org.apache.lucene.index.IndexReader.AtomicReaderContext; import org.apache.lucene.search.*; +import org.apache.lucene.search.TimeLimitingCollector.TimerThread; import org.apache.lucene.store.Directory; import org.apache.lucene.store.FSDirectory; import org.apache.lucene.util.Bits; @@ -1294,7 +1295,7 @@ public class SolrIndexSearcher extends IndexSearcher implements SolrInfoMBean { } if( timeAllowed > 0 ) { - collector = new TimeLimitingCollector(collector, timeAllowed); + collector = new TimeLimitingCollector(collector, TimeLimitingCollector.getDefaultCounter(), timeAllowed); } if (pf.postFilter != null) { pf.postFilter.setLastDelegate(collector); @@ -1323,7 +1324,7 @@ public class SolrIndexSearcher extends IndexSearcher implements SolrInfoMBean { } Collector collector = topCollector; if( timeAllowed > 0 ) { - collector = new TimeLimitingCollector(collector, timeAllowed); + collector = new TimeLimitingCollector(collector, TimeLimitingCollector.getDefaultCounter(), timeAllowed); } if (pf.postFilter != null) { pf.postFilter.setLastDelegate(collector); @@ -1413,7 +1414,7 @@ public class SolrIndexSearcher extends IndexSearcher implements SolrInfoMBean { } if( timeAllowed > 0 ) { - collector = new TimeLimitingCollector(collector, timeAllowed); + collector = new TimeLimitingCollector(collector, TimeLimitingCollector.getDefaultCounter(), timeAllowed); } if (pf.postFilter != null) { pf.postFilter.setLastDelegate(collector); @@ -1449,7 +1450,7 @@ public class SolrIndexSearcher extends IndexSearcher implements SolrInfoMBean { Collector collector = setCollector; if( timeAllowed > 0 ) { - collector = new TimeLimitingCollector(collector, timeAllowed ); + collector = new TimeLimitingCollector(collector, TimeLimitingCollector.getDefaultCounter(), timeAllowed ); } if (pf.postFilter != null) { pf.postFilter.setLastDelegate(collector);