You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2012/01/14 17:16:34 UTC

svn commit: r1231521 - in /lucene/dev/branches/branch_3x: ./ lucene/ lucene/contrib/ lucene/contrib/analyzers/ lucene/contrib/analyzers/common/ lucene/contrib/join/src/java/org/apache/lucene/search/join/ lucene/contrib/join/src/test/org/apache/lucene/s...

Author: mikemccand
Date: Sat Jan 14 16:16:33 2012
New Revision: 1231521

URL: http://svn.apache.org/viewvc?rev=1231521&view=rev
Log:
LUCENE-3685: add ToChildBlockJoinQuery, to join from parent to child

Added:
    lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java   (with props)
    lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java
      - copied, changed from r1231512, lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/BlockJoinCollector.java
    lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
      - copied, changed from r1231512, lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/BlockJoinQuery.java
Removed:
    lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/BlockJoinCollector.java
    lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/BlockJoinQuery.java
Modified:
    lucene/dev/branches/branch_3x/   (props changed)
    lucene/dev/branches/branch_3x/lucene/   (props changed)
    lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt
    lucene/dev/branches/branch_3x/lucene/contrib/analyzers/   (props changed)
    lucene/dev/branches/branch_3x/lucene/contrib/analyzers/common/   (props changed)
    lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/package.html
    lucene/dev/branches/branch_3x/lucene/contrib/join/src/test/org/apache/lucene/search/TestBlockJoin.java
    lucene/dev/branches/branch_3x/solr/   (props changed)

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=1231521&r1=1231520&r2=1231521&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt Sat Jan 14 16:16:33 2012
@@ -130,6 +130,11 @@ New Features
    (next page, drill down, etc.) use the same searcher as before (Mike
    McCandless)
 
+* LUCENE-3685: Add ToChildBlockJoinQuery and renamed previous
+  BlockJoinQuery to ToParentBlockJoinQuery, so that you can now do
+  joins in both parent to child and child to parent directions.
+  (Mike McCandless)
+  
 API Changes
 
  * LUCENE-3431: Deprecated QueryAutoStopWordAnalyzer.addStopWords* since they

