You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by mi...@apache.org on 2011/05/25 00:01:57 UTC

svn commit: r1127314 - in /lucene/java/branches/lucene_3_0/src/test/org/apache/lucene: index/TestNRTThreads.java util/LineFileDocs.java util/LuceneTestCase.java util/_TestUtil.java util/europarl.lines.txt.gz

Author: mikemccand
Date: Tue May 24 22:01:57 2011
New Revision: 1127314

URL: http://svn.apache.org/viewvc?rev=1127314&view=rev
Log:
port TestNRTThreads to 3.0.x

Added:
    lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/index/TestNRTThreads.java   (with props)
    lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/LineFileDocs.java   (with props)
    lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/europarl.lines.txt.gz   (with props)
Modified:
    lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/LuceneTestCase.java
    lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/_TestUtil.java

Added: lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/index/TestNRTThreads.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/index/TestNRTThreads.java?rev=1127314&view=auto
==============================================================================
--- lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/index/TestNRTThreads.java (added)
+++ lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/index/TestNRTThreads.java Tue May 24 22:01:57 2011
@@ -0,0 +1,450 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.lucene.analysis.WhitespaceAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.PhraseQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+
+// TODO
+//   - mix in optimize, addIndexes
+//   - randomoly mix in non-congruent docs
+
+public class TestNRTThreads extends LuceneTestCase {
+
+  private static class SubDocs {
+    public final String packID;
+    public final List<String> subIDs;
+    public boolean deleted;
+
+    public SubDocs(String packID, List<String> subIDs) {
+      this.packID = packID;
+      this.subIDs = subIDs;
+    }
+  }
+
+  public void testNRTThreads() throws Exception {
+
+    final long t0 = System.currentTimeMillis();
+
+    final Random random = new Random(17);
+
+    final LineFileDocs docs = new LineFileDocs(random);
+    final File tempDir = _TestUtil.getTempDir("nrtopenfiles");
+    final Directory dir = FSDirectory.open(tempDir);
+
+    final IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer(), IndexWriter.MaxFieldLength.UNLIMITED);
+    writer.setMergedSegmentWarmer(new IndexWriter.IndexReaderWarmer() {
+      @Override
+      public void warm(IndexReader reader) throws IOException {
+        if (VERBOSE) {
+          System.out.println("TEST: now warm merged reader=" + reader);
+        }
+        final int maxDoc = reader.maxDoc();
+        int sum = 0;
+        final int inc = Math.max(1, maxDoc/50);
+        for(int docID=0;docID<maxDoc;docID += inc) {
+          if (reader.isDeleted(docID)) {
+            final Document doc = reader.document(docID);
+            sum += doc.getFields().size();
+          }
+        }
+
+        IndexSearcher searcher = new IndexSearcher(reader);
+        sum += searcher.search(new TermQuery(new Term("body", "united")), 10).totalHits;
+        searcher.close();
+
+        if (VERBOSE) {
+          System.out.println("TEST: warm visited " + sum + " fields");
+        }
+      }
+      });
+    
+    if (VERBOSE) {
+      writer.setInfoStream(System.out);
+    }
+
+    final int NUM_INDEX_THREADS = 2;
+    final int NUM_SEARCH_THREADS = 3;
+
+    final int RUN_TIME_SEC = 5;
+    //final int RUN_TIME_SEC = 1800;
+
+    final AtomicBoolean failed = new AtomicBoolean();
+    final AtomicInteger addCount = new AtomicInteger();
+    final AtomicInteger delCount = new AtomicInteger();
+
+    final Set<String> delIDs = Collections.synchronizedSet(new HashSet<String>());
+    final List<SubDocs> allSubDocs = Collections.synchronizedList(new ArrayList<SubDocs>());
+
+    final long stopTime = System.currentTimeMillis() + RUN_TIME_SEC*1000;
+    Thread[] threads = new Thread[NUM_INDEX_THREADS];
+    for(int thread=0;thread<NUM_INDEX_THREADS;thread++) {
+      threads[thread] = new Thread() {
+          @Override
+          public void run() {
+            // TODO: would be better if this were cross thread, so that we make sure one thread deleting anothers added docs works:
+            final List<String> toDeleteIDs = new ArrayList<String>();
+            final List<SubDocs> toDeleteSubDocs = new ArrayList<SubDocs>();
+            while(System.currentTimeMillis() < stopTime && !failed.get()) {
+              try {
+                Document doc = docs.nextDoc();
+                if (doc == null) {
+                  break;
+                }
+                final String addedField;
+                if (random.nextBoolean()) {
+                  addedField = "extra" + random.nextInt(10);
+                  doc.add(new Field(addedField, "a random field", Field.Store.NO, Field.Index.ANALYZED));
+                } else {
+                  addedField = null;
+                }
+                if (random.nextBoolean()) {
+                  if (VERBOSE) {
+                    System.out.println(Thread.currentThread().getName() + ": add doc id:" + doc.get("docid"));
+                  }
+
+                  writer.addDocument(doc);
+                  addCount.getAndIncrement();
+                  
+                  if (random.nextInt(5) == 3) {
+                    if (VERBOSE) {
+                      //System.out.println(Thread.currentThread().getName() + ": buffer del id:" + doc.get("docid"));
+                    }
+                    toDeleteIDs.add(doc.get("docid"));
+                  }
+                } else {
+                  // we use update but it never replaces a
+                  // prior doc
+                  if (VERBOSE) {
+                    System.out.println(Thread.currentThread().getName() + ": update doc id:" + doc.get("docid"));
+                  }
+                  writer.updateDocument(new Term("docid", doc.get("docid")), doc);
+                  addCount.getAndIncrement();
+
+                  if (random.nextInt(5) == 3) {
+                    if (VERBOSE) {
+                      //System.out.println(Thread.currentThread().getName() + ": buffer del id:" + doc.get("docid"));
+                    }
+                    toDeleteIDs.add(doc.get("docid"));
+                  }
+                }
+
+                if (random.nextInt(30) == 17) {
+                  if (VERBOSE) {
+                    System.out.println(Thread.currentThread().getName() + ": apply " + toDeleteIDs.size() + " deletes");
+                  }
+                  for(String id : toDeleteIDs) {
+                    if (VERBOSE) {
+                      System.out.println(Thread.currentThread().getName() + ": del term=id:" + id);
+                    }
+                    writer.deleteDocuments(new Term("docid", id));
+                  }
+                  final int count = delCount.addAndGet(toDeleteIDs.size());
+                  if (VERBOSE) {
+                    System.out.println(Thread.currentThread().getName() + ": tot " + count + " deletes");
+                  }
+                  delIDs.addAll(toDeleteIDs);
+                  toDeleteIDs.clear();
+
+                  for(SubDocs subDocs : toDeleteSubDocs) {
+                    assert !subDocs.deleted;
+                    writer.deleteDocuments(new Term("packID", subDocs.packID));
+                    subDocs.deleted = true;
+                    if (VERBOSE) {
+                      System.out.println("  del subs: " + subDocs.subIDs + " packID=" + subDocs.packID);
+                    }
+                    delIDs.addAll(subDocs.subIDs);
+                    delCount.addAndGet(subDocs.subIDs.size());
+                  }
+                  toDeleteSubDocs.clear();
+                }
+                if (addedField != null) {
+                  doc.removeField(addedField);
+                }
+              } catch (Exception exc) {
+                System.out.println(Thread.currentThread().getName() + ": hit exc");
+                exc.printStackTrace();
+                failed.set(true);
+                throw new RuntimeException(exc);
+              }
+            }
+            if (VERBOSE) {
+              System.out.println(Thread.currentThread().getName() + ": indexing done");
+            }
+          }
+        };
+      threads[thread].setDaemon(true);
+      threads[thread].start();
+    }
+
+    if (VERBOSE) {
+      System.out.println("TEST: DONE start indexing threads [" + (System.currentTimeMillis()-t0) + " ms]");
+    }
+
+    // let index build up a bit
+    Thread.sleep(100);
+
+    IndexReader r = writer.getReader();
+    boolean any = false;
+
+    // silly starting guess:
+    final AtomicInteger totTermCount = new AtomicInteger(100);
+
+    while(System.currentTimeMillis() < stopTime && !failed.get()) {
+      if (random.nextBoolean()) {
+        if (VERBOSE) {
+          System.out.println("TEST: now reopen r=" + r);
+        }
+        final IndexReader r2 = r.reopen();
+        if (r != r2) {
+          r.close();
+          r = r2;
+        }
+      } else {
+        if (VERBOSE) {
+          System.out.println("TEST: now close reader=" + r);
+        }
+        r.close();
+        writer.commit();
+        //assertEquals("open but deleted: " + openDeletedFiles, 0, openDeletedFiles.size());
+        if (VERBOSE) {
+          System.out.println("TEST: now open");
+        }
+        r = writer.getReader();
+      }
+      if (VERBOSE) {
+        System.out.println("TEST: got new reader=" + r);
+      }
+      //System.out.println("numDocs=" + r.numDocs() + "
+      //openDelFileCount=" + dir.openDeleteFileCount());
+
+      smokeTestReader(r);
+
+      if (r.numDocs() > 0) {
+
+        final IndexSearcher s = new IndexSearcher(r);
+
+        // run search threads
+        final long searchStopTime = System.currentTimeMillis() + 500;
+        final Thread[] searchThreads = new Thread[NUM_SEARCH_THREADS];
+        final AtomicInteger totHits = new AtomicInteger();
+        for(int thread=0;thread<NUM_SEARCH_THREADS;thread++) {
+          searchThreads[thread] = new Thread() {
+              @Override
+                public void run() {
+                try {
+                  TermEnum termEnum = s.getIndexReader().terms(new Term("body", ""));
+                  int seenTermCount = 0;
+                  int shift;
+                  int trigger;
+                  if (totTermCount.get() == 0) {
+                    shift = 0;
+                    trigger = 1;
+                  } else {
+                    shift = random.nextInt(totTermCount.get()/10);
+                    trigger = totTermCount.get()/10;
+                  }
+                  while(System.currentTimeMillis() < searchStopTime) {
+                    Term term = termEnum.term();
+                    if (term == null) {
+                      if (seenTermCount == 0) {
+                        break;
+                      }
+                      totTermCount.set(seenTermCount);
+                      seenTermCount = 0;
+                      trigger = totTermCount.get()/10;
+                      //System.out.println("trigger " + trigger);
+                      shift = random.nextInt(totTermCount.get()/10);
+                      termEnum = s.getIndexReader().terms(new Term("body", ""));
+                      continue;
+                    }
+                    seenTermCount++;
+                    // search 10 terms
+                    if (trigger == 0) {
+                      trigger = 1;
+                    }
+                    if ((seenTermCount + shift) % trigger == 0) {
+                      //if (VERBOSE) {
+                      //System.out.println(Thread.currentThread().getName() + " now search body:" + term.utf8ToString());
+                      //}
+                      totHits.addAndGet(runQuery(s, new TermQuery(term)));
+                    }
+                  }
+                  if (VERBOSE) {
+                    System.out.println(Thread.currentThread().getName() + ": search done");
+                  }
+                } catch (Throwable t) {
+                  failed.set(true);
+                  t.printStackTrace(System.out);
+                  throw new RuntimeException(t);
+                }
+              }
+            };
+          searchThreads[thread].setDaemon(true);
+          searchThreads[thread].start();
+        }
+
+        for(int thread=0;thread<NUM_SEARCH_THREADS;thread++) {
+          searchThreads[thread].join();
+        }
+
+        if (VERBOSE) {
+          System.out.println("TEST: DONE search: totHits=" + totHits);
+        }
+      } else {
+        Thread.sleep(100);
+      }
+    }
+
+    if (VERBOSE) {
+      System.out.println("TEST: all searching done [" + (System.currentTimeMillis()-t0) + " ms]");
+    }
+
+    //System.out.println("numDocs=" + r.numDocs() + " openDelFileCount=" + dir.openDeleteFileCount());
+    r.close();
+
+    assertFalse("saw non-zero open-but-deleted count", any);
+    if (VERBOSE) {
+      System.out.println("TEST: now join");
+    }
+    for(int thread=0;thread<NUM_INDEX_THREADS;thread++) {
+      threads[thread].join();
+    }
+    if (VERBOSE) {
+      System.out.println("TEST: done join [" + (System.currentTimeMillis()-t0) + " ms]; addCount=" + addCount + " delCount=" + delCount);
+    }
+
+    final IndexReader r2 = writer.getReader();
+    final IndexSearcher s = new IndexSearcher(r2);
+    boolean doFail = false;
+    for(String id : delIDs) {
+      final TopDocs hits = s.search(new TermQuery(new Term("docid", id)), 1);
+      if (hits.totalHits != 0) {
+        System.out.println("doc id=" + id + " is supposed to be deleted, but got docID=" + hits.scoreDocs[0].doc);
+        doFail = true;
+      }
+    }
+
+    // Make sure each group of sub-docs are still in docID order:
+    for(SubDocs subDocs : allSubDocs) {
+      if (!subDocs.deleted) {
+        // We sort by relevance but the scores should be identical so sort falls back to by docID:
+        TopDocs hits = s.search(new TermQuery(new Term("packID", subDocs.packID)), 20);
+        assertEquals(subDocs.subIDs.size(), hits.totalHits);
+        int lastDocID = -1;
+        int startDocID = -1;
+        for(ScoreDoc scoreDoc : hits.scoreDocs) {
+          final int docID = scoreDoc.doc;
+          if (lastDocID != -1) {
+            assertEquals(1+lastDocID, docID);
+          } else {
+            startDocID = docID;
+          }
+          lastDocID = docID;
+          final Document doc = s.doc(docID);
+          assertEquals(subDocs.packID, doc.get("packID"));
+        }
+
+        lastDocID = startDocID - 1;
+        for(String subID : subDocs.subIDs) {
+          hits = s.search(new TermQuery(new Term("docid", subID)), 1);
+          assertEquals(1, hits.totalHits);
+          final int docID = hits.scoreDocs[0].doc;
+          if (lastDocID != -1) {
+            assertEquals(1+lastDocID, docID);
+          }
+          lastDocID = docID;
+        }          
+      } else {
+        for(String subID : subDocs.subIDs) {
+          assertEquals(0, s.search(new TermQuery(new Term("docid", subID)), 1).totalHits);
+        }
+      }
+    }
+    
+    final int endID = Integer.parseInt(docs.nextDoc().get("docid"));
+    for(int id=0;id<endID;id++) {
+      String stringID = ""+id;
+      if (!delIDs.contains(stringID)) {
+        final TopDocs hits = s.search(new TermQuery(new Term("docid", stringID)), 1);
+        if (hits.totalHits != 1) {
+          System.out.println("doc id=" + stringID + " is not supposed to be deleted, but got hitCount=" + hits.totalHits);
+          doFail = true;
+        }
+      }
+    }
+    assertFalse(doFail);
+    
+    assertEquals("index=" + writer.segString() + " addCount=" + addCount + " delCount=" + delCount, addCount.get() - delCount.get(), r2.numDocs());
+    r2.close();
+
+    writer.commit();
+    assertEquals("index=" + writer.segString() + " addCount=" + addCount + " delCount=" + delCount, addCount.get() - delCount.get(), writer.numDocs());
+
+    writer.close(false);
+    _TestUtil.checkIndex(dir);
+    s.close();
+    dir.close();
+    _TestUtil.rmDir(tempDir);
+    docs.close();
+    if (VERBOSE) {
+      System.out.println("TEST: done [" + (System.currentTimeMillis()-t0) + " ms]");
+    }
+  }
+
+  private int runQuery(IndexSearcher s, Query q) throws Exception {
+    s.search(q, 10);
+    return s.search(q, null, 10, new Sort(new SortField("title", SortField.STRING))).totalHits;
+  }
+
+  private void smokeTestReader(IndexReader r) throws Exception {
+    IndexSearcher s = new IndexSearcher(r);
+    runQuery(s, new TermQuery(new Term("body", "united")));
+    runQuery(s, new TermQuery(new Term("titleTokenized", "states")));
+    PhraseQuery pq = new PhraseQuery();
+    pq.add(new Term("body", "united"));
+    pq.add(new Term("body", "states"));
+    runQuery(s, pq);
+    s.close();
+  }
+}

