You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mv...@apache.org on 2012/02/07 22:22:50 UTC

svn commit: r1241634 - in /lucene/dev/branches/branch_3x/lucene/contrib: ./ join/src/java/org/apache/lucene/search/join/ join/src/test/org/apache/lucene/search/

Author: mvg
Date: Tue Feb  7 21:22:47 2012
New Revision: 1241634

URL: http://svn.apache.org/viewvc?rev=1241634&view=rev
Log:
LUCENE-3602: Added query time joining under the join contrib.

Added:
    lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/JoinUtil.java
      - copied, changed from r1233078, lucene/dev/trunk/modules/join/src/java/org/apache/lucene/search/join/JoinUtil.java
    lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/TermsCollector.java
      - copied, changed from r1233078, lucene/dev/trunk/modules/join/src/java/org/apache/lucene/search/join/TermsCollector.java
    lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/TermsQuery.java
      - copied, changed from r1233078, lucene/dev/trunk/modules/join/src/java/org/apache/lucene/search/join/TermsQuery.java
    lucene/dev/branches/branch_3x/lucene/contrib/join/src/test/org/apache/lucene/search/TestJoinUtil.java
      - copied, changed from r1233078, lucene/dev/trunk/modules/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
Modified:
    lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt
    lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/package.html

Modified: lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt?rev=1241634&r1=1241633&r2=1241634&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt Tue Feb  7 21:22:47 2012
@@ -49,6 +49,8 @@ New Features
   
  * LUCENE-1812: Added static index pruning contrib module.
    (Andrzej Bialecki, Doron Cohen)
+
+ * LUCENE-3602: Added query time joining under the join contrib. (Martijn van Groningen, Michael McCandless)
   
 API Changes
 