Added: lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java?rev=1231521&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java (added)
+++ lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java Sat Jan 14 16:16:33 2012
@@ -0,0 +1,319 @@
+package org.apache.lucene.search.join;
+
+/**
+ * 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.IOException;
+import java.util.Set;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;       // javadocs
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * Just like {@link ToParentBlockJoinQuery}, except this
+ * query joins in reverse: you provide a Query matching
+ * parent documents and it joins down to child
+ * documents.
+ *
+ * @lucene.experimental
+ */
+
+public class ToChildBlockJoinQuery extends Query {
+
+  private final Filter parentsFilter;
+  private final Query parentQuery;
+
+  // If we are rewritten, this is the original parentQuery we
+  // were passed; we use this for .equals() and
+  // .hashCode().  This makes rewritten query equal the
+  // original, so that user does not have to .rewrite() their
+  // query before searching:
+  private final Query origParentQuery;
+  private final boolean doScores;
+
+  public ToChildBlockJoinQuery(Query parentQuery, Filter parentsFilter, boolean doScores) {
+    super();
+    this.origParentQuery = parentQuery;
+    this.parentQuery = parentQuery;
+    this.parentsFilter = parentsFilter;
+    this.doScores = doScores;
+  }
+
+  private ToChildBlockJoinQuery(Query origParentQuery, Query parentQuery, Filter parentsFilter, boolean doScores) {
+    super();
+    this.origParentQuery = origParentQuery;
+    this.parentQuery = parentQuery;
+    this.parentsFilter = parentsFilter;
+    this.doScores = doScores;
+  }
+
+  @Override
+  public Weight createWeight(Searcher searcher) throws IOException {
+    return new ToChildBlockJoinWeight(this, parentQuery.createWeight(searcher), parentsFilter, doScores);
+  }
+
+  private static class ToChildBlockJoinWeight extends Weight {
+    private final Query joinQuery;
+    private final Weight parentWeight;
+    private final Filter parentsFilter;
+    private final boolean doScores;
+
+    public ToChildBlockJoinWeight(Query joinQuery, Weight parentWeight, Filter parentsFilter, boolean doScores) {
+      super();
+      this.joinQuery = joinQuery;
+      this.parentWeight = parentWeight;
+      this.parentsFilter = parentsFilter;
+      this.doScores = doScores;
+    }
+
+    @Override
+    public Query getQuery() {
+      return joinQuery;
+    }
+
+    @Override
+    public float getValue() {
+      return parentWeight.getValue();
+    }
+
+    @Override
+    public float sumOfSquaredWeights() throws IOException {
+      return parentWeight.sumOfSquaredWeights() * joinQuery.getBoost() * joinQuery.getBoost();
+    }
+
+    @Override
+    public void normalize(float norm) {
+      parentWeight.normalize(norm * joinQuery.getBoost());
+    }
+
+    @Override
+    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
+      // Pass scoreDocsInOrder true, topScorer false to our sub:
+      final Scorer parentScorer = parentWeight.scorer(reader, true, false);
+
+      if (parentScorer == null) {
+        // No matches
+        return null;
+      }
+
+      final DocIdSet parents = parentsFilter.getDocIdSet(reader);
+      // TODO: once we do random-access filters we can
+      // generalize this:
+      if (parents == null) {
+        // No matches
+        return null;
+      }
+      if (!(parents instanceof FixedBitSet)) {
+        throw new IllegalStateException("parentFilter must return FixedBitSet; got " + parents);
+      }
+
+      return new ToChildBlockJoinScorer(this, parentScorer, (FixedBitSet) parents, doScores);
+    }
+
+    @Override
+    public Explanation explain(IndexReader reader, int doc) throws IOException {
+      // TODO
+      throw new UnsupportedOperationException(getClass().getName() +
+                                              " cannot explain match on parent document");
+    }
+
+    @Override
+    public boolean scoresDocsOutOfOrder() {
+      return false;
+    }
+  }
+
+  static class ToChildBlockJoinScorer extends Scorer {
+    private final Scorer parentScorer;
+    private final FixedBitSet parentBits;
+    private final boolean doScores;
+    private float parentScore;
+
+    private int childDoc = -1;
+    private int parentDoc;
+
+    public ToChildBlockJoinScorer(Weight weight, Scorer parentScorer, FixedBitSet parentBits, boolean doScores) {
+      super(weight);
+      this.doScores = doScores;
+      this.parentBits = parentBits;
+      this.parentScorer = parentScorer;
+    }
+
+    @Override
+    public void visitSubScorers(Query parent, BooleanClause.Occur relationship,
+                                ScorerVisitor<Query, Query, Scorer> visitor) {
+      super.visitSubScorers(parent, relationship, visitor);
+      //parentScorer.visitSubScorers(weight.getQuery(), BooleanClause.Occur.MUST, visitor);
+      parentScorer.visitScorers(visitor);
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      //System.out.println("Q.nextDoc() parentDoc=" + parentDoc + " childDoc=" + childDoc);
+
+      if (childDoc+1 == parentDoc) {
+        // OK, we are done iterating through all children
+        // matching this one parent doc, so we now nextDoc()
+        // the parent.  Use a while loop because we may have
+        // to skip over some number of parents w/ no
+        // children:
+        while (true) {
+          parentDoc = parentScorer.nextDoc();
+          if (parentDoc == 0) {
+            // Degenerate but allowed: parent has no children
+            // TODO: would be nice to pull initial parent
+            // into ctor so we can skip this if... but it's
+            // tricky because scorer must return -1 for
+            // .doc() on init...
+            parentDoc = parentScorer.nextDoc();
+          }
+
+          if (parentDoc == NO_MORE_DOCS) {
+            childDoc = NO_MORE_DOCS;
+            //System.out.println("  END");
+            return childDoc;
+          }
+
+          childDoc = 1 + parentBits.prevSetBit(parentDoc-1);
+          if (childDoc < parentDoc) {
+            if (doScores) {
+              parentScore = parentScorer.score();
+            }
+            //System.out.println("  " + childDoc);
+            return childDoc;
+          } else {
+            // Degenerate but allowed: parent has no children
+          }
+        }
+      } else {
+        assert childDoc < parentDoc: "childDoc=" + childDoc + " parentDoc=" + parentDoc;
+        childDoc++;
+        //System.out.println("  " + childDoc);
+        return childDoc;
+      }
+    }
+
+    @Override
+    public int docID() {
+      return childDoc;
+    }
+
+    @Override
+    public float score() throws IOException {
+      return parentScore;
+    }
+
+    @Override
+    public int advance(int childTarget) throws IOException {
+
+      //System.out.println("Q.advance childTarget=" + childTarget);
+      if (childTarget == NO_MORE_DOCS) {
+        //System.out.println("  END");
+        return childDoc = parentDoc = NO_MORE_DOCS;
+      }
+
+      assert childTarget != parentDoc;
+      if (childTarget > parentDoc) {
+        // Advance to new parent:
+        parentDoc = parentScorer.advance(childTarget);
+        //System.out.println("  advance to parentDoc=" + parentDoc);
+        assert parentDoc > childTarget;
+        if (parentDoc == NO_MORE_DOCS) {
+          //System.out.println("  END");
+          return childDoc = NO_MORE_DOCS;
+        }
+        if (doScores) {
+          parentScore = parentScorer.score();
+        }
+        final int firstChild = parentBits.prevSetBit(parentDoc-1);
+        //System.out.println("  firstChild=" + firstChild);
+        childTarget = Math.max(childTarget, firstChild);
+      }
+
+      assert childTarget < parentDoc;
+
+      // Advance within children of current parent:
+      childDoc = childTarget;
+      //System.out.println("  " + childDoc);
+      return childDoc;
+    }
+  }
+
+  @Override
+  public void extractTerms(Set<Term> terms) {
+    parentQuery.extractTerms(terms);
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    final Query parentRewrite = parentQuery.rewrite(reader);
+    if (parentRewrite != parentQuery) {
+      Query rewritten = new ToChildBlockJoinQuery(parentQuery,
+                                parentRewrite,
+                                parentsFilter,
+                                doScores);
+      rewritten.setBoost(getBoost());
+      return rewritten;
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public String toString(String field) {
+    return "ToChildBlockJoinQuery ("+parentQuery.toString()+")";
+  }
+
+  @Override
+  public boolean equals(Object _other) {
+    if (_other instanceof ToChildBlockJoinQuery) {
+      final ToChildBlockJoinQuery other = (ToChildBlockJoinQuery) _other;
+      return origParentQuery.equals(other.origParentQuery) &&
+        parentsFilter.equals(other.parentsFilter) &&
+        doScores == other.doScores;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int hash = 1;
+    hash = prime * hash + origParentQuery.hashCode();
+    hash = prime * hash + new Boolean(doScores).hashCode();
+    hash = prime * hash + parentsFilter.hashCode();
+    return hash;
+  }
+
+  @Override
+  public Object clone() {
+    return new ToChildBlockJoinQuery((Query) origParentQuery.clone(),
+                                     parentsFilter,
+                                     doScores);
+  }
+}

Copied: lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java (from r1231512, lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/BlockJoinCollector.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java?p2=lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java&p1=lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/BlockJoinCollector.java&r1=1231512&r2=1231521&rev=1231521&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/BlockJoinCollector.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java Sat Jan 14 16:16:33 2012
@@ -56,7 +56,7 @@ import org.apache.lucene.util.ArrayUtil;
  *
  *  <p>You should only use this
  *  collector if one or more of the clauses in the query is
- *  a {@link BlockJoinQuery}.  This collector will find those query
+ *  a {@link ToParentBlockJoinQuery}.  This collector will find those query
  *  clauses and record the matching child documents for the
  *  top scoring parent documents.</p>
  *
@@ -66,11 +66,11 @@ import org.apache.lucene.util.ArrayUtil;
  *  parent table were indexed as a doc block.</p>
  *
  *  <p>For the simple star join you can retrieve the
- *  {@link TopGroups} instance containing each {@link BlockJoinQuery}'s
+ *  {@link TopGroups} instance containing each {@link ToParentBlockJoinQuery}'s
  *  matching child documents for the top parent groups,
  *  using {@link #getTopGroups}.  Ie,
  *  a single query, which will contain two or more
- *  {@link BlockJoinQuery}'s as clauses representing the star join,
+ *  {@link ToParentBlockJoinQuery}'s as clauses representing the star join,
  *  can then retrieve two or more {@link TopGroups} instances.</p>
  *
  *  <p>For nested joins, the query will run correctly (ie,
@@ -86,7 +86,7 @@ import org.apache.lucene.util.ArrayUtil;
  *
  * @lucene.experimental
  */
-public class BlockJoinCollector extends Collector {
+public class ToParentBlockJoinCollector extends Collector {
 
   private final Sort sort;
 
@@ -102,7 +102,7 @@ public class BlockJoinCollector extends 
   private final boolean trackScores;
 
   private int docBase;
-  private BlockJoinQuery.BlockJoinScorer[] joinScorers = new BlockJoinQuery.BlockJoinScorer[0];
+  private ToParentBlockJoinQuery.BlockJoinScorer[] joinScorers = new ToParentBlockJoinQuery.BlockJoinScorer[0];
   private IndexReader currentReader;
   private Scorer scorer;
   private boolean queueFull;
@@ -111,9 +111,9 @@ public class BlockJoinCollector extends 
   private int totalHitCount;
   private float maxScore = Float.NaN;
 
-  /*  Creates a BlockJoinCollector.  The provided sort must
+  /*  Creates a ToParentBlockJoinCollector.  The provided sort must
    *  not be null. */
-  public BlockJoinCollector(Sort sort, int numParentHits, boolean trackScores, boolean trackMaxScore) throws IOException {
+  public ToParentBlockJoinCollector(Sort sort, int numParentHits, boolean trackScores, boolean trackMaxScore) throws IOException {
     // TODO: allow null sort to be specialized to relevance
     // only collector
     this.sort = sort;
@@ -251,7 +251,7 @@ public class BlockJoinCollector extends 
 
     //System.out.println("copyGroups parentDoc=" + og.doc);
     for(int scorerIDX = 0;scorerIDX < numSubScorers;scorerIDX++) {
-      final BlockJoinQuery.BlockJoinScorer joinScorer = joinScorers[scorerIDX];
+      final ToParentBlockJoinQuery.BlockJoinScorer joinScorer = joinScorers[scorerIDX];
       //System.out.println("  scorer=" + joinScorer);
       if (joinScorer != null) {
         og.counts[scorerIDX] = joinScorer.getChildCount();
@@ -283,6 +283,20 @@ public class BlockJoinCollector extends 
     return false;
   }
 
+  private void enroll(ToParentBlockJoinQuery query, ToParentBlockJoinQuery.BlockJoinScorer scorer) {
+    final Integer slot = joinQueryID.get(query);
+    if (slot == null) {
+      joinQueryID.put(query, joinScorers.length);
+      //System.out.println("found JQ: " + query + " slot=" + joinScorers.length);
+      final ToParentBlockJoinQuery.BlockJoinScorer[] newArray = new ToParentBlockJoinQuery.BlockJoinScorer[1+joinScorers.length];
+      System.arraycopy(joinScorers, 0, newArray, 0, joinScorers.length);
+      joinScorers = newArray;
+      joinScorers[joinScorers.length-1] = scorer;
+    } else {
+      joinScorers[slot] = scorer;
+    }
+  }
+  
   @Override
   public void setScorer(Scorer scorer) {
     //System.out.println("C.setScorer scorer=" + scorer);
@@ -297,44 +311,30 @@ public class BlockJoinCollector extends 
 
     // Find any BlockJoinScorers out there:
     scorer.visitScorers(new Scorer.ScorerVisitor<Query,Query,Scorer>() {
-        private void enroll(BlockJoinQuery query, BlockJoinQuery.BlockJoinScorer scorer) {
-          final Integer slot = joinQueryID.get(query);
-          if (slot == null) {
-            joinQueryID.put(query, joinScorers.length);
-            //System.out.println("found JQ: " + query + " slot=" + joinScorers.length);
-            final BlockJoinQuery.BlockJoinScorer[] newArray = new BlockJoinQuery.BlockJoinScorer[1+joinScorers.length];
-            System.arraycopy(joinScorers, 0, newArray, 0, joinScorers.length);
-            joinScorers = newArray;
-            joinScorers[joinScorers.length-1] = scorer;
-          } else {
-            joinScorers[slot] = scorer;
-          }
-        }
-
         @Override
         public void visitOptional(Query parent, Query child, Scorer scorer) {
           //System.out.println("visitOpt");
-          if (child instanceof BlockJoinQuery) {
-            enroll((BlockJoinQuery) child,
-                   (BlockJoinQuery.BlockJoinScorer) scorer);
+          if (child instanceof ToParentBlockJoinQuery) {
+            enroll((ToParentBlockJoinQuery) child,
+                   (ToParentBlockJoinQuery.BlockJoinScorer) scorer);
           }
         }
 
         @Override
         public void visitRequired(Query parent, Query child, Scorer scorer) {
           //System.out.println("visitReq parent=" + parent + " child=" + child + " scorer=" + scorer);
-          if (child instanceof BlockJoinQuery) {
-            enroll((BlockJoinQuery) child,
-                   (BlockJoinQuery.BlockJoinScorer) scorer);
+          if (child instanceof ToParentBlockJoinQuery) {
+            enroll((ToParentBlockJoinQuery) child,
+                   (ToParentBlockJoinQuery.BlockJoinScorer) scorer);
           }
         }
 
         @Override
         public void visitProhibited(Query parent, Query child, Scorer scorer) {
           //System.out.println("visitProh");
-          if (child instanceof BlockJoinQuery) {
-            enroll((BlockJoinQuery) child,
-                   (BlockJoinQuery.BlockJoinScorer) scorer);
+          if (child instanceof ToParentBlockJoinQuery) {
+            enroll((ToParentBlockJoinQuery) child,
+                   (ToParentBlockJoinQuery.BlockJoinScorer) scorer);
           }
         }
       });
@@ -387,7 +387,7 @@ public class BlockJoinCollector extends 
    *  is not computed (will always be 0).  Returns null if
    *  no groups matched. */
   @SuppressWarnings("unchecked")
-  public TopGroups<Integer> getTopGroups(BlockJoinQuery query, Sort withinGroupSort, int offset, int maxDocsPerGroup, int withinGroupOffset, boolean fillSortFields) 
+  public TopGroups<Integer> getTopGroups(ToParentBlockJoinQuery query, Sort withinGroupSort, int offset, int maxDocsPerGroup, int withinGroupOffset, boolean fillSortFields) 
 
     throws IOException {
 

Copied: lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java (from r1231512, lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/BlockJoinQuery.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java?p2=lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java&p1=lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/BlockJoinQuery.java&r1=1231512&r2=1231521&rev=1231521&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/BlockJoinQuery.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java Sat Jan 14 16:16:33 2012
@@ -52,12 +52,15 @@ import org.apache.lucene.util.FixedBitSe
  * You can then use this Query as a clause with
  * other queries in the parent document space.</p>
  *
+ * <p>See {@link ToChildBlockJoinQuery} if you need to join
+ * in the reverse order.
+ *
  * <p>The child documents must be orthogonal to the parent
  * documents: the wrapped child query must never
  * return a parent document.</p>
  *
  * If you'd like to retrieve {@link TopGroups} for the
- * resulting query, use the {@link BlockJoinCollector}.
+ * resulting query, use the {@link ToParentBlockJoinCollector}.
  * Note that this is not necessary, ie, if you simply want
  * to collect the parent documents and don't need to see
  * which child documents matched under that parent, then
@@ -67,7 +70,7 @@ import org.apache.lucene.util.FixedBitSe
  * matches, for example you OR a parent-only query with a
  * joined child-only query, then the resulting collected documents
  * will be correct, however the {@link TopGroups} you get
- * from {@link BlockJoinCollector} will not contain every
+ * from {@link ToParentBlockJoinCollector} will not contain every
  * child for parents that had matched.
  *
  * <p>See {@link org.apache.lucene.search.join} for an
@@ -76,9 +79,22 @@ import org.apache.lucene.util.FixedBitSe
  * @lucene.experimental
  */
 
-public class BlockJoinQuery extends Query {
+public class ToParentBlockJoinQuery extends Query {
 
-  public static enum ScoreMode {None, Avg, Max, Total};
+  /** How to aggregate multiple child hit scores into a
+   *  single parent score. */
+  public static enum ScoreMode {
+    /** Do no scoring. */
+    None,
+    /** Parent hit's score is the average of all child
+        scores. */
+    Avg,
+    /** Parent hit's score is the max of all child
+        scores. */
+    Max,
+    /** Parent hit's score is the sum of all child
+        scores. */
+    Total};
 
   private final Filter parentsFilter;
   private final Query childQuery;
@@ -91,7 +107,15 @@ public class BlockJoinQuery extends Quer
   private final Query origChildQuery;
   private final ScoreMode scoreMode;
 
-  public BlockJoinQuery(Query childQuery, Filter parentsFilter, ScoreMode scoreMode) {
+  /** Create a ToParentBlockJoinQuery.
+   * 
+   * @param childQuery Query matching child documents.
+   * @param parentsFilter Filter (must produce FixedBitSet
+   * per-seegment) identifying the parent documents.
+   * @param scoreMode How to aggregate multiple child scores
+   * into a single parent score.
+   **/
+  public ToParentBlockJoinQuery(Query childQuery, Filter parentsFilter, ScoreMode scoreMode) {
     super();
     this.origChildQuery = childQuery;
     this.childQuery = childQuery;
@@ -99,7 +123,7 @@ public class BlockJoinQuery extends Quer
     this.scoreMode = scoreMode;
   }
 
-  private BlockJoinQuery(Query origChildQuery, Query childQuery, Filter parentsFilter, ScoreMode scoreMode) {
+  private ToParentBlockJoinQuery(Query origChildQuery, Query childQuery, Filter parentsFilter, ScoreMode scoreMode) {
     super();
     this.origChildQuery = origChildQuery;
     this.childQuery = childQuery;
@@ -270,9 +294,9 @@ public class BlockJoinQuery extends Quer
         //System.out.println("  c=" + nextChildDoc);
         if (pendingChildDocs.length == childDocUpto) {
           pendingChildDocs = ArrayUtil.grow(pendingChildDocs);
-          if (scoreMode != ScoreMode.None) {
-            pendingChildScores = ArrayUtil.grow(pendingChildScores);
-          }
+        }
+        if (scoreMode != ScoreMode.None && pendingChildScores.length == childDocUpto) {
+          pendingChildScores = ArrayUtil.grow(pendingChildScores);
         }
         pendingChildDocs[childDocUpto] = nextChildDoc;
         if (scoreMode != ScoreMode.None) {
@@ -365,7 +389,7 @@ public class BlockJoinQuery extends Quer
   public Query rewrite(IndexReader reader) throws IOException {
     final Query childRewrite = childQuery.rewrite(reader);
     if (childRewrite != childQuery) {
-      Query rewritten = new BlockJoinQuery(childQuery,
+      Query rewritten = new ToParentBlockJoinQuery(childQuery,
                                 childRewrite,
                                 parentsFilter,
                                 scoreMode);
@@ -378,13 +402,13 @@ public class BlockJoinQuery extends Quer
 
   @Override
   public String toString(String field) {
-    return "BlockJoinQuery ("+childQuery.toString()+")";
+    return "ToParentBlockJoinQuery ("+childQuery.toString()+")";
   }
 
   @Override
   public boolean equals(Object _other) {
-    if (_other instanceof BlockJoinQuery) {
-      final BlockJoinQuery other = (BlockJoinQuery) _other;
+    if (_other instanceof ToParentBlockJoinQuery) {
+      final ToParentBlockJoinQuery other = (ToParentBlockJoinQuery) _other;
       return origChildQuery.equals(other.origChildQuery) &&
         parentsFilter.equals(other.parentsFilter) &&
         scoreMode == other.scoreMode;
@@ -405,7 +429,7 @@ public class BlockJoinQuery extends Quer
 
   @Override
   public Object clone() {
-    return new BlockJoinQuery((Query) origChildQuery.clone(),
+    return new ToParentBlockJoinQuery((Query) origChildQuery.clone(),
                               parentsFilter,
                               scoreMode);
   }

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=1231521&r1=1231520&r2=1231521&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 Sat Jan 14 16:16:33 2012
@@ -14,19 +14,25 @@
   parent documents, as Lucene does not currently record any information
   about doc blocks.</p>
 
-<p>At search time, use {@link org.apache.lucene.search.join.BlockJoinQuery} to remap
-  matches from any child {@link org.apache.lucene.search.Query} (ie, a query that matches only
-  child documents) up to the parent document space.  The resulting
-  {@link org.apache.lucene.search.join.BlockJoinQuery} can then be used as a clause in any query that
-  matches parent documents.</p>
+<p>At search time, use {@link
+  org.apache.lucene.search.join.ToParentBlockJoinQuery} to remap/join
+  matches from any child {@link org.apache.lucene.search.Query} (ie, a
+  query that matches only child documents) up to the parent document
+  space.  The
+  resulting query can then be used as a clause in any query that
+  matches parent.</p>
 
 <p>If you only care about the parent documents matching the query, you
   can use any collector to collect the parent hits, but if you'd also
   like to see which child documents match for each parent document,
-  use the {@link org.apache.lucene.search.join.BlockJoinCollector} to collect the hits. Once the
+  use the {@link org.apache.lucene.search.join.ToParentBlockJoinCollector} to collect the hits. Once the
   search is done, you retrieve a {@link
   org.apache.lucene.search.grouping.TopGroups} instance from the
-  {@link org.apache.lucene.search.join.BlockJoinCollector#getTopGroups} method.</p>
+  {@link org.apache.lucene.search.join.ToParentBlockJoinCollector#getTopGroups} method.</p>
 
+<p>To map/join in the opposite direction, use {@link
+  org.apache.lucene.search.join.ToChildBlockJoinQuery}.  This wraps
+  any query matching parent documents, creating the joined query
+  matching only child documents.
 </body>
 </html>

Modified: lucene/dev/branches/branch_3x/lucene/contrib/join/src/test/org/apache/lucene/search/TestBlockJoin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/join/src/test/org/apache/lucene/search/TestBlockJoin.java?rev=1231521&r1=1231520&r2=1231521&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/join/src/test/org/apache/lucene/search/TestBlockJoin.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/join/src/test/org/apache/lucene/search/TestBlockJoin.java Sat Jan 14 16:16:33 2012
@@ -17,6 +17,7 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -32,11 +33,14 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.grouping.GroupDocs;
 import org.apache.lucene.search.grouping.TopGroups;
-import org.apache.lucene.search.join.BlockJoinCollector;
-import org.apache.lucene.search.join.BlockJoinQuery;
+import org.apache.lucene.search.join.ToChildBlockJoinQuery;
+import org.apache.lucene.search.join.ToParentBlockJoinCollector;
+import org.apache.lucene.search.join.ToParentBlockJoinQuery;
 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.ReaderUtil;
 import org.apache.lucene.util._TestUtil;
 
 public class TestBlockJoin extends LuceneTestCase {
@@ -54,7 +58,7 @@ public class TestBlockJoin extends Lucen
   private Document makeJob(String skill, int year) {
     Document job = new Document();
     job.add(newField("skill", skill, Field.Store.YES, Field.Index.NOT_ANALYZED));
-    job.add(new NumericField("year").setIntValue(year));
+    job.add(new NumericField("year", Field.Store.YES, true).setIntValue(year));
     return job;
   }
 
@@ -101,14 +105,14 @@ public class TestBlockJoin extends Lucen
 
     // Wrap the child document query to 'join' any matches
     // up to corresponding parent:
-    BlockJoinQuery childJoinQuery = new BlockJoinQuery(childQuery, parentsFilter, BlockJoinQuery.ScoreMode.Avg);
+    ToParentBlockJoinQuery childJoinQuery = new ToParentBlockJoinQuery(childQuery, parentsFilter, ToParentBlockJoinQuery.ScoreMode.Avg);
 
     // Combine the parent and nested child queries into a single query for a candidate
     BooleanQuery fullQuery = new BooleanQuery();
     fullQuery.add(new BooleanClause(parentQuery, Occur.MUST));
     fullQuery.add(new BooleanClause(childJoinQuery, Occur.MUST));
 
-    BlockJoinCollector c = new BlockJoinCollector(Sort.RELEVANCE, 1, true, false);
+    ToParentBlockJoinCollector c = new ToParentBlockJoinCollector(Sort.RELEVANCE, 1, true, false);
 
     s.search(fullQuery, c);
     
@@ -128,9 +132,44 @@ public class TestBlockJoin extends Lucen
     Document parentDoc = s.doc(group.groupValue);
     assertEquals("Lisa", parentDoc.get("name"));
 
+    //System.out.println("TEST: now test up");
+
+    // Now join "up" (map parent hits to child docs) instead...:
+    ToChildBlockJoinQuery parentJoinQuery = new ToChildBlockJoinQuery(parentQuery, parentsFilter, random.nextBoolean());
+    BooleanQuery fullChildQuery = new BooleanQuery();
+    fullChildQuery.add(new BooleanClause(parentJoinQuery, Occur.MUST));
+    fullChildQuery.add(new BooleanClause(childQuery, Occur.MUST));
+    
+    //System.out.println("FULL: " + fullChildQuery);
+    TopDocs hits = s.search(fullChildQuery, 10);
+    assertEquals(1, hits.totalHits);
+    childDoc = s.doc(hits.scoreDocs[0].doc);
+    //System.out.println("CHILD = " + childDoc + " docID=" + hits.scoreDocs[0].doc);
+    assertEquals("java", childDoc.get("skill"));
+    assertEquals(2007, ((NumericField) childDoc.getFieldable("year")).getNumericValue());
+    assertEquals("Lisa", getParentDoc(r, parentsFilter, hits.scoreDocs[0].doc).get("name"));
+
     s.close();
     r.close();
     dir.close();
+   }
+
+  private Document getParentDoc(IndexReader reader, Filter parents, int childDocID) throws IOException {
+    List<IndexReader> subReadersList = new ArrayList<IndexReader>();
+    ReaderUtil.gatherSubReaders(subReadersList, reader);
+    IndexReader[] subReaders = subReadersList
+        .toArray(new IndexReader[subReadersList.size()]);
+    int[] docStarts = new int[subReaders.length];
+    int maxDoc = 0;
+    for (int i = 0; i < subReaders.length; i++) {
+      docStarts[i] = maxDoc;
+      maxDoc += subReaders[i].maxDoc();
+    }
+
+    final int subIndex = ReaderUtil.subIndex(childDocID, docStarts);
+    final IndexReader subReader = subReaders[subIndex];
+    final FixedBitSet bits = (FixedBitSet) parents.getDocIdSet(subReader);
+    return subReader.document(bits.nextSetBit(childDocID - docStarts[subIndex]));
   }
   
   public void testBoostBug() throws Exception {
@@ -140,7 +179,7 @@ public class TestBlockJoin extends Lucen
     w.close();
     IndexSearcher s = newSearcher(r);
     
-    BlockJoinQuery q = new BlockJoinQuery(new MatchAllDocsQuery(), new QueryWrapperFilter(new MatchAllDocsQuery()), BlockJoinQuery.ScoreMode.Avg);
+    ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(new MatchAllDocsQuery(), new QueryWrapperFilter(new MatchAllDocsQuery()), ToParentBlockJoinQuery.ScoreMode.Avg);
     s.search(q, 10);
     BooleanQuery bq = new BooleanQuery();
     bq.setBoost(2f); // we boost the BQ
@@ -198,8 +237,9 @@ public class TestBlockJoin extends Lucen
 
   public void testRandom() throws Exception {
     // We build two indices at once: one normalized (which
-    // BlockJoinQuery/Collector can query) and the other w/
-    // same docs just fully denormalized:
+    // ToParentBlockJoinQuery/Collector,
+    // ToChildBlockJoinQuery can query) and the other w/
+    // the same docs, just fully denormalized:
     final Directory dir = newDirectory();
     final Directory joinDir = newDirectory();
 
@@ -211,7 +251,7 @@ public class TestBlockJoin extends Lucen
     // Values for child fields:
     final String[][] childFields = getRandomFields(numParentDocs);
 
-    // TODO: test star join, nested join cases too!
+    // TODO: parallel star join, nested join cases too!
     final RandomIndexWriter w = new RandomIndexWriter(random, dir);
     final RandomIndexWriter joinW = new RandomIndexWriter(random, joinDir);
     for(int parentDocID=0;parentDocID<numParentDocs;parentDocID++) {
@@ -234,7 +274,15 @@ public class TestBlockJoin extends Lucen
       final List<Document> joinDocs = new ArrayList<Document>();
 
       if (VERBOSE) {
-        System.out.println("  " + parentDoc);
+        StringBuilder sb = new StringBuilder();
+        sb.append("parentID=" + parentDoc.get("parentID"));
+        for(int fieldID=0;fieldID<parentFields.length;fieldID++) {
+          String s = parentDoc.get("parent" + fieldID);
+          if (s != null) {
+            sb.append(" parent" + fieldID + "=" + s);
+          }
+        }
+        System.out.println("  " + sb.toString());
       }
 
       final int numChildDocs = _TestUtil.nextInt(random, 1, 20);
@@ -259,7 +307,15 @@ public class TestBlockJoin extends Lucen
         }
 
         if (VERBOSE) {
-          System.out.println("    " + joinChildDoc);
+          StringBuilder sb = new StringBuilder();
+          sb.append("childID=" + joinChildDoc.get("childID"));
+          for(int fieldID=0;fieldID<childFields.length;fieldID++) {
+            String s = joinChildDoc.get("child" + fieldID);
+            if (s != null) {
+              sb.append(" child" + fieldID + "=" + s);
+            }
+          }
+          System.out.println("    " + sb.toString());
         }
 
         w.addDocument(childDoc);
@@ -334,14 +390,26 @@ public class TestBlockJoin extends Lucen
                random.nextBoolean() ? BooleanClause.Occur.MUST : BooleanClause.Occur.MUST_NOT);
       }
 
-      final BlockJoinQuery childJoinQuery = new BlockJoinQuery(childQuery, parentsFilter, BlockJoinQuery.ScoreMode.Avg);
+      final int x = random.nextInt(4);
+      final ToParentBlockJoinQuery.ScoreMode agg;
+      if (x == 0) {
+        agg = ToParentBlockJoinQuery.ScoreMode.None;
+      } else if (x == 1) {
+        agg = ToParentBlockJoinQuery.ScoreMode.Max;
+      } else if (x == 2) {
+        agg = ToParentBlockJoinQuery.ScoreMode.Total;
+      } else {
+        agg = ToParentBlockJoinQuery.ScoreMode.Avg;
+      }
+
+      final ToParentBlockJoinQuery childJoinQuery = new ToParentBlockJoinQuery(childQuery, parentsFilter, agg);
 
       // To run against the block-join index:
       final Query parentJoinQuery;
 
       // Same query as parentJoinQuery, but to run against
-      // the fully denormalized index (so we can compare)
-      // results:
+      // the fully denormalized index (so we can compare
+      // results):
       final Query parentQuery;
 
       if (random.nextBoolean()) {
@@ -382,7 +450,7 @@ public class TestBlockJoin extends Lucen
         System.out.println("\nTEST: query=" + parentQuery + " joinQuery=" + parentJoinQuery + " parentSort=" + parentSort + " childSort=" + childSort);
       }
 
-      // Merge both sorst:
+      // Merge both sorts:
       final List<SortField> sortFields = new ArrayList<SortField>(Arrays.asList(parentSort.getSort()));
       sortFields.addAll(Arrays.asList(childSort.getSort()));
       final Sort parentAndChildSort = new Sort(sortFields.toArray(new SortField[sortFields.size()]));
@@ -411,8 +479,17 @@ public class TestBlockJoin extends Lucen
           }
         }
       }
-      
-      final BlockJoinCollector c = new BlockJoinCollector(parentSort, 10, true, true);
+
+      final boolean trackScores;
+      final boolean trackMaxScore;
+      if (agg == ToParentBlockJoinQuery.ScoreMode.None) {
+        trackScores = false;
+        trackMaxScore = false;
+      } else {
+        trackScores = random.nextBoolean();
+        trackMaxScore = random.nextBoolean();
+      }
+      final ToParentBlockJoinCollector c = new ToParentBlockJoinCollector(parentSort, 10, trackScores, trackMaxScore);
 
       joinS.search(parentJoinQuery, c);
 
@@ -455,6 +532,124 @@ public class TestBlockJoin extends Lucen
       } else {
         compareHits(r, joinR, results, joinResults);
       }
+
+      // Test joining in the opposite direction (parent to
+      // child):
+
+      // Get random query against parent documents:
+      final Query parentQuery2;
+      if (random.nextInt(3) == 2) {
+        final int fieldID = random.nextInt(parentFields.length);
+        parentQuery2 = new TermQuery(new Term("parent" + fieldID,
+                                              parentFields[fieldID][random.nextInt(parentFields[fieldID].length)]));
+      } else if (random.nextInt(3) == 2) {
+        BooleanQuery bq = new BooleanQuery();
+        parentQuery2 = bq;
+        final int numClauses = _TestUtil.nextInt(random, 2, 4);
+        boolean didMust = false;
+        for(int clauseIDX=0;clauseIDX<numClauses;clauseIDX++) {
+          Query clause;
+          BooleanClause.Occur occur;
+          if (!didMust && random.nextBoolean()) {
+            occur = random.nextBoolean() ? BooleanClause.Occur.MUST : BooleanClause.Occur.MUST_NOT;
+            clause = new TermQuery(randomParentTerm(parentFields[0]));
+            didMust = true;
+          } else {
+            occur = BooleanClause.Occur.SHOULD;
+            final int fieldID = _TestUtil.nextInt(random, 1, parentFields.length-1);
+            clause = new TermQuery(new Term("parent" + fieldID,
+                                            parentFields[fieldID][random.nextInt(parentFields[fieldID].length)]));
+          }
+          bq.add(clause, occur);
+        }
+      } else {
+        BooleanQuery bq = new BooleanQuery();
+        parentQuery2 = bq;
+        
+        bq.add(new TermQuery(randomParentTerm(parentFields[0])),
+               BooleanClause.Occur.MUST);
+        final int fieldID = _TestUtil.nextInt(random, 1, parentFields.length-1);
+        bq.add(new TermQuery(new Term("parent" + fieldID, parentFields[fieldID][random.nextInt(parentFields[fieldID].length)])),
+               random.nextBoolean() ? BooleanClause.Occur.MUST : BooleanClause.Occur.MUST_NOT);
+      }
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: top down: parentQuery2=" + parentQuery2);
+      }
+
+      // Maps parent query to child docs:
+      final ToChildBlockJoinQuery parentJoinQuery2 = new ToChildBlockJoinQuery(parentQuery2, parentsFilter, random.nextBoolean());
+
+      // To run against the block-join index:
+      final Query childJoinQuery2;
+
+      // Same query as parentJoinQuery, but to run against
+      // the fully denormalized index (so we can compare
+      // results):
+      final Query childQuery2;
+
+      if (random.nextBoolean()) {
+        childQuery2 = parentQuery2;
+        childJoinQuery2 = parentJoinQuery2;
+      } else {
+        // AND child field w/ parent query:
+        final BooleanQuery bq = new BooleanQuery();
+        childJoinQuery2 = bq;
+        final Term childTerm = randomChildTerm(childFields[0]);
+        if (random.nextBoolean()) {
+          bq.add(parentJoinQuery2, BooleanClause.Occur.MUST);
+          bq.add(new TermQuery(childTerm),
+                 BooleanClause.Occur.MUST);
+        } else {
+          bq.add(new TermQuery(childTerm),
+                 BooleanClause.Occur.MUST);
+          bq.add(parentJoinQuery2, BooleanClause.Occur.MUST);
+        }
+
+        final BooleanQuery bq2 = new BooleanQuery();
+        childQuery2 = bq2;
+        if (random.nextBoolean()) {
+          bq2.add(parentQuery2, BooleanClause.Occur.MUST);
+          bq2.add(new TermQuery(childTerm),
+                  BooleanClause.Occur.MUST);
+        } else {
+          bq2.add(new TermQuery(childTerm),
+                  BooleanClause.Occur.MUST);
+          bq2.add(parentQuery2, BooleanClause.Occur.MUST);
+        }
+      }
+
+      final Sort childSort2 = getRandomSort("child", childFields.length);
+
+      // Search denormalized index:
+      if (VERBOSE) {
+        System.out.println("TEST: run top down query=" + childQuery2 + " sort=" + childSort2);
+      }
+      final TopDocs results2 = s.search(childQuery2, null, r.numDocs(),
+                                        childSort2);
+      if (VERBOSE) {
+        System.out.println("  " + results2.totalHits + " totalHits:");
+        for(ScoreDoc sd : results2.scoreDocs) {
+          final Document doc = s.doc(sd.doc);
+          System.out.println("  childID=" + doc.get("childID") + " parentID=" + doc.get("parentID") + " docID=" + sd.doc);
+        }
+      }
+
+      // Search join index:
+      if (VERBOSE) {
+        System.out.println("TEST: run top down join query=" + childJoinQuery2 + " sort=" + childSort2);
+      }
+      TopDocs joinResults2 = joinS.search(childJoinQuery2, null, joinR.numDocs(), childSort2);
+      if (VERBOSE) {
+        System.out.println("  " + joinResults2.totalHits + " totalHits:");
+        for(ScoreDoc sd : joinResults2.scoreDocs) {
+          final Document doc = joinS.doc(sd.doc);
+          final Document parentDoc = getParentDoc(joinR, parentsFilter, sd.doc);
+          System.out.println("  childID=" + doc.get("childID") + " parentID=" + parentDoc.get("parentID") + " docID=" + sd.doc);
+        }
+      }
+
+      compareChildHits(r, joinR, results2, joinResults2);
     }
 
     s.close();
@@ -465,6 +660,28 @@ public class TestBlockJoin extends Lucen
     joinDir.close();
   }
 
+  private void compareChildHits(IndexReader r, IndexReader joinR, TopDocs results, TopDocs joinResults) throws Exception {
+    assertEquals(results.totalHits, joinResults.totalHits);
+    assertEquals(results.scoreDocs.length, joinResults.scoreDocs.length);
+    for(int hitCount=0;hitCount<results.scoreDocs.length;hitCount++) {
+      ScoreDoc hit = results.scoreDocs[hitCount];
+      ScoreDoc joinHit = joinResults.scoreDocs[hitCount];
+      Document doc1 = r.document(hit.doc);
+      Document doc2 = joinR.document(joinHit.doc);
+      assertEquals("hit " + hitCount + " differs",
+                   doc1.get("childID"), doc2.get("childID"));
+      // don't compare scores -- they are expected to differ
+
+
+      assertTrue(hit instanceof FieldDoc);
+      assertTrue(joinHit instanceof FieldDoc);
+
+      FieldDoc hit0 = (FieldDoc) hit;
+      FieldDoc joinHit0 = (FieldDoc) joinHit;
+      assertEquals(hit0.fields, joinHit0.fields);
+    }
+  }
+
   private void compareHits(IndexReader r, IndexReader joinR, TopDocs results, TopGroups<Integer> joinResults) throws Exception {
     // results is 'complete'; joinResults is a subset
     int resultUpto = 0;
@@ -540,8 +757,8 @@ public class TestBlockJoin extends Lucen
 
     // Wrap the child document query to 'join' any matches
     // up to corresponding parent:
-    BlockJoinQuery childJobJoinQuery = new BlockJoinQuery(childJobQuery, parentsFilter, BlockJoinQuery.ScoreMode.Avg);
-    BlockJoinQuery childQualificationJoinQuery = new BlockJoinQuery(childQualificationQuery, parentsFilter, BlockJoinQuery.ScoreMode.Avg);
+    ToParentBlockJoinQuery childJobJoinQuery = new ToParentBlockJoinQuery(childJobQuery, parentsFilter, ToParentBlockJoinQuery.ScoreMode.Avg);
+    ToParentBlockJoinQuery childQualificationJoinQuery = new ToParentBlockJoinQuery(childQualificationQuery, parentsFilter, ToParentBlockJoinQuery.ScoreMode.Avg);
 
     // Combine the parent and nested child queries into a single query for a candidate
     BooleanQuery fullQuery = new BooleanQuery();
@@ -549,12 +766,13 @@ public class TestBlockJoin extends Lucen
     fullQuery.add(new BooleanClause(childJobJoinQuery, Occur.MUST));
     fullQuery.add(new BooleanClause(childQualificationJoinQuery, Occur.MUST));
 
-    //????? How do I control volume of jobs vs qualifications per parent?
-    BlockJoinCollector c = new BlockJoinCollector(Sort.RELEVANCE, 10, true, false);
+    // Collects all job and qualification child docs for
+    // each resume hit in the top N (sorted by score):
+    ToParentBlockJoinCollector c = new ToParentBlockJoinCollector(Sort.RELEVANCE, 10, true, false);
 
     s.search(fullQuery, c);
 
-    //Examine "Job" children
+    // Examine "Job" children
     boolean showNullPointerIssue=true;
     if (showNullPointerIssue) {
       TopGroups<Integer> jobResults = c.getTopGroups(childJobJoinQuery, null, 0, 10, 0, true);
@@ -574,10 +792,9 @@ public class TestBlockJoin extends Lucen
       assertEquals("Lisa", parentDoc.get("name"));
     }
 
-    //Now Examine qualification children
+    // Now Examine qualification children
     TopGroups<Integer> qualificationResults = c.getTopGroups(childQualificationJoinQuery, null, 0, 10, 0, true);
 
-    //!!!!! This next line can null pointer - but only if prior "jobs" section called first
     assertEquals(1, qualificationResults.totalGroupedHitCount);
     assertEquals(1, qualificationResults.groups.length);
 
@@ -612,7 +829,7 @@ public class TestBlockJoin extends Lucen
                             new QueryWrapperFilter(
                               new TermQuery(new Term("parent", "1"))));
 
-    BlockJoinQuery q = new BlockJoinQuery(tq, parentFilter, BlockJoinQuery.ScoreMode.Avg);
+    ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ToParentBlockJoinQuery.ScoreMode.Avg);
     Weight weight = s.createNormalizedWeight(q);
     
     IndexReader atomic;
@@ -654,7 +871,7 @@ public class TestBlockJoin extends Lucen
                             new QueryWrapperFilter(
                               new TermQuery(new Term("isparent", "yes"))));
 
-    BlockJoinQuery q = new BlockJoinQuery(tq, parentFilter, BlockJoinQuery.ScoreMode.Avg);
+    ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ToParentBlockJoinQuery.ScoreMode.Avg);
     Weight weight = s.createNormalizedWeight(q);
     final IndexReader[] subs = s.getIndexReader().getSequentialSubReaders();
     final IndexReader sub;