mirror of https://github.com/apache/lucene.git
add threads to this nice test
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1392610 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
637541add4
commit
013753ac26
|
@ -17,17 +17,20 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
|
||||
/**
|
||||
|
@ -37,43 +40,78 @@ import org.apache.lucene.util._TestUtil;
|
|||
@SuppressCodecs({"Direct", "Memory"}) // at night this makes like 200k/300k docs and will make Direct's heart beat!
|
||||
public class TestBagOfPostings extends LuceneTestCase {
|
||||
public void test() throws Exception {
|
||||
LinkedList<String> postings = new LinkedList<String>();
|
||||
List<String> postingsList = new ArrayList<String>();
|
||||
int numTerms = atLeast(300);
|
||||
int maxTermsPerDoc = 10;
|
||||
final int maxTermsPerDoc = _TestUtil.nextInt(random(), 10, 20);
|
||||
//System.out.println("maxTermsPerDoc=" + maxTermsPerDoc);
|
||||
//System.out.println("numTerms=" + numTerms);
|
||||
for (int i = 0; i < numTerms; i++) {
|
||||
String term = Integer.toString(i);
|
||||
for (int j = 0; j < i; j++) {
|
||||
postings.add(term);
|
||||
postingsList.add(term);
|
||||
}
|
||||
}
|
||||
Collections.shuffle(postings, random());
|
||||
|
||||
Collections.shuffle(postingsList, random());
|
||||
|
||||
final ConcurrentLinkedQueue<String> postings = new ConcurrentLinkedQueue<String>(postingsList);
|
||||
|
||||
Directory dir = newFSDirectory(_TestUtil.getTempDir("bagofpostings"));
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
|
||||
Document document = new Document();
|
||||
Field field = newTextField("field", "", Field.Store.NO);
|
||||
document.add(field);
|
||||
final RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
|
||||
|
||||
int threadCount = _TestUtil.nextInt(random(), 1, 5);
|
||||
//System.out.println("threadCount=" + threadCount);
|
||||
|
||||
Thread[] threads = new Thread[threadCount];
|
||||
final CountDownLatch startingGun = new CountDownLatch(1);
|
||||
|
||||
while (!postings.isEmpty()) {
|
||||
StringBuilder text = new StringBuilder();
|
||||
Set<String> visited = new HashSet<String>();
|
||||
for (int i = 0; i < maxTermsPerDoc; i++) {
|
||||
if (postings.isEmpty() || visited.contains(postings.peek())) {
|
||||
break;
|
||||
}
|
||||
String element = postings.remove();
|
||||
text.append(' ');
|
||||
text.append(element);
|
||||
visited.add(element);
|
||||
}
|
||||
field.setStringValue(text.toString());
|
||||
iw.addDocument(document);
|
||||
for(int threadID=0;threadID<threadCount;threadID++) {
|
||||
threads[threadID] = new Thread() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
Document document = new Document();
|
||||
Field field = newTextField("field", "", Field.Store.NO);
|
||||
document.add(field);
|
||||
startingGun.await();
|
||||
while (!postings.isEmpty()) {
|
||||
StringBuilder text = new StringBuilder();
|
||||
Set<String> visited = new HashSet<String>();
|
||||
for (int i = 0; i < maxTermsPerDoc; i++) {
|
||||
String token = postings.poll();
|
||||
if (token == null) {
|
||||
break;
|
||||
}
|
||||
if (visited.contains(token)) {
|
||||
// Put it back:
|
||||
postings.add(token);
|
||||
break;
|
||||
}
|
||||
text.append(' ');
|
||||
text.append(token);
|
||||
visited.add(token);
|
||||
}
|
||||
field.setStringValue(text.toString());
|
||||
iw.addDocument(document);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
};
|
||||
threads[threadID].start();
|
||||
}
|
||||
startingGun.countDown();
|
||||
for(Thread t : threads) {
|
||||
t.join();
|
||||
}
|
||||
|
||||
iw.forceMerge(1);
|
||||
DirectoryReader ir = iw.getReader();
|
||||
assertEquals(1, ir.leaves().size());
|
||||
AtomicReader air = ir.leaves().get(0).reader();
|
||||
Terms terms = air.terms("field");
|
||||
// numTerms-1 because there cannot be a term 0 with 0 postings:
|
||||
assertEquals(numTerms-1, terms.size());
|
||||
TermsEnum termsEnum = terms.iterator(null);
|
||||
BytesRef term;
|
||||
while ((term = termsEnum.next()) != null) {
|
||||
|
|
Loading…
Reference in New Issue