mirror of https://github.com/apache/lucene.git
sequence numbers: fix test bug
This commit is contained in:
parent
06ff43718f
commit
b0b7c81cfc
|
@ -149,6 +149,11 @@ public abstract class IndexInput extends DataInput implements Cloneable,Closeabl
|
|||
slice.seek(pos);
|
||||
return slice.readLong();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "RandomAccessInput(" + IndexInput.this.toString() + ")";
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,6 +17,14 @@
|
|||
|
||||
package org.apache.lucene.index;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
|
@ -28,17 +36,10 @@ import org.apache.lucene.search.TermQuery;
|
|||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
public class TestIndexingSequenceNumbers extends LuceneTestCase {
|
||||
|
||||
public void testBasic() throws Exception {
|
||||
|
@ -316,6 +317,9 @@ public class TestIndexingSequenceNumbers extends LuceneTestCase {
|
|||
final IndexWriter w = new IndexWriter(dir, iwc);
|
||||
|
||||
final int numThreads = TestUtil.nextInt(random(), 2, 10);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: numThreads=" + numThreads);
|
||||
}
|
||||
Thread[] threads = new Thread[numThreads];
|
||||
//System.out.println("TEST: iter=" + iter + " opCount=" + opCount + " idCount=" + idCount + " threadCount=" + threads.length);
|
||||
final CountDownLatch startingGun = new CountDownLatch(1);
|
||||
|
@ -383,6 +387,7 @@ public class TestIndexingSequenceNumbers extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
};
|
||||
threads[i].setName("thread" + i);
|
||||
threads[i].start();
|
||||
}
|
||||
startingGun.countDown();
|
||||
|
@ -416,9 +421,8 @@ public class TestIndexingSequenceNumbers extends LuceneTestCase {
|
|||
for(Operation op : threadOps.get(threadID)) {
|
||||
if (op.seqNo <= commitSeqNo && op.seqNo > seqNos[op.id]) {
|
||||
seqNos[op.id] = op.seqNo;
|
||||
if (op.what == 0) {
|
||||
expectedThreadIDs[op.id] = threadID;
|
||||
}
|
||||
assert op.what == 0;
|
||||
expectedThreadIDs[op.id] = threadID;
|
||||
}
|
||||
|
||||
assertTrue(op.seqNo > lastSeqNo);
|
||||
|
@ -437,9 +441,9 @@ public class TestIndexingSequenceNumbers extends LuceneTestCase {
|
|||
// We pre-add all ids up front:
|
||||
assert expectedThreadIDs[id] != -1;
|
||||
assertEquals(1, hits.totalHits);
|
||||
int actualThreadID = (int) docValues.get(id);
|
||||
int actualThreadID = (int) docValues.get(hits.scoreDocs[0].doc);
|
||||
if (expectedThreadIDs[id] != actualThreadID) {
|
||||
System.out.println("FAIL: commit=" + i + " (of " + commits.size() + ") id=" + id + " expectedThreadID=" + expectedThreadIDs[id] + " vs actualThreadID=" + actualThreadID + " commitSeqNo=" + commitSeqNo + " numThreads=" + numThreads);
|
||||
System.out.println("FAIL: commit=" + i + " (of " + commits.size() + ") id=" + id + " expectedThreadID=" + expectedThreadIDs[id] + " vs actualThreadID=" + actualThreadID + " commitSeqNo=" + commitSeqNo + " numThreads=" + numThreads + " reader=" + r + " commit=" + indexCommits.get(i));
|
||||
for(int threadID=0;threadID<threadOps.size();threadID++) {
|
||||
for(Operation op : threadOps.get(threadID)) {
|
||||
if (id == op.id) {
|
||||
|
@ -447,7 +451,7 @@ public class TestIndexingSequenceNumbers extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
}
|
||||
assertEquals("id=" + id, expectedThreadIDs[id], actualThreadID);
|
||||
assertEquals("id=" + id + " docID=" + hits.scoreDocs[0].doc, expectedThreadIDs[id], actualThreadID);
|
||||
}
|
||||
}
|
||||
w.close();
|
||||
|
|
Loading…
Reference in New Issue