diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 33e3261..d9354ea 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -199,6 +199,14 @@ Changes in backwards compatibility policy
as these are no longer used by the scoring system. See MIGRATE.txt for more
details. (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
* LUCENE-2846: omitNorms now behaves like omitTermFrequencyAndPositions, if you
@@ -747,11 +755,6 @@ Optimizations
* LUCENE-2205: Very substantial (3-5X) RAM reduction required to hold
the terms index on opening an IndexReader (Aaron McCurry via Mike McCandless)
-* LUCENE-3443: FieldCache can now set docsWithField, and create an
- array, in a single pass. This results in faster init time for apps
- that need both (such as sorting by a field with a missing value).
- (Mike McCandless)
-
Test Cases
* LUCENE-3420: Disable the finalness checks in TokenStream and Analyzer
diff --git a/lucene/common-build.xml b/lucene/common-build.xml
index 3271f80..761fa8b 100644
--- a/lucene/common-build.xml
+++ b/lucene/common-build.xml
@@ -477,7 +477,7 @@
-
+
diff --git a/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java b/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
index 7bafcc9..784ab3a 100644
--- a/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
+++ b/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
@@ -138,7 +138,6 @@ final class TermVectorsTermsWriter extends TermsHashConsumer {
}
long lastPos = tvf.getFilePointer();
perFields[0].finishDocument();
- perFields[0].fieldInfo.commitVectors();
for(int i=1;i
+ * 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 long getResolution() {
- return resolution;
+ 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.
- *
Note that:
- *
- * - Finer (smaller) resolution is more accurate but less efficient.
- * - Setting resolution to less than 5 milliseconds will be silently modified to 5 milliseconds.
- * - 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.
- *
+ * 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 +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.
+ *
Note that:
+ *
+ * - Finer (smaller) resolution is more accurate but less efficient.
+ * - Setting resolution to less than 5 milliseconds will be silently modified to 5 milliseconds.
+ * - 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.
+ *
+ */
+ 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.
+ }
+ }
+
}
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-framework/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/src/test-framework/java/org/apache/lucene/util/LuceneTestCase.java
index be50dfe..4a7b01e 100644
--- a/lucene/src/test-framework/java/org/apache/lucene/util/LuceneTestCase.java
+++ b/lucene/src/test-framework/java/org/apache/lucene/util/LuceneTestCase.java
@@ -289,7 +289,7 @@ public abstract class LuceneTestCase extends Assert {
final Codec codec;
int randomVal = random.nextInt(10);
- if ("Lucene3x".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal < 2)) { // preflex-only setup
+ if ("Lucene3x".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal <= 3)) { // preflex-only setup
codec = new PreFlexRWCodec();
PREFLEX_IMPERSONATION_IS_ACTIVE = true;
} else if ("SimpleText".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 9)) {
diff --git a/lucene/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java b/lucene/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java
index 0a3b1a7..38119a2 100644
--- a/lucene/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java
+++ b/lucene/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java
@@ -28,17 +28,17 @@ 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;
/**
* Tests the {@link TimeLimitingCollector}. This test checks (1) search
* 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 +57,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 +66,9 @@ public class TestTimeLimitingCollector extends LuceneTestCase {
@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",
@@ -98,7 +103,6 @@ public class TestTimeLimitingCollector extends LuceneTestCase {
// warm the searcher
searcher.search(query, null, 1000);
-
}
@Override
@@ -106,6 +110,8 @@ public class TestTimeLimitingCollector extends LuceneTestCase {
searcher.close();
reader.close();
directory.close();
+ counterThread.stopTimer();
+ counterThread.join();
super.tearDown();
}
@@ -147,7 +153,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 +205,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 +221,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 +232,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;
@@ -240,22 +246,22 @@ public class TestTimeLimitingCollector extends LuceneTestCase {
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);
}
}
diff --git a/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/ContentItemsSource.java b/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/ContentItemsSource.java
index 20f7582..d89d888 100644
--- a/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/ContentItemsSource.java
+++ b/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/ContentItemsSource.java
@@ -149,9 +149,6 @@ public abstract class ContentItemsSource {
}
public void printStatistics(String itemsName) {
- if (!verbose) {
- return;
- }
boolean print = false;
String col = " ";
StringBuilder sb = new StringBuilder();
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..6c8d7c6 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -1294,7 +1294,7 @@ public class SolrIndexSearcher extends IndexSearcher implements SolrInfoMBean {
}
if( timeAllowed > 0 ) {
- collector = new TimeLimitingCollector(collector, timeAllowed);
+ collector = new TimeLimitingCollector(collector, TimeLimitingCollector.getGlobalCounter(), timeAllowed);
}
if (pf.postFilter != null) {
pf.postFilter.setLastDelegate(collector);
@@ -1323,7 +1323,7 @@ public class SolrIndexSearcher extends IndexSearcher implements SolrInfoMBean {
}
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);
@@ -1413,7 +1413,7 @@ public class SolrIndexSearcher extends IndexSearcher implements SolrInfoMBean {
}
if( timeAllowed > 0 ) {
- collector = new TimeLimitingCollector(collector, timeAllowed);
+ collector = new TimeLimitingCollector(collector, TimeLimitingCollector.getGlobalCounter(), timeAllowed);
}
if (pf.postFilter != null) {
pf.postFilter.setLastDelegate(collector);
@@ -1449,7 +1449,7 @@ public class SolrIndexSearcher extends IndexSearcher implements SolrInfoMBean {
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);