Propchange: lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/index/TestNRTThreads.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/LineFileDocs.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/LineFileDocs.java?rev=1127314&view=auto
==============================================================================
--- lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/LineFileDocs.java (added)
+++ lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/LineFileDocs.java Tue May 24 22:01:57 2011
@@ -0,0 +1,178 @@
+package org.apache.lucene.util;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
+import java.io.InputStream;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.zip.GZIPInputStream;
+import java.util.Random;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+
+/** Minimal port of contrib/benchmark's LneDocSource +
+ * DocMaker, so tests can enum docs from a line file created
+ * by contrib/benchmark's WriteLineDoc task */
+public class LineFileDocs implements Closeable {
+
+  private BufferedReader reader;
+  private final static int BUFFER_SIZE = 1 << 16;     // 64K
+  private final AtomicInteger id = new AtomicInteger();
+  private final String path;
+
+  /** If forever is true, we rewind the file at EOF (repeat
+   * the docs over and over) */
+  public LineFileDocs(Random random, String path) throws IOException {
+    this.path = path;
+    open(random);
+  }
+
+  public LineFileDocs(Random random) throws IOException {
+    this(random, LuceneTestCase.TEST_LINE_DOCS_FILE);
+  }
+
+  public synchronized void close() throws IOException {
+    if (reader != null) {
+      reader.close();
+      reader = null;
+    }
+  }
+
+  private synchronized void open(Random random) throws IOException {
+    InputStream is = getClass().getResourceAsStream(path);
+    if (is == null) {
+      // if its not in classpath, we load it as absolute filesystem path (e.g. Hudson's home dir)
+      is = new FileInputStream(path);
+    }
+    File file = new File(path);
+    long size;
+    if (file.exists()) {
+      size = file.length();
+    } else {
+      size = is.available();
+    }
+    if (path.endsWith(".gz")) {
+      is = new GZIPInputStream(is);
+      // guestimate:
+      size *= 2.8;
+    }
+
+    reader = new BufferedReader(new InputStreamReader(is, "UTF-8"), BUFFER_SIZE);
+
+    // Override sizes for currently "known" line files:
+    if (path.equals("europarl.lines.txt.gz")) {
+      size = 15129506L;
+    } else if (path.equals("/home/hudson/lucene-data/enwiki.random.lines.txt.gz")) {
+      size = 3038178822L;
+    }
+
+    // Randomly seek to starting point:
+    if (random != null && size > 3) {
+      final long seekTo = (random.nextLong()&Long.MAX_VALUE) % (size/3);
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("TEST: LineFileDocs: seek to fp=" + seekTo + " on open");
+      }
+      reader.skip(seekTo);
+      reader.readLine();
+    }
+  }
+
+  public synchronized void reset(Random random) throws IOException {
+    close();
+    open(random);
+    id.set(0);
+  }
+
+  private final static char SEP = '\t';
+
+  private static final class DocState {
+    final Document doc;
+    final Field titleTokenized;
+    final Field title;
+    final Field body;
+    final Field id;
+    final Field date;
+
+    public DocState() {
+      doc = new Document();
+      
+      title = new Field("title", "", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS);
+      doc.add(title);
+
+      titleTokenized = new Field("titleTokenized", "", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS);
+      doc.add(titleTokenized);
+
+      body = new Field("body", "", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS);
+      doc.add(body);
+
+      id = new Field("docid", "", Field.Store.YES, Field.Index.NOT_ANALYZED_NO_NORMS);
+      doc.add(id);
+
+      date = new Field("date", "", Field.Store.YES, Field.Index.NOT_ANALYZED_NO_NORMS);
+      doc.add(date);
+    }
+  }
+
+  private final ThreadLocal<DocState> threadDocs = new ThreadLocal<DocState>();
+
+  /** Note: Document instance is re-used per-thread */
+  public Document nextDoc() throws IOException {
+    String line;
+    synchronized(this) {
+      line = reader.readLine();
+      if (line == null) {
+        // Always rewind at end:
+        if (LuceneTestCase.VERBOSE) {
+          System.out.println("TEST: LineFileDocs: now rewind file...");
+        }
+        close();
+        open(null);
+        line = reader.readLine();
+      }
+    }
+
+    DocState docState = threadDocs.get();
+    if (docState == null) {
+      docState = new DocState();
+      threadDocs.set(docState);
+    }
+
+    int spot = line.indexOf(SEP);
+    if (spot == -1) {
+      throw new RuntimeException("line: [" + line + "] is in an invalid format !");
+    }
+    int spot2 = line.indexOf(SEP, 1 + spot);
+    if (spot2 == -1) {
+      throw new RuntimeException("line: [" + line + "] is in an invalid format !");
+    }
+
+    docState.body.setValue(line.substring(1+spot2, line.length()));
+    final String title = line.substring(0, spot);
+    docState.title.setValue(title);
+    docState.titleTokenized.setValue(title);
+    docState.date.setValue(line.substring(1+spot, spot2));
+    docState.id.setValue(Integer.toString(id.getAndIncrement()));
+    return docState.doc;
+  }
+}

