You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/07/04 21:47:36 UTC

svn commit: r1689192 - in /lucene/dev/trunk/lucene: ./ core/src/java/org/apache/lucene/util/automaton/ core/src/test/org/apache/lucene/util/automaton/ suggest/src/java/org/apache/lucene/search/suggest/analyzing/ suggest/src/java/org/apache/lucene/searc...

Author: mikemccand
Date: Sat Jul  4 19:47:35 2015
New Revision: 1689192

URL: http://svn.apache.org/r1689192
Log:
LUCENE-6365: switch to iterator API to get all finite strings from an Automaton

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/FiniteStringsIterator.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/LimitedFiniteStringsIterator.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/FiniteStringsIteratorTest.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/LimitedFiniteStringsIteratorTest.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestOperations.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java
    lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1689192&r1=1689191&r2=1689192&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Sat Jul  4 19:47:35 2015
@@ -126,6 +126,10 @@ New Features
 * LUCENE-6365: Add Operations.topoSort, to run topological sort of the
   states in an Automaton (Markus Heiden via Mike McCandless)
 
+* LUCENE-6365: Replace Operations.getFiniteStrings with a
+  more scalable iterator API (FiniteStringsIterator) (Markus Heiden
+  via Mike McCandless)
+
 API Changes
 
 * LUCENE-6508: Simplify Lock api, there is now just 

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/FiniteStringsIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/FiniteStringsIterator.java?rev=1689192&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/FiniteStringsIterator.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/FiniteStringsIterator.java Sat Jul  4 19:47:35 2015
@@ -0,0 +1,216 @@
+package org.apache.lucene.util.automaton;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import 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.
+ *
+ * <p>If the {@link Automaton} has cycles then this iterator may throw an {@code
+ * IllegalArgumentException}, but this is not guaranteed!
+ *
+ * <p>Be aware that the iteration order is implementation dependent
+ * and may change across releases.
+ *
+ * @lucene.experimental
+ */
+public class FiniteStringsIterator {
+  /**
+   * Empty string.
+   */
+  private static final IntsRef EMPTY = new IntsRef();
+
+  /**
+   * Automaton to create finite string from.
+   */
+  private final Automaton a;
+
+  /**
+   * Tracks which states are in the current path, for cycle detection.
+   */
+  private final BitSet pathStates;
+
+  /**
+   * Builder for current finite string.
+   */
+  private final IntsRefBuilder string;
+
+  /**
+   * Stack to hold our current state in the recursion/iteration.
+   */
+  private PathNode[] nodes;
+
+  /**
+   * Emit empty string?.
+   */
+  private boolean emitEmptyString;
+
+  /**
+   * Constructor.
+   *
+   * @param a Automaton to create finite string from.
+   */
+  public FiniteStringsIterator(Automaton a) {
+    this.a = a;
+    this.nodes = new PathNode[16];
+    for (int i = 0, end = nodes.length; i < end; i++) {
+      nodes[i] = new PathNode();
+    }
+    this.string = new IntsRefBuilder();
+    this.pathStates = new BitSet(a.getNumStates());
+    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);
+    }
+  }
+
+  /**
+   * Generate next finite string.
+   * The return value is just valid until the next call of this method!
+   *
+   * @return Finite string or null, if no more finite strings are available.
+   */
+  public IntsRef next() {
+    // Special case the empty string, as usual:
+    if (emitEmptyString) {
+      emitEmptyString = false;
+      return EMPTY;
+    }
+
+    for (int depth = string.length(); depth > 0;) {
+      PathNode node = nodes[depth-1];
+
+      // Get next label leaving the current node:
+      int label = node.nextLabel(a);
+      if (label != -1) {
+        string.setIntAt(depth - 1, label);
+
+        int to = node.to;
+        if (a.getNumTransitions(to) != 0) {
+          // Now recurse: the destination of this transition has outgoing transitions:
+          if (pathStates.get(to)) {
+            throw new IllegalArgumentException("automaton has cycles");
+          }
+          pathStates.set(to);
+
+          // Push node onto stack:
+          growStack(depth);
+          nodes[depth].resetState(a, to);
+          depth++;
+          string.setLength(depth);
+          string.grow(depth);
+        } else if (a.isAccept(to)) {
+          // This transition leads to an accept state, so we save the current string:
+          return string.get();
+        }
+      } else {
+        // No more transitions leaving this state, pop/return back to previous state:
+        int state = node.state;
+        assert pathStates.get(state);
+        pathStates.clear(state);
+        depth--;
+        string.setLength(depth);
+
+        if (a.isAccept(state)) {
+          // This transition leads to an accept state, so we save the current string:
+          return string.get();
+        }
+      }
+    }
+
+    // Finished iteration.
+    return null;
+  }
+
+  /**
+   * Grow path stack, if required.
+   */
+  private void growStack(int depth) {
+    if (nodes.length == depth) {
+      PathNode[] newNodes = new PathNode[ArrayUtil.oversize(nodes.length + 1, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+      System.arraycopy(nodes, 0, newNodes, 0, nodes.length);
+      for (int i = depth, end = newNodes.length; i < end; i++) {
+        newNodes[i] = new PathNode();
+      }
+      nodes = newNodes;
+    }
+  }
+
+  /**
+   * Nodes for path stack.
+   */
+  private static class PathNode {
+
+    /** Which state the path node ends on, whose
+     *  transitions we are enumerating. */
+    public int state;
+
+    /** Which state the current transition leads to. */
+    public int to;
+
+    /** Which transition we are on. */
+    public int transition;
+
+    /** Which label we are on, in the min-max range of the
+     *  current Transition */
+    public int label;
+
+    private final Transition t = new Transition();
+
+    public void resetState(Automaton a, int state) {
+      assert a.getNumTransitions(state) != 0;
+      this.state = state;
+      transition = 0;
+      a.getTransition(state, 0, t);
+      label = t.min;
+      to = t.dest;
+    }
+
+    /** Returns next label of current transition, or
+     *  advances to next transition and returns its first
+     *  label, if current one is exhausted.  If there are
+     *  no more transitions, returns -1. */
+    public int nextLabel(Automaton a) {
+      if (label > t.max) {
+        // We've exhaused the current transition's labels;
+        // move to next transitions:
+        transition++;
+        if (transition >= a.getNumTransitions(state)) {
+          // We're done iterating transitions leaving this state
+          label = -1;
+          return -1;
+        }
+        a.getTransition(state, transition, t);
+        label = t.min;
+        to = t.dest;
+      }
+      return label++;
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/LimitedFiniteStringsIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/LimitedFiniteStringsIterator.java?rev=1689192&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/LimitedFiniteStringsIterator.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/LimitedFiniteStringsIterator.java Sat Jul  4 19:47:35 2015
@@ -0,0 +1,83 @@
+package org.apache.lucene.util.automaton;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.util.IntsRef;
+
+/**
+ * {@link FiniteStringsIterator} which limits the number of iterated accepted strings.
+ * If more than <code>limit</code> strings are accepted,
+ * the first <code>limit</code> strings found are returned.
+ *
+ * <p>If the {@link Automaton} has cycles then this iterator may throw an {@code
+ * IllegalArgumentException}, but this is not guaranteed!
+ *
+ * <p>Be aware that the iteration order is implementation dependent
+ * and may change across releases.
+ *
+ * @lucene.experimental
+ */
+public class LimitedFiniteStringsIterator extends FiniteStringsIterator {
+  /**
+   * Maximum number of finite strings to create.
+   */
+  private int limit = Integer.MAX_VALUE;
+
+  /**
+   * Number of generated finite strings.
+   */
+  private int count = 0;
+
+  /**
+   * Constructor.
+   *
+   * @param a Automaton to create finite string from.
+   * @param limit Maximum number of finite strings to create, or -1 for infinite.
+   */
+  public LimitedFiniteStringsIterator(Automaton a, int limit) {
+    super(a);
+
+    if (limit != -1 && limit <= 0) {
+      throw new IllegalArgumentException("limit must be -1 (which means no limit), or > 0; got: " + limit);
+    }
+
+    this.limit = limit > 0? limit : Integer.MAX_VALUE;
+  }
+
+  @Override
+  public IntsRef next() {
+    if (count >= limit) {
+      // Abort on limit.
+      return null;
+    }
+
+    IntsRef result = super.next();
+    if (result != null) {
+      count++;
+    }
+
+    return result;
+  }
+
+  /**
+   * Number of iterated finite strings.
+   */
+  public int size() {
+    return count;
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java?rev=1689192&r1=1689191&r2=1689192&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java Sat Jul  4 19:47:35 2015
@@ -1232,156 +1232,6 @@ final public class Operations {
     return result;
   }
 
-  private static class PathNode {
-
-    /** Which state the path node ends on, whose
-     *  transitions we are enumerating. */
-    public int state;
-
-    /** Which state the current transition leads to. */
-    public int to;
-
-    /** Which transition we are on. */
-    public int transition;
-
-    /** Which label we are on, in the min-max range of the
-     *  current Transition */
-    public int label;
-
-    private final Transition t = new Transition();
-
-    public void resetState(Automaton a, int state) {
-      assert a.getNumTransitions(state) != 0;
-      this.state = state;
-      transition = 0;
-      a.getTransition(state, 0, t);
-      label = t.min;
-      to = t.dest;
-    }
-
-    /** Returns next label of current transition, or
-     *  advances to next transition and returns its first
-     *  label, if current one is exhausted.  If there are
-     *  no more transitions, returns -1. */
-    public int nextLabel(Automaton a) {
-      if (label > t.max) {
-        // We've exhaused the current transition's labels;
-        // move to next transitions:
-        transition++;
-        if (transition >= a.getNumTransitions(state)) {
-          // We're done iterating transitions leaving this state
-          return -1;
-        }
-        a.getTransition(state, transition, t);
-        label = t.min;
-        to = t.dest;
-      }
-      return label++;
-    }
-  }
-
-  private static PathNode getNode(PathNode[] nodes, int index) {
-    assert index < nodes.length;
-    if (nodes[index] == null) {
-      nodes[index] = new PathNode();
-    }
-    return nodes[index];
-  }
-
-  // TODO: this is a dangerous method ... Automaton could be
-  // huge ... and it's better in general for caller to
-  // enumerate & process in a single walk:
-
-  /** Returns the set of accepted strings, up to at most
-   *  <code>limit</code> strings. If more than <code>limit</code> 
-   *  strings are accepted, the first limit strings found are returned. If <code>limit</code> == -1, then 
-   *  the limit is infinite.  If the {@link Automaton} has
-   *  cycles then this method might throw {@code
-   *  IllegalArgumentException} but that is not guaranteed
-   *  when the limit is set. */
-  public static Set<IntsRef> getFiniteStrings(Automaton a, int limit) {
-    Set<IntsRef> results = new HashSet<>();
-
-    if (limit == -1 || limit > 0) {
-      // OK
-    } else {
-      throw new IllegalArgumentException("limit must be -1 (which means no limit), or > 0; got: " + limit);
-    }
-
-    if (a.isAccept(0)) {
-      // Special case the empty string, as usual:
-      results.add(new IntsRef());
-    }
-
-    if (a.getNumTransitions(0) > 0 && (limit == -1 || results.size() < limit)) {
-
-      int numStates = a.getNumStates();
-
-      // Tracks which states are in the current path, for
-      // cycle detection:
-      BitSet pathStates = new BitSet(numStates);
-
-      // Stack to hold our current state in the
-      // recursion/iteration:
-      PathNode[] nodes = new PathNode[4];
-
-      pathStates.set(0);
-      PathNode root = getNode(nodes, 0);
-      root.resetState(a, 0);
-
-      IntsRefBuilder string = new IntsRefBuilder();
-      string.append(0);
-
-      while (string.length() > 0) {
-
-        PathNode node = nodes[string.length()-1];
-
-        // Get next label leaving the current node:
-        int label = node.nextLabel(a);
-
-        if (label != -1) {
-          string.setIntAt(string.length()-1, label);
-
-          if (a.isAccept(node.to)) {
-            // This transition leads to an accept state,
-            // so we save the current string:
-            results.add(string.toIntsRef());
-            if (results.size() == limit) {
-              break;
-            }
-          }
-
-          if (a.getNumTransitions(node.to) != 0) {
-            // Now recurse: the destination of this transition has
-            // outgoing transitions:
-            if (pathStates.get(node.to)) {
-              throw new IllegalArgumentException("automaton has cycles");
-            }
-            pathStates.set(node.to);
-
-            // Push node onto stack:
-            if (nodes.length == string.length()) {
-              PathNode[] newNodes = new PathNode[ArrayUtil.oversize(nodes.length+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-              System.arraycopy(nodes, 0, newNodes, 0, nodes.length);
-              nodes = newNodes;
-            }
-            getNode(nodes, string.length()).resetState(a, node.to);
-            string.setLength(string.length() + 1);
-            string.grow(string.length());
-          }
-        } else {
-          // No more transitions leaving this state,
-          // pop/return back to previous state:
-          assert pathStates.get(node.state);
-          pathStates.clear(node.state);
-          string.setLength(string.length() - 1);
-        }
-      }
-    }
-
-    return results;
-  }
-
   /** Returns a new automaton accepting the same language with added
    *  transitions to a dead state so that from every state and every label
    *  there is a transition. */

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/FiniteStringsIteratorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/FiniteStringsIteratorTest.java?rev=1689192&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/FiniteStringsIteratorTest.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/FiniteStringsIteratorTest.java Sat Jul  4 19:47:35 2015
@@ -0,0 +1,217 @@
+package org.apache.lucene.util.automaton;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.fst.Util;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
+
+/**
+ * Test for {@link FiniteStringsIterator}.
+ */
+public class FiniteStringsIteratorTest extends LuceneTestCase {
+  public void testRandomFiniteStrings1() {
+    int numStrings = atLeast(100);
+    if (VERBOSE) {
+      System.out.println("TEST: numStrings=" + numStrings);
+    }
+
+    Set<IntsRef> strings = new HashSet<>();
+    List<Automaton> automata = new ArrayList<>();
+    IntsRefBuilder scratch = new IntsRefBuilder();
+    for(int i=0;i<numStrings;i++) {
+      String s = TestUtil.randomSimpleString(random(), 1, 200);
+      automata.add(Automata.makeString(s));
+      Util.toUTF32(s.toCharArray(), 0, s.length(), scratch);
+      strings.add(scratch.toIntsRef());
+      if (VERBOSE) {
+        System.out.println("  add string=" + s);
+      }
+    }
+
+    // TODO: we could sometimes use
+    // DaciukMihovAutomatonBuilder here
+
+    // TODO: what other random things can we do here...
+    Automaton a = Operations.union(automata);
+    if (random().nextBoolean()) {
+      a = MinimizationOperations.minimize(a, 1000000);
+      if (VERBOSE) {
+        System.out.println("TEST: a.minimize numStates=" + a.getNumStates());
+      }
+    } else if (random().nextBoolean()) {
+      if (VERBOSE) {
+        System.out.println("TEST: a.determinize");
+      }
+      a = Operations.determinize(a, 1000000);
+    } else if (random().nextBoolean()) {
+      if (VERBOSE) {
+        System.out.println("TEST: a.removeDeadStates");
+      }
+      a = Operations.removeDeadStates(a);
+    }
+
+    FiniteStringsIterator iterator = new FiniteStringsIterator(a);
+    List<IntsRef> actual = getFiniteStrings(iterator);
+    assertFiniteStringsRecursive(a, actual);
+
+    if (!strings.equals(new HashSet<>(actual))) {
+      System.out.println("strings.size()=" + strings.size() + " actual.size=" + actual.size());
+      List<IntsRef> x = new ArrayList<>(strings);
+      Collections.sort(x);
+      List<IntsRef> y = new ArrayList<>(actual);
+      Collections.sort(y);
+      int end = Math.min(x.size(), y.size());
+      for(int i=0;i<end;i++) {
+        System.out.println("  i=" + i + " string=" + toString(x.get(i)) + " actual=" + toString(y.get(i)));
+      }
+      fail("wrong strings found");
+    }
+  }
+
+  /**
+   * Basic test for getFiniteStrings
+   */
+  public void testFiniteStringsBasic() {
+    Automaton a = Operations.union(Automata.makeString("dog"), Automata.makeString("duck"));
+    a = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
+    FiniteStringsIterator iterator = new FiniteStringsIterator(a);
+    List<IntsRef> actual = getFiniteStrings(iterator);
+    assertFiniteStringsRecursive(a, actual);
+    assertEquals(2, actual.size());
+    IntsRefBuilder dog = new IntsRefBuilder();
+    Util.toIntsRef(new BytesRef("dog"), dog);
+    assertTrue(actual.contains(dog.get()));
+    IntsRefBuilder duck = new IntsRefBuilder();
+    Util.toIntsRef(new BytesRef("duck"), duck);
+    assertTrue(actual.contains(duck.get()));
+  }
+
+  public void testFiniteStringsEatsStack() {
+    char[] chars = new char[50000];
+    TestUtil.randomFixedLengthUnicodeString(random(), chars, 0, chars.length);
+    String bigString1 = new String(chars);
+    TestUtil.randomFixedLengthUnicodeString(random(), chars, 0, chars.length);
+    String bigString2 = new String(chars);
+    Automaton a = Operations.union(Automata.makeString(bigString1), Automata.makeString(bigString2));
+    FiniteStringsIterator iterator = new FiniteStringsIterator(a);
+    List<IntsRef> actual = getFiniteStrings(iterator);
+    assertEquals(2, actual.size());
+    IntsRefBuilder scratch = new IntsRefBuilder();
+    Util.toUTF32(bigString1.toCharArray(), 0, bigString1.length(), scratch);
+    assertTrue(actual.contains(scratch.get()));
+    Util.toUTF32(bigString2.toCharArray(), 0, bigString2.length(), scratch);
+    assertTrue(actual.contains(scratch.get()));
+  }
+
+
+  public void testWithCycle() throws Exception {
+    try {
+      Automaton a = new RegExp("abc.*", RegExp.NONE).toAutomaton();
+      FiniteStringsIterator iterator = new FiniteStringsIterator(a);
+      getFiniteStrings(iterator);
+      fail("did not hit exception");
+    } catch (IllegalArgumentException iae) {
+      // expected
+    }
+  }
+
+  public void testSingletonNoLimit() {
+    Automaton a = Automata.makeString("foobar");
+    FiniteStringsIterator iterator = new FiniteStringsIterator(a);
+    List<IntsRef> actual = getFiniteStrings(iterator);
+    assertEquals(1, actual.size());
+    IntsRefBuilder scratch = new IntsRefBuilder();
+    Util.toUTF32("foobar".toCharArray(), 0, 6, scratch);
+    assertTrue(actual.contains(scratch.get()));
+  }
+
+  public void testShortAccept() {
+    Automaton a = Operations.union(Automata.makeString("x"), Automata.makeString("xy"));
+    a = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
+    FiniteStringsIterator iterator = new FiniteStringsIterator(a);
+    List<IntsRef> actual = getFiniteStrings(iterator);
+    assertEquals(2, actual.size());
+    IntsRefBuilder x = new IntsRefBuilder();
+    Util.toIntsRef(new BytesRef("x"), x);
+    assertTrue(actual.contains(x.get()));
+    IntsRefBuilder xy = new IntsRefBuilder();
+    Util.toIntsRef(new BytesRef("xy"), xy);
+    assertTrue(actual.contains(xy.get()));
+  }
+
+  public void testSingleString() {
+    Automaton a = new Automaton();
+    int start = a.createState();
+    int end = a.createState();
+    a.setAccept(end, true);
+    a.addTransition(start, end, 'a', 'a');
+    a.finishState();
+    Set<IntsRef> accepted = TestOperations.getFiniteStrings(a);
+    assertEquals(1, accepted.size());
+    IntsRefBuilder intsRef = new IntsRefBuilder();
+    intsRef.append('a');
+    assertTrue(accepted.contains(intsRef.toIntsRef()));
+  }
+
+  /**
+   * All strings generated by the iterator.
+   */
+  static List<IntsRef> getFiniteStrings(FiniteStringsIterator iterator) {
+    List<IntsRef> result = new ArrayList<>();
+    for (IntsRef finiteString; (finiteString = iterator.next()) != null;) {
+      result.add(IntsRef.deepCopyOf(finiteString));
+    }
+
+    return result;
+  }
+
+  /**
+   * Check that strings the automaton returns are as expected.
+   *
+   * @param automaton Automaton.
+   * @param actual Strings generated by automaton.
+   */
+  private void assertFiniteStringsRecursive(Automaton automaton, List<IntsRef> actual) {
+    Set<IntsRef> expected = AutomatonTestUtil.getFiniteStringsRecursive(automaton, -1);
+    // Check that no string is emitted twice.
+    assertEquals(expected.size(), actual.size());
+    assertEquals(expected, new HashSet<>(actual));
+  }
+
+  // ascii only!
+  private static String toString(IntsRef ints) {
+    BytesRef br = new BytesRef(ints.length);
+    for(int i=0;i<ints.length;i++) {
+      br.bytes[i] = (byte) ints.ints[i];
+    }
+    br.length = ints.length;
+    return br.utf8ToString();
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/LimitedFiniteStringsIteratorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/LimitedFiniteStringsIteratorTest.java?rev=1689192&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/LimitedFiniteStringsIteratorTest.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/LimitedFiniteStringsIteratorTest.java Sat Jul  4 19:47:35 2015
@@ -0,0 +1,101 @@
+package org.apache.lucene.util.automaton;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.List;
+
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.fst.Util;
+
+import static org.apache.lucene.util.automaton.FiniteStringsIteratorTest.getFiniteStrings;
+
+/**
+ * Test for {@link FiniteStringsIterator}.
+ */
+public class LimitedFiniteStringsIteratorTest extends LuceneTestCase {
+ public void testRandomFiniteStrings() {
+    // Just makes sure we can run on any random finite
+    // automaton:
+    int iters = atLeast(100);
+    for(int i=0;i<iters;i++) {
+      Automaton a = AutomatonTestUtil.randomAutomaton(random());
+      try {
+        // Must pass a limit because the random automaton
+        // can accept MANY strings:
+        getFiniteStrings(new LimitedFiniteStringsIterator(a, TestUtil.nextInt(random(), 1, 1000)));
+        // NOTE: cannot do this, because the method is not
+        // guaranteed to detect cycles when you have a limit
+        //assertTrue(Operations.isFinite(a));
+      } catch (IllegalArgumentException iae) {
+        assertFalse(Operations.isFinite(a));
+      }
+    }
+  }
+
+  public void testInvalidLimitNegative() {
+    Automaton a = AutomatonTestUtil.randomAutomaton(random());
+    try {
+      new LimitedFiniteStringsIterator(a, -7);
+      fail("did not hit exception");
+    } catch (IllegalArgumentException iae) {
+      // expected
+    }
+  }
+
+  public void testInvalidLimitNull() {
+    Automaton a = AutomatonTestUtil.randomAutomaton(random());
+    try {
+      new LimitedFiniteStringsIterator(a, 0);
+      fail("did not hit exception");
+    } catch (IllegalArgumentException iae) {
+      // expected
+    }
+  }
+
+  public void testSingleton() {
+    Automaton a = Automata.makeString("foobar");
+    List<IntsRef> actual = getFiniteStrings(new LimitedFiniteStringsIterator(a, 1));
+    assertEquals(1, actual.size());
+    IntsRefBuilder scratch = new IntsRefBuilder();
+    Util.toUTF32("foobar".toCharArray(), 0, 6, scratch);
+    assertTrue(actual.contains(scratch.get()));
+  }
+
+  public void testLimit() {
+    Automaton a = Operations.union(Automata.makeString("foo"), Automata.makeString("bar"));
+
+    // Test without limit
+    FiniteStringsIterator withoutLimit = new LimitedFiniteStringsIterator(a, -1);
+    assertEquals(2, getFiniteStrings(withoutLimit).size());
+
+    // Test with limit
+    FiniteStringsIterator withLimit = new LimitedFiniteStringsIterator(a, 1);
+    assertEquals(1, getFiniteStrings(withLimit).size());
+  }
+
+  public void testSize() {
+    Automaton a = Operations.union(Automata.makeString("foo"), Automata.makeString("bar"));
+    LimitedFiniteStringsIterator iterator = new LimitedFiniteStringsIterator(a, -1);
+    List<IntsRef> actual = getFiniteStrings(iterator);
+    assertEquals(2, actual.size());
+    assertEquals(2, iterator.size());
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java?rev=1689192&r1=1689191&r2=1689192&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java Sat Jul  4 19:47:35 2015
@@ -505,8 +505,8 @@ public class TestAutomaton extends Lucen
       expected.add(Util.toUTF32(s, ints));
     }
 
-    assertEquals(expected, Operations.getFiniteStrings(Operations.determinize(a,
-      DEFAULT_MAX_DETERMINIZED_STATES), -1)); 
+    assertEquals(expected, TestOperations.getFiniteStrings(
+        Operations.determinize(a, DEFAULT_MAX_DETERMINIZED_STATES)));
   }
 
   public void testConcatenatePreservesDet() throws Exception {
@@ -552,11 +552,11 @@ public class TestAutomaton extends Lucen
     // If you concat empty automaton to anything the result should still be empty:
     Automaton a = Operations.concatenate(Automata.makeEmpty(),
                                                         Automata.makeString("foo"));
-    assertEquals(new HashSet<IntsRef>(), Operations.getFiniteStrings(a, -1));
+    assertEquals(new HashSet<IntsRef>(), TestOperations.getFiniteStrings(a));
 
     a = Operations.concatenate(Automata.makeString("foo"),
                                          Automata.makeEmpty());
-    assertEquals(new HashSet<IntsRef>(), Operations.getFiniteStrings(a, -1));
+    assertEquals(new HashSet<IntsRef>(), TestOperations.getFiniteStrings(a));
   }
 
   public void testSeemsNonEmptyButIsNot1() throws Exception {
@@ -1097,7 +1097,7 @@ public class TestAutomaton extends Lucen
         Util.toUTF32(term.utf8ToString(), intsRef);
         expected.add(intsRef.toIntsRef());
       }
-      Set<IntsRef> actual = Operations.getFiniteStrings(a, -1);
+      Set<IntsRef> actual = TestOperations.getFiniteStrings(a);
 
       if (expected.equals(actual) == false) {
         System.out.println("FAILED:");
@@ -1129,7 +1129,7 @@ public class TestAutomaton extends Lucen
         Util.toIntsRef(term, intsRef);
         expected2.add(intsRef.toIntsRef());
       }
-      assertEquals(expected2, Operations.getFiniteStrings(utf8, -1));
+      assertEquals(expected2, TestOperations.getFiniteStrings(utf8));
     } catch (AssertionError ae) {
       System.out.println("TEST: FAILED: not same");
       System.out.println("  terms (count=" + terms.size() + "):");
@@ -1259,7 +1259,7 @@ public class TestAutomaton extends Lucen
         continue;
       } else {
         // Enumerate all finite strings and verify the count matches what we expect:
-        assertEquals(expectedCount, Operations.getFiniteStrings(a, expectedCount).size());
+        assertEquals(expectedCount, TestOperations.getFiniteStrings(a, expectedCount).size());
       }
 
       b = new BytesRefBuilder();
@@ -1339,7 +1339,7 @@ public class TestAutomaton extends Lucen
     Automaton a = Automata.makeBinaryInterval(new BytesRef("bar"), true, new BytesRef("bar"), true);
     assertTrue(Operations.run(a, intsRef("bar")));
     assertTrue(Operations.isFinite(a));
-    assertEquals(1, Operations.getFiniteStrings(a, 10).size());
+    assertEquals(1, TestOperations.getFiniteStrings(a).size());
   }
 
   public void testMakeBinaryIntervalCommonPrefix() throws Exception {

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestOperations.java?rev=1689192&r1=1689191&r2=1689192&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestOperations.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestOperations.java Sat Jul  4 19:47:35 2015
@@ -20,7 +20,6 @@ package org.apache.lucene.util.automaton
 import java.util.*;
 
 import org.apache.lucene.util.*;
-import org.apache.lucene.util.fst.Util;
 
 import com.carrotsearch.randomizedtesting.generators.RandomInts;
 
@@ -125,176 +124,39 @@ public class TestOperations extends Luce
     }
   }
 
-  /** Pass false for testRecursive if the expected strings
-   *  may be too long */
-  private Set<IntsRef> getFiniteStrings(Automaton a, int limit, boolean testRecursive) {
-    Set<IntsRef> result = Operations.getFiniteStrings(a, limit);
-    if (testRecursive) {
-      assertEquals(AutomatonTestUtil.getFiniteStringsRecursive(a, limit), result);
-    }
-    return result;
-  }
-  
   /**
-   * Basic test for getFiniteStrings
+   * Returns the set of all accepted strings.
+   *
+   * This method exist just to ease testing.
+   * For production code directly use {@link FiniteStringsIterator} instead.
+   *
+   * @see FiniteStringsIterator
    */
-  public void testFiniteStringsBasic() {
-    Automaton a = Operations.union(Automata.makeString("dog"), Automata.makeString("duck"));
-    a = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
-    Set<IntsRef> strings = getFiniteStrings(a, -1, true);
-    assertEquals(2, strings.size());
-    IntsRefBuilder dog = new IntsRefBuilder();
-    Util.toIntsRef(new BytesRef("dog"), dog);
-    assertTrue(strings.contains(dog.get()));
-    IntsRefBuilder duck = new IntsRefBuilder();
-    Util.toIntsRef(new BytesRef("duck"), duck);
-    assertTrue(strings.contains(duck.get()));
-  }
-
-  public void testFiniteStringsEatsStack() {
-    char[] chars = new char[50000];
-    TestUtil.randomFixedLengthUnicodeString(random(), chars, 0, chars.length);
-    String bigString1 = new String(chars);
-    TestUtil.randomFixedLengthUnicodeString(random(), chars, 0, chars.length);
-    String bigString2 = new String(chars);
-    Automaton a = Operations.union(Automata.makeString(bigString1), Automata.makeString(bigString2));
-    Set<IntsRef> strings = getFiniteStrings(a, -1, false);
-    assertEquals(2, strings.size());
-    IntsRefBuilder scratch = new IntsRefBuilder();
-    Util.toUTF32(bigString1.toCharArray(), 0, bigString1.length(), scratch);
-    assertTrue(strings.contains(scratch.get()));
-    Util.toUTF32(bigString2.toCharArray(), 0, bigString2.length(), scratch);
-    assertTrue(strings.contains(scratch.get()));
-  }
-
-  public void testRandomFiniteStrings1() {
-
-    int numStrings = atLeast(100);
-    if (VERBOSE) {
-      System.out.println("TEST: numStrings=" + numStrings);
-    }
-
-    Set<IntsRef> strings = new HashSet<IntsRef>();
-    List<Automaton> automata = new ArrayList<>();
-    IntsRefBuilder scratch = new IntsRefBuilder();
-    for(int i=0;i<numStrings;i++) {
-      String s = TestUtil.randomSimpleString(random(), 1, 200);
-      automata.add(Automata.makeString(s));
-      Util.toUTF32(s.toCharArray(), 0, s.length(), scratch);
-      strings.add(scratch.toIntsRef());
-      if (VERBOSE) {
-        System.out.println("  add string=" + s);
-      }
-    }
-
-    // TODO: we could sometimes use
-    // DaciukMihovAutomatonBuilder here
-
-    // TODO: what other random things can we do here...
-    Automaton a = Operations.union(automata);
-    if (random().nextBoolean()) {
-      a = MinimizationOperations.minimize(a, 1000000);
-      if (VERBOSE) {
-        System.out.println("TEST: a.minimize numStates=" + a.getNumStates());
-      }
-    } else if (random().nextBoolean()) {
-      if (VERBOSE) {
-        System.out.println("TEST: a.determinize");
-      }
-      a = Operations.determinize(a, 1000000);
-    } else if (random().nextBoolean()) {
-      if (VERBOSE) {
-        System.out.println("TEST: a.removeDeadStates");
-      }
-      a = Operations.removeDeadStates(a);
-    }
-
-    Set<IntsRef> actual = getFiniteStrings(a, -1, true);
-    if (strings.equals(actual) == false) {
-      System.out.println("strings.size()=" + strings.size() + " actual.size=" + actual.size());
-      List<IntsRef> x = new ArrayList<>(strings);
-      Collections.sort(x);
-      List<IntsRef> y = new ArrayList<>(actual);
-      Collections.sort(y);
-      int end = Math.min(x.size(), y.size());
-      for(int i=0;i<end;i++) {
-        System.out.println("  i=" + i + " string=" + toString(x.get(i)) + " actual=" + toString(y.get(i)));
-      }
-      fail("wrong strings found");
-    }
-  }
-
-  // ascii only!
-  private static String toString(IntsRef ints) {
-    BytesRef br = new BytesRef(ints.length);
-    for(int i=0;i<ints.length;i++) {
-      br.bytes[i] = (byte) ints.ints[i];
-    }
-    br.length = ints.length;
-    return br.utf8ToString();
+  public static Set<IntsRef> getFiniteStrings(Automaton a) {
+    return getFiniteStrings(new FiniteStringsIterator(a));
   }
 
-  public void testWithCycle() throws Exception {
-    try {
-      Operations.getFiniteStrings(new RegExp("abc.*", RegExp.NONE).toAutomaton(), -1);
-      fail("did not hit exception");
-    } catch (IllegalArgumentException iae) {
-      // expected
-    }
-  }
-
-  public void testRandomFiniteStrings2() {
-    // Just makes sure we can run on any random finite
-    // automaton:
-    int iters = atLeast(100);
-    for(int i=0;i<iters;i++) {
-      Automaton a = AutomatonTestUtil.randomAutomaton(random());
-      try {
-        // Must pass a limit because the random automaton
-        // can accept MANY strings:
-        Operations.getFiniteStrings(a, TestUtil.nextInt(random(), 1, 1000));
-        // NOTE: cannot do this, because the method is not
-        // guaranteed to detect cycles when you have a limit
-        //assertTrue(Operations.isFinite(a));
-      } catch (IllegalArgumentException iae) {
-        assertFalse(Operations.isFinite(a));
-      }
-    }
-  }
-
-  public void testInvalidLimit() {
-    Automaton a = AutomatonTestUtil.randomAutomaton(random());
-    try {
-      Operations.getFiniteStrings(a, -7);
-      fail("did not hit exception");
-    } catch (IllegalArgumentException iae) {
-      // expected
-    }
+  /**
+   * Returns the set of accepted strings, up to at most <code>limit</code> strings.
+   *
+   * This method exist just to ease testing.
+   * For production code directly use {@link LimitedFiniteStringsIterator} instead.
+   *
+   * @see LimitedFiniteStringsIterator
+   */
+  public static Set<IntsRef> getFiniteStrings(Automaton a, int limit) {
+    return getFiniteStrings(new LimitedFiniteStringsIterator(a, limit));
   }
 
-  public void testInvalidLimit2() {
-    Automaton a = AutomatonTestUtil.randomAutomaton(random());
-    try {
-      Operations.getFiniteStrings(a, 0);
-      fail("did not hit exception");
-    } catch (IllegalArgumentException iae) {
-      // expected
+  /**
+   * Get all finite strings of an iterator.
+   */
+  private static Set<IntsRef> getFiniteStrings(FiniteStringsIterator iterator) {
+    Set<IntsRef> result = new HashSet<>();
+    for (IntsRef finiteString; (finiteString = iterator.next()) != null;) {
+      result.add(IntsRef.deepCopyOf(finiteString));
     }
-  }
 
-  public void testSingletonNoLimit() {
-    Set<IntsRef> result = Operations.getFiniteStrings(Automata.makeString("foobar"), -1);
-    assertEquals(1, result.size());
-    IntsRefBuilder scratch = new IntsRefBuilder();
-    Util.toUTF32("foobar".toCharArray(), 0, 6, scratch);
-    assertTrue(result.contains(scratch.get()));
-  }
-
-  public void testSingletonLimit1() {
-    Set<IntsRef> result = Operations.getFiniteStrings(Automata.makeString("foobar"), 1);
-    assertEquals(1, result.size());
-    IntsRefBuilder scratch = new IntsRefBuilder();
-    Util.toUTF32("foobar".toCharArray(), 0, 6, scratch);
-    assertTrue(result.contains(scratch.get()));
+    return result;
   }
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java?rev=1689192&r1=1689191&r2=1689192&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java Sat Jul  4 19:47:35 2015
@@ -218,7 +218,7 @@ public class TestUTF32ToUTF8 extends Luc
       Util.toIntsRef(new BytesRef(s), ints);
       Set<IntsRef> set = new HashSet<>();
       set.add(ints.get());
-      assertEquals(set, Operations.getFiniteStrings(utf8, -1));
+      assertEquals(set, TestOperations.getFiniteStrings(utf8));
     }
   }
   

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java?rev=1689192&r1=1689191&r2=1689192&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java Sat Jul  4 19:47:35 2015
@@ -50,6 +50,7 @@ import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.OfflineSorter;
 import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.LimitedFiniteStringsIterator;
 import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 import org.apache.lucene.util.fst.Builder;
@@ -413,16 +414,13 @@ public class AnalyzingSuggester extends
     byte buffer[] = new byte[8];
     try {
       ByteArrayDataOutput output = new ByteArrayDataOutput(buffer);
-      BytesRef surfaceForm;
 
-      while ((surfaceForm = iterator.next()) != null) {
-        Set<IntsRef> paths = toFiniteStrings(surfaceForm, ts2a);
-        
-        maxAnalyzedPathsForOneInput = Math.max(maxAnalyzedPathsForOneInput, paths.size());
-
-        for (IntsRef path : paths) {
+      for (BytesRef surfaceForm; (surfaceForm = iterator.next()) != null;) {
+        LimitedFiniteStringsIterator finiteStrings =
+            new LimitedFiniteStringsIterator(toAutomaton(surfaceForm, ts2a), maxGraphExpansions);
 
-          Util.toBytesRef(path, scratch);
+        for (IntsRef string; (string = finiteStrings.next()) != null; count++) {
+          Util.toBytesRef(string, scratch);
           
           // length of the analyzed text (FST input)
           if (scratch.length() > Short.MAX_VALUE-2) {
@@ -473,7 +471,8 @@ public class AnalyzingSuggester extends
           assert output.getPosition() == requiredLength: output.getPosition() + " vs " + requiredLength;
           writer.write(buffer, 0, output.getPosition());
         }
-        count++;
+
+        maxAnalyzedPathsForOneInput = Math.max(maxAnalyzedPathsForOneInput, finiteStrings.size());
       }
       writer.close();
 
@@ -833,9 +832,9 @@ public class AnalyzingSuggester extends
     return prefixPaths;
   }
   
-  final Set<IntsRef> toFiniteStrings(final BytesRef surfaceForm, final TokenStreamToAutomaton ts2a) throws IOException {
+  final Automaton toAutomaton(final BytesRef surfaceForm, final TokenStreamToAutomaton ts2a) throws IOException {
     // Analyze surface form:
-    Automaton automaton = null;
+    Automaton automaton;
     try (TokenStream ts = indexAnalyzer.tokenStream("", surfaceForm.utf8ToString())) {
 
       // Create corresponding automaton: labels are bytes
@@ -853,12 +852,7 @@ public class AnalyzingSuggester extends
     // Get all paths from the automaton (there can be
     // more than one path, eg if the analyzer created a
     // graph using SynFilter or WDF):
-
-    // TODO: we could walk & add simultaneously, so we
-    // don't have to alloc [possibly biggish]
-    // intermediate HashSet in RAM:
-
-    return Operations.getFiniteStrings(automaton, maxGraphExpansions);
+    return automaton;
   }
 
   final Automaton toLookupAutomaton(final CharSequence key) throws IOException {

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java?rev=1689192&r1=1689191&r2=1689192&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java Sat Jul  4 19:47:35 2015
@@ -17,9 +17,8 @@ package org.apache.lucene.search.suggest
  */
 
 import java.io.IOException;
-import java.util.Arrays;
+import java.util.ArrayList;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
@@ -30,6 +29,7 @@ import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.automaton.Automata;
 import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.FiniteStringsIterator;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
 import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.UTF32ToUTF8;
@@ -221,37 +221,34 @@ public final class FuzzySuggester extend
   }
 
   Automaton toLevenshteinAutomata(Automaton automaton) {
-    final Set<IntsRef> ref = Operations.getFiniteStrings(automaton, -1);
-    Automaton subs[] = new Automaton[ref.size()];
-    int upto = 0;
-    for (IntsRef path : ref) {
-      if (path.length <= nonFuzzyPrefix || path.length < minFuzzyLength) {
-        subs[upto] = Automata.makeString(path.ints, path.offset, path.length);
-        upto++;
+    List<Automaton> subs = new ArrayList<>();
+    FiniteStringsIterator finiteStrings = new FiniteStringsIterator(automaton);
+    for (IntsRef string; (string = finiteStrings.next()) != null;) {
+      if (string.length <= nonFuzzyPrefix || string.length < minFuzzyLength) {
+        subs.add(Automata.makeString(string.ints, string.offset, string.length));
       } else {
-        int ints[] = new int[path.length-nonFuzzyPrefix];
-        System.arraycopy(path.ints, path.offset+nonFuzzyPrefix, ints, 0, ints.length);
+        int ints[] = new int[string.length-nonFuzzyPrefix];
+        System.arraycopy(string.ints, string.offset+nonFuzzyPrefix, ints, 0, ints.length);
         // TODO: maybe add alphaMin to LevenshteinAutomata,
         // and pass 1 instead of 0?  We probably don't want
         // to allow the trailing dedup bytes to be
         // edited... but then 0 byte is "in general" allowed
         // on input (but not in UTF8).
         LevenshteinAutomata lev = new LevenshteinAutomata(ints, unicodeAware ? Character.MAX_CODE_POINT : 255, transpositions);
-        subs[upto] = lev.toAutomaton(maxEdits, UnicodeUtil.newString(path.ints, path.offset, nonFuzzyPrefix));
-        upto++;
+        subs.add(lev.toAutomaton(maxEdits, UnicodeUtil.newString(string.ints, string.offset, nonFuzzyPrefix)));
       }
     }
 
-    if (subs.length == 0) {
+    if (subs.isEmpty()) {
       // automaton is empty, there is no accepted paths through it
       return Automata.makeEmpty(); // matches nothing
-    } else if (subs.length == 1) {
+    } else if (subs.size() == 1) {
       // no synonyms or anything: just a single path through the tokenstream
-      return subs[0];
+      return subs.get(0);
     } else {
       // multiple paths: this is really scary! is it slow?
       // maybe we should not do this and throw UOE?
-      Automaton a = Operations.union(Arrays.asList(subs));
+      Automaton a = Operations.union(subs);
       // TODO: we could call toLevenshteinAutomata() before det? 
       // this only happens if you have multiple paths anyway (e.g. synonyms)
       return Operations.determinize(a, DEFAULT_MAX_DETERMINIZED_STATES);

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java?rev=1689192&r1=1689191&r2=1689192&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java Sat Jul  4 19:47:35 2015
@@ -18,14 +18,11 @@ package org.apache.lucene.search.suggest
  */
 
 import java.io.IOException;
-import java.util.Iterator;
-import java.util.Set;
 
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.TokenStreamToAutomaton;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
-import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.util.AttributeImpl;
 import org.apache.lucene.util.AttributeReflector;
@@ -35,6 +32,8 @@ import org.apache.lucene.util.CharsRefBu
 import org.apache.lucene.util.IOUtils;
 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.LimitedFiniteStringsIterator;
 import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 import org.apache.lucene.util.fst.Util;
@@ -56,7 +55,6 @@ import static org.apache.lucene.search.s
 public final class CompletionTokenStream extends TokenStream {
 
   private final PayloadAttribute payloadAttr = addAttribute(PayloadAttribute.class);
-  private final PositionIncrementAttribute posAttr = addAttribute(PositionIncrementAttribute.class);
   private final BytesRefBuilderTermAttribute bytesAtt = addAttribute(BytesRefBuilderTermAttribute.class);
 
   private final TokenStream input;
@@ -64,9 +62,8 @@ public final class CompletionTokenStream
   final boolean preservePositionIncrements;
   final int maxGraphExpansions;
 
+  private FiniteStringsIterator finiteStrings;
   private BytesRef payload;
-  private Iterator<IntsRef> finiteStrings;
-  private int posInc = -1;
   private CharTermAttribute charTermAttribute;
 
   /**
@@ -100,45 +97,38 @@ public final class CompletionTokenStream
   public boolean incrementToken() throws IOException {
     clearAttributes();
     if (finiteStrings == null) {
-      //TODO: make this return a Iterator<IntsRef> instead?
       Automaton automaton = toAutomaton();
-      Set<IntsRef> strings = Operations.getFiniteStrings(automaton, maxGraphExpansions);
+      finiteStrings = new LimitedFiniteStringsIterator(automaton, maxGraphExpansions);
+    }
+
+    IntsRef string = finiteStrings.next();
+    if (string == null) {
+      return false;
+    }
 
-      posInc = strings.size();
-      finiteStrings = strings.iterator();
+    Util.toBytesRef(string, bytesAtt.builder()); // now we have UTF-8
+    if (charTermAttribute != null) {
+      charTermAttribute.setLength(0);
+      charTermAttribute.append(bytesAtt.toUTF16());
     }
-    if (finiteStrings.hasNext()) {
-      posAttr.setPositionIncrement(posInc);
-      /*
-       * this posInc encodes the number of paths that this surface form
-       * produced. Multi Fields have the same surface form and therefore sum up
-       */
-      posInc = 0;
-      Util.toBytesRef(finiteStrings.next(), bytesAtt.builder()); // now we have UTF-8
-      if (charTermAttribute != null) {
-        charTermAttribute.setLength(0);
-        charTermAttribute.append(bytesAtt.toUTF16());
-      }
-      if (payload != null) {
-        payloadAttr.setPayload(this.payload);
-      }
-      return true;
+    if (payload != null) {
+      payloadAttr.setPayload(this.payload);
     }
 
-    return false;
+    return true;
   }
 
   @Override
   public void end() throws IOException {
     super.end();
-    if (posInc == -1) {
+    if (finiteStrings == null) {
       input.end();
     }
   }
 
   @Override
   public void close() throws IOException {
-    if (posInc == -1) {
+    if (finiteStrings == null) {
       input.close();
     }
   }
@@ -151,7 +141,6 @@ public final class CompletionTokenStream
       charTermAttribute = getAttribute(CharTermAttribute.class);
     }
     finiteStrings = null;
-    posInc = -1;
   }
 
   /**

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java?rev=1689192&r1=1689191&r2=1689192&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java Sat Jul  4 19:47:35 2015
@@ -18,7 +18,9 @@ package org.apache.lucene.search.suggest
  */
 
 import java.io.IOException;
-import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -30,6 +32,7 @@ import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.automaton.Automata;
 import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.FiniteStringsIterator;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
 import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.UTF32ToUTF8;
@@ -142,28 +145,29 @@ public class FuzzyCompletionQuery extend
   @Override
   public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
     CompletionTokenStream stream = (CompletionTokenStream) analyzer.tokenStream(getField(), getTerm().text());
-    Automaton a = stream.toAutomaton(unicodeAware);
-    final Set<IntsRef> refs = Operations.getFiniteStrings(a, -1);
-    assert refs.size() > 0;
-    Automaton automaton = toLevenshteinAutomata(refs);
+    Set<IntsRef> refs = new HashSet<>();
+    Automaton automaton = toLevenshteinAutomata(stream.toAutomaton(unicodeAware), refs);
     if (unicodeAware) {
       Automaton utf8automaton = new UTF32ToUTF8().convert(automaton);
       utf8automaton = Operations.determinize(utf8automaton, maxDeterminizedStates);
       automaton = utf8automaton;
     }
+    // TODO Accumulating all refs is bad, because the resulting set may be very big.
+    // TODO Better iterate over automaton again inside FuzzyCompletionWeight?
     return new FuzzyCompletionWeight(this, automaton, refs);
   }
 
-  private Automaton toLevenshteinAutomata(Set<IntsRef> ref) {
-    Automaton subs[] = new Automaton[ref.size()];
-    int upto = 0;
-    for (IntsRef path : ref) {
-      if (path.length <= nonFuzzyPrefix || path.length < minFuzzyLength) {
-        subs[upto] = Automata.makeString(path.ints, path.offset, path.length);
-        upto++;
+  private Automaton toLevenshteinAutomata(Automaton automaton, Set<IntsRef> refs) {
+    List<Automaton> subs = new ArrayList<>();
+    FiniteStringsIterator finiteStrings = new FiniteStringsIterator(automaton);
+    for (IntsRef string; (string = finiteStrings.next()) != null;) {
+      refs.add(IntsRef.deepCopyOf(string));
+
+      if (string.length <= nonFuzzyPrefix || string.length < minFuzzyLength) {
+        subs.add(Automata.makeString(string.ints, string.offset, string.length));
       } else {
-        int ints[] = new int[path.length - nonFuzzyPrefix];
-        System.arraycopy(path.ints, path.offset + nonFuzzyPrefix, ints, 0, ints.length);
+        int ints[] = new int[string.length - nonFuzzyPrefix];
+        System.arraycopy(string.ints, string.offset + nonFuzzyPrefix, ints, 0, ints.length);
         // TODO: maybe add alphaMin to LevenshteinAutomata,
         // and pass 1 instead of 0?  We probably don't want
         // to allow the trailing dedup bytes to be
@@ -172,22 +176,21 @@ public class FuzzyCompletionQuery extend
         LevenshteinAutomata lev = new LevenshteinAutomata(ints,
             unicodeAware ? Character.MAX_CODE_POINT : 255,
             transpositions);
-        subs[upto] = lev.toAutomaton(maxEdits,
-            UnicodeUtil.newString(path.ints, path.offset, nonFuzzyPrefix));
-        upto++;
+        subs.add(lev.toAutomaton(maxEdits,
+            UnicodeUtil.newString(string.ints, string.offset, nonFuzzyPrefix)));
       }
     }
 
-    if (subs.length == 0) {
+    if (subs.isEmpty()) {
       // automaton is empty, there is no accepted paths through it
       return Automata.makeEmpty(); // matches nothing
-    } else if (subs.length == 1) {
+    } else if (subs.size() == 1) {
       // no synonyms or anything: just a single path through the tokenstream
-      return subs[0];
+      return subs.get(0);
     } else {
       // multiple paths: this is really scary! is it slow?
       // maybe we should not do this and throw UOE?
-      Automaton a = Operations.union(Arrays.asList(subs));
+      Automaton a = Operations.union(subs);
       // TODO: we could call toLevenshteinAutomata() before det?
       // this only happens if you have multiple paths anyway (e.g. synonyms)
       return Operations.determinize(a, maxDeterminizedStates);

Modified: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java?rev=1689192&r1=1689191&r2=1689192&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java Sat Jul  4 19:47:35 2015
@@ -49,6 +49,7 @@ import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.FiniteStringsIterator;
 import org.apache.lucene.util.fst.Util;
 
 public class FuzzySuggesterTest extends LuceneTestCase {
@@ -773,10 +774,11 @@ public class FuzzySuggesterTest extends
       BytesRefBuilder spare = new BytesRefBuilder();
       for (TermFreqPayload2 e : slowCompletor) {
         spare.copyChars(e.analyzedForm);
-        Set<IntsRef> finiteStrings = suggester.toFiniteStrings(spare.get(), tokenStreamToAutomaton);
-        for (IntsRef intsRef : finiteStrings) {
+        FiniteStringsIterator finiteStrings =
+            new FiniteStringsIterator(suggester.toAutomaton(spare.get(), tokenStreamToAutomaton));
+        for (IntsRef string; (string = finiteStrings.next()) != null;) {
           int p = 0;
-          BytesRef ref = Util.toBytesRef(intsRef, spare);
+          BytesRef ref = Util.toBytesRef(string, spare);
           boolean added = false;
           for (int i = ref.offset; i < ref.length; i++) {
             int q = automaton.step(p, ref.bytes[i] & 0xff);