Copied: lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/JoinUtil.java (from r1233078, lucene/dev/trunk/modules/join/src/java/org/apache/lucene/search/join/JoinUtil.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/JoinUtil.java?p2=lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/JoinUtil.java&p1=lucene/dev/trunk/modules/join/src/java/org/apache/lucene/search/join/JoinUtil.java&r1=1233078&r2=1241634&rev=1241634&view=diff
==============================================================================
--- lucene/dev/trunk/modules/join/src/java/org/apache/lucene/search/join/JoinUtil.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/JoinUtil.java Tue Feb  7 21:22:47 2012
@@ -36,24 +36,25 @@ public final class JoinUtil {
   /**
    * Method for query time joining.
    * <p/>
-   * Execute the returned query with a {@link IndexSearcher} to retrieve all documents that have the same terms in the
+   * Execute the returned query with a {@link org.apache.lucene.search.IndexSearcher} to retrieve all documents that have the same terms in the
    * to field that match with documents matching the specified fromQuery and have the same terms in the from field.
    *
+   * Notice: Can't join documents with a fromField that holds more then one term.
+   *
+   *
    * @param fromField                 The from field to join from
-   * @param multipleValuesPerDocument Whether the from field has multiple terms per document
    * @param toField                   The to field to join to
    * @param fromQuery                 The query to match documents on the from side
    * @param fromSearcher              The searcher that executed the specified fromQuery
-   * @return a {@link Query} instance that can be used to join documents based on the
+   * @return a {@link org.apache.lucene.search.Query} instance that can be used to join documents based on the
    *         terms in the from and to field
-   * @throws IOException If I/O related errors occur
+   * @throws java.io.IOException If I/O related errors occur
    */
   public static Query createJoinQuery(String fromField,
-                                      boolean multipleValuesPerDocument,
                                       String toField,
                                       Query fromQuery,
                                       IndexSearcher fromSearcher) throws IOException {
-    TermsCollector termsCollector = TermsCollector.create(fromField, multipleValuesPerDocument);
+    TermsCollector termsCollector = new TermsCollector(fromField);
     fromSearcher.search(fromQuery, termsCollector);
     return new TermsQuery(toField, termsCollector.getCollectorTerms());
   }

Copied: lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/TermsCollector.java (from r1233078, lucene/dev/trunk/modules/join/src/java/org/apache/lucene/search/join/TermsCollector.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/TermsCollector.java?p2=lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/TermsCollector.java&p1=lucene/dev/trunk/modules/join/src/java/org/apache/lucene/search/join/TermsCollector.java&r1=1233078&r2=1241634&rev=1241634&view=diff
==============================================================================
--- lucene/dev/trunk/modules/join/src/java/org/apache/lucene/search/join/TermsCollector.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/TermsCollector.java Tue Feb  7 21:22:47 2012
@@ -17,32 +17,32 @@ package org.apache.lucene.search.join;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocTermOrds;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.Scorer;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefHash;
 
 import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
 
 /**
  * A collector that collects all terms from a specified field matching the query.
  *
  * @lucene.experimental
  */
-abstract class TermsCollector extends Collector {
+class TermsCollector extends Collector {
 
   final String field;
-  final BytesRefHash collectorTerms = new BytesRefHash();
+  final Set<String> collectorTerms = new HashSet<String>();
+  
+  String[] fromDocTerms;
 
   TermsCollector(String field) {
     this.field = field;
   }
 
-  public BytesRefHash getCollectorTerms() {
+  public Set<String> getCollectorTerms() {
     return collectorTerms;
   }
 
@@ -53,71 +53,12 @@ abstract class TermsCollector extends Co
     return true;
   }
 
-  /**
-   * Chooses the right {@link TermsCollector} implementation.
-   *
-   * @param field                     The field to collect terms for
-   * @param multipleValuesPerDocument Whether the field to collect terms for has multiple values per document.
-   * @return a {@link TermsCollector} instance
-   */
-  static TermsCollector create(String field, boolean multipleValuesPerDocument) {
-    return multipleValuesPerDocument ? new MV(field) : new SV(field);
-  }
-
-  // impl that works with multiple values per document
-  static class MV extends TermsCollector {
-
-    private DocTermOrds docTermOrds;
-    private TermsEnum docTermsEnum;
-    private DocTermOrds.TermOrdsIterator reuse;
-
-    MV(String field) {
-      super(field);
-    }
-
-    public void collect(int doc) throws IOException {
-      reuse = docTermOrds.lookup(doc, reuse);
-      int[] buffer = new int[5];
-
-      int chunk;
-      do {
-        chunk = reuse.read(buffer);
-        if (chunk == 0) {
-          return;
-        }
-
-        for (int idx = 0; idx < chunk; idx++) {
-          int key = buffer[idx];
-          docTermsEnum.seekExact((long) key);
-          collectorTerms.add(docTermsEnum.term());
-        }
-      } while (chunk >= buffer.length);
-    }
-
-    public void setNextReader(IndexReader.AtomicReaderContext context) throws IOException {
-      docTermOrds = FieldCache.DEFAULT.getDocTermOrds(context.reader, field);
-      docTermsEnum = docTermOrds.getOrdTermsEnum(context.reader);
-      reuse = null; // LUCENE-3377 needs to be fixed first then this statement can be removed...
-    }
-  }
-
-  // impl that works with single value per document
-  static class SV extends TermsCollector {
-
-    final BytesRef spare = new BytesRef();
-    private FieldCache.DocTerms fromDocTerms;
-
-    SV(String field) {
-      super(field);
-    }
-
-    public void collect(int doc) throws IOException {
-      collectorTerms.add(fromDocTerms.getTerm(doc, spare));
-    }
-
-    public void setNextReader(IndexReader.AtomicReaderContext context) throws IOException {
-      fromDocTerms = FieldCache.DEFAULT.getTerms(context.reader, field);
-    }
+  public void collect(int doc) throws IOException {
+    collectorTerms.add(fromDocTerms[doc]);
+  }
+
+  public void setNextReader(IndexReader indexReader, int docBase) throws IOException {
+    fromDocTerms = FieldCache.DEFAULT.getStrings(indexReader, field);
   }
 
 }

Copied: lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/TermsQuery.java (from r1233078, lucene/dev/trunk/modules/join/src/java/org/apache/lucene/search/join/TermsQuery.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/TermsQuery.java?p2=lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/TermsQuery.java&p1=lucene/dev/trunk/modules/join/src/java/org/apache/lucene/search/join/TermsQuery.java&r1=1233078&r2=1241634&rev=1241634&view=diff
==============================================================================
--- lucene/dev/trunk/modules/join/src/java/org/apache/lucene/search/join/TermsQuery.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/TermsQuery.java Tue Feb  7 21:22:47 2012
@@ -17,16 +17,15 @@ package org.apache.lucene.search.join;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.FilteredTermsEnum;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermEnum;
+import org.apache.lucene.search.FilteredTermEnum;
 import org.apache.lucene.search.MultiTermQuery;
-import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefHash;
 
 import java.io.IOException;
-import java.util.Comparator;
+import java.util.Arrays;
+import java.util.Set;
 
 /**
  * A query that has an array of terms from a specific field. This query will match documents have one or more terms in
@@ -36,23 +35,61 @@ import java.util.Comparator;
  */
 class TermsQuery extends MultiTermQuery {
 
-  private final BytesRefHash terms;
+  private static final FilteredTermEnum EMPTY = new FilteredTermEnum() {
+
+    public boolean next() throws IOException {
+      return false;
+    }
+
+    public Term term() {
+      return null;
+    }
+
+    public int docFreq() {
+      throw new IllegalStateException("this method should never be called");
+    }
+
+    public void close() throws IOException {
+    }
+
+    protected boolean termCompare(Term term) {
+      throw new IllegalStateException("this method should never be called");
+    }
+
+    public float difference() {
+      throw new IllegalStateException("this method should never be called");
+    }
+
+    protected boolean endEnum() {
+      throw new IllegalStateException("this method should never be called");
+    }
+  };
+
+  private final String[] terms;
+  private final String field;
 
   /**
    * @param field The field that should contain terms that are specified in the previous parameter
-   * @param terms The terms that matching documents should have. The terms must be sorted by natural order.
+   * @param terms A set terms that matching documents should have.
    */
-  TermsQuery(String field, BytesRefHash terms) {
-    super(field);
-    this.terms = terms;
+  TermsQuery(String field, Set<String> terms) {
+    this.field = field;
+    this.terms = terms.toArray(new String[terms.size()]);
+    Arrays.sort(this.terms);
   }
 
-  protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-    if (this.terms.size() == 0) {
-      return TermsEnum.EMPTY;
+  protected FilteredTermEnum getEnum(IndexReader reader) throws IOException {
+    if (terms.length == 0) {
+      return EMPTY;
     }
 
-    return new SeekingTermSetTermsEnum(terms.iterator(null), this.terms);
+    TermEnum termEnum = reader.terms(new Term(field, terms[0]));
+    Term firstTerm = termEnum.term();
+    if (firstTerm == null || field != firstTerm.field()) { // interned comparison
+      return EMPTY;
+    }
+
+    return new SeekingTermsEnum(termEnum, firstTerm, field, terms);
   }
 
   public String toString(String string) {
@@ -61,75 +98,71 @@ class TermsQuery extends MultiTermQuery 
         '}';
   }
 
-  static class SeekingTermSetTermsEnum extends FilteredTermsEnum {
-
-    private final BytesRefHash terms;
-    private final int[] ords;
-    private final int lastElement;
+  static class SeekingTermsEnum extends FilteredTermEnum {
 
-    private final BytesRef lastTerm;
-    private final BytesRef spare = new BytesRef();
-    private final Comparator<BytesRef> comparator;
+    private final String[] terms;
+    private final String field;
+    private final int lastPosition;
 
-    private BytesRef seekTerm;
-    private int upto = 0;
+    private boolean endEnum;
+    private int upto;
 
-    SeekingTermSetTermsEnum(TermsEnum tenum, BytesRefHash terms) throws IOException {
-      super(tenum);
+    SeekingTermsEnum(TermEnum termEnum, Term firstTerm, String field, String[] terms) throws IOException {
       this.terms = terms;
+      this.field = field;
+      this.lastPosition = terms.length - 1;
 
-      lastElement = terms.size() - 1;
-      ords = terms.sort(comparator = tenum.getComparator());
-      lastTerm = terms.get(ords[lastElement], new BytesRef());
-      seekTerm = terms.get(ords[upto], spare);
-    }
-
-    @Override
-    protected BytesRef nextSeekTerm(BytesRef currentTerm) throws IOException {
-      BytesRef temp = seekTerm;
-      seekTerm = null;
-      return temp;
-    }
-
-    protected AcceptStatus accept(BytesRef term) throws IOException {
-      if (comparator.compare(term, lastTerm) > 0) {
-        return AcceptStatus.END;
+      upto = Arrays.binarySearch(terms, firstTerm.text());
+      if (upto < 0) {
+        upto = 0;
       }
+      endEnum = upto == lastPosition;
+      setEnum(termEnum);
+    }
 
-      BytesRef currentTerm = terms.get(ords[upto], spare);
-      if (comparator.compare(term, currentTerm) == 0) {
-        if (upto == lastElement) {
-          return AcceptStatus.YES;
-        } else {
-          seekTerm = terms.get(ords[++upto], spare);
-          return AcceptStatus.YES_AND_SEEK;
+    protected boolean termCompare(Term term) {
+      if (term == null || term.field() != field) { // interned comparison
+        endEnum = true;
+        return false;
+      }
+
+      int cmp = terms[upto].compareTo(term.text());
+      if (cmp < 0) {
+        if (upto == lastPosition) {
+          return false;
         }
-      } else {
-        if (upto == lastElement) {
-          return AcceptStatus.NO;
-        } else { // Our current term doesn't match the the given term.
-          int cmp;
-          do { // We maybe are behind the given term by more than one step. Keep incrementing till we're the same or higher.
-            if (upto == lastElement) {
-              return AcceptStatus.NO;
-            }
-            // typically the terms dict is a superset of query's terms so it's unusual that we have to skip many of
-            // our terms so we don't do a binary search here
-            seekTerm = terms.get(ords[++upto], spare);
-          } while ((cmp = comparator.compare(seekTerm, term)) < 0);
-          if (cmp == 0) {
-            if (upto == lastElement) {
-              return AcceptStatus.YES;
-            }
-            seekTerm = terms.get(ords[++upto], spare);
-            return AcceptStatus.YES_AND_SEEK;
-          } else {
-            return AcceptStatus.NO_AND_SEEK;
+
+        while ((cmp = terms[++upto].compareTo(term.text())) < 0) {
+          if (upto == lastPosition) {
+            endEnum = true;
+            return false;
           }
         }
+        assert cmp >= 0 : "cmp cannot be lower than zero";
+        if (cmp == 0) {
+          upto++;
+          endEnum = upto > lastPosition;
+          return true;
+        } else {
+          return false;
+        }
+      } else if (cmp > 0) {
+        return false;
+      } else {
+        upto++;
+        endEnum = upto > lastPosition;
+        return true;
       }
     }
 
+    public float difference() {
+      return 1.0f;
+    }
+
+    protected boolean endEnum() {
+      return endEnum;
+    }
+
   }
 
 }

Modified: lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/package.html?rev=1241634&r1=1241633&r2=1241634&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/package.html (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/package.html Tue Feb  7 21:22:47 2012
@@ -1,7 +1,11 @@
 <html>
 <body>
 
-<p>This module supports index-time joins while searching, where joined
+<p>This modules support index-time and query-time joins.</p>
+
+<h2>Index-time joins</h2>
+
+<p>The index-time joining support joins while searching, where joined
   documents are indexed as a single document block using
   {@link org.apache.lucene.index.IndexWriter#addDocuments}.  This is useful for any normalized content (XML documents or database tables).  In database terms, all rows for all
   joined tables matching a single row of the primary table must be
@@ -34,5 +38,35 @@
   org.apache.lucene.search.join.ToChildBlockJoinQuery}.  This wraps
   any query matching parent documents, creating the joined query
   matching only child documents.
+
+<h2>Search-time joins</h2>
+
+<p>
+  The query time joining is index term based and implemented as two pass search. The first pass collects all the terms from a fromField
+  that match the fromQuery. The second pass returns all documents that have matching terms in a toField to the terms
+  collected in the first pass.
+</p>
+<p>Query time joining has the following input:</p>
+<ul>
+  <li><code>fromField</code>: The from field to join from.
+  <li><code>fromQuery</code>:  The query executed to collect the from terms. This is usually the user specified query.
+  <li><code>toField</code>: The to field to join to
+</ul>
+<p>
+  Basically the query-time joining is accessible from one static method. The user of this method supplies the method
+  with the described input and a <code>IndexSearcher</code> where the from terms need to be collected from. The returned
+  query can be executed with the same <code>IndexSearcher</code>, but also with another <code>IndexSearcher</code>.
+  Example usage of the {@link org.apache.lucene.search.join.JoinUtil#createJoinQuery(String, String, org.apache.lucene.search.Query, org.apache.lucene.search.IndexSearcher)} :
+</p>
+<pre class="prettyprint">
+  String fromField = "from"; // Name of the from field
+  String toField = "to"; // Name of the to field
+  Query fromQuery = new TermQuery(new Term("content", searchTerm)); // Query executed to collect from values to join to the to values
+
+  Query joinQuery = JoinUtil.createJoinQuery(fromField, toField, fromQuery, fromSearcher);
+  TopDocs topDocs = toSearcher.search(joinQuery, 10); // Note: toSearcher can be the same as the fromSearcher
+  // Render topDocs...
+</pre>
+
 </body>
 </html>

Copied: lucene/dev/branches/branch_3x/lucene/contrib/join/src/test/org/apache/lucene/search/TestJoinUtil.java (from r1233078, lucene/dev/trunk/modules/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/join/src/test/org/apache/lucene/search/TestJoinUtil.java?p2=lucene/dev/branches/branch_3x/lucene/contrib/join/src/test/org/apache/lucene/search/TestJoinUtil.java&p1=lucene/dev/trunk/modules/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java&r1=1233078&r2=1241634&rev=1241634&view=diff
==============================================================================
--- lucene/dev/trunk/modules/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/join/src/test/org/apache/lucene/search/TestJoinUtil.java Tue Feb  7 21:22:47 2012
@@ -1,4 +1,4 @@
-package org.apache.lucene.search.join;
+package org.apache.lucene.search;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -21,11 +21,9 @@ import org.apache.lucene.analysis.MockAn
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.*;
-import org.apache.lucene.search.*;
+import org.apache.lucene.search.join.JoinUtil;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
@@ -49,45 +47,45 @@ public class TestJoinUtil extends Lucene
 
     // 0
     Document doc = new Document();
-    doc.add(new Field("description", "random text", TextField.TYPE_STORED));
-    doc.add(new Field("name", "name1", TextField.TYPE_STORED));
-    doc.add(new Field(idField, "1", TextField.TYPE_STORED));
+    doc.add(new Field("description", "random text", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
+    doc.add(new Field("name", "name1", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
+    doc.add(new Field(idField, "1", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
     w.addDocument(doc);
 
     // 1
     doc = new Document();
-    doc.add(new Field("price", "10.0", TextField.TYPE_STORED));
-    doc.add(new Field(idField, "2", TextField.TYPE_STORED));
-    doc.add(new Field(toField, "1", TextField.TYPE_STORED));
+    doc.add(new Field("price", "10.0", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
+    doc.add(new Field(idField, "2", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
+    doc.add(new Field(toField, "1", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
     w.addDocument(doc);
 
     // 2
     doc = new Document();
-    doc.add(new Field("price", "20.0", TextField.TYPE_STORED));
-    doc.add(new Field(idField, "3", TextField.TYPE_STORED));
-    doc.add(new Field(toField, "1", TextField.TYPE_STORED));
+    doc.add(new Field("price", "20.0", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
+    doc.add(new Field(idField, "3", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
+    doc.add(new Field(toField, "1", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
     w.addDocument(doc);
 
     // 3
     doc = new Document();
-    doc.add(new Field("description", "more random text", TextField.TYPE_STORED));
-    doc.add(new Field("name", "name2", TextField.TYPE_STORED));
-    doc.add(new Field(idField, "4", TextField.TYPE_STORED));
+    doc.add(new Field("description", "more random text", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
+    doc.add(new Field("name", "name2", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
+    doc.add(new Field(idField, "4", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
     w.addDocument(doc);
     w.commit();
 
     // 4
     doc = new Document();
-    doc.add(new Field("price", "10.0", TextField.TYPE_STORED));
-    doc.add(new Field(idField, "5", TextField.TYPE_STORED));
-    doc.add(new Field(toField, "4", TextField.TYPE_STORED));
+    doc.add(new Field("price", "10.0", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
+    doc.add(new Field(idField, "5", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
+    doc.add(new Field(toField, "4", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
     w.addDocument(doc);
 
     // 5
     doc = new Document();
-    doc.add(new Field("price", "20.0", TextField.TYPE_STORED));
-    doc.add(new Field(idField, "6", TextField.TYPE_STORED));
-    doc.add(new Field(toField, "4", TextField.TYPE_STORED));
+    doc.add(new Field("price", "20.0", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
+    doc.add(new Field(idField, "6", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
+    doc.add(new Field(toField, "4", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
     w.addDocument(doc);
 
     IndexSearcher indexSearcher = new IndexSearcher(w.getReader());
@@ -95,21 +93,21 @@ public class TestJoinUtil extends Lucene
 
     // Search for product
     Query joinQuery =
-        JoinUtil.createJoinQuery(idField, false, toField, new TermQuery(new Term("name", "name2")), indexSearcher);
+        JoinUtil.createJoinQuery(idField, toField, new TermQuery(new Term("name", "name2")), indexSearcher);
 
     TopDocs result = indexSearcher.search(joinQuery, 10);
     assertEquals(2, result.totalHits);
     assertEquals(4, result.scoreDocs[0].doc);
     assertEquals(5, result.scoreDocs[1].doc);
 
-    joinQuery = JoinUtil.createJoinQuery(idField, false, toField, new TermQuery(new Term("name", "name1")), indexSearcher);
+    joinQuery = JoinUtil.createJoinQuery(idField, toField, new TermQuery(new Term("name", "name1")), indexSearcher);
     result = indexSearcher.search(joinQuery, 10);
     assertEquals(2, result.totalHits);
     assertEquals(1, result.scoreDocs[0].doc);
     assertEquals(2, result.scoreDocs[1].doc);
 
     // Search for offer
-    joinQuery = JoinUtil.createJoinQuery(toField, false, idField, new TermQuery(new Term("id", "5")), indexSearcher);
+    joinQuery = JoinUtil.createJoinQuery(toField, idField, new TermQuery(new Term("id", "5")), indexSearcher);
     result = indexSearcher.search(joinQuery, 10);
     assertEquals(1, result.totalHits);
     assertEquals(3, result.scoreDocs[0].doc);
@@ -119,21 +117,8 @@ public class TestJoinUtil extends Lucene
   }
 
   @Test
-  public void testSingleValueRandomJoin() throws Exception {
+  public void testRandom() throws Exception {
     int maxIndexIter = _TestUtil.nextInt(random, 6, 12);
-    int maxSearchIter = _TestUtil.nextInt(random, 13, 26);
-    executeRandomJoin(false, maxIndexIter, maxSearchIter);
-  }
-
-  @Test
-  // This test really takes more time, that is why the number of iterations are smaller.
-  public void testMultiValueRandomJoin() throws Exception {
-    int maxIndexIter = _TestUtil.nextInt(random, 3, 6);
-    int maxSearchIter = _TestUtil.nextInt(random, 6, 12);
-    executeRandomJoin(true, maxIndexIter, maxSearchIter);
-  }
-
-  private void executeRandomJoin(boolean multipleValuesPerDocument, int maxIndexIter, int maxSearchIter) throws Exception {
     for (int indexIter = 1; indexIter <= maxIndexIter; indexIter++) {
       if (VERBOSE) {
         System.out.println("indexIter=" + indexIter);
@@ -144,11 +129,12 @@ public class TestJoinUtil extends Lucene
           dir,
           newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random, MockTokenizer.KEYWORD, false)).setMergePolicy(newLogMergePolicy())
       );
-      int numberOfDocumentsToIndex = _TestUtil.nextInt(random, 87, 764);
-      IndexIterationContext context = createContext(numberOfDocumentsToIndex, w, multipleValuesPerDocument);
+      int numberOfDocumentsToIndex = TEST_NIGHTLY ? _TestUtil.nextInt(random, 7889, 14632) : _TestUtil.nextInt(random, 87, 764);
+      IndexIterationContext context = createContext(numberOfDocumentsToIndex, w, false);
 
       IndexReader topLevelReader = w.getReader();
       w.close();
+      int maxSearchIter = _TestUtil.nextInt(random, 13, 26);
       for (int searchIter = 1; searchIter <= maxSearchIter; searchIter++) {
         if (VERBOSE) {
           System.out.println("searchIter=" + searchIter);
@@ -158,7 +144,7 @@ public class TestJoinUtil extends Lucene
         int r = random.nextInt(context.randomUniqueValues.length);
         boolean from = context.randomFrom[r];
         String randomValue = context.randomUniqueValues[r];
-        FixedBitSet expectedResult = createExpectedResult(randomValue, from, indexSearcher.getIndexReader(), context);
+        FixedBitSet expectedResult = createExpectedResult(randomValue, from, indexSearcher, context);
 
         Query actualQuery = new TermQuery(new Term("value", randomValue));
         if (VERBOSE) {
@@ -166,9 +152,9 @@ public class TestJoinUtil extends Lucene
         }
         Query joinQuery;
         if (from) {
-          joinQuery = JoinUtil.createJoinQuery("from", multipleValuesPerDocument, "to", actualQuery, indexSearcher);
+          joinQuery = JoinUtil.createJoinQuery("from", "to", actualQuery, indexSearcher);
         } else {
-          joinQuery = JoinUtil.createJoinQuery("to", multipleValuesPerDocument, "from", actualQuery, indexSearcher);
+          joinQuery = JoinUtil.createJoinQuery("to", "from", actualQuery, indexSearcher);
         }
         if (VERBOSE) {
           System.out.println("joinQuery=" + joinQuery);
@@ -184,8 +170,8 @@ public class TestJoinUtil extends Lucene
             actualResult.set(doc + docBase);
           }
 
-          public void setNextReader(IndexReader.AtomicReaderContext context) throws IOException {
-            docBase = context.docBase;
+          public void setNextReader(IndexReader reader, int docBase) throws IOException {
+            this.docBase = docBase;
           }
 
           public void setScorer(Scorer scorer) throws IOException {
@@ -210,6 +196,7 @@ public class TestJoinUtil extends Lucene
         }
 
         assertEquals(expectedResult, actualResult);
+        indexSearcher.close();
       }
       topLevelReader.close();
       dir.close();
@@ -243,8 +230,8 @@ public class TestJoinUtil extends Lucene
       int randomI = random.nextInt(context.randomUniqueValues.length);
       String value = context.randomUniqueValues[randomI];
       Document document = new Document();
-      document.add(newField(random, "id", id, TextField.TYPE_STORED));
-      document.add(newField(random, "value", value, TextField.TYPE_STORED));
+      document.add(newField(random, "id", id, Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
+      document.add(newField(random, "value", value, Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
 
       boolean from = context.randomFrom[randomI];
       int numberOfLinkValues = multipleValuesPerDocument ? 2 + random.nextInt(10) : 1;
@@ -262,7 +249,7 @@ public class TestJoinUtil extends Lucene
 
           context.fromDocuments.get(linkValue).add(doc);
           context.randomValueFromDocs.get(value).add(doc);
-          document.add(newField(random, "from", linkValue, TextField.TYPE_STORED));
+          document.add(newField(random, "from", linkValue, Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
         } else {
           if (!context.toDocuments.containsKey(linkValue)) {
             context.toDocuments.put(linkValue, new ArrayList<RandomDoc>());
@@ -273,7 +260,7 @@ public class TestJoinUtil extends Lucene
 
           context.toDocuments.get(linkValue).add(doc);
           context.randomValueToDocs.get(value).add(doc);
-          document.add(newField(random, "to", linkValue, TextField.TYPE_STORED));
+          document.add(newField(random, "to", linkValue, Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS));
         }
       }
 
@@ -295,7 +282,7 @@ public class TestJoinUtil extends Lucene
     return context;
   }
 
-  private FixedBitSet createExpectedResult(String queryValue, boolean from, IndexReader topLevelReader, IndexIterationContext context) throws IOException {
+  private FixedBitSet createExpectedResult(String queryValue, boolean from, IndexSearcher searcher, IndexIterationContext context) throws IOException {
     final Map<String, List<RandomDoc>> randomValueDocs;
     final Map<String, List<RandomDoc>> linkValueDocuments;
     if (from) {
@@ -306,10 +293,10 @@ public class TestJoinUtil extends Lucene
       linkValueDocuments = context.fromDocuments;
     }
 
-    FixedBitSet expectedResult = new FixedBitSet(topLevelReader.maxDoc());
+    FixedBitSet expectedResult = new FixedBitSet(searcher.maxDoc());
     List<RandomDoc> matchingDocs = randomValueDocs.get(queryValue);
     if (matchingDocs == null) {
-      return new FixedBitSet(topLevelReader.maxDoc());
+      return new FixedBitSet(searcher.maxDoc());
     }
 
     for (RandomDoc matchingDoc : matchingDocs) {
@@ -320,9 +307,7 @@ public class TestJoinUtil extends Lucene
         }
 
         for (RandomDoc otherSideDoc : otherMatchingDocs) {
-          DocsEnum docsEnum = MultiFields.getTermDocsEnum(topLevelReader, MultiFields.getLiveDocs(topLevelReader), "id", new BytesRef(otherSideDoc.id), false);
-          assert docsEnum != null;
-          int doc = docsEnum.nextDoc();
+          int doc = searcher.search(new TermQuery(new Term("id", otherSideDoc.id)), 1).scoreDocs[0].doc;
           expectedResult.set(doc);
         }
       }