Propchange: lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/LineFileDocs.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/LuceneTestCase.java?rev=1127314&r1=1127313&r2=1127314&view=diff
==============================================================================
--- lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/LuceneTestCase.java Tue May 24 22:01:57 2011
@@ -50,6 +50,9 @@ import org.apache.lucene.util.FieldCache
  */
 public abstract class LuceneTestCase extends TestCase {
 
+  public static final boolean VERBOSE = Boolean.getBoolean("tests.verbose");
+  public static final String TEST_LINE_DOCS_FILE = System.getProperty("tests.linedocsfile", "europarl.lines.txt.gz");
+
   public static final File TEMP_DIR;
   static {
     String s = System.getProperty("tempDir", System.getProperty("java.io.tmpdir"));

Modified: lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/_TestUtil.java?rev=1127314&r1=1127313&r2=1127314&view=diff
==============================================================================
--- lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/_TestUtil.java Tue May 24 22:01:57 2011
@@ -117,4 +117,8 @@ public class _TestUtil {
     return 1024 + new Random().nextInt(64512);
   }
 
+  /** start and end are BOTH inclusive */
+  public static int nextInt(Random r, int start, int end) {
+    return start + r.nextInt(end-start+1);
+  }
 }

Added: lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/europarl.lines.txt.gz
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/europarl.lines.txt.gz?rev=1127314&view=auto
==============================================================================
Binary file - no diff available.

Propchange: lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/europarl.lines.txt.gz
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream