You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by is...@apache.org on 2017/02/12 13:18:41 UTC

[17/18] lucene-solr:jira/solr-5944: Updating branch by merging latest changes from master

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java b/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
index a8c0a82..b8d8c29 100644
--- a/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
+++ b/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
@@ -19,6 +19,7 @@ package org.apache.lucene.util;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -56,6 +57,7 @@ import org.apache.lucene.util.graph.GraphTokenStreamFiniteStrings;
 public class QueryBuilder {
   protected Analyzer analyzer;
   protected boolean enablePositionIncrements = true;
+  protected boolean autoGenerateMultiTermSynonymsPhraseQuery = false;
   
   /** Creates a new QueryBuilder using the given analyzer. */
   public QueryBuilder(Analyzer analyzer) {
@@ -131,7 +133,7 @@ public class QueryBuilder {
       throw new IllegalArgumentException("fraction should be >= 0 and <= 1");
     }
     
-    // TODO: wierd that BQ equals/rewrite/scorer doesn't handle this?
+    // TODO: weird that BQ equals/rewrite/scorer doesn't handle this?
     if (fraction == 1) {
       return createBooleanQuery(field, queryText, BooleanClause.Occur.MUST);
     }
@@ -139,21 +141,6 @@ public class QueryBuilder {
     Query query = createFieldQuery(analyzer, BooleanClause.Occur.SHOULD, field, queryText, false, 0);
     if (query instanceof BooleanQuery) {
       query = addMinShouldMatchToBoolean((BooleanQuery) query, fraction);
-    } else if (query instanceof GraphQuery && ((GraphQuery) query).hasBoolean()) {
-      // we have a graph query that has at least one boolean sub-query
-      // re-build and set minimum should match on each boolean found
-      List<Query> oldQueries = ((GraphQuery) query).getQueries();
-      Query[] queries = new Query[oldQueries.size()];
-      for (int i = 0; i < queries.length; i++) {
-        Query oldQuery = oldQueries.get(i);
-        if (oldQuery instanceof BooleanQuery) {
-          queries[i] = addMinShouldMatchToBoolean((BooleanQuery) oldQuery, fraction);
-        } else {
-          queries[i] = oldQuery;
-        }
-      }
-
-      query = new GraphQuery(queries);
     }
     return query;
   }
@@ -208,6 +195,22 @@ public class QueryBuilder {
     this.enablePositionIncrements = enable;
   }
 
+  /**
+   * Returns true if phrase query should be automatically generated for multi terms synonyms.
+   * @see #setAutoGenerateMultiTermSynonymsPhraseQuery(boolean)
+   */
+  public boolean getAutoGenerateMultiTermSynonymsPhraseQuery() {
+    return autoGenerateMultiTermSynonymsPhraseQuery;
+  }
+
+  /**
+   * Set to <code>true</code> if phrase queries should be automatically generated
+   * for multi terms synonyms.
+   * Default: false.
+   */
+  public void setAutoGenerateMultiTermSynonymsPhraseQuery(boolean enable) {
+    this.autoGenerateMultiTermSynonymsPhraseQuery = enable;
+  }
 
   /**
    * Creates a query from the analysis chain.
@@ -294,7 +297,11 @@ public class QueryBuilder {
         return analyzeTerm(field, stream);
       } else if (isGraph) {
         // graph
-        return analyzeGraph(stream, operator, field, quoted, phraseSlop);
+        if (quoted) {
+          return analyzeGraphPhrase(stream, operator, field, phraseSlop);
+        } else {
+          return analyzeGraphBoolean(field, stream, operator);
+        }
       } else if (quoted && positionCount > 1) {
         // phrase
         if (hasSynonyms) {
@@ -444,25 +451,74 @@ public class QueryBuilder {
   }
 
   /**
-   * Creates a query from a graph token stream by extracting all the finite strings from the graph and using them to create the query.
+   * Creates a boolean query from a graph token stream. The articulation points of the graph are visited in order and the queries
+   * created at each point are merged in the returned boolean query.
    */
-  protected Query analyzeGraph(TokenStream source, BooleanClause.Occur operator, String field, boolean quoted, int phraseSlop)
-      throws IOException {
+  protected Query analyzeGraphBoolean(String field, TokenStream source, BooleanClause.Occur operator) throws IOException {
     source.reset();
-    List<TokenStream> tokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(source);
-
-    if (tokenStreams.isEmpty()) {
-      return null;
+    GraphTokenStreamFiniteStrings graph = new GraphTokenStreamFiniteStrings(source);
+    BooleanQuery.Builder builder = new BooleanQuery.Builder();
+    int[] articulationPoints = graph.articulationPoints();
+    int lastState = 0;
+    for (int i = 0; i <= articulationPoints.length; i++) {
+      int start = lastState;
+      int end = -1;
+      if (i < articulationPoints.length) {
+        end = articulationPoints[i];
+      }
+      lastState = end;
+      final Query queryPos;
+      if (graph.hasSidePath(start)) {
+        List<Query> queries = new ArrayList<> ();
+        Iterator<TokenStream> it = graph.getFiniteStrings(start, end);
+        while (it.hasNext()) {
+          TokenStream ts = it.next();
+          // This is a synonym path so all terms are mandatory (MUST).
+          Query q = createFieldQuery(ts, BooleanClause.Occur.MUST, field, getAutoGenerateMultiTermSynonymsPhraseQuery(), 0);
+          if (q != null) {
+            queries.add(q);
+          }
+        }
+        if (queries.size() > 0) {
+          queryPos = newGraphSynonymQuery(queries.toArray(new Query[queries.size()]));
+        } else {
+          queryPos = null;
+        }
+      } else {
+        Term[] terms = graph.getTerms(field, start);
+        assert terms.length > 0;
+        if (terms.length == 1) {
+          queryPos = newTermQuery(terms[0]);
+        } else {
+          queryPos = newSynonymQuery(terms);
+        }
+      }
+      if (queryPos != null) {
+        builder.add(queryPos, operator);
+      }
     }
+    BooleanQuery bq =  builder.build();
+    if (bq.clauses().size() == 1) {
+      return bq.clauses().get(0).getQuery();
+    }
+    return bq;
+  }
 
-    List<Query> queries = new ArrayList<>(tokenStreams.size());
-    for (TokenStream ts : tokenStreams) {
-      Query query = createFieldQuery(ts, operator, field, quoted, phraseSlop);
+  /**
+   * Creates a query from a graph token stream by extracting all the finite strings from the graph and using them to create the query.
+   */
+  protected Query analyzeGraphPhrase(TokenStream source, BooleanClause.Occur operator, String field, int phraseSlop)
+      throws IOException {
+    source.reset();
+    GraphTokenStreamFiniteStrings visitor = new GraphTokenStreamFiniteStrings(source);
+    Iterator<TokenStream> it = visitor.getFiniteStrings();
+    List<Query> queries = new ArrayList<>();
+    while (it.hasNext()) {
+      Query query = createFieldQuery(it.next(), operator, field, true, phraseSlop);
       if (query != null) {
         queries.add(query);
       }
     }
-
     return new GraphQuery(queries.toArray(new Query[0]));
   }
 
@@ -485,6 +541,16 @@ public class QueryBuilder {
   protected Query newSynonymQuery(Term terms[]) {
     return new SynonymQuery(terms);
   }
+
+  /**
+   * Builds a new GraphQuery for multi-terms synonyms.
+   * <p>
+   * This is intended for subclasses that wish to customize the generated queries.
+   * @return new Query instance
+   */
+  protected Query newGraphSynonymQuery(Query queries[]) {
+    return new GraphQuery(queries);
+  }
   
   /**
    * Builds a new TermQuery instance.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/lucene/core/src/java/org/apache/lucene/util/Version.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/Version.java b/lucene/core/src/java/org/apache/lucene/util/Version.java
index 00fb329..6477816 100644
--- a/lucene/core/src/java/org/apache/lucene/util/Version.java
+++ b/lucene/core/src/java/org/apache/lucene/util/Version.java
@@ -81,6 +81,13 @@ public final class Version {
   public static final Version LUCENE_6_4_0 = new Version(6, 4, 0);
 
   /**
+   * Match settings and bugs in Lucene's 6.4.1 release.
+   * @deprecated Use latest
+   */
+  @Deprecated
+  public static final Version LUCENE_6_4_1 = new Version(6, 4, 1);
+
+  /**
    * Match settings and bugs in Lucene's 6.5.0 release.
    * @deprecated Use latest
    */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/lucene/core/src/java/org/apache/lucene/util/automaton/FiniteStringsIterator.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/FiniteStringsIterator.java b/lucene/core/src/java/org/apache/lucene/util/automaton/FiniteStringsIterator.java
index 229cdc9..4472bda 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/FiniteStringsIterator.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/FiniteStringsIterator.java
@@ -17,13 +17,13 @@
 package org.apache.lucene.util.automaton;
 
 
+import java.util.BitSet;
+
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.RamUsageEstimator;
 
-import java.util.BitSet;
-
 /**
  * Iterates all accepted strings.
  *
@@ -50,6 +50,11 @@ public class FiniteStringsIterator {
   private final Automaton a;
 
   /**
+   * The state where each path should stop or -1 if only accepted states should be final.
+   */
+  private final int endState;
+
+  /**
    * Tracks which states are in the current path, for cycle detection.
    */
   private final BitSet pathStates;
@@ -75,7 +80,20 @@ public class FiniteStringsIterator {
    * @param a Automaton to create finite string from.
    */
   public FiniteStringsIterator(Automaton a) {
+    this(a, 0, -1);
+  }
+
+
+  /**
+   * Constructor.
+   *
+   * @param a Automaton to create finite string from.
+   * @param startState The starting state for each path.
+   * @param endState The state where each path should stop or -1 if only accepted states should be final.
+   */
+  public FiniteStringsIterator(Automaton a, int startState, int endState) {
     this.a = a;
+    this.endState = endState;
     this.nodes = new PathNode[16];
     for (int i = 0, end = nodes.length; i < end; i++) {
       nodes[i] = new PathNode();
@@ -85,11 +103,11 @@ public class FiniteStringsIterator {
     this.string.setLength(0);
     this.emitEmptyString = a.isAccept(0);
 
-    // Start iteration with node 0.
-    if (a.getNumTransitions(0) > 0) {
-      pathStates.set(0);
-      nodes[0].resetState(a, 0);
-      string.append(0);
+    // Start iteration with node startState.
+    if (a.getNumTransitions(startState) > 0) {
+      pathStates.set(startState);
+      nodes[0].resetState(a, startState);
+      string.append(startState);
     }
   }
 
@@ -115,7 +133,7 @@ public class FiniteStringsIterator {
         string.setIntAt(depth - 1, label);
 
         int to = node.to;
-        if (a.getNumTransitions(to) != 0) {
+        if (a.getNumTransitions(to) != 0 && to != endState) {
           // Now recurse: the destination of this transition has outgoing transitions:
           if (pathStates.get(to)) {
             throw new IllegalArgumentException("automaton has cycles");
@@ -128,7 +146,7 @@ public class FiniteStringsIterator {
           depth++;
           string.setLength(depth);
           string.grow(depth);
-        } else if (a.isAccept(to)) {
+        } else if (endState == to || a.isAccept(to)) {
           // This transition leads to an accept state, so we save the current string:
           return string.get();
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
index e120435..ad65990 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
@@ -528,10 +528,19 @@ public final class BKDReader extends PointValues implements Accountable {
   }
 
   /** Fast path: this is called when the query box fully encompasses all cells under this node. */
-  private void addAll(IntersectState state) throws IOException {
+  private void addAll(IntersectState state, boolean grown) throws IOException {
     //System.out.println("R: addAll nodeID=" + nodeID);
 
+    if (grown == false) {
+      final long maxPointCount = (long) maxPointsInLeafNode * state.index.getNumLeaves();
+      if (maxPointCount <= Integer.MAX_VALUE) { // could be >MAX_VALUE if there are more than 2B points in total
+        state.visitor.grow((int) maxPointCount);
+        grown = true;
+      }
+    }
+
     if (state.index.isLeafNode()) {
+      assert grown;
       //System.out.println("ADDALL");
       if (state.index.nodeExists()) {
         visitDocIDs(state.in, state.index.getLeafBlockFP(), state.visitor);
@@ -539,11 +548,11 @@ public final class BKDReader extends PointValues implements Accountable {
       // TODO: we can assert that the first value here in fact matches what the index claimed?
     } else {
       state.index.pushLeft();
-      addAll(state);
+      addAll(state, grown);
       state.index.pop();
 
       state.index.pushRight();
-      addAll(state);
+      addAll(state, grown);
       state.index.pop();
     }
   }
@@ -579,7 +588,7 @@ public final class BKDReader extends PointValues implements Accountable {
 
     // How many points are stored in this leaf cell:
     int count = in.readVInt();
-    visitor.grow(count);
+    // No need to call grow(), it has been called up-front
 
     if (version < BKDWriter.VERSION_COMPRESSED_DOC_IDS) {
       DocIdsWriter.readInts32(in, count, visitor);
@@ -687,7 +696,7 @@ public final class BKDReader extends PointValues implements Accountable {
       // This cell is fully outside of the query shape: stop recursing
     } else if (r == Relation.CELL_INSIDE_QUERY) {
       // This cell is fully inside of the query shape: recursively add all points in this cell without filtering
-      addAll(state);
+      addAll(state, false);
       // The cell crosses the shape boundary, or the cell fully contains the query, so we fall through and do full filtering:
     } else if (state.index.isLeafNode()) {
       

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java b/lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java
index cec65fa..fd85836 100644
--- a/lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java
+++ b/lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java
@@ -19,7 +19,11 @@ package org.apache.lucene.util.graph;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Collections;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
@@ -28,25 +32,27 @@ import org.apache.lucene.analysis.tokenattributes.BytesTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.FiniteStringsIterator;
 import org.apache.lucene.util.automaton.Operations;
+import org.apache.lucene.util.automaton.Transition;
 
 import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
 
 /**
- * Creates a list of {@link TokenStream} where each stream is the tokens that make up a finite string in graph token stream.  To do this,
- * the graph token stream is converted to an {@link Automaton} and from there we use a {@link FiniteStringsIterator} to collect the various
- * token streams for each finite string.
+ * Consumes a TokenStream and creates an {@link Automaton} where the transition labels are terms from
+ * the {@link TermToBytesRefAttribute}.
+ * This class also provides helpers to explore the different paths of the {@link Automaton}.
  */
 public final class GraphTokenStreamFiniteStrings {
-  private final Automaton.Builder builder = new Automaton.Builder();
   private final Map<BytesRef, Integer> termToID = new HashMap<>();
   private final Map<Integer, BytesRef> idToTerm = new HashMap<>();
   private final Map<Integer, Integer> idToInc = new HashMap<>();
-  private Automaton det;
+  private final Automaton det;
+  private final Transition transition = new Transition();
 
   private class FiniteStringsTokenStream extends TokenStream {
     private final BytesTermAttribute termAtt = addAttribute(BytesTermAttribute.class);
@@ -82,37 +88,121 @@ public final class GraphTokenStreamFiniteStrings {
     }
   }
 
-  private GraphTokenStreamFiniteStrings() {
+  public GraphTokenStreamFiniteStrings(TokenStream in) throws IOException {
+    Automaton aut = build(in);
+    this.det = Operations.removeDeadStates(Operations.determinize(aut, DEFAULT_MAX_DETERMINIZED_STATES));
   }
 
   /**
-   * Gets the list of finite string token streams from the given input graph token stream.
+   * Returns whether the provided state is the start of multiple side paths of different length (eg: new york, ny)
    */
-  public static List<TokenStream> getTokenStreams(final TokenStream in) throws IOException {
-    GraphTokenStreamFiniteStrings gfs = new GraphTokenStreamFiniteStrings();
-    return gfs.process(in);
+  public boolean hasSidePath(int state) {
+    int numT = det.initTransition(state, transition);
+    if (numT <= 1) {
+      return false;
+    }
+    det.getNextTransition(transition);
+    int dest = transition.dest;
+    for (int i = 1; i < numT; i++) {
+      det.getNextTransition(transition);
+      if (dest != transition.dest) {
+        return true;
+      }
+    }
+    return false;
   }
 
   /**
-   * Builds automaton and builds the finite string token streams.
+   * Returns the list of terms that start at the provided state
    */
-  private List<TokenStream> process(final TokenStream in) throws IOException {
-    build(in);
-
-    List<TokenStream> tokenStreams = new ArrayList<>();
-    final FiniteStringsIterator finiteStrings = new FiniteStringsIterator(det);
-    for (IntsRef ids; (ids = finiteStrings.next()) != null; ) {
-      tokenStreams.add(new FiniteStringsTokenStream(IntsRef.deepCopyOf(ids)));
+  public Term[] getTerms(String field, int state) {
+    int numT = det.initTransition(state, transition);
+    List<Term> terms = new ArrayList<> ();
+    for (int i = 0; i < numT; i++) {
+      det.getNextTransition(transition);
+      for (int id = transition.min; id <= transition.max; id++) {
+        Term term = new Term(field, idToTerm.get(id));
+        terms.add(term);
+      }
     }
+    return terms.toArray(new Term[terms.size()]);
+  }
+
+  /**
+   * Get all finite strings from the automaton.
+   */
+  public Iterator<TokenStream> getFiniteStrings() throws IOException {
+    return getFiniteStrings(0, -1);
+  }
+
+
+  /**
+   * Get all finite strings that start at {@code startState} and end at {@code endState}.
+   */
+  public Iterator<TokenStream> getFiniteStrings(int startState, int endState) throws IOException {
+    final FiniteStringsIterator it = new FiniteStringsIterator(det, startState, endState);
+    return new Iterator<TokenStream> () {
+      IntsRef current;
+      boolean finished = false;
+
+      @Override
+      public boolean hasNext() {
+        if (finished == false && current == null) {
+          current = it.next();
+          if (current == null) {
+            finished = true;
+          }
+        }
+        return current != null;
+      }
 
-    return tokenStreams;
+      @Override
+      public TokenStream next() {
+        if (current == null) {
+          hasNext();
+        }
+        TokenStream next =  new FiniteStringsTokenStream(current);
+        current = null;
+        return next;
+      }
+    };
   }
 
-  private void build(final TokenStream in) throws IOException {
-    if (det != null) {
-      throw new IllegalStateException("Automation already built");
+  /**
+   * Returns the articulation points (or cut vertices) of the graph:
+   * https://en.wikipedia.org/wiki/Biconnected_component
+   */
+  public int[] articulationPoints() {
+    if (det.getNumStates() == 0) {
+      return new int[0];
+    }
+    //
+    Automaton.Builder undirect = new Automaton.Builder();
+    undirect.copy(det);
+    for (int i = 0; i < det.getNumStates(); i++) {
+      int numT = det.initTransition(i, transition);
+      for (int j = 0; j < numT; j++) {
+        det.getNextTransition(transition);
+        undirect.addTransition(transition.dest, i, transition.min);
+      }
     }
+    int numStates = det.getNumStates();
+    BitSet visited = new BitSet(numStates);
+    int[] depth = new int[det.getNumStates()];
+    int[] low = new int[det.getNumStates()];
+    int[] parent = new int[det.getNumStates()];
+    Arrays.fill(parent, -1);
+    List<Integer> points = new ArrayList<>();
+    articulationPointsRecurse(undirect.finish(), 0, 0, depth, low, parent, visited, points);
+    Collections.reverse(points);
+    return points.stream().mapToInt(p -> p).toArray();
+  }
 
+  /**
+   * Build an automaton from the provided {@link TokenStream}.
+   */
+  private Automaton build(final TokenStream in) throws IOException {
+    Automaton.Builder builder = new Automaton.Builder();
     final TermToBytesRefAttribute termBytesAtt = in.addAttribute(TermToBytesRefAttribute.class);
     final PositionIncrementAttribute posIncAtt = in.addAttribute(PositionIncrementAttribute.class);
     final PositionLengthAttribute posLengthAtt = in.addAttribute(PositionLengthAttribute.class);
@@ -136,12 +226,12 @@ public final class GraphTokenStreamFiniteStrings {
 
       int endPos = pos + posLengthAtt.getPositionLength();
       while (state < endPos) {
-        state = createState();
+        state = builder.createState();
       }
 
       BytesRef term = termBytesAtt.getBytesRef();
       int id = getTermID(currentIncr, prevIncr, term);
-      addTransition(pos, endPos, currentIncr, id);
+      builder.addTransition(pos, endPos, id);
 
       // only save last increment on non-zero increment in case we have multiple stacked tokens
       if (currentIncr > 0) {
@@ -150,47 +240,10 @@ public final class GraphTokenStreamFiniteStrings {
     }
 
     in.end();
-    setAccept(state, true);
-    finish();
-  }
-
-  /**
-   * Returns a new state; state 0 is always the initial state.
-   */
-  private int createState() {
-    return builder.createState();
-  }
-
-  /**
-   * Marks the specified state as accept or not.
-   */
-  private void setAccept(int state, boolean accept) {
-    builder.setAccept(state, accept);
-  }
-
-  /**
-   * Adds a transition to the automaton.
-   */
-  private void addTransition(int source, int dest, int incr, int id) {
-    builder.addTransition(source, dest, id);
-  }
-
-  /**
-   * Call this once you are done adding states/transitions.
-   */
-  private void finish() {
-    finish(DEFAULT_MAX_DETERMINIZED_STATES);
-  }
-
-  /**
-   * Call this once you are done adding states/transitions.
-   *
-   * @param maxDeterminizedStates Maximum number of states created when determinizing the automaton.  Higher numbers allow this operation
-   *                              to consume more memory but allow more complex automatons.
-   */
-  private void finish(int maxDeterminizedStates) {
-    Automaton automaton = builder.finish();
-    det = Operations.removeDeadStates(Operations.determinize(automaton, maxDeterminizedStates));
+    if (state != -1) {
+      builder.setAccept(state, true);
+    }
+    return builder.finish();
   }
 
   /**
@@ -224,7 +277,34 @@ public final class GraphTokenStreamFiniteStrings {
         idToTerm.put(id, term);
       }
     }
-
     return id;
   }
+
+  private static void articulationPointsRecurse(Automaton a, int state, int d, int[] depth, int[] low, int[] parent,
+                                                BitSet visited, List<Integer> points) {
+    visited.set(state);
+    depth[state] = d;
+    low[state] = d;
+    int childCount = 0;
+    boolean isArticulation = false;
+    Transition t = new Transition();
+    int numT = a.initTransition(state, t);
+    for (int i = 0; i < numT; i++) {
+      a.getNextTransition(t);
+      if (visited.get(t.dest) == false) {
+        parent[t.dest] = state;
+        articulationPointsRecurse(a, t.dest, d + 1, depth, low, parent, visited, points);
+        childCount++;
+        if (low[t.dest] >= depth[state]) {
+          isArticulation = true;
+        }
+        low[state] = Math.min(low[state], low[t.dest]);
+      } else if (t.dest != parent[state]) {
+        low[state] = Math.min(low[state], depth[t.dest]);
+      }
+    }
+    if ((parent[state] != -1 && isArticulation) || (parent[state] == -1 && childCount > 1)) {
+      points.add(state);
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
index 8f0be3a..b4bfcbe 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
@@ -160,8 +160,8 @@ public class TestLucene60PointsFormat extends BasePointsFormatTestCase {
         }));
 
     // If only one point matches, then the point count is (maxPointsInLeafNode + 1) / 2
-    assertEquals((maxPointsInLeafNode + 1) / 2,
-        points.estimatePointCount(new IntersectVisitor() {
+    // in general, or maybe 2x that if the point is a split value
+    final long pointCount = points.estimatePointCount(new IntersectVisitor() {
           @Override
           public void visit(int docID, byte[] packedValue) throws IOException {}
           
@@ -176,7 +176,10 @@ public class TestLucene60PointsFormat extends BasePointsFormatTestCase {
             }
             return Relation.CELL_CROSSES_QUERY;
           }
-        }));
+        });
+    assertTrue(""+pointCount,
+        pointCount == (maxPointsInLeafNode + 1) / 2 || // common case
+        pointCount == 2*((maxPointsInLeafNode + 1) / 2)); // if the point is a split value
 
     r.close();
     dir.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/lucene/core/src/test/org/apache/lucene/geo/TestGeoUtils.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/geo/TestGeoUtils.java b/lucene/core/src/test/org/apache/lucene/geo/TestGeoUtils.java
index 2cfb2f8..3d95a6d 100644
--- a/lucene/core/src/test/org/apache/lucene/geo/TestGeoUtils.java
+++ b/lucene/core/src/test/org/apache/lucene/geo/TestGeoUtils.java
@@ -293,4 +293,16 @@ public class TestGeoUtils extends LuceneTestCase {
 
     return false;
   }
+
+  public void testWithin90LonDegrees() {
+    assertTrue(GeoUtils.within90LonDegrees(0, -80, 80));
+    assertFalse(GeoUtils.within90LonDegrees(0, -100, 80));
+    assertFalse(GeoUtils.within90LonDegrees(0, -80, 100));
+
+    assertTrue(GeoUtils.within90LonDegrees(-150, 140, 170));
+    assertFalse(GeoUtils.within90LonDegrees(-150, 120, 150));
+
+    assertTrue(GeoUtils.within90LonDegrees(150, -170, -140));
+    assertFalse(GeoUtils.within90LonDegrees(150, -150, -120));
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/lucene/core/src/test/org/apache/lucene/index/TestFilterCodecReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestFilterCodecReader.java b/lucene/core/src/test/org/apache/lucene/index/TestFilterCodecReader.java
new file mode 100644
index 0000000..feb803f
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/index/TestFilterCodecReader.java
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+package org.apache.lucene.index;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestFilterCodecReader extends LuceneTestCase {
+
+  public void testDeclaredMethodsOverridden() throws Exception {
+    final Class<?> subClass = FilterCodecReader.class;
+    implTestDeclaredMethodsOverridden(subClass.getSuperclass(), subClass);
+  }
+
+  private void implTestDeclaredMethodsOverridden(Class<?> superClass, Class<?> subClass) throws Exception {
+    for (final Method superClassMethod : superClass.getDeclaredMethods()) {
+      final int modifiers = superClassMethod.getModifiers();
+      if (Modifier.isPrivate(modifiers)) continue;
+      if (Modifier.isFinal(modifiers)) continue;
+      if (Modifier.isStatic(modifiers)) continue;
+      try {
+        final Method subClassMethod = subClass.getDeclaredMethod(
+            superClassMethod.getName(),
+            superClassMethod.getParameterTypes());
+        assertEquals("getReturnType() difference",
+            superClassMethod.getReturnType(),
+            subClassMethod.getReturnType());
+      } catch (NoSuchMethodException e) {
+        fail(subClass + " needs to override '" + superClassMethod + "'");
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
index 4885fc2..c2b180a 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
@@ -204,7 +204,7 @@ public class TestIndexSorting extends LuceneTestCase {
     // segment sort is needed
     codec.needsIndexSort = true;
     codec.numCalls = 0;
-    for (int i = 200; i < 300; i++) {
+    for (int i = 201; i < 300; i++) {
       Document doc = new Document();
       doc.add(new StringField("id", Integer.toString(i), Store.YES));
       doc.add(new NumericDocValuesField("id", i));
@@ -1698,7 +1698,6 @@ public class TestIndexSorting extends LuceneTestCase {
     dir.close();
   }
 
-
   // docvalues fields involved in the index sort cannot be updated
   public void testBadDVUpdate() throws Exception {
     Directory dir = newDirectory();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/lucene/core/src/test/org/apache/lucene/util/TestQueryBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestQueryBuilder.java b/lucene/core/src/test/org/apache/lucene/util/TestQueryBuilder.java
index 9cd8390..e349e98 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestQueryBuilder.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestQueryBuilder.java
@@ -164,6 +164,77 @@ public class TestQueryBuilder extends LuceneTestCase {
         queryBuilder.createPhraseQuery("field", "guinea pig"));
   }
 
+  /** forms graph query */
+  public void testMultiWordSynonymsBoolean() throws Exception {
+    for (BooleanClause.Occur occur : new BooleanClause.Occur[] {BooleanClause.Occur.SHOULD, BooleanClause.Occur.MUST}) {
+      Query syn1 = new BooleanQuery.Builder()
+          .add(new TermQuery(new Term("field", "guinea")), BooleanClause.Occur.MUST)
+          .add(new TermQuery(new Term("field", "pig")), BooleanClause.Occur.MUST)
+          .build();
+      Query syn2 = new TermQuery(new Term("field", "cavy"));
+      GraphQuery expectedGraphQuery = new GraphQuery(syn1, syn2);
+      QueryBuilder queryBuilder = new QueryBuilder(new MockSynonymAnalyzer());
+      assertEquals(expectedGraphQuery, queryBuilder.createBooleanQuery("field", "guinea pig", occur));
+
+      BooleanQuery expectedBooleanQuery = new BooleanQuery.Builder()
+          .add(expectedGraphQuery, occur)
+          .add(new TermQuery(new Term("field", "story")), occur)
+          .build();
+      assertEquals(expectedBooleanQuery, queryBuilder.createBooleanQuery("field", "guinea pig story", occur));
+
+      expectedBooleanQuery = new BooleanQuery.Builder()
+          .add(new TermQuery(new Term("field", "the")), occur)
+          .add(expectedGraphQuery, occur)
+          .add(new TermQuery(new Term("field", "story")), occur)
+          .build();
+      assertEquals(expectedBooleanQuery, queryBuilder.createBooleanQuery("field", "the guinea pig story", occur));
+
+      expectedBooleanQuery = new BooleanQuery.Builder()
+          .add(new TermQuery(new Term("field", "the")), occur)
+          .add(expectedGraphQuery, occur)
+          .add(new TermQuery(new Term("field", "story")), occur)
+          .add(expectedGraphQuery, occur)
+          .build();
+      assertEquals(expectedBooleanQuery, queryBuilder.createBooleanQuery("field", "the guinea pig story guinea pig", occur));
+    }
+  }
+
+  /** forms graph query */
+  public void testMultiWordPhraseSynonymsBoolean() throws Exception {
+    for (BooleanClause.Occur occur : new BooleanClause.Occur[] {BooleanClause.Occur.SHOULD, BooleanClause.Occur.MUST}) {
+      Query syn1 = new PhraseQuery.Builder()
+          .add(new Term("field", "guinea"))
+          .add(new Term("field", "pig"))
+          .build();
+      Query syn2 = new TermQuery(new Term("field", "cavy"));
+      GraphQuery expectedGraphQuery = new GraphQuery(syn1, syn2);
+      QueryBuilder queryBuilder = new QueryBuilder(new MockSynonymAnalyzer());
+      queryBuilder.setAutoGenerateMultiTermSynonymsPhraseQuery(true);
+      assertEquals(expectedGraphQuery, queryBuilder.createBooleanQuery("field", "guinea pig", occur));
+
+      BooleanQuery expectedBooleanQuery = new BooleanQuery.Builder()
+          .add(expectedGraphQuery, occur)
+          .add(new TermQuery(new Term("field", "story")), occur)
+          .build();
+      assertEquals(expectedBooleanQuery, queryBuilder.createBooleanQuery("field", "guinea pig story", occur));
+
+      expectedBooleanQuery = new BooleanQuery.Builder()
+          .add(new TermQuery(new Term("field", "the")), occur)
+          .add(expectedGraphQuery, occur)
+          .add(new TermQuery(new Term("field", "story")), occur)
+          .build();
+      assertEquals(expectedBooleanQuery, queryBuilder.createBooleanQuery("field", "the guinea pig story", occur));
+
+      expectedBooleanQuery = new BooleanQuery.Builder()
+          .add(new TermQuery(new Term("field", "the")), occur)
+          .add(expectedGraphQuery, occur)
+          .add(new TermQuery(new Term("field", "story")), occur)
+          .add(expectedGraphQuery, occur)
+          .build();
+      assertEquals(expectedBooleanQuery, queryBuilder.createBooleanQuery("field", "the guinea pig story guinea pig", occur));
+    }
+  }
+
   protected static class SimpleCJKTokenizer extends Tokenizer {
     private CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/lucene/core/src/test/org/apache/lucene/util/graph/TestGraphTokenStreamFiniteStrings.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/graph/TestGraphTokenStreamFiniteStrings.java b/lucene/core/src/test/org/apache/lucene/util/graph/TestGraphTokenStreamFiniteStrings.java
index 4e636e2..8c336cd 100644
--- a/lucene/core/src/test/org/apache/lucene/util/graph/TestGraphTokenStreamFiniteStrings.java
+++ b/lucene/core/src/test/org/apache/lucene/util/graph/TestGraphTokenStreamFiniteStrings.java
@@ -16,13 +16,14 @@
  */
 package org.apache.lucene.util.graph;
 
-import java.util.List;
+import java.util.Iterator;
 
 import org.apache.lucene.analysis.CannedTokenStream;
 import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.BytesTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.util.LuceneTestCase;
 
 /**
@@ -65,10 +66,19 @@ public class TestGraphTokenStreamFiniteStrings extends LuceneTestCase {
           token("b", 1, 1)
       );
 
-      GraphTokenStreamFiniteStrings.getTokenStreams(ts);
+      new GraphTokenStreamFiniteStrings(ts).getFiniteStrings();
     });
   }
 
+  public void testEmpty() throws Exception {
+    TokenStream ts = new CannedTokenStream(
+    );
+    GraphTokenStreamFiniteStrings graph = new GraphTokenStreamFiniteStrings(ts);
+    Iterator<TokenStream> it = graph.getFiniteStrings();
+    assertFalse(it.hasNext());
+    assertArrayEquals(new int[0], graph.articulationPoints());
+  }
+
   public void testSingleGraph() throws Exception {
     TokenStream ts = new CannedTokenStream(
         token("fast", 1, 1),
@@ -78,11 +88,41 @@ public class TestGraphTokenStreamFiniteStrings extends LuceneTestCase {
         token("network", 1, 1)
     );
 
-    List<TokenStream> finiteTokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(ts);
-
-    assertEquals(2, finiteTokenStreams.size());
-    assertTokenStream(finiteTokenStreams.get(0), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 1, 1, 1});
-    assertTokenStream(finiteTokenStreams.get(1), new String[]{"fast", "wifi", "network"}, new int[]{1, 1, 1});
+    GraphTokenStreamFiniteStrings graph = new GraphTokenStreamFiniteStrings(ts);
+
+    Iterator<TokenStream> it = graph.getFiniteStrings();
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 1, 1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast", "wifi", "network"}, new int[]{1, 1, 1});
+    assertFalse(it.hasNext());
+
+    int[] points = graph.articulationPoints();
+    assertArrayEquals(points, new int[] {1, 3});
+
+    assertFalse(graph.hasSidePath(0));
+    it = graph.getFiniteStrings(0, 1);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast"}, new int[] {1});
+    assertFalse(it.hasNext());
+    Term[] terms = graph.getTerms("field", 0);
+    assertArrayEquals(terms, new Term[] {new Term("field", "fast")});
+
+    assertTrue(graph.hasSidePath(1));
+    it = graph.getFiniteStrings(1, 3);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"wi", "fi"}, new int[]{1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"wifi"}, new int[]{1});
+    assertFalse(it.hasNext());
+
+    assertFalse(graph.hasSidePath(3));
+    it = graph.getFiniteStrings(3, -1);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"network"}, new int[] {1});
+    assertFalse(it.hasNext());
+    terms = graph.getTerms("field", 3);
+    assertArrayEquals(terms, new Term[] {new Term("field", "network")});
   }
 
   public void testSingleGraphWithGap() throws Exception {
@@ -96,13 +136,51 @@ public class TestGraphTokenStreamFiniteStrings extends LuceneTestCase {
         token("network", 1, 1)
     );
 
-    List<TokenStream> finiteTokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(ts);
+    GraphTokenStreamFiniteStrings graph = new GraphTokenStreamFiniteStrings(ts);
 
-    assertEquals(2, finiteTokenStreams.size());
-    assertTokenStream(finiteTokenStreams.get(0),
+    Iterator<TokenStream> it = graph.getFiniteStrings();
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(),
         new String[]{"hey", "fast", "wi", "fi", "network"}, new int[]{1, 2, 1, 1, 1});
-    assertTokenStream(finiteTokenStreams.get(1),
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(),
         new String[]{"hey", "fast", "wifi", "network"}, new int[]{1, 2, 1, 1});
+    assertFalse(it.hasNext());
+
+    int[] points = graph.articulationPoints();
+    assertArrayEquals(points, new int[] {1, 2, 4});
+
+    assertFalse(graph.hasSidePath(0));
+    it = graph.getFiniteStrings(0, 1);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"hey"}, new int[] {1});
+    assertFalse(it.hasNext());
+    Term[] terms = graph.getTerms("field", 0);
+    assertArrayEquals(terms, new Term[] {new Term("field", "hey")});
+
+    assertFalse(graph.hasSidePath(1));
+    it = graph.getFiniteStrings(1, 2);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast"}, new int[] {2});
+    assertFalse(it.hasNext());
+    terms = graph.getTerms("field", 1);
+    assertArrayEquals(terms, new Term[] {new Term("field", "fast")});
+
+    assertTrue(graph.hasSidePath(2));
+    it = graph.getFiniteStrings(2, 4);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"wi", "fi"}, new int[]{1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"wifi"}, new int[]{1});
+    assertFalse(it.hasNext());
+
+    assertFalse(graph.hasSidePath(4));
+    it = graph.getFiniteStrings(4, -1);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"network"}, new int[] {1});
+    assertFalse(it.hasNext());
+    terms = graph.getTerms("field", 4);
+    assertArrayEquals(terms, new Term[] {new Term("field", "network")});
   }
 
 
@@ -115,11 +193,41 @@ public class TestGraphTokenStreamFiniteStrings extends LuceneTestCase {
         token("network", 1, 1)
     );
 
-    List<TokenStream> finiteTokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(ts);
-
-    assertEquals(2, finiteTokenStreams.size());
-    assertTokenStream(finiteTokenStreams.get(0), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 2, 1, 1});
-    assertTokenStream(finiteTokenStreams.get(1), new String[]{"fast", "wifi", "network"}, new int[]{1, 2, 1});
+    GraphTokenStreamFiniteStrings graph = new GraphTokenStreamFiniteStrings(ts);
+
+    Iterator<TokenStream> it = graph.getFiniteStrings();
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 2, 1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast", "wifi", "network"}, new int[]{1, 2, 1});
+    assertFalse(it.hasNext());
+
+    int[] points = graph.articulationPoints();
+    assertArrayEquals(points, new int[] {1, 3});
+
+    assertFalse(graph.hasSidePath(0));
+    it = graph.getFiniteStrings(0, 1);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast"}, new int[] {1});
+    assertFalse(it.hasNext());
+    Term[] terms = graph.getTerms("field", 0);
+    assertArrayEquals(terms, new Term[] {new Term("field", "fast")});
+
+    assertTrue(graph.hasSidePath(1));
+    it = graph.getFiniteStrings(1, 3);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"wi", "fi"}, new int[]{2, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"wifi"}, new int[]{2});
+    assertFalse(it.hasNext());
+
+    assertFalse(graph.hasSidePath(3));
+    it = graph.getFiniteStrings(3, -1);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"network"}, new int[] {1});
+    assertFalse(it.hasNext());
+    terms = graph.getTerms("field", 3);
+    assertArrayEquals(terms, new Term[] {new Term("field", "network")});
   }
 
   public void testGraphAndGapSameTokenTerm() throws Exception {
@@ -131,11 +239,41 @@ public class TestGraphTokenStreamFiniteStrings extends LuceneTestCase {
         token("d", 1, 1)
     );
 
-    List<TokenStream> finiteTokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(ts);
-
-    assertEquals(2, finiteTokenStreams.size());
-    assertTokenStream(finiteTokenStreams.get(0), new String[]{"a", "b", "c", "d"}, new int[]{1, 1, 2, 1});
-    assertTokenStream(finiteTokenStreams.get(1), new String[]{"a", "b", "a"}, new int[]{1, 1, 2});
+    GraphTokenStreamFiniteStrings graph = new GraphTokenStreamFiniteStrings(ts);
+
+    Iterator<TokenStream> it = graph.getFiniteStrings();
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"a", "b", "c", "d"}, new int[]{1, 1, 2, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"a", "b", "a"}, new int[]{1, 1, 2});
+    assertFalse(it.hasNext());
+
+    int[] points = graph.articulationPoints();
+    assertArrayEquals(points, new int[] {1, 2});
+
+    assertFalse(graph.hasSidePath(0));
+    it = graph.getFiniteStrings(0, 1);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"a"}, new int[] {1});
+    assertFalse(it.hasNext());
+    Term[] terms = graph.getTerms("field", 0);
+    assertArrayEquals(terms, new Term[] {new Term("field", "a")});
+
+    assertFalse(graph.hasSidePath(1));
+    it = graph.getFiniteStrings(1, 2);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"b"}, new int[] {1});
+    assertFalse(it.hasNext());
+    terms = graph.getTerms("field", 1);
+    assertArrayEquals(terms, new Term[] {new Term("field", "b")});
+
+    assertTrue(graph.hasSidePath(2));
+    it = graph.getFiniteStrings(2, -1);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"c", "d"}, new int[] {2, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"a"}, new int[] {2});
+    assertFalse(it.hasNext());
   }
 
   public void testStackedGraph() throws Exception {
@@ -148,12 +286,45 @@ public class TestGraphTokenStreamFiniteStrings extends LuceneTestCase {
         token("network", 1, 1)
     );
 
-    List<TokenStream> finiteTokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(ts);
-
-    assertEquals(3, finiteTokenStreams.size());
-    assertTokenStream(finiteTokenStreams.get(0), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 1, 1, 1});
-    assertTokenStream(finiteTokenStreams.get(1), new String[]{"fast", "wifi", "network"}, new int[]{1, 1, 1});
-    assertTokenStream(finiteTokenStreams.get(2), new String[]{"fast", "wireless", "network"}, new int[]{1, 1, 1});
+    GraphTokenStreamFiniteStrings graph = new GraphTokenStreamFiniteStrings(ts);
+
+    Iterator<TokenStream> it = graph.getFiniteStrings();
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 1, 1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast", "wifi", "network"}, new int[]{1, 1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast", "wireless", "network"}, new int[]{1, 1, 1});
+    assertFalse(it.hasNext());
+
+    int[] points = graph.articulationPoints();
+    assertArrayEquals(points, new int[] {1, 3});
+
+    assertFalse(graph.hasSidePath(0));
+    it = graph.getFiniteStrings(0, 1);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast"}, new int[] {1});
+    assertFalse(it.hasNext());
+    Term[] terms = graph.getTerms("field", 0);
+    assertArrayEquals(terms, new Term[] {new Term("field", "fast")});
+
+    assertTrue(graph.hasSidePath(1));
+    it = graph.getFiniteStrings(1, 3);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"wi", "fi"}, new int[]{1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"wifi"}, new int[]{1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"wireless"}, new int[]{1});
+    assertFalse(it.hasNext());
+
+    assertFalse(graph.hasSidePath(3));
+    it = graph.getFiniteStrings(3, -1);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"network"}, new int[] {1});
+    assertFalse(it.hasNext());
+    terms = graph.getTerms("field", 3);
+    assertArrayEquals(terms, new Term[] {new Term("field", "network")});
   }
 
   public void testStackedGraphWithGap() throws Exception {
@@ -166,12 +337,45 @@ public class TestGraphTokenStreamFiniteStrings extends LuceneTestCase {
         token("network", 1, 1)
     );
 
-    List<TokenStream> finiteTokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(ts);
-
-    assertEquals(3, finiteTokenStreams.size());
-    assertTokenStream(finiteTokenStreams.get(0), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 2, 1, 1});
-    assertTokenStream(finiteTokenStreams.get(1), new String[]{"fast", "wifi", "network"}, new int[]{1, 2, 1});
-    assertTokenStream(finiteTokenStreams.get(2), new String[]{"fast", "wireless", "network"}, new int[]{1, 2, 1});
+    GraphTokenStreamFiniteStrings graph = new GraphTokenStreamFiniteStrings(ts);
+
+    Iterator<TokenStream> it = graph.getFiniteStrings();
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 2, 1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast", "wifi", "network"}, new int[]{1, 2, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast", "wireless", "network"}, new int[]{1, 2, 1});
+    assertFalse(it.hasNext());
+
+    int[] points = graph.articulationPoints();
+    assertArrayEquals(points, new int[] {1, 3});
+
+    assertFalse(graph.hasSidePath(0));
+    it = graph.getFiniteStrings(0, 1);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast"}, new int[] {1});
+    assertFalse(it.hasNext());
+    Term[] terms = graph.getTerms("field", 0);
+    assertArrayEquals(terms, new Term[] {new Term("field", "fast")});
+
+    assertTrue(graph.hasSidePath(1));
+    it = graph.getFiniteStrings(1, 3);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"wi", "fi"}, new int[]{2, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"wifi"}, new int[]{2});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"wireless"}, new int[]{2});
+    assertFalse(it.hasNext());
+
+    assertFalse(graph.hasSidePath(3));
+    it = graph.getFiniteStrings(3, -1);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"network"}, new int[] {1});
+    assertFalse(it.hasNext());
+    terms = graph.getTerms("field", 3);
+    assertArrayEquals(terms, new Term[] {new Term("field", "network")});
   }
 
   public void testGraphWithRegularSynonym() throws Exception {
@@ -184,13 +388,47 @@ public class TestGraphTokenStreamFiniteStrings extends LuceneTestCase {
         token("network", 1, 1)
     );
 
-    List<TokenStream> finiteTokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(ts);
-
-    assertEquals(4, finiteTokenStreams.size());
-    assertTokenStream(finiteTokenStreams.get(0), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 1, 1, 1});
-    assertTokenStream(finiteTokenStreams.get(1), new String[]{"fast", "wifi", "network"}, new int[]{1, 1, 1});
-    assertTokenStream(finiteTokenStreams.get(2), new String[]{"speedy", "wi", "fi", "network"}, new int[]{1, 1, 1, 1});
-    assertTokenStream(finiteTokenStreams.get(3), new String[]{"speedy", "wifi", "network"}, new int[]{1, 1, 1});
+    GraphTokenStreamFiniteStrings graph = new GraphTokenStreamFiniteStrings(ts);
+
+    Iterator<TokenStream> it = graph.getFiniteStrings();
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 1, 1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast", "wifi", "network"}, new int[]{1, 1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"speedy", "wi", "fi", "network"}, new int[]{1, 1, 1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"speedy", "wifi", "network"}, new int[]{1, 1, 1});
+    assertFalse(it.hasNext());
+
+    int[] points = graph.articulationPoints();
+    assertArrayEquals(points, new int[] {1, 3});
+
+    assertFalse(graph.hasSidePath(0));
+    it = graph.getFiniteStrings(0, 1);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast"}, new int[] {1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"speedy"}, new int[] {1});
+    assertFalse(it.hasNext());
+    Term[] terms = graph.getTerms("field", 0);
+    assertArrayEquals(terms, new Term[] {new Term("field", "fast"), new Term("field", "speedy")});
+
+    assertTrue(graph.hasSidePath(1));
+    it = graph.getFiniteStrings(1, 3);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"wi", "fi"}, new int[]{1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"wifi"}, new int[]{1});
+    assertFalse(it.hasNext());
+
+    assertFalse(graph.hasSidePath(3));
+    it = graph.getFiniteStrings(3, -1);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"network"}, new int[] {1});
+    assertFalse(it.hasNext());
+    terms = graph.getTerms("field", 3);
+    assertArrayEquals(terms, new Term[] {new Term("field", "network")});
   }
 
   public void testMultiGraph() throws Exception {
@@ -204,14 +442,105 @@ public class TestGraphTokenStreamFiniteStrings extends LuceneTestCase {
         token("network", 1, 1)
     );
 
-    List<TokenStream> finiteTokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(ts);
+    GraphTokenStreamFiniteStrings graph = new GraphTokenStreamFiniteStrings(ts);
 
-    assertEquals(4, finiteTokenStreams.size());
-    assertTokenStream(finiteTokenStreams.get(0),
+    Iterator<TokenStream> it = graph.getFiniteStrings();
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(),
         new String[]{"turbo", "charged", "wi", "fi", "network"}, new int[]{1, 1, 1, 1, 1});
-    assertTokenStream(finiteTokenStreams.get(1),
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(),
         new String[]{"turbo", "charged", "wifi", "network"}, new int[]{1, 1, 1, 1});
-    assertTokenStream(finiteTokenStreams.get(2), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 1, 1, 1});
-    assertTokenStream(finiteTokenStreams.get(3), new String[]{"fast", "wifi", "network"}, new int[]{1, 1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 1, 1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast", "wifi", "network"}, new int[]{1, 1, 1});
+    assertFalse(it.hasNext());
+
+    int[] points = graph.articulationPoints();
+    assertArrayEquals(points, new int[] {2, 4});
+
+    assertTrue(graph.hasSidePath(0));
+    it = graph.getFiniteStrings(0, 2);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"turbo", "charged"}, new int[]{1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"fast"}, new int[]{1});
+    assertFalse(it.hasNext());
+
+    assertTrue(graph.hasSidePath(2));
+    it = graph.getFiniteStrings(2, 4);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"wi", "fi"}, new int[]{1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"wifi"}, new int[]{1});
+    assertFalse(it.hasNext());
+
+    assertFalse(graph.hasSidePath(4));
+    it = graph.getFiniteStrings(4, -1);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"network"}, new int[] {1});
+    assertFalse(it.hasNext());
+    Term[] terms = graph.getTerms("field", 4);
+    assertArrayEquals(terms, new Term[] {new Term("field", "network")});
+  }
+
+  public void testMultipleSidePaths() throws Exception {
+    TokenStream ts = new CannedTokenStream(
+        token("the", 1, 1),
+        token("ny", 1, 4),
+        token("new", 0, 1),
+        token("york", 1, 1),
+        token("wifi", 1, 4),
+        token("wi", 0, 1),
+        token("fi", 1, 3),
+        token("wifi", 2, 2),
+        token("wi", 0, 1),
+        token("fi", 1, 1),
+        token("network", 1, 1)
+    );
+    GraphTokenStreamFiniteStrings graph = new GraphTokenStreamFiniteStrings(ts);
+
+    Iterator<TokenStream> it = graph.getFiniteStrings();
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"the", "ny", "wifi", "network"}, new int[]{1, 1, 2, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"the", "ny", "wi", "fi", "network"}, new int[]{1, 1, 2, 1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"the", "new", "york", "wifi", "network"}, new int[]{1, 1, 1, 1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"the", "new", "york", "wi", "fi", "network"}, new int[]{1, 1, 1, 1, 1, 1});
+    assertFalse(it.hasNext());
+
+    int[] points = graph.articulationPoints();
+    assertArrayEquals(points, new int[] {1, 7});
+
+    assertFalse(graph.hasSidePath(0));
+    it = graph.getFiniteStrings(0, 1);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"the"}, new int[]{1});
+    assertFalse(it.hasNext());
+    Term[] terms = graph.getTerms("field", 0);
+    assertArrayEquals(terms, new Term[] {new Term("field", "the")});
+
+    assertTrue(graph.hasSidePath(1));
+    it = graph.getFiniteStrings(1, 7);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"ny", "wifi"}, new int[]{1, 2});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"ny", "wi", "fi"}, new int[]{1, 2, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"new", "york", "wifi"}, new int[]{1, 1, 1});
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"new", "york", "wi", "fi"}, new int[]{1, 1, 1, 1});
+    assertFalse(it.hasNext());
+
+    assertFalse(graph.hasSidePath(7));
+    it = graph.getFiniteStrings(7, -1);
+    assertTrue(it.hasNext());
+    assertTokenStream(it.next(), new String[]{"network"}, new int[]{1});
+    assertFalse(it.hasNext());
+    terms = graph.getTerms("field", 7);
+    assertArrayEquals(terms, new Term[] {new Term("field", "network")});
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/lucene/join/build.xml
----------------------------------------------------------------------
diff --git a/lucene/join/build.xml b/lucene/join/build.xml
index b5360c4..c411dbe 100644
--- a/lucene/join/build.xml
+++ b/lucene/join/build.xml
@@ -24,25 +24,4 @@
 
   <import file="../module-build.xml"/>
 
-  <path id="classpath">
-    <pathelement path="${grouping.jar}"/>
-    <path refid="base.classpath"/>
-  </path>
-
-  <path id="run.classpath">
-    <path refid="classpath"/>
-    <pathelement location="${build.dir}/classes/java"/>
-  </path>
-
-  <target name="init" depends="module-build.init,jar-grouping"/>
-
-  <target name="javadocs" depends="javadocs-grouping,compile-core,check-javadocs-uptodate"
-          unless="javadocs-uptodate-${name}">
-    <invoke-module-javadoc>
-      <links>
-        <link href="../grouping"/>
-      </links>
-    </invoke-module-javadoc>
-  </target>
-
 </project>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/lucene/join/src/java/org/apache/lucene/search/join/ParentChildrenBlockJoinQuery.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/ParentChildrenBlockJoinQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/ParentChildrenBlockJoinQuery.java
new file mode 100644
index 0000000..a739294
--- /dev/null
+++ b/lucene/join/src/java/org/apache/lucene/search/join/ParentChildrenBlockJoinQuery.java
@@ -0,0 +1,199 @@
+/*
+ * 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.
+ */
+
+package org.apache.lucene.search.join;
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.ReaderUtil;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.BitSet;
+
+/**
+ * A query that returns all the matching child documents for a specific parent document
+ * indexed together in the same block. The provided child query determines which matching
+ * child doc is being returned.
+ *
+ * @lucene.experimental
+ */
+public class ParentChildrenBlockJoinQuery extends Query {
+
+  private final BitSetProducer parentFilter;
+  private final Query childQuery;
+  private final int parentDocId;
+
+  /**
+   * Creates a <code>ParentChildrenBlockJoinQuery</code> instance
+   *
+   * @param parentFilter  A filter identifying parent documents.
+   * @param childQuery    A child query that determines which child docs are matching
+   * @param parentDocId   The top level doc id of that parent to return children documents for
+   */
+  public ParentChildrenBlockJoinQuery(BitSetProducer parentFilter, Query childQuery, int parentDocId) {
+    this.parentFilter = parentFilter;
+    this.childQuery = childQuery;
+    this.parentDocId = parentDocId;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (sameClassAs(obj) == false) {
+      return false;
+    }
+    ParentChildrenBlockJoinQuery other = (ParentChildrenBlockJoinQuery) obj;
+    return parentFilter.equals(other.parentFilter)
+        && childQuery.equals(other.childQuery)
+        && parentDocId == other.parentDocId;
+  }
+
+  @Override
+  public int hashCode() {
+    int hash = classHash();
+    hash = 31 * hash + parentFilter.hashCode();
+    hash = 31 * hash +  childQuery.hashCode();
+    hash = 31 * hash + parentDocId;
+    return hash;
+  }
+
+  @Override
+  public String toString(String field) {
+    return "ParentChildrenBlockJoinQuery (" + childQuery + ")";
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    final Query childRewrite = childQuery.rewrite(reader);
+    if (childRewrite != childQuery) {
+      return new ParentChildrenBlockJoinQuery(parentFilter, childRewrite, parentDocId);
+    } else {
+      return super.rewrite(reader);
+    }
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
+    final Weight childWeight = childQuery.createWeight(searcher, needsScores, boost);
+    final int readerIndex = ReaderUtil.subIndex(parentDocId, searcher.getIndexReader().leaves());
+    return new Weight(this) {
+
+      @Override
+      public void extractTerms(Set<Term> terms) {
+        childWeight.extractTerms(terms);
+      }
+
+      @Override
+      public Explanation explain(LeafReaderContext context, int doc) throws IOException {
+        return Explanation.noMatch("Not implemented, use ToParentBlockJoinQuery explain why a document matched");
+      }
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        // Childs docs only reside in a single segment, so no need to evaluate all segments
+        if (context.ord != readerIndex) {
+          return null;
+        }
+
+        final int localParentDocId = parentDocId - context.docBase;
+        // If parentDocId == 0 then a parent doc doesn't have child docs, because child docs are stored
+        // before the parent doc and because parent doc is 0 we can safely assume that there are no child docs.
+        if (localParentDocId == 0) {
+          return null;
+        }
+
+        final BitSet parents = parentFilter.getBitSet(context);
+        final int firstChildDocId = parents.prevSetBit(localParentDocId - 1) + 1;
+        // A parent doc doesn't have child docs, so we can early exit here:
+        if (firstChildDocId == localParentDocId) {
+          return null;
+        }
+
+        final Scorer childrenScorer = childWeight.scorer(context);
+        if (childrenScorer == null) {
+          return null;
+        }
+        DocIdSetIterator childrenIterator = childrenScorer.iterator();
+        final DocIdSetIterator it = new DocIdSetIterator() {
+
+          int doc = -1;
+
+          @Override
+          public int docID() {
+            return doc;
+          }
+
+          @Override
+          public int nextDoc() throws IOException {
+            return advance(doc + 1);
+          }
+
+          @Override
+          public int advance(int target) throws IOException {
+            target = Math.max(firstChildDocId, target);
+            if (target >= localParentDocId) {
+              // We're outside the child nested scope, so it is done
+              return doc = NO_MORE_DOCS;
+            } else {
+              int advanced = childrenIterator.advance(target);
+              if (advanced >= localParentDocId) {
+                // We're outside the child nested scope, so it is done
+                return doc = NO_MORE_DOCS;
+              } else {
+                return doc = advanced;
+              }
+            }
+          }
+
+          @Override
+          public long cost() {
+            return Math.min(childrenIterator.cost(), localParentDocId - firstChildDocId);
+          }
+
+        };
+        return new Scorer(this) {
+          @Override
+          public int docID() {
+            return it.docID();
+          }
+
+          @Override
+          public float score() throws IOException {
+            return childrenScorer.score();
+          }
+
+          @Override
+          public int freq() throws IOException {
+            return childrenScorer.freq();
+          }
+
+          @Override
+          public DocIdSetIterator iterator() {
+            return it;
+          }
+        };
+      }
+    };
+  }
+}