mirror of https://github.com/apache/lucene.git
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:
parent
7ddbf59fbd
commit
3d55eaeaa8
|
@ -34,6 +34,7 @@ import java.util.Set;
|
||||||
import java.util.SortedMap;
|
import java.util.SortedMap;
|
||||||
import java.util.TreeMap;
|
import java.util.TreeMap;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.MockAnalyzer;
|
import org.apache.lucene.analysis.MockAnalyzer;
|
||||||
import org.apache.lucene.codecs.Codec;
|
import org.apache.lucene.codecs.Codec;
|
||||||
|
@ -1381,7 +1382,12 @@ public abstract class BasePostingsFormatTestCase extends LuceneTestCase {
|
||||||
Directory dir = newDirectory();
|
Directory dir = newDirectory();
|
||||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
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 sumDocFreq = new AtomicLong();
|
||||||
final AtomicLong sumTotalTermFreq = new AtomicLong();
|
final AtomicLong sumTotalTermFreq = new AtomicLong();
|
||||||
|
|
||||||
|
@ -1398,6 +1404,8 @@ public abstract class BasePostingsFormatTestCase extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
final PostingsFormat defaultPostingsFormat = p;
|
final PostingsFormat defaultPostingsFormat = p;
|
||||||
|
|
||||||
|
final Thread mainThread = Thread.currentThread();
|
||||||
|
|
||||||
if (field.equals("body")) {
|
if (field.equals("body")) {
|
||||||
|
|
||||||
// A PF that counts up some stats and then in
|
// 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;
|
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();
|
boolean addOnSecondPass = random().nextBoolean();
|
||||||
|
|
||||||
//System.out.println("write isMerge=" + isMerge + " 2ndPass=" + addOnSecondPass);
|
//System.out.println("write isMerge=" + isMerge + " 2ndPass=" + addOnSecondPass);
|
||||||
|
@ -1456,10 +1473,12 @@ public abstract class BasePostingsFormatTestCase extends LuceneTestCase {
|
||||||
String termString = term.utf8ToString();
|
String termString = term.utf8ToString();
|
||||||
|
|
||||||
// During merge we should only see terms
|
// 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));
|
assertTrue(isMerge==false || termFreqs.containsKey(termString));
|
||||||
|
|
||||||
if (isMerge == false && addOnSecondPass == false) {
|
if (isMerge == false) {
|
||||||
|
if (addOnSecondPass == false) {
|
||||||
TermFreqs tf = termFreqs.get(termString);
|
TermFreqs tf = termFreqs.get(termString);
|
||||||
if (tf == null) {
|
if (tf == null) {
|
||||||
tf = new TermFreqs();
|
tf = new TermFreqs();
|
||||||
|
@ -1470,9 +1489,12 @@ public abstract class BasePostingsFormatTestCase extends LuceneTestCase {
|
||||||
sumDocFreq.addAndGet(docFreq);
|
sumDocFreq.addAndGet(docFreq);
|
||||||
sumTotalTermFreq.addAndGet(totalTermFreq);
|
sumTotalTermFreq.addAndGet(totalTermFreq);
|
||||||
} else if (termFreqs.containsKey(termString) == false) {
|
} else if (termFreqs.containsKey(termString) == false) {
|
||||||
|
// Add placeholder (2nd pass will
|
||||||
|
// set its counts):
|
||||||
termFreqs.put(termString, new TermFreqs());
|
termFreqs.put(termString, new TermFreqs());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// Also test seeking the TermsEnum:
|
// Also test seeking the TermsEnum:
|
||||||
// must make copy because this very code sometimes makes puts while iterating?!
|
// must make copy because this very code sometimes makes puts while iterating?!
|
||||||
|
@ -1503,10 +1525,7 @@ public abstract class BasePostingsFormatTestCase extends LuceneTestCase {
|
||||||
|
|
||||||
if (isMerge == false && addOnSecondPass) {
|
if (isMerge == false && addOnSecondPass) {
|
||||||
TermFreqs tf = termFreqs.get(term);
|
TermFreqs tf = termFreqs.get(term);
|
||||||
if (tf == null) {
|
assert tf != null;
|
||||||
tf = new TermFreqs();
|
|
||||||
termFreqs.put(term, tf);
|
|
||||||
}
|
|
||||||
tf.docFreq += docFreq;
|
tf.docFreq += docFreq;
|
||||||
tf.totalTermFreq += totalTermFreq;
|
tf.totalTermFreq += totalTermFreq;
|
||||||
sumDocFreq.addAndGet(docFreq);
|
sumDocFreq.addAndGet(docFreq);
|
||||||
|
|
Loading…
Reference in New Issue