LUCENE-5752: merge trunk

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5752@1603492 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestDuelingAnalyzers.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestDuelingAnalyzers.java
index 7c4334b..8d97f33 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestDuelingAnalyzers.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestDuelingAnalyzers.java
@@ -31,11 +31,9 @@
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.BasicOperations;
+import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
-import org.apache.lucene.util.automaton.State;
-import org.apache.lucene.util.automaton.Transition;
+import org.apache.lucene.util.automaton.Automaton;
 
 /**
  * Compares MockTokenizer (which is simple with no optimizations) with equivalent 
@@ -50,18 +48,18 @@
   @Override
   public void setUp() throws Exception {
     super.setUp();
+    Automaton single = new Automaton();
+    int initial = single.createState();
+    int accept = single.createState();
+    single.setAccept(accept, true);
+
     // build an automaton matching this jvm's letter definition
-    State initial = new State();
-    State accept = new State();
-    accept.setAccept(true);
     for (int i = 0; i <= 0x10FFFF; i++) {
       if (Character.isLetter(i)) {
-        initial.addTransition(new Transition(i, i, accept));
+        single.addTransition(initial, accept, i);
       }
     }
-    Automaton single = new Automaton(initial);
-    single.reduce();
-    Automaton repeat = BasicOperations.repeat(single);
+    Automaton repeat = Operations.repeat(single);
     jvmLetter = new CharacterRunAutomaton(repeat);
   }
   
diff --git a/lucene/build.xml b/lucene/build.xml
index 39cf296..280800b 100644
--- a/lucene/build.xml
+++ b/lucene/build.xml
@@ -258,6 +258,7 @@
     <!-- test-framework: problems -->
 
     <!-- too much to fix core/ for now, but enforce full javadocs for key packages -->
+    <check-missing-javadocs dir="build/docs/core/org/apache/lucene/util/automaton" level="method"/>
     <check-missing-javadocs dir="build/docs/core/org/apache/lucene/analysis" level="method"/>
     <check-missing-javadocs dir="build/docs/core/org/apache/lucene/document" level="method"/>
     <check-missing-javadocs dir="build/docs/core/org/apache/lucene/search/similarities" level="method"/>
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
index 149a63d..e8d5d62 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
@@ -29,8 +29,8 @@
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.OrdTermState;
 import org.apache.lucene.index.SegmentReadState;
@@ -940,10 +940,11 @@
       private final class State {
         int changeOrd;
         int state;
-        Transition[] transitions;
         int transitionUpto;
+        int transitionCount;
         int transitionMax;
         int transitionMin;
+        final Transition transition = new Transition();
       }
 
       private State[] states;
@@ -957,7 +958,8 @@
         states[0] = new State();
         states[0].changeOrd = terms.length;
         states[0].state = runAutomaton.getInitialState();
-        states[0].transitions = compiledAutomaton.sortedTransitions[states[0].state];
+        states[0].transitionCount = compiledAutomaton.automaton.getNumTransitions(states[0].state);
+        compiledAutomaton.automaton.initTransition(states[0].state, states[0].transition);
         states[0].transitionUpto = -1;
         states[0].transitionMax = -1;
 
@@ -978,9 +980,10 @@
 
               while (label > states[i].transitionMax) {
                 states[i].transitionUpto++;
-                assert states[i].transitionUpto < states[i].transitions.length;
-                states[i].transitionMin = states[i].transitions[states[i].transitionUpto].getMin();
-                states[i].transitionMax = states[i].transitions[states[i].transitionUpto].getMax();
+                assert states[i].transitionUpto < states[i].transitionCount;
+                compiledAutomaton.automaton.getNextTransition(states[i].transition);
+                states[i].transitionMin = states[i].transition.min;
+                states[i].transitionMax = states[i].transition.max;
                 assert states[i].transitionMin >= 0;
                 assert states[i].transitionMin <= 255;
                 assert states[i].transitionMax >= 0;
@@ -1037,7 +1040,8 @@
                     stateUpto++;
                     states[stateUpto].changeOrd = skips[skipOffset + skipUpto++];
                     states[stateUpto].state = nextState;
-                    states[stateUpto].transitions = compiledAutomaton.sortedTransitions[nextState];
+                    states[stateUpto].transitionCount = compiledAutomaton.automaton.getNumTransitions(nextState);
+                    compiledAutomaton.automaton.initTransition(states[stateUpto].state, states[stateUpto].transition);
                     states[stateUpto].transitionUpto = -1;
                     states[stateUpto].transitionMax = -1;
                     //System.out.println("  push " + states[stateUpto].transitions.length + " trans");
@@ -1191,7 +1195,7 @@
           while (label > state.transitionMax) {
             //System.out.println("  label=" + label + " vs max=" + state.transitionMax + " transUpto=" + state.transitionUpto + " vs " + state.transitions.length);
             state.transitionUpto++;
-            if (state.transitionUpto == state.transitions.length) {
+            if (state.transitionUpto == state.transitionCount) {
               // We've exhausted transitions leaving this
               // state; force pop+next/skip now:
               //System.out.println("forcepop: stateUpto=" + stateUpto);
@@ -1210,9 +1214,10 @@
               }
               continue nextTerm;
             }
-            assert state.transitionUpto < state.transitions.length: " state.transitionUpto=" + state.transitionUpto + " vs " + state.transitions.length;
-            state.transitionMin = state.transitions[state.transitionUpto].getMin();
-            state.transitionMax = state.transitions[state.transitionUpto].getMax();
+            compiledAutomaton.automaton.getNextTransition(state.transition);
+            assert state.transitionUpto < state.transitionCount: " state.transitionUpto=" + state.transitionUpto + " vs " + state.transitionCount;
+            state.transitionMin = state.transition.min;
+            state.transitionMax = state.transition.max;
             assert state.transitionMin >= 0;
             assert state.transitionMin <= 255;
             assert state.transitionMax >= 0;
@@ -1310,7 +1315,8 @@
             stateUpto++;
             states[stateUpto].state = nextState;
             states[stateUpto].changeOrd = skips[skipOffset + skipUpto++];
-            states[stateUpto].transitions = compiledAutomaton.sortedTransitions[nextState];
+            states[stateUpto].transitionCount = compiledAutomaton.automaton.getNumTransitions(nextState);
+            compiledAutomaton.automaton.initTransition(nextState, states[stateUpto].transition);
             states[stateUpto].transitionUpto = -1;
             states[stateUpto].transitionMax = -1;
             
diff --git a/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java b/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
index 2bb3aec..aedaa43 100644
--- a/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
+++ b/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
@@ -26,8 +26,6 @@
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.RollingBuffer;
 import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.State;
-import org.apache.lucene.util.automaton.Transition;
 
 // TODO: maybe also toFST?  then we can translate atts into FST outputs/weights
 
@@ -61,15 +59,15 @@
 
   private static class Position implements RollingBuffer.Resettable {
     // Any tokens that ended at our position arrive to this state:
-    State arriving;
+    int arriving = -1;
 
     // Any tokens that start at our position leave from this state:
-    State leaving;
+    int leaving = -1;
 
     @Override
     public void reset() {
-      arriving = null;
-      leaving = null;
+      arriving = -1;
+      leaving = -1;
     }
   }
 
@@ -99,8 +97,8 @@
    *  automaton where arcs are bytes (or Unicode code points 
    *  if unicodeArcs = true) from each term. */
   public Automaton toAutomaton(TokenStream in) throws IOException {
-    final Automaton a = new Automaton();
-    boolean deterministic = true;
+    final Automaton.Builder builder = new Automaton.Builder();
+    builder.createState();
 
     final TermToBytesRefAttribute termBytesAtt = in.addAttribute(TermToBytesRefAttribute.class);
     final PositionIncrementAttribute posIncAtt = in.addAttribute(PositionIncrementAttribute.class);
@@ -132,34 +130,29 @@
         pos += posInc;
 
         posData = positions.get(pos);
-        assert posData.leaving == null;
+        assert posData.leaving == -1;
 
-        if (posData.arriving == null) {
+        if (posData.arriving == -1) {
           // No token ever arrived to this position
           if (pos == 0) {
             // OK: this is the first token
-            posData.leaving = a.getInitialState();
+            posData.leaving = 0;
           } else {
             // This means there's a hole (eg, StopFilter
             // does this):
-            posData.leaving = new State();
-            addHoles(a.getInitialState(), positions, pos);
+            posData.leaving = builder.createState();
+            addHoles(builder, positions, pos);
           }
         } else {
-          posData.leaving = new State();
-          posData.arriving.addTransition(new Transition(POS_SEP, posData.leaving));
+          posData.leaving = builder.createState();
+          builder.addTransition(posData.arriving, posData.leaving, POS_SEP);
           if (posInc > 1) {
             // A token spanned over a hole; add holes
             // "under" it:
-            addHoles(a.getInitialState(), positions, pos);
+            addHoles(builder, positions, pos);
           }
         }
         positions.freeBefore(pos);
-      } else {
-        // note: this isn't necessarily true. its just that we aren't surely det.
-        // we could optimize this further (e.g. buffer and sort synonyms at a position)
-        // but thats probably overkill. this is cheap and dirty
-        deterministic = false;
       }
 
       final int endPos = pos + posLengthAtt.getPositionLength();
@@ -168,31 +161,33 @@
       final BytesRef termUTF8 = changeToken(term);
       int[] termUnicode = null;
       final Position endPosData = positions.get(endPos);
-      if (endPosData.arriving == null) {
-        endPosData.arriving = new State();
+      if (endPosData.arriving == -1) {
+        endPosData.arriving = builder.createState();
       }
 
-      State state = posData.leaving;
       int termLen;
       if (unicodeArcs) {
         final String utf16 = termUTF8.utf8ToString();
         termUnicode = new int[utf16.codePointCount(0, utf16.length())];
         termLen = termUnicode.length;
-        for (int cp, i = 0, j = 0; i < utf16.length(); i += Character.charCount(cp))
+        for (int cp, i = 0, j = 0; i < utf16.length(); i += Character.charCount(cp)) {
           termUnicode[j++] = cp = utf16.codePointAt(i);
+        }
       } else {
         termLen = termUTF8.length;
       }
 
+      int state = posData.leaving;
+
       for(int byteIDX=0;byteIDX<termLen;byteIDX++) {
-        final State nextState = byteIDX == termLen-1 ? endPosData.arriving : new State();
+        final int nextState = byteIDX == termLen-1 ? endPosData.arriving : builder.createState();
         int c;
         if (unicodeArcs) {
           c = termUnicode[byteIDX];
         } else {
           c = termUTF8.bytes[termUTF8.offset + byteIDX] & 0xff;
         }
-        state.addTransition(new Transition(c, nextState));
+        builder.addTransition(state, nextState, c);
         state = nextState;
       }
 
@@ -200,28 +195,26 @@
     }
 
     in.end();
-    State endState = null;
+    int endState = -1;
     if (offsetAtt.endOffset() > maxOffset) {
-      endState = new State();
-      endState.setAccept(true);
+      endState = builder.createState();
+      builder.setAccept(endState, true);
     }
 
     pos++;
     while (pos <= positions.getMaxPos()) {
       posData = positions.get(pos);
-      if (posData.arriving != null) {
-        if (endState != null) {
-          posData.arriving.addTransition(new Transition(POS_SEP, endState));
+      if (posData.arriving != -1) {
+        if (endState != -1) {
+          builder.addTransition(posData.arriving, endState, POS_SEP);
         } else {
-          posData.arriving.setAccept(true);
+          builder.setAccept(posData.arriving, true);
         }
       }
       pos++;
     }
 
-    //toDot(a);
-    a.setDeterministic(deterministic);
-    return a;
+    return builder.finish();
   }
 
   // for debugging!
@@ -235,26 +228,26 @@
   }
   */
 
-  private static void addHoles(State startState, RollingBuffer<Position> positions, int pos) {
+  private static void addHoles(Automaton.Builder builder, RollingBuffer<Position> positions, int pos) {
     Position posData = positions.get(pos);
     Position prevPosData = positions.get(pos-1);
 
-    while(posData.arriving == null || prevPosData.leaving == null) {
-      if (posData.arriving == null) {
-        posData.arriving = new State();
-        posData.arriving.addTransition(new Transition(POS_SEP, posData.leaving));
+    while(posData.arriving == -1 || prevPosData.leaving == -1) {
+      if (posData.arriving == -1) {
+        posData.arriving = builder.createState();
+        builder.addTransition(posData.arriving, posData.leaving, POS_SEP);
       }
-      if (prevPosData.leaving == null) {
+      if (prevPosData.leaving == -1) {
         if (pos == 1) {
-          prevPosData.leaving = startState;
+          prevPosData.leaving = 0;
         } else {
-          prevPosData.leaving = new State();
+          prevPosData.leaving = builder.createState();
         }
-        if (prevPosData.arriving != null) {
-          prevPosData.arriving.addTransition(new Transition(POS_SEP, prevPosData.leaving));
+        if (prevPosData.arriving != -1) {
+          builder.addTransition(prevPosData.arriving, prevPosData.leaving, POS_SEP);
         }
       }
-      prevPosData.leaving.addTransition(new Transition(HOLE, posData.arriving));
+      builder.addTransition(prevPosData.leaving, posData.arriving, HOLE);
       pos--;
       if (pos <= 0) {
         break;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
index 5ef41bc..91d2abe 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
@@ -18,43 +18,25 @@
  */
 
 import java.io.IOException;
-import java.io.PrintStream;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.TreeMap;
 
-import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.StringHelper;
-import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.RunAutomaton;
-import org.apache.lucene.util.automaton.Transition;
-import org.apache.lucene.util.fst.ByteSequenceOutputs;
-import org.apache.lucene.util.fst.FST;
-import org.apache.lucene.util.fst.Outputs;
-import org.apache.lucene.util.fst.Util;
 
 /** A block-based terms index and dictionary that assigns
  *  terms to variable length blocks according to how they
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java
index e910ed0..a7a569b 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java
@@ -347,7 +347,7 @@
       if (currentFrame.suffix != 0) {
         final int label = currentFrame.suffixBytes[currentFrame.startBytePos] & 0xff;
         while (label > currentFrame.curTransitionMax) {
-          if (currentFrame.transitionIndex >= currentFrame.transitions.length-1) {
+          if (currentFrame.transitionIndex >= currentFrame.transitionCount-1) {
             // Stop processing this frame -- no further
             // matches are possible because we've moved
             // beyond what the max transition will allow
@@ -359,7 +359,8 @@
             continue nextTerm;
           }
           currentFrame.transitionIndex++;
-          currentFrame.curTransitionMax = currentFrame.transitions[currentFrame.transitionIndex].getMax();
+          compiledAutomaton.automaton.getNextTransition(currentFrame.transition);
+          currentFrame.curTransitionMax = currentFrame.transition.max;
           //if (DEBUG) System.out.println("      next trans=" + currentFrame.transitions[currentFrame.transitionIndex]);
         }
       }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java
index a39f74b..e676228 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java
@@ -68,9 +68,10 @@
   int numFollowFloorBlocks;
   int nextFloorLabel;
         
-  Transition[] transitions;
+  Transition transition = new Transition();
   int curTransitionMax;
   int transitionIndex;
+  int transitionCount;
 
   FST.Arc<BytesRef> arc;
 
@@ -112,7 +113,7 @@
         nextFloorLabel = 256;
       }
       // if (DEBUG) System.out.println("    nextFloorLabel=" + (char) nextFloorLabel);
-    } while (numFollowFloorBlocks != 0 && nextFloorLabel <= transitions[transitionIndex].getMin());
+    } while (numFollowFloorBlocks != 0 && nextFloorLabel <= transition.min);
 
     load(null);
   }
@@ -120,9 +121,11 @@
   public void setState(int state) {
     this.state = state;
     transitionIndex = 0;
-    transitions = ite.compiledAutomaton.sortedTransitions[state];
-    if (transitions.length != 0) {
-      curTransitionMax = transitions[0].getMax();
+    transitionCount = ite.compiledAutomaton.automaton.getNumTransitions(state);
+    if (transitionCount != 0) {
+      ite.compiledAutomaton.automaton.initTransition(state, transition);
+      ite.compiledAutomaton.automaton.getNextTransition(transition);
+      curTransitionMax = transition.max;
     } else {
       curTransitionMax = -1;
     }
@@ -132,7 +135,7 @@
 
     // if (DEBUG) System.out.println("    load fp=" + fp + " fpOrig=" + fpOrig + " frameIndexData=" + frameIndexData + " trans=" + (transitions.length != 0 ? transitions[0] : "n/a" + " state=" + state));
 
-    if (frameIndexData != null && transitions.length != 0) {
+    if (frameIndexData != null && transitionCount != 0) {
       // Floor frame
       if (floorData.length < frameIndexData.length) {
         this.floorData = new byte[ArrayUtil.oversize(frameIndexData.length, 1)];
@@ -151,7 +154,8 @@
         // first block in case it has empty suffix:
         if (!ite.runAutomaton.isAccept(state)) {
           // Maybe skip floor blocks:
-          while (numFollowFloorBlocks != 0 && nextFloorLabel <= transitions[0].getMin()) {
+          assert transitionIndex == 0: "transitionIndex=" + transitionIndex;
+          while (numFollowFloorBlocks != 0 && nextFloorLabel <= transition.min) {
             fp = fpOrig + (floorDataReader.readVLong() >>> 1);
             numFollowFloorBlocks--;
             // if (DEBUG) System.out.println("    skip floor block!  nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[0].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks);
diff --git a/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java b/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java
index e7984e9..dcae44f 100644
--- a/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java
+++ b/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java
@@ -24,6 +24,7 @@
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
@@ -51,7 +52,7 @@
   // true if the automaton accepts a finite language
   private final boolean finite;
   // array of sorted transitions for each state, indexed by state number
-  private final Transition[][] allTransitions;
+  private final Automaton automaton;
   // for path tracking: each long records gen when we last
   // visited the state; we use gens to avoid having to clear
   private final long[] visited;
@@ -79,7 +80,7 @@
     this.runAutomaton = compiled.runAutomaton;
     assert this.runAutomaton != null;
     this.commonSuffixRef = compiled.commonSuffixRef;
-    this.allTransitions = compiled.sortedTransitions;
+    this.automaton = compiled.automaton;
 
     // used for path tracking, where each bit is a numbered state.
     visited = new long[runAutomaton.getSize()];
@@ -124,6 +125,8 @@
     }
   }
 
+  private Transition transition = new Transition();
+
   /**
    * Sets the enum to operate in linear fashion, as we have found
    * a looping transition at position: we set an upper bound and 
@@ -133,16 +136,20 @@
     assert linear == false;
     
     int state = runAutomaton.getInitialState();
+    assert state == 0;
     int maxInterval = 0xff;
+    //System.out.println("setLinear pos=" + position + " seekbytesRef=" + seekBytesRef);
     for (int i = 0; i < position; i++) {
       state = runAutomaton.step(state, seekBytesRef.bytes[i] & 0xff);
       assert state >= 0: "state=" + state;
     }
-    for (int i = 0; i < allTransitions[state].length; i++) {
-      Transition t = allTransitions[state][i];
-      if (t.getMin() <= (seekBytesRef.bytes[position] & 0xff) && 
-          (seekBytesRef.bytes[position] & 0xff) <= t.getMax()) {
-        maxInterval = t.getMax();
+    final int numTransitions = automaton.getNumTransitions(state);
+    automaton.initTransition(state, transition);
+    for (int i = 0; i < numTransitions; i++) {
+      automaton.getNextTransition(transition);
+      if (transition.min <= (seekBytesRef.bytes[position] & 0xff) && 
+          (seekBytesRef.bytes[position] & 0xff) <= transition.max) {
+        maxInterval = transition.max;
         break;
       }
     }
@@ -250,19 +257,19 @@
     seekBytesRef.length = position;
     visited[state] = curGen;
 
-    Transition transitions[] = allTransitions[state];
-
+    final int numTransitions = automaton.getNumTransitions(state);
+    automaton.initTransition(state, transition);
     // find the minimal path (lexicographic order) that is >= c
     
-    for (int i = 0; i < transitions.length; i++) {
-      Transition transition = transitions[i];
-      if (transition.getMax() >= c) {
-        int nextChar = Math.max(c, transition.getMin());
+    for (int i = 0; i < numTransitions; i++) {
+      automaton.getNextTransition(transition);
+      if (transition.max >= c) {
+        int nextChar = Math.max(c, transition.min);
         // append either the next sequential char, or the minimum transition
         seekBytesRef.grow(seekBytesRef.length + 1);
         seekBytesRef.length++;
         seekBytesRef.bytes[seekBytesRef.length - 1] = (byte) nextChar;
-        state = transition.getDest().getNumber();
+        state = transition.dest;
         /* 
          * as long as is possible, continue down the minimal path in
          * lexicographic order. if a loop or accept state is encountered, stop.
@@ -274,13 +281,14 @@
            * so the below is ok, if it is not an accept state,
            * then there MUST be at least one transition.
            */
-          transition = allTransitions[state][0];
-          state = transition.getDest().getNumber();
+          automaton.initTransition(state, transition);
+          automaton.getNextTransition(transition);
+          state = transition.dest;
           
           // append the minimum transition
           seekBytesRef.grow(seekBytesRef.length + 1);
           seekBytesRef.length++;
-          seekBytesRef.bytes[seekBytesRef.length - 1] = (byte) transition.getMin();
+          seekBytesRef.bytes[seekBytesRef.length - 1] = (byte) transition.min;
           
           // we found a loop, record it for faster enumeration
           if (!finite && !linear && visited[state] == curGen) {
diff --git a/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java b/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
index 6d73239..7aeb520 100644
--- a/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
+++ b/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
@@ -24,11 +24,11 @@
 
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.FilteredTermsEnum;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.FilteredTermsEnum;
 import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.AttributeImpl;
 import org.apache.lucene.util.AttributeSource;
@@ -36,8 +36,6 @@
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.BasicAutomata;
-import org.apache.lucene.util.automaton.BasicOperations;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
@@ -152,7 +150,7 @@
       throws IOException {
     final List<CompiledAutomaton> runAutomata = initAutomata(editDistance);
     if (editDistance < runAutomata.size()) {
-      //if (BlockTreeTermsWriter.DEBUG) System.out.println("FuzzyTE.getAEnum: ed=" + editDistance + " lastTerm=" + (lastTerm==null ? "null" : lastTerm.utf8ToString()));
+      //System.out.println("FuzzyTE.getAEnum: ed=" + editDistance + " lastTerm=" + (lastTerm==null ? "null" : lastTerm.utf8ToString()));
       final CompiledAutomaton compiled = runAutomata.get(editDistance);
       return new AutomatonFuzzyTermsEnum(terms.intersect(compiled, lastTerm == null ? null : compiled.floor(lastTerm, new BytesRef())),
                                          runAutomata.subList(0, editDistance + 1).toArray(new CompiledAutomaton[editDistance + 1]));
@@ -165,20 +163,15 @@
   private List<CompiledAutomaton> initAutomata(int maxDistance) {
     final List<CompiledAutomaton> runAutomata = dfaAtt.automata();
     //System.out.println("cached automata size: " + runAutomata.size());
-    if (runAutomata.size() <= maxDistance && 
+    if (runAutomata.size() <= maxDistance &&
         maxDistance <= LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE) {
       LevenshteinAutomata builder = 
         new LevenshteinAutomata(UnicodeUtil.newString(termText, realPrefixLength, termText.length - realPrefixLength), transpositions);
 
+      String prefix = UnicodeUtil.newString(termText, 0, realPrefixLength);
       for (int i = runAutomata.size(); i <= maxDistance; i++) {
-        Automaton a = builder.toAutomaton(i);
+        Automaton a = builder.toAutomaton(i, prefix);
         //System.out.println("compute automaton n=" + i);
-        // constant prefix
-        if (realPrefixLength > 0) {
-          Automaton prefix = BasicAutomata.makeString(
-            UnicodeUtil.newString(termText, 0, realPrefixLength));
-          a = BasicOperations.concatenate(prefix, a);
-        }
         runAutomata.add(new CompiledAutomaton(a, true, false));
       }
     }
diff --git a/lucene/core/src/java/org/apache/lucene/search/RegexpQuery.java b/lucene/core/src/java/org/apache/lucene/search/RegexpQuery.java
index e5664c0..2aba28c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/RegexpQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/RegexpQuery.java
@@ -1,7 +1,6 @@
 package org.apache.lucene.search;
 
 import org.apache.lucene.index.Term;
-
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.AutomatonProvider;
diff --git a/lucene/core/src/java/org/apache/lucene/search/WildcardQuery.java b/lucene/core/src/java/org/apache/lucene/search/WildcardQuery.java
index 12cd770..91b473e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/WildcardQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/WildcardQuery.java
@@ -17,15 +17,15 @@
  * limitations under the License.
  */
 
-import org.apache.lucene.index.Term;
-import org.apache.lucene.util.ToStringUtils;
-import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.BasicAutomata;
-import org.apache.lucene.util.automaton.BasicOperations;
-
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.lucene.index.Term;
+import org.apache.lucene.util.ToStringUtils;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.Operations;
+import org.apache.lucene.util.automaton.Automaton;
+
 /** Implements the wildcard search query. Supported wildcards are <code>*</code>, which
  * matches any character sequence (including the empty one), and <code>?</code>,
  * which matches any single character. '\' is the escape character.
@@ -72,26 +72,26 @@
       int length = Character.charCount(c);
       switch(c) {
         case WILDCARD_STRING: 
-          automata.add(BasicAutomata.makeAnyString());
+          automata.add(Automata.makeAnyString());
           break;
         case WILDCARD_CHAR:
-          automata.add(BasicAutomata.makeAnyChar());
+          automata.add(Automata.makeAnyChar());
           break;
         case WILDCARD_ESCAPE:
           // add the next codepoint instead, if it exists
           if (i + length < wildcardText.length()) {
             final int nextChar = wildcardText.codePointAt(i + length);
             length += Character.charCount(nextChar);
-            automata.add(BasicAutomata.makeChar(nextChar));
+            automata.add(Automata.makeChar(nextChar));
             break;
           } // else fallthru, lenient parsing with a trailing \
         default:
-          automata.add(BasicAutomata.makeChar(c));
+          automata.add(Automata.makeChar(c));
       }
       i += length;
     }
     
-    return BasicOperations.concatenate(automata);
+    return Operations.concatenate(automata);
   }
   
   /**
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/Automata.java b/lucene/core/src/java/org/apache/lucene/util/automaton/Automata.java
new file mode 100644
index 0000000..2d327cf
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/Automata.java
@@ -0,0 +1,330 @@
+/*
+ * dk.brics.automaton
+ * 
+ * Copyright (c) 2001-2009 Anders Moeller
+ * All rights reserved.
+ * 
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the above copyright
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ * 3. The name of the author may not be used to endorse or promote products
+ *    derived from this software without specific prior written permission.
+ * 
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
+ * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+ * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+ * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+package org.apache.lucene.util.automaton;
+
+import java.util.*;
+
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Construction of basic automata.
+ * 
+ * @lucene.experimental
+ */
+final public class Automata {
+  
+  private Automata() {}
+  
+  /**
+   * Returns a new (deterministic) automaton with the empty language.
+   */
+  public static Automaton makeEmpty() {
+    Automaton a = new Automaton();
+    a.finishState();
+    return a;
+  }
+  
+  /**
+   * Returns a new (deterministic) automaton that accepts only the empty string.
+   */
+  public static Automaton makeEmptyString() {
+    Automaton a = new Automaton();
+    a.createState();
+    a.setAccept(0, true);
+    return a;
+  }
+  
+  /**
+   * Returns a new (deterministic) automaton that accepts all strings.
+   */
+  public static Automaton makeAnyString() {
+    Automaton a = new Automaton();
+    int s = a.createState();
+    a.setAccept(s, true);
+    a.addTransition(s, s, Character.MIN_CODE_POINT, Character.MAX_CODE_POINT);
+    a.finishState();
+    return a;
+  }
+  
+  /**
+   * Returns a new (deterministic) automaton that accepts any single codepoint.
+   */
+  public static Automaton makeAnyChar() {
+    return makeCharRange(Character.MIN_CODE_POINT, Character.MAX_CODE_POINT);
+  }
+
+  /** Accept any single character starting from the specified state, returning the new state */
+  public static int appendAnyChar(Automaton a, int state) {
+    int newState = a.createState();
+    a.addTransition(state, newState, Character.MIN_CODE_POINT, Character.MAX_CODE_POINT);
+    return newState;
+  }
+
+  /**
+   * Returns a new (deterministic) automaton that accepts a single codepoint of
+   * the given value.
+   */
+  public static Automaton makeChar(int c) {
+    return makeCharRange(c, c);
+  }
+
+  /** Appends the specified character to the specified state, returning a new state. */
+  public static int appendChar(Automaton a, int state, int c) {
+    int newState = a.createState();
+    a.addTransition(state, newState, c, c);
+    return newState;
+  }
+
+  /**
+   * Returns a new (deterministic) automaton that accepts a single codepoint whose
+   * value is in the given interval (including both end points).
+   */
+  public static Automaton makeCharRange(int min, int max) {
+    if (min > max) {
+      return makeEmpty();
+    }
+    Automaton a = new Automaton();
+    int s1 = a.createState();
+    int s2 = a.createState();
+    a.setAccept(s2, true);
+    a.addTransition(s1, s2, min, max);
+    a.finishState();
+    return a;
+  }
+  
+  /**
+   * Constructs sub-automaton corresponding to decimal numbers of length
+   * x.substring(n).length().
+   */
+  private static int anyOfRightLength(Automaton.Builder builder, String x, int n) {
+    int s = builder.createState();
+    if (x.length() == n) {
+      builder.setAccept(s, true);
+    } else {
+      builder.addTransition(s, anyOfRightLength(builder, x, n + 1), '0', '9');
+    }
+    return s;
+  }
+  
+  /**
+   * Constructs sub-automaton corresponding to decimal numbers of value at least
+   * x.substring(n) and length x.substring(n).length().
+   */
+  private static int atLeast(Automaton.Builder builder, String x, int n, Collection<Integer> initials,
+      boolean zeros) {
+    int s = builder.createState();
+    if (x.length() == n) {
+      builder.setAccept(s, true);
+    } else {
+      if (zeros) {
+        initials.add(s);
+      }
+      char c = x.charAt(n);
+      builder.addTransition(s, atLeast(builder, x, n + 1, initials, zeros && c == '0'), c);
+      if (c < '9') {
+        builder.addTransition(s, anyOfRightLength(builder, x, n + 1), (char) (c + 1), '9');
+      }
+    }
+    return s;
+  }
+  
+  /**
+   * Constructs sub-automaton corresponding to decimal numbers of value at most
+   * x.substring(n) and length x.substring(n).length().
+   */
+  private static int atMost(Automaton.Builder builder, String x, int n) {
+    int s = builder.createState();
+    if (x.length() == n) {
+      builder.setAccept(s, true);
+    } else {
+      char c = x.charAt(n);
+      builder.addTransition(s, atMost(builder, x, (char) n + 1), c);
+      if (c > '0') {
+        builder.addTransition(s, anyOfRightLength(builder, x, n + 1), '0', (char) (c - 1));
+      }
+    }
+    return s;
+  }
+  
+  /**
+   * Constructs sub-automaton corresponding to decimal numbers of value between
+   * x.substring(n) and y.substring(n) and of length x.substring(n).length()
+   * (which must be equal to y.substring(n).length()).
+   */
+  private static int between(Automaton.Builder builder,
+      String x, String y, int n,
+      Collection<Integer> initials, boolean zeros) {
+    int s = builder.createState();
+    if (x.length() == n) {
+      builder.setAccept(s, true);
+    } else {
+      if (zeros) {
+        initials.add(s);
+      }
+      char cx = x.charAt(n);
+      char cy = y.charAt(n);
+      if (cx == cy) {
+        builder.addTransition(s, between(builder, x, y, n + 1, initials, zeros && cx == '0'), cx);
+      } else { // cx<cy
+        builder.addTransition(s, atLeast(builder, x, n + 1, initials, zeros && cx == '0'), cx);
+        builder.addTransition(s, atMost(builder, y, n + 1), cy);
+        if (cx + 1 < cy) {
+          builder.addTransition(s, anyOfRightLength(builder, x, n+1), (char) (cx + 1), (char) (cy - 1));
+        }
+      }
+    }
+
+    return s;
+  }
+
+  /**
+   * Returns a new automaton that accepts strings representing decimal
+   * non-negative integers in the given interval.
+   * 
+   * @param min minimal value of interval
+   * @param max maximal value of interval (both end points are included in the
+   *          interval)
+   * @param digits if >0, use fixed number of digits (strings must be prefixed
+   *          by 0's to obtain the right length) - otherwise, the number of
+   *          digits is not fixed (any number of leading 0s is accepted)
+   * @exception IllegalArgumentException if min>max or if numbers in the
+   *              interval cannot be expressed with the given fixed number of
+   *              digits
+   */
+  public static Automaton makeInterval(int min, int max, int digits)
+      throws IllegalArgumentException {
+    String x = Integer.toString(min);
+    String y = Integer.toString(max);
+    if (min > max || (digits > 0 && y.length() > digits)) {
+      throw new IllegalArgumentException();
+    }
+    int d;
+    if (digits > 0) d = digits;
+    else d = y.length();
+    StringBuilder bx = new StringBuilder();
+    for (int i = x.length(); i < d; i++) {
+      bx.append('0');
+    }
+    bx.append(x);
+    x = bx.toString();
+    StringBuilder by = new StringBuilder();
+    for (int i = y.length(); i < d; i++) {
+      by.append('0');
+    }
+    by.append(y);
+    y = by.toString();
+
+    Automaton.Builder builder = new Automaton.Builder();
+
+    if (digits <= 0) {
+      // Reserve the "real" initial state:
+      builder.createState();
+    }
+
+    Collection<Integer> initials = new ArrayList<>();
+
+    between(builder, x, y, 0, initials, digits <= 0);
+
+    Automaton a1 = builder.finish();
+
+    if (digits <= 0) {
+      a1.addTransition(0, 0, '0');
+      for (int p : initials) {
+        a1.addEpsilon(0, p);
+      }
+      a1.finishState();
+    }
+
+    return a1;
+  }
+  
+  /**
+   * Returns a new (deterministic) automaton that accepts the single given
+   * string.
+   */
+  public static Automaton makeString(String s) {
+    Automaton a = new Automaton();
+    int lastState = a.createState();
+    for (int i = 0, cp = 0; i < s.length(); i += Character.charCount(cp)) {
+      int state = a.createState();
+      cp = s.codePointAt(i);
+      a.addTransition(lastState, state, cp, cp);
+      lastState = state;
+    }
+
+    a.setAccept(lastState, true);
+    a.finishState();
+
+    assert a.isDeterministic();
+    assert Operations.hasDeadStates(a) == false;
+
+    return a;
+  }
+  
+  /**
+   * Returns a new (deterministic) automaton that accepts the single given
+   * string from the specified unicode code points.
+   */
+  public static Automaton makeString(int[] word, int offset, int length) {
+    Automaton a = new Automaton();
+    a.createState();
+    int s = 0;
+    for (int i = offset; i < offset+length; i++) {
+      int s2 = a.createState();
+      a.addTransition(s, s2, word[i]);
+      s = s2;
+    }
+    a.setAccept(s, true);
+    a.finishState();
+
+    return a;
+  }
+
+  /**
+   * Returns a new (deterministic and minimal) automaton that accepts the union
+   * of the given collection of {@link BytesRef}s representing UTF-8 encoded
+   * strings.
+   * 
+   * @param utf8Strings
+   *          The input strings, UTF-8 encoded. The collection must be in sorted
+   *          order.
+   * 
+   * @return An {@link Automaton} accepting all input strings. The resulting
+   *         automaton is codepoint based (full unicode codepoints on
+   *         transitions).
+   */
+  public static Automaton makeStringUnion(Collection<BytesRef> utf8Strings) {
+    if (utf8Strings.isEmpty()) {
+      return makeEmpty();
+    } else {
+      return DaciukMihovAutomatonBuilder.build(utf8Strings);
+    }
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java b/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java
index 9cdab39..3603aba 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java
@@ -1,777 +1,783 @@
-/*
- * dk.brics.automaton
- * 
- * Copyright (c) 2001-2009 Anders Moeller
- * All rights reserved.
- * 
- * Redistribution and use in source and binary forms, with or without
- * modification, are permitted provided that the following conditions
- * are met:
- * 1. Redistributions of source code must retain the above copyright
- *    notice, this list of conditions and the following disclaimer.
- * 2. Redistributions in binary form must reproduce the above copyright
- *    notice, this list of conditions and the following disclaimer in the
- *    documentation and/or other materials provided with the distribution.
- * 3. The name of the author may not be used to endorse or promote products
- *    derived from this software without specific prior written permission.
- * 
- * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
- * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
- * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
- * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
- * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
- * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
- * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
- */
-
 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.io.IOException;
+//import java.io.PrintWriter;
 import java.util.Arrays;
-import java.util.BitSet;
-import java.util.Collection;
-import java.util.HashMap;
 import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.InPlaceMergeSorter;
+import org.apache.lucene.util.Sorter;
 
-/**
- * Finite-state automaton with regular expression operations.
- * <p>
- * Class invariants:
- * <ul>
- * <li>An automaton is either represented explicitly (with {@link State} and
- * {@link Transition} objects) or with a singleton string (see
- * {@link #getSingleton()} and {@link #expandSingleton()}) in case the automaton
- * is known to accept exactly one string. (Implicitly, all states and
- * transitions of an automaton are reachable from its initial state.)
- * <li>Automata are always reduced (see {@link #reduce()}) and have no
- * transitions to dead states (see {@link #removeDeadTransitions()}).
- * <li>If an automaton is nondeterministic, then {@link #isDeterministic()}
- * returns false (but the converse is not required).
- * <li>Automata provided as input to operations are generally assumed to be
- * disjoint.
- * </ul>
- * <p>
- * If the states or transitions are manipulated manually, the
- * {@link #restoreInvariant()} and {@link #setDeterministic(boolean)} methods
- * should be used afterwards to restore representation invariants that are
- * assumed by the built-in automata operations.
- * 
- * <p>
- * <p>
- * Note: This class has internal mutable state and is not thread safe. It is 
- * the caller's responsibility to ensure any necessary synchronization if you
- * wish to use the same Automaton from multiple threads. In general it is instead
- * recommended to use a {@link RunAutomaton} for multithreaded matching: it is immutable, 
- * thread safe, and much faster.  
- * </p>
- * @lucene.experimental
- */
-public class Automaton implements Cloneable {
-  
-  /**
-   * Minimize using Hopcroft's O(n log n) algorithm. This is regarded as one of
-   * the most generally efficient algorithms that exist.
-   * 
-   * @see #setMinimization(int)
-   */
-  public static final int MINIMIZE_HOPCROFT = 2;
-  
-  /** Selects minimization algorithm (default: <code>MINIMIZE_HOPCROFT</code>). */
-  static int minimization = MINIMIZE_HOPCROFT;
-  
-  /** Initial state of this automaton. */
-  State initial;
-  
-  /**
-   * If true, then this automaton is definitely deterministic (i.e., there are
-   * no choices for any run, but a run may crash).
-   */
-  boolean deterministic;
-  
-  /** Extra data associated with this automaton. */
-  transient Object info;
-  
-  /**
-   * Hash code. Recomputed by {@link MinimizationOperations#minimize(Automaton)}
-   */
-  //int hash_code;
-  
-  /** Singleton string. Null if not applicable. */
-  String singleton;
-  
-  /** Minimize always flag. */
-  static boolean minimize_always = false;
-  
-  /**
-   * Selects whether operations may modify the input automata (default:
-   * <code>false</code>).
-   */
-  static boolean allow_mutation = false;
-  
-  /**
-   * Constructs a new automaton that accepts the empty language. Using this
-   * constructor, automata can be constructed manually from {@link State} and
-   * {@link Transition} objects.
-   * 
-   * @see State
-   * @see Transition
-   */
-  public Automaton(State initial) {
-    this.initial = initial;
-    deterministic = true;
-    singleton = null;
-  }
 
+// TODO
+//   - could use packed int arrays instead
+//   - could encode dest w/ delta from to?
+
+/** Represents an automaton and all its states and transitions.  States
+ *  are integers and must be created using {@link #createState}.  Mark a
+ *  state as an accept state using {@link #setAccept}.  Add transitions
+ *  using {@link #addTransition}.  Each state must have all of its
+ *  transitions added at once; if this is too restrictive then use
+ *  {@link Automaton.Builder} instead.  State 0 is always the
+ *  initial state.  Once a state is finished, either
+ *  because you've starting adding transitions to another state or you
+ *  call {@link #finishState}, then that states transitions are sorted
+ *  (first by min, then max, then dest) and reduced (transitions with
+ *  adjacent labels going to the same dest are combined).
+ *
+ * @lucene.experimental */
+
+public class Automaton {
+  /** Where we next write to the int[] states; this increments by 2 for
+   *  each added state because we pack a pointer to the transitions
+   *  array and a count of how many transitions leave the state.  */
+  private int nextState;
+
+  /** Where we next write to in int[] transitions; this
+   *  increments by 3 for each added transition because we
+   *  pack min, max, dest in sequence. */
+  private int nextTransition;
+
+  /** Current state we are adding transitions to; the caller
+   *  must add all transitions for this state before moving
+   *  onto another state. */
+  private int curState = -1;
+
+  /** Index in the transitions array, where this states
+   *  leaving transitions are stored, or -1 if this state
+   *  has not added any transitions yet, followed by number
+   *  of transitions. */
+  private int[] states = new int[4];
+
+  /** Holds toState, min, max for each transition. */
+  private int[] transitions = new int[6];
+
+  private FixedBitSet isAccept = new FixedBitSet(4);
+
+  /** True if no state has two transitions leaving with the same label. */
+  private boolean deterministic = true;
+
+  /** Sole constructor; creates an automaton with no states. */
   public Automaton() {
-    this(new State());
   }
-  
-  /**
-   * Selects minimization algorithm (default: <code>MINIMIZE_HOPCROFT</code>).
-   * 
-   * @param algorithm minimization algorithm
-   */
-  static public void setMinimization(int algorithm) {
-    minimization = algorithm;
+
+  /** Create a new state. */
+  public int createState() {
+    growStates();
+    int state = nextState/2;
+    states[nextState] = -1;
+    nextState += 2;
+    if (state >= isAccept.length()) {
+      FixedBitSet newBits = new FixedBitSet(ArrayUtil.oversize(state+1, 1));
+      newBits.or(isAccept);
+      isAccept = newBits;
+    }
+    return state;
   }
-  
-  /**
-   * Sets or resets minimize always flag. If this flag is set, then
-   * {@link MinimizationOperations#minimize(Automaton)} will automatically be
-   * invoked after all operations that otherwise may produce non-minimal
-   * automata. By default, the flag is not set.
-   * 
-   * @param flag if true, the flag is set
-   */
-  static public void setMinimizeAlways(boolean flag) {
-    minimize_always = flag;
+
+  /** Set or clear this state as an accept state. */
+  public void setAccept(int state, boolean accept) {
+    if (state >= getNumStates()) {
+      throw new IllegalArgumentException("state=" + state + " is out of bounds (numStates=" + getNumStates() + ")");
+    }
+    if (accept) {
+      isAccept.set(state);
+    } else {
+      isAccept.clear(state);
+    }
   }
-  
-  /**
-   * Sets or resets allow mutate flag. If this flag is set, then all automata
-   * operations may modify automata given as input; otherwise, operations will
-   * always leave input automata languages unmodified. By default, the flag is
-   * not set.
-   * 
-   * @param flag if true, the flag is set
-   * @return previous value of the flag
-   */
-  static public boolean setAllowMutate(boolean flag) {
-    boolean b = allow_mutation;
-    allow_mutation = flag;
-    return b;
+
+  /** Sugar to get all transitions for all states.  This is
+   *  object-heavy; it's better to iterate state by state instead. */
+  public Transition[][] getSortedTransitions() {
+    int numStates = getNumStates();
+    Transition[][] transitions = new Transition[numStates][];
+    for(int s=0;s<numStates;s++) {
+      int numTransitions = getNumTransitions(s);
+      transitions[s] = new Transition[numTransitions];
+      for(int t=0;t<numTransitions;t++) {
+        Transition transition = new Transition();
+        getTransition(s, t, transition);
+        transitions[s][t] = transition;
+      }
+    }
+
+    return transitions;
   }
-  
-  /**
-   * Returns the state of the allow mutate flag. If this flag is set, then all
-   * automata operations may modify automata given as input; otherwise,
-   * operations will always leave input automata languages unmodified. By
-   * default, the flag is not set.
-   * 
-   * @return current value of the flag
-   */
-  static boolean getAllowMutate() {
-    return allow_mutation;
+
+  /** Returns accept states.  If the bit is set then that state is an accept state. */
+  FixedBitSet getAcceptStates() {
+    return isAccept;
   }
-  
-  void checkMinimizeAlways() {
-    if (minimize_always) MinimizationOperations.minimize(this);
+
+  /** Returns true if this state is an accept state. */
+  public boolean isAccept(int state) {
+    return isAccept.get(state);
   }
-  
-  boolean isSingleton() {
-    return singleton != null;
+
+  /** Add a new transition with min = max = label. */
+  public void addTransition(int source, int dest, int label) {
+    addTransition(source, dest, label, label);
   }
-  
-  /**
-   * Returns the singleton string for this automaton. An automaton that accepts
-   * exactly one string <i>may</i> be represented in singleton mode. In that
-   * case, this method may be used to obtain the string.
-   * 
-   * @return string, null if this automaton is not in singleton mode.
-   */
-  public String getSingleton() {
-    return singleton;
+
+  /** Add a new transition with the specified source, dest, min, max. */
+  public void addTransition(int source, int dest, int min, int max) {
+    assert nextTransition%3 == 0;
+
+    if (source >= nextState/2) {
+      throw new IllegalArgumentException("source=" + source + " is out of bounds (maxState is " + (nextState/2-1) + ")");
+    }
+    if (dest >= nextState/2) {
+      throw new IllegalArgumentException("dest=" + dest + " is out of bounds (max state is " + (nextState/2-1) + ")");
+    }
+
+    growTransitions();
+    if (curState != source) {
+      if (curState != -1) {
+        finishCurrentState();
+      }
+
+      // Move to next source:
+      curState = source;
+      if (states[2*curState] != -1) {
+        throw new IllegalStateException("from state (" + source + ") already had transitions added");
+      }
+      assert states[2*curState+1] == 0;
+      states[2*curState] = nextTransition;
+    }
+
+    transitions[nextTransition++] = dest;
+    transitions[nextTransition++] = min;
+    transitions[nextTransition++] = max;
+
+    // Increment transition count for this state
+    states[2*curState+1]++;
   }
-  
-  /**
-   * Sets initial state.
-   * 
-   * @param s state
-   */
-  /*
-  public void setInitialState(State s) {
-    initial = s;
-    singleton = null;
+
+  /** Add a [virtual] epsilon transition between source and dest.
+   *  Dest state must already have all transitions added because this
+   *  method simply copies those same transitions over to source. */
+  public void addEpsilon(int source, int dest) {
+    Transition t = new Transition();
+    int count = initTransition(dest, t);
+    for(int i=0;i<count;i++) {
+      getNextTransition(t);
+      addTransition(source, t.dest, t.min, t.max);
+    }
+    if (isAccept(dest)) {
+      setAccept(source, true);
+    }
   }
-  */
-  
-  /**
-   * Gets initial state.
-   * 
-   * @return state
-   */
-  public State getInitialState() {
-    expandSingleton();
-    return initial;
+
+  /** Copies over all states/transitions from other.  The states numbers
+   *  are sequentially assigned (appended). */
+  public void copy(Automaton other) {
+
+    // Bulk copy and then fixup the state pointers:
+    int stateOffset = getNumStates();
+    states = ArrayUtil.grow(states, nextState + other.nextState);
+    System.arraycopy(other.states, 0, states, nextState, other.nextState);
+    for(int i=0;i<other.nextState;i += 2) {
+      if (states[nextState+i] != -1) {
+        states[nextState+i] += nextTransition;
+      }
+    }
+    nextState += other.nextState;
+    if (isAccept.length() < nextState/2) {
+      FixedBitSet newBits = new FixedBitSet(ArrayUtil.oversize(nextState/2, 1));
+      newBits.or(isAccept);
+      isAccept = newBits;
+    }
+    int otherNumStates = other.getNumStates();
+    FixedBitSet otherAcceptStates = other.getAcceptStates();
+    int state = 0;
+    while (state < otherNumStates && (state = otherAcceptStates.nextSetBit(state)) != -1) {
+      setAccept(stateOffset + state, true);
+      state++;
+    }
+
+    // Bulk copy and then fixup dest for each transition:
+    transitions = ArrayUtil.grow(transitions, nextTransition + other.nextTransition);
+    System.arraycopy(other.transitions, 0, transitions, nextTransition, other.nextTransition);
+    for(int i=0;i<other.nextTransition;i += 3) {
+      transitions[nextTransition+i] += stateOffset;
+    }
+    nextTransition += other.nextTransition;
+
+    if (other.deterministic == false) {
+      deterministic = false;
+    }
   }
-  
-  /**
-   * Returns deterministic flag for this automaton.
-   * 
-   * @return true if the automaton is definitely deterministic, false if the
-   *         automaton may be nondeterministic
-   */
+
+  /** Freezes the last state, sorting and reducing the transitions. */
+  private void finishCurrentState() {
+    int numTransitions = states[2*curState+1];
+    assert numTransitions > 0;
+
+    int offset = states[2*curState];
+    int start = offset/3;
+    destMinMaxSorter.sort(start, start+numTransitions);
+
+    // Reduce any "adjacent" transitions:
+    int upto = 0;
+    int min = -1;
+    int max = -1;
+    int dest = -1;
+
+    for(int i=0;i<numTransitions;i++) {
+      int tDest = transitions[offset+3*i];
+      int tMin = transitions[offset+3*i+1];
+      int tMax = transitions[offset+3*i+2];
+
+      if (dest == tDest) {
+        if (tMin <= max+1) {
+          if (tMax > max) {
+            max = tMax;
+          }
+        } else {
+          if (dest != -1) {
+            transitions[offset+3*upto] = dest;
+            transitions[offset+3*upto+1] = min;
+            transitions[offset+3*upto+2] = max;
+            upto++;
+          }
+          min = tMin;
+          max = tMax;
+        }
+      } else {
+        if (dest != -1) {
+          transitions[offset+3*upto] = dest;
+          transitions[offset+3*upto+1] = min;
+          transitions[offset+3*upto+2] = max;
+          upto++;
+        }
+        dest = tDest;
+        min = tMin;
+        max = tMax;
+      }
+    }
+
+    if (dest != -1) {
+      // Last transition
+      transitions[offset+3*upto] = dest;
+      transitions[offset+3*upto+1] = min;
+      transitions[offset+3*upto+2] = max;
+      upto++;
+    }
+
+    nextTransition -= (numTransitions-upto)*3;
+    states[2*curState+1] = upto;
+
+    // Sort transitions by min/max/dest:
+    minMaxDestSorter.sort(start, start+upto);
+
+    if (deterministic && upto > 1) {
+      int lastMax = transitions[offset+2];
+      for(int i=1;i<upto;i++) {
+        min = transitions[offset + 3*i + 1];
+        if (min <= lastMax) {
+          deterministic = false;
+          break;
+        }
+        lastMax = transitions[offset + 3*i + 2];
+      }
+    }
+  }
+
+  /** Returns true if this automaton is deterministic (for ever state
+   *  there is only one transition for each label). */
   public boolean isDeterministic() {
     return deterministic;
   }
-  
-  /**
-   * Sets deterministic flag for this automaton. This method should (only) be
-   * used if automata are constructed manually.
-   * 
-   * @param deterministic true if the automaton is definitely deterministic,
-   *          false if the automaton may be nondeterministic
-   */
-  public void setDeterministic(boolean deterministic) {
-    this.deterministic = deterministic;
-  }
-  
-  /**
-   * Associates extra information with this automaton.
-   * 
-   * @param info extra information
-   */
-  public void setInfo(Object info) {
-    this.info = info;
-  }
-  
-  /**
-   * Returns extra information associated with this automaton.
-   * 
-   * @return extra information
-   * @see #setInfo(Object)
-   */
-  public Object getInfo() {
-    return info;
-  }
 
-  // cached
-  private State[] numberedStates;
-
-  public State[] getNumberedStates() {
-    if (numberedStates == null) {
-      expandSingleton();
-      final Set<State> visited = new HashSet<>();
-      final LinkedList<State> worklist = new LinkedList<>();
-      State states[] = new State[4];
-      int upto = 0;
-      worklist.add(initial);
-      visited.add(initial);
-      initial.number = upto;
-      states[upto] = initial;
-      upto++;
-      while (worklist.size() > 0) {
-        State s = worklist.removeFirst();
-        for (int i=0;i<s.numTransitions;i++) {
-          final Transition t = s.transitionsArray[i];
-          if (!visited.contains(t.to)) {
-            visited.add(t.to);
-            worklist.add(t.to);
-            t.to.number = upto;
-            if (upto == states.length) {
-              final State[] newArray = new State[ArrayUtil.oversize(1+upto, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-              System.arraycopy(states, 0, newArray, 0, upto);
-              states = newArray;
-            }
-            states[upto] = t.to;
-            upto++;
-          }
-        }
-      }
-      if (states.length != upto) {
-        final State[] newArray = new State[upto];
-        System.arraycopy(states, 0, newArray, 0, upto);
-        states = newArray;
-      }
-      numberedStates = states;
+  /** Finishes the current state; call this once you are done adding
+   *  transitions for a state.  This is automatically called if you
+   *  start adding transitions to a new source state, but for the last
+   *  state you add you need to this method yourself. */
+  public void finishState() {
+    if (curState != -1) {
+      finishCurrentState();
+      curState = -1;
     }
-
-    return numberedStates;
   }
 
-  public void setNumberedStates(State[] states) {
-    setNumberedStates(states, states.length);
+  // TODO: add finish() to shrink wrap the arrays?
+
+  /** How many states this automaton has. */
+  public int getNumStates() {
+    return nextState/2;
   }
 
-  public void setNumberedStates(State[] states, int count) {
-    assert count <= states.length;
-    // TODO: maybe we can eventually allow for oversizing here...
-    if (count < states.length) {
-      final State[] newArray = new State[count];
-      System.arraycopy(states, 0, newArray, 0, count);
-      numberedStates = newArray;
+  /** How many transitions this state has. */
+  public int getNumTransitions(int state) {
+    int count = states[2*state+1];
+    if (count == -1) {
+      return 0;
     } else {
-      numberedStates = states;
+      return count;
     }
   }
 
-  public void clearNumberedStates() {
-    numberedStates = null;
+  private void growStates() {
+    if (nextState+2 >= states.length) {
+      states = ArrayUtil.grow(states, nextState+2);
+    }
   }
 
-  /**
-   * Returns the set of reachable accept states.
-   * 
-   * @return set of {@link State} objects
-   */
-  public Set<State> getAcceptStates() {
-    expandSingleton();
-    HashSet<State> accepts = new HashSet<>();
-    HashSet<State> visited = new HashSet<>();
-    LinkedList<State> worklist = new LinkedList<>();
-    worklist.add(initial);
-    visited.add(initial);
-    while (worklist.size() > 0) {
-      State s = worklist.removeFirst();
-      if (s.accept) accepts.add(s);
-      for (Transition t : s.getTransitions())
-        if (!visited.contains(t.to)) {
-          visited.add(t.to);
-          worklist.add(t.to);
-        }
+  private void growTransitions() {
+    if (nextTransition+3 >= transitions.length) {
+      transitions = ArrayUtil.grow(transitions, nextTransition+3);
     }
-    return accepts;
   }
-  
-  /**
-   * Adds transitions to explicit crash state to ensure that transition function
-   * is total.
-   */
-  void totalize() {
-    State s = new State();
-    s.addTransition(new Transition(Character.MIN_CODE_POINT, Character.MAX_CODE_POINT,
-        s));
-    for (State p : getNumberedStates()) {
-      int maxi = Character.MIN_CODE_POINT;
-      p.sortTransitions(Transition.CompareByMinMaxThenDest);
-      for (Transition t : p.getTransitions()) {
-        if (t.min > maxi) p.addTransition(new Transition(maxi,
-            (t.min - 1), s));
-        if (t.max + 1 > maxi) maxi = t.max + 1;
+
+  /** Sorts transitions by dest, ascending, then min label ascending, then max label ascending */
+  private final Sorter destMinMaxSorter = new InPlaceMergeSorter() {
+
+      private void swapOne(int i, int j) {
+        int x = transitions[i];
+        transitions[i] = transitions[j];
+        transitions[j] = x;
       }
-      if (maxi <= Character.MAX_CODE_POINT) p.addTransition(new Transition(
-          maxi, Character.MAX_CODE_POINT, s));
+
+      @Override
+      protected void swap(int i, int j) {
+        int iStart = 3*i;
+        int jStart = 3*j;
+        swapOne(iStart, jStart);
+        swapOne(iStart+1, jStart+1);
+        swapOne(iStart+2, jStart+2);
+      };
+
+      @Override
+      protected int compare(int i, int j) {
+        int iStart = 3*i;
+        int jStart = 3*j;
+
+        // First dest:
+        int iDest = transitions[iStart];
+        int jDest = transitions[jStart];
+        if (iDest < jDest) {
+          return -1;
+        } else if (iDest > jDest) {
+          return 1;
+        }
+
+        // Then min:
+        int iMin = transitions[iStart+1];
+        int jMin = transitions[jStart+1];
+        if (iMin < jMin) {
+          return -1;
+        } else if (iMin > jMin) {
+          return 1;
+        }
+
+        // Then max:
+        int iMax = transitions[iStart+2];
+        int jMax = transitions[jStart+2];
+        if (iMax < jMax) {
+          return -1;
+        } else if (iMax > jMax) {
+          return 1;
+        }
+
+        return 0;
+      }
+    };
+
+  /** Sorts transitions by min label, ascending, then max label ascending, then dest ascending */
+  private final Sorter minMaxDestSorter = new InPlaceMergeSorter() {
+
+      private void swapOne(int i, int j) {
+        int x = transitions[i];
+        transitions[i] = transitions[j];
+        transitions[j] = x;
+      }
+
+      @Override
+      protected void swap(int i, int j) {
+        int iStart = 3*i;
+        int jStart = 3*j;
+        swapOne(iStart, jStart);
+        swapOne(iStart+1, jStart+1);
+        swapOne(iStart+2, jStart+2);
+      };
+
+      @Override
+      protected int compare(int i, int j) {
+        int iStart = 3*i;
+        int jStart = 3*j;
+
+        // First min:
+        int iMin = transitions[iStart+1];
+        int jMin = transitions[jStart+1];
+        if (iMin < jMin) {
+          return -1;
+        } else if (iMin > jMin) {
+          return 1;
+        }
+
+        // Then max:
+        int iMax = transitions[iStart+2];
+        int jMax = transitions[jStart+2];
+        if (iMax < jMax) {
+          return -1;
+        } else if (iMax > jMax) {
+          return 1;
+        }
+
+        // Then dest:
+        int iDest = transitions[iStart];
+        int jDest = transitions[jStart];
+        if (iDest < jDest) {
+          return -1;
+        } else if (iDest > jDest) {
+          return 1;
+        }
+
+        return 0;
+      }
+    };
+
+  /** Initialize the provided Transition to iterate through all transitions
+   *  leaving the specified state.  You must call {@link #getNextTransition} to
+   *  get each transition.  Returns the number of transitions
+   *  leaving this state. */
+  public int initTransition(int state, Transition t) {
+    assert state < nextState/2: "state=" + state + " nextState=" + nextState;
+    t.source = state;
+    t.transitionUpto = states[2*state];
+    return getNumTransitions(state);
+  }
+
+  /** Iterate to the next transition after the provided one */
+  public void getNextTransition(Transition t) {
+    // Make sure there is still a transition left:
+    assert (t.transitionUpto+3 - states[2*t.source]) <= 3*states[2*t.source+1];
+    t.dest = transitions[t.transitionUpto++];
+    t.min = transitions[t.transitionUpto++];
+    t.max = transitions[t.transitionUpto++];
+  }
+
+  /** Fill the provided {@link Transition} with the index'th
+   *  transition leaving the specified state. */
+  public void getTransition(int state, int index, Transition t) {
+    int i = states[2*state] + 3*index;
+    t.source = state;
+    t.dest = transitions[i++];
+    t.min = transitions[i++];
+    t.max = transitions[i++];
+  }
+
+  static void appendCharString(int c, StringBuilder b) {
+    if (c >= 0x21 && c <= 0x7e && c != '\\' && c != '"') b.appendCodePoint(c);
+    else {
+      b.append("\\\\U");
+      String s = Integer.toHexString(c);
+      if (c < 0x10) b.append("0000000").append(s);
+      else if (c < 0x100) b.append("000000").append(s);
+      else if (c < 0x1000) b.append("00000").append(s);
+      else if (c < 0x10000) b.append("0000").append(s);
+      else if (c < 0x100000) b.append("000").append(s);
+      else if (c < 0x1000000) b.append("00").append(s);
+      else if (c < 0x10000000) b.append("0").append(s);
+      else b.append(s);
     }
-    clearNumberedStates();
   }
-  
-  /**
-   * Restores representation invariant. This method must be invoked before any
-   * built-in automata operation is performed if automaton states or transitions
-   * are manipulated manually.
-   * 
-   * @see #setDeterministic(boolean)
-   */
-  public void restoreInvariant() {
-    removeDeadTransitions();
+
+  /*
+  public void writeDot(String fileName) {
+    if (fileName.indexOf('/') == -1) {
+      fileName = "/l/la/lucene/core/" + fileName + ".dot";
+    }
+    try {
+      PrintWriter pw = new PrintWriter(fileName);
+      pw.println(toDot());
+      pw.close();
+    } catch (IOException ioe) {
+      throw new RuntimeException(ioe);
+    }
   }
-  
-  /**
-   * Reduces this automaton. An automaton is "reduced" by combining overlapping
-   * and adjacent edge intervals with same destination.
-   */
-  public void reduce() {
-    final State[] states = getNumberedStates();
-    if (isSingleton()) return;
-    for (State s : states)
-      s.reduce();
+  */
+
+  /** Returns the dot (graphviz) representation of this automaton.
+   *  This is extremely useful for visualizing the automaton. */
+  public String toDot() {
+    // TODO: breadth first search so we can see get layered output...
+
+    StringBuilder b = new StringBuilder();
+    b.append("digraph Automaton {\n");
+    b.append("  rankdir = LR\n");
+    final int numStates = getNumStates();
+    if (numStates > 0) {
+      b.append("  initial [shape=plaintext,label=\"0\"]\n");
+      b.append("  initial -> 0\n");
+    }
+
+    Transition t = new Transition();
+
+    for(int state=0;state<numStates;state++) {
+      b.append("  ");
+      b.append(state);
+      if (isAccept(state)) {
+        b.append(" [shape=doublecircle,label=\"" + state + "\"]\n");
+      } else {
+        b.append(" [shape=circle,label=\"" + state + "\"]\n");
+      }
+      int numTransitions = getNumTransitions(state);
+      initTransition(state, t);
+      //System.out.println("toDot: state " + state + " has " + numTransitions + " transitions; t.nextTrans=" + t.transitionUpto);
+      for(int i=0;i<numTransitions;i++) {
+        getNextTransition(t);
+        //System.out.println("  t.nextTrans=" + t.transitionUpto);
+        assert t.max >= t.min;
+        b.append("  ");
+        b.append(state);
+        b.append(" -> ");
+        b.append(t.dest);
+        b.append(" [label=\"");
+        appendCharString(t.min, b);
+        if (t.max != t.min) {
+          b.append('-');
+          appendCharString(t.max, b);
+        }
+        b.append("\"]\n");
+        //System.out.println("  t=" + t);
+      }
+    }
+    b.append('}');
+    return b.toString();
   }
-  
+
   /**
    * Returns sorted array of all interval start points.
    */
   int[] getStartPoints() {
-    final State[] states = getNumberedStates();
     Set<Integer> pointset = new HashSet<>();
     pointset.add(Character.MIN_CODE_POINT);
-    for (State s : states) {
-      for (Transition t : s.getTransitions()) {
-        pointset.add(t.min);
-        if (t.max < Character.MAX_CODE_POINT) pointset.add((t.max + 1));
+    //System.out.println("getStartPoints");
+    for (int s=0;s<nextState;s+=2) {
+      int trans = states[s];
+      int limit = trans+3*states[s+1];
+      //System.out.println("  state=" + (s/2) + " trans=" + trans + " limit=" + limit);
+      while (trans < limit) {
+        int min = transitions[trans+1];
+        int max = transitions[trans+2];
+        //System.out.println("    min=" + min);
+        pointset.add(min);
+        if (max < Character.MAX_CODE_POINT) {
+          pointset.add(max + 1);
+        }
+        trans += 3;
       }
     }
     int[] points = new int[pointset.size()];
     int n = 0;
-    for (Integer m : pointset)
+    for (Integer m : pointset) {
       points[n++] = m;
+    }
     Arrays.sort(points);
     return points;
   }
-  
+
   /**
-   * Returns the set of live states. A state is "live" if an accept state is
-   * reachable from it.
+   * Performs lookup in transitions, assuming determinism.
    * 
-   * @return set of {@link State} objects
+   * @param state starting state
+   * @param label codepoint to look up
+   * @return destination state, -1 if no matching outgoing transition
    */
-  private State[] getLiveStates() {
-    final State[] states = getNumberedStates();
-    Set<State> live = new HashSet<>();
-    for (State q : states) {
-      if (q.isAccept()) {
-        live.add(q);
+  public int step(int state, int label) {
+    assert state >= 0;
+    assert label >= 0;
+    int trans = states[2*state];
+    int limit = trans + 3*states[2*state+1];
+    // TODO: we could do bin search; transitions are sorted
+    while (trans < limit) {
+      int dest = transitions[trans];
+      int min = transitions[trans+1];
+      int max = transitions[trans+2];
+      if (min <= label && label <= max) {
+        return dest;
       }
-    }
-    // map<state, set<state>>
-    @SuppressWarnings({"rawtypes","unchecked"}) Set<State> map[] = new Set[states.length];
-    for (int i = 0; i < map.length; i++)
-      map[i] = new HashSet<>();
-    for (State s : states) {
-      for(int i=0;i<s.numTransitions;i++) {
-        map[s.transitionsArray[i].to.number].add(s);
-      }
-    }
-    LinkedList<State> worklist = new LinkedList<>(live);
-    while (worklist.size() > 0) {
-      State s = worklist.removeFirst();
-      for (State p : map[s.number])
-        if (!live.contains(p)) {
-          live.add(p);
-          worklist.add(p);
-        }
+      trans += 3;
     }
 
-    return live.toArray(new State[live.size()]);
+    return -1;
   }
 
-  /**
-   * Removes transitions to dead states and calls {@link #reduce()}.
-   * (A state is "dead" if no accept state is
-   * reachable from it.)
-   */
-  public void removeDeadTransitions() {
-    final State[] states = getNumberedStates();
-    //clearHashCode();
-    if (isSingleton()) return;
-    State[] live = getLiveStates();
+  /** Records new states and transitions and then {@link
+   *  #finish} creates the {@link Automaton}.  Use this
+   *  when you cannot create the Automaton directly because
+   *  it's too restrictive to have to add all transitions
+   *  leaving each state at once. */
+  public static class Builder {
+    private int[] transitions = new int[4];
+    private int nextTransition;
+    private final Automaton a = new Automaton();
 
-    BitSet liveSet = new BitSet(states.length);
-    for (State s : live)
-      liveSet.set(s.number);
+    /** Sole constructor. */
+    public Builder() {
+    }
 
-    for (State s : states) {
-      // filter out transitions to dead states:
+    /** Add a new transition with min = max = label. */
+    public void addTransition(int source, int dest, int label) {
+      addTransition(source, dest, label, label);
+    }
+
+    /** Add a new transition with the specified source, dest, min, max. */
+    public void addTransition(int source, int dest, int min, int max) {
+      if (transitions.length < nextTransition+4) {
+        transitions = ArrayUtil.grow(transitions, nextTransition+4);
+      }
+      transitions[nextTransition++] = source;
+      transitions[nextTransition++] = dest;
+      transitions[nextTransition++] = min;
+      transitions[nextTransition++] = max;
+    }
+
+    /** Sorts transitions first then min label ascending, then
+     *  max label ascending, then dest ascending */
+    private final Sorter sorter = new InPlaceMergeSorter() {
+
+        private void swapOne(int i, int j) {
+          int x = transitions[i];
+          transitions[i] = transitions[j];
+          transitions[j] = x;
+        }
+
+        @Override
+        protected void swap(int i, int j) {
+          int iStart = 4*i;
+          int jStart = 4*j;
+          swapOne(iStart, jStart);
+          swapOne(iStart+1, jStart+1);
+          swapOne(iStart+2, jStart+2);
+          swapOne(iStart+3, jStart+3);
+        };
+
+        @Override
+        protected int compare(int i, int j) {
+          int iStart = 4*i;
+          int jStart = 4*j;
+
+          // First src:
+          int iSrc = transitions[iStart];
+          int jSrc = transitions[jStart];
+          if (iSrc < jSrc) {
+            return -1;
+          } else if (iSrc > jSrc) {
+            return 1;
+          }
+
+          // Then min:
+          int iMin = transitions[iStart+2];
+          int jMin = transitions[jStart+2];
+          if (iMin < jMin) {
+            return -1;
+          } else if (iMin > jMin) {
+            return 1;
+          }
+
+          // Then max:
+          int iMax = transitions[iStart+3];
+          int jMax = transitions[jStart+3];
+          if (iMax < jMax) {
+            return -1;
+          } else if (iMax > jMax) {
+            return 1;
+          }
+
+          // First dest:
+          int iDest = transitions[iStart+1];
+          int jDest = transitions[jStart+1];
+          if (iDest < jDest) {
+            return -1;
+          } else if (iDest > jDest) {
+            return 1;
+          }
+
+          return 0;
+        }
+      };
+
+    /** Compiles all added states and transitions into a new {@code Automaton}
+     *  and returns it. */
+    public Automaton finish() {
+      //System.out.println("LA.Builder.finish: count=" + (nextTransition/4));
+      // TODO: we could make this more efficient,
+      // e.g. somehow xfer the int[] to the automaton, or
+      // alloc exactly the right size from the automaton
+      //System.out.println("finish pending");
+      sorter.sort(0, nextTransition/4);
       int upto = 0;
-      for(int i=0;i<s.numTransitions;i++) {
-        final Transition t = s.transitionsArray[i];
-        if (liveSet.get(t.to.number)) {
-          s.transitionsArray[upto++] = s.transitionsArray[i];
-        }
+      while (upto < nextTransition) {
+        a.addTransition(transitions[upto],
+                        transitions[upto+1],
+                        transitions[upto+2],
+                        transitions[upto+3]);
+        upto += 4;
       }
-      s.numTransitions = upto;
-    }
-    for(int i=0;i<live.length;i++) {
-      live[i].number = i;
-    }
-    if (live.length > 0) {
-      setNumberedStates(live);
-    } else {
-      // sneaky corner case -- if machine accepts no strings
-      clearNumberedStates();
-    }
-    reduce();
-  }
-  
-  /**
-   * Returns a sorted array of transitions for each state (and sets state
-   * numbers).
-   */
-  public Transition[][] getSortedTransitions() {
-    final State[] states = getNumberedStates();
-    Transition[][] transitions = new Transition[states.length][];
-    for (State s : states) {
-      s.sortTransitions(Transition.CompareByMinMaxThenDest);
-      s.trimTransitionsArray();
-      transitions[s.number] = s.transitionsArray;
-      assert s.transitionsArray != null;
-    }
-    return transitions;
-  }
-  
-  /**
-   * Expands singleton representation to normal representation. Does nothing if
-   * not in singleton representation.
-   */
-  public void expandSingleton() {
-    if (isSingleton()) {
-      State p = new State();
-      initial = p;
-      for (int i = 0, cp = 0; i < singleton.length(); i += Character.charCount(cp)) {
-        State q = new State();
-        p.addTransition(new Transition(cp = singleton.codePointAt(i), q));
-        p = q;
-      }
-      p.accept = true;
-      deterministic = true;
-      singleton = null;
-    }
-  }
-  
-  /**
-   * Returns the number of states in this automaton.
-   */
-  public int getNumberOfStates() {
-    if (isSingleton()) return singleton.codePointCount(0, singleton.length()) + 1;
-    return getNumberedStates().length;
-  }
-  
-  /**
-   * Returns the number of transitions in this automaton. This number is counted
-   * as the total number of edges, where one edge may be a character interval.
-   */
-  public int getNumberOfTransitions() {
-    if (isSingleton()) return singleton.codePointCount(0, singleton.length());
-    int c = 0;
-    for (State s : getNumberedStates())
-      c += s.numTransitions();
-    return c;
-  }
-  
-  @Override
-  public boolean equals(Object obj) {
-    throw new UnsupportedOperationException("use BasicOperations.sameLanguage instead");
-  }
 
-  @Override
-  public int hashCode() {
-    throw new UnsupportedOperationException();
-  }
-  
-  /**
-   * Must be invoked when the stored hash code may no longer be valid.
-   */
-  /*
-  void clearHashCode() {
-    hash_code = 0;
-  }
-  */
-  
-  /**
-   * Returns a string representation of this automaton.
-   */
-  @Override
-  public String toString() {
-    StringBuilder b = new StringBuilder();
-    if (isSingleton()) {
-      b.append("singleton: ");
-      int length = singleton.codePointCount(0, singleton.length());
-      int codepoints[] = new int[length];
-      for (int i = 0, j = 0, cp = 0; i < singleton.length(); i += Character.charCount(cp))
-        codepoints[j++] = cp = singleton.codePointAt(i);
-      for (int c : codepoints)
-        Transition.appendCharString(c, b);
-      b.append("\n");
-    } else {
-      State[] states = getNumberedStates();
-      b.append("initial state: ").append(initial.number).append("\n");
-      for (State s : states)
-        b.append(s.toString());
+      a.finishState();
+      return a;
     }
-    return b.toString();
-  }
-  
-  /**
-   * Returns <a href="http://www.research.att.com/sw/tools/graphviz/"
-   * target="_top">Graphviz Dot</a> representation of this automaton.
-   */
-  public String toDot() {
-    StringBuilder b = new StringBuilder("digraph Automaton {\n");
-    b.append("  rankdir = LR;\n");
-    State[] states = getNumberedStates();
-    for (State s : states) {
-      b.append("  ").append(s.number);
-      if (s.accept) b.append(" [shape=doublecircle,label=\"" + s.number + "\"];\n");
-      else b.append(" [shape=circle,label=\" " + s.number + "\"];\n");
-      if (s == initial) {
-        b.append("  initial [shape=plaintext,label=\"\"];\n");
-        b.append("  initial -> ").append(s.number).append("\n");
-      }
-      for (Transition t : s.getTransitions()) {
-        b.append("  ").append(s.number);
-        t.appendDot(b);
-      }
+
+    /** Create a new state. */
+    public int createState() {
+      return a.createState();
     }
-    return b.append("}\n").toString();
-  }
-  
-  /**
-   * Returns a clone of this automaton, expands if singleton.
-   */
-  Automaton cloneExpanded() {
-    Automaton a = clone();
-    a.expandSingleton();
-    return a;
-  }
-  
-  /**
-   * Returns a clone of this automaton unless <code>allow_mutation</code> is
-   * set, expands if singleton.
-   */
-  Automaton cloneExpandedIfRequired() {
-    if (allow_mutation) {
-      expandSingleton();
-      return this;
-    } else return cloneExpanded();
-  }
-  
-  /**
-   * Returns a clone of this automaton.
-   */
-  @Override
-  public Automaton clone() {
-    try {
-      Automaton a = (Automaton) super.clone();
-      if (!isSingleton()) {
-        HashMap<State,State> m = new HashMap<>();
-        State[] states = getNumberedStates();
-        for (State s : states)
-          m.put(s, new State());
-        for (State s : states) {
-          State p = m.get(s);
-          p.accept = s.accept;
-          if (s == initial) a.initial = p;
-          for (Transition t : s.getTransitions())
-            p.addTransition(new Transition(t.min, t.max, m.get(t.to)));
+
+    /** Set or clear this state as an accept state. */
+    public void setAccept(int state, boolean accept) {
+      a.setAccept(state, accept);
+    }
+
+    /** Returns true if this state is an accept state. */
+    public boolean isAccept(int state) {
+      return a.isAccept(state);
+    }
+
+    /** How many states this automaton has. */
+    public int getNumStates() {
+      return a.getNumStates();
+    }
+
+    /** Copies over all states/transitions from other. */
+    public void copy(Automaton other) {
+      int offset = getNumStates();
+      int otherNumStates = other.getNumStates();
+      for(int s=0;s<otherNumStates;s++) {
+        int newState = createState();
+        setAccept(newState, other.isAccept(s));
+      }
+      Transition t = new Transition();
+      for(int s=0;s<otherNumStates;s++) {
+        int count = other.initTransition(s, t);
+        for(int i=0;i<count;i++) {
+          other.getNextTransition(t);
+          addTransition(offset + s, offset + t.dest, t.min, t.max);
         }
       }
-      a.clearNumberedStates();
-      return a;
-    } catch (CloneNotSupportedException e) {
-      throw new RuntimeException(e);
     }
   }
-  
-  /**
-   * Returns a clone of this automaton, or this automaton itself if
-   * <code>allow_mutation</code> flag is set.
-   */
-  Automaton cloneIfRequired() {
-    if (allow_mutation) return this;
-    else return clone();
-  }
-  
-  /**
-   * See {@link BasicOperations#concatenate(Automaton, Automaton)}.
-   */
-  public Automaton concatenate(Automaton a) {
-    return BasicOperations.concatenate(this, a);
-  }
-  
-  /**
-   * See {@link BasicOperations#concatenate(List)}.
-   */
-  static public Automaton concatenate(List<Automaton> l) {
-    return BasicOperations.concatenate(l);
-  }
-  
-  /**
-   * See {@link BasicOperations#optional(Automaton)}.
-   */
-  public Automaton optional() {
-    return BasicOperations.optional(this);
-  }
-  
-  /**
-   * See {@link BasicOperations#repeat(Automaton)}.
-   */
-  public Automaton repeat() {
-    return BasicOperations.repeat(this);
-  }
-  
-  /**
-   * See {@link BasicOperations#repeat(Automaton, int)}.
-   */
-  public Automaton repeat(int min) {
-    return BasicOperations.repeat(this, min);
-  }
-  
-  /**
-   * See {@link BasicOperations#repeat(Automaton, int, int)}.
-   */
-  public Automaton repeat(int min, int max) {
-    return BasicOperations.repeat(this, min, max);
-  }
-  
-  /**
-   * See {@link BasicOperations#complement(Automaton)}.
-   */
-  public Automaton complement() {
-    return BasicOperations.complement(this);
-  }
-  
-  /**
-   * See {@link BasicOperations#minus(Automaton, Automaton)}.
-   */
-  public Automaton minus(Automaton a) {
-    return BasicOperations.minus(this, a);
-  }
-  
-  /**
-   * See {@link BasicOperations#intersection(Automaton, Automaton)}.
-   */
-  public Automaton intersection(Automaton a) {
-    return BasicOperations.intersection(this, a);
-  }
-  
-  /**
-   * See {@link BasicOperations#subsetOf(Automaton, Automaton)}.
-   */
-  public boolean subsetOf(Automaton a) {
-    return BasicOperations.subsetOf(this, a);
-  }
-  
-  /**
-   * See {@link BasicOperations#union(Automaton, Automaton)}.
-   */
-  public Automaton union(Automaton a) {
-    return BasicOperations.union(this, a);
-  }
-  
-  /**
-   * See {@link BasicOperations#union(Collection)}.
-   */
-  static public Automaton union(Collection<Automaton> l) {
-    return BasicOperations.union(l);
-  }
-  
-  /**
-   * See {@link BasicOperations#determinize(Automaton)}.
-   */
-  public void determinize() {
-    BasicOperations.determinize(this);
-  }
-  
-  /**
-   * See {@link BasicOperations#isEmptyString(Automaton)}.
-   */
-  public boolean isEmptyString() {
-    return BasicOperations.isEmptyString(this);
-  }
-  
-  /**
-   * See {@link MinimizationOperations#minimize(Automaton)}. Returns the
-   * automaton being given as argument.
-   */
-  public static Automaton minimize(Automaton a) {
-    MinimizationOperations.minimize(a);
-    return a;
-  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/BasicAutomata.java b/lucene/core/src/java/org/apache/lucene/util/automaton/BasicAutomata.java
deleted file mode 100644
index 7f51a37..0000000
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/BasicAutomata.java
+++ /dev/null
@@ -1,278 +0,0 @@
-/*
- * dk.brics.automaton
- * 
- * Copyright (c) 2001-2009 Anders Moeller
- * All rights reserved.
- * 
- * Redistribution and use in source and binary forms, with or without
- * modification, are permitted provided that the following conditions
- * are met:
- * 1. Redistributions of source code must retain the above copyright
- *    notice, this list of conditions and the following disclaimer.
- * 2. Redistributions in binary form must reproduce the above copyright
- *    notice, this list of conditions and the following disclaimer in the
- *    documentation and/or other materials provided with the distribution.
- * 3. The name of the author may not be used to endorse or promote products
- *    derived from this software without specific prior written permission.
- * 
- * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
- * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
- * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
- * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
- * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
- * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
- * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
- */
-
-package org.apache.lucene.util.automaton;
-
-import java.util.*;
-
-import org.apache.lucene.util.BytesRef;
-
-/**
- * Construction of basic automata.
- * 
- * @lucene.experimental
- */
-final public class BasicAutomata {
-  
-  private BasicAutomata() {}
-  
-  /**
-   * Returns a new (deterministic) automaton with the empty language.
-   */
-  public static Automaton makeEmpty() {
-    Automaton a = new Automaton();
-    State s = new State();
-    a.initial = s;
-    a.deterministic = true;
-    return a;
-  }
-  
-  /**
-   * Returns a new (deterministic) automaton that accepts only the empty string.
-   */
-  public static Automaton makeEmptyString() {
-    Automaton a = new Automaton();
-    a.singleton = "";
-    a.deterministic = true;
-    return a;
-  }
-  
-  /**
-   * Returns a new (deterministic) automaton that accepts all strings.
-   */
-  public static Automaton makeAnyString() {
-    Automaton a = new Automaton();
-    State s = new State();
-    a.initial = s;
-    s.accept = true;
-    s.addTransition(new Transition(Character.MIN_CODE_POINT, Character.MAX_CODE_POINT,
-        s));
-    a.deterministic = true;
-    return a;
-  }
-  
-  /**
-   * Returns a new (deterministic) automaton that accepts any single codepoint.
-   */
-  public static Automaton makeAnyChar() {
-    return makeCharRange(Character.MIN_CODE_POINT, Character.MAX_CODE_POINT);
-  }
-  
-  /**
-   * Returns a new (deterministic) automaton that accepts a single codepoint of
-   * the given value.
-   */
-  public static Automaton makeChar(int c) {
-    Automaton a = new Automaton();
-    a.singleton = new String(Character.toChars(c));
-    a.deterministic = true;
-    return a;
-  }
-  
-  /**
-   * Returns a new (deterministic) automaton that accepts a single codepoint whose
-   * value is in the given interval (including both end points).
-   */
-  public static Automaton makeCharRange(int min, int max) {
-    if (min == max) return makeChar(min);
-    Automaton a = new Automaton();
-    State s1 = new State();
-    State s2 = new State();
-    a.initial = s1;
-    s2.accept = true;
-    if (min <= max) s1.addTransition(new Transition(min, max, s2));
-    a.deterministic = true;
-    return a;
-  }
-  
-  /**
-   * Constructs sub-automaton corresponding to decimal numbers of length
-   * x.substring(n).length().
-   */
-  private static State anyOfRightLength(String x, int n) {
-    State s = new State();
-    if (x.length() == n) s.setAccept(true);
-    else s.addTransition(new Transition('0', '9', anyOfRightLength(x, n + 1)));
-    return s;
-  }
-  
-  /**
-   * Constructs sub-automaton corresponding to decimal numbers of value at least
-   * x.substring(n) and length x.substring(n).length().
-   */
-  private static State atLeast(String x, int n, Collection<State> initials,
-      boolean zeros) {
-    State s = new State();
-    if (x.length() == n) s.setAccept(true);
-    else {
-      if (zeros) initials.add(s);
-      char c = x.charAt(n);
-      s.addTransition(new Transition(c, atLeast(x, n + 1, initials, zeros
-          && c == '0')));
-      if (c < '9') s.addTransition(new Transition((char) (c + 1), '9',
-          anyOfRightLength(x, n + 1)));
-    }
-    return s;
-  }
-  
-  /**
-   * Constructs sub-automaton corresponding to decimal numbers of value at most
-   * x.substring(n) and length x.substring(n).length().
-   */
-  private static State atMost(String x, int n) {
-    State s = new State();
-    if (x.length() == n) s.setAccept(true);
-    else {
-      char c = x.charAt(n);
-      s.addTransition(new Transition(c, atMost(x, (char) n + 1)));
-      if (c > '0') s.addTransition(new Transition('0', (char) (c - 1),
-          anyOfRightLength(x, n + 1)));
-    }
-    return s;
-  }
-  
-  /**
-   * Constructs sub-automaton corresponding to decimal numbers of value between
-   * x.substring(n) and y.substring(n) and of length x.substring(n).length()
-   * (which must be equal to y.substring(n).length()).
-   */
-  private static State between(String x, String y, int n,
-      Collection<State> initials, boolean zeros) {
-    State s = new State();
-    if (x.length() == n) s.setAccept(true);
-    else {
-      if (zeros) initials.add(s);
-      char cx = x.charAt(n);
-      char cy = y.charAt(n);
-      if (cx == cy) s.addTransition(new Transition(cx, between(x, y, n + 1,
-          initials, zeros && cx == '0')));
-      else { // cx<cy
-        s.addTransition(new Transition(cx, atLeast(x, n + 1, initials, zeros
-            && cx == '0')));
-        s.addTransition(new Transition(cy, atMost(y, n + 1)));
-        if (cx + 1 < cy) s.addTransition(new Transition((char) (cx + 1),
-            (char) (cy - 1), anyOfRightLength(x, n + 1)));
-      }
-    }
-    return s;
-  }
-  
-  /**
-   * Returns a new automaton that accepts strings representing decimal
-   * non-negative integers in the given interval.
-   * 
-   * @param min minimal value of interval
-   * @param max maximal value of interval (both end points are included in the
-   *          interval)
-   * @param digits if >0, use fixed number of digits (strings must be prefixed
-   *          by 0's to obtain the right length) - otherwise, the number of
-   *          digits is not fixed
-   * @exception IllegalArgumentException if min>max or if numbers in the
-   *              interval cannot be expressed with the given fixed number of
-   *              digits
-   */
-  public static Automaton makeInterval(int min, int max, int digits)
-      throws IllegalArgumentException {
-    Automaton a = new Automaton();
-    String x = Integer.toString(min);
-    String y = Integer.toString(max);
-    if (min > max || (digits > 0 && y.length() > digits)) throw new IllegalArgumentException();
-    int d;
-    if (digits > 0) d = digits;
-    else d = y.length();
-    StringBuilder bx = new StringBuilder();
-    for (int i = x.length(); i < d; i++)
-      bx.append('0');
-    bx.append(x);
-    x = bx.toString();
-    StringBuilder by = new StringBuilder();
-    for (int i = y.length(); i < d; i++)
-      by.append('0');
-    by.append(y);
-    y = by.toString();
-    Collection<State> initials = new ArrayList<>();
-    a.initial = between(x, y, 0, initials, digits <= 0);
-    if (digits <= 0) {
-      ArrayList<StatePair> pairs = new ArrayList<>();
-      for (State p : initials)
-        if (a.initial != p) pairs.add(new StatePair(a.initial, p));
-      BasicOperations.addEpsilons(a, pairs);
-      a.initial.addTransition(new Transition('0', a.initial));
-      a.deterministic = false;
-    } else a.deterministic = true;
-    a.checkMinimizeAlways();
-    return a;
-  }
-  
-  /**
-   * Returns a new (deterministic) automaton that accepts the single given
-   * string.
-   */
-  public static Automaton makeString(String s) {
-    Automaton a = new Automaton();
-    a.singleton = s;
-    a.deterministic = true;
-    return a;
-  }
-  
-  public static Automaton makeString(int[] word, int offset, int length) {
-    Automaton a = new Automaton();
-    a.setDeterministic(true);
-    State s = new State();
-    a.initial = s;
-    for (int i = offset; i < offset+length; i++) {
-      State s2 = new State();
-      s.addTransition(new Transition(word[i], s2));
-      s = s2;
-    }
-    s.accept = true;
-    return a;
-  }
-
-  /**
-   * Returns a new (deterministic and minimal) automaton that accepts the union
-   * of the given collection of {@link BytesRef}s representing UTF-8 encoded
-   * strings.
-   * 
-   * @param utf8Strings
-   *          The input strings, UTF-8 encoded. The collection must be in sorted
-   *          order.
-   * 
-   * @return An {@link Automaton} accepting all input strings. The resulting
-   *         automaton is codepoint based (full unicode codepoints on
-   *         transitions).
-   */
-  public static Automaton makeStringUnion(Collection<BytesRef> utf8Strings) {
-    if (utf8Strings.isEmpty()) {
-      return makeEmpty();
-    } else {
-      return DaciukMihovAutomatonBuilder.build(utf8Strings);
-    }
-  }
-}
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/BasicOperations.java b/lucene/core/src/java/org/apache/lucene/util/automaton/BasicOperations.java
deleted file mode 100644
index 1d95358..0000000
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/BasicOperations.java
+++ /dev/null
@@ -1,853 +0,0 @@
-/*
- * dk.brics.automaton
- * 
- * Copyright (c) 2001-2009 Anders Moeller
- * All rights reserved.
- * 
- * Redistribution and use in source and binary forms, with or without
- * modification, are permitted provided that the following conditions
- * are met:
- * 1. Redistributions of source code must retain the above copyright
- *    notice, this list of conditions and the following disclaimer.
- * 2. Redistributions in binary form must reproduce the above copyright
- *    notice, this list of conditions and the following disclaimer in the
- *    documentation and/or other materials provided with the distribution.
- * 3. The name of the author may not be used to endorse or promote products
- *    derived from this software without specific prior written permission.
- * 
- * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
- * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
- * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
- * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
- * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
- * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
- * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
- */
-
-package org.apache.lucene.util.automaton;
-
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.RamUsageEstimator;
-
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Basic automata operations.
- * 
- * @lucene.experimental
- */
-final public class BasicOperations {
-  
-  private BasicOperations() {}
-  
-  /**
-   * Returns an automaton that accepts the concatenation of the languages of the
-   * given automata.
-   * <p>
-   * Complexity: linear in number of states.
-   */
-  static public Automaton concatenate(Automaton a1, Automaton a2) {
-    if (a1.isSingleton() && a2.isSingleton()) return BasicAutomata
-        .makeString(a1.singleton + a2.singleton);
-    if (isEmpty(a1) || isEmpty(a2))
-      return BasicAutomata.makeEmpty();
-    // adding epsilon transitions with the NFA concatenation algorithm
-    // in this case always produces a resulting DFA, preventing expensive
-    // redundant determinize() calls for this common case.
-    boolean deterministic = a1.isSingleton() && a2.isDeterministic();
-    if (a1 == a2) {
-      a1 = a1.cloneExpanded();
-      a2 = a2.cloneExpanded();
-    } else {
-      a1 = a1.cloneExpandedIfRequired();
-      a2 = a2.cloneExpandedIfRequired();
-    }
-    for (State s : a1.getAcceptStates()) {
-      s.accept = false;
-      s.addEpsilon(a2.initial);
-    }
-    a1.deterministic = deterministic;
-    //a1.clearHashCode();
-    a1.clearNumberedStates();
-    a1.checkMinimizeAlways();
-    return a1;
-  }
-  
-  /**
-   * Returns an automaton that accepts the concatenation of the languages of the
-   * given automata.
-   * <p>
-   * Complexity: linear in total number of states.
-   */
-  static public Automaton concatenate(List<Automaton> l) {
-    if (l.isEmpty()) return BasicAutomata.makeEmptyString();
-    boolean all_singleton = true;
-    for (Automaton a : l)
-      if (!a.isSingleton()) {
-        all_singleton = false;
-        break;
-      }
-    if (all_singleton) {
-      StringBuilder b = new StringBuilder();
-      for (Automaton a : l)
-        b.append(a.singleton);
-      return BasicAutomata.makeString(b.toString());
-    } else {
-      for (Automaton a : l)
-        if (BasicOperations.isEmpty(a)) return BasicAutomata.makeEmpty();
-      Set<Integer> ids = new HashSet<>();
-      for (Automaton a : l)
-        ids.add(System.identityHashCode(a));
-      boolean has_aliases = ids.size() != l.size();
-      Automaton b = l.get(0);
-      if (has_aliases) b = b.cloneExpanded();
-      else b = b.cloneExpandedIfRequired();
-      Set<State> ac = b.getAcceptStates();
-      boolean first = true;
-      for (Automaton a : l)
-        if (first) first = false;
-        else {
-          if (a.isEmptyString()) continue;
-          Automaton aa = a;
-          if (has_aliases) aa = aa.cloneExpanded();
-          else aa = aa.cloneExpandedIfRequired();
-          Set<State> ns = aa.getAcceptStates();
-          for (State s : ac) {
-            s.accept = false;
-            s.addEpsilon(aa.initial);
-            if (s.accept) ns.add(s);
-          }
-          ac = ns;
-        }
-      b.deterministic = false;
-      //b.clearHashCode();
-      b.clearNumberedStates();
-      b.checkMinimizeAlways();
-      return b;
-    }
-  }
-  
-  /**
-   * Returns an automaton that accepts the union of the empty string and the
-   * language of the given automaton.
-   * <p>
-   * Complexity: linear in number of states.
-   */
-  static public Automaton optional(Automaton a) {
-    a = a.cloneExpandedIfRequired();
-    State s = new State();
-    s.addEpsilon(a.initial);
-    s.accept = true;
-    a.initial = s;
-    a.deterministic = false;
-    //a.clearHashCode();
-    a.clearNumberedStates();
-    a.checkMinimizeAlways();
-    return a;
-  }
-  
-  /**
-   * Returns an automaton that accepts the Kleene star (zero or more
-   * concatenated repetitions) of the language of the given automaton. Never
-   * modifies the input automaton language.
-   * <p>
-   * Complexity: linear in number of states.
-   */
-  static public Automaton repeat(Automaton a) {
-    a = a.cloneExpanded();
-    State s = new State();
-    s.accept = true;
-    s.addEpsilon(a.initial);
-    for (State p : a.getAcceptStates())
-      p.addEpsilon(s);
-    a.initial = s;
-    a.deterministic = false;
-    //a.clearHashCode();
-    a.clearNumberedStates();
-    a.checkMinimizeAlways();
-    return a;
-  }
-  
-  /**
-   * Returns an automaton that accepts <code>min</code> or more concatenated
-   * repetitions of the language of the given automaton.
-   * <p>
-   * Complexity: linear in number of states and in <code>min</code>.
-   */
-  static public Automaton repeat(Automaton a, int min) {
-    if (min == 0) return repeat(a);
-    List<Automaton> as = new ArrayList<>();
-    while (min-- > 0)
-      as.add(a);
-    as.add(repeat(a));
-    return concatenate(as);
-  }
-  
-  /**
-   * Returns an automaton that accepts between <code>min</code> and
-   * <code>max</code> (including both) concatenated repetitions of the language
-   * of the given automaton.
-   * <p>
-   * Complexity: linear in number of states and in <code>min</code> and
-   * <code>max</code>.
-   */
-  static public Automaton repeat(Automaton a, int min, int max) {
-    if (min > max) return BasicAutomata.makeEmpty();
-    max -= min;
-    a.expandSingleton();
-    Automaton b;
-    if (min == 0) b = BasicAutomata.makeEmptyString();
-    else if (min == 1) b = a.clone();
-    else {
-      List<Automaton> as = new ArrayList<>();
-      while (min-- > 0)
-        as.add(a);
-      b = concatenate(as);
-    }
-    if (max > 0) {
-      Automaton d = a.clone();
-      while (--max > 0) {
-        Automaton c = a.clone();
-        for (State p : c.getAcceptStates())
-          p.addEpsilon(d.initial);
-        d = c;
-      }
-      for (State p : b.getAcceptStates())
-        p.addEpsilon(d.initial);
-      b.deterministic = false;
-      //b.clearHashCode();
-      b.clearNumberedStates();
-      b.checkMinimizeAlways();
-    }
-    return b;
-  }
-  
-  /**
-   * Returns a (deterministic) automaton that accepts the complement of the
-   * language of the given automaton.
-   * <p>
-   * Complexity: linear in number of states (if already deterministic).
-   */
-  static public Automaton complement(Automaton a) {
-    a = a.cloneExpandedIfRequired();
-    a.determinize();
-    a.totalize();
-    for (State p : a.getNumberedStates())
-      p.accept = !p.accept;
-    a.removeDeadTransitions();
-    return a;
-  }
-  
-  /**
-   * Returns a (deterministic) automaton that accepts the intersection of the
-   * language of <code>a1</code> and the complement of the language of
-   * <code>a2</code>. As a side-effect, the automata may be determinized, if not
-   * already deterministic.
-   * <p>
-   * Complexity: quadratic in number of states (if already deterministic).
-   */
-  static public Automaton minus(Automaton a1, Automaton a2) {
-    if (BasicOperations.isEmpty(a1) || a1 == a2) return BasicAutomata
-        .makeEmpty();
-    if (BasicOperations.isEmpty(a2)) return a1.cloneIfRequired();
-    if (a1.isSingleton()) {
-      if (BasicOperations.run(a2, a1.singleton)) return BasicAutomata.makeEmpty();
-      else return a1.cloneIfRequired();
-    }
-    return intersection(a1, a2.complement());
-  }
-  
-  /**
-   * Returns an automaton that accepts the intersection of the languages of the
-   * given automata. Never modifies the input automata languages.
-   * <p>
-   * Complexity: quadratic in number of states.
-   */
-  static public Automaton intersection(Automaton a1, Automaton a2) {
-    if (a1.isSingleton()) {
-      if (BasicOperations.run(a2, a1.singleton)) return a1.cloneIfRequired();
-      else return BasicAutomata.makeEmpty();
-    }
-    if (a2.isSingleton()) {
-      if (BasicOperations.run(a1, a2.singleton)) return a2.cloneIfRequired();
-      else return BasicAutomata.makeEmpty();
-    }
-    if (a1 == a2) return a1.cloneIfRequired();
-    Transition[][] transitions1 = a1.getSortedTransitions();
-    Transition[][] transitions2 = a2.getSortedTransitions();
-    Automaton c = new Automaton();
-    LinkedList<StatePair> worklist = new LinkedList<>();
-    HashMap<StatePair,StatePair> newstates = new HashMap<>();
-    StatePair p = new StatePair(c.initial, a1.initial, a2.initial);
-    worklist.add(p);
-    newstates.put(p, p);
-    while (worklist.size() > 0) {
-      p = worklist.removeFirst();
-      p.s.accept = p.s1.accept && p.s2.accept;
-      Transition[] t1 = transitions1[p.s1.number];
-      Transition[] t2 = transitions2[p.s2.number];
-      for (int n1 = 0, b2 = 0; n1 < t1.length; n1++) {
-        while (b2 < t2.length && t2[b2].max < t1[n1].min)
-          b2++;
-        for (int n2 = b2; n2 < t2.length && t1[n1].max >= t2[n2].min; n2++)
-          if (t2[n2].max >= t1[n1].min) {
-            StatePair q = new StatePair(t1[n1].to, t2[n2].to);
-            StatePair r = newstates.get(q);
-            if (r == null) {
-              q.s = new State();
-              worklist.add(q);
-              newstates.put(q, q);
-              r = q;
-            }
-            int min = t1[n1].min > t2[n2].min ? t1[n1].min : t2[n2].min;
-            int max = t1[n1].max < t2[n2].max ? t1[n1].max : t2[n2].max;
-            p.s.addTransition(new Transition(min, max, r.s));
-          }
-      }
-    }
-    c.deterministic = a1.deterministic && a2.deterministic;
-    c.removeDeadTransitions();
-    c.checkMinimizeAlways();
-    return c;
-  }
-
-  /** Returns true if these two automata accept exactly the
-   *  same language.  This is a costly computation!  Note
-   *  also that a1 and a2 will be determinized as a side
-   *  effect. */
-  public static boolean sameLanguage(Automaton a1, Automaton a2) {
-    if (a1 == a2) {
-      return true;
-    }
-    if (a1.isSingleton() && a2.isSingleton()) {
-      return a1.singleton.equals(a2.singleton);
-    } else if (a1.isSingleton()) {
-      // subsetOf is faster if the first automaton is a singleton
-      return subsetOf(a1, a2) && subsetOf(a2, a1);
-    } else {
-      return subsetOf(a2, a1) && subsetOf(a1, a2);
-    }
-  }
-  
-  /**
-   * Returns true if the language of <code>a1</code> is a subset of the language
-   * of <code>a2</code>. As a side-effect, <code>a2</code> is determinized if
-   * not already marked as deterministic.
-   * <p>
-   * Complexity: quadratic in number of states.
-   */
-  public static boolean subsetOf(Automaton a1, Automaton a2) {
-    if (a1 == a2) return true;
-    if (a1.isSingleton()) {
-      if (a2.isSingleton()) return a1.singleton.equals(a2.singleton);
-      return BasicOperations.run(a2, a1.singleton);
-    }
-    a2.determinize();
-    Transition[][] transitions1 = a1.getSortedTransitions();
-    Transition[][] transitions2 = a2.getSortedTransitions();
-    LinkedList<StatePair> worklist = new LinkedList<>();
-    HashSet<StatePair> visited = new HashSet<>();
-    StatePair p = new StatePair(a1.initial, a2.initial);
-    worklist.add(p);
-    visited.add(p);
-    while (worklist.size() > 0) {
-      p = worklist.removeFirst();
-      if (p.s1.accept && !p.s2.accept) {
-        return false;
-      }
-      Transition[] t1 = transitions1[p.s1.number];
-      Transition[] t2 = transitions2[p.s2.number];
-      for (int n1 = 0, b2 = 0; n1 < t1.length; n1++) {
-        while (b2 < t2.length && t2[b2].max < t1[n1].min)
-          b2++;
-        int min1 = t1[n1].min, max1 = t1[n1].max;
-
-        for (int n2 = b2; n2 < t2.length && t1[n1].max >= t2[n2].min; n2++) {
-          if (t2[n2].min > min1) {
-            return false;
-          }
-          if (t2[n2].max < Character.MAX_CODE_POINT) min1 = t2[n2].max + 1;
-          else {
-            min1 = Character.MAX_CODE_POINT;
-            max1 = Character.MIN_CODE_POINT;
-          }
-          StatePair q = new StatePair(t1[n1].to, t2[n2].to);
-          if (!visited.contains(q)) {
-            worklist.add(q);
-            visited.add(q);
-          }
-        }
-        if (min1 <= max1) {
-          return false;
-        }
-      }
-    }
-    return true;
-  }
-  
-  /**
-   * Returns an automaton that accepts the union of the languages of the given
-   * automata.
-   * <p>
-   * Complexity: linear in number of states.
-   */
-  public static Automaton union(Automaton a1, Automaton a2) {
-    if ((a1.isSingleton() && a2.isSingleton() && a1.singleton
-        .equals(a2.singleton))
-        || a1 == a2) return a1.cloneIfRequired();
-    if (a1 == a2) {
-      a1 = a1.cloneExpanded();
-      a2 = a2.cloneExpanded();
-    } else {
-      a1 = a1.cloneExpandedIfRequired();
-      a2 = a2.cloneExpandedIfRequired();
-    }
-    State s = new State();
-    s.addEpsilon(a1.initial);
-    s.addEpsilon(a2.initial);
-    a1.initial = s;
-    a1.deterministic = false;
-    //a1.clearHashCode();
-    a1.clearNumberedStates();
-    a1.checkMinimizeAlways();
-    return a1;
-  }
-  
-  /**
-   * Returns an automaton that accepts the union of the languages of the given
-   * automata.
-   * <p>
-   * Complexity: linear in number of states.
-   */
-  public static Automaton union(Collection<Automaton> l) {
-    Set<Integer> ids = new HashSet<>();
-    for (Automaton a : l)
-      ids.add(System.identityHashCode(a));
-    boolean has_aliases = ids.size() != l.size();
-    State s = new State();
-    for (Automaton b : l) {
-      if (BasicOperations.isEmpty(b)) continue;
-      Automaton bb = b;
-      if (has_aliases) bb = bb.cloneExpanded();
-      else bb = bb.cloneExpandedIfRequired();
-      s.addEpsilon(bb.initial);
-    }
-    Automaton a = new Automaton();
-    a.initial = s;
-    a.deterministic = false;
-    //a.clearHashCode();
-    a.clearNumberedStates();
-    a.checkMinimizeAlways();
-    return a;
-  }
-
-  // Simple custom ArrayList<Transition>
-  private final static class TransitionList {
-    Transition[] transitions = new Transition[2];
-    int count;
-
-    public void add(Transition t) {
-      if (transitions.length == count) {
-        Transition[] newArray = new Transition[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-        System.arraycopy(transitions, 0, newArray, 0, count);
-        transitions = newArray;
-      }
-      transitions[count++] = t;
-    }
-  }
-
-  // Holds all transitions that start on this int point, or
-  // end at this point-1
-  private final static class PointTransitions implements Comparable<PointTransitions> {
-    int point;
-    final TransitionList ends = new TransitionList();
-    final TransitionList starts = new TransitionList();
-    @Override
-    public int compareTo(PointTransitions other) {
-      return point - other.point;
-    }
-
-    public void reset(int point) {
-      this.point = point;
-      ends.count = 0;
-      starts.count = 0;
-    }
-
-    @Override
-    public boolean equals(Object other) {
-      return ((PointTransitions) other).point == point;
-    }
-
-    @Override
-    public int hashCode() {
-      return point;
-    }
-  }
-
-  private final static class PointTransitionSet {
-    int count;
-    PointTransitions[] points = new PointTransitions[5];
-
-    private final static int HASHMAP_CUTOVER = 30;
-    private final HashMap<Integer,PointTransitions> map = new HashMap<>();
-    private boolean useHash = false;
-
-    private PointTransitions next(int point) {
-      // 1st time we are seeing this point
-      if (count == points.length) {
-        final PointTransitions[] newArray = new PointTransitions[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-        System.arraycopy(points, 0, newArray, 0, count);
-        points = newArray;
-      }
-      PointTransitions points0 = points[count];
-      if (points0 == null) {
-        points0 = points[count] = new PointTransitions();
-      }
-      points0.reset(point);
-      count++;
-      return points0;
-    }
-
-    private PointTransitions find(int point) {
-      if (useHash) {
-        final Integer pi = point;
-        PointTransitions p = map.get(pi);
-        if (p == null) {
-          p = next(point);
-          map.put(pi, p);
-        }
-        return p;
-      } else {
-        for(int i=0;i<count;i++) {
-          if (points[i].point == point) {
-            return points[i];
-          }
-        }
-
-        final PointTransitions p = next(point);
-        if (count == HASHMAP_CUTOVER) {
-          // switch to HashMap on the fly
-          assert map.size() == 0;
-          for(int i=0;i<count;i++) {
-            map.put(points[i].point, points[i]);
-          }
-          useHash = true;
-        }
-        return p;
-      }
-    }
-
-    public void reset() {
-      if (useHash) {
-        map.clear();
-        useHash = false;
-      }
-      count = 0;
-    }
-
-    public void sort() {
-      // Tim sort performs well on already sorted arrays:
-      if (count > 1) ArrayUtil.timSort(points, 0, count);
-    }
-
-    public void add(Transition t) {
-      find(t.min).starts.add(t);
-      find(1+t.max).ends.add(t);
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder s = new StringBuilder();
-      for(int i=0;i<count;i++) {
-        if (i > 0) {
-          s.append(' ');
-        }
-        s.append(points[i].point).append(':').append(points[i].starts.count).append(',').append(points[i].ends.count);
-      }
-      return s.toString();
-    }
-  }
-
-  /**
-   * Determinizes the given automaton.
-   * <p>
-   * Worst case complexity: exponential in number of states.
-   */
-  public static void determinize(Automaton a) {
-    if (a.deterministic || a.isSingleton()) {
-      return;
-    }
-
-    final State[] allStates = a.getNumberedStates();
-
-    // subset construction
-    final boolean initAccept = a.initial.accept;
-    final int initNumber = a.initial.number;
-    a.initial = new State();
-    SortedIntSet.FrozenIntSet initialset = new SortedIntSet.FrozenIntSet(initNumber, a.initial);
-
-    LinkedList<SortedIntSet.FrozenIntSet> worklist = new LinkedList<>();
-    Map<SortedIntSet.FrozenIntSet,State> newstate = new HashMap<>();
-
-    worklist.add(initialset);
-
-    a.initial.accept = initAccept;
-    newstate.put(initialset, a.initial);
-
-    int newStateUpto = 0;
-    State[] newStatesArray = new State[5];
-    newStatesArray[newStateUpto] = a.initial;
-    a.initial.number = newStateUpto;
-    newStateUpto++;
-
-    // like Set<Integer,PointTransitions>
-    final PointTransitionSet points = new PointTransitionSet();
-
-    // like SortedMap<Integer,Integer>
-    final SortedIntSet statesSet = new SortedIntSet(5);
-
-    while (worklist.size() > 0) {
-      SortedIntSet.FrozenIntSet s = worklist.removeFirst();
-
-      // Collate all outgoing transitions by min/1+max:
-      for(int i=0;i<s.values.length;i++) {
-        final State s0 = allStates[s.values[i]];
-        for(int j=0;j<s0.numTransitions;j++) {
-          points.add(s0.transitionsArray[j]);
-        }
-      }
-
-      if (points.count == 0) {
-        // No outgoing transitions -- skip it
-        continue;
-      }
-
-      points.sort();
-
-      int lastPoint = -1;
-      int accCount = 0;
-
-      final State r = s.state;
-      for(int i=0;i<points.count;i++) {
-
-        final int point = points.points[i].point;
-
-        if (statesSet.upto > 0) {
-          assert lastPoint != -1;
-
-          statesSet.computeHash();
-          
-          State q = newstate.get(statesSet);
-          if (q == null) {
-            q = new State();
-            final SortedIntSet.FrozenIntSet p = statesSet.freeze(q);
-            worklist.add(p);
-            if (newStateUpto == newStatesArray.length) {
-              final State[] newArray = new State[ArrayUtil.oversize(1+newStateUpto, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-              System.arraycopy(newStatesArray, 0, newArray, 0, newStateUpto);
-              newStatesArray = newArray;
-            }
-            newStatesArray[newStateUpto] = q;
-            q.number = newStateUpto;
-            newStateUpto++;
-            q.accept = accCount > 0;
-            newstate.put(p, q);
-          } else {
-            assert (accCount > 0 ? true:false) == q.accept: "accCount=" + accCount + " vs existing accept=" + q.accept + " states=" + statesSet;
-          }
-
-          r.addTransition(new Transition(lastPoint, point-1, q));
-        }
-
-        // process transitions that end on this point
-        // (closes an overlapping interval)
-        Transition[] transitions = points.points[i].ends.transitions;
-        int limit = points.points[i].ends.count;
-        for(int j=0;j<limit;j++) {
-          final Transition t = transitions[j];
-          final Integer num = t.to.number;
-          statesSet.decr(num);
-          accCount -= t.to.accept ? 1:0;
-        }
-        points.points[i].ends.count = 0;
-
-        // process transitions that start on this point
-        // (opens a new interval)
-        transitions = points.points[i].starts.transitions;
-        limit = points.points[i].starts.count;
-        for(int j=0;j<limit;j++) {
-          final Transition t = transitions[j];
-          final Integer num = t.to.number;
-          statesSet.incr(num);
-          accCount += t.to.accept ? 1:0;
-        }
-        lastPoint = point;
-        points.points[i].starts.count = 0;
-      }
-      points.reset();
-      assert statesSet.upto == 0: "upto=" + statesSet.upto;
-    }
-    a.deterministic = true;
-    a.setNumberedStates(newStatesArray, newStateUpto);
-  }
-  
-  /**
-   * Adds epsilon transitions to the given automaton. This method adds extra
-   * character interval transitions that are equivalent to the given set of
-   * epsilon transitions.
-   * 
-   * @param pairs collection of {@link StatePair} objects representing pairs of
-   *          source/destination states where epsilon transitions should be
-   *          added
-   */
-  public static void addEpsilons(Automaton a, Collection<StatePair> pairs) {
-    a.expandSingleton();
-    HashMap<State,HashSet<State>> forward = new HashMap<>();
-    HashMap<State,HashSet<State>> back = new HashMap<>();
-    for (StatePair p : pairs) {
-      HashSet<State> to = forward.get(p.s1);
-      if (to == null) {
-        to = new HashSet<>();
-        forward.put(p.s1, to);
-      }
-      to.add(p.s2);
-      HashSet<State> from = back.get(p.s2);
-      if (from == null) {
-        from = new HashSet<>();
-        back.put(p.s2, from);
-      }
-      from.add(p.s1);
-    }
-    // calculate epsilon closure
-    LinkedList<StatePair> worklist = new LinkedList<>(pairs);
-    HashSet<StatePair> workset = new HashSet<>(pairs);
-    while (!worklist.isEmpty()) {
-      StatePair p = worklist.removeFirst();
-      workset.remove(p);
-      HashSet<State> to = forward.get(p.s2);
-      HashSet<State> from = back.get(p.s1);
-      if (to != null) {
-        for (State s : to) {
-          StatePair pp = new StatePair(p.s1, s);
-          if (!pairs.contains(pp)) {
-            pairs.add(pp);
-            forward.get(p.s1).add(s);
-            back.get(s).add(p.s1);
-            worklist.add(pp);
-            workset.add(pp);
-            if (from != null) {
-              for (State q : from) {
-                StatePair qq = new StatePair(q, p.s1);
-                if (!workset.contains(qq)) {
-                  worklist.add(qq);
-                  workset.add(qq);
-                }
-              }
-            }
-          }
-        }
-      }
-    }
-    // add transitions
-    for (StatePair p : pairs)
-      p.s1.addEpsilon(p.s2);
-    a.deterministic = false;
-    //a.clearHashCode();
-    a.clearNumberedStates();
-    a.checkMinimizeAlways();
-  }
-  
-  /**
-   * Returns true if the given automaton accepts the empty string and nothing
-   * else.
-   */
-  public static boolean isEmptyString(Automaton a) {
-    if (a.isSingleton()) return a.singleton.length() == 0;
-    else return a.initial.accept && a.initial.numTransitions() == 0;
-  }
-  
-  /**
-   * Returns true if the given automaton accepts no strings.
-   */
-  public static boolean isEmpty(Automaton a) {
-    if (a.isSingleton()) return false;
-    return !a.initial.accept && a.initial.numTransitions() == 0;
-  }
-  
-  /**
-   * Returns true if the given automaton accepts all strings.
-   */
-  public static boolean isTotal(Automaton a) {
-    if (a.isSingleton()) return false;
-    if (a.initial.accept && a.initial.numTransitions() == 1) {
-      Transition t = a.initial.getTransitions().iterator().next();
-      return t.to == a.initial && t.min == Character.MIN_CODE_POINT
-          && t.max == Character.MAX_CODE_POINT;
-    }
-    return false;
-  }
-  
-  /**
-   * Returns true if the given string is accepted by the automaton.
-   * <p>
-   * Complexity: linear in the length of the string.
-   * <p>
-   * <b>Note:</b> for full performance, use the {@link RunAutomaton} class.
-   */
-  public static boolean run(Automaton a, String s) {
-    if (a.isSingleton()) return s.equals(a.singleton);
-    if (a.deterministic) {
-      State p = a.initial;
-      for (int i = 0, cp = 0; i < s.length(); i += Character.charCount(cp)) {
-        State q = p.step(cp = s.codePointAt(i));
-        if (q == null) return false;
-        p = q;
-      }
-      return p.accept;
-    } else {
-      State[] states = a.getNumberedStates();
-      LinkedList<State> pp = new LinkedList<>();
-      LinkedList<State> pp_other = new LinkedList<>();
-      BitSet bb = new BitSet(states.length);
-      BitSet bb_other = new BitSet(states.length);
-      pp.add(a.initial);
-      ArrayList<State> dest = new ArrayList<>();
-      boolean accept = a.initial.accept;
-      for (int i = 0, c = 0; i < s.length(); i += Character.charCount(c)) {
-        c = s.codePointAt(i);
-        accept = false;
-        pp_other.clear();
-        bb_other.clear();
-        for (State p : pp) {
-          dest.clear();
-          p.step(c, dest);
-          for (State q : dest) {
-            if (q.accept) accept = true;
-            if (!bb_other.get(q.number)) {
-              bb_other.set(q.number);
-              pp_other.add(q);
-            }
-          }
-        }
-        LinkedList<State> tp = pp;
-        pp = pp_other;
-        pp_other = tp;
-        BitSet tb = bb;
-        bb = bb_other;
-        bb_other = tb;
-      }
-      return accept;
-    }
-  }
-}
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/ByteRunAutomaton.java b/lucene/core/src/java/org/apache/lucene/util/automaton/ByteRunAutomaton.java
index 8c8d68a..5804ef1 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/ByteRunAutomaton.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/ByteRunAutomaton.java
@@ -21,7 +21,8 @@
  * Automaton representation for matching UTF-8 byte[].
  */
 public class ByteRunAutomaton extends RunAutomaton {
-  
+
+  /** Converts incoming automaton to byte-based (UTF32ToUTF8) first */
   public ByteRunAutomaton(Automaton a) {
     this(a, false);
   }
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/CharacterRunAutomaton.java b/lucene/core/src/java/org/apache/lucene/util/automaton/CharacterRunAutomaton.java
index 2dcd922..8582870 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/CharacterRunAutomaton.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/CharacterRunAutomaton.java
@@ -22,6 +22,7 @@
  */
 public class CharacterRunAutomaton extends RunAutomaton {
 
+  /** Sole constructor. */
   public CharacterRunAutomaton(Automaton a) {
     super(a, Character.MAX_CODE_POINT, false);
   }
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java b/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
index 6d8666f..4384958 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
@@ -19,7 +19,6 @@
   
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 
 import org.apache.lucene.index.Terms;
@@ -52,6 +51,8 @@
     /** Catch-all for any other automata. */
     NORMAL
   };
+
+  /** If simplify is true this will be the "simplified" type; else, this is NORMAL */
   public final AUTOMATON_TYPE type;
 
   /** 
@@ -65,21 +66,22 @@
    * only valid for {@link AUTOMATON_TYPE#NORMAL}.
    */
   public final ByteRunAutomaton runAutomaton;
-  // TODO: would be nice if these sortedTransitions had "int
-  // to;" instead of "State to;" somehow:
+
   /**
    * Two dimensional array of transitions, indexed by state
    * number for traversal. The state numbering is consistent with
    * {@link #runAutomaton}. 
    * Only valid for {@link AUTOMATON_TYPE#NORMAL}.
    */
-  public final Transition[][] sortedTransitions;
+  public final Automaton automaton;
+
   /**
    * Shared common suffix accepted by the automaton. Only valid
    * for {@link AUTOMATON_TYPE#NORMAL}, and only when the
    * automaton accepts an infinite language.
    */
   public final BytesRef commonSuffixRef;
+
   /**
    * Indicates if the automaton accepts a finite set of strings.
    * Null if this was not computed.
@@ -87,125 +89,152 @@
    */
   public final Boolean finite;
 
+  /** Create this, passing simplify=true and finite=null, so that we try
+   *  to simplify the automaton and determine if it is finite. */
   public CompiledAutomaton(Automaton automaton) {
     this(automaton, null, true);
   }
 
+  /** Create this.  If finite is null, we use {@link Operations#isFinite}
+   *  to determine whether it is finite.  If simplify is true, we run
+   *  possibly expensive operations to determine if the automaton is one
+   *  the cases in {@link CompiledAutomaton.AUTOMATON_TYPE}. */
   public CompiledAutomaton(Automaton automaton, Boolean finite, boolean simplify) {
 
     if (simplify) {
+
       // Test whether the automaton is a "simple" form and
       // if so, don't create a runAutomaton.  Note that on a
       // large automaton these tests could be costly:
-      if (BasicOperations.isEmpty(automaton)) {
+
+      if (Operations.isEmpty(automaton)) {
         // matches nothing
         type = AUTOMATON_TYPE.NONE;
         term = null;
         commonSuffixRef = null;
         runAutomaton = null;
-        sortedTransitions = null;
+        this.automaton = null;
         this.finite = null;
         return;
-      } else if (BasicOperations.isTotal(automaton)) {
+      // NOTE: only approximate, because automaton may not be minimal:
+      } else if (Operations.isTotal(automaton)) {
         // matches all possible strings
         type = AUTOMATON_TYPE.ALL;
         term = null;
         commonSuffixRef = null;
         runAutomaton = null;
-        sortedTransitions = null;
+        this.automaton = null;
         this.finite = null;
         return;
       } else {
-        final String commonPrefix;
+
+        automaton = Operations.determinize(automaton);
+
+        final String commonPrefix = Operations.getCommonPrefix(automaton);
         final String singleton;
-        if (automaton.getSingleton() == null) {
-          commonPrefix = SpecialOperations.getCommonPrefix(automaton);
-          if (commonPrefix.length() > 0 && BasicOperations.sameLanguage(automaton, BasicAutomata.makeString(commonPrefix))) {
-            singleton = commonPrefix;
-          } else {
-            singleton = null;
-          }
+
+        if (commonPrefix.length() > 0 && Operations.sameLanguage(automaton, Automata.makeString(commonPrefix))) {
+          singleton = commonPrefix;
         } else {
-          commonPrefix = null;
-          singleton = automaton.getSingleton();
+          singleton = null;
         }
-      
+
         if (singleton != null) {
-          // matches a fixed string in singleton or expanded
-          // representation
+          // matches a fixed string
           type = AUTOMATON_TYPE.SINGLE;
           term = new BytesRef(singleton);
           commonSuffixRef = null;
           runAutomaton = null;
-          sortedTransitions = null;
+          this.automaton = null;
           this.finite = null;
           return;
-        } else if (BasicOperations.sameLanguage(automaton, BasicOperations.concatenate(
-                                                                                       BasicAutomata.makeString(commonPrefix), BasicAutomata.makeAnyString()))) {
-          // matches a constant prefix
-          type = AUTOMATON_TYPE.PREFIX;
-          term = new BytesRef(commonPrefix);
-          commonSuffixRef = null;
-          runAutomaton = null;
-          sortedTransitions = null;
-          this.finite = null;
-          return;
+        } else if (commonPrefix.length() > 0) {
+          Automaton other = Operations.concatenate(Automata.makeString(commonPrefix), Automata.makeAnyString());
+          other = Operations.determinize(other);
+          assert Operations.hasDeadStates(other) == false;
+          if (Operations.sameLanguage(automaton, other)) {
+            // matches a constant prefix
+            type = AUTOMATON_TYPE.PREFIX;
+            term = new BytesRef(commonPrefix);
+            commonSuffixRef = null;
+            runAutomaton = null;
+            this.automaton = null;
+            this.finite = null;
+            return;
+          }
         }
       }
     }
 
     type = AUTOMATON_TYPE.NORMAL;
     term = null;
+
     if (finite == null) {
-      this.finite = SpecialOperations.isFinite(automaton);
+      this.finite = Operations.isFinite(automaton);
     } else {
       this.finite = finite;
     }
+
     Automaton utf8 = new UTF32ToUTF8().convert(automaton);
     if (this.finite) {
       commonSuffixRef = null;
     } else {
-      commonSuffixRef = SpecialOperations.getCommonSuffixBytesRef(utf8);
+      commonSuffixRef = Operations.getCommonSuffixBytesRef(utf8);
     }
     runAutomaton = new ByteRunAutomaton(utf8, true);
-    sortedTransitions = utf8.getSortedTransitions();
+
+    this.automaton = runAutomaton.automaton;
   }
+
+  private Transition transition = new Transition();
   
   //private static final boolean DEBUG = BlockTreeTermsWriter.DEBUG;
 
   private BytesRef addTail(int state, BytesRef term, int idx, int leadLabel) {
-
+    //System.out.println("addTail state=" + state + " term=" + term.utf8ToString() + " idx=" + idx + " leadLabel=" + (char) leadLabel);
+    //System.out.println(automaton.toDot());
     // Find biggest transition that's < label
     // TODO: use binary search here
-    Transition maxTransition = null;
-    for (Transition transition : sortedTransitions[state]) {
+    int maxIndex = -1;
+    int numTransitions = automaton.initTransition(state, transition);
+    for(int i=0;i<numTransitions;i++) {
+      automaton.getNextTransition(transition);
       if (transition.min < leadLabel) {
-        maxTransition = transition;
+        maxIndex = i;
+      } else {
+        // Transitions are alway sorted
+        break;
       }
     }
 
-    assert maxTransition != null;
+    //System.out.println("  maxIndex=" + maxIndex);
+
+    assert maxIndex != -1;
+    automaton.getTransition(state, maxIndex, transition);
 
     // Append floorLabel
     final int floorLabel;
-    if (maxTransition.max > leadLabel-1) {
+    if (transition.max > leadLabel-1) {
       floorLabel = leadLabel-1;
     } else {
-      floorLabel = maxTransition.max;
+      floorLabel = transition.max;
     }
+    //System.out.println("  floorLabel=" + (char) floorLabel);
     if (idx >= term.bytes.length) {
       term.grow(1+idx);
     }
     //if (DEBUG) System.out.println("  add floorLabel=" + (char) floorLabel + " idx=" + idx);
     term.bytes[idx] = (byte) floorLabel;
 
-    state = maxTransition.to.getNumber();
+    state = transition.dest;
+    //System.out.println("  dest: " + state);
     idx++;
 
     // Push down to last accept state
     while (true) {
-      Transition[] transitions = sortedTransitions[state];
-      if (transitions.length == 0) {
+      numTransitions = automaton.getNumTransitions(state);
+      if (numTransitions == 0) {
+        //System.out.println("state=" + state + " 0 trans");
         assert runAutomaton.isAccept(state);
         term.length = idx;
         //if (DEBUG) System.out.println("  return " + term.utf8ToString());
@@ -213,14 +242,15 @@
       } else {
         // We are pushing "top" -- so get last label of
         // last transition:
-        assert transitions.length != 0;
-        Transition lastTransition = transitions[transitions.length-1];
+        //System.out.println("get state=" + state + " numTrans=" + numTransitions);
+        automaton.getTransition(state, numTransitions-1, transition);
         if (idx >= term.bytes.length) {
           term.grow(1+idx);
         }
         //if (DEBUG) System.out.println("  push maxLabel=" + (char) lastTransition.max + " idx=" + idx);
-        term.bytes[idx] = (byte) lastTransition.max;
-        state = lastTransition.to.getNumber();
+        //System.out.println("  add trans dest=" + scratch.dest + " label=" + (char) scratch.max);
+        term.bytes[idx] = (byte) transition.max;
+        state = transition.dest;
         idx++;
       }
     }
@@ -229,6 +259,8 @@
   // TODO: should this take startTerm too?  This way
   // Terms.intersect could forward to this method if type !=
   // NORMAL:
+  /** Return a {@link TermsEnum} intersecting the provided {@link Terms}
+   *  with the terms accepted by this automaton. */
   public TermsEnum getTermsEnum(Terms terms) throws IOException {
     switch(type) {
     case NONE:
@@ -301,33 +333,36 @@
         // Pop back to a state that has a transition
         // <= our label:
         while (true) {
-          Transition[] transitions = sortedTransitions[state];
-          if (transitions.length == 0) {
+          int numTransitions = automaton.getNumTransitions(state);
+          if (numTransitions == 0) {
             assert runAutomaton.isAccept(state);
             output.length = idx;
             //if (DEBUG) System.out.println("  return " + output.utf8ToString());
             return output;
-          } else if (label-1 < transitions[0].min) {
-
-            if (runAutomaton.isAccept(state)) {
-              output.length = idx;
-              //if (DEBUG) System.out.println("  return " + output.utf8ToString());
-              return output;
-            }
-            // pop
-            if (stack.size() == 0) {
-              //if (DEBUG) System.out.println("  pop ord=" + idx + " return null");
-              return null;
-            } else {
-              state = stack.remove(stack.size()-1);
-              idx--;
-              //if (DEBUG) System.out.println("  pop ord=" + (idx+1) + " label=" + (char) label + " first trans.min=" + (char) transitions[0].min);
-              label = input.bytes[input.offset + idx] & 0xff;
-            }
-
           } else {
-            //if (DEBUG) System.out.println("  stop pop ord=" + idx + " first trans.min=" + (char) transitions[0].min);
-            break;
+            automaton.getTransition(state, 0, transition);
+
+            if (label-1 < transition.min) {
+
+              if (runAutomaton.isAccept(state)) {
+                output.length = idx;
+                //if (DEBUG) System.out.println("  return " + output.utf8ToString());
+                return output;
+              }
+              // pop
+              if (stack.size() == 0) {
+                //if (DEBUG) System.out.println("  pop ord=" + idx + " return null");
+                return null;
+              } else {
+                state = stack.remove(stack.size()-1);
+                idx--;
+                //if (DEBUG) System.out.println("  pop ord=" + (idx+1) + " label=" + (char) label + " first trans.min=" + (char) transitions[0].min);
+                label = input.bytes[input.offset + idx] & 0xff;
+              }
+            } else {
+              //if (DEBUG) System.out.println("  stop pop ord=" + idx + " first trans.min=" + (char) transitions[0].min);
+              break;
+            }
           }
         }
 
@@ -346,26 +381,6 @@
       }
     }
   }
-  
-  public String toDot() {
-    StringBuilder b = new StringBuilder("digraph CompiledAutomaton {\n");
-    b.append("  rankdir = LR;\n");
-    int initial = runAutomaton.getInitialState();
-    for (int i = 0; i < sortedTransitions.length; i++) {
-      b.append("  ").append(i);
-      if (runAutomaton.isAccept(i)) b.append(" [shape=doublecircle,label=\"\"];\n");
-      else b.append(" [shape=circle,label=\"\"];\n");
-      if (i == initial) {
-        b.append("  initial [shape=plaintext,label=\"\"];\n");
-        b.append("  initial -> ").append(i).append("\n");
-      }
-      for (int j = 0; j < sortedTransitions[i].length; j++) {
-        b.append("  ").append(i);
-        sortedTransitions[i][j].appendDot(b);
-      }
-    }
-    return b.append("}\n").toString();
-  }
 
   @Override
   public int hashCode() {
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java b/lucene/core/src/java/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java
index 68ce1e9..f96b837 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java
@@ -29,7 +29,7 @@
  * (nearly linear with the input size).
  * 
  * @see #build(Collection)
- * @see BasicAutomata#makeStringUnion(Collection)
+ * @see Automata#makeStringUnion(Collection)
  */
 final class DaciukMihovAutomatonBuilder {
   /**
@@ -249,20 +249,22 @@
   /**
    * Internal recursive traversal for conversion.
    */
-  private static org.apache.lucene.util.automaton.State convert(State s,
-      IdentityHashMap<State,org.apache.lucene.util.automaton.State> visited) {
-    org.apache.lucene.util.automaton.State converted = visited.get(s);
-    if (converted != null) return converted;
+  private static int convert(Automaton.Builder a, State s,
+      IdentityHashMap<State,Integer> visited) {
+
+    Integer converted = visited.get(s);
+    if (converted != null) {
+      return converted;
+    }
     
-    converted = new org.apache.lucene.util.automaton.State();
-    converted.setAccept(s.is_final);
+    converted = a.createState();
+    a.setAccept(converted, s.is_final);
     
     visited.put(s, converted);
     int i = 0;
     int[] labels = s.labels;
     for (DaciukMihovAutomatonBuilder.State target : s.states) {
-      converted.addTransition(
-          new Transition(labels[i++], convert(target, visited)));
+      a.addTransition(converted, convert(a, target, visited), labels[i++]);
     }
     
     return converted;
@@ -281,12 +283,12 @@
       builder.add(scratch);
     }
     
-    Automaton a = new Automaton();
-    a.initial = convert(
+    Automaton.Builder a = new Automaton.Builder();
+    convert(a,
         builder.complete(), 
-        new IdentityHashMap<State,org.apache.lucene.util.automaton.State>());
-    a.deterministic = true;
-    return a;
+        new IdentityHashMap<State,Integer>());
+
+    return a.finish();
   }
 
   /**
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/LevenshteinAutomata.java b/lucene/core/src/java/org/apache/lucene/util/automaton/LevenshteinAutomata.java
index 869dc3a..01badf0 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/LevenshteinAutomata.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/LevenshteinAutomata.java
@@ -21,6 +21,8 @@
 import java.util.SortedSet;
 import java.util.TreeSet;
 
+import org.apache.lucene.util.UnicodeUtil;
+
 /**
  * Class to construct DFAs that match a word within some edit distance.
  * <p>
@@ -30,7 +32,8 @@
  * @lucene.experimental
  */
 public class LevenshteinAutomata {
-  /** @lucene.internal */
+  /** Maximum edit distance this class can generate an automaton for.
+   *  @lucene.internal */
   public static final int MAXIMUM_SUPPORTED_DISTANCE = 2;
   /* input word */
   final int word[];
@@ -112,7 +115,7 @@
     }
     return word;
   }
-  
+
   /**
    * Compute a DFA that accepts all strings within an edit distance of <code>n</code>.
    * <p>
@@ -125,8 +128,25 @@
    * </p>
    */
   public Automaton toAutomaton(int n) {
+    return toAutomaton(n, "");
+  }
+
+  /**
+   * Compute a DFA that accepts all strings within an edit distance of <code>n</code>,
+   * matching the specified exact prefix.
+   * <p>
+   * All automata have the following properties:
+   * <ul>
+   * <li>They are deterministic (DFA).
+   * <li>There are no transitions to dead states.
+   * <li>They are not minimal (some transitions could be combined).
+   * </ul>
+   * </p>
+   */
+  public Automaton toAutomaton(int n, String prefix) {
+    assert prefix != null;
     if (n == 0) {
-      return BasicAutomata.makeString(word, 0, word.length);
+      return Automata.makeString(prefix + UnicodeUtil.newString(word, 0, word.length));
     }
     
     if (n >= descriptions.length)
@@ -135,15 +155,36 @@
     final int range = 2*n+1;
     ParametricDescription description = descriptions[n];
     // the number of states is based on the length of the word and n
-    State states[] = new State[description.size()];
-    // create all states, and mark as accept states if appropriate
-    for (int i = 0; i < states.length; i++) {
-      states[i] = new State();
-      states[i].number = i;
-      states[i].setAccept(description.isAccept(i));
+    int numStates = description.size();
+
+    Automaton a = new Automaton();
+    int lastState;
+    if (prefix != null) {
+      // Insert prefix
+      lastState = a.createState();
+      for (int i = 0, cp = 0; i < prefix.length(); i += Character.charCount(cp)) {
+        int state = a.createState();
+        cp = prefix.codePointAt(i);
+        a.addTransition(lastState, state, cp, cp);
+        lastState = state;
+      }
+    } else {
+      lastState = a.createState();
     }
+
+    int stateOffset = lastState;
+    a.setAccept(lastState, description.isAccept(0));
+
+    // create all states, and mark as accept states if appropriate
+    for (int i = 1; i < numStates; i++) {
+      int state = a.createState();
+      a.setAccept(state, description.isAccept(i));
+    }
+
+    // TODO: this creates bogus states/transitions (states are final, have self loops, and can't be reached from an init state)
+
     // create transitions from state to state
-    for (int k = 0; k < states.length; k++) {
+    for (int k = 0; k < numStates; k++) {
       final int xpos = description.getPosition(k);
       if (xpos < 0)
         continue;
@@ -154,31 +195,26 @@
         // get the characteristic vector at this position wrt ch
         final int cvec = getVector(ch, xpos, end);
         int dest = description.transition(k, xpos, cvec);
-        if (dest >= 0)
-          states[k].addTransition(new Transition(ch, states[dest]));
+        if (dest >= 0) {
+          a.addTransition(stateOffset+k, stateOffset+dest, ch);
+        }
       }
       // add transitions for all other chars in unicode
       // by definition, their characteristic vectors are always 0,
       // because they do not exist in the input string.
       int dest = description.transition(k, xpos, 0); // by definition
-      if (dest >= 0)
-        for (int r = 0; r < numRanges; r++)
-          states[k].addTransition(new Transition(rangeLower[r], rangeUpper[r], states[dest]));      
+      if (dest >= 0) {
+        for (int r = 0; r < numRanges; r++) {
+          a.addTransition(stateOffset+k, stateOffset+dest, rangeLower[r], rangeUpper[r]);
+        }
+      }
     }
 
-    Automaton a = new Automaton(states[0]);
-    a.setDeterministic(true);
-    // we create some useless unconnected states, and its a net-win overall to remove these,
-    // as well as to combine any adjacent transitions (it makes later algorithms more efficient).
-    // so, while we could set our numberedStates here, its actually best not to, and instead to
-    // force a traversal in reduce, pruning the unconnected states while we combine adjacent transitions.
-    //a.setNumberedStates(states);
-    a.reduce();
-    // we need not trim transitions to dead states, as they are not created.
-    //a.restoreInvariant();
+    a.finishState();
+    assert a.isDeterministic();
     return a;
   }
-  
+
   /**
    * Get the characteristic vector <code>X(x, V)</code> 
    * where V is <code>substring(pos, end)</code>
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java b/lucene/core/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java
index 85f8d58..223b25b 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java
@@ -46,40 +46,43 @@
   /**
    * Minimizes (and determinizes if not already deterministic) the given
    * automaton.
-   * 
-   * @see Automaton#setMinimization(int)
    */
-  public static void minimize(Automaton a) {
-    if (!a.isSingleton()) {
-      minimizeHopcroft(a);
-    }
-    // recompute hash code
-    //a.hash_code = 1a.getNumberOfStates() * 3 + a.getNumberOfTransitions() * 2;
-    //if (a.hash_code == 0) a.hash_code = 1;
+  public static Automaton minimize(Automaton a) {
+    return minimizeHopcroft(a);
   }
   
   /**
    * Minimizes the given automaton using Hopcroft's algorithm.
    */
-  public static void minimizeHopcroft(Automaton a) {
-    a.determinize();
-    if (a.initial.numTransitions == 1) {
-      Transition t = a.initial.transitionsArray[0];
-      if (t.to == a.initial && t.min == Character.MIN_CODE_POINT
-          && t.max == Character.MAX_CODE_POINT) return;
+  public static Automaton minimizeHopcroft(Automaton a) {
+    if (a.getNumStates() == 0 || (a.isAccept(0) == false && a.getNumTransitions(0) == 0)) {
+      // Fastmatch for common case
+      return new Automaton();
     }
-    a.totalize();
+    a = Operations.determinize(a);
+    //a.writeDot("adet");
+    if (a.getNumTransitions(0) == 1) {
+      Transition t = new Transition();
+      a.getTransition(0, 0, t);
+      if (t.dest == 0 && t.min == Character.MIN_CODE_POINT
+          && t.max == Character.MAX_CODE_POINT) {
+        // Accepts all strings
+        return a;
+      }
+    }
+    a = Operations.totalize(a);
+    //a.writeDot("atot");
 
     // initialize data structures
     final int[] sigma = a.getStartPoints();
-    final State[] states = a.getNumberedStates();
-    final int sigmaLen = sigma.length, statesLen = states.length;
-    @SuppressWarnings({"rawtypes","unchecked"}) final ArrayList<State>[][] reverse =
-      (ArrayList<State>[][]) new ArrayList[statesLen][sigmaLen];
-    @SuppressWarnings({"rawtypes","unchecked"}) final HashSet<State>[] partition =
-      (HashSet<State>[]) new HashSet[statesLen];
-    @SuppressWarnings({"rawtypes","unchecked"}) final ArrayList<State>[] splitblock =
-      (ArrayList<State>[]) new ArrayList[statesLen];
+    final int sigmaLen = sigma.length, statesLen = a.getNumStates();
+
+    @SuppressWarnings({"rawtypes","unchecked"}) final ArrayList<Integer>[][] reverse =
+      (ArrayList<Integer>[][]) new ArrayList[statesLen][sigmaLen];
+    @SuppressWarnings({"rawtypes","unchecked"}) final HashSet<Integer>[] partition =
+      (HashSet<Integer>[]) new HashSet[statesLen];
+    @SuppressWarnings({"rawtypes","unchecked"}) final ArrayList<Integer>[] splitblock =
+      (ArrayList<Integer>[]) new ArrayList[statesLen];
     final int[] block = new int[statesLen];
     final StateList[][] active = new StateList[statesLen][sigmaLen];
     final StateListNode[][] active2 = new StateListNode[statesLen][sigmaLen];
@@ -96,71 +99,78 @@
     }
     // find initial partition and reverse edges
     for (int q = 0; q < statesLen; q++) {
-      final State qq = states[q];
-      final int j = qq.accept ? 0 : 1;
-      partition[j].add(qq);
+      final int j = a.isAccept(q) ? 0 : 1;
+      partition[j].add(q);
       block[q] = j;
       for (int x = 0; x < sigmaLen; x++) {
-        final ArrayList<State>[] r =
-          reverse[qq.step(sigma[x]).number];
-        if (r[x] == null)
+        final ArrayList<Integer>[] r = reverse[a.step(q, sigma[x])];
+        if (r[x] == null) {
           r[x] = new ArrayList<>();
-        r[x].add(qq);
+        }
+        r[x].add(q);
       }
     }
     // initialize active sets
     for (int j = 0; j <= 1; j++) {
       for (int x = 0; x < sigmaLen; x++) {
-        for (final State qq : partition[j]) {
-          if (reverse[qq.number][x] != null)
-            active2[qq.number][x] = active[j][x].add(qq);
+        for (int q : partition[j]) {
+          if (reverse[q][x] != null) {
+            active2[q][x] = active[j][x].add(q);
+          }
         }
       }
     }
+
     // initialize pending
     for (int x = 0; x < sigmaLen; x++) {
       final int j = (active[0][x].size <= active[1][x].size) ? 0 : 1;
       pending.add(new IntPair(j, x));
       pending2.set(x*statesLen + j);
     }
+
     // process pending until fixed point
     int k = 2;
+    //System.out.println("start min");
     while (!pending.isEmpty()) {
+      //System.out.println("  cycle pending");
       final IntPair ip = pending.removeFirst();
       final int p = ip.n1;
       final int x = ip.n2;
+      //System.out.println("    pop n1=" + ip.n1 + " n2=" + ip.n2);
       pending2.clear(x*statesLen + p);
       // find states that need to be split off their blocks
       for (StateListNode m = active[p][x].first; m != null; m = m.next) {
-        final ArrayList<State> r = reverse[m.q.number][x];
-        if (r != null) for (final State s : r) {
-          final int i = s.number;
-          if (!split.get(i)) {
-            split.set(i);
-            final int j = block[i];
-            splitblock[j].add(s);
-            if (!refine2.get(j)) {
-              refine2.set(j);
-              refine.set(j);
+        final ArrayList<Integer> r = reverse[m.q][x];
+        if (r != null) {
+          for (int i : r) {
+            if (!split.get(i)) {
+              split.set(i);
+              final int j = block[i];
+              splitblock[j].add(i);
+              if (!refine2.get(j)) {
+                refine2.set(j);
+                refine.set(j);
+              }
             }
           }
         }
       }
+
       // refine blocks
       for (int j = refine.nextSetBit(0); j >= 0; j = refine.nextSetBit(j+1)) {
-        final ArrayList<State> sb = splitblock[j];
+        final ArrayList<Integer> sb = splitblock[j];
         if (sb.size() < partition[j].size()) {
-          final HashSet<State> b1 = partition[j];
-          final HashSet<State> b2 = partition[k];
-          for (final State s : sb) {
+          final HashSet<Integer> b1 = partition[j];
+          final HashSet<Integer> b2 = partition[k];
+          for (int s : sb) {
             b1.remove(s);
             b2.add(s);
-            block[s.number] = k;
+            block[s] = k;
             for (int c = 0; c < sigmaLen; c++) {
-              final StateListNode sn = active2[s.number][c];
+              final StateListNode sn = active2[s][c];
               if (sn != null && sn.sl == active[j][c]) {
                 sn.remove();
-                active2[s.number][c] = active[k][c].add(s);
+                active2[s][c] = active[k][c].add(s);
               }
             }
           }
@@ -180,33 +190,69 @@
           k++;
         }
         refine2.clear(j);
-        for (final State s : sb)
-          split.clear(s.number);
+        for (int s : sb) {
+          split.clear(s);
+        }
         sb.clear();
       }
       refine.clear();
     }
+
+    Automaton result = new Automaton();
+
+    Transition t = new Transition();
+
+    //System.out.println("  k=" + k);
+
     // make a new state for each equivalence class, set initial state
-    State[] newstates = new State[k];
-    for (int n = 0; n < newstates.length; n++) {
-      final State s = new State();
-      newstates[n] = s;
-      for (State q : partition[n]) {
-        if (q == a.initial) a.initial = s;
-        s.accept = q.accept;
-        s.number = q.number; // select representative
-        q.number = n;
+    int[] stateMap = new int[statesLen];
+    int[] stateRep = new int[k];
+
+    result.createState();
+
+    //System.out.println("min: k=" + k);
+    for (int n = 0; n < k; n++) {
+      //System.out.println("    n=" + n);
+
+      boolean isInitial = false;
+      for (int q : partition[n]) {
+        if (q == 0) {
+          isInitial = true;
+          //System.out.println("    isInitial!");
+          break;
+        }
+      }
+
+      int newState;
+      if (isInitial) {
+        newState = 0;
+      } else {
+        newState = result.createState();
+      }
+
+      //System.out.println("  newState=" + newState);
+
+      for (int q : partition[n]) {
+        stateMap[q] = newState;
+        //System.out.println("      q=" + q + " isAccept?=" + a.isAccept(q));
+        result.setAccept(newState, a.isAccept(q));
+        stateRep[newState] = q;   // select representative
       }
     }
+
     // build transitions and set acceptance
-    for (int n = 0; n < newstates.length; n++) {
-      final State s = newstates[n];
-      s.accept = states[s.number].accept;
-      for (Transition t : states[s.number].getTransitions())
-        s.addTransition(new Transition(t.min, t.max, newstates[t.to.number]));
+    for (int n = 0; n < k; n++) {
+      int numTransitions = a.initTransition(stateRep[n], t);
+      for(int i=0;i<numTransitions;i++) {
+        a.getNextTransition(t);
+        //System.out.println("  add trans");
+        result.addTransition(n, stateMap[t.dest], t.min, t.max);
+      }
     }
-    a.clearNumberedStates();
-    a.removeDeadTransitions();
+    result.finishState();
+    //System.out.println(result.getNumStates() + " states");
+
+    return Operations.removeDeadStates(result);
   }
   
   static final class IntPair {
@@ -225,20 +271,20 @@
     
     StateListNode first, last;
     
-    StateListNode add(State q) {
+    StateListNode add(int q) {
       return new StateListNode(q, this);
     }
   }
   
   static final class StateListNode {
     
-    final State q;
+    final int q;
     
     StateListNode next, prev;
     
     final StateList sl;
     
-    StateListNode(State q, StateList sl) {
+    StateListNode(int q, StateList sl) {
       this.q = q;
       this.sl = sl;
       if (sl.size++ == 0) sl.first = sl.last = this;
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java b/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java
new file mode 100644
index 0000000..6d2ecac
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java
@@ -0,0 +1,1365 @@
+/*
+ * dk.brics.automaton
+ * 
+ * Copyright (c) 2001-2009 Anders Moeller
+ * All rights reserved.
+ * 
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the above copyright
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ * 3. The name of the author may not be used to endorse or promote products
+ *    derived from this software without specific prior written permission.
+ * 
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
+ * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+ * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+ * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+package org.apache.lucene.util.automaton;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/**
+ * Automata operations.
+ * 
+ * @lucene.experimental
+ */
+final public class Operations {
+  
+  private Operations() {}
+
+  /**
+   * Returns an automaton that accepts the concatenation of the languages of the
+   * given automata.
+   * <p>
+   * Complexity: linear in total number of states.
+   */
+  static public Automaton concatenate(Automaton a1, Automaton a2) {
+    return concatenate(Arrays.asList(a1, a2));
+  }
+
+  /**
+   * Returns an automaton that accepts the concatenation of the languages of the
+   * given automata.
+   * <p>
+   * Complexity: linear in total number of states.
+   */
+  static public Automaton concatenate(List<Automaton> l) {
+    Automaton result = new Automaton();
+
+    // First pass: create all states
+    for(Automaton a : l) {
+      if (a.getNumStates() == 0) {
+        result.finishState();
+        return result;
+      }
+      int numStates = a.getNumStates();
+      for(int s=0;s<numStates;s++) {
+        result.createState();
+      }
+    }
+
+    // Second pass: add transitions, carefully linking accept
+    // states of A to init state of next A:
+    int stateOffset = 0;
+    Transition t = new Transition();
+    for(int i=0;i<l.size();i++) {
+      Automaton a = l.get(i);
+      int numStates = a.getNumStates();
+
+      Automaton nextA = (i == l.size()-1) ? null : l.get(i+1);
+
+      for(int s=0;s<numStates;s++) {
+        int numTransitions = a.initTransition(s, t);
+        for(int j=0;j<numTransitions;j++) {
+          a.getNextTransition(t);
+          result.addTransition(stateOffset + s, stateOffset + t.dest, t.min, t.max);
+        }
+
+        if (a.isAccept(s)) {
+          Automaton followA = nextA;
+          int followOffset = stateOffset;
+          int upto = i+1;
+          while (true) {
+            if (followA != null) {
+              // Adds a "virtual" epsilon transition:
+              numTransitions = followA.initTransition(0, t);
+              for(int j=0;j<numTransitions;j++) {
+                followA.getNextTransition(t);
+                result.addTransition(stateOffset + s, followOffset + numStates + t.dest, t.min, t.max);
+              }
+              if (followA.isAccept(0)) {
+                // Keep chaining if followA accepts empty string
+                followOffset += followA.getNumStates();
+                followA = (upto == l.size()-1) ? null : l.get(upto+1);
+                upto++;
+              } else {
+                break;
+              }
+            } else {
+              result.setAccept(stateOffset + s, true);
+              break;
+            }
+          }
+        }
+      }
+
+      stateOffset += numStates;
+    }
+
+    if (result.getNumStates() == 0) {
+      result.createState();
+    }
+
+    result.finishState();
+
+    return result;
+  }
+
+  /**
+   * Returns an automaton that accepts the union of the empty string and the
+   * language of the given automaton.
+   * <p>
+   * Complexity: linear in number of states.
+   */
+  static public Automaton optional(Automaton a) {
+    Automaton result = new Automaton();
+    result.createState();
+    result.setAccept(0, true);
+    if (a.getNumStates() > 0) {
+      result.copy(a);
+      result.addEpsilon(0, 1);
+    }
+    result.finishState();
+    return result;
+  }
+  
+  /**
+   * Returns an automaton that accepts the Kleene star (zero or more
+   * concatenated repetitions) of the language of the given automaton. Never
+   * modifies the input automaton language.
+   * <p>
+   * Complexity: linear in number of states.
+   */
+  static public Automaton repeat(Automaton a) {
+    Automaton.Builder builder = new Automaton.Builder();
+    builder.createState();
+    builder.setAccept(0, true);
+    builder.copy(a);
+
+    Transition t = new Transition();
+    int count = a.initTransition(0, t);
+    for(int i=0;i<count;i++) {
+      a.getNextTransition(t);
+      builder.addTransition(0, t.dest+1, t.min, t.max);
+    }
+
+    int numStates = a.getNumStates();
+    for(int s=0;s<numStates;s++) {
+      if (a.isAccept(s)) {
+        count = a.initTransition(0, t);
+        for(int i=0;i<count;i++) {
+          a.getNextTransition(t);
+          builder.addTransition(s+1, t.dest+1, t.min, t.max);
+        }
+      }
+    }
+
+    return builder.finish();
+  }
+
+  /**
+   * Returns an automaton that accepts <code>min</code> or more concatenated
+   * repetitions of the language of the given automaton.
+   * <p>
+   * Complexity: linear in number of states and in <code>min</code>.
+   */
+  static public Automaton repeat(Automaton a, int min) {
+    if (min == 0) {
+      return repeat(a);
+    }
+    List<Automaton> as = new ArrayList<>();
+    while (min-- > 0) {
+      as.add(a);
+    }
+    as.add(repeat(a));
+    return concatenate(as);
+  }
+  
+  /**
+   * Returns an automaton that accepts between <code>min</code> and
+   * <code>max</code> (including both) concatenated repetitions of the language
+   * of the given automaton.
+   * <p>
+   * Complexity: linear in number of states and in <code>min</code> and
+   * <code>max</code>.
+   */
+  static public Automaton repeat(Automaton a, int min, int max) {
+    if (min > max) {
+      return Automata.makeEmpty();
+    }
+
+    Automaton b;
+    if (min == 0) {
+      b = Automata.makeEmptyString();
+    } else if (min == 1) {
+      b = new Automaton();
+      b.copy(a);
+    } else {
+      List<Automaton> as = new ArrayList<>();
+      for(int i=0;i<min;i++) {
+        as.add(a);
+      }
+      b = concatenate(as);
+    }
+
+    Set<Integer> prevAcceptStates = toSet(b, 0);
+
+    for(int i=min;i<max;i++) {
+      int numStates = b.getNumStates();
+      b.copy(a);
+      for(int s : prevAcceptStates) {
+        b.addEpsilon(s, numStates);
+      }
+      prevAcceptStates = toSet(a, numStates);
+    }
+
+    b.finishState();
+
+    return b;
+  }
+
+  private static Set<Integer> toSet(Automaton a, int offset) {
+    int numStates = a.getNumStates();
+    FixedBitSet isAccept = a.getAcceptStates();
+    Set<Integer> result = new HashSet<Integer>();
+    int upto = 0;
+    while (upto < numStates && (upto = isAccept.nextSetBit(upto)) != -1) {
+      result.add(offset+upto);
+      upto++;
+    }
+
+    return result;
+  }
+  
+  /**
+   * Returns a (deterministic) automaton that accepts the complement of the
+   * language of the given automaton.
+   * <p>
+   * Complexity: linear in number of states (if already deterministic).
+   */
+  static public Automaton complement(Automaton a) {
+    a = totalize(determinize(a));
+    int numStates = a.getNumStates();
+    for (int p=0;p<numStates;p++) {
+      a.setAccept(p, !a.isAccept(p));
+    }
+    return removeDeadStates(a);
+  }
+  
+  /**
+   * Returns a (deterministic) automaton that accepts the intersection of the
+   * language of <code>a1</code> and the complement of the language of
+   * <code>a2</code>. As a side-effect, the automata may be determinized, if not
+   * already deterministic.
+   * <p>
+   * Complexity: quadratic in number of states (if already deterministic).
+   */
+  static public Automaton minus(Automaton a1, Automaton a2) {
+    if (Operations.isEmpty(a1) || a1 == a2) {
+      return Automata.makeEmpty();
+    }
+    if (Operations.isEmpty(a2)) {
+      return a1;
+    }
+    return intersection(a1, complement(a2));
+  }
+  
+  /**
+   * Returns an automaton that accepts the intersection of the languages of the
+   * given automata. Never modifies the input automata languages.
+   * <p>
+   * Complexity: quadratic in number of states.
+   */
+  static public Automaton intersection(Automaton a1, Automaton a2) {
+    if (a1 == a2) {
+      return a1;
+    }
+    if (a1.getNumStates() == 0) {
+      return a1;
+    }
+    if (a2.getNumStates() == 0) {
+      return a2;
+    }
+    Transition[][] transitions1 = a1.getSortedTransitions();
+    Transition[][] transitions2 = a2.getSortedTransitions();
+    Automaton c = new Automaton();
+    c.createState();
+    LinkedList<StatePair> worklist = new LinkedList<>();
+    HashMap<StatePair,StatePair> newstates = new HashMap<>();
+    StatePair p = new StatePair(0, 0, 0);
+    worklist.add(p);
+    newstates.put(p, p);
+    while (worklist.size() > 0) {
+      p = worklist.removeFirst();
+      c.setAccept(p.s, a1.isAccept(p.s1) && a2.isAccept(p.s2));
+      Transition[] t1 = transitions1[p.s1];
+      Transition[] t2 = transitions2[p.s2];
+      for (int n1 = 0, b2 = 0; n1 < t1.length; n1++) {
+        while (b2 < t2.length && t2[b2].max < t1[n1].min)
+          b2++;
+        for (int n2 = b2; n2 < t2.length && t1[n1].max >= t2[n2].min; n2++)
+          if (t2[n2].max >= t1[n1].min) {
+            StatePair q = new StatePair(t1[n1].dest, t2[n2].dest);
+            StatePair r = newstates.get(q);
+            if (r == null) {
+              q.s = c.createState();
+              worklist.add(q);
+              newstates.put(q, q);
+              r = q;
+            }
+            int min = t1[n1].min > t2[n2].min ? t1[n1].min : t2[n2].min;
+            int max = t1[n1].max < t2[n2].max ? t1[n1].max : t2[n2].max;
+            c.addTransition(p.s, r.s, min, max);
+          }
+      }
+    }
+    c.finishState();
+
+    return removeDeadStates(c);
+  }
+
+  /** Returns true if these two automata accept exactly the
+   *  same language.  This is a costly computation!  Note
+   *  also that a1 and a2 will be determinized as a side
+   *  effect.  Both automata must be determinized and have
+   *  no dead states! */
+  public static boolean sameLanguage(Automaton a1, Automaton a2) {
+    if (a1 == a2) {
+      return true;
+    }
+    return subsetOf(a2, a1) && subsetOf(a1, a2);
+  }
+
+  // TODO: move to test-framework?
+  /** Returns true if this automaton has any states that cannot
+   *  be reached from the initial state or cannot reach an accept state.
+   *  Cost is O(numTransitions+numStates). */
+  public static boolean hasDeadStates(Automaton a) {
+    BitSet liveStates = getLiveStates(a);
+    int numLive = liveStates.cardinality();
+    int numStates = a.getNumStates();
+    assert numLive <= numStates: "numLive=" + numLive + " numStates=" + numStates + " " + liveStates;
+    return numLive < numStates;
+  }
+
+  // TODO: move to test-framework?
+  /** Returns true if there are dead states reachable from an initial state. */
+  public static boolean hasDeadStatesFromInitial(Automaton a) {
+    BitSet reachableFromInitial = getLiveStatesFromInitial(a);
+    BitSet reachableFromAccept = getLiveStatesToAccept(a);
+    reachableFromInitial.andNot(reachableFromAccept);
+    return reachableFromInitial.isEmpty() == false;
+  }
+
+  // TODO: move to test-framework?
+  /** Returns true if there are dead states that reach an accept state. */
+  public static boolean hasDeadStatesToAccept(Automaton a) {
+    BitSet reachableFromInitial = getLiveStatesFromInitial(a);
+    BitSet reachableFromAccept = getLiveStatesToAccept(a);
+    reachableFromAccept.andNot(reachableFromInitial);
+    return reachableFromAccept.isEmpty() == false;
+  }
+
+  /**
+   * Returns true if the language of <code>a1</code> is a subset of the language
+   * of <code>a2</code>. Both automata must be determinized and must have no dead
+   * states.
+   * <p>
+   * Complexity: quadratic in number of states.
+   */
+  public static boolean subsetOf(Automaton a1, Automaton a2) {
+    if (a1.isDeterministic() == false) {
+      throw new IllegalArgumentException("a1 must be deterministic");
+    }
+    if (a2.isDeterministic() == false) {
+      throw new IllegalArgumentException("a2 must be deterministic");
+    }
+    assert hasDeadStatesFromInitial(a1) == false;
+    assert hasDeadStatesFromInitial(a2) == false;
+    if (a1.getNumStates() == 0) {
+      // Empty language is alwyas a subset of any other language
+      return true;
+    } else if (a2.getNumStates() == 0) {
+      return isEmpty(a1);
+    }
+
+    // TODO: cutover to iterators instead
+    Transition[][] transitions1 = a1.getSortedTransitions();
+    Transition[][] transitions2 = a2.getSortedTransitions();
+    LinkedList<StatePair> worklist = new LinkedList<>();
+    HashSet<StatePair> visited = new HashSet<>();
+    StatePair p = new StatePair(0, 0);
+    worklist.add(p);
+    visited.add(p);
+    while (worklist.size() > 0) {
+      p = worklist.removeFirst();
+      if (a1.isAccept(p.s1) && a2.isAccept(p.s2) == false) {
+        return false;
+      }
+      Transition[] t1 = transitions1[p.s1];
+      Transition[] t2 = transitions2[p.s2];
+      for (int n1 = 0, b2 = 0; n1 < t1.length; n1++) {
+        while (b2 < t2.length && t2[b2].max < t1[n1].min) {
+          b2++;
+        }
+        int min1 = t1[n1].min, max1 = t1[n1].max;
+
+        for (int n2 = b2; n2 < t2.length && t1[n1].max >= t2[n2].min; n2++) {
+          if (t2[n2].min > min1) {
+            return false;
+          }
+          if (t2[n2].max < Character.MAX_CODE_POINT) {
+            min1 = t2[n2].max + 1;
+          } else {
+            min1 = Character.MAX_CODE_POINT;
+            max1 = Character.MIN_CODE_POINT;
+          }
+          StatePair q = new StatePair(t1[n1].dest, t2[n2].dest);
+          if (!visited.contains(q)) {
+            worklist.add(q);
+            visited.add(q);
+          }
+        }
+        if (min1 <= max1) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Returns an automaton that accepts the union of the languages of the given
+   * automata.
+   * <p>
+   * Complexity: linear in number of states.
+   */
+  public static Automaton union(Automaton a1, Automaton a2) {
+    return union(Arrays.asList(a1, a2));
+  }
+
+  /**
+   * Returns an automaton that accepts the union of the languages of the given
+   * automata.
+   * <p>
+   * Complexity: linear in number of states.
+   */
+  public static Automaton union(Collection<Automaton> l) {
+    Automaton result = new Automaton();
+
+    // Create initial state:
+    result.createState();
+
+    // Copy over all automata
+    Transition t = new Transition();
+    for(Automaton a : l) {
+      result.copy(a);
+    }
+    
+    // Add epsilon transition from new initial state
+    int stateOffset = 1;
+    for(Automaton a : l) {
+      if (a.getNumStates() == 0) {
+        continue;
+      }
+      result.addEpsilon(0, stateOffset);
+      stateOffset += a.getNumStates();
+    }
+
+    result.finishState();
+
+    return result;
+  }
+
+  // Simple custom ArrayList<Transition>
+  private final static class TransitionList {
+    // dest, min, max
+    int[] transitions = new int[3];
+    int next;
+
+    public void add(Transition t) {
+      if (transitions.length < next+3) {
+        transitions = ArrayUtil.grow(transitions, next+3);
+      }
+      transitions[next] = t.dest;
+      transitions[next+1] = t.min;
+      transitions[next+2] = t.max;
+      next += 3;
+    }
+  }
+
+  // Holds all transitions that start on this int point, or
+  // end at this point-1
+  private final static class PointTransitions implements Comparable<PointTransitions> {
+    int point;
+    final TransitionList ends = new TransitionList();
+    final TransitionList starts = new TransitionList();
+
+    @Override
+    public int compareTo(PointTransitions other) {
+      return point - other.point;
+    }
+
+    public void reset(int point) {
+      this.point = point;
+      ends.next = 0;
+      starts.next = 0;
+    }
+
+    @Override
+    public boolean equals(Object other) {
+      return ((PointTransitions) other).point == point;
+    }
+
+    @Override
+    public int hashCode() {
+      return point;
+    }
+  }
+
+  private final static class PointTransitionSet {
+    int count;
+    PointTransitions[] points = new PointTransitions[5];
+
+    private final static int HASHMAP_CUTOVER = 30;
+    private final HashMap<Integer,PointTransitions> map = new HashMap<>();
+    private boolean useHash = false;
+
+    private PointTransitions next(int point) {
+      // 1st time we are seeing this point
+      if (count == points.length) {
+        final PointTransitions[] newArray = new PointTransitions[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+        System.arraycopy(points, 0, newArray, 0, count);
+        points = newArray;
+      }
+      PointTransitions points0 = points[count];
+      if (points0 == null) {
+        points0 = points[count] = new PointTransitions();
+      }
+      points0.reset(point);
+      count++;
+      return points0;
+    }
+
+    private PointTransitions find(int point) {
+      if (useHash) {
+        final Integer pi = point;
+        PointTransitions p = map.get(pi);
+        if (p == null) {
+          p = next(point);
+          map.put(pi, p);
+        }
+        return p;
+      } else {
+        for(int i=0;i<count;i++) {
+          if (points[i].point == point) {
+            return points[i];
+          }
+        }
+
+        final PointTransitions p = next(point);
+        if (count == HASHMAP_CUTOVER) {
+          // switch to HashMap on the fly
+          assert map.size() == 0;
+          for(int i=0;i<count;i++) {
+            map.put(points[i].point, points[i]);
+          }
+          useHash = true;
+        }
+        return p;
+      }
+    }
+
+    public void reset() {
+      if (useHash) {
+        map.clear();
+        useHash = false;
+      }
+      count = 0;
+    }
+
+    public void sort() {
+      // Tim sort performs well on already sorted arrays:
+      if (count > 1) ArrayUtil.timSort(points, 0, count);
+    }
+
+    public void add(Transition t) {
+      find(t.min).starts.add(t);
+      find(1+t.max).ends.add(t);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder s = new StringBuilder();
+      for(int i=0;i<count;i++) {
+        if (i > 0) {
+          s.append(' ');
+        }
+        s.append(points[i].point).append(':').append(points[i].starts.next/3).append(',').append(points[i].ends.next/3);
+      }
+      return s.toString();
+    }
+  }
+
+  /**
+   * Determinizes the given automaton.
+   * <p>
+   * Worst case complexity: exponential in number of states.
+   */
+  public static Automaton determinize(Automaton a) {
+    if (a.isDeterministic()) {
+      // Already determinized
+      return a;
+    }
+    if (a.getNumStates() <= 1) {
+      // Already determinized
+      return a;
+    }
+
+    // subset construction
+    Automaton.Builder b = new Automaton.Builder();
+
+    //System.out.println("DET:");
+    //a.writeDot("/l/la/lucene/core/detin.dot");
+
+    SortedIntSet.FrozenIntSet initialset = new SortedIntSet.FrozenIntSet(0, 0);
+
+    // Create state 0:
+    b.createState();
+
+    LinkedList<SortedIntSet.FrozenIntSet> worklist = new LinkedList<>();
+    Map<SortedIntSet.FrozenIntSet,Integer> newstate = new HashMap<>();
+
+    worklist.add(initialset);
+
+    b.setAccept(0, a.isAccept(0));
+    newstate.put(initialset, 0);
+
+    int newStateUpto = 0;
+    int[] newStatesArray = new int[5];
+    newStatesArray[newStateUpto] = 0;
+    newStateUpto++;
+
+    // like Set<Integer,PointTransitions>
+    final PointTransitionSet points = new PointTransitionSet();
+
+    // like SortedMap<Integer,Integer>
+    final SortedIntSet statesSet = new SortedIntSet(5);
+
+    Transition t = new Transition();
+
+    while (worklist.size() > 0) {
+      SortedIntSet.FrozenIntSet s = worklist.removeFirst();
+      //System.out.println("det: pop set=" + s);
+
+      // Collate all outgoing transitions by min/1+max:
+      for(int i=0;i<s.values.length;i++) {
+        final int s0 = s.values[i];
+        int numTransitions = a.getNumTransitions(s0);
+        a.initTransition(s0, t);
+        for(int j=0;j<numTransitions;j++) {
+          a.getNextTransition(t);
+          points.add(t);
+        }
+      }
+
+      if (points.count == 0) {
+        // No outgoing transitions -- skip it
+        continue;
+      }
+
+      points.sort();
+
+      int lastPoint = -1;
+      int accCount = 0;
+
+      final int r = s.state;
+
+      for(int i=0;i<points.count;i++) {
+
+        final int point = points.points[i].point;
+
+        if (statesSet.upto > 0) {
+          assert lastPoint != -1;
+
+          statesSet.computeHash();
+          
+          Integer q = newstate.get(statesSet);
+          if (q == null) {
+            q = b.createState();
+            final SortedIntSet.FrozenIntSet p = statesSet.freeze(q);
+            //System.out.println("  make new state=" + q + " -> " + p + " accCount=" + accCount);
+            worklist.add(p);
+            b.setAccept(q, accCount > 0);
+            newstate.put(p, q);
+          } else {
+            assert (accCount > 0 ? true:false) == b.isAccept(q): "accCount=" + accCount + " vs existing accept=" +
+              b.isAccept(q) + " states=" + statesSet;
+          }
+
+          // System.out.println("  add trans src=" + r + " dest=" + q + " min=" + lastPoint + " max=" + (point-1));
+
+          b.addTransition(r, q, lastPoint, point-1);
+        }
+
+        // process transitions that end on this point
+        // (closes an overlapping interval)
+        int[] transitions = points.points[i].ends.transitions;
+        int limit = points.points[i].ends.next;
+        for(int j=0;j<limit;j+=3) {
+          int dest = transitions[j];
+          statesSet.decr(dest);
+          accCount -= a.isAccept(dest) ? 1:0;
+        }
+        points.points[i].ends.next = 0;
+
+        // process transitions that start on this point
+        // (opens a new interval)
+        transitions = points.points[i].starts.transitions;
+        limit = points.points[i].starts.next;
+        for(int j=0;j<limit;j+=3) {
+          int dest = transitions[j];
+          statesSet.incr(dest);
+          accCount += a.isAccept(dest) ? 1:0;
+        }
+        lastPoint = point;
+        points.points[i].starts.next = 0;
+      }
+      points.reset();
+      assert statesSet.upto == 0: "upto=" + statesSet.upto;
+    }
+
+    Automaton result = b.finish();
+    assert result.isDeterministic();
+    return result;
+  }
+
+  /**
+   * Returns true if the given automaton accepts no strings.
+   */
+  public static boolean isEmpty(Automaton a) {
+    if (a.getNumStates() == 0) {
+      // Common case: no states
+      return true;
+    }
+    if (a.isAccept(0) == false && a.getNumTransitions(0) == 0) {
+      // Common case: just one initial state
+      return true;
+    }
+    if (a.isAccept(0) == true) {
+      // Apparently common case: it accepts the damned empty string
+      return false;
+    }
+    
+    LinkedList<Integer> workList = new LinkedList<>();
+    BitSet seen = new BitSet(a.getNumStates());
+    workList.add(0);
+    seen.set(0);
+
+    Transition t = new Transition();
+    while (workList.isEmpty() == false) {
+      int state = workList.removeFirst();
+      if (a.isAccept(state)) {
+        return false;
+      }
+      int count = a.initTransition(state, t);
+      for(int i=0;i<count;i++) {
+        a.getNextTransition(t);
+        if (seen.get(t.dest) == false) {
+          workList.add(t.dest);
+          seen.set(t.dest);
+        }
+      }
+    }
+
+    return true;
+  }
+  
+  /**
+   * Returns true if the given automaton accepts all strings.  The automaton must be minimized.
+   */
+  public static boolean isTotal(Automaton a) {
+    if (a.isAccept(0) && a.getNumTransitions(0) == 1) {
+      Transition t = new Transition();
+      a.getTransition(0, 0, t);
+      return t.dest == 0 && t.min == Character.MIN_CODE_POINT
+          && t.max == Character.MAX_CODE_POINT;
+    }
+    return false;
+  }
+  
+  /**
+   * Returns true if the given string is accepted by the automaton.  The input must be deterministic.
+   * <p>
+   * Complexity: linear in the length of the string.
+   * <p>
+   * <b>Note:</b> for full performance, use the {@link RunAutomaton} class.
+   */
+  public static boolean run(Automaton a, String s) {
+    assert a.isDeterministic();
+    int state = 0;
+    for (int i = 0, cp = 0; i < s.length(); i += Character.charCount(cp)) {
+      int nextState = a.step(state, cp = s.codePointAt(i));
+      if (nextState == -1) {
+        return false;
+      }
+      state = nextState;
+    }
+    return a.isAccept(state);
+  }
+
+  /**
+   * Returns true if the given string (expressed as unicode codepoints) is accepted by the automaton.  The input must be deterministic.
+   * <p>
+   * Complexity: linear in the length of the string.
+   * <p>
+   * <b>Note:</b> for full performance, use the {@link RunAutomaton} class.
+   */
+  public static boolean run(Automaton a, IntsRef s) {
+    assert a.isDeterministic();
+    int state = 0;
+    for (int i=0;i<s.length;i++) {
+      int nextState = a.step(state, s.ints[s.offset+i]);
+      if (nextState == -1) {
+        return false;
+      }
+      state = nextState;
+    }
+    return a.isAccept(state);
+  }
+
+  /**
+   * Returns the set of live states. A state is "live" if an accept state is
+   * reachable from it and if it is reachable from the initial state.
+   */
+  private static BitSet getLiveStates(Automaton a) {
+    BitSet live = getLiveStatesFromInitial(a);
+    live.and(getLiveStatesToAccept(a));
+    return live;
+  }
+
+  /** Returns bitset marking states reachable from the initial state. */
+  private static BitSet getLiveStatesFromInitial(Automaton a) {
+    int numStates = a.getNumStates();
+    BitSet live = new BitSet(numStates);
+    if (numStates == 0) {
+      return live;
+    }
+    LinkedList<Integer> workList = new LinkedList<>();
+    live.set(0);
+    workList.add(0);
+
+    Transition t = new Transition();
+    while (workList.isEmpty() == false) {
+      int s = workList.removeFirst();
+      int count = a.initTransition(s, t);
+      for(int i=0;i<count;i++) {
+        a.getNextTransition(t);
+        if (live.get(t.dest) == false) {
+          live.set(t.dest);
+          workList.add(t.dest);
+        }
+      }
+    }
+
+    return live;
+  }
+
+  /** Returns bitset marking states that can reach an accept state. */
+  private static BitSet getLiveStatesToAccept(Automaton a) {
+    Automaton.Builder builder = new Automaton.Builder();
+
+    // NOTE: not quite the same thing as what SpecialOperations.reverse does:
+    Transition t = new Transition();
+    int numStates = a.getNumStates();
+    for(int s=0;s<numStates;s++) {
+      builder.createState();
+    }
+    for(int s=0;s<numStates;s++) {
+      int count = a.initTransition(s, t);
+      for(int i=0;i<count;i++) {
+        a.getNextTransition(t);
+        builder.addTransition(t.dest, s, t.min, t.max);
+      }
+    }
+    Automaton a2 = builder.finish();
+
+    LinkedList<Integer> workList = new LinkedList<>();
+    BitSet live = new BitSet(numStates);
+    FixedBitSet acceptBits = a.getAcceptStates();
+    int s = 0;
+    while (s < numStates && (s = acceptBits.nextSetBit(s)) != -1) {
+      live.set(s);
+      workList.add(s);
+      s++;
+    }
+
+    while (workList.isEmpty() == false) {
+      s = workList.removeFirst();
+      int count = a2.initTransition(s, t);
+      for(int i=0;i<count;i++) {
+        a2.getNextTransition(t);
+        if (live.get(t.dest) == false) {
+          live.set(t.dest);
+          workList.add(t.dest);
+        }
+      }
+    }
+
+    return live;
+  }
+
+  /**
+   * Removes transitions to dead states (a state is "dead" if it is not
+   * reachable from the initial state or no accept state is reachable from it.)
+   */
+  public static Automaton removeDeadStates(Automaton a) {
+    int numStates = a.getNumStates();
+    BitSet liveSet = getLiveStates(a);
+
+    int[] map = new int[numStates];
+
+    Automaton result = new Automaton();
+    //System.out.println("liveSet: " + liveSet + " numStates=" + numStates);
+    for(int i=0;i<numStates;i++) {
+      if (liveSet.get(i)) {
+        map[i] = result.createState();
+        result.setAccept(map[i], a.isAccept(i));
+      }
+    }
+
+    Transition t = new Transition();
+
+    for (int i=0;i<numStates;i++) {
+      if (liveSet.get(i)) {
+        int numTransitions = a.initTransition(i, t);
+        // filter out transitions to dead states:
+        for(int j=0;j<numTransitions;j++) {
+          a.getNextTransition(t);
+          if (liveSet.get(t.dest)) {
+            result.addTransition(map[i], map[t.dest], t.min, t.max);
+          }
+        }
+      }
+    }
+
+    result.finishState();
+    assert hasDeadStates(result) == false;
+    return result;
+  }
+
+  /**
+   * Finds the largest entry whose value is less than or equal to c, or 0 if
+   * there is no such entry.
+   */
+  static int findIndex(int c, int[] points) {
+    int a = 0;
+    int b = points.length;
+    while (b - a > 1) {
+      int d = (a + b) >>> 1;
+      if (points[d] > c) b = d;
+      else if (points[d] < c) a = d;
+      else return d;
+    }
+    return a;
+  }
+  
+  /**
+   * Returns true if the language of this automaton is finite.  The
+   * automaton must not have any dead states.
+   */
+  public static boolean isFinite(Automaton a) {
+    if (a.getNumStates() == 0) {
+      return true;
+    }
+    return isFinite(new Transition(), a, 0, new BitSet(a.getNumStates()), new BitSet(a.getNumStates()));
+  }
+  
+  /**
+   * Checks whether there is a loop containing state. (This is sufficient since
+   * there are never transitions to dead states.)
+   */
+  // TODO: not great that this is recursive... in theory a
+  // large automata could exceed java's stack
+  private static boolean isFinite(Transition scratch, Automaton a, int state, BitSet path, BitSet visited) {
+    path.set(state);
+    int numTransitions = a.initTransition(state, scratch);
+    for(int t=0;t<numTransitions;t++) {
+      a.getTransition(state, t, scratch);
+      if (path.get(scratch.dest) || (!visited.get(scratch.dest) && !isFinite(scratch, a, scratch.dest, path, visited))) {
+        return false;
+      }
+    }
+    path.clear(state);
+    visited.set(state);
+    return true;
+  }
+  
+  /**
+   * Returns the longest string that is a prefix of all accepted strings and
+   * visits each state at most once.  The automaton must be deterministic.
+   * 
+   * @return common prefix
+   */
+  public static String getCommonPrefix(Automaton a) {
+    if (a.isDeterministic() == false) {
+      throw new IllegalArgumentException("input automaton must be deterministic");
+    }
+    StringBuilder b = new StringBuilder();
+    HashSet<Integer> visited = new HashSet<>();
+    int s = 0;
+    boolean done;
+    Transition t = new Transition();
+    do {
+      done = true;
+      visited.add(s);
+      if (a.isAccept(s) == false && a.getNumTransitions(s) == 1) {
+        a.getTransition(s, 0, t);
+        if (t.min == t.max && !visited.contains(t.dest)) {
+          b.appendCodePoint(t.min);
+          s = t.dest;
+          done = false;
+        }
+      }
+    } while (!done);
+
+    return b.toString();
+  }
+  
+  // TODO: this currently requites a determinized machine,
+  // but it need not -- we can speed it up by walking the
+  // NFA instead.  it'd still be fail fast.
+  /**
+   * Returns the longest BytesRef that is a prefix of all accepted strings and
+   * visits each state at most once.  The automaton must be deterministic.
+   * 
+   * @return common prefix
+   */
+  public static BytesRef getCommonPrefixBytesRef(Automaton a) {
+    BytesRef ref = new BytesRef(10);
+    HashSet<Integer> visited = new HashSet<>();
+    int s = 0;
+    boolean done;
+    Transition t = new Transition();
+    do {
+      done = true;
+      visited.add(s);
+      if (a.isAccept(s) == false && a.getNumTransitions(s) == 1) {
+        a.getTransition(s, 0, t);
+        if (t.min == t.max && !visited.contains(t.dest)) {
+          ref.grow(++ref.length);
+          ref.bytes[ref.length - 1] = (byte) t.min;
+          s = t.dest;
+          done = false;
+        }
+      }
+    } while (!done);
+
+    return ref;
+  }
+
+  /**
+   * Returns the longest BytesRef that is a suffix of all accepted strings.
+   * Worst case complexity: exponential in number of states (this calls
+   * determinize).
+   *
+   * @return common suffix
+   */
+  public static BytesRef getCommonSuffixBytesRef(Automaton a) {
+    // reverse the language of the automaton, then reverse its common prefix.
+    Automaton r = Operations.determinize(reverse(a));
+    BytesRef ref = getCommonPrefixBytesRef(r);
+    reverseBytes(ref);
+    return ref;
+  }
+  
+  private static void reverseBytes(BytesRef ref) {
+    if (ref.length <= 1) return;
+    int num = ref.length >> 1;
+    for (int i = ref.offset; i < ( ref.offset + num ); i++) {
+      byte b = ref.bytes[i];
+      ref.bytes[i] = ref.bytes[ref.offset * 2 + ref.length - i - 1];
+      ref.bytes[ref.offset * 2 + ref.length - i - 1] = b;
+    }
+  }
+
+  /** Returns an automaton accepting the reverse language. */
+  public static Automaton reverse(Automaton a) {
+    return reverse(a, null);
+  }
+
+  /** Reverses the automaton, returning the new initial states. */
+  static Automaton reverse(Automaton a, Set<Integer> initialStates) {
+
+    if (Operations.isEmpty(a)) {
+      return new Automaton();
+    }
+
+    int numStates = a.getNumStates();
+
+    // Build a new automaton with all edges reversed
+    Automaton.Builder builder = new Automaton.Builder();
+
+    // Initial node; we'll add epsilon transitions in the end:
+    builder.createState();
+
+    for(int s=0;s<numStates;s++) {
+      builder.createState();
+    }
+
+    // Old initial state becomes new accept state:
+    builder.setAccept(1, true);
+
+    Transition t = new Transition();
+    for (int s=0;s<numStates;s++) {
+      int numTransitions = a.getNumTransitions(s);
+      a.initTransition(s, t);
+      for(int i=0;i<numTransitions;i++) {
+        a.getNextTransition(t);
+        builder.addTransition(t.dest+1, s+1, t.min, t.max);
+      }
+    }
+
+    Automaton result = builder.finish();
+
+    int s = 0;
+    FixedBitSet acceptStates = a.getAcceptStates();
+    while (s < numStates && (s = acceptStates.nextSetBit(s)) != -1) {
+      result.addEpsilon(0, s+1);
+      if (initialStates != null) {
+        initialStates.add(s+1);
+      }
+      s++;
+    }
+
+    result.finishState();
+
+    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);
+
+      IntsRef string = new IntsRef(1);
+      string.length = 1;
+
+      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.ints[string.length-1] = label;
+
+          if (a.isAccept(node.to)) {
+            // This transition leads to an accept state,
+            // so we save the current string:
+            results.add(IntsRef.deepCopyOf(string));
+            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.length++;
+            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.length--;
+        }
+      }
+    }
+
+    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. */
+  static Automaton totalize(Automaton a) {
+    Automaton result = new Automaton();
+    int numStates = a.getNumStates();
+    for(int i=0;i<numStates;i++) {
+      result.createState();
+      result.setAccept(i, a.isAccept(i));
+    }
+
+    int deadState = result.createState();
+    result.addTransition(deadState, deadState, Character.MIN_CODE_POINT, Character.MAX_CODE_POINT);
+
+    Transition t = new Transition();
+    for(int i=0;i<numStates;i++) {
+      int maxi = Character.MIN_CODE_POINT;
+      int count = a.initTransition(i, t);
+      for(int j=0;j<count;j++) {
+        a.getNextTransition(t);
+        result.addTransition(i, t.dest, t.min, t.max);
+        if (t.min > maxi) {
+          result.addTransition(i, deadState, maxi, t.min-1);
+        }
+        if (t.max + 1 > maxi) {
+          maxi = t.max + 1;
+        }
+      }
+
+      if (maxi <= Character.MAX_CODE_POINT) {
+        result.addTransition(i, deadState, maxi, Character.MAX_CODE_POINT);
+      }
+    }
+
+    result.finishState();
+    return result;
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java b/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java
index bf5b4be..76a040a 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java
@@ -361,8 +361,6 @@
    */
   public static final int NONE = 0x0000;
   
-  private static boolean allow_mutation = false;
-  
   Kind kind;
   RegExp exp1, exp2;
   String s;
@@ -419,13 +417,13 @@
     to = e.to;
     b = null;
   }
-  
+
   /**
    * Constructs new <code>Automaton</code> from this <code>RegExp</code>. Same
    * as <code>toAutomaton(null)</code> (empty automaton map).
    */
   public Automaton toAutomaton() {
-    return toAutomatonAllowMutate(null, null);
+    return toAutomaton(null, null);
   }
   
   /**
@@ -439,7 +437,7 @@
    */
   public Automaton toAutomaton(AutomatonProvider automaton_provider)
       throws IllegalArgumentException {
-    return toAutomatonAllowMutate(null, automaton_provider);
+    return toAutomaton(null, automaton_provider);
   }
   
   /**
@@ -454,32 +452,9 @@
    */
   public Automaton toAutomaton(Map<String,Automaton> automata)
       throws IllegalArgumentException {
-    return toAutomatonAllowMutate(automata, null);
+    return toAutomaton(automata, null);
   }
-  
-  /**
-   * Sets or resets allow mutate flag. If this flag is set, then automata
-   * construction uses mutable automata, which is slightly faster but not thread
-   * safe. By default, the flag is not set.
-   * 
-   * @param flag if true, the flag is set
-   * @return previous value of the flag
-   */
-  public boolean setAllowMutate(boolean flag) {
-    boolean b = allow_mutation;
-    allow_mutation = flag;
-    return b;
-  }
-  
-  private Automaton toAutomatonAllowMutate(Map<String,Automaton> automata,
-      AutomatonProvider automaton_provider) throws IllegalArgumentException {
-    boolean b = false;
-    if (allow_mutation) b = Automaton.setAllowMutate(true); // thread unsafe
-    Automaton a = toAutomaton(automata, automaton_provider);
-    if (allow_mutation) Automaton.setAllowMutate(b);
-    return a;
-  }
-  
+
   private Automaton toAutomaton(Map<String,Automaton> automata,
       AutomatonProvider automaton_provider) throws IllegalArgumentException {
     List<Automaton> list;
@@ -489,8 +464,8 @@
         list = new ArrayList<>();
         findLeaves(exp1, Kind.REGEXP_UNION, list, automata, automaton_provider);
         findLeaves(exp2, Kind.REGEXP_UNION, list, automata, automaton_provider);
-        a = BasicOperations.union(list);
-        MinimizationOperations.minimize(a);
+        a = Operations.union(list);
+        a = MinimizationOperations.minimize(a);
         break;
       case REGEXP_CONCATENATION:
         list = new ArrayList<>();
@@ -498,66 +473,72 @@
             automaton_provider);
         findLeaves(exp2, Kind.REGEXP_CONCATENATION, list, automata,
             automaton_provider);
-        a = BasicOperations.concatenate(list);
-        MinimizationOperations.minimize(a);
+        a = Operations.concatenate(list);
+        a = MinimizationOperations.minimize(a);
         break;
       case REGEXP_INTERSECTION:
-        a = exp1.toAutomaton(automata, automaton_provider).intersection(
+        a = Operations.intersection(
+            exp1.toAutomaton(automata, automaton_provider),
             exp2.toAutomaton(automata, automaton_provider));
-        MinimizationOperations.minimize(a);
+        a = MinimizationOperations.minimize(a);
         break;
       case REGEXP_OPTIONAL:
-        a = exp1.toAutomaton(automata, automaton_provider).optional();
-        MinimizationOperations.minimize(a);
+        a = Operations.optional(exp1.toAutomaton(automata, automaton_provider));
+        a = MinimizationOperations.minimize(a);
         break;
       case REGEXP_REPEAT:
-        a = exp1.toAutomaton(automata, automaton_provider).repeat();
-        MinimizationOperations.minimize(a);
+        a = Operations.repeat(exp1.toAutomaton(automata, automaton_provider));
+        a = MinimizationOperations.minimize(a);
         break;
       case REGEXP_REPEAT_MIN:
-        a = exp1.toAutomaton(automata, automaton_provider).repeat(min);
-        MinimizationOperations.minimize(a);
+        a = Operations.repeat(exp1.toAutomaton(automata, automaton_provider), min);
+        a = MinimizationOperations.minimize(a);
         break;
       case REGEXP_REPEAT_MINMAX:
-        a = exp1.toAutomaton(automata, automaton_provider).repeat(min, max);
-        MinimizationOperations.minimize(a);
+        a = Operations.repeat(exp1.toAutomaton(automata, automaton_provider), min, max);
+        a = MinimizationOperations.minimize(a);
         break;
       case REGEXP_COMPLEMENT:
-        a = exp1.toAutomaton(automata, automaton_provider).complement();
-        MinimizationOperations.minimize(a);
+        a = Operations.complement(exp1.toAutomaton(automata, automaton_provider));
+        a = MinimizationOperations.minimize(a);
         break;
       case REGEXP_CHAR:
-        a = BasicAutomata.makeChar(c);
+        a = Automata.makeChar(c);
         break;
       case REGEXP_CHAR_RANGE:
-        a = BasicAutomata.makeCharRange(from, to);
+        a = Automata.makeCharRange(from, to);
         break;
       case REGEXP_ANYCHAR:
-        a = BasicAutomata.makeAnyChar();
+        a = Automata.makeAnyChar();
         break;
       case REGEXP_EMPTY:
-        a = BasicAutomata.makeEmpty();
+        a = Automata.makeEmpty();
         break;
       case REGEXP_STRING:
-        a = BasicAutomata.makeString(s);
+        a = Automata.makeString(s);
         break;
       case REGEXP_ANYSTRING:
-        a = BasicAutomata.makeAnyString();
+        a = Automata.makeAnyString();
         break;
       case REGEXP_AUTOMATON:
         Automaton aa = null;
-        if (automata != null) aa = automata.get(s);
-        if (aa == null && automaton_provider != null) try {
-          aa = automaton_provider.getAutomaton(s);
-        } catch (IOException e) {
-          throw new IllegalArgumentException(e);
+        if (automata != null) {
+          aa = automata.get(s);
         }
-        if (aa == null) throw new IllegalArgumentException("'" + s
-            + "' not found");
-        a = aa.clone(); // always clone here (ignore allow_mutate)
+        if (aa == null && automaton_provider != null) {
+          try {
+            aa = automaton_provider.getAutomaton(s);
+          } catch (IOException e) {
+            throw new IllegalArgumentException(e);
+          }
+        }
+        if (aa == null) {
+          throw new IllegalArgumentException("'" + s + "' not found");
+        }
+        a = aa;
         break;
       case REGEXP_INTERVAL:
-        a = BasicAutomata.makeInterval(min, max, digits);
+        a = Automata.makeInterval(min, max, digits);
         break;
     }
     return a;
@@ -568,7 +549,9 @@
     if (exp.kind == kind) {
       findLeaves(exp.exp1, kind, list, automata, automaton_provider);
       findLeaves(exp.exp2, kind, list, automata, automaton_provider);
-    } else list.add(exp.toAutomaton(automata, automaton_provider));
+    } else {
+      list.add(exp.toAutomaton(automata, automaton_provider));
+    }
   }
   
   /**
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/RunAutomaton.java b/lucene/core/src/java/org/apache/lucene/util/automaton/RunAutomaton.java
index bbcadd3..7c21632 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/RunAutomaton.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/RunAutomaton.java
@@ -37,6 +37,7 @@
  * @lucene.experimental
  */
 public abstract class RunAutomaton {
+  final Automaton automaton;
   final int maxInterval;
   final int size;
   final boolean[] accept;
@@ -65,10 +66,10 @@
           if (j + 1 < points.length) max = (points[j + 1] - 1);
           else max = maxInterval;
           b.append(" ");
-          Transition.appendCharString(min, b);
+          Automaton.appendCharString(min, b);
           if (min != max) {
             b.append("-");
-            Transition.appendCharString(max, b);
+            Automaton.appendCharString(max, b);
           }
           b.append(" -> ").append(k).append("\n");
         }
@@ -110,7 +111,7 @@
    * Gets character class of given codepoint
    */
   final int getCharClass(int c) {
-    return SpecialOperations.findIndex(c, points);
+    return Operations.findIndex(c, points);
   }
 
   /**
@@ -121,23 +122,23 @@
    */
   public RunAutomaton(Automaton a, int maxInterval, boolean tableize) {
     this.maxInterval = maxInterval;
-    a.determinize();
+    a = Operations.determinize(a);
+    this.automaton = a;
     points = a.getStartPoints();
-    final State[] states = a.getNumberedStates();
-    initial = a.initial.number;
-    size = states.length;
+    initial = 0;
+    size = Math.max(1,a.getNumStates());
     accept = new boolean[size];
     transitions = new int[size * points.length];
-    for (int n = 0; n < size * points.length; n++)
-      transitions[n] = -1;
-    for (State s : states) {
-      int n = s.number;
-      accept[n] = s.accept;
+    Arrays.fill(transitions, -1);
+    for (int n=0;n<size;n++) {
+      accept[n] = a.isAccept(n);
       for (int c = 0; c < points.length; c++) {
-        State q = s.step(points[c]);
-        if (q != null) transitions[n * points.length + c] = q.number;
+        int dest = a.step(n, points[c]);
+        assert dest == -1 || dest < size;
+        transitions[n * points.length + c] = dest;
       }
     }
+
     /*
      * Set alphabet table for optimal run performance.
      */
@@ -145,8 +146,9 @@
       classmap = new int[maxInterval + 1];
       int i = 0;
       for (int j = 0; j <= maxInterval; j++) {
-        if (i + 1 < points.length && j == points[i + 1])
+        if (i + 1 < points.length && j == points[i + 1]) {
           i++;
+        }
         classmap[j] = i;
       }
     } else {
@@ -162,10 +164,11 @@
    * transition function.)
    */
   public final int step(int state, int c) {
-    if (classmap == null)
+    if (classmap == null) {
       return transitions[state * points.length + getCharClass(c)];
-    else
+    } else {
       return transitions[state * points.length + classmap[c]];
+    }
   }
 
   @Override
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/SortedIntSet.java b/lucene/core/src/java/org/apache/lucene/util/automaton/SortedIntSet.java
index 546c307..d66f9de 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/SortedIntSet.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/SortedIntSet.java
@@ -39,7 +39,7 @@
 
   private boolean useTreeMap;
 
-  State state;
+  int state;
 
   public SortedIntSet(int capacity) {
     values = new int[capacity];
@@ -153,7 +153,7 @@
     }
   }
 
-  public FrozenIntSet freeze(State state) {
+  public FrozenIntSet freeze(int state) {
     final int[] c = new int[upto];
     System.arraycopy(values, 0, c, 0, upto);
     return new FrozenIntSet(c, hashCode, state);
@@ -204,15 +204,15 @@
   public final static class FrozenIntSet {
     final int[] values;
     final int hashCode;
-    final State state;
+    final int state;
 
-    public FrozenIntSet(int[] values, int hashCode, State state) {
+    public FrozenIntSet(int[] values, int hashCode, int state) {
       this.values = values;
       this.hashCode = hashCode;
       this.state = state;
     }
 
-    public FrozenIntSet(int num, State state) {
+    public FrozenIntSet(int num, int state) {
       this.values = new int[] {num};
       this.state = state;
       this.hashCode = 683+num;
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java b/lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java
deleted file mode 100644
index a0eb957..0000000
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java
+++ /dev/null
@@ -1,374 +0,0 @@
-/*
- * dk.brics.automaton
- * 
- * Copyright (c) 2001-2009 Anders Moeller
- * All rights reserved.
- * 
- * Redistribution and use in source and binary forms, with or without
- * modification, are permitted provided that the following conditions
- * are met:
- * 1. Redistributions of source code must retain the above copyright
- *    notice, this list of conditions and the following disclaimer.
- * 2. Redistributions in binary form must reproduce the above copyright
- *    notice, this list of conditions and the following disclaimer in the
- *    documentation and/or other materials provided with the distribution.
- * 3. The name of the author may not be used to endorse or promote products
- *    derived from this software without specific prior written permission.
- * 
- * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
- * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
- * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
- * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
- * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
- * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
- * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
- */
-
-package org.apache.lucene.util.automaton;
-
-import java.util.BitSet;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.IdentityHashMap;
-import java.util.Set;
-
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IntsRef;
-import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.fst.Util;
-
-/**
- * Special automata operations.
- * 
- * @lucene.experimental
- */
-final public class SpecialOperations {
-  
-  private SpecialOperations() {}
-  
-  /**
-   * Finds the largest entry whose value is less than or equal to c, or 0 if
-   * there is no such entry.
-   */
-  static int findIndex(int c, int[] points) {
-    int a = 0;
-    int b = points.length;
-    while (b - a > 1) {
-      int d = (a + b) >>> 1;
-      if (points[d] > c) b = d;
-      else if (points[d] < c) a = d;
-      else return d;
-    }
-    return a;
-  }
-  
-  /**
-   * Returns true if the language of this automaton is finite.
-   */
-  public static boolean isFinite(Automaton a) {
-    if (a.isSingleton()) return true;
-    return isFinite(a.initial, new BitSet(a.getNumberOfStates()), new BitSet(a.getNumberOfStates()));
-  }
-  
-  /**
-   * Checks whether there is a loop containing s. (This is sufficient since
-   * there are never transitions to dead states.)
-   */
-  // TODO: not great that this is recursive... in theory a
-  // large automata could exceed java's stack
-  private static boolean isFinite(State s, BitSet path, BitSet visited) {
-    path.set(s.number);
-    for (Transition t : s.getTransitions())
-      if (path.get(t.to.number) || (!visited.get(t.to.number) && !isFinite(t.to, path, visited))) return false;
-    path.clear(s.number);
-    visited.set(s.number);
-    return true;
-  }
-  
-  /**
-   * Returns the longest string that is a prefix of all accepted strings and
-   * visits each state at most once.
-   * 
-   * @return common prefix
-   */
-  public static String getCommonPrefix(Automaton a) {
-    if (a.isSingleton()) return a.singleton;
-    StringBuilder b = new StringBuilder();
-    HashSet<State> visited = new HashSet<>();
-    State s = a.initial;
-    boolean done;
-    do {
-      done = true;
-      visited.add(s);
-      if (!s.accept && s.numTransitions() == 1) {
-        Transition t = s.getTransitions().iterator().next();
-        if (t.min == t.max && !visited.contains(t.to)) {
-          b.appendCodePoint(t.min);
-          s = t.to;
-          done = false;
-        }
-      }
-    } while (!done);
-    return b.toString();
-  }
-  
-  // TODO: this currently requites a determinized machine,
-  // but it need not -- we can speed it up by walking the
-  // NFA instead.  it'd still be fail fast.
-  public static BytesRef getCommonPrefixBytesRef(Automaton a) {
-    if (a.isSingleton()) return new BytesRef(a.singleton);
-    BytesRef ref = new BytesRef(10);
-    HashSet<State> visited = new HashSet<>();
-    State s = a.initial;
-    boolean done;
-    do {
-      done = true;
-      visited.add(s);
-      if (!s.accept && s.numTransitions() == 1) {
-        Transition t = s.getTransitions().iterator().next();
-        if (t.min == t.max && !visited.contains(t.to)) {
-          ref.grow(++ref.length);
-          ref.bytes[ref.length - 1] = (byte)t.min;
-          s = t.to;
-          done = false;
-        }
-      }
-    } while (!done);
-    return ref;
-  }
-  
-  /**
-   * Returns the longest string that is a suffix of all accepted strings and
-   * visits each state at most once.
-   * 
-   * @return common suffix
-   */
-  public static String getCommonSuffix(Automaton a) {
-    if (a.isSingleton()) // if singleton, the suffix is the string itself.
-      return a.singleton;
-    
-    // reverse the language of the automaton, then reverse its common prefix.
-    Automaton r = a.clone();
-    reverse(r);
-    r.determinize();
-    return new StringBuilder(SpecialOperations.getCommonPrefix(r)).reverse().toString();
-  }
-  
-  public static BytesRef getCommonSuffixBytesRef(Automaton a) {
-    if (a.isSingleton()) // if singleton, the suffix is the string itself.
-      return new BytesRef(a.singleton);
-    
-    // reverse the language of the automaton, then reverse its common prefix.
-    Automaton r = a.clone();
-    reverse(r);
-    r.determinize();
-    BytesRef ref = SpecialOperations.getCommonPrefixBytesRef(r);
-    reverseBytes(ref);
-    return ref;
-  }
-  
-  private static void reverseBytes(BytesRef ref) {
-    if (ref.length <= 1) return;
-    int num = ref.length >> 1;
-    for (int i = ref.offset; i < ( ref.offset + num ); i++) {
-      byte b = ref.bytes[i];
-      ref.bytes[i] = ref.bytes[ref.offset * 2 + ref.length - i - 1];
-      ref.bytes[ref.offset * 2 + ref.length - i - 1] = b;
-    }
-  }
-  
-  /**
-   * Reverses the language of the given (non-singleton) automaton while returning
-   * the set of new initial states.
-   */
-  public static Set<State> reverse(Automaton a) {
-    a.expandSingleton();
-    // reverse all edges
-    HashMap<State, HashSet<Transition>> m = new HashMap<>();
-    State[] states = a.getNumberedStates();
-    Set<State> accept = new HashSet<>();
-    for (State s : states)
-      if (s.isAccept())
-        accept.add(s);
-    for (State r : states) {
-      m.put(r, new HashSet<Transition>());
-      r.accept = false;
-    }
-    for (State r : states)
-      for (Transition t : r.getTransitions())
-        m.get(t.to).add(new Transition(t.min, t.max, r));
-    for (State r : states) {
-      Set<Transition> tr = m.get(r);
-      r.setTransitions(tr.toArray(new Transition[tr.size()]));
-    }
-    // make new initial+final states
-    a.initial.accept = true;
-    a.initial = new State();
-    for (State r : accept)
-      a.initial.addEpsilon(r); // ensures that all initial states are reachable
-    a.deterministic = false;
-    a.clearNumberedStates();
-    return accept;
-  }
-
-  private static class PathNode {
-
-    /** Which state the path node ends on, whose
-     *  transitions we are enumerating. */
-    public State state;
-
-    /** Which state the current transition leads to. */
-    public State 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;
-
-    public void resetState(State state) {
-      assert state.numTransitions() != 0;
-      this.state = state;
-      transition = 0;
-      Transition t = state.transitionsArray[transition];
-      label = t.min;
-      to = t.to;
-    }
-
-    /** 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() {
-      if (label > state.transitionsArray[transition].max) {
-        // We've exhaused the current transition's labels;
-        // move to next transitions:
-        transition++;
-        if (transition >= state.numTransitions()) {
-          // We're done iterating transitions leaving this state
-          return -1;
-        }
-        Transition t = state.transitionsArray[transition];
-        label = t.min;
-        to = t.to;
-      }
-      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.isSingleton()) {
-      // Easy case: automaton accepts only 1 string
-      results.add(Util.toUTF32(a.singleton, new IntsRef()));
-    } else {
-
-      if (a.initial.accept) {
-        // Special case the empty string, as usual:
-        results.add(new IntsRef());
-      }
-
-      if (a.initial.numTransitions() > 0 && (limit == -1 || results.size() < limit)) {
-
-        // TODO: we could use state numbers here and just
-        // alloc array, but asking for states array can be
-        // costly (it's lazily computed):
-
-        // Tracks which states are in the current path, for
-        // cycle detection:
-        Set<State> pathStates = Collections.newSetFromMap(new IdentityHashMap<State,Boolean>());
-
-        // Stack to hold our current state in the
-        // recursion/iteration:
-        PathNode[] nodes = new PathNode[4];
-
-        pathStates.add(a.initial);
-        PathNode root = getNode(nodes, 0);
-        root.resetState(a.initial);
-
-        IntsRef string = new IntsRef(1);
-        string.length = 1;
-
-        while (string.length > 0) {
-
-          PathNode node = nodes[string.length-1];
-
-          // Get next label leaving the current node:
-          int label = node.nextLabel();
-
-          if (label != -1) {
-            string.ints[string.length-1] = label;
-
-            if (node.to.accept) {
-              // This transition leads to an accept state,
-              // so we save the current string:
-              results.add(IntsRef.deepCopyOf(string));
-              if (results.size() == limit) {
-                break;
-              }
-            }
-
-            if (node.to.numTransitions() != 0) {
-              // Now recurse: the destination of this transition has
-              // outgoing transitions:
-              if (pathStates.contains(node.to)) {
-                throw new IllegalArgumentException("automaton has cycles");
-              }
-              pathStates.add(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(node.to);
-              string.length++;
-              string.grow(string.length);
-            }
-          } else {
-            // No more transitions leaving this state,
-            // pop/return back to previous state:
-            assert pathStates.contains(node.state);
-            pathStates.remove(node.state);
-            string.length--;
-          }
-        }
-      }
-    }
-
-    return results;
-  }
-}
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/State.java b/lucene/core/src/java/org/apache/lucene/util/automaton/State.java
deleted file mode 100644
index d1639e4..0000000
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/State.java
+++ /dev/null
@@ -1,280 +0,0 @@
-/*
- * dk.brics.automaton
- * 
- * Copyright (c) 2001-2009 Anders Moeller
- * All rights reserved.
- * 
- * Redistribution and use in source and binary forms, with or without
- * modification, are permitted provided that the following conditions
- * are met:
- * 1. Redistributions of source code must retain the above copyright
- *    notice, this list of conditions and the following disclaimer.
- * 2. Redistributions in binary form must reproduce the above copyright
- *    notice, this list of conditions and the following disclaimer in the
- *    documentation and/or other materials provided with the distribution.
- * 3. The name of the author may not be used to endorse or promote products
- *    derived from this software without specific prior written permission.
- * 
- * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
- * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
- * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
- * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
- * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
- * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
- * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
- */
-
-package org.apache.lucene.util.automaton;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.RamUsageEstimator;
-
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.Iterator;
-
-/**
- * <tt>Automaton</tt> state.
- * 
- * @lucene.experimental
- */
-public class State implements Comparable<State> {
-  
-  boolean accept;
-  public Transition[] transitionsArray;
-  public int numTransitions;
-  
-  int number;
-
-  int id;
-  static int next_id;
-  
-  /**
-   * Constructs a new state. Initially, the new state is a reject state.
-   */
-  public State() {
-    resetTransitions();
-    id = next_id++;
-  }
-  
-  /**
-   * Resets transition set.
-   */
-  final void resetTransitions() {
-    transitionsArray = new Transition[0];
-    numTransitions = 0;
-  }
-
-  private class TransitionsIterable implements Iterable<Transition> {
-    @Override
-    public Iterator<Transition> iterator() {
-      return new Iterator<Transition>() {
-        int upto;
-        @Override
-        public boolean hasNext() {
-          return upto < numTransitions;
-        }
-        @Override
-        public Transition next() {
-          return transitionsArray[upto++];
-        }
-        @Override
-        public void remove() {
-          throw new UnsupportedOperationException();
-        }
-      };
-    }
-  }
-  
-  /**
-   * Returns the set of outgoing transitions. Subsequent changes are reflected
-   * in the automaton.
-   * 
-   * @return transition set
-   */
-  public Iterable<Transition> getTransitions() {
-    return new TransitionsIterable();
-  }
-
-  public int numTransitions() {
-    return numTransitions;
-  }
-
-  public void setTransitions(Transition[] transitions) {
-    this.numTransitions = transitions.length;
-    this.transitionsArray = transitions;
-  }
-  
-  /**
-   * Adds an outgoing transition.
-   * 
-   * @param t transition
-   */
-  public void addTransition(Transition t) {
-    if (numTransitions == transitionsArray.length) {
-      final Transition[] newArray = new Transition[ArrayUtil.oversize(1+numTransitions, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-      System.arraycopy(transitionsArray, 0, newArray, 0, numTransitions);
-      transitionsArray = newArray;
-    }
-    transitionsArray[numTransitions++] = t;
-  }
-  
-  /**
-   * Sets acceptance for this state.
-   * 
-   * @param accept if true, this state is an accept state
-   */
-  public void setAccept(boolean accept) {
-    this.accept = accept;
-  }
-  
-  /**
-   * Returns acceptance status.
-   * 
-   * @return true is this is an accept state
-   */
-  public boolean isAccept() {
-    return accept;
-  }
-  
-  /**
-   * Performs lookup in transitions, assuming determinism.
-   * 
-   * @param c codepoint to look up
-   * @return destination state, null if no matching outgoing transition
-   * @see #step(int, Collection)
-   */
-  public State step(int c) {
-    assert c >= 0;
-    for (int i=0;i<numTransitions;i++) {
-      final Transition t = transitionsArray[i];
-      if (t.min <= c && c <= t.max) return t.to;
-    }
-    return null;
-  }
-  
-  /**
-   * Performs lookup in transitions, allowing nondeterminism.
-   * 
-   * @param c codepoint to look up
-   * @param dest collection where destination states are stored
-   * @see #step(int)
-   */
-  public void step(int c, Collection<State> dest) {
-    for (int i=0;i<numTransitions;i++) {
-      final Transition t = transitionsArray[i];
-      if (t.min <= c && c <= t.max) dest.add(t.to);
-    }
-  }
-  
-  /** Virtually adds an epsilon transition to the target
-   *  {@code to} state.  This is implemented by copying all
-   *  transitions from {@code to} to this state, and if {@code
-   *  to} is an accept state then set accept for this state. */
-  void addEpsilon(State to) {
-    if (to.accept) accept = true;
-    for (Transition t : to.getTransitions())
-      addTransition(t);
-  }
-
-  /** Downsizes transitionArray to numTransitions */
-  public void trimTransitionsArray() {
-    if (numTransitions < transitionsArray.length) {
-      final Transition[] newArray = new Transition[numTransitions];
-      System.arraycopy(transitionsArray, 0, newArray, 0, numTransitions);
-      transitionsArray = newArray;
-    }
-  }
-  
-  /**
-   * Reduces this state. A state is "reduced" by combining overlapping
-   * and adjacent edge intervals with same destination.
-   */
-  public void reduce() {
-    if (numTransitions <= 1) {
-      return;
-    }
-    sortTransitions(Transition.CompareByDestThenMinMax);
-    State p = null;
-    int min = -1, max = -1;
-    int upto = 0;
-    for (int i=0;i<numTransitions;i++) {
-      final Transition t = transitionsArray[i];
-      if (p == t.to) {
-        if (t.min <= max + 1) {
-          if (t.max > max) max = t.max;
-        } else {
-          if (p != null) {
-            transitionsArray[upto++] = new Transition(min, max, p);
-          }
-          min = t.min;
-          max = t.max;
-        }
-      } else {
-        if (p != null) {
-          transitionsArray[upto++] = new Transition(min, max, p);
-        }
-        p = t.to;
-        min = t.min;
-        max = t.max;
-      }
-    }
-
-    if (p != null) {
-      transitionsArray[upto++] = new Transition(min, max, p);
-    }
-    numTransitions = upto;
-  }
-
-  /**
-   * Returns sorted list of outgoing transitions.
-   * 
-   * @param to_first if true, order by (to, min, reverse max); otherwise (min,
-   *          reverse max, to)
-   * @return transition list
-   */
-  
-  /** Sorts transitions array in-place. */
-  public void sortTransitions(Comparator<Transition> comparator) {
-    // mergesort seems to perform better on already sorted arrays:
-    if (numTransitions > 1) ArrayUtil.timSort(transitionsArray, 0, numTransitions, comparator);
-  }
-  
-  /**
-   * Return this state's number. 
-   * <p>
-   * Expert: Will be useless unless {@link Automaton#getNumberedStates}
-   * has been called first to number the states.
-   * @return the number
-   */
-  public int getNumber() {
-    return number;
-  }
-  
-  /**
-   * Returns string describing this state. Normally invoked via
-   * {@link Automaton#toString()}.
-   */
-  @Override
-  public String toString() {
-    StringBuilder b = new StringBuilder();
-    b.append("state ").append(number);
-    if (accept) b.append(" [accept]");
-    else b.append(" [reject]");
-    b.append(":\n");
-    for (Transition t : getTransitions())
-      b.append("  ").append(t.toString()).append("\n");
-    return b.toString();
-  }
-  
-  /**
-   * Compares this object with the specified object for order. States are
-   * ordered by the time of construction.
-   */
-  @Override
-  public int compareTo(State s) {
-    return s.id - id;
-  }
-}
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/StatePair.java b/lucene/core/src/java/org/apache/lucene/util/automaton/StatePair.java
index 4124b9e..4ce81ab 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/StatePair.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/StatePair.java
@@ -35,11 +35,11 @@
  * @lucene.experimental
  */
 public class StatePair {
-  State s;
-  State s1;
-  State s2;
+  int s;
+  int s1;
+  int s2;
   
-  StatePair(State s, State s1, State s2) {
+  StatePair(int s, int s1, int s2) {
     this.s = s;
     this.s1 = s1;
     this.s2 = s2;
@@ -51,27 +51,10 @@
    * @param s1 first state
    * @param s2 second state
    */
-  public StatePair(State s1, State s2) {
+  public StatePair(int s1, int s2) {
     this.s1 = s1;
     this.s2 = s2;
-  }
-  
-  /**
-   * Returns first component of this pair.
-   * 
-   * @return first state
-   */
-  public State getFirstState() {
-    return s1;
-  }
-  
-  /**
-   * Returns second component of this pair.
-   * 
-   * @return second state
-   */
-  public State getSecondState() {
-    return s2;
+    this.s = -1;
   }
   
   /**
@@ -96,6 +79,11 @@
    */
   @Override
   public int hashCode() {
-    return s1.hashCode() + s2.hashCode();
+    return s1 ^ s2;
+  }
+
+  @Override
+  public String toString() {
+    return "StatePair(s1=" + s1 + " s2=" + s2 + ")";
   }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/Transition.java b/lucene/core/src/java/org/apache/lucene/util/automaton/Transition.java
index d22c6db..fc5b658 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/Transition.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/Transition.java
@@ -1,214 +1,51 @@
-/*
- * dk.brics.automaton
- * 
- * Copyright (c) 2001-2009 Anders Moeller
- * All rights reserved.
- * 
- * Redistribution and use in source and binary forms, with or without
- * modification, are permitted provided that the following conditions
- * are met:
- * 1. Redistributions of source code must retain the above copyright
- *    notice, this list of conditions and the following disclaimer.
- * 2. Redistributions in binary form must reproduce the above copyright
- *    notice, this list of conditions and the following disclaimer in the
- *    documentation and/or other materials provided with the distribution.
- * 3. The name of the author may not be used to endorse or promote products
- *    derived from this software without specific prior written permission.
- * 
- * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
- * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
- * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
- * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
- * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
- * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
- * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
- */
-
 package org.apache.lucene.util.automaton;
 
-import java.util.Comparator;
-
-/**
- * <tt>Automaton</tt> transition.
- * <p>
- * A transition, which belongs to a source state, consists of a Unicode
- * codepoint interval and a destination state.
- * 
- * @lucene.experimental
+/*
+ * 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.
  */
-public class Transition implements Cloneable {
-  
-  /*
-   * CLASS INVARIANT: min<=max
-   */
 
-  final int min;
-  final int max;
-  final State to;
-  
-  /**
-   * Constructs a new singleton interval transition.
-   * 
-   * @param c transition codepoint
-   * @param to destination state
-   */
-  public Transition(int c, State to) {
-    assert c >= 0;
-    min = max = c;
-    this.to = to;
+/** Holds one transition from an {@link Automaton}.  This is typically
+ *  used temporarily when iterating through transitions by invoking
+ *  {@link Automaton#initTransition} and {@link Automaton#getNextTransition}. */
+
+public class Transition {
+
+  /** Sole constructor. */
+  public Transition() {
   }
-  
-  /**
-   * Constructs a new transition. Both end points are included in the interval.
-   * 
-   * @param min transition interval minimum
-   * @param max transition interval maximum
-   * @param to destination state
-   */
-  public Transition(int min, int max, State to) {
-    assert min >= 0;
-    assert max >= 0;
-    if (max < min) {
-      int t = max;
-      max = min;
-      min = t;
-    }
-    this.min = min;
-    this.max = max;
-    this.to = to;
-  }
-  
-  /** Returns minimum of this transition interval. */
-  public int getMin() {
-    return min;
-  }
-  
-  /** Returns maximum of this transition interval. */
-  public int getMax() {
-    return max;
-  }
-  
-  /** Returns destination of this transition. */
-  public State getDest() {
-    return to;
-  }
-  
-  /**
-   * Checks for equality.
-   * 
-   * @param obj object to compare with
-   * @return true if <tt>obj</tt> is a transition with same character interval
-   *         and destination state as this transition.
-   */
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof Transition) {
-      Transition t = (Transition) obj;
-      return t.min == min && t.max == max && t.to == to;
-    } else return false;
-  }
-  
-  /**
-   * Returns hash code. The hash code is based on the character interval (not
-   * the destination state).
-   * 
-   * @return hash code
-   */
-  @Override
-  public int hashCode() {
-    return min * 2 + max * 3;
-  }
-  
-  /**
-   * Clones this transition.
-   * 
-   * @return clone with same character interval and destination state
-   */
-  @Override
-  public Transition clone() {
-    try {
-      return (Transition) super.clone();
-    } catch (CloneNotSupportedException e) {
-      throw new RuntimeException(e);
-    }
-  }
-  
-  static void appendCharString(int c, StringBuilder b) {
-    if (c >= 0x21 && c <= 0x7e && c != '\\' && c != '"') b.appendCodePoint(c);
-    else {
-      b.append("\\\\U");
-      String s = Integer.toHexString(c);
-      if (c < 0x10) b.append("0000000").append(s);
-      else if (c < 0x100) b.append("000000").append(s);
-      else if (c < 0x1000) b.append("00000").append(s);
-      else if (c < 0x10000) b.append("0000").append(s);
-      else if (c < 0x100000) b.append("000").append(s);
-      else if (c < 0x1000000) b.append("00").append(s);
-      else if (c < 0x10000000) b.append("0").append(s);
-      else b.append(s);
-    }
-  }
-  
-  /**
-   * Returns a string describing this state. Normally invoked via
-   * {@link Automaton#toString()}.
-   */
+
+  /** Source state. */
+  public int source;
+
+  /** Destination state. */
+  public int dest;
+
+  /** Minimum accepted label (inclusive). */
+  public int min;
+
+  /** Maximum accepted label (inclusive). */
+  public int max;
+
+  /** Remembers where we are in the iteration; init to -1 to provoke
+   *  exception if nextTransition is called without first initTransition. */
+  int transitionUpto = -1;
+
   @Override
   public String toString() {
-    StringBuilder b = new StringBuilder();
-    appendCharString(min, b);
-    if (min != max) {
-      b.append("-");
-      appendCharString(max, b);
-    }
-    b.append(" -> ").append(to.number);
-    return b.toString();
+    return source + " --> " + dest + " " + (char) min + "-" + (char) max;
   }
-  
-  void appendDot(StringBuilder b) {
-    b.append(" -> ").append(to.number).append(" [label=\"");
-    appendCharString(min, b);
-    if (min != max) {
-      b.append("-");
-      appendCharString(max, b);
-    }
-    b.append("\"]\n");
-  }
-
-  private static final class CompareByDestThenMinMaxSingle implements Comparator<Transition> {
-    @Override
-    public int compare(Transition t1, Transition t2) {
-      if (t1.to != t2.to) {
-        if (t1.to.number < t2.to.number) return -1;
-        else if (t1.to.number > t2.to.number) return 1;
-      }
-      if (t1.min < t2.min) return -1;
-      if (t1.min > t2.min) return 1;
-      if (t1.max > t2.max) return -1;
-      if (t1.max < t2.max) return 1;
-      return 0;
-    }
-  }
-
-  public static final Comparator<Transition> CompareByDestThenMinMax = new CompareByDestThenMinMaxSingle();
-
-  private static final class CompareByMinMaxThenDestSingle implements Comparator<Transition> {
-    @Override
-    public int compare(Transition t1, Transition t2) {
-      if (t1.min < t2.min) return -1;
-      if (t1.min > t2.min) return 1;
-      if (t1.max > t2.max) return -1;
-      if (t1.max < t2.max) return 1;
-      if (t1.to != t2.to) {
-        if (t1.to.number < t2.to.number) return -1;
-        if (t1.to.number > t2.to.number) return 1;
-      }
-      return 0;
-    }
-  }
-
-  public static final Comparator<Transition> CompareByMinMaxThenDest = new CompareByMinMaxThenDestSingle();
 }
+
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java b/lucene/core/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java
index 17be0ec..059ee09 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java
@@ -17,11 +17,9 @@
  * limitations under the License.
  */
 
-import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.ArrayUtil;
-
-import java.util.List;
+import java.util.Arrays;
 import java.util.ArrayList;
+import java.util.List;
 
 // TODO
 //   - do we really need the .bits...?  if not we can make util in UnicodeUtil to convert 1 char into a BytesRef
@@ -122,6 +120,10 @@
     }
   }
 
+  /** Sole constructor. */
+  public UTF32ToUTF8() {
+  }
+
   private final UTF8Sequence startUTF8 = new UTF8Sequence();
   private final UTF8Sequence endUTF8 = new UTF8Sequence();
 
@@ -129,37 +131,37 @@
   private final UTF8Sequence tmpUTF8b = new UTF8Sequence();
 
   // Builds necessary utf8 edges between start & end
-  void convertOneEdge(State start, State end, int startCodePoint, int endCodePoint) {
+  void convertOneEdge(int start, int end, int startCodePoint, int endCodePoint) {
     startUTF8.set(startCodePoint);
     endUTF8.set(endCodePoint);
-    //System.out.println("start = " + startUTF8);
-    //System.out.println("  end = " + endUTF8);
     build(start, end, startUTF8, endUTF8, 0);
   }
 
-  private void build(State start, State end, UTF8Sequence startUTF8, UTF8Sequence endUTF8, int upto) {
+  private void build(int start, int end, UTF8Sequence startUTF8, UTF8Sequence endUTF8, int upto) {
 
     // Break into start, middle, end:
     if (startUTF8.byteAt(upto) == endUTF8.byteAt(upto)) {
       // Degen case: lead with the same byte:
       if (upto == startUTF8.len-1 && upto == endUTF8.len-1) {
         // Super degen: just single edge, one UTF8 byte:
-        start.addTransition(new Transition(startUTF8.byteAt(upto), endUTF8.byteAt(upto), end));
+        utf8.addTransition(start, end, startUTF8.byteAt(upto), endUTF8.byteAt(upto));
         return;
       } else {
         assert startUTF8.len > upto+1;
         assert endUTF8.len > upto+1;
-        State n = newUTF8State();
+        int n = utf8.createState();
 
         // Single value leading edge
-        start.addTransition(new Transition(startUTF8.byteAt(upto), n));  // type=single
+        utf8.addTransition(start, n, startUTF8.byteAt(upto));
+        //start.addTransition(new Transition(startUTF8.byteAt(upto), n));  // type=single
 
         // Recurse for the rest
         build(n, end, startUTF8, endUTF8, 1+upto);
       }
     } else if (startUTF8.len == endUTF8.len) {
       if (upto == startUTF8.len-1) {
-        start.addTransition(new Transition(startUTF8.byteAt(upto), endUTF8.byteAt(upto), end));        // type=startend
+        //start.addTransition(new Transition(startUTF8.byteAt(upto), endUTF8.byteAt(upto), end));        // type=startend
+        utf8.addTransition(start, end, startUTF8.byteAt(upto), endUTF8.byteAt(upto));
       } else {
         start(start, end, startUTF8, upto, false);
         if (endUTF8.byteAt(upto) - startUTF8.byteAt(upto) > 1) {
@@ -193,62 +195,69 @@
     }
   }
 
-  private void start(State start, State end, UTF8Sequence utf8, int upto, boolean doAll) {
-    if (upto == utf8.len-1) {
+  private void start(int start, int end, UTF8Sequence startUTF8, int upto, boolean doAll) {
+    if (upto == startUTF8.len-1) {
       // Done recursing
-      start.addTransition(new Transition(utf8.byteAt(upto), utf8.byteAt(upto) | MASKS[utf8.numBits(upto)-1], end));  // type=start
+      utf8.addTransition(start, end, startUTF8.byteAt(upto), startUTF8.byteAt(upto) | MASKS[startUTF8.numBits(upto)-1]); // type=start
+      //start.addTransition(new Transition(startUTF8.byteAt(upto), startUTF8.byteAt(upto) | MASKS[startUTF8.numBits(upto)-1], end));  // type=start
     } else {
-      State n = newUTF8State();
-      start.addTransition(new Transition(utf8.byteAt(upto), n));  // type=start
-      start(n, end, utf8, 1+upto, true);
-      int endCode = utf8.byteAt(upto) | MASKS[utf8.numBits(upto)-1];
-      if (doAll && utf8.byteAt(upto) != endCode) {
-        all(start, end, utf8.byteAt(upto)+1, endCode, utf8.len-upto-1);
+      int n = utf8.createState();
+      utf8.addTransition(start, n, startUTF8.byteAt(upto));
+      //start.addTransition(new Transition(startUTF8.byteAt(upto), n));  // type=start
+      start(n, end, startUTF8, 1+upto, true);
+      int endCode = startUTF8.byteAt(upto) | MASKS[startUTF8.numBits(upto)-1];
+      if (doAll && startUTF8.byteAt(upto) != endCode) {
+        all(start, end, startUTF8.byteAt(upto)+1, endCode, startUTF8.len-upto-1);
       }
     }
   }
 
-  private void end(State start, State end, UTF8Sequence utf8, int upto, boolean doAll) {
-    if (upto == utf8.len-1) {
+  private void end(int start, int end, UTF8Sequence endUTF8, int upto, boolean doAll) {
+    if (upto == endUTF8.len-1) {
       // Done recursing
-      start.addTransition(new Transition(utf8.byteAt(upto) & (~MASKS[utf8.numBits(upto)-1]), utf8.byteAt(upto), end));   // type=end
+      //start.addTransition(new Transition(endUTF8.byteAt(upto) & (~MASKS[endUTF8.numBits(upto)-1]), endUTF8.byteAt(upto), end));   // type=end
+      utf8.addTransition(start, end, endUTF8.byteAt(upto) & (~MASKS[endUTF8.numBits(upto)-1]), endUTF8.byteAt(upto));
     } else {
       final int startCode;
-      if (utf8.numBits(upto) == 5) {
-        // special case -- avoid created unused edges (utf8
+      if (endUTF8.numBits(upto) == 5) {
+        // special case -- avoid created unused edges (endUTF8
         // doesn't accept certain byte sequences) -- there
         // are other cases we could optimize too:
         startCode = 194;
       } else {
-        startCode = utf8.byteAt(upto) & (~MASKS[utf8.numBits(upto)-1]);
+        startCode = endUTF8.byteAt(upto) & (~MASKS[endUTF8.numBits(upto)-1]);
       }
-      if (doAll && utf8.byteAt(upto) != startCode) {
-        all(start, end, startCode, utf8.byteAt(upto)-1, utf8.len-upto-1);
+      if (doAll && endUTF8.byteAt(upto) != startCode) {
+        all(start, end, startCode, endUTF8.byteAt(upto)-1, endUTF8.len-upto-1);
       }
-      State n = newUTF8State();
-      start.addTransition(new Transition(utf8.byteAt(upto), n));  // type=end
-      end(n, end, utf8, 1+upto, true);
+      int n = utf8.createState();
+      //start.addTransition(new Transition(endUTF8.byteAt(upto), n));  // type=end
+      utf8.addTransition(start, n, endUTF8.byteAt(upto));
+      end(n, end, endUTF8, 1+upto, true);
     }
   }
 
-  private void all(State start, State end, int startCode, int endCode, int left) {
+  private void all(int start, int end, int startCode, int endCode, int left) {
     if (left == 0) {
-      start.addTransition(new Transition(startCode, endCode, end));  // type=all
+      //start.addTransition(new Transition(startCode, endCode, end));  // type=all
+      utf8.addTransition(start, end, startCode, endCode);
     } else {
-      State lastN = newUTF8State();
-      start.addTransition(new Transition(startCode, endCode, lastN));  // type=all
+      int lastN = utf8.createState();
+      //start.addTransition(new Transition(startCode, endCode, lastN));  // type=all
+      utf8.addTransition(start, lastN, startCode, endCode);
       while (left > 1) {
-        State n = newUTF8State();
-        lastN.addTransition(new Transition(128, 191, n));  // type=all*
+        int n = utf8.createState();
+        //lastN.addTransition(new Transition(128, 191, n));  // type=all*
+        utf8.addTransition(lastN, n, 128, 191); // type=all*
         left--;
         lastN = n;
       }
-      lastN.addTransition(new Transition(128, 191, end)); // type = all*
+      //lastN.addTransition(new Transition(128, 191, end)); // type = all*
+      utf8.addTransition(lastN, end, 128, 191); // type = all*
     }
   }
 
-  private State[] utf8States;
-  private int utf8StateCount;
+  Automaton.Builder utf8;
 
   /** Converts an incoming utf32 automaton to an equivalent
    *  utf8 one.  The incoming automaton need not be
@@ -256,61 +265,49 @@
    *  not in general be deterministic, so you must
    *  determinize it if that's needed. */
   public Automaton convert(Automaton utf32) {
-    if (utf32.isSingleton()) {
-      utf32 = utf32.cloneExpanded();
+    if (utf32.getNumStates() == 0) {
+      return utf32;
     }
 
-    State[] map = new State[utf32.getNumberedStates().length];
-    List<State> pending = new ArrayList<>();
-    State utf32State = utf32.getInitialState();
+    int[] map = new int[utf32.getNumStates()];
+    Arrays.fill(map, -1);
+
+    List<Integer> pending = new ArrayList<>();
+    int utf32State = 0;
     pending.add(utf32State);
-    Automaton utf8 = new Automaton();
-    utf8.setDeterministic(false);
+    utf8 = new Automaton.Builder();
+       
+    int utf8State = utf8.createState();
 
-    State utf8State = utf8.getInitialState();
+    utf8.setAccept(utf8State, utf32.isAccept(utf32State));
 
-    utf8States = new State[5];
-    utf8StateCount = 0;
-    utf8State.number = utf8StateCount;
-    utf8States[utf8StateCount] = utf8State;
-    utf8StateCount++;
-
-    utf8State.setAccept(utf32State.isAccept());
-
-    map[utf32State.number] = utf8State;
+    map[utf32State] = utf8State;
     
-    while(pending.size() != 0) {
+    Transition scratch = new Transition();
+    
+    while (pending.size() != 0) {
       utf32State = pending.remove(pending.size()-1);
-      utf8State = map[utf32State.number];
-      for(int i=0;i<utf32State.numTransitions;i++) {
-        final Transition t = utf32State.transitionsArray[i];
-        final State destUTF32 = t.to;
-        State destUTF8 = map[destUTF32.number];
-        if (destUTF8 == null) {
-          destUTF8 = newUTF8State();
-          destUTF8.accept = destUTF32.accept;
-          map[destUTF32.number] = destUTF8;
+      utf8State = map[utf32State];
+      assert utf8State != -1;
+
+      int numTransitions = utf32.getNumTransitions(utf32State);
+      utf32.initTransition(utf32State, scratch);
+      for(int i=0;i<numTransitions;i++) {
+        utf32.getNextTransition(scratch);
+        int destUTF32 = scratch.dest;
+        int destUTF8 = map[destUTF32];
+        if (destUTF8 == -1) {
+          destUTF8 = utf8.createState();
+          utf8.setAccept(destUTF8, utf32.isAccept(destUTF32));
+          map[destUTF32] = destUTF8;
           pending.add(destUTF32);
         }
-        convertOneEdge(utf8State, destUTF8, t.min, t.max);
+
+        // Writes new transitions into pendingTransitions:
+        convertOneEdge(utf8State, destUTF8, scratch.min, scratch.max);
       }
     }
 
-    utf8.setNumberedStates(utf8States, utf8StateCount);
-
-    return utf8;
-  }
-
-  private State newUTF8State() {
-    State s = new State();
-    if (utf8StateCount == utf8States.length) {
-      final State[] newArray = new State[ArrayUtil.oversize(1+utf8StateCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-      System.arraycopy(utf8States, 0, newArray, 0, utf8StateCount);
-      utf8States = newArray;
-    }
-    utf8States[utf8StateCount] = s;
-    s.number = utf8StateCount;
-    utf8StateCount++;
-    return s;
+    return utf8.finish();
   }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java b/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
index eb688f0..6802688 100644
--- a/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
+++ b/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
@@ -18,8 +18,8 @@
  */
 
 import java.io.IOException;
-import java.io.StringWriter;
 import java.io.PrintWriter;
+import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -29,9 +29,9 @@
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.BasicAutomata;
-import org.apache.lucene.util.automaton.BasicOperations;
 
 public class TestGraphTokenizers extends BaseTokenStreamTestCase {
 
@@ -410,8 +410,9 @@
         token("abc", 1, 1),
       });
     final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
-    final Automaton expected = BasicAutomata.makeString("abc");
-    assertTrue(BasicOperations.sameLanguage(expected, actual));
+    final Automaton expected = s2a("abc");
+    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
+                                       Operations.determinize(Operations.removeDeadStates(actual))));
   }
 
   public void testMultipleHoles() throws Exception {
@@ -422,7 +423,8 @@
       });
     final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
     final Automaton expected = join(s2a("a"), SEP_A, HOLE_A, SEP_A, HOLE_A, SEP_A, s2a("b")); 
-    assertTrue(BasicOperations.sameLanguage(expected, actual));
+    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
+                                       Operations.determinize(Operations.removeDeadStates(actual))));
   }
 
   public void testSynOverMultipleHoles() throws Exception {
@@ -435,8 +437,9 @@
     final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
     final Automaton a1 = join(s2a("a"), SEP_A, HOLE_A, SEP_A, HOLE_A, SEP_A, s2a("b")); 
     final Automaton a2 = join(s2a("x"), SEP_A, s2a("b")); 
-    final Automaton expected = BasicOperations.union(a1, a2);
-    assertTrue(BasicOperations.sameLanguage(expected, actual));
+    final Automaton expected = Operations.union(a1, a2);
+    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
+                                       Operations.determinize(Operations.removeDeadStates(actual))));
   }
 
   // for debugging!
@@ -450,25 +453,25 @@
   }
   */
 
-  private static final Automaton SEP_A = BasicAutomata.makeChar(TokenStreamToAutomaton.POS_SEP);
-  private static final Automaton HOLE_A = BasicAutomata.makeChar(TokenStreamToAutomaton.HOLE);
+  private static final Automaton SEP_A = Automata.makeChar(TokenStreamToAutomaton.POS_SEP);
+  private static final Automaton HOLE_A = Automata.makeChar(TokenStreamToAutomaton.HOLE);
 
   private Automaton join(String ... strings) {
     List<Automaton> as = new ArrayList<>();
     for(String s : strings) {
-      as.add(BasicAutomata.makeString(s));
+      as.add(s2a(s));
       as.add(SEP_A);
     }
     as.remove(as.size()-1);
-    return BasicOperations.concatenate(as);
+    return Operations.concatenate(as);
   }
 
   private Automaton join(Automaton ... as) {
-    return BasicOperations.concatenate(Arrays.asList(as));
+    return Operations.concatenate(Arrays.asList(as));
   }
 
   private Automaton s2a(String s) {
-    return BasicAutomata.makeString(s);
+    return Automata.makeString(s);
   }
 
   public void testTwoTokens() throws Exception {
@@ -482,7 +485,8 @@
     final Automaton expected =  join("abc", "def");
 
     //toDot(actual);
-    assertTrue(BasicOperations.sameLanguage(expected, actual));
+    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
+                                       Operations.determinize(Operations.removeDeadStates(actual))));
   }
 
   public void testHole() throws Exception {
@@ -497,7 +501,8 @@
     final Automaton expected = join(s2a("abc"), SEP_A, HOLE_A, SEP_A, s2a("def"));
 
     //toDot(actual);
-    assertTrue(BasicOperations.sameLanguage(expected, actual));
+    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
+                                       Operations.determinize(Operations.removeDeadStates(actual))));
   }
 
   public void testOverlappedTokensSausage() throws Exception {
@@ -509,10 +514,11 @@
         token("xyz", 0, 1)
       });
     final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
-    final Automaton a1 = BasicAutomata.makeString("abc");
-    final Automaton a2 = BasicAutomata.makeString("xyz");
-    final Automaton expected = BasicOperations.union(a1, a2);
-    assertTrue(BasicOperations.sameLanguage(expected, actual));
+    final Automaton a1 = s2a("abc");
+    final Automaton a2 = s2a("xyz");
+    final Automaton expected = Operations.union(a1, a2);
+    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
+                                       Operations.determinize(Operations.removeDeadStates(actual))));
   }
 
   public void testOverlappedTokensLattice() throws Exception {
@@ -524,12 +530,13 @@
         token("def", 1, 1),
       });
     final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
-    final Automaton a1 = BasicAutomata.makeString("xyz");
+    final Automaton a1 = s2a("xyz");
     final Automaton a2 = join("abc", "def");
                                                                    
-    final Automaton expected = BasicOperations.union(a1, a2);
+    final Automaton expected = Operations.union(a1, a2);
     //toDot(actual);
-    assertTrue(BasicOperations.sameLanguage(expected, actual));
+    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
+                                       Operations.determinize(Operations.removeDeadStates(actual))));
   }
 
   public void testSynOverHole() throws Exception {
@@ -541,13 +548,14 @@
         token("b", 2, 1),
       });
     final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
-    final Automaton a1 = BasicOperations.union(
+    final Automaton a1 = Operations.union(
                                                join(s2a("a"), SEP_A, HOLE_A),
-                                               BasicAutomata.makeString("X"));
-    final Automaton expected = BasicOperations.concatenate(a1,
+                                               s2a("X"));
+    final Automaton expected = Operations.concatenate(a1,
                                                            join(SEP_A, s2a("b")));
     //toDot(actual);
-    assertTrue(BasicOperations.sameLanguage(expected, actual));
+    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
+                                       Operations.determinize(Operations.removeDeadStates(actual))));
   }
 
   public void testSynOverHole2() throws Exception {
@@ -559,10 +567,11 @@
         token("def", 2, 1),
       });
     final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
-    final Automaton expected = BasicOperations.union(
+    final Automaton expected = Operations.union(
                                                      join(s2a("xyz"), SEP_A, HOLE_A, SEP_A, s2a("def")),
-                                                     BasicAutomata.makeString("abc"));
-    assertTrue(BasicOperations.sameLanguage(expected, actual));
+                                                     s2a("abc"));
+    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
+                                       Operations.determinize(Operations.removeDeadStates(actual))));
   }
 
   public void testOverlappedTokensLattice2() throws Exception {
@@ -575,11 +584,12 @@
         token("ghi", 1, 1),
       });
     final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
-    final Automaton a1 = BasicAutomata.makeString("xyz");
+    final Automaton a1 = s2a("xyz");
     final Automaton a2 = join("abc", "def", "ghi");
-    final Automaton expected = BasicOperations.union(a1, a2);
+    final Automaton expected = Operations.union(a1, a2);
     //toDot(actual);
-    assertTrue(BasicOperations.sameLanguage(expected, actual));
+    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
+                                       Operations.determinize(Operations.removeDeadStates(actual))));
   }
 
   public void testToDot() throws Exception {
@@ -597,7 +607,8 @@
     final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
     final Automaton expected = join(HOLE_A, SEP_A, s2a("abc"));
     //toDot(actual);
-    assertTrue(BasicOperations.sameLanguage(expected, actual));
+    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
+                                       Operations.determinize(Operations.removeDeadStates(actual))));
   }
 
   // TODO: testEndsWithHole... but we need posInc to set in TS.end()
@@ -609,8 +620,9 @@
         token("X", 0, 10),
       });
     final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
-    final Automaton expected = BasicOperations.union(BasicAutomata.makeString("a"),
-                                                     BasicAutomata.makeString("X"));
-    assertTrue(BasicOperations.sameLanguage(expected, actual));
+    final Automaton expected = Operations.union(s2a("a"),
+                                                               s2a("X"));
+    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
+                                       Operations.determinize(Operations.removeDeadStates(actual))));
   }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java b/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java
index 18d5f80..9d68ea8 100644
--- a/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java
+++ b/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java
@@ -34,10 +34,9 @@
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.AutomatonTestUtil;
-import org.apache.lucene.util.automaton.BasicAutomata;
-import org.apache.lucene.util.automaton.BasicOperations;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 import org.apache.lucene.util.automaton.RegExp;
 
@@ -165,9 +164,9 @@
   public void testKeep() throws Exception {
     CharacterRunAutomaton keepWords = 
       new CharacterRunAutomaton(
-          BasicOperations.complement(
-              Automaton.union(
-                  Arrays.asList(BasicAutomata.makeString("foo"), BasicAutomata.makeString("bar")))));
+          Operations.complement(
+              Operations.union(
+                  Arrays.asList(Automata.makeString("foo"), Automata.makeString("bar")))));
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true, keepWords);
     assertAnalyzesTo(a, "quick foo brown bar bar fox foo",
         new String[] { "foo", "bar", "bar", "foo" },
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
index 03881d9..533cc72 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
@@ -84,9 +84,9 @@
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.ThreadInterruptedException;
 import org.apache.lucene.util.Version;
-import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.BasicAutomata;
+import org.apache.lucene.util.automaton.Automata;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
+import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.packed.PackedInts;
 import org.junit.Test;
 
@@ -2006,7 +2006,7 @@
   public void testStopwordsPosIncHole2() throws Exception {
     // use two stopfilters for testing here
     Directory dir = newDirectory();
-    final Automaton secondSet = BasicAutomata.makeString("foobar");
+    final Automaton secondSet = Automata.makeString("foobar");
     Analyzer a = new Analyzer() {
       @Override
       protected TokenStreamComponents createComponents(String fieldName) {
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java b/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java
index ecf4e9b..9acc4fd 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java
@@ -33,9 +33,9 @@
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 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.BasicAutomata;
+import org.apache.lucene.util.automaton.Automata;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.RegExp;
 
 @SuppressCodecs({ "SimpleText", "Memory", "Direct" })
@@ -244,7 +244,7 @@
         if (VERBOSE) {
           System.out.println("\nTEST: empty automaton");
         }
-        a = BasicAutomata.makeEmpty();
+        a = Automata.makeEmpty();
       } else {
         if (VERBOSE) {
           System.out.println("\nTEST: keepPct=" + keepPct);
@@ -259,16 +259,9 @@
           acceptTerms.add(s2);
           sortedAcceptTerms.add(new BytesRef(s2));
         }
-        a = BasicAutomata.makeStringUnion(sortedAcceptTerms);
+        a = Automata.makeStringUnion(sortedAcceptTerms);
       }
       
-      if (random().nextBoolean()) {
-        if (VERBOSE) {
-          System.out.println("TEST: reduce the automaton");
-        }
-        a.reduce();
-      }
-
       final CompiledAutomaton c = new CompiledAutomaton(a, true, false);
 
       final BytesRef[] acceptTermsArray = new BytesRef[acceptTerms.size()];
@@ -745,7 +738,7 @@
     w.shutdown();
     AtomicReader sub = getOnlySegmentReader(r);
     Terms terms = sub.fields().terms("field");
-    Automaton automaton = new RegExp(".*", RegExp.NONE).toAutomaton();    
+    Automaton automaton = new RegExp(".*", RegExp.NONE).toAutomaton();
     CompiledAutomaton ca = new CompiledAutomaton(automaton, false, false);    
     TermsEnum te = terms.intersect(ca, null);
     assertEquals("aaa", te.next().utf8ToString());
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum2.java b/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum2.java
index 55f8f0d..f261f16 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum2.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum2.java
@@ -31,6 +31,7 @@
 import org.apache.lucene.search.AutomatonQuery;
 import org.apache.lucene.search.CheckHits;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
@@ -68,7 +69,7 @@
       writer.addDocument(doc);
     }
     
-    termsAutomaton = BasicAutomata.makeStringUnion(terms);
+    termsAutomaton = Automata.makeStringUnion(terms);
     
     reader = writer.getReader();
     searcher = newSearcher(reader);
@@ -84,23 +85,27 @@
   
   /** tests a pre-intersected automaton against the original */
   public void testFiniteVersusInfinite() throws Exception {
+
     for (int i = 0; i < numIterations; i++) {
       String reg = AutomatonTestUtil.randomRegexp(random());
-      Automaton automaton = new RegExp(reg, RegExp.NONE).toAutomaton();
+      Automaton automaton = Operations.determinize(new RegExp(reg, RegExp.NONE).toAutomaton());
       final List<BytesRef> matchedTerms = new ArrayList<>();
       for(BytesRef t : terms) {
-        if (BasicOperations.run(automaton, t.utf8ToString())) {
+        if (Operations.run(automaton, t.utf8ToString())) {
           matchedTerms.add(t);
         }
       }
 
-      Automaton alternate = BasicAutomata.makeStringUnion(matchedTerms);
+      Automaton alternate = Automata.makeStringUnion(matchedTerms);
       //System.out.println("match " + matchedTerms.size() + " " + alternate.getNumberOfStates() + " states, sigma=" + alternate.getStartPoints().length);
       //AutomatonTestUtil.minimizeSimple(alternate);
       //System.out.println("minmize done");
       AutomatonQuery a1 = new AutomatonQuery(new Term("field", ""), automaton);
       AutomatonQuery a2 = new AutomatonQuery(new Term("field", ""), alternate);
-      CheckHits.checkEqual(a1, searcher.search(a1, 25).scoreDocs, searcher.search(a2, 25).scoreDocs);
+
+      ScoreDoc[] origHits = searcher.search(a1, 25).scoreDocs;
+      ScoreDoc[] newHits = searcher.search(a2, 25).scoreDocs;
+      CheckHits.checkEqual(a1, origHits, newHits);
     }
   }
   
@@ -108,13 +113,13 @@
   public void testSeeking() throws Exception {
     for (int i = 0; i < numIterations; i++) {
       String reg = AutomatonTestUtil.randomRegexp(random());
-      Automaton automaton = new RegExp(reg, RegExp.NONE).toAutomaton();
+      Automaton automaton = Operations.determinize(new RegExp(reg, RegExp.NONE).toAutomaton());
       TermsEnum te = MultiFields.getTerms(reader, "field").iterator(null);
       ArrayList<BytesRef> unsortedTerms = new ArrayList<>(terms);
       Collections.shuffle(unsortedTerms, random());
 
       for (BytesRef term : unsortedTerms) {
-        if (BasicOperations.run(automaton, term.utf8ToString())) {
+        if (Operations.run(automaton, term.utf8ToString())) {
           // term is accepted
           if (random().nextBoolean()) {
             // seek exact
@@ -153,16 +158,16 @@
     for (int i = 0; i < numIterations; i++) {
       String reg = AutomatonTestUtil.randomRegexp(random());
       Automaton automaton = new RegExp(reg, RegExp.NONE).toAutomaton();
-      CompiledAutomaton ca = new CompiledAutomaton(automaton, SpecialOperations.isFinite(automaton), false);
+      CompiledAutomaton ca = new CompiledAutomaton(automaton, Operations.isFinite(automaton), false);
       TermsEnum te = MultiFields.getTerms(reader, "field").intersect(ca, null);
-      Automaton expected = BasicOperations.intersection(termsAutomaton, automaton);
+      Automaton expected = Operations.determinize(Operations.intersection(termsAutomaton, automaton));
       TreeSet<BytesRef> found = new TreeSet<>();
       while (te.next() != null) {
         found.add(BytesRef.deepCopyOf(te.term()));
       }
       
-      Automaton actual = BasicAutomata.makeStringUnion(found);     
-      assertTrue(BasicOperations.sameLanguage(expected, actual));
+      Automaton actual = Operations.determinize(Automata.makeStringUnion(found));
+      assertTrue(Operations.sameLanguage(expected, actual));
     }
   }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java
index 1887ed0..c1cc032 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java
@@ -33,10 +33,10 @@
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.Rethrow;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.AutomatonTestUtil;
-import org.apache.lucene.util.automaton.BasicAutomata;
-import org.apache.lucene.util.automaton.BasicOperations;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.Operations;
+import org.apache.lucene.util.automaton.Automaton;
 
 public class TestAutomatonQuery extends LuceneTestCase {
   private Directory directory;
@@ -106,24 +106,24 @@
   /**
    * Test some very simple automata.
    */
-  public void testBasicAutomata() throws IOException {
-    assertAutomatonHits(0, BasicAutomata.makeEmpty());
-    assertAutomatonHits(0, BasicAutomata.makeEmptyString());
-    assertAutomatonHits(2, BasicAutomata.makeAnyChar());
-    assertAutomatonHits(3, BasicAutomata.makeAnyString());
-    assertAutomatonHits(2, BasicAutomata.makeString("doc"));
-    assertAutomatonHits(1, BasicAutomata.makeChar('a'));
-    assertAutomatonHits(2, BasicAutomata.makeCharRange('a', 'b'));
-    assertAutomatonHits(2, BasicAutomata.makeInterval(1233, 2346, 0));
-    assertAutomatonHits(1, BasicAutomata.makeInterval(0, 2000, 0));
-    assertAutomatonHits(2, BasicOperations.union(BasicAutomata.makeChar('a'),
-        BasicAutomata.makeChar('b')));
-    assertAutomatonHits(0, BasicOperations.intersection(BasicAutomata
-        .makeChar('a'), BasicAutomata.makeChar('b')));
-    assertAutomatonHits(1, BasicOperations.minus(BasicAutomata.makeCharRange('a', 'b'), 
-        BasicAutomata.makeChar('a')));
+  public void testAutomata() throws IOException {
+    assertAutomatonHits(0, Automata.makeEmpty());
+    assertAutomatonHits(0, Automata.makeEmptyString());
+    assertAutomatonHits(2, Automata.makeAnyChar());
+    assertAutomatonHits(3, Automata.makeAnyString());
+    assertAutomatonHits(2, Automata.makeString("doc"));
+    assertAutomatonHits(1, Automata.makeChar('a'));
+    assertAutomatonHits(2, Automata.makeCharRange('a', 'b'));
+    assertAutomatonHits(2, Automata.makeInterval(1233, 2346, 0));
+    assertAutomatonHits(1, Automata.makeInterval(0, 2000, 0));
+    assertAutomatonHits(2, Operations.union(Automata.makeChar('a'),
+        Automata.makeChar('b')));
+    assertAutomatonHits(0, Operations.intersection(Automata
+        .makeChar('a'), Automata.makeChar('b')));
+    assertAutomatonHits(1, Operations.minus(Automata.makeCharRange('a', 'b'), 
+        Automata.makeChar('a')));
   }
-  
+
   /**
    * Test that a nondeterministic automaton works correctly. (It should will be
    * determinized)
@@ -131,26 +131,27 @@
   public void testNFA() throws IOException {
     // accept this or three, the union is an NFA (two transitions for 't' from
     // initial state)
-    Automaton nfa = BasicOperations.union(BasicAutomata.makeString("this"),
-        BasicAutomata.makeString("three"));
+    Automaton nfa = Operations.union(Automata.makeString("this"),
+        Automata.makeString("three"));
     assertAutomatonHits(2, nfa);
   }
   
   public void testEquals() {
-    AutomatonQuery a1 = new AutomatonQuery(newTerm("foobar"), BasicAutomata
+    AutomatonQuery a1 = new AutomatonQuery(newTerm("foobar"), Automata
         .makeString("foobar"));
     // reference to a1
     AutomatonQuery a2 = a1;
     // same as a1 (accepts the same language, same term)
-    AutomatonQuery a3 = new AutomatonQuery(newTerm("foobar"), BasicOperations
-        .concatenate(BasicAutomata.makeString("foo"), BasicAutomata
-            .makeString("bar")));
+    AutomatonQuery a3 = new AutomatonQuery(newTerm("foobar"),
+                            Operations.concatenate(
+                                 Automata.makeString("foo"),
+                                 Automata.makeString("bar")));
     // different than a1 (same term, but different language)
-    AutomatonQuery a4 = new AutomatonQuery(newTerm("foobar"), BasicAutomata
-        .makeString("different"));
+    AutomatonQuery a4 = new AutomatonQuery(newTerm("foobar"),
+                                           Automata.makeString("different"));
     // different than a1 (different term, same language)
-    AutomatonQuery a5 = new AutomatonQuery(newTerm("blah"), BasicAutomata
-        .makeString("foobar"));
+    AutomatonQuery a5 = new AutomatonQuery(newTerm("blah"),
+                                           Automata.makeString("foobar"));
     
     assertEquals(a1.hashCode(), a2.hashCode());
     assertEquals(a1, a2);
@@ -176,8 +177,7 @@
    * MultiTermQuery semantics.
    */
   public void testRewriteSingleTerm() throws IOException {
-    AutomatonQuery aq = new AutomatonQuery(newTerm("bogus"), BasicAutomata
-        .makeString("piece"));
+    AutomatonQuery aq = new AutomatonQuery(newTerm("bogus"), Automata.makeString("piece"));
     Terms terms = MultiFields.getTerms(searcher.getIndexReader(), FN);
     assertTrue(aq.getTermsEnum(terms) instanceof SingleTermsEnum);
     assertEquals(1, automatonQueryNrHits(aq));
@@ -188,10 +188,8 @@
    * MultiTermQuery semantics.
    */
   public void testRewritePrefix() throws IOException {
-    Automaton pfx = BasicAutomata.makeString("do");
-    pfx.expandSingleton(); // expand singleton representation for testing
-    Automaton prefixAutomaton = BasicOperations.concatenate(pfx, BasicAutomata
-        .makeAnyString());
+    Automaton pfx = Automata.makeString("do");
+    Automaton prefixAutomaton = Operations.concatenate(pfx, Automata.makeAnyString());
     AutomatonQuery aq = new AutomatonQuery(newTerm("bogus"), prefixAutomaton);
     Terms terms = MultiFields.getTerms(searcher.getIndexReader(), FN);
     assertTrue(aq.getTermsEnum(terms) instanceof PrefixTermsEnum);
@@ -202,8 +200,7 @@
    * Test handling of the empty language
    */
   public void testEmptyOptimization() throws IOException {
-    AutomatonQuery aq = new AutomatonQuery(newTerm("bogus"), BasicAutomata
-        .makeEmpty());
+    AutomatonQuery aq = new AutomatonQuery(newTerm("bogus"), Automata.makeEmpty());
     // not yet available: assertTrue(aq.getEnum(searcher.getIndexReader())
     // instanceof EmptyTermEnum);
     Terms terms = MultiFields.getTerms(searcher.getIndexReader(), FN);
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestDocTermOrdsRewriteMethod.java b/lucene/core/src/test/org/apache/lucene/search/TestDocTermOrdsRewriteMethod.java
index d80d703..9dd845f 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestDocTermOrdsRewriteMethod.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestDocTermOrdsRewriteMethod.java
@@ -78,7 +78,7 @@
       Collections.sort(terms);
       System.out.println("UTF16 order:");
       for(String s : terms) {
-        System.out.println("  " + UnicodeUtil.toHexString(s));
+        System.out.println("  " + UnicodeUtil.toHexString(s) + " " + s);
       }
     }
     
@@ -115,7 +115,7 @@
   /** check that the # of hits is the same as if the query
    * is run against the inverted index
    */
-  protected void assertSame(String regexp) throws IOException {   
+  protected void assertSame(String regexp) throws IOException {
     RegexpQuery docValues = new RegexpQuery(new Term(fieldName, regexp), RegExp.NONE);
     docValues.setRewriteMethod(new DocTermOrdsRewriteMethod());
     RegexpQuery inverted = new RegexpQuery(new Term(fieldName, regexp), RegExp.NONE);
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestFuzzyQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestFuzzyQuery.java
index b276b89..327a003 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestFuzzyQuery.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestFuzzyQuery.java
@@ -38,6 +38,21 @@
  */
 public class TestFuzzyQuery extends LuceneTestCase {
 
+  public void testBasicPrefix() throws Exception {
+    Directory directory = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), directory);
+    addDoc("abc", writer);
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    writer.shutdown();
+
+    FuzzyQuery query = new FuzzyQuery(new Term("field", "abc"), FuzzyQuery.defaultMaxEdits, 1);
+    ScoreDoc[] hits = searcher.search(query, null, 1000).scoreDocs;
+    assertEquals(1, hits.length);
+    reader.close();
+    directory.close();
+  }
+
   public void testFuzziness() throws Exception {
     Directory directory = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), directory);
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestRegexpQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestRegexpQuery.java
index 05ea7a1..6ed3140 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestRegexpQuery.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestRegexpQuery.java
@@ -27,10 +27,10 @@
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.AutomatonProvider;
-import org.apache.lucene.util.automaton.BasicAutomata;
-import org.apache.lucene.util.automaton.BasicOperations;
 import org.apache.lucene.util.automaton.RegExp;
 
 /**
@@ -97,10 +97,10 @@
   public void testCustomProvider() throws IOException {
     AutomatonProvider myProvider = new AutomatonProvider() {
       // automaton that matches quick or brown
-      private Automaton quickBrownAutomaton = BasicOperations.union(Arrays
-          .asList(BasicAutomata.makeString("quick"),
-          BasicAutomata.makeString("brown"),
-          BasicAutomata.makeString("bob")));
+      private Automaton quickBrownAutomaton = Operations.union(Arrays
+          .asList(Automata.makeString("quick"),
+          Automata.makeString("brown"),
+          Automata.makeString("bob")));
       
       @Override
       public Automaton getAutomaton(String name) {
@@ -108,8 +108,7 @@
         else return null;
       }
     };
-    RegexpQuery query = new RegexpQuery(newTerm("<quickBrown>"), RegExp.ALL,
-        myProvider);
+    RegexpQuery query = new RegexpQuery(newTerm("<quickBrown>"), RegExp.ALL, myProvider);
     assertEquals(1, searcher.search(query, 5).totalHits);
   }
   
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestRegexpRandom2.java b/lucene/core/src/test/org/apache/lucene/search/TestRegexpRandom2.java
index f6f2be5..ead284b 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestRegexpRandom2.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestRegexpRandom2.java
@@ -40,9 +40,9 @@
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.UnicodeUtil;
-import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.AutomatonTestUtil;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
+import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.RegExp;
 
 /**
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestWildcard.java b/lucene/core/src/test/org/apache/lucene/search/TestWildcard.java
index 5d65451..195c3fd 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestWildcard.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestWildcard.java
@@ -268,7 +268,7 @@
    * Test that wild card queries are parsed to the correct type and are searched correctly.
    * This test looks at both parsing and execution of wildcard queries.
    * Although placed here, it also tests prefix queries, verifying that
-   * prefix queries are not parsed into wild card queries, and viceversa.
+   * prefix queries are not parsed into wild card queries, and vice-versa.
    */
   public void testParsingAndSearching() throws Exception {
     String field = "content";
diff --git a/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java b/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java
new file mode 100644
index 0000000..fe0842f
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java
@@ -0,0 +1,1065 @@
+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.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util.automaton.AutomatonTestUtil.RandomAcceptedStrings;
+import org.apache.lucene.util.fst.Util;
+
+public class TestAutomaton extends LuceneTestCase {
+
+  public void testBasic() throws Exception {
+    Automaton a = new Automaton();
+    int start = a.createState();
+    int x = a.createState();
+    int y = a.createState();
+    int end = a.createState();
+    a.setAccept(end, true);
+
+    a.addTransition(start, x, 'a', 'a');
+    a.addTransition(start, end, 'd', 'd');
+    a.addTransition(x, y, 'b', 'b');
+    a.addTransition(y, end, 'c', 'c');
+    a.finishState();
+  }
+
+  public void testReduceBasic() throws Exception {
+    Automaton a = new Automaton();
+    int start = a.createState();
+    int end = a.createState();
+    a.setAccept(end, true);
+    // Should collapse to a-b:
+    a.addTransition(start, end, 'a', 'a');
+    a.addTransition(start, end, 'b', 'b');
+    a.addTransition(start, end, 'm', 'm');
+    // Should collapse to x-y:
+    a.addTransition(start, end, 'x', 'x');
+    a.addTransition(start, end, 'y', 'y');
+
+    a.finishState();
+    assertEquals(3, a.getNumTransitions(start));
+    Transition scratch = new Transition();
+    a.initTransition(start, scratch);
+    a.getNextTransition(scratch);
+    assertEquals('a', scratch.min);
+    assertEquals('b', scratch.max);
+    a.getNextTransition(scratch);
+    assertEquals('m', scratch.min);
+    assertEquals('m', scratch.max);
+    a.getNextTransition(scratch);
+    assertEquals('x', scratch.min);
+    assertEquals('y', scratch.max);
+  }
+
+  public void testSameLanguage() throws Exception {
+    Automaton a1 = Automata.makeString("foobar");
+    Automaton a2 = Operations.removeDeadStates(Operations.concatenate(
+                            Automata.makeString("foo"),
+                            Automata.makeString("bar")));
+    assertTrue(Operations.sameLanguage(a1, a2));
+  }
+
+  public void testCommonPrefix() throws Exception {
+    Automaton a = Operations.concatenate(
+                            Automata.makeString("foobar"),
+                            Automata.makeAnyString());
+    assertEquals("foobar", Operations.getCommonPrefix(a));
+  }
+
+  public void testConcatenate1() throws Exception {
+    Automaton a = Operations.concatenate(
+                            Automata.makeString("m"),
+                            Automata.makeAnyString());
+    assertTrue(Operations.run(a, "m"));
+    assertTrue(Operations.run(a, "me"));
+    assertTrue(Operations.run(a, "me too"));
+  }
+
+  public void testConcatenate2() throws Exception {
+    Automaton a = Operations.concatenate(Arrays.asList(
+                            Automata.makeString("m"),
+                            Automata.makeAnyString(),
+                            Automata.makeString("n"),
+                            Automata.makeAnyString()));
+    a = Operations.determinize(a);
+    assertTrue(Operations.run(a, "mn"));
+    assertTrue(Operations.run(a, "mone"));
+    assertFalse(Operations.run(a, "m"));
+    assertFalse(Operations.isFinite(a));
+  }
+
+  public void testUnion1() throws Exception {
+    Automaton a = Operations.union(Arrays.asList(
+                            Automata.makeString("foobar"),
+                            Automata.makeString("barbaz")));
+    a = Operations.determinize(a);
+    assertTrue(Operations.run(a, "foobar"));
+    assertTrue(Operations.run(a, "barbaz"));
+
+    assertMatches(a, "foobar", "barbaz");
+  }
+
+  public void testUnion2() throws Exception {
+    Automaton a = Operations.union(Arrays.asList(
+                            Automata.makeString("foobar"),
+                            Automata.makeString(""),
+                            Automata.makeString("barbaz")));
+    a = Operations.determinize(a);
+    assertTrue(Operations.run(a, "foobar"));
+    assertTrue(Operations.run(a, "barbaz"));
+    assertTrue(Operations.run(a, ""));
+
+    assertMatches(a, "", "foobar", "barbaz");
+  }
+
+  public void testMinimizeSimple() throws Exception {
+    Automaton a = Automata.makeString("foobar");
+    Automaton aMin = MinimizationOperations.minimize(a);
+
+    assertTrue(Operations.sameLanguage(a, aMin));
+  }
+
+  public void testMinimize2() throws Exception {
+    Automaton a = Operations.union(Arrays.asList(Automata.makeString("foobar"),
+                                                           Automata.makeString("boobar")));
+    Automaton aMin = MinimizationOperations.minimize(a);
+    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(a)), aMin));
+  }
+
+  public void testReverse() throws Exception {
+    Automaton a = Automata.makeString("foobar");
+    Automaton ra = Operations.reverse(a);
+    Automaton a2 = Operations.determinize(Operations.reverse(ra));
+    
+    assertTrue(Operations.sameLanguage(a, a2));
+  }
+
+  public void testOptional() throws Exception {
+    Automaton a = Automata.makeString("foobar");
+    Automaton a2 = Operations.optional(a);
+    a2 = Operations.determinize(a2);
+    
+    assertTrue(Operations.run(a, "foobar"));
+    assertFalse(Operations.run(a, ""));
+    assertTrue(Operations.run(a2, "foobar"));
+    assertTrue(Operations.run(a2, ""));
+  }
+
+  public void testRepeatAny() throws Exception {
+    Automaton a = Automata.makeString("zee");
+    Automaton a2 = Operations.determinize(Operations.repeat(a));
+    assertTrue(Operations.run(a2, ""));
+    assertTrue(Operations.run(a2, "zee"));    
+    assertTrue(Operations.run(a2, "zeezee"));
+    assertTrue(Operations.run(a2, "zeezeezee"));
+  }
+
+  public void testRepeatMin() throws Exception {
+    Automaton a = Automata.makeString("zee");
+    Automaton a2 = Operations.determinize(Operations.repeat(a, 2));
+    assertFalse(Operations.run(a2, ""));
+    assertFalse(Operations.run(a2, "zee"));    
+    assertTrue(Operations.run(a2, "zeezee"));
+    assertTrue(Operations.run(a2, "zeezeezee"));
+  }
+
+  public void testRepeatMinMax1() throws Exception {
+    Automaton a = Automata.makeString("zee");
+    Automaton a2 = Operations.determinize(Operations.repeat(a, 0, 2));
+    assertTrue(Operations.run(a2, ""));
+    assertTrue(Operations.run(a2, "zee"));    
+    assertTrue(Operations.run(a2, "zeezee"));
+    assertFalse(Operations.run(a2, "zeezeezee"));
+  }
+
+  public void testRepeatMinMax2() throws Exception {
+    Automaton a = Automata.makeString("zee");
+    Automaton a2 = Operations.determinize(Operations.repeat(a, 2, 4));
+    assertFalse(Operations.run(a2, ""));
+    assertFalse(Operations.run(a2, "zee"));    
+    assertTrue(Operations.run(a2, "zeezee"));
+    assertTrue(Operations.run(a2, "zeezeezee"));
+    assertTrue(Operations.run(a2, "zeezeezeezee"));
+    assertFalse(Operations.run(a2, "zeezeezeezeezee"));
+  }
+
+  public void testComplement() throws Exception {
+    Automaton a = Automata.makeString("zee");
+    Automaton a2 = Operations.determinize(Operations.complement(a));
+    assertTrue(Operations.run(a2, ""));
+    assertFalse(Operations.run(a2, "zee"));    
+    assertTrue(Operations.run(a2, "zeezee"));
+    assertTrue(Operations.run(a2, "zeezeezee"));
+  }
+
+  public void testInterval() throws Exception {
+    Automaton a = Operations.determinize(Automata.makeInterval(17, 100, 3));
+    assertFalse(Operations.run(a, ""));
+    assertTrue(Operations.run(a, "017"));
+    assertTrue(Operations.run(a, "100"));
+    assertTrue(Operations.run(a, "073"));
+  }
+
+  public void testCommonSuffix() throws Exception {
+    Automaton a = new Automaton();
+    int init = a.createState();
+    int fini = a.createState();
+    a.setAccept(init, true);
+    a.setAccept(fini, true);
+    a.addTransition(init, fini, 'm');
+    a.addTransition(fini, fini, 'm');
+    a.finishState();
+    assertEquals(0, Operations.getCommonSuffixBytesRef(a).length);
+  }
+
+  public void testReverseRandom1() throws Exception {
+    int ITERS = atLeast(100);
+    for(int i=0;i<ITERS;i++) {
+      Automaton a = AutomatonTestUtil.randomAutomaton(random());
+      Automaton ra = Operations.reverse(a);
+      Automaton rra = Operations.reverse(ra);
+      assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(a)),
+                                              Operations.determinize(Operations.removeDeadStates(rra))));
+    }
+  }
+
+  public void testReverseRandom2() throws Exception {
+    int ITERS = atLeast(100);
+    for(int iter=0;iter<ITERS;iter++) {
+      //System.out.println("TEST: iter=" + iter);
+      Automaton a = AutomatonTestUtil.randomAutomaton(random());
+      if (random().nextBoolean()) {
+        a = Operations.removeDeadStates(a);
+      }
+      Automaton ra = Operations.reverse(a);
+      Automaton rda = Operations.determinize(ra);
+
+      if (Operations.isEmpty(a)) {
+        assertTrue(Operations.isEmpty(rda));
+        continue;
+      }
+
+      RandomAcceptedStrings ras = new RandomAcceptedStrings(a);
+
+      for(int iter2=0;iter2<20;iter2++) {
+        // Find string accepted by original automaton
+        int[] s = ras.getRandomAcceptedString(random());
+
+        // Reverse it
+        for(int j=0;j<s.length/2;j++) {
+          int x = s[j];
+          s[j] = s[s.length-j-1];
+          s[s.length-j-1] = x;
+        }
+        //System.out.println("TEST:   iter2=" + iter2 + " s=" + Arrays.toString(s));
+
+        // Make sure reversed automaton accepts it
+        assertTrue(Operations.run(rda, new IntsRef(s, 0, s.length)));
+      }
+    }
+  }
+
+  public void testAnyStringEmptyString() throws Exception {
+    Automaton a = Operations.determinize(Automata.makeAnyString());
+    assertTrue(Operations.run(a, ""));
+  }
+
+  public void testBasicIsEmpty() throws Exception {
+    Automaton a = new Automaton();
+    a.createState();
+    assertTrue(Operations.isEmpty(a));
+  }
+
+  public void testRemoveDeadTransitionsEmpty() throws Exception {
+    Automaton a = Automata.makeEmpty();
+    Automaton a2 = Operations.removeDeadStates(a);
+    assertTrue(Operations.isEmpty(a2));
+  }
+
+  public void testInvalidAddTransition() throws Exception {
+    Automaton a = new Automaton();
+    int s1 = a.createState();
+    int s2 = a.createState();
+    a.addTransition(s1, s2, 'a');
+    a.addTransition(s2, s2, 'a');
+    try {
+      a.addTransition(s1, s2, 'b');
+      fail("didn't hit expected exception");
+    } catch (IllegalStateException ise) {
+      // expected
+    }
+  }
+
+  public void testBuilderRandom() throws Exception {
+    int ITERS = atLeast(100);
+    for(int iter=0;iter<ITERS;iter++) {
+      Automaton a = AutomatonTestUtil.randomAutomaton(random());
+
+      // Just get all transitions, shuffle, and build a new automaton with the same transitions:
+      List<Transition> allTrans = new ArrayList<>();
+      int numStates = a.getNumStates();
+      for(int s=0;s<numStates;s++) {
+        int count = a.getNumTransitions(s);
+        for(int i=0;i<count;i++) {
+          Transition t = new Transition();
+          a.getTransition(s, i, t);
+          allTrans.add(t);
+        }
+      }
+
+      Automaton.Builder builder = new Automaton.Builder();
+      for(int i=0;i<numStates;i++) {
+        int s = builder.createState();
+        builder.setAccept(s, a.isAccept(s));
+      }
+
+      Collections.shuffle(allTrans, random());
+      for(Transition t : allTrans) {
+        builder.addTransition(t.source, t.dest, t.min, t.max);
+      }
+
+      assertTrue(Operations.sameLanguage(
+                    Operations.determinize(Operations.removeDeadStates(a)),
+                    Operations.determinize(Operations.removeDeadStates(builder.finish()))));
+      
+    }
+  }
+
+  public void testIsTotal() throws Exception {
+    assertFalse(Operations.isTotal(new Automaton()));
+    Automaton a = new Automaton();
+    int init = a.createState();
+    int fini = a.createState();
+    a.setAccept(fini, true);
+    a.addTransition(init, fini, Character.MIN_CODE_POINT, Character.MAX_CODE_POINT);
+    a.finishState();
+    assertFalse(Operations.isTotal(a));
+    a.addTransition(fini, fini, Character.MIN_CODE_POINT, Character.MAX_CODE_POINT);
+    a.finishState();
+    assertFalse(Operations.isTotal(a));
+    a.setAccept(init, true);
+    assertTrue(Operations.isTotal(MinimizationOperations.minimize(a)));
+  }
+
+  public void testMinimizeEmpty() throws Exception {
+    Automaton a = new Automaton();
+    int init = a.createState();
+    int fini = a.createState();
+    a.addTransition(init, fini, 'a');
+    a.finishState();
+    a = MinimizationOperations.minimize(a);
+    assertEquals(0, a.getNumStates());
+  }
+
+  public void testMinus() throws Exception {
+    Automaton a1 = Automata.makeString("foobar");
+    Automaton a2 = Automata.makeString("boobar");
+    Automaton a3 = Automata.makeString("beebar");
+    Automaton a = Operations.union(Arrays.asList(a1, a2, a3));
+    if (random().nextBoolean()) {
+      a = Operations.determinize(a);
+    } else if (random().nextBoolean()) {
+      a = MinimizationOperations.minimize(a);
+    }
+    assertMatches(a, "foobar", "beebar", "boobar");
+
+    Automaton a4 = Operations.determinize(Operations.minus(a, a2));
+    
+    assertTrue(Operations.run(a4, "foobar"));
+    assertFalse(Operations.run(a4, "boobar"));
+    assertTrue(Operations.run(a4, "beebar"));
+    assertMatches(a4, "foobar", "beebar");
+
+    a4 = Operations.determinize(Operations.minus(a4, a1));
+    assertFalse(Operations.run(a4, "foobar"));
+    assertFalse(Operations.run(a4, "boobar"));
+    assertTrue(Operations.run(a4, "beebar"));
+    assertMatches(a4, "beebar");
+
+    a4 = Operations.determinize(Operations.minus(a4, a3));
+    assertFalse(Operations.run(a4, "foobar"));
+    assertFalse(Operations.run(a4, "boobar"));
+    assertFalse(Operations.run(a4, "beebar"));
+    assertMatches(a4);
+  }
+
+  public void testOneInterval() throws Exception {
+    Automaton a = Automata.makeInterval(999, 1032, 0);
+    a = Operations.determinize(a);
+    assertTrue(Operations.run(a, "0999"));
+    assertTrue(Operations.run(a, "00999"));
+    assertTrue(Operations.run(a, "000999"));
+  }
+
+  public void testAnotherInterval() throws Exception {
+    Automaton a = Automata.makeInterval(1, 2, 0);
+    a = Operations.determinize(a);
+    assertTrue(Operations.run(a, "01"));
+  }
+
+  public void testIntervalRandom() throws Exception {
+    int ITERS = atLeast(100);
+    for(int iter=0;iter<ITERS;iter++) {
+      int min = TestUtil.nextInt(random(), 0, 100000);
+      int max = TestUtil.nextInt(random(), min, min+100000);
+      int digits;
+      if (random().nextBoolean()) {
+        digits = 0;
+      } else {
+        String s = Integer.toString(max);
+        digits = TestUtil.nextInt(random(), s.length(), 2*s.length());
+      }
+      StringBuilder b = new StringBuilder();
+      for(int i=0;i<digits;i++) {
+        b.append('0');
+      }
+      String prefix = b.toString();
+
+      Automaton a = Operations.determinize(Automata.makeInterval(min, max, digits));
+      if (random().nextBoolean()) {
+        a = MinimizationOperations.minimize(a);
+      }
+      String mins = Integer.toString(min);
+      String maxs = Integer.toString(max);
+      if (digits > 0) {
+        mins = prefix.substring(mins.length()) + mins;
+        maxs = prefix.substring(maxs.length()) + maxs;
+      }
+      assertTrue(Operations.run(a, mins));
+      assertTrue(Operations.run(a, maxs));
+
+      for(int iter2=0;iter2<100;iter2++) {
+        int x = random().nextInt(2*max);
+        boolean expected = x >= min && x <= max;
+        String sx = Integer.toString(x);
+        if (sx.length() < digits) {
+          // Left-fill with 0s
+          sx = b.substring(sx.length()) + sx;
+        } else if (digits == 0) {
+          // Left-fill with random number of 0s:
+          int numZeros = random().nextInt(10);
+          StringBuilder sb = new StringBuilder();
+          for(int i=0;i<numZeros;i++) {
+            sb.append('0');
+          }
+          sb.append(sx);
+          sx = sb.toString();
+        }
+        assertEquals(expected, Operations.run(a, sx));
+      }
+    }
+  }
+
+  private void assertMatches(Automaton a, String... strings) {
+    Set<IntsRef> expected = new HashSet<>();
+    for(String s : strings) {
+      IntsRef ints = new IntsRef();
+      expected.add(Util.toUTF32(s, ints));
+    }
+
+    assertEquals(expected, Operations.getFiniteStrings(Operations.determinize(a), -1)); 
+  }
+
+  public void testConcatenatePreservesDet() throws Exception {
+    Automaton a1 = Automata.makeString("foobar");
+    assertTrue(a1.isDeterministic());
+    Automaton a2 = Automata.makeString("baz");
+    assertTrue(a2.isDeterministic());
+    assertTrue((Operations.concatenate(Arrays.asList(a1, a2)).isDeterministic()));
+  }
+
+  public void testRemoveDeadStates() throws Exception {
+    Automaton a = Operations.concatenate(Arrays.asList(Automata.makeString("x"),
+                                                                      Automata.makeString("y")));
+    assertEquals(4, a.getNumStates());
+    a = Operations.removeDeadStates(a);
+    assertEquals(3, a.getNumStates());
+  }
+
+  public void testRemoveDeadStatesEmpty1() throws Exception {
+    Automaton a = new Automaton();
+    a.finishState();
+    assertTrue(Operations.isEmpty(a));
+    assertTrue(Operations.isEmpty(Operations.removeDeadStates(a)));
+  }
+
+  public void testRemoveDeadStatesEmpty2() throws Exception {
+    Automaton a = new Automaton();
+    a.finishState();
+    assertTrue(Operations.isEmpty(a));
+    assertTrue(Operations.isEmpty(Operations.removeDeadStates(a)));
+  }
+
+  public void testRemoveDeadStatesEmpty3() throws Exception {
+    Automaton a = new Automaton();
+    int init = a.createState();
+    int fini = a.createState();
+    a.addTransition(init, fini, 'a');
+    Automaton a2 = Operations.removeDeadStates(a);
+    assertEquals(0, a2.getNumStates());
+  }
+
+  public void testConcatEmpty() throws Exception {
+    // 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));
+
+    a = Operations.concatenate(Automata.makeString("foo"),
+                                         Automata.makeEmpty());
+    assertEquals(new HashSet<IntsRef>(), Operations.getFiniteStrings(a, -1));
+  }
+
+  public void testSeemsNonEmptyButIsNot1() throws Exception {
+    Automaton a = new Automaton();
+    // Init state has a transition but doesn't lead to accept
+    int init = a.createState();
+    int s = a.createState();
+    a.addTransition(init, s, 'a');
+    a.finishState();
+    assertTrue(Operations.isEmpty(a));
+  }
+
+  public void testSeemsNonEmptyButIsNot2() throws Exception {
+    Automaton a = new Automaton();
+    int init = a.createState();
+    int s = a.createState();
+    a.addTransition(init, s, 'a');
+    // An orphan'd accept state
+    s = a.createState();
+    a.setAccept(s, true);
+    a.finishState();
+    assertTrue(Operations.isEmpty(a));
+  }
+
+  public void testSameLanguage1() throws Exception {
+    Automaton a = Automata.makeEmptyString();
+    Automaton a2 = Automata.makeEmptyString();
+    int state = a2.createState();
+    a2.addTransition(0, state, 'a');
+    a2.finishState();
+    assertTrue(Operations.sameLanguage(Operations.removeDeadStates(a),
+                                            Operations.removeDeadStates(a2)));
+  }
+
+  private Automaton randomNoOp(Automaton a) {
+    switch (random().nextInt(7)) {
+    case 0:
+      if (VERBOSE) {
+        System.out.println("  randomNoOp: determinize");
+      }
+      return Operations.determinize(a);
+    case 1:
+      if (VERBOSE) {
+        System.out.println("  randomNoOp: minimize");
+      }
+      return MinimizationOperations.minimize(a);
+    case 2:
+      if (VERBOSE) {
+        System.out.println("  randomNoOp: removeDeadStates");
+      }
+      return Operations.removeDeadStates(a);
+    case 3:
+      if (VERBOSE) {
+        System.out.println("  randomNoOp: reverse reverse");
+      }
+      a = Operations.reverse(a);
+      a = randomNoOp(a);
+      return Operations.reverse(a);
+    case 4:
+      if (VERBOSE) {
+        System.out.println("  randomNoOp: concat empty string");
+      }
+      return Operations.concatenate(a, Automata.makeEmptyString());
+    case 5:
+      if (VERBOSE) {
+        System.out.println("  randomNoOp: union empty automaton");
+      }
+      return Operations.union(a, Automata.makeEmpty());
+    case 6:
+      if (VERBOSE) {
+        System.out.println("  randomNoOp: do nothing!");
+      }
+      return a;
+    }
+    assert false;
+    return null;
+  }
+
+  private Automaton unionTerms(Collection<BytesRef> terms) {
+    Automaton a;
+    if (random().nextBoolean()) {
+      if (VERBOSE) {
+        System.out.println("TEST: unionTerms: use union");
+      }
+      List<Automaton> as = new ArrayList<>();
+      for(BytesRef term : terms) {
+        as.add(Automata.makeString(term.utf8ToString()));
+      }
+      a = Operations.union(as);
+    } else {
+      if (VERBOSE) {
+        System.out.println("TEST: unionTerms: use makeStringUnion");
+      }
+      List<BytesRef> termsList = new ArrayList<>(terms);
+      Collections.sort(termsList);
+      a = Automata.makeStringUnion(termsList);
+    }
+
+    return randomNoOp(a);
+  }
+
+  private String getRandomString() {
+    //return TestUtil.randomSimpleString(random());
+    return TestUtil.randomRealisticUnicodeString(random());
+  }
+
+  public void testRandomFinite() throws Exception {
+
+    int numTerms = atLeast(10);
+    int iters = atLeast(100);
+
+    if (VERBOSE) {
+      System.out.println("TEST: numTerms" + numTerms + " iters=" + iters);
+    }
+
+    Set<BytesRef> terms = new HashSet<>();
+    while (terms.size() < numTerms) {
+      terms.add(new BytesRef(getRandomString()));
+    }
+
+    Automaton a = unionTerms(terms);
+    assertSame(terms, a);
+
+    for(int iter=0;iter<iters;iter++) {
+      if (VERBOSE) {
+        System.out.println("TEST: iter=" + iter + " numTerms=" + terms.size());
+        System.out.println("  terms:");
+        for(BytesRef term : terms) {
+          System.out.println("    " + term);
+        }
+      }
+      switch(random().nextInt(15)) {
+
+      case 0:
+        // concatenate prefix
+        {
+          if (VERBOSE) {
+            System.out.println("  op=concat prefix");
+          }
+          Set<BytesRef> newTerms = new HashSet<>();
+          BytesRef prefix = new BytesRef(getRandomString());
+          for(BytesRef term : terms) {
+            BytesRef newTerm = BytesRef.deepCopyOf(prefix);
+            newTerm.append(term);
+            newTerms.add(newTerm);
+          }
+          terms = newTerms;
+          boolean wasDeterministic1 = a.isDeterministic();
+          a = Operations.concatenate(Automata.makeString(prefix.utf8ToString()), a);
+          assertEquals(wasDeterministic1, a.isDeterministic());
+        }
+        break;
+
+      case 1:
+        // concatenate suffix
+        {
+          BytesRef suffix = new BytesRef(getRandomString());
+          if (VERBOSE) {
+            System.out.println("  op=concat suffix " + suffix);
+          }
+          Set<BytesRef> newTerms = new HashSet<>();
+          for(BytesRef term : terms) {
+            BytesRef newTerm = BytesRef.deepCopyOf(term);
+            newTerm.append(suffix);
+            newTerms.add(newTerm);
+          }
+          terms = newTerms;
+          a = Operations.concatenate(a, Automata.makeString(suffix.utf8ToString()));
+        }
+        break;
+
+      case 2:
+        // determinize
+        if (VERBOSE) {
+          System.out.println("  op=determinize");
+        }
+        a = Operations.determinize(a);
+        assertTrue(a.isDeterministic());
+        break;
+
+      case 3:
+        if (VERBOSE) {
+          System.out.println("  op=minimize");
+        }
+        // minimize
+        a = MinimizationOperations.minimize(a);
+        break;
+
+      case 4:
+        // union
+        {
+          if (VERBOSE) {
+            System.out.println("  op=union");
+          }
+          Set<BytesRef> newTerms = new HashSet<>();
+          int numNewTerms = random().nextInt(5);
+          while (newTerms.size() < numNewTerms) {
+            newTerms.add(new BytesRef(getRandomString()));
+          }
+          terms.addAll(newTerms);
+          Automaton newA = unionTerms(newTerms);
+          a = Operations.union(a, newA);
+        }
+        break;
+
+      case 5:
+        // optional
+        {
+          if (VERBOSE) {
+            System.out.println("  op=optional");
+          }
+          a = Operations.optional(a);
+          terms.add(new BytesRef());
+        }
+        break;
+
+      case 6:
+        // minus finite 
+        {
+          if (VERBOSE) {
+            System.out.println("  op=minus finite");
+          }
+          if (terms.size() > 0) {
+            RandomAcceptedStrings rasl = new RandomAcceptedStrings(Operations.removeDeadStates(a));
+            Set<BytesRef> toRemove = new HashSet<>();
+            int numToRemove = TestUtil.nextInt(random(), 1, (terms.size()+1)/2);
+            while (toRemove.size() < numToRemove) {
+              int[] ints = rasl.getRandomAcceptedString(random());
+              BytesRef term = new BytesRef(UnicodeUtil.newString(ints, 0, ints.length));
+              if (toRemove.contains(term) == false) {
+                toRemove.add(term);
+              }
+            }
+            for(BytesRef term : toRemove) {
+              boolean removed = terms.remove(term);
+              assertTrue(removed);
+            }
+            Automaton a2 = unionTerms(toRemove);
+            a = Operations.minus(a, a2);
+          }
+        }
+        break;
+
+      case 7:
+        {
+          // minus infinite
+          List<Automaton> as = new ArrayList<>();
+          int count = TestUtil.nextInt(random(), 1, 5);
+          Set<Integer> prefixes = new HashSet<>();
+          while(prefixes.size() < count) {
+            // prefix is a leading ascii byte; we remove <prefix>* from a
+            int prefix = random().nextInt(128);
+            prefixes.add(prefix);
+          }
+
+          if (VERBOSE) {
+            System.out.println("  op=minus infinite prefixes=" + prefixes);
+          }
+
+          for(int prefix : prefixes) {
+            // prefix is a leading ascii byte; we remove <prefix>* from a
+            Automaton a2 = new Automaton();
+            int init = a2.createState();
+            int state = a2.createState();
+            a2.addTransition(init, state, prefix);
+            a2.setAccept(state, true);
+            a2.addTransition(state, state, Character.MIN_CODE_POINT, Character.MAX_CODE_POINT);
+            a2.finishState();
+            as.add(a2);
+            Iterator<BytesRef> it = terms.iterator();
+            while (it.hasNext()) {
+              BytesRef term = it.next();
+              if (term.length > 0 && (term.bytes[term.offset] & 0xFF) == prefix) {
+                it.remove();
+              }
+            }
+          }
+          Automaton a2 = randomNoOp(Operations.union(as));
+          a = Operations.minus(a, a2);
+        }
+        break;
+
+      case 8:
+        {
+          int count = TestUtil.nextInt(random(), 10, 20);
+          if (VERBOSE) {
+            System.out.println("  op=intersect infinite count=" + count);
+          }
+          // intersect infinite
+          List<Automaton> as = new ArrayList<>();
+
+          Set<Integer> prefixes = new HashSet<>();
+          while(prefixes.size() < count) {
+            int prefix = random().nextInt(128);
+            prefixes.add(prefix);
+          }
+          if (VERBOSE) {
+            System.out.println("  prefixes=" + prefixes);
+          }
+
+          for(int prefix : prefixes) {
+            // prefix is a leading ascii byte; we retain <prefix>* in a
+            Automaton a2 = new Automaton();
+            int init = a2.createState();
+            int state = a2.createState();
+            a2.addTransition(init, state, prefix);
+            a2.setAccept(state, true);
+            a2.addTransition(state, state, Character.MIN_CODE_POINT, Character.MAX_CODE_POINT);
+            a2.finishState();
+            as.add(a2);
+            prefixes.add(prefix);
+          }
+
+          Automaton a2 = Operations.union(as);
+          if (random().nextBoolean()) {
+            a2 = Operations.determinize(a2);
+          } else if (random().nextBoolean()) {
+            a2 = MinimizationOperations.minimize(a2);
+          }
+          a = Operations.intersection(a, a2);
+
+          Iterator<BytesRef> it = terms.iterator();
+          while (it.hasNext()) {
+            BytesRef term = it.next();
+            if (term.length == 0 || prefixes.contains(term.bytes[term.offset]&0xff) == false) {
+              if (VERBOSE) {
+                System.out.println("  drop term=" + term);
+              }
+              it.remove();
+            } else {
+              if (VERBOSE) {
+                System.out.println("  keep term=" + term);
+              }
+            }
+          }
+        }        
+        break;
+
+      case 9:
+        // reverse
+        {
+          if (VERBOSE) {
+            System.out.println("  op=reverse");
+          }
+          a = Operations.reverse(a);
+          Set<BytesRef> newTerms = new HashSet<>();
+          for(BytesRef term : terms) {
+            newTerms.add(new BytesRef(new StringBuilder(term.utf8ToString()).reverse().toString()));
+          }
+          terms = newTerms;
+        }
+        break;
+
+      case 10:
+        if (VERBOSE) {
+          System.out.println("  op=randomNoOp");
+        }
+        a = randomNoOp(a);
+        break;
+
+      case 11:
+        // interval
+        {
+          int min = random().nextInt(1000);
+          int max = min + random().nextInt(50);
+          // digits must be non-zero else we make cycle
+          int digits = Integer.toString(max).length();
+          if (VERBOSE) {
+            System.out.println("  op=union interval min=" + min + " max=" + max + " digits=" + digits);
+          }
+          a = Operations.union(a, Automata.makeInterval(min, max, digits));
+          StringBuilder b = new StringBuilder();
+          for(int i=0;i<digits;i++) {
+            b.append('0');
+          }
+          String prefix = b.toString();
+          for(int i=min;i<=max;i++) {
+            String s = Integer.toString(i);
+            if (s.length() < digits) {
+              // Left-fill with 0s
+              s = prefix.substring(s.length()) + s;
+            }
+            terms.add(new BytesRef(s));
+          }
+        }
+        break;
+
+      case 12:
+        if (VERBOSE) {
+          System.out.println("  op=remove the empty string");
+        }
+        a = Operations.minus(a, Automata.makeEmptyString());
+        terms.remove(new BytesRef());
+        break;
+
+      case 13:
+        if (VERBOSE) {
+          System.out.println("  op=add the empty string");
+        }
+        a = Operations.union(a, Automata.makeEmptyString());
+        terms.add(new BytesRef());
+        break;
+
+      case 14:
+        // Safety in case we are really unlucky w/ the dice:
+        if (terms.size() <= numTerms * 10) {
+          if (VERBOSE) {
+            System.out.println("  op=concat finite automaton");
+          }
+          int count = random().nextBoolean() ? 2 : 3;
+          Set<BytesRef> addTerms = new HashSet<>();
+          while (addTerms.size() < count) {
+            addTerms.add(new BytesRef(getRandomString()));
+          }
+          if (VERBOSE) {
+            for(BytesRef term : addTerms) {
+              System.out.println("    term=" + term);
+            }
+          }
+          Automaton a2 = unionTerms(addTerms);
+          Set<BytesRef> newTerms = new HashSet<>();
+          if (random().nextBoolean()) {
+            // suffix
+            if (VERBOSE) {
+              System.out.println("  do suffix");
+            }
+            a = Operations.concatenate(a, randomNoOp(a2));
+            for(BytesRef term : terms) {
+              for(BytesRef suffix : addTerms) {
+                BytesRef newTerm = BytesRef.deepCopyOf(term);
+                newTerm.append(suffix);
+                newTerms.add(newTerm);
+              }
+            }
+          } else {
+            // prefix
+            if (VERBOSE) {
+              System.out.println("  do prefix");
+            }
+            a = Operations.concatenate(randomNoOp(a2), a);
+            for(BytesRef term : terms) {
+              for(BytesRef prefix : addTerms) {
+                BytesRef newTerm = BytesRef.deepCopyOf(prefix);
+                newTerm.append(term);
+                newTerms.add(newTerm);
+              }
+            }
+          }
+
+          terms = newTerms;
+        }
+        break;
+      }
+
+      // assertSame(terms, a);
+      assertEquals(AutomatonTestUtil.isDeterministicSlow(a), a.isDeterministic());
+    }
+
+    assertSame(terms, a);
+  }
+
+  private void assertSame(Collection<BytesRef> terms, Automaton a) {
+
+    try {
+      assertTrue(Operations.isFinite(a));
+      assertFalse(Operations.isTotal(a));
+
+      Automaton detA = Operations.determinize(a);
+
+      // Make sure all terms are accepted:
+      IntsRef scratch = new IntsRef();
+      for(BytesRef term : terms) {
+        Util.toIntsRef(term, scratch);
+        assertTrue("failed to accept term=" + term.utf8ToString(), Operations.run(detA, term.utf8ToString()));
+      }
+
+      // Use getFiniteStrings:
+      Set<IntsRef> expected = new HashSet<>();
+      for(BytesRef term : terms) {
+        IntsRef intsRef = new IntsRef();
+        Util.toUTF32(term.utf8ToString(), intsRef);
+        expected.add(intsRef);
+      }
+      Set<IntsRef> actual = Operations.getFiniteStrings(a, -1);
+
+      if (expected.equals(actual) == false) {
+        System.out.println("FAILED:");
+        for(IntsRef term : expected) {
+          if (actual.contains(term) == false) {
+            System.out.println("  term=" + term + " should be accepted but isn't");
+          }
+        }
+        for(IntsRef term : actual) {
+          if (expected.contains(term) == false) {
+            System.out.println("  term=" + term + " is accepted but should not be");
+          }
+        }
+        throw new AssertionError("mismatch");
+      }
+
+      // Use sameLanguage:
+      Automaton a2 = Operations.removeDeadStates(Operations.determinize(unionTerms(terms)));
+      assertTrue(Operations.sameLanguage(a2, Operations.removeDeadStates(Operations.determinize(a))));
+
+      // Do same check, in UTF8 space
+      Automaton utf8 = randomNoOp(new UTF32ToUTF8().convert(a));
+    
+      Set<IntsRef> expected2 = new HashSet<>();
+      for(BytesRef term : terms) {
+        IntsRef intsRef = new IntsRef();
+        Util.toIntsRef(term, intsRef);
+        expected2.add(intsRef);
+      }
+      assertEquals(expected2, Operations.getFiniteStrings(utf8, -1));
+    } catch (AssertionError ae) {
+      System.out.println("TEST: FAILED: not same");
+      System.out.println("  terms (count=" + terms.size() + "):");
+      for(BytesRef term : terms) {
+        System.out.println("    " + term);
+      }
+      System.out.println("  automaton:");
+      System.out.println(a.toDot());
+      //a.writeDot("fail");
+      throw ae;
+    }
+  }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/util/automaton/TestBasicOperations.java b/lucene/core/src/test/org/apache/lucene/util/automaton/TestBasicOperations.java
deleted file mode 100644
index 1b2e62d..0000000
--- a/lucene/core/src/test/org/apache/lucene/util/automaton/TestBasicOperations.java
+++ /dev/null
@@ -1,146 +0,0 @@
-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.*;
-
-import org.apache.lucene.util.*;
-
-import com.carrotsearch.randomizedtesting.generators.RandomInts;
-
-public class TestBasicOperations extends LuceneTestCase {
-  /** Test string union. */
-  public void testStringUnion() {
-    List<BytesRef> strings = new ArrayList<>();
-    for (int i = RandomInts.randomIntBetween(random(), 0, 1000); --i >= 0;) {
-      strings.add(new BytesRef(TestUtil.randomUnicodeString(random())));
-    }
-
-    Collections.sort(strings);
-    Automaton union = BasicAutomata.makeStringUnion(strings);
-    assertTrue(union.isDeterministic());
-    assertTrue(BasicOperations.sameLanguage(union, naiveUnion(strings)));
-  }
-
-  private static Automaton naiveUnion(List<BytesRef> strings) {
-    Automaton [] eachIndividual = new Automaton [strings.size()];
-    int i = 0;
-    for (BytesRef bref : strings) {
-      eachIndividual[i++] = BasicAutomata.makeString(bref.utf8ToString());
-    }
-    return BasicOperations.union(Arrays.asList(eachIndividual));
-  }
-
-  /** Test optimization to concatenate() */
-  public void testSingletonConcatenate() {
-    Automaton singleton = BasicAutomata.makeString("prefix");
-    Automaton expandedSingleton = singleton.cloneExpanded();
-    Automaton other = BasicAutomata.makeCharRange('5', '7');
-    Automaton concat = BasicOperations.concatenate(singleton, other);
-    assertTrue(concat.isDeterministic());
-    assertTrue(BasicOperations.sameLanguage(BasicOperations.concatenate(expandedSingleton, other), concat));
-  }
-  
-  /** Test optimization to concatenate() to an NFA */
-  public void testSingletonNFAConcatenate() {
-    Automaton singleton = BasicAutomata.makeString("prefix");
-    Automaton expandedSingleton = singleton.cloneExpanded();
-    // an NFA (two transitions for 't' from initial state)
-    Automaton nfa = BasicOperations.union(BasicAutomata.makeString("this"),
-        BasicAutomata.makeString("three"));
-    Automaton concat = BasicOperations.concatenate(singleton, nfa);
-    assertFalse(concat.isDeterministic());
-    assertTrue(BasicOperations.sameLanguage(BasicOperations.concatenate(expandedSingleton, nfa), concat));
-  }
-  
-  /** Test optimization to concatenate() with empty String */
-  public void testEmptySingletonConcatenate() {
-    Automaton singleton = BasicAutomata.makeString("");
-    Automaton expandedSingleton = singleton.cloneExpanded();
-    Automaton other = BasicAutomata.makeCharRange('5', '7');
-    Automaton concat1 = BasicOperations.concatenate(expandedSingleton, other);
-    Automaton concat2 = BasicOperations.concatenate(singleton, other);
-    assertTrue(concat2.isDeterministic());
-    assertTrue(BasicOperations.sameLanguage(concat1, concat2));
-    assertTrue(BasicOperations.sameLanguage(other, concat1));
-    assertTrue(BasicOperations.sameLanguage(other, concat2));
-  }
-  
-  /** Test concatenation with empty language returns empty */
-  public void testEmptyLanguageConcatenate() {
-    Automaton a = BasicAutomata.makeString("a");
-    Automaton concat = BasicOperations.concatenate(a, BasicAutomata.makeEmpty());
-    assertTrue(BasicOperations.isEmpty(concat));
-  }
-  
-  /** Test optimization to concatenate() with empty String to an NFA */
-  public void testEmptySingletonNFAConcatenate() {
-    Automaton singleton = BasicAutomata.makeString("");
-    Automaton expandedSingleton = singleton.cloneExpanded();
-    // an NFA (two transitions for 't' from initial state)
-    Automaton nfa = BasicOperations.union(BasicAutomata.makeString("this"),
-        BasicAutomata.makeString("three"));
-    Automaton concat1 = BasicOperations.concatenate(expandedSingleton, nfa);
-    Automaton concat2 = BasicOperations.concatenate(singleton, nfa);
-    assertFalse(concat2.isDeterministic());
-    assertTrue(BasicOperations.sameLanguage(concat1, concat2));
-    assertTrue(BasicOperations.sameLanguage(nfa, concat1));
-    assertTrue(BasicOperations.sameLanguage(nfa, concat2));
-  }
-
-  /** Test singletons work correctly */
-  public void testSingleton() {
-    Automaton singleton = BasicAutomata.makeString("foobar");
-    Automaton expandedSingleton = singleton.cloneExpanded();
-    assertTrue(BasicOperations.sameLanguage(singleton, expandedSingleton));
-    
-    singleton = BasicAutomata.makeString("\ud801\udc1c");
-    expandedSingleton = singleton.cloneExpanded();
-    assertTrue(BasicOperations.sameLanguage(singleton, expandedSingleton));
-  }
-
-  public void testGetRandomAcceptedString() throws Throwable {
-    final int ITER1 = atLeast(100);
-    final int ITER2 = atLeast(100);
-    for(int i=0;i<ITER1;i++) {
-
-      final RegExp re = new RegExp(AutomatonTestUtil.randomRegexp(random()), RegExp.NONE);
-      final Automaton a = re.toAutomaton();
-      assertFalse(BasicOperations.isEmpty(a));
-
-      final AutomatonTestUtil.RandomAcceptedStrings rx = new AutomatonTestUtil.RandomAcceptedStrings(a);
-      for(int j=0;j<ITER2;j++) {
-        int[] acc = null;
-        try {
-          acc = rx.getRandomAcceptedString(random());
-          final String s = UnicodeUtil.newString(acc, 0, acc.length);
-          assertTrue(BasicOperations.run(a, s));
-        } catch (Throwable t) {
-          System.out.println("regexp: " + re);
-          if (acc != null) {
-            System.out.println("fail acc re=" + re + " count=" + acc.length);
-            for(int k=0;k<acc.length;k++) {
-              System.out.println("  " + Integer.toHexString(acc[k]));
-            }
-          }
-          throw t;
-        }
-      }
-    }
-  }
-}
diff --git a/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminism.java b/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminism.java
index 2191b37..f8d2d76 100644
--- a/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminism.java
+++ b/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminism.java
@@ -28,8 +28,9 @@
   /** test a bunch of random regular expressions */
   public void testRegexps() throws Exception {
       int num = atLeast(500);
-      for (int i = 0; i < num; i++)
+      for (int i = 0; i < num; i++) {
         assertAutomaton(new RegExp(AutomatonTestUtil.randomRegexp(random()), RegExp.NONE).toAutomaton());
+      }
   }
   
   /** test against a simple, unoptimized det */
@@ -37,42 +38,41 @@
     int num = atLeast(200);
     for (int i = 0; i < num; i++) {
       Automaton a = AutomatonTestUtil.randomAutomaton(random());
-      Automaton b = a.clone();
-      AutomatonTestUtil.determinizeSimple(a);
-      b.deterministic = false; // force det
-      b.determinize();
+      a = AutomatonTestUtil.determinizeSimple(a);
+      Automaton b = Operations.determinize(a);
       // TODO: more verifications possible?
-      assertTrue(BasicOperations.sameLanguage(a, b));
+      assertTrue(Operations.sameLanguage(a, b));
     }
   }
   
   private static void assertAutomaton(Automaton a) {
-    Automaton clone = a.clone();
+    a = Operations.determinize(Operations.removeDeadStates(a));
+
     // complement(complement(a)) = a
-    Automaton equivalent = BasicOperations.complement(BasicOperations.complement(a));
-    assertTrue(BasicOperations.sameLanguage(a, equivalent));
+    Automaton equivalent = Operations.complement(Operations.complement(a));
+    assertTrue(Operations.sameLanguage(a, equivalent));
     
     // a union a = a
-    equivalent = BasicOperations.union(a, clone);
-    assertTrue(BasicOperations.sameLanguage(a, equivalent));
+    equivalent = Operations.determinize(Operations.removeDeadStates(Operations.union(a, a)));
+    assertTrue(Operations.sameLanguage(a, equivalent));
     
     // a intersect a = a
-    equivalent = BasicOperations.intersection(a, clone);
-    assertTrue(BasicOperations.sameLanguage(a, equivalent));
+    equivalent = Operations.determinize(Operations.removeDeadStates(Operations.intersection(a, a)));
+    assertTrue(Operations.sameLanguage(a, equivalent));
     
     // a minus a = empty
-    Automaton empty = BasicOperations.minus(a, clone);
-    assertTrue(BasicOperations.isEmpty(empty));
+    Automaton empty = Operations.minus(a, a);
+    assertTrue(Operations.isEmpty(empty));
     
     // as long as don't accept the empty string
     // then optional(a) - empty = a
-    if (!BasicOperations.run(a, "")) {
+    if (!Operations.run(a, "")) {
       //System.out.println("test " + a);
-      Automaton optional = BasicOperations.optional(a);
+      Automaton optional = Operations.optional(a);
       //System.out.println("optional " + optional);
-      equivalent = BasicOperations.minus(optional, BasicAutomata.makeEmptyString());
+      equivalent = Operations.minus(optional, Automata.makeEmptyString());
       //System.out.println("equiv " + equivalent);
-      assertTrue(BasicOperations.sameLanguage(a, equivalent));
+      assertTrue(Operations.sameLanguage(a, equivalent));
     }
   } 
 }
diff --git a/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java b/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java
index be40c6c..1ae00e4 100644
--- a/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java
+++ b/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java
@@ -41,7 +41,7 @@
       for (int j = 0; j < 5000; j++) {
         String randomString = TestUtil.randomUnicodeString(random());
         terms.add(randomString);
-        automata.add(BasicAutomata.makeString(randomString));
+        automata.add(Automata.makeString(randomString));
       }
       assertLexicon();
     }
@@ -49,11 +49,11 @@
   
   public void assertLexicon() throws Exception {
     Collections.shuffle(automata, random());
-    final Automaton lex = BasicOperations.union(automata);
-    lex.determinize();
-    assertTrue(SpecialOperations.isFinite(lex));
+    Automaton lex = Operations.union(automata);
+    lex = Operations.determinize(lex);
+    assertTrue(Operations.isFinite(lex));
     for (String s : terms) {
-      assertTrue(BasicOperations.run(lex, s));
+      assertTrue(Operations.run(lex, s));
     }
     final ByteRunAutomaton lexByte = new ByteRunAutomaton(lex);
     for (String s : terms) {
diff --git a/lucene/core/src/test/org/apache/lucene/util/automaton/TestLevenshteinAutomata.java b/lucene/core/src/test/org/apache/lucene/util/automaton/TestLevenshteinAutomata.java
index c301493..a4f9a0d 100644
--- a/lucene/core/src/test/org/apache/lucene/util/automaton/TestLevenshteinAutomata.java
+++ b/lucene/core/src/test/org/apache/lucene/util/automaton/TestLevenshteinAutomata.java
@@ -41,7 +41,7 @@
   
   // LUCENE-3094
   public void testNoWastedStates() throws Exception {
-    AutomatonTestUtil.assertNoDetachedStates(new LevenshteinAutomata("abc", false).toAutomaton(1));
+    assertFalse(Operations.hasDeadStatesFromInitial(new LevenshteinAutomata("abc", false).toAutomaton(1)));
   }
   
   /** 
@@ -75,31 +75,36 @@
       assertNotNull(tautomata[n]);
       assertTrue(automata[n].isDeterministic());
       assertTrue(tautomata[n].isDeterministic());
-      assertTrue(SpecialOperations.isFinite(automata[n]));
-      assertTrue(SpecialOperations.isFinite(tautomata[n]));
-      AutomatonTestUtil.assertNoDetachedStates(automata[n]);
-      AutomatonTestUtil.assertNoDetachedStates(tautomata[n]);
+      assertTrue(Operations.isFinite(automata[n]));
+      assertTrue(Operations.isFinite(tautomata[n]));
+      assertFalse(Operations.hasDeadStatesFromInitial(automata[n]));
+      assertFalse(Operations.hasDeadStatesFromInitial(tautomata[n]));
       // check that the dfa for n-1 accepts a subset of the dfa for n
       if (n > 0) {
-        assertTrue(automata[n-1].subsetOf(automata[n]));
-        assertTrue(automata[n-1].subsetOf(tautomata[n]));
-        assertTrue(tautomata[n-1].subsetOf(automata[n]));
-        assertTrue(tautomata[n-1].subsetOf(tautomata[n]));
+        assertTrue(Operations.subsetOf(Operations.removeDeadStates(automata[n-1]),
+                                       Operations.removeDeadStates(automata[n])));
+        assertTrue(Operations.subsetOf(Operations.removeDeadStates(automata[n-1]),
+                                       Operations.removeDeadStates(tautomata[n])));
+        assertTrue(Operations.subsetOf(Operations.removeDeadStates(tautomata[n-1]),
+                                       Operations.removeDeadStates(automata[n])));
+        assertTrue(Operations.subsetOf(Operations.removeDeadStates(tautomata[n-1]),
+                                       Operations.removeDeadStates(tautomata[n])));
         assertNotSame(automata[n-1], automata[n]);
       }
       // check that Lev(N) is a subset of LevT(N)
-      assertTrue(automata[n].subsetOf(tautomata[n]));
+      assertTrue(Operations.subsetOf(Operations.removeDeadStates(automata[n]),
+                                     Operations.removeDeadStates(tautomata[n])));
       // special checks for specific n
       switch(n) {
         case 0:
           // easy, matches the string itself
-          assertTrue(BasicOperations.sameLanguage(BasicAutomata.makeString(s), automata[0]));
-          assertTrue(BasicOperations.sameLanguage(BasicAutomata.makeString(s), tautomata[0]));
+          assertTrue(Operations.sameLanguage(Automata.makeString(s), Operations.removeDeadStates(automata[0])));
+          assertTrue(Operations.sameLanguage(Automata.makeString(s), Operations.removeDeadStates(tautomata[0])));
           break;
         case 1:
           // generate a lev1 naively, and check the accepted lang is the same.
-          assertTrue(BasicOperations.sameLanguage(naiveLev1(s), automata[1]));
-          assertTrue(BasicOperations.sameLanguage(naiveLev1T(s), tautomata[1]));
+          assertTrue(Operations.sameLanguage(naiveLev1(s), Operations.removeDeadStates(automata[1])));
+          assertTrue(Operations.sameLanguage(naiveLev1T(s), Operations.removeDeadStates(tautomata[1])));
           break;
         default:
           assertBruteForce(s, automata[n], n);
@@ -114,13 +119,13 @@
    * substitutions of s.
    */
   private Automaton naiveLev1(String s) {
-    Automaton a = BasicAutomata.makeString(s);
-    a = BasicOperations.union(a, insertionsOf(s));
-    MinimizationOperations.minimize(a);
-    a = BasicOperations.union(a, deletionsOf(s));
-    MinimizationOperations.minimize(a);
-    a = BasicOperations.union(a, substitutionsOf(s));
-    MinimizationOperations.minimize(a);
+    Automaton a = Automata.makeString(s);
+    a = Operations.union(a, insertionsOf(s));
+    a = MinimizationOperations.minimize(a);
+    a = Operations.union(a, deletionsOf(s));
+    a = MinimizationOperations.minimize(a);
+    a = Operations.union(a, substitutionsOf(s));
+    a = MinimizationOperations.minimize(a);
     
     return a;
   }
@@ -131,8 +136,8 @@
    */
   private Automaton naiveLev1T(String s) {
     Automaton a = naiveLev1(s);
-    a = BasicOperations.union(a, transpositionsOf(s));
-    MinimizationOperations.minimize(a);
+    a = Operations.union(a, transpositionsOf(s));
+    a = MinimizationOperations.minimize(a);
     return a;
   }
   
@@ -144,15 +149,14 @@
     List<Automaton> list = new ArrayList<>();
     
     for (int i = 0; i <= s.length(); i++) {
-      Automaton a = BasicAutomata.makeString(s.substring(0, i));
-      a = BasicOperations.concatenate(a, BasicAutomata.makeAnyChar());
-      a = BasicOperations.concatenate(a, BasicAutomata.makeString(s
-          .substring(i)));
+      Automaton a = Automata.makeString(s.substring(0, i));
+      a = Operations.concatenate(a, Automata.makeAnyChar());
+      a = Operations.concatenate(a, Automata.makeString(s.substring(i)));
       list.add(a);
     }
     
-    Automaton a = BasicOperations.union(list);
-    MinimizationOperations.minimize(a);
+    Automaton a = Operations.union(list);
+    a = MinimizationOperations.minimize(a);
     return a;
   }
   
@@ -164,15 +168,13 @@
     List<Automaton> list = new ArrayList<>();
     
     for (int i = 0; i < s.length(); i++) {
-      Automaton a = BasicAutomata.makeString(s.substring(0, i));
-      a = BasicOperations.concatenate(a, BasicAutomata.makeString(s
-          .substring(i + 1)));
-      a.expandSingleton();
+      Automaton a = Automata.makeString(s.substring(0, i));
+      a = Operations.concatenate(a, Automata.makeString(s.substring(i + 1)));
       list.add(a);
     }
     
-    Automaton a = BasicOperations.union(list);
-    MinimizationOperations.minimize(a);
+    Automaton a = Operations.union(list);
+    a = MinimizationOperations.minimize(a);
     return a;
   }
   
@@ -184,15 +186,14 @@
     List<Automaton> list = new ArrayList<>();
     
     for (int i = 0; i < s.length(); i++) {
-      Automaton a = BasicAutomata.makeString(s.substring(0, i));
-      a = BasicOperations.concatenate(a, BasicAutomata.makeAnyChar());
-      a = BasicOperations.concatenate(a, BasicAutomata.makeString(s
-          .substring(i + 1)));
+      Automaton a = Automata.makeString(s.substring(0, i));
+      a = Operations.concatenate(a, Automata.makeAnyChar());
+      a = Operations.concatenate(a, Automata.makeString(s.substring(i + 1)));
       list.add(a);
     }
     
-    Automaton a = BasicOperations.union(list);
-    MinimizationOperations.minimize(a);
+    Automaton a = Operations.union(list);
+    a = MinimizationOperations.minimize(a);
     return a;
   }
   
@@ -201,8 +202,9 @@
    * (transposing two adjacent characters)
    */
   private Automaton transpositionsOf(String s) {
-    if (s.length() < 2)
-      return BasicAutomata.makeEmpty();
+    if (s.length() < 2) {
+      return Automata.makeEmpty();
+    }
     List<Automaton> list = new ArrayList<>();
     for (int i = 0; i < s.length()-1; i++) {
       StringBuilder sb = new StringBuilder();
@@ -211,11 +213,12 @@
       sb.append(s.charAt(i));
       sb.append(s.substring(i+2, s.length()));
       String st = sb.toString();
-      if (!st.equals(s))
-        list.add(BasicAutomata.makeString(st));
+      if (!st.equals(s)) {
+        list.add(Automata.makeString(st));
+      }
     }
-    Automaton a = BasicOperations.union(list);
-    MinimizationOperations.minimize(a);
+    Automaton a = Operations.union(list);
+    a = MinimizationOperations.minimize(a);
     return a;
   }
   
diff --git a/lucene/core/src/test/org/apache/lucene/util/automaton/TestMinimize.java b/lucene/core/src/test/org/apache/lucene/util/automaton/TestMinimize.java
index e306253..82a2914 100644
--- a/lucene/core/src/test/org/apache/lucene/util/automaton/TestMinimize.java
+++ b/lucene/core/src/test/org/apache/lucene/util/automaton/TestMinimize.java
@@ -24,13 +24,13 @@
  */
 public class TestMinimize extends LuceneTestCase {
   /** the minimal and non-minimal are compared to ensure they are the same. */
-  public void test() {
+  public void testBasic() {
     int num = atLeast(200);
     for (int i = 0; i < num; i++) {
       Automaton a = AutomatonTestUtil.randomAutomaton(random());
-      Automaton b = a.clone();
-      MinimizationOperations.minimize(b);
-      assertTrue(BasicOperations.sameLanguage(a, b));
+      Automaton la = Operations.determinize(Operations.removeDeadStates(a));
+      Automaton lb = MinimizationOperations.minimize(a);
+      assertTrue(Operations.sameLanguage(la, lb));
     }
   }
   
@@ -41,12 +41,22 @@
     int num = atLeast(200);
     for (int i = 0; i < num; i++) {
       Automaton a = AutomatonTestUtil.randomAutomaton(random());
-      AutomatonTestUtil.minimizeSimple(a);
-      Automaton b = a.clone();
-      MinimizationOperations.minimize(b);
-      assertTrue(BasicOperations.sameLanguage(a, b));
-      assertEquals(a.getNumberOfStates(), b.getNumberOfStates());
-      assertEquals(a.getNumberOfTransitions(), b.getNumberOfTransitions());
+      a = AutomatonTestUtil.minimizeSimple(a);
+      Automaton b = MinimizationOperations.minimize(a);
+      assertTrue(Operations.sameLanguage(a, b));
+      assertEquals(a.getNumStates(), b.getNumStates());
+      int numStates = a.getNumStates();
+
+      int sum1 = 0;
+      for(int s=0;s<numStates;s++) {
+        sum1 += a.getNumTransitions(s);
+      }
+      int sum2 = 0;
+      for(int s=0;s<numStates;s++) {
+        sum2 += b.getNumTransitions(s);
+      }
+
+      assertEquals(sum1, sum2);
     }
   }
   
diff --git a/lucene/core/src/test/org/apache/lucene/util/automaton/TestOperations.java b/lucene/core/src/test/org/apache/lucene/util/automaton/TestOperations.java
new file mode 100644
index 0000000..e4f2280
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/util/automaton/TestOperations.java
@@ -0,0 +1,289 @@
+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.*;
+
+import org.apache.lucene.util.*;
+import org.apache.lucene.util.fst.Util;
+import com.carrotsearch.randomizedtesting.generators.RandomInts;
+
+public class TestOperations extends LuceneTestCase {
+  /** Test string union. */
+  public void testStringUnion() {
+    List<BytesRef> strings = new ArrayList<>();
+    for (int i = RandomInts.randomIntBetween(random(), 0, 1000); --i >= 0;) {
+      strings.add(new BytesRef(TestUtil.randomUnicodeString(random())));
+    }
+
+    Collections.sort(strings);
+    Automaton union = Automata.makeStringUnion(strings);
+    assertTrue(union.isDeterministic());
+    assertTrue(Operations.sameLanguage(union, naiveUnion(strings)));
+  }
+
+  private static Automaton naiveUnion(List<BytesRef> strings) {
+    Automaton[] eachIndividual = new Automaton[strings.size()];
+    int i = 0;
+    for (BytesRef bref : strings) {
+      eachIndividual[i++] = Automata.makeString(bref.utf8ToString());
+    }
+    return Operations.determinize(Operations.union(Arrays.asList(eachIndividual)));
+  }
+
+  /** Test concatenation with empty language returns empty */
+  public void testEmptyLanguageConcatenate() {
+    Automaton a = Automata.makeString("a");
+    Automaton concat = Operations.concatenate(a, Automata.makeEmpty());
+    assertTrue(Operations.isEmpty(concat));
+  }
+  
+  /** Test optimization to concatenate() with empty String to an NFA */
+  public void testEmptySingletonNFAConcatenate() {
+    Automaton singleton = Automata.makeString("");
+    Automaton expandedSingleton = singleton;
+    // an NFA (two transitions for 't' from initial state)
+    Automaton nfa = Operations.union(Automata.makeString("this"),
+        Automata.makeString("three"));
+    Automaton concat1 = Operations.concatenate(expandedSingleton, nfa);
+    Automaton concat2 = Operations.concatenate(singleton, nfa);
+    assertFalse(concat2.isDeterministic());
+    assertTrue(Operations.sameLanguage(Operations.determinize(concat1),
+                                       Operations.determinize(concat2)));
+    assertTrue(Operations.sameLanguage(Operations.determinize(nfa),
+                                       Operations.determinize(concat1)));
+    assertTrue(Operations.sameLanguage(Operations.determinize(nfa),
+                                       Operations.determinize(concat2)));
+  }
+
+  public void testGetRandomAcceptedString() throws Throwable {
+    final int ITER1 = atLeast(100);
+    final int ITER2 = atLeast(100);
+    for(int i=0;i<ITER1;i++) {
+
+      final RegExp re = new RegExp(AutomatonTestUtil.randomRegexp(random()), RegExp.NONE);
+      //System.out.println("TEST i=" + i + " re=" + re);
+      final Automaton a = Operations.determinize(re.toAutomaton());
+      assertFalse(Operations.isEmpty(a));
+
+      final AutomatonTestUtil.RandomAcceptedStrings rx = new AutomatonTestUtil.RandomAcceptedStrings(a);
+      for(int j=0;j<ITER2;j++) {
+        //System.out.println("TEST: j=" + j);
+        int[] acc = null;
+        try {
+          acc = rx.getRandomAcceptedString(random());
+          final String s = UnicodeUtil.newString(acc, 0, acc.length);
+          //a.writeDot("adot");
+          assertTrue(Operations.run(a, s));
+        } catch (Throwable t) {
+          System.out.println("regexp: " + re);
+          if (acc != null) {
+            System.out.println("fail acc re=" + re + " count=" + acc.length);
+            for(int k=0;k<acc.length;k++) {
+              System.out.println("  " + Integer.toHexString(acc[k]));
+            }
+          }
+          throw t;
+        }
+      }
+    }
+  }
+  /**
+   * tests against the original brics implementation.
+   */
+  public void testIsFinite() {
+    int num = atLeast(200);
+    for (int i = 0; i < num; i++) {
+      Automaton a = AutomatonTestUtil.randomAutomaton(random());
+      assertEquals(AutomatonTestUtil.isFiniteSlow(a), Operations.isFinite(a));
+    }
+  }
+
+  /** 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
+   */
+  public void testFiniteStringsBasic() {
+    Automaton a = Operations.union(Automata.makeString("dog"), Automata.makeString("duck"));
+    a = MinimizationOperations.minimize(a);
+    Set<IntsRef> strings = getFiniteStrings(a, -1, true);
+    assertEquals(2, strings.size());
+    IntsRef dog = new IntsRef();
+    Util.toIntsRef(new BytesRef("dog"), dog);
+    assertTrue(strings.contains(dog));
+    IntsRef duck = new IntsRef();
+    Util.toIntsRef(new BytesRef("duck"), duck);
+    assertTrue(strings.contains(duck));
+  }
+
+  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());
+    IntsRef scratch = new IntsRef();
+    Util.toUTF32(bigString1.toCharArray(), 0, bigString1.length(), scratch);
+    assertTrue(strings.contains(scratch));
+    Util.toUTF32(bigString2.toCharArray(), 0, bigString2.length(), scratch);
+    assertTrue(strings.contains(scratch));
+  }
+
+  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<>();
+    for(int i=0;i<numStrings;i++) {
+      String s = TestUtil.randomSimpleString(random(), 1, 200);
+      automata.add(Automata.makeString(s));
+      IntsRef scratch = new IntsRef();
+      Util.toUTF32(s.toCharArray(), 0, s.length(), scratch);
+      strings.add(scratch);
+      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);
+      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);
+    } 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 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
+    }
+  }
+
+  public void testInvalidLimit2() {
+    Automaton a = AutomatonTestUtil.randomAutomaton(random());
+    try {
+      Operations.getFiniteStrings(a, 0);
+      fail("did not hit exception");
+    } catch (IllegalArgumentException iae) {
+      // expected
+    }
+  }
+
+  public void testSingletonNoLimit() {
+    Set<IntsRef> result = Operations.getFiniteStrings(Automata.makeString("foobar"), -1);
+    assertEquals(1, result.size());
+    IntsRef scratch = new IntsRef();
+    Util.toUTF32("foobar".toCharArray(), 0, 6, scratch);
+    assertTrue(result.contains(scratch));
+  }
+
+  public void testSingletonLimit1() {
+    Set<IntsRef> result = Operations.getFiniteStrings(Automata.makeString("foobar"), 1);
+    assertEquals(1, result.size());
+    IntsRef scratch = new IntsRef();
+    Util.toUTF32("foobar".toCharArray(), 0, 6, scratch);
+    assertTrue(result.contains(scratch));
+  }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/util/automaton/TestSpecialOperations.java b/lucene/core/src/test/org/apache/lucene/util/automaton/TestSpecialOperations.java
deleted file mode 100644
index d1f3e47..0000000
--- a/lucene/core/src/test/org/apache/lucene/util/automaton/TestSpecialOperations.java
+++ /dev/null
@@ -1,222 +0,0 @@
-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.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IntsRef;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.fst.Util;
-
-public class TestSpecialOperations extends LuceneTestCase {
-  /**
-   * tests against the original brics implementation.
-   */
-  public void testIsFinite() {
-    int num = atLeast(200);
-    for (int i = 0; i < num; i++) {
-      Automaton a = AutomatonTestUtil.randomAutomaton(random());
-      Automaton b = a.clone();
-      assertEquals(AutomatonTestUtil.isFiniteSlow(a), SpecialOperations.isFinite(b));
-    }
-  }
-
-  /** 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 = SpecialOperations.getFiniteStrings(a, limit);
-    if (testRecursive) {
-      assertEquals(AutomatonTestUtil.getFiniteStringsRecursive(a, limit), result);
-    }
-    return result;
-  }
-  
-  /**
-   * Basic test for getFiniteStrings
-   */
-  public void testFiniteStringsBasic() {
-    Automaton a = BasicOperations.union(BasicAutomata.makeString("dog"), BasicAutomata.makeString("duck"));
-    MinimizationOperations.minimize(a);
-    Set<IntsRef> strings = getFiniteStrings(a, -1, true);
-    assertEquals(2, strings.size());
-    IntsRef dog = new IntsRef();
-    Util.toIntsRef(new BytesRef("dog"), dog);
-    assertTrue(strings.contains(dog));
-    IntsRef duck = new IntsRef();
-    Util.toIntsRef(new BytesRef("duck"), duck);
-    assertTrue(strings.contains(duck));
-  }
-
-  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 = BasicOperations.union(BasicAutomata.makeString(bigString1), BasicAutomata.makeString(bigString2));
-    Set<IntsRef> strings = getFiniteStrings(a, -1, false);
-    assertEquals(2, strings.size());
-    IntsRef scratch = new IntsRef();
-    Util.toUTF32(bigString1.toCharArray(), 0, bigString1.length(), scratch);
-    assertTrue(strings.contains(scratch));
-    Util.toUTF32(bigString2.toCharArray(), 0, bigString2.length(), scratch);
-    assertTrue(strings.contains(scratch));
-  }
-
-  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<Automaton>();
-    for(int i=0;i<numStrings;i++) {
-      String s = TestUtil.randomSimpleString(random(), 1, 200);
-      automata.add(BasicAutomata.makeString(s));
-      IntsRef scratch = new IntsRef();
-      Util.toUTF32(s.toCharArray(), 0, s.length(), scratch);
-      strings.add(scratch);
-      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 = BasicOperations.union(automata);
-    if (random().nextBoolean()) {
-      Automaton.minimize(a);
-      if (VERBOSE) {
-        System.out.println("TEST: a.minimize numStates=" + a.getNumberOfStates());
-      }
-    } else if (random().nextBoolean()) {
-      if (VERBOSE) {
-        System.out.println("TEST: a.determinize");
-      }
-      a.determinize();
-    } else if (random().nextBoolean()) {
-      if (VERBOSE) {
-        System.out.println("TEST: a.reduce");
-      }
-      a.reduce();
-    } else if (random().nextBoolean()) {
-      if (VERBOSE) {
-        System.out.println("TEST: a.getNumberedStates");
-      }
-      a.getNumberedStates();
-    }
-
-    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 void testWithCycle() throws Exception {
-    try {
-      SpecialOperations.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:
-        SpecialOperations.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(SpecialOperations.isFinite(a));
-      } catch (IllegalArgumentException iae) {
-        assertFalse(SpecialOperations.isFinite(a));
-      }
-    }
-  }
-
-  public void testInvalidLimit() {
-    Automaton a = AutomatonTestUtil.randomAutomaton(random());
-    try {
-      SpecialOperations.getFiniteStrings(a, -7);
-      fail("did not hit exception");
-    } catch (IllegalArgumentException iae) {
-      // expected
-    }
-  }
-
-  public void testInvalidLimit2() {
-    Automaton a = AutomatonTestUtil.randomAutomaton(random());
-    try {
-      SpecialOperations.getFiniteStrings(a, 0);
-      fail("did not hit exception");
-    } catch (IllegalArgumentException iae) {
-      // expected
-    }
-  }
-
-  public void testSingletonNoLimit() {
-    Set<IntsRef> result = SpecialOperations.getFiniteStrings(BasicAutomata.makeString("foobar"), -1);
-    assertEquals(1, result.size());
-    IntsRef scratch = new IntsRef();
-    Util.toUTF32("foobar".toCharArray(), 0, 6, scratch);
-    assertTrue(result.contains(scratch));
-  }
-
-  public void testSingletonLimit1() {
-    Set<IntsRef> result = SpecialOperations.getFiniteStrings(BasicAutomata.makeString("foobar"), 1);
-    assertEquals(1, result.size());
-    IntsRef scratch = new IntsRef();
-    Util.toUTF32("foobar".toCharArray(), 0, 6, scratch);
-    assertTrue(result.contains(scratch));
-  }
-}
diff --git a/lucene/core/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java b/lucene/core/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java
index d5faa4d..95b19e0 100644
--- a/lucene/core/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java
+++ b/lucene/core/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java
@@ -17,13 +17,17 @@
  * limitations under the License.
  */
 
+import java.nio.charset.StandardCharsets;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.UnicodeUtil;
-
-import java.nio.charset.StandardCharsets;
-import java.util.Random;
+import org.apache.lucene.util.fst.Util;
 
 public class TestUTF32ToUTF8 extends LuceneTestCase {
   
@@ -151,12 +155,7 @@
         continue;
       }
       
-      final Automaton a = new Automaton();
-      final State end = new State();
-      end.setAccept(true);
-      a.getInitialState().addTransition(new Transition(startCode, endCode, end));
-      a.setDeterministic(true);
-
+      Automaton a = Automata.makeCharRange(startCode, endCode);
       testOne(r, new ByteRunAutomaton(a), startCode, endCode, ITERS_PER_DFA);
     }
   }
@@ -208,6 +207,20 @@
       assertAutomaton(new RegExp(AutomatonTestUtil.randomRegexp(random()), RegExp.NONE).toAutomaton());
     }
   }
+
+  public void testSingleton() throws Exception {
+    int iters = atLeast(100);
+    for(int iter=0;iter<iters;iter++) {
+      String s = TestUtil.randomRealisticUnicodeString(random());
+      Automaton a = Automata.makeString(s);
+      Automaton utf8 = new UTF32ToUTF8().convert(a);
+      IntsRef ints = new IntsRef();
+      Util.toIntsRef(new BytesRef(s), ints);
+      Set<IntsRef> set = new HashSet<>();
+      set.add(ints);
+      assertEquals(set, Operations.getFiniteStrings(utf8, -1));
+    }
+  }
   
   private void assertAutomaton(Automaton automaton) throws Exception {
     CharacterRunAutomaton cra = new CharacterRunAutomaton(automaton);
diff --git a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
index 26aaeb0..8c3f60c 100644
--- a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
+++ b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
@@ -17,43 +17,6 @@
  * limitations under the License.
  */
 
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.MultiFields;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.FSDirectory;
-import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IntsRef;
-import org.apache.lucene.util.LineFileDocs;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.LuceneTestCase.Slow;
-import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
-import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.RegExp;
-import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
-import org.apache.lucene.util.fst.FST.Arc;
-import org.apache.lucene.util.fst.FST.BytesReader;
-import org.apache.lucene.util.fst.PairOutputs.Pair;
-import org.apache.lucene.util.fst.Util.Result;
-import org.apache.lucene.util.packed.PackedInts;
-
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
@@ -78,6 +41,43 @@
 import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.RegExp;
+import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
+import org.apache.lucene.util.fst.FST.Arc;
+import org.apache.lucene.util.fst.FST.BytesReader;
+import org.apache.lucene.util.fst.PairOutputs.Pair;
+import org.apache.lucene.util.fst.Util.Result;
+import org.apache.lucene.util.packed.PackedInts;
+
 import static org.apache.lucene.util.fst.FSTTester.getRandomString;
 import static org.apache.lucene.util.fst.FSTTester.simpleRandomString;
 import static org.apache.lucene.util.fst.FSTTester.toIntsRef;
@@ -346,7 +346,7 @@
       BytesRef term;
       int ord = 0;
 
-      Automaton automaton = new RegExp(".*", RegExp.NONE).toAutomaton();    
+      Automaton automaton = new RegExp(".*", RegExp.NONE).toAutomaton();
       final TermsEnum termsEnum2 = terms.intersect(new CompiledAutomaton(automaton, false, false), null);
 
       while((term = termsEnum.next()) != null) {
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/MultiTermHighlighting.java b/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/MultiTermHighlighting.java
index a061025..bf2f1d2 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/MultiTermHighlighting.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/MultiTermHighlighting.java
@@ -46,11 +46,11 @@
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.UnicodeUtil;
-import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.BasicAutomata;
-import org.apache.lucene.util.automaton.BasicOperations;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
+import org.apache.lucene.util.automaton.Automaton;
 
 /**
  * Support for highlighting multiterm queries in PostingsHighlighter.
@@ -106,8 +106,8 @@
       final PrefixQuery pq = (PrefixQuery) query;
       Term prefix = pq.getPrefix();
       if (prefix.field().equals(field)) {
-        list.add(new CharacterRunAutomaton(BasicOperations.concatenate(BasicAutomata.makeString(prefix.text()), 
-                                                                       BasicAutomata.makeAnyString())) {
+        list.add(new CharacterRunAutomaton(Operations.concatenate(Automata.makeString(prefix.text()), 
+                                                                       Automata.makeAnyString())) {
           @Override
           public String toString() {
             return pq.toString();
@@ -126,11 +126,8 @@
         int prefixLength = Math.min(fq.getPrefixLength(), termLength);
         String suffix = UnicodeUtil.newString(termText, prefixLength, termText.length - prefixLength);
         LevenshteinAutomata builder = new LevenshteinAutomata(suffix, fq.getTranspositions());
-        Automaton automaton = builder.toAutomaton(fq.getMaxEdits());
-        if (prefixLength > 0) {
-          Automaton prefix = BasicAutomata.makeString(UnicodeUtil.newString(termText, 0, prefixLength));
-          automaton = BasicOperations.concatenate(prefix, automaton);
-        }
+        String prefix = UnicodeUtil.newString(termText, 0, prefixLength);
+        Automaton automaton = builder.toAutomaton(fq.getMaxEdits(), prefix);
         list.add(new CharacterRunAutomaton(automaton) {
           @Override
           public String toString() {
@@ -161,7 +158,7 @@
         final Comparator<CharsRef> comparator = CharsRef.getUTF16SortedAsUTF8Comparator();
         
         // this is *not* an automaton, but its very simple
-        list.add(new CharacterRunAutomaton(BasicAutomata.makeEmpty()) {
+        list.add(new CharacterRunAutomaton(Automata.makeEmpty()) {
           @Override
           public boolean run(char[] s, int offset, int length) {
             scratch.chars = s;
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
index 30b86f6..ee6d8b8 100644
--- a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
@@ -55,7 +55,7 @@
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.automaton.BasicAutomata;
+import org.apache.lucene.util.automaton.Automata;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 import org.apache.lucene.util.automaton.RegExp;
 import org.w3c.dom.Element;
@@ -1340,7 +1340,7 @@
       @Override
       public void run() throws Exception {
         String goodWord = "goodtoken";
-        CharacterRunAutomaton stopWords = new CharacterRunAutomaton(BasicAutomata.makeString("stoppedtoken"));
+        CharacterRunAutomaton stopWords = new CharacterRunAutomaton(Automata.makeString("stoppedtoken"));
         // we disable MockTokenizer checks because we will forcefully limit the 
         // tokenstream and call end() before incrementToken() returns false.
         final MockAnalyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true, stopWords);
diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
index 48c33fe..acf07b6 100644
--- a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
+++ b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
@@ -629,7 +629,7 @@
       }
       DocsEnum parents = MultiFields.getTermDocsEnum(joinR, null, "isParent", new BytesRef("x"));
       System.out.println("parent docIDs:");
-      while (parents.nextDoc() != parents.NO_MORE_DOCS) {
+      while (parents.nextDoc() != DocsEnum.NO_MORE_DOCS) {
         System.out.println("  " + parents.docID());
       }
     }
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/precedence/TestPrecedenceQueryParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/precedence/TestPrecedenceQueryParser.java
index a758a6f..389bd0f 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/precedence/TestPrecedenceQueryParser.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/precedence/TestPrecedenceQueryParser.java
@@ -46,7 +46,7 @@
 import org.apache.lucene.search.TermRangeQuery;
 import org.apache.lucene.search.WildcardQuery;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.automaton.BasicAutomata;
+import org.apache.lucene.util.automaton.Automata;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -557,7 +557,7 @@
   }
 
   public void testBoost() throws Exception {
-    CharacterRunAutomaton stopSet = new CharacterRunAutomaton(BasicAutomata.makeString("on"));
+    CharacterRunAutomaton stopSet = new CharacterRunAutomaton(Automata.makeString("on"));
     Analyzer oneStopAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true, stopSet);
 
     PrecedenceQueryParser qp = new PrecedenceQueryParser();
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java
index abcf7a7..5068210 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java
@@ -67,7 +67,7 @@
 import org.apache.lucene.search.WildcardQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.automaton.BasicAutomata;
+import org.apache.lucene.util.automaton.Automata;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 import org.apache.lucene.util.automaton.RegExp;
 import org.junit.AfterClass;
@@ -957,7 +957,7 @@
   }
 
   public void testBoost() throws Exception {
-    CharacterRunAutomaton stopSet = new CharacterRunAutomaton(BasicAutomata.makeString("on"));
+    CharacterRunAutomaton stopSet = new CharacterRunAutomaton(Automata.makeString("on"));
     Analyzer oneStopAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true, stopSet);
     StandardQueryParser qp = new StandardQueryParser();
     qp.setAnalyzer(oneStopAnalyzer);
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
index 613a63d..f76c9e1 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
@@ -47,7 +47,7 @@
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.automaton.BasicAutomata;
+import org.apache.lucene.util.automaton.Automata;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 import org.apache.lucene.util.automaton.RegExp;
 import org.junit.AfterClass;
@@ -868,7 +868,7 @@
 
   public void testBoost()
     throws Exception {
-    CharacterRunAutomaton stopWords = new CharacterRunAutomaton(BasicAutomata.makeString("on"));
+    CharacterRunAutomaton stopWords = new CharacterRunAutomaton(Automata.makeString("on"));
     Analyzer oneStopAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true, stopWords);
     CommonQueryParserConfiguration qp = getParserConfig(oneStopAnalyzer);
     Query q = getQuery("on^1.0",qp);
diff --git a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsReader.java b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsReader.java
index b4e69e1..3919b45 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsReader.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsReader.java
@@ -18,45 +18,25 @@
  */
 
 import java.io.IOException;
-import java.io.PrintStream;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.TreeMap;
 
-import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.StringHelper;
-import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.RunAutomaton;
-import org.apache.lucene.util.automaton.Transition;
-import org.apache.lucene.util.fst.ByteSequenceOutputs;
-import org.apache.lucene.util.fst.FST;
-import org.apache.lucene.util.fst.Outputs;
 import org.apache.lucene.util.fst.PairOutputs.Pair;
-import org.apache.lucene.util.fst.PairOutputs;
-import org.apache.lucene.util.fst.Util;
 
 /**
  * See {@link VersionBlockTreeTermsWriter}.
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
index 483cbb8..afae341 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
@@ -17,6 +17,16 @@
  * limitations under the License.
  */
 
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.TokenStreamToAutomaton;
@@ -33,30 +43,19 @@
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.OfflineSorter;
 import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.BasicOperations;
-import org.apache.lucene.util.automaton.SpecialOperations;
-import org.apache.lucene.util.automaton.State;
 import org.apache.lucene.util.automaton.Transition;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.ByteSequenceOutputs;
-import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.FST.BytesReader;
-import org.apache.lucene.util.fst.PairOutputs;
+import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.PairOutputs.Pair;
+import org.apache.lucene.util.fst.PairOutputs;
 import org.apache.lucene.util.fst.PositiveIntOutputs;
-import org.apache.lucene.util.fst.Util;
 import org.apache.lucene.util.fst.Util.Result;
 import org.apache.lucene.util.fst.Util.TopResults;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
+import org.apache.lucene.util.fst.Util;
 
 /**
  * Suggester that first analyzes the surface form, adds the
@@ -255,37 +254,64 @@
     return fst == null ? 0 : fst.ramBytesUsed();
   }
 
-  private void copyDestTransitions(State from, State to, List<Transition> transitions) {
-    if (to.isAccept()) {
-      from.setAccept(true);
+  private int[] topoSortStates(Automaton a) {
+    int[] states = new int[a.getNumStates()];
+    final Set<Integer> visited = new HashSet<>();
+    final LinkedList<Integer> worklist = new LinkedList<>();
+    worklist.add(0);
+    visited.add(0);
+    int upto = 0;
+    states[upto] = 0;
+    upto++;
+    Transition t = new Transition();
+    while (worklist.size() > 0) {
+      int s = worklist.removeFirst();
+      int count = a.initTransition(s, t);
+      for (int i=0;i<count;i++) {
+        a.getNextTransition(t);
+        if (!visited.contains(t.dest)) {
+          visited.add(t.dest);
+          worklist.add(t.dest);
+          states[upto++] = t.dest;
+        }
+      }
     }
-    for(Transition t : to.getTransitions()) {
-      transitions.add(t);
-    }
+    return states;
   }
 
+
   // Replaces SEP with epsilon or remaps them if
   // we were asked to preserve them:
-  private void replaceSep(Automaton a) {
+  private Automaton replaceSep(Automaton a) {
 
-    State[] states = a.getNumberedStates();
+    Automaton result = new Automaton();
+
+    // Copy all states over
+    int numStates = a.getNumStates();
+    for(int s=0;s<numStates;s++) {
+      result.createState();
+      result.setAccept(s, a.isAccept(s));
+    }
 
     // Go in reverse topo sort so we know we only have to
     // make one pass:
-    for(int stateNumber=states.length-1;stateNumber >=0;stateNumber--) {
-      final State state = states[stateNumber];
-      List<Transition> newTransitions = new ArrayList<>();
-      for(Transition t : state.getTransitions()) {
-        assert t.getMin() == t.getMax();
-        if (t.getMin() == TokenStreamToAutomaton.POS_SEP) {
+    Transition t = new Transition();
+    int[] topoSortStates = topoSortStates(a);
+    for(int i=0;i<topoSortStates.length;i++) {
+      int state = topoSortStates[topoSortStates.length-1-i];
+      int count = a.initTransition(state, t);
+      for(int j=0;j<count;j++) {
+        a.getNextTransition(t);
+        if (t.min == TokenStreamToAutomaton.POS_SEP) {
+          assert t.max == TokenStreamToAutomaton.POS_SEP;
           if (preserveSep) {
             // Remap to SEP_LABEL:
-            newTransitions.add(new Transition(SEP_LABEL, t.getDest()));
+            result.addTransition(state, t.dest, SEP_LABEL);
           } else {
-            copyDestTransitions(state, t.getDest(), newTransitions);
-            a.setDeterministic(false);
+            result.addEpsilon(state, t.dest);
           }
-        } else if (t.getMin() == TokenStreamToAutomaton.HOLE) {
+        } else if (t.min == TokenStreamToAutomaton.HOLE) {
+          assert t.max == TokenStreamToAutomaton.HOLE;
 
           // Just remove the hole: there will then be two
           // SEP tokens next to each other, which will only
@@ -294,14 +320,16 @@
           // that's somehow a problem we can always map HOLE
           // to a dedicated byte (and escape it in the
           // input).
-          copyDestTransitions(state, t.getDest(), newTransitions);
-          a.setDeterministic(false);
+          result.addEpsilon(state, t.dest);
         } else {
-          newTransitions.add(t);
+          result.addTransition(state, t.dest, t.min, t.max);
         }
       }
-      state.setTransitions(newTransitions.toArray(new Transition[newTransitions.size()]));
     }
+
+    result.finishState();
+
+    return result;
   }
 
   /** Used by subclass to change the lookup automaton, if
@@ -665,7 +693,6 @@
     }
     final BytesRef utf8Key = new BytesRef(key);
     try {
-
       Automaton lookupAutomaton = toLookupAutomaton(key);
 
       final CharsRef spare = new CharsRef();
@@ -835,7 +862,7 @@
       automaton = ts2a.toAutomaton(ts);
     }
 
-    replaceSep(automaton);
+    automaton = replaceSep(automaton);
     automaton = convertAutomaton(automaton);
 
     // TODO: LUCENE-5660 re-enable this once we disallow massive suggestion strings
@@ -848,7 +875,8 @@
     // TODO: we could walk & add simultaneously, so we
     // don't have to alloc [possibly biggish]
     // intermediate HashSet in RAM:
-    return SpecialOperations.getFiniteStrings(automaton, maxGraphExpansions);
+
+    return Operations.getFiniteStrings(automaton, maxGraphExpansions);
   }
 
   final Automaton toLookupAutomaton(final CharSequence key) throws IOException {
@@ -856,24 +884,16 @@
     // Turn tokenstream into automaton:
     Automaton automaton = null;
     try (TokenStream ts = queryAnalyzer.tokenStream("", key.toString())) {
-      automaton = (getTokenStreamToAutomaton()).toAutomaton(ts);
+        automaton = getTokenStreamToAutomaton().toAutomaton(ts);
     }
 
-    // TODO: we could use the end offset to "guess"
-    // whether the final token was a partial token; this
-    // would only be a heuristic ... but maybe an OK one.
-    // This way we could eg differentiate "net" from "net ",
-    // which we can't today...
-
-    replaceSep(automaton);
+    automaton = replaceSep(automaton);
 
     // TODO: we can optimize this somewhat by determinizing
     // while we convert
-    BasicOperations.determinize(automaton);
+    automaton = Operations.determinize(automaton);
     return automaton;
   }
-  
-  
 
   /**
    * Returns the weight associated with an input string,
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java
index b9e886f..ef6ea60 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java
@@ -17,13 +17,12 @@
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.io.IOException;
 
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.State;
 import org.apache.lucene.util.automaton.Transition;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.Util;
@@ -43,7 +42,7 @@
   public static final class Path<T> {
 
     /** Node in the automaton where path ends: */
-    public final State state;
+    public final int state;
 
     /** Node in the FST where path ends: */
     public final FST.Arc<T> fstNode;
@@ -55,7 +54,7 @@
     public final IntsRef input;
 
     /** Sole constructor. */
-    public Path(State state, FST.Arc<T> fstNode, T output, IntsRef input) {
+    public Path(int state, FST.Arc<T> fstNode, T output, IntsRef input) {
       this.state = state;
       this.fstNode = fstNode;
       this.output = output;
@@ -72,16 +71,22 @@
     assert a.isDeterministic();
     final List<Path<T>> queue = new ArrayList<>();
     final List<Path<T>> endNodes = new ArrayList<>();
-    queue.add(new Path<>(a.getInitialState(), fst
+    if (a.getNumStates() == 0) {
+      return endNodes;
+    }
+
+    queue.add(new Path<>(0, fst
         .getFirstArc(new FST.Arc<T>()), fst.outputs.getNoOutput(),
         new IntsRef()));
     
     final FST.Arc<T> scratchArc = new FST.Arc<>();
     final FST.BytesReader fstReader = fst.getBytesReader();
-    
+
+    Transition t = new Transition();
+
     while (queue.size() != 0) {
       final Path<T> path = queue.remove(queue.size() - 1);
-      if (path.state.isAccept()) {
+      if (a.isAccept(path.state)) {
         endNodes.add(path);
         // we can stop here if we accept this path,
         // we accept all further paths too
@@ -89,18 +94,20 @@
       }
       
       IntsRef currentInput = path.input;
-      for (Transition t : path.state.getTransitions()) {
-        final int min = t.getMin();
-        final int max = t.getMax();
+      int count = a.initTransition(path.state, t);
+      for (int i=0;i<count;i++) {
+        a.getNextTransition(t);
+        final int min = t.min;
+        final int max = t.max;
         if (min == max) {
-          final FST.Arc<T> nextArc = fst.findTargetArc(t.getMin(),
+          final FST.Arc<T> nextArc = fst.findTargetArc(t.min,
               path.fstNode, scratchArc, fstReader);
           if (nextArc != null) {
             final IntsRef newInput = new IntsRef(currentInput.length + 1);
             newInput.copyInts(currentInput);
-            newInput.ints[currentInput.length] = t.getMin();
+            newInput.ints[currentInput.length] = t.min;
             newInput.length = currentInput.length + 1;
-            queue.add(new Path<>(t.getDest(), new FST.Arc<T>()
+            queue.add(new Path<>(t.dest, new FST.Arc<T>()
                 .copyFrom(nextArc), fst.outputs
                 .add(path.output, nextArc.output), newInput));
           }
@@ -122,7 +129,7 @@
             newInput.copyInts(currentInput);
             newInput.ints[currentInput.length] = nextArc.label;
             newInput.length = currentInput.length + 1;
-            queue.add(new Path<>(t.getDest(), new FST.Arc<T>()
+            queue.add(new Path<>(t.dest, new FST.Arc<T>()
                 .copyFrom(nextArc), fst.outputs
                 .add(path.output, nextArc.output), newInput));
             final int label = nextArc.label; // used in assert
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java
index 458d59d..c4f91aa 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java
@@ -28,11 +28,11 @@
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute; // javadocs
 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.BasicAutomata;
-import org.apache.lucene.util.automaton.BasicOperations;
+import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
-import org.apache.lucene.util.automaton.SpecialOperations;
+import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.UTF32ToUTF8;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.PairOutputs.Pair;
@@ -205,7 +205,7 @@
   protected Automaton convertAutomaton(Automaton a) {
     if (unicodeAware) {
       Automaton utf8automaton = new UTF32ToUTF8().convert(a);
-      BasicOperations.determinize(utf8automaton);
+      utf8automaton = Operations.determinize(utf8automaton);
       return utf8automaton;
     } else {
       return a;
@@ -220,15 +220,14 @@
   }
 
   Automaton toLevenshteinAutomata(Automaton automaton) {
-    final Set<IntsRef> ref = SpecialOperations.getFiniteStrings(automaton, -1);
+    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] = BasicAutomata.makeString(path.ints, path.offset, path.length);
+        subs[upto] = Automata.makeString(path.ints, path.offset, path.length);
         upto++;
       } else {
-        Automaton prefix = BasicAutomata.makeString(path.ints, path.offset, nonFuzzyPrefix);
         int ints[] = new int[path.length-nonFuzzyPrefix];
         System.arraycopy(path.ints, path.offset+nonFuzzyPrefix, ints, 0, ints.length);
         // TODO: maybe add alphaMin to LevenshteinAutomata,
@@ -237,29 +236,24 @@
         // 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);
-        Automaton levAutomaton = lev.toAutomaton(maxEdits);
-        Automaton combined = BasicOperations.concatenate(Arrays.asList(prefix, levAutomaton));
-        combined.setDeterministic(true); // its like the special case in concatenate itself, except we cloneExpanded already
-        subs[upto] = combined;
+        subs[upto] = lev.toAutomaton(maxEdits, UnicodeUtil.newString(path.ints, path.offset, nonFuzzyPrefix));
         upto++;
       }
     }
 
     if (subs.length == 0) {
       // automaton is empty, there is no accepted paths through it
-      return BasicAutomata.makeEmpty(); // matches nothing
+      return Automata.makeEmpty(); // matches nothing
     } else if (subs.length == 1) {
       // no synonyms or anything: just a single path through the tokenstream
       return subs[0];
     } else {
       // multiple paths: this is really scary! is it slow?
       // maybe we should not do this and throw UOE?
-      Automaton a = BasicOperations.union(Arrays.asList(subs));
+      Automaton a = Operations.union(Arrays.asList(subs));
       // TODO: we could call toLevenshteinAutomata() before det? 
       // this only happens if you have multiple paths anyway (e.g. synonyms)
-      BasicOperations.determinize(a);
-
-      return a;
+      return Operations.determinize(a);
     }
   }
 }
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/InputArrayIterator.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/InputArrayIterator.java
index 75301e9..937fb9e 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/InputArrayIterator.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/InputArrayIterator.java
@@ -92,4 +92,4 @@
   public boolean hasContexts() {
     return hasContexts;
   }
-}
\ No newline at end of file
+}
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java
index 6a0a58c..0859055 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java
@@ -40,15 +40,14 @@
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
-import org.apache.lucene.search.suggest.Lookup.LookupResult;
 import org.apache.lucene.search.suggest.Input;
 import org.apache.lucene.search.suggest.InputArrayIterator;
+import org.apache.lucene.search.suggest.Lookup.LookupResult;
 import org.apache.lucene.util.BytesRef;
 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.State;
 import org.apache.lucene.util.fst.Util;
 
 public class FuzzySuggesterTest extends LuceneTestCase {
@@ -754,27 +753,28 @@
       // this:
       Automaton automaton = suggester.convertAutomaton(suggester.toLevenshteinAutomata(suggester.toLookupAutomaton(analyzedKey)));
       assertTrue(automaton.isDeterministic());
+
       // TODO: could be faster... but its slowCompletor for a reason
       BytesRef spare = new BytesRef();
       for (TermFreqPayload2 e : slowCompletor) {
         spare.copyChars(e.analyzedForm);
         Set<IntsRef> finiteStrings = suggester.toFiniteStrings(spare, tokenStreamToAutomaton);
         for (IntsRef intsRef : finiteStrings) {
-          State p = automaton.getInitialState();
+          int p = 0;
           BytesRef ref = Util.toBytesRef(intsRef, spare);
           boolean added = false;
           for (int i = ref.offset; i < ref.length; i++) {
-            State q = p.step(ref.bytes[i] & 0xff);
-            if (q == null) {
+            int q = automaton.step(p, ref.bytes[i] & 0xff);
+            if (q == -1) {
               break;
-            } else if (q.isAccept()) {
+            } else if (automaton.isAccept(q)) {
               matches.add(new LookupResult(e.surfaceForm, e.weight));
               added = true;
               break;
             }
             p = q;
           }
-          if (!added && p.isAccept()) {
+          if (!added && automaton.isAccept(p)) {
             matches.add(new LookupResult(e.surfaceForm, e.weight));
           } 
         }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/analysis/MockTokenFilter.java b/lucene/test-framework/src/java/org/apache/lucene/analysis/MockTokenFilter.java
index 0aea06c..25619a8 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/analysis/MockTokenFilter.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/analysis/MockTokenFilter.java
@@ -17,15 +17,15 @@
  * limitations under the License.
  */
 
-import static org.apache.lucene.util.automaton.BasicAutomata.makeEmpty;
-import static org.apache.lucene.util.automaton.BasicAutomata.makeString;
+import static org.apache.lucene.util.automaton.Automata.makeEmpty;
+import static org.apache.lucene.util.automaton.Automata.makeString;
 
 import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
-import org.apache.lucene.util.automaton.BasicOperations;
+import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 
 /**
@@ -43,7 +43,7 @@
   
   /** Set of common english stopwords */
   public static final CharacterRunAutomaton ENGLISH_STOPSET = 
-    new CharacterRunAutomaton(BasicOperations.union(Arrays.asList(
+    new CharacterRunAutomaton(Operations.union(Arrays.asList(
       makeString("a"), makeString("an"), makeString("and"), makeString("are"),
       makeString("as"), makeString("at"), makeString("be"), makeString("but"), 
       makeString("by"), makeString("for"), makeString("if"), makeString("in"), 
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java b/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java
index 2ce1e1c..a47dd3d 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java
@@ -33,7 +33,7 @@
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.automaton.BasicAutomata;
+import org.apache.lucene.util.automaton.Automata;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -57,7 +57,7 @@
     Random random = random();
     directory = newDirectory();
     stopword = "" + randomChar();
-    CharacterRunAutomaton stopset = new CharacterRunAutomaton(BasicAutomata.makeString(stopword));
+    CharacterRunAutomaton stopset = new CharacterRunAutomaton(Automata.makeString(stopword));
     analyzer = new MockAnalyzer(random, MockTokenizer.WHITESPACE, false, stopset);
     RandomIndexWriter iw = new RandomIndexWriter(random, directory, analyzer);
     Document doc = new Document();
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/automaton/AutomatonTestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/automaton/AutomatonTestUtil.java
index f34183c..41d7c51 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/automaton/AutomatonTestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/automaton/AutomatonTestUtil.java
@@ -20,7 +20,6 @@
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.IdentityHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -31,7 +30,6 @@
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.UnicodeUtil;
-import org.apache.lucene.util.fst.Util;
 
 /**
  * Utilities for testing automata.
@@ -92,40 +90,40 @@
   /** picks a random int code point, avoiding surrogates;
    * throws IllegalArgumentException if this transition only
    * accepts surrogates */
-  private static int getRandomCodePoint(final Random r, final Transition t) {
+  private static int getRandomCodePoint(final Random r, int min, int max) {
     final int code;
-    if (t.max < UnicodeUtil.UNI_SUR_HIGH_START ||
-        t.min > UnicodeUtil.UNI_SUR_HIGH_END) {
+    if (max < UnicodeUtil.UNI_SUR_HIGH_START ||
+        min > UnicodeUtil.UNI_SUR_HIGH_END) {
       // easy: entire range is before or after surrogates
-      code = t.min+r.nextInt(t.max-t.min+1);
-    } else if (t.min >= UnicodeUtil.UNI_SUR_HIGH_START) {
-      if (t.max > UnicodeUtil.UNI_SUR_LOW_END) {
+      code = min+r.nextInt(max-min+1);
+    } else if (min >= UnicodeUtil.UNI_SUR_HIGH_START) {
+      if (max > UnicodeUtil.UNI_SUR_LOW_END) {
         // after surrogates
-        code = 1+UnicodeUtil.UNI_SUR_LOW_END+r.nextInt(t.max-UnicodeUtil.UNI_SUR_LOW_END);
+        code = 1+UnicodeUtil.UNI_SUR_LOW_END+r.nextInt(max-UnicodeUtil.UNI_SUR_LOW_END);
       } else {
-        throw new IllegalArgumentException("transition accepts only surrogates: " + t);
+        throw new IllegalArgumentException("transition accepts only surrogates: min=" + min + " max=" + max);
       }
-    } else if (t.max <= UnicodeUtil.UNI_SUR_LOW_END) {
-      if (t.min < UnicodeUtil.UNI_SUR_HIGH_START) {
+    } else if (max <= UnicodeUtil.UNI_SUR_LOW_END) {
+      if (min < UnicodeUtil.UNI_SUR_HIGH_START) {
         // before surrogates
-        code = t.min + r.nextInt(UnicodeUtil.UNI_SUR_HIGH_START - t.min);
+        code = min + r.nextInt(UnicodeUtil.UNI_SUR_HIGH_START - min);
       } else {
-        throw new IllegalArgumentException("transition accepts only surrogates: " + t);
+        throw new IllegalArgumentException("transition accepts only surrogates: min=" + min + " max=" + max);
       }
     } else {
       // range includes all surrogates
-      int gap1 = UnicodeUtil.UNI_SUR_HIGH_START - t.min;
-      int gap2 = t.max - UnicodeUtil.UNI_SUR_LOW_END;
+      int gap1 = UnicodeUtil.UNI_SUR_HIGH_START - min;
+      int gap2 = max - UnicodeUtil.UNI_SUR_LOW_END;
       int c = r.nextInt(gap1+gap2);
       if (c < gap1) {
-        code = t.min + c;
+        code = min + c;
       } else {
         code = UnicodeUtil.UNI_SUR_LOW_END + c - gap1 + 1;
       }
     }
 
-    assert code >= t.min && code <= t.max && (code < UnicodeUtil.UNI_SUR_HIGH_START || code > UnicodeUtil.UNI_SUR_LOW_END):
-      "code=" + code + " min=" + t.min + " max=" + t.max;
+    assert code >= min && code <= max && (code < UnicodeUtil.UNI_SUR_HIGH_START || code > UnicodeUtil.UNI_SUR_LOW_END):
+      "code=" + code + " min=" + min + " max=" + max;
     return code;
   }
 
@@ -140,11 +138,13 @@
 
     private final Map<Transition,Boolean> leadsToAccept;
     private final Automaton a;
+    private final Transition[][] transitions;
 
     private static class ArrivingTransition {
-      final State from;
+      final int from;
       final Transition t;
-      public ArrivingTransition(State from, Transition t) {
+
+      public ArrivingTransition(int from, Transition t) {
         this.from = from;
         this.t = t;
       }
@@ -152,32 +152,30 @@
 
     public RandomAcceptedStrings(Automaton a) {
       this.a = a;
-      if (a.isSingleton()) {
-        leadsToAccept = null;
-        return;
+      if (a.getNumStates() == 0) {
+        throw new IllegalArgumentException("this automaton accepts nothing");
       }
+      this.transitions = a.getSortedTransitions();
 
-      // must use IdentityHashmap because two Transitions w/
-      // different start nodes can be considered the same
-      leadsToAccept = new IdentityHashMap<>();
-      final Map<State,List<ArrivingTransition>> allArriving = new HashMap<>();
+      leadsToAccept = new HashMap<>();
+      final Map<Integer,List<ArrivingTransition>> allArriving = new HashMap<>();
 
-      final LinkedList<State> q = new LinkedList<>();
-      final Set<State> seen = new HashSet<>();
+      final LinkedList<Integer> q = new LinkedList<>();
+      final Set<Integer> seen = new HashSet<>();
 
       // reverse map the transitions, so we can quickly look
       // up all arriving transitions to a given state
-      for(State s: a.getNumberedStates()) {
-        for(int i=0;i<s.numTransitions;i++) {
-          final Transition t = s.transitionsArray[i];
-          List<ArrivingTransition> tl = allArriving.get(t.to);
+      int numStates = a.getNumStates();
+      for(int s=0;s<numStates;s++) {
+        for(Transition t : transitions[s]) {
+          List<ArrivingTransition> tl = allArriving.get(t.dest);
           if (tl == null) {
             tl = new ArrayList<>();
-            allArriving.put(t.to, tl);
+            allArriving.put(t.dest, tl);
           }
           tl.add(new ArrivingTransition(s, t));
         }
-        if (s.accept) {
+        if (a.isAccept(s)) {
           q.add(s);
           seen.add(s);
         }
@@ -185,12 +183,12 @@
 
       // Breadth-first search, from accept states,
       // backwards:
-      while(!q.isEmpty()) {
-        final State s = q.removeFirst();
+      while (q.isEmpty() == false) {
+        final int s = q.removeFirst();
         List<ArrivingTransition> arriving = allArriving.get(s);
         if (arriving != null) {
           for(ArrivingTransition at : arriving) {
-            final State from = at.from;
+            final int from = at.from;
             if (!seen.contains(from)) {
               q.add(from);
               seen.add(from);
@@ -204,62 +202,49 @@
     public int[] getRandomAcceptedString(Random r) {
 
       final List<Integer> soFar = new ArrayList<>();
-      if (a.isSingleton()) {
-        // accepts only one
-        final String s = a.singleton;
+
+      int s = 0;
+
+      while(true) {
       
-        int charUpto = 0;
-        while(charUpto < s.length()) {
-          final int cp = s.codePointAt(charUpto);
-          charUpto += Character.charCount(cp);
-          soFar.add(cp);
-        }
-      } else {
-
-        State s = a.initial;
-
-        while(true) {
-      
-          if (s.accept) {
-            if (s.numTransitions == 0) {
-              // stop now
-              break;
-            } else {
-              if (r.nextBoolean()) {
-                break;
-              }
-            }
-          }
-
-          if (s.numTransitions == 0) {
-            throw new RuntimeException("this automaton has dead states");
-          }
-
-          boolean cheat = r.nextBoolean();
-
-          final Transition t;
-          if (cheat) {
-            // pick a transition that we know is the fastest
-            // path to an accept state
-            List<Transition> toAccept = new ArrayList<>();
-            for(int i=0;i<s.numTransitions;i++) {
-              final Transition t0 = s.transitionsArray[i];
-              if (leadsToAccept.containsKey(t0)) {
-                toAccept.add(t0);
-              }
-            }
-            if (toAccept.size() == 0) {
-              // this is OK -- it means we jumped into a cycle
-              t = s.transitionsArray[r.nextInt(s.numTransitions)];
-            } else {
-              t = toAccept.get(r.nextInt(toAccept.size()));
-            }
+        if (a.isAccept(s)) {
+          if (a.getNumTransitions(s) == 0) {
+            // stop now
+            break;
           } else {
-            t = s.transitionsArray[r.nextInt(s.numTransitions)];
+            if (r.nextBoolean()) {
+              break;
+            }
           }
-          soFar.add(getRandomCodePoint(r, t));
-          s = t.to;
         }
+
+        if (a.getNumTransitions(s) == 0) {
+          throw new RuntimeException("this automaton has dead states");
+        }
+
+        boolean cheat = r.nextBoolean();
+
+        final Transition t;
+        if (cheat) {
+          // pick a transition that we know is the fastest
+          // path to an accept state
+          List<Transition> toAccept = new ArrayList<>();
+          for(Transition t0 : transitions[s]) {
+            if (leadsToAccept.containsKey(t0)) {
+              toAccept.add(t0);
+            }
+          }
+          if (toAccept.size() == 0) {
+            // this is OK -- it means we jumped into a cycle
+            t = transitions[s][r.nextInt(transitions[s].length)];
+          } else {
+            t = toAccept.get(r.nextInt(toAccept.size()));
+          }
+        } else {
+          t = transitions[s][r.nextInt(transitions[s].length)];
+        }
+        soFar.add(getRandomCodePoint(r, t.min, t.max));
+        s = t.dest;
       }
 
       return ArrayUtil.toIntArray(soFar);
@@ -270,19 +255,21 @@
   public static Automaton randomAutomaton(Random random) {
     // get two random Automata from regexps
     Automaton a1 = new RegExp(AutomatonTestUtil.randomRegexp(random), RegExp.NONE).toAutomaton();
-    if (random.nextBoolean())
-      a1 = BasicOperations.complement(a1);
+    if (random.nextBoolean()) {
+      a1 = Operations.complement(a1);
+    }
     
     Automaton a2 = new RegExp(AutomatonTestUtil.randomRegexp(random), RegExp.NONE).toAutomaton();
-    if (random.nextBoolean()) 
-      a2 = BasicOperations.complement(a2);
-    
+    if (random.nextBoolean()) {
+      a2 = Operations.complement(a2);
+    }
+
     // combine them in random ways
-    switch(random.nextInt(4)) {
-      case 0: return BasicOperations.concatenate(a1, a2);
-      case 1: return BasicOperations.union(a1, a2);
-      case 2: return BasicOperations.intersection(a1, a2);
-      default: return BasicOperations.minus(a1, a2);
+    switch (random.nextInt(4)) {
+      case 0: return Operations.concatenate(a1, a2);
+      case 1: return Operations.union(a1, a2);
+      case 2: return Operations.intersection(a1, a2);
+      default: return Operations.minus(a1, a2);
     }
   }
   
@@ -323,70 +310,81 @@
   /**
    * Simple, original brics implementation of Brzozowski minimize()
    */
-  public static void minimizeSimple(Automaton a) {
-    if (a.isSingleton())
-      return;
-    determinizeSimple(a, SpecialOperations.reverse(a));
-    determinizeSimple(a, SpecialOperations.reverse(a));
+  public static Automaton minimizeSimple(Automaton a) {
+    Set<Integer> initialSet = new HashSet<Integer>();
+    a = determinizeSimple(Operations.reverse(a, initialSet), initialSet);
+    initialSet.clear();
+    a = determinizeSimple(Operations.reverse(a, initialSet), initialSet);
+    return a;
   }
   
   /**
    * Simple, original brics implementation of determinize()
    */
-  public static void determinizeSimple(Automaton a) {
-    if (a.deterministic || a.isSingleton())
-      return;
-    Set<State> initialset = new HashSet<>();
-    initialset.add(a.initial);
-    determinizeSimple(a, initialset);
+  public static Automaton determinizeSimple(Automaton a) {
+    Set<Integer> initialset = new HashSet<>();
+    initialset.add(0);
+    return determinizeSimple(a, initialset);
   }
-  
+
   /** 
    * Simple, original brics implementation of determinize()
    * Determinizes the given automaton using the given set of initial states. 
    */
-  public static void determinizeSimple(Automaton a, Set<State> initialset) {
+  public static Automaton determinizeSimple(Automaton a, Set<Integer> initialset) {
+    if (a.getNumStates() == 0) {
+      return a;
+    }
     int[] points = a.getStartPoints();
     // subset construction
-    Map<Set<State>, Set<State>> sets = new HashMap<>();
-    LinkedList<Set<State>> worklist = new LinkedList<>();
-    Map<Set<State>, State> newstate = new HashMap<>();
+    Map<Set<Integer>, Set<Integer>> sets = new HashMap<>();
+    LinkedList<Set<Integer>> worklist = new LinkedList<>();
+    Map<Set<Integer>, Integer> newstate = new HashMap<>();
     sets.put(initialset, initialset);
     worklist.add(initialset);
-    a.initial = new State();
-    newstate.put(initialset, a.initial);
+    Automaton.Builder result = new Automaton.Builder();
+    result.createState();
+    newstate.put(initialset, 0);
+    Transition t = new Transition();
     while (worklist.size() > 0) {
-      Set<State> s = worklist.removeFirst();
-      State r = newstate.get(s);
-      for (State q : s)
-        if (q.accept) {
-          r.accept = true;
+      Set<Integer> s = worklist.removeFirst();
+      int r = newstate.get(s);
+      for (int q : s) {
+        if (a.isAccept(q)) {
+          result.setAccept(r, true);
           break;
         }
+      }
       for (int n = 0; n < points.length; n++) {
-        Set<State> p = new HashSet<>();
-        for (State q : s)
-          for (Transition t : q.getTransitions())
-            if (t.min <= points[n] && points[n] <= t.max)
-              p.add(t.to);
+        Set<Integer> p = new HashSet<>();
+        for (int q : s) {
+          int count = a.initTransition(q, t);
+          for(int i=0;i<count;i++) {
+            a.getNextTransition(t);
+            if (t.min <= points[n] && points[n] <= t.max) {
+              p.add(t.dest);
+            }
+          }
+        }
+
         if (!sets.containsKey(p)) {
           sets.put(p, p);
           worklist.add(p);
-          newstate.put(p, new State());
+          newstate.put(p, result.createState());
         }
-        State q = newstate.get(p);
+        int q = newstate.get(p);
         int min = points[n];
         int max;
-        if (n + 1 < points.length)
+        if (n + 1 < points.length) {
           max = points[n + 1] - 1;
-        else
+        } else {
           max = Character.MAX_CODE_POINT;
-        r.addTransition(new Transition(min, max, q));
+        }
+        result.addTransition(r, q, min, max);
       }
     }
-    a.deterministic = true;
-    a.clearNumberedStates();
-    a.removeDeadTransitions();
+
+    return Operations.removeDeadStates(result.finish());
   }
 
   /**
@@ -403,11 +401,7 @@
    */
   public static Set<IntsRef> getFiniteStringsRecursive(Automaton a, int limit) {
     HashSet<IntsRef> strings = new HashSet<>();
-    if (a.isSingleton()) {
-      if (limit > 0) {
-        strings.add(Util.toUTF32(a.singleton, new IntsRef()));
-      }
-    } else if (!getFiniteStrings(a.initial, new HashSet<State>(), strings, new IntsRef(), limit)) {
+    if (!getFiniteStrings(a, 0, new HashSet<Integer>(), strings, new IntsRef(), limit)) {
       return strings;
     }
     return strings;
@@ -418,24 +412,27 @@
    * false if more than <code>limit</code> strings are found. 
    * <code>limit</code>&lt;0 means "infinite".
    */
-  private static boolean getFiniteStrings(State s, HashSet<State> pathstates, 
+  private static boolean getFiniteStrings(Automaton a, int s, HashSet<Integer> pathstates, 
       HashSet<IntsRef> strings, IntsRef path, int limit) {
     pathstates.add(s);
-    for (Transition t : s.getTransitions()) {
-      if (pathstates.contains(t.to)) {
+    Transition t = new Transition();
+    int count = a.initTransition(s, t);
+    for (int i=0;i<count;i++) {
+      a.getNextTransition(t);
+      if (pathstates.contains(t.dest)) {
         return false;
       }
       for (int n = t.min; n <= t.max; n++) {
         path.grow(path.length+1);
         path.ints[path.length] = n;
         path.length++;
-        if (t.to.accept) {
+        if (a.isAccept(t.dest)) {
           strings.add(IntsRef.deepCopyOf(path));
           if (limit >= 0 && strings.size() > limit) {
             return false;
           }
         }
-        if (!getFiniteStrings(t.to, pathstates, strings, path, limit)) {
+        if (!getFiniteStrings(a, t.dest, pathstates, strings, path, limit)) {
           return false;
         }
         path.length--;
@@ -452,8 +449,10 @@
    * this is only used to test the correctness of our faster implementation.
    */
   public static boolean isFiniteSlow(Automaton a) {
-    if (a.isSingleton()) return true;
-    return isFiniteSlow(a.initial, new HashSet<State>());
+    if (a.getNumStates() == 0) {
+      return true;
+    }
+    return isFiniteSlow(a, 0, new HashSet<Integer>());
   }
   
   /**
@@ -462,22 +461,48 @@
    */
   // TODO: not great that this is recursive... in theory a
   // large automata could exceed java's stack
-  private static boolean isFiniteSlow(State s, HashSet<State> path) {
+  private static boolean isFiniteSlow(Automaton a, int s, HashSet<Integer> path) {
     path.add(s);
-    for (Transition t : s.getTransitions())
-      if (path.contains(t.to) || !isFiniteSlow(t.to, path)) return false;
+    Transition t = new Transition();
+    int count = a.initTransition(s, t);
+    for (int i=0;i<count;i++) {
+      a.getNextTransition(t);
+      if (path.contains(t.dest) || !isFiniteSlow(a, t.dest, path)) {
+        return false;
+      }
+    }
     path.remove(s);
     return true;
   }
   
-  
   /**
    * Checks that an automaton has no detached states that are unreachable
    * from the initial state.
    */
   public static void assertNoDetachedStates(Automaton a) {
-    int numStates = a.getNumberOfStates();
-    a.clearNumberedStates(); // force recomputation of cached numbered states
-    assert numStates == a.getNumberOfStates() : "automaton has " + (numStates - a.getNumberOfStates()) + " detached states";
+    Automaton a2 = Operations.removeDeadStates(a);
+    assert a.getNumStates() == a2.getNumStates() : "automaton has " + (a.getNumStates() - a2.getNumStates()) + " detached states";
   }
+
+  /** Returns true if the automaton is deterministic. */
+  public static boolean isDeterministicSlow(Automaton a) {
+    Transition t = new Transition();
+    int numStates = a.getNumStates();
+    for(int s=0;s<numStates;s++) {
+      int count = a.initTransition(s, t);
+      int lastMax = -1;
+      for(int i=0;i<count;i++) {
+        a.getNextTransition(t);
+        if (t.min <= lastMax) {
+          assert a.isDeterministic() == false;
+          return false;
+        }
+        lastMax = t.max;
+      }
+    }
+
+    assert a.isDeterministic() == true;
+    return true;
+  }
+  
 }
diff --git a/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java b/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
index fae81bb..1ef4d0a 100644
--- a/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
+++ b/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
@@ -17,6 +17,12 @@
 
 package org.apache.solr.parser;
 
+import java.io.StringReader;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.util.TokenFilterFactory;
 import org.apache.lucene.index.Term;
@@ -34,10 +40,9 @@
 import org.apache.lucene.util.QueryBuilder;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.Version;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.BasicAutomata;
-import org.apache.lucene.util.automaton.BasicOperations;
-import org.apache.lucene.util.automaton.SpecialOperations;
 import org.apache.solr.analysis.ReversedWildcardFilterFactory;
 import org.apache.solr.analysis.TokenizerChain;
 import org.apache.solr.common.SolrException;
@@ -49,12 +54,6 @@
 import org.apache.solr.search.QParser;
 import org.apache.solr.search.SyntaxError;
 
-import java.io.StringReader;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 /** This class is overridden by QueryParser in QueryParser.jj
  * and acts to separate the majority of the Java code from the .jj grammar file. 
  */
@@ -780,16 +779,16 @@
       Automaton automaton = WildcardQuery.toAutomaton(term);
       // TODO: we should likely use the automaton to calculate shouldReverse, too.
       if (factory.shouldReverse(termStr)) {
-        automaton = BasicOperations.concatenate(automaton, BasicAutomata.makeChar(factory.getMarkerChar()));
-        SpecialOperations.reverse(automaton);
+        automaton = Operations.concatenate(automaton, Automata.makeChar(factory.getMarkerChar()));
+        automaton = Operations.reverse(automaton);
       } else {
         // reverse wildcardfilter is active: remove false positives
         // fsa representing false positives (markerChar*)
-        Automaton falsePositives = BasicOperations.concatenate(
-            BasicAutomata.makeChar(factory.getMarkerChar()),
-            BasicAutomata.makeAnyString());
+        Automaton falsePositives = Operations.concatenate(
+            Automata.makeChar(factory.getMarkerChar()),
+            Automata.makeAnyString());
         // subtract these away
-        automaton = BasicOperations.minus(automaton, falsePositives);
+        automaton = Operations.minus(automaton, falsePositives);
       }
       return new AutomatonQuery(term, automaton) {
         // override toString so its completely transparent
diff --git a/solr/core/src/java/org/apache/solr/search/SolrQueryParser.java b/solr/core/src/java/org/apache/solr/search/SolrQueryParser.java
index f28262f..21fbf5e 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrQueryParser.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrQueryParser.java
@@ -25,10 +25,7 @@
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.*;
 import org.apache.lucene.util.ToStringUtils;
-import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.BasicAutomata;
-import org.apache.lucene.util.automaton.BasicOperations;
-import org.apache.lucene.util.automaton.SpecialOperations;
+import org.apache.lucene.util.automaton.Automata;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.solr.analysis.ReversedWildcardFilterFactory;
 import org.apache.solr.analysis.TokenizerChain;
diff --git a/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java b/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java
index f367e11..2ce6a23 100644
--- a/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java
+++ b/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java
@@ -19,7 +19,6 @@
 
 import java.io.IOException;
 import java.lang.reflect.Field;
-
 import java.util.HashMap;
 import java.util.Map;
 
@@ -27,8 +26,8 @@
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.search.AutomatonQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.SpecialOperations;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.IndexSchema;
@@ -158,14 +157,11 @@
   /** fragile assert: depends on our implementation, but cleanest way to check for now */ 
   private boolean wasReversed(SolrQueryParser qp, String query) throws Exception {
     Query q = qp.parse(query);
-    if (!(q instanceof AutomatonQuery))
+    if (!(q instanceof AutomatonQuery)) {
       return false;
-    // this is a hack to get the protected Automaton field in AutomatonQuery,
-    // may break in later lucene versions - we have no getter... for good reasons.
-    final Field automatonField = AutomatonQuery.class.getDeclaredField("automaton");
-    automatonField.setAccessible(true);
-    Automaton automaton = (Automaton) automatonField.get(q);
-    String prefix = SpecialOperations.getCommonPrefix(automaton);
+    }
+    Automaton automaton = ((AutomatonQuery) q).getAutomaton();
+    String prefix = Operations.getCommonPrefix(Operations.determinize(automaton));
     return prefix.length() > 0 && prefix.charAt(0) == '\u0001';
   }
 
diff --git a/solr/test-framework/src/java/org/apache/solr/analysis/MockTokenFilterFactory.java b/solr/test-framework/src/java/org/apache/solr/analysis/MockTokenFilterFactory.java
index f7a5183..56d7c57 100644
--- a/solr/test-framework/src/java/org/apache/solr/analysis/MockTokenFilterFactory.java
+++ b/solr/test-framework/src/java/org/apache/solr/analysis/MockTokenFilterFactory.java
@@ -62,4 +62,4 @@
   public MockTokenFilter create(TokenStream stream) {
     return new MockTokenFilter(stream, filter);
   }
-}
\ No newline at end of file
+}