fix test bug

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1540287 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2013-11-09 11:25:16 +00:00
parent 7ddbf59fbd
commit 3d55eaeaa8
1 changed files with 36 additions and 17 deletions

View File

@ -34,6 +34,7 @@ import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
@ -1381,7 +1382,12 @@ public abstract class BasePostingsFormatTestCase extends LuceneTestCase {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
final Map<String,TermFreqs> termFreqs = new HashMap<String,TermFreqs>();
// Must be concurrent because thread(s) can be merging
// while up to one thread flushes, and each of those
// threads iterates over the map while the flushing
// thread might be adding to it:
final Map<String,TermFreqs> termFreqs = new ConcurrentHashMap<String,TermFreqs>();
final AtomicLong sumDocFreq = new AtomicLong();
final AtomicLong sumTotalTermFreq = new AtomicLong();
@ -1398,6 +1404,8 @@ public abstract class BasePostingsFormatTestCase extends LuceneTestCase {
}
final PostingsFormat defaultPostingsFormat = p;
final Thread mainThread = Thread.currentThread();
if (field.equals("body")) {
// A PF that counts up some stats and then in
@ -1420,6 +1428,15 @@ public abstract class BasePostingsFormatTestCase extends LuceneTestCase {
boolean isMerge = state.context.context == IOContext.Context.MERGE;
// We only use one thread for flushing
// in this test:
assert isMerge || Thread.currentThread() == mainThread;
// We iterate the provided TermsEnum
// twice, so we excercise this new freedom
// with the inverted API; if
// addOnSecondPass is true, we add up
// term stats on the 2nd iteration:
boolean addOnSecondPass = random().nextBoolean();
//System.out.println("write isMerge=" + isMerge + " 2ndPass=" + addOnSecondPass);
@ -1456,21 +1473,26 @@ public abstract class BasePostingsFormatTestCase extends LuceneTestCase {
String termString = term.utf8ToString();
// During merge we should only see terms
// we had already seen during flush:
// we had already seen during a
// previous flush:
assertTrue(isMerge==false || termFreqs.containsKey(termString));
if (isMerge == false && addOnSecondPass == false) {
TermFreqs tf = termFreqs.get(termString);
if (tf == null) {
tf = new TermFreqs();
termFreqs.put(termString, tf);
if (isMerge == false) {
if (addOnSecondPass == false) {
TermFreqs tf = termFreqs.get(termString);
if (tf == null) {
tf = new TermFreqs();
termFreqs.put(termString, tf);
}
tf.docFreq += docFreq;
tf.totalTermFreq += totalTermFreq;
sumDocFreq.addAndGet(docFreq);
sumTotalTermFreq.addAndGet(totalTermFreq);
} else if (termFreqs.containsKey(termString) == false) {
// Add placeholder (2nd pass will
// set its counts):
termFreqs.put(termString, new TermFreqs());
}
tf.docFreq += docFreq;
tf.totalTermFreq += totalTermFreq;
sumDocFreq.addAndGet(docFreq);
sumTotalTermFreq.addAndGet(totalTermFreq);
} else if (termFreqs.containsKey(termString) == false) {
termFreqs.put(termString, new TermFreqs());
}
}
@ -1503,10 +1525,7 @@ public abstract class BasePostingsFormatTestCase extends LuceneTestCase {
if (isMerge == false && addOnSecondPass) {
TermFreqs tf = termFreqs.get(term);
if (tf == null) {
tf = new TermFreqs();
termFreqs.put(term, tf);
}
assert tf != null;
tf.docFreq += docFreq;
tf.totalTermFreq += totalTermFreq;
sumDocFreq.addAndGet(docFreq);