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

lucene-solr:branch_6x: LUCENE-7638: Query parsers now analyze the token graph for articulation points (or cut vertices) in order to create more efficient queries for multi-token synonyms.

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x d1135b4d2 -> 5386116e9


LUCENE-7638: Query parsers now analyze the token graph for articulation points (or cut vertices) in order to create more efficient queries for multi-token synonyms.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/5386116e
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/5386116e
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/5386116e

Branch: refs/heads/branch_6x
Commit: 5386116e9a3b54db4674b21e39a41cc4d43553f8
Parents: d1135b4
Author: Jim Ferenczi <ji...@elastic.co>
Authored: Tue Feb 7 11:27:59 2017 +0100
Committer: Jim Ferenczi <ji...@elastic.co>
Committed: Tue Feb 7 12:51:56 2017 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   4 +
 .../org/apache/lucene/util/QueryBuilder.java    | 122 ++++--
 .../util/automaton/FiniteStringsIterator.java   |  36 +-
 .../graph/GraphTokenStreamFiniteStrings.java    | 212 +++++++---
 .../apache/lucene/util/TestQueryBuilder.java    |  71 ++++
 .../TestGraphTokenStreamFiniteStrings.java      | 421 +++++++++++++++++--
 .../classic/TestMultiFieldQueryParser.java      |   4 +-
 .../queryparser/classic/TestQueryParser.java    |  53 ++-
 8 files changed, 745 insertions(+), 178 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5386116e/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 4dbfc8a..f834ca4 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -88,6 +88,10 @@ Optimizations
 * LUCENE-7667: BKDReader now calls `IntersectVisitor.grow()` on larger
   increments. (Adrien Grand)
 
+* LUCENE-7638: Query parsers now analyze the token graph for articulation
+  points (or cut vertices) in order to create more efficient queries for
+  multi-token synonyms. (Jim Ferenczi)
+
 Build
 
 * LUCENE-7653: Update randomizedtesting to version 2.5.0. (Dawid Weiss)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5386116e/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 a632614..f6517e4 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;
   }
@@ -209,6 +196,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.
@@ -295,7 +298,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) {
@@ -445,25 +452,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]));
   }
 
@@ -486,6 +542,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/5386116e/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/5386116e/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/5386116e/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/5386116e/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/5386116e/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiFieldQueryParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiFieldQueryParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiFieldQueryParser.java
index 0147b86..6108938 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiFieldQueryParser.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiFieldQueryParser.java
@@ -356,7 +356,7 @@ public class TestMultiFieldQueryParser extends LuceneTestCase {
     parser.setSplitOnWhitespace(false);
     q = parser.parse("guinea pig");
     assertFalse(parser.getSplitOnWhitespace());
-    assertEquals("Graph(b:guinea b:pig, b:cavy, hasBoolean=true, hasPhrase=false) "
-        + "Graph(t:guinea t:pig, t:cavy, hasBoolean=true, hasPhrase=false)", q.toString());
+    assertEquals("Graph(+b:guinea +b:pig, b:cavy, hasBoolean=true, hasPhrase=false) "
+        + "Graph(+t:guinea +t:pig, t:cavy, hasBoolean=true, hasPhrase=false)", q.toString());
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5386116e/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java
index d3d288a..524e60e 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java
@@ -505,8 +505,8 @@ public class TestQueryParser extends QueryParserTestBase {
 
     // A multi-word synonym source will form a graph query for synonyms that formed the graph token stream
     BooleanQuery.Builder synonym = new BooleanQuery.Builder();
-    synonym.add(guinea, BooleanClause.Occur.SHOULD);
-    synonym.add(pig, BooleanClause.Occur.SHOULD);
+    synonym.add(guinea, BooleanClause.Occur.MUST);
+    synonym.add(pig, BooleanClause.Occur.MUST);
     BooleanQuery guineaPig = synonym.build();
 
     GraphQuery graphQuery = new GraphQuery(guineaPig, cavy);
@@ -579,30 +579,30 @@ public class TestQueryParser extends QueryParserTestBase {
     assertQueryEquals("guinea /pig/", a, "guinea /pig/");
 
     // Operators should not interrupt multiword analysis if not don't associate
-    assertQueryEquals("(guinea pig)", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
-    assertQueryEquals("+(guinea pig)", a, "+Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
-    assertQueryEquals("-(guinea pig)", a, "-Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
-    assertQueryEquals("!(guinea pig)", a, "-Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
-    assertQueryEquals("NOT (guinea pig)", a, "-Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
-    assertQueryEquals("(guinea pig)^2", a, "(Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false))^2.0");
-
-    assertQueryEquals("field:(guinea pig)", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
-
-    assertQueryEquals("+small guinea pig", a, "+small Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
-    assertQueryEquals("-small guinea pig", a, "-small Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
-    assertQueryEquals("!small guinea pig", a, "-small Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
-    assertQueryEquals("NOT small guinea pig", a, "-small Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
-    assertQueryEquals("small* guinea pig", a, "small* Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
-    assertQueryEquals("small? guinea pig", a, "small? Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
-    assertQueryEquals("\"small\" guinea pig", a, "small Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
-
-    assertQueryEquals("guinea pig +running", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false) +running");
-    assertQueryEquals("guinea pig -running", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false) -running");
-    assertQueryEquals("guinea pig !running", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false) -running");
-    assertQueryEquals("guinea pig NOT running", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false) -running");
-    assertQueryEquals("guinea pig running*", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false) running*");
-    assertQueryEquals("guinea pig running?", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false) running?");
-    assertQueryEquals("guinea pig \"running\"", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false) running");
+    assertQueryEquals("(guinea pig)", a, "Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("+(guinea pig)", a, "+Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("-(guinea pig)", a, "-Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("!(guinea pig)", a, "-Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("NOT (guinea pig)", a, "-Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("(guinea pig)^2", a, "(Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false))^2.0");
+
+    assertQueryEquals("field:(guinea pig)", a, "Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+
+    assertQueryEquals("+small guinea pig", a, "+small Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("-small guinea pig", a, "-small Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("!small guinea pig", a, "-small Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("NOT small guinea pig", a, "-small Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("small* guinea pig", a, "small* Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("small? guinea pig", a, "small? Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("\"small\" guinea pig", a, "small Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+
+    assertQueryEquals("guinea pig +running", a, "Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false) +running");
+    assertQueryEquals("guinea pig -running", a, "Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false) -running");
+    assertQueryEquals("guinea pig !running", a, "Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false) -running");
+    assertQueryEquals("guinea pig NOT running", a, "Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false) -running");
+    assertQueryEquals("guinea pig running*", a, "Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false) running*");
+    assertQueryEquals("guinea pig running?", a, "Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false) running?");
+    assertQueryEquals("guinea pig \"running\"", a, "Graph(+field:guinea +field:pig, field:cavy, hasBoolean=true, hasPhrase=false) running");
 
     assertQueryEquals("\"guinea pig\"~2", a, "Graph(field:\"guinea pig\"~2, field:cavy, hasBoolean=false, hasPhrase=true)");
 
@@ -703,7 +703,6 @@ public class TestQueryParser extends QueryParserTestBase {
     boolean oldSplitOnWhitespace = splitOnWhitespace;
     splitOnWhitespace = QueryParser.DEFAULT_SPLIT_ON_WHITESPACE;
     assertQueryEquals("guinea pig", new MockSynonymAnalyzer(), "guinea pig");
-    splitOnWhitespace = oldSplitOnWhitespace;
   }
 
   // LUCENE-7533