You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2021/08/23 15:12:28 UTC

[GitHub] [lucene] mikemccand commented on a change in pull request #225: LUCENE-10010 Introduce NFARunAutomaton to run NFA directly

mikemccand commented on a change in pull request #225:
URL: https://github.com/apache/lucene/pull/225#discussion_r694036897



##########
File path: lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
##########
@@ -261,6 +266,21 @@ public CompiledAutomaton(
     sinkState = findSinkState(this.automaton);
   }
 
+  public CompiledAutomaton(Automaton automaton, boolean isNFA) {

Review comment:
       Maybe we could introduce an effectively boolean valued `enum` so that caller would have to use e.g. `AutomatonType.NFA|DFA` to make it clearer?

##########
File path: lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java
##########
@@ -100,8 +100,12 @@ public AutomatonQuery(
     this.term = term;
     this.automaton = automaton;
     this.automatonIsBinary = isBinary;
-    // TODO: we could take isFinite too, to save a bit of CPU in CompiledAutomaton ctor?:
-    this.compiled = new CompiledAutomaton(automaton, null, true, determinizeWorkLimit, isBinary);
+    if (determinizeWorkLimit == 0) {

Review comment:
       Hmm need to update javadoc above to say that passing `determinizeWorkLimit==0` means to use NFA instead?  Or rather it means "determinize on demand, state by state, as terms in the index require"?

##########
File path: lucene/core/src/java/org/apache/lucene/util/automaton/Stepable.java
##########
@@ -0,0 +1,13 @@
+package org.apache.lucene.util.automaton;

Review comment:
       Add copyright header?

##########
File path: lucene/core/src/test/org/apache/lucene/util/automaton/TestNFARunAutomaton.java
##########
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.automaton;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.codecs.lucene90.Lucene90Codec;
+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.Term;
+import org.apache.lucene.search.AutomatonQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.ToStringUtils;
+
+public class TestNFARunAutomaton extends LuceneTestCase {
+
+  private static final String FIELD = "field";
+
+  public void testWithRandomAutomaton() {
+    for (int i = 0; i < 100; i++) {
+      RegExp regExp = null;
+      while (regExp == null) {
+        try {
+          regExp = new RegExp(AutomatonTestUtil.randomRegexp(random()));
+        } catch (IllegalArgumentException e) {
+          ignoreException(e);
+        }
+      }
+      Automaton dfa = regExp.toDFA();
+      NFARunAutomaton candidate = new NFARunAutomaton(regExp.toNFA());

Review comment:
       Another maybe powerful way to test NFA behavior would be to create any random DFA (e.g. make random set of strings and call that Daciuk/Mihov builder, or random RegExp.toDFA() like here, or maybe even randomly construct something state by state and transition by transition), then create a new test-only method that converts any DFA back into an NFA by randomly picking a DFA state and duplicating it (preserve all incoming and leaving transitions), or maybe by generating N strings accepted by the DFA and unioning them back into it.  Do that N times so N states get duplicated.  This should not alter the language accepted by the automaton, but should make it very "N".
   
   Finally, from the DFA, randomly enumerate strings it accepts, and then assert the `NFARunAutomaton` also accepts them.  And, sometimes randomly generate random strings that are not accepted by the DFA, and confirm the NFA also does not accept them.

##########
File path: lucene/core/src/java/org/apache/lucene/util/automaton/NFARunAutomaton.java
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.automaton;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.hppc.BitMixer;
+
+/**
+ * A RunAutomaton that does not require DFA, it will determinize and memorize the generated DFA
+ * state along with the run
+ *
+ * <p>implemented based on: https://swtch.com/~rsc/regexp/regexp1.html
+ */
+public class NFARunAutomaton {
+
+  /** state ordinal of "no such state" */
+  public static final int MISSING = -1;
+
+  private static final int NOT_COMPUTED = -2;
+
+  private final Automaton automaton;
+  private final int[] points;
+  private final Map<DState, Integer> dStateToOrd = new HashMap<>(); // could init lazily?
+  private DState[] dStates;
+  private final int alphabetSize;
+
+  /**
+   * Constructor, assuming alphabet size is the whole codepoint space
+   *
+   * @param automaton incoming automaton, should be NFA, for DFA please use {@link RunAutomaton} for
+   *     better efficiency
+   */
+  public NFARunAutomaton(Automaton automaton) {
+    this(automaton, Character.MAX_CODE_POINT);
+  }
+
+  /**
+   * Constructor
+   *
+   * @param automaton incoming automaton, should be NFA, for DFA please use {@link RunAutomaton} *
+   *     for better efficiency
+   * @param alphabetSize alphabet size
+   */
+  public NFARunAutomaton(Automaton automaton, int alphabetSize) {
+    this.automaton = automaton;
+    points = automaton.getStartPoints();
+    this.alphabetSize = alphabetSize;
+    dStates = new DState[10];
+    findDState(new DState(new int[] {0}));
+  }
+
+  /**
+   * For a given state and an incoming character (codepoint), return the next state
+   *
+   * @param state incoming state, should either be 0 or some state that is returned previously by
+   *     this function
+   * @param c codepoint
+   * @return the next state or {@link #MISSING} if the transition doesn't exist
+   */
+  public int step(int state, int c) {
+    assert dStates[state] != null;
+    return step(dStates[state], c);
+  }
+
+  /**
+   * Run through a given codepoint array, return accepted or not, should only be used in test
+   *
+   * @param s String represented by an int array
+   * @return accept or not
+   */
+  boolean run(int[] s) {
+    int p = 0;
+    for (int c : s) {
+      p = step(p, c);
+      if (p == MISSING) return false;
+    }
+    return dStates[p].isAccept;
+  }
+
+  /**
+   * From an existing DFA state, step to next DFA state given character c if the transition is
+   * previously tried then this operation will just use the cached result, otherwise it will call
+   * {@link #step(int[], int)} to get the next state and cache the result
+   */
+  private int step(DState dState, int c) {
+    int charClass = getCharClass(c);
+    if (dState.nextState(charClass) == NOT_COMPUTED) {
+      // the next dfa state has not been computed yet
+      dState.setNextState(charClass, findDState(step(dState.nfaStates, c)));
+    }
+    return dState.nextState(charClass);
+  }
+
+  /**
+   * given a list of NFA states and a character c, compute the output list of NFA state which is
+   * wrapped as a DFA state
+   */
+  private DState step(int[] nfaStates, int c) {
+    Transition transition = new Transition();
+    StateSet stateSet = new StateSet(5); // fork IntHashSet from hppc instead?
+    int numTransitions;
+    for (int nfaState : nfaStates) {
+      numTransitions = automaton.initTransition(nfaState, transition);
+      for (int i = 0; i < numTransitions; i++) {
+        automaton.getNextTransition(transition);
+        if (transition.min <= c && transition.max >= c) {
+          stateSet.incr(transition.dest);
+        }
+      }
+    }
+    if (stateSet.size() == 0) {
+      return null;
+    }
+    return new DState(stateSet.getArray());
+  }
+
+  /**
+   * return the ordinal of given DFA state, generate a new ordinal if the given DFA state is a new
+   * one
+   */
+  private int findDState(DState dState) {
+    if (dState == null) {
+      return MISSING;
+    }
+    int ord = dStateToOrd.getOrDefault(dState, -1);
+    if (ord >= 0) {
+      return ord;
+    }
+    ord = dStateToOrd.size();
+    dStateToOrd.put(dState, ord);
+    assert ord >= dStates.length || dStates[ord] == null;
+    if (ord >= dStates.length) {
+      dStates = ArrayUtil.grow(dStates, ord + 1);
+    }
+    dStates[ord] = dState;
+    return ord;
+  }
+
+  /** Gets character class of given codepoint */
+  final int getCharClass(int c) {
+    assert c < alphabetSize;
+    // binary search
+    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;
+  }
+
+  private class DState {
+    private final int[] nfaStates;
+    private int[] transitions;
+    private final int hashCode;
+    private final boolean isAccept;
+
+    private DState(int[] nfaStates) {
+      assert nfaStates != null && nfaStates.length > 0;
+      this.nfaStates = nfaStates;
+      int hashCode = nfaStates.length;
+      boolean isAccept = false;
+      for (int s : nfaStates) {
+        hashCode += BitMixer.mix(s);
+        if (automaton.isAccept(s)) {
+          isAccept = true;
+        }
+      }
+      this.isAccept = isAccept;
+      this.hashCode = hashCode;
+    }
+
+    private int nextState(int charClass) {
+      initTransitions();
+      assert charClass < transitions.length;
+      return transitions[charClass];
+    }
+
+    private void setNextState(int charClass, int nextState) {
+      initTransitions();
+      assert charClass < transitions.length;
+      transitions[charClass] = nextState;
+    }
+
+    private void initTransitions() {
+      if (transitions == null) {
+        transitions = new int[points.length];

Review comment:
       Yeah, I think it's fair.  You are basically building up the same table, node by node lazily on-demand, that `RunAutomaton` on a DFA creates entirely up front.

##########
File path: lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
##########
@@ -261,6 +266,21 @@ public CompiledAutomaton(
     sinkState = findSinkState(this.automaton);
   }
 
+  public CompiledAutomaton(Automaton automaton, boolean isNFA) {
+    // nocommit: the parameter "isNFA" makes no sense, is only used to distinguish the ctor
+    assert automaton.isDeterministic() == false;

Review comment:
       Maybe make this a real `if`?

##########
File path: lucene/core/src/test/org/apache/lucene/util/automaton/TestNFARunAutomaton.java
##########
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.automaton;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.codecs.lucene90.Lucene90Codec;
+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.Term;
+import org.apache.lucene.search.AutomatonQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.ToStringUtils;
+
+public class TestNFARunAutomaton extends LuceneTestCase {
+
+  private static final String FIELD = "field";
+
+  public void testWithRandomAutomaton() {
+    for (int i = 0; i < 100; i++) {
+      RegExp regExp = null;
+      while (regExp == null) {
+        try {
+          regExp = new RegExp(AutomatonTestUtil.randomRegexp(random()));
+        } catch (IllegalArgumentException e) {
+          ignoreException(e);
+        }
+      }
+      Automaton dfa = regExp.toDFA();
+      NFARunAutomaton candidate = new NFARunAutomaton(regExp.toNFA());
+      AutomatonTestUtil.RandomAcceptedStrings randomStringGen =
+          new AutomatonTestUtil.RandomAcceptedStrings(dfa);
+
+      for (int round = 0; round < 20; round++) {
+        // test order of accepted strings and random (likely rejected) strings alternatively to make
+        // sure caching system works correctly
+        if (random().nextBoolean()) {
+          testAcceptedString(regExp, randomStringGen, candidate, 10);
+          testRandomString(regExp, dfa, candidate, 10);
+        } else {
+          testRandomString(regExp, dfa, candidate, 10);
+          testAcceptedString(regExp, randomStringGen, candidate, 10);
+        }
+      }
+    }
+  }
+
+  public void testWithRandomAutomatonQuery() throws IOException {
+    final int docNum = 50;
+    final int automatonNum = 50;
+    Directory directory = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig();
+    iwc.setCodec(new Lucene90Codec());
+    IndexWriter writer = new IndexWriter(directory, iwc);
+
+    Set<String> vocab = new HashSet<>();
+    Set<String> perLoopReuse = new HashSet<>();
+    for (int i = 0; i < docNum; i++) {
+      perLoopReuse.clear();
+      int termNum = random().nextInt(20) + 30;
+      while (perLoopReuse.size() < termNum) {
+        String randomString;
+        while ((randomString = TestUtil.randomUnicodeString(random())).length() == 0);
+        perLoopReuse.add(randomString);
+        vocab.add(randomString);
+      }
+      Document document = new Document();
+      document.add(newTextField(FIELD,
+                   perLoopReuse.stream().reduce("", (s1, s2) -> s1 + " " + s2),
+                   Field.Store.NO));
+      writer.addDocument(document);
+    }
+    writer.commit();
+    IndexReader reader = DirectoryReader.open(writer);
+    IndexSearcher searcher = new IndexSearcher(reader);
+
+    Set<String> foreignVocab = new HashSet<>();
+    while (foreignVocab.size() < vocab.size()) {
+      String randomString;
+      while ((randomString = TestUtil.randomUnicodeString(random())).length() == 0);
+      foreignVocab.add(randomString);
+    }
+
+    ArrayList<String> vocabList = new ArrayList<>(vocab);
+    ArrayList<String> foreignVocabList = new ArrayList<>(foreignVocab);
+
+    for (int i = 0; i < automatonNum; i++) {
+      perLoopReuse.clear();
+      int termNum = random().nextInt(40) + 30;
+      while (perLoopReuse.size() < termNum) {
+        if (random().nextBoolean()) {
+          perLoopReuse.add(vocabList.get(random().nextInt(vocabList.size())));
+        } else {
+          perLoopReuse.add(foreignVocabList.get(random().nextInt(foreignVocabList.size())));
+        }
+      }
+      Automaton a = null;
+      for (String term: perLoopReuse) {
+        if (a == null) {
+          a = Automata.makeString(term);
+        } else {
+          a = Operations.union(a, Automata.makeString(term));
+        }
+      }
+      if (a.isDeterministic()) {
+        i--;
+        continue;
+      }
+      Query dfaQuery = new AutomatonQuery(new Term(FIELD), a);
+      Query nfaQuery = new AutomatonQuery(new Term(FIELD), a, 0);

Review comment:
       It's very cool you are able to completely reuse `AutomatonQuery` to run either an NFA or DFA by using `determinizeWorkLimit = 0` to mean "create an NFA", since you made the two interfaces.  Versus forking and then requiring the user make an `NFAAutomatonQuery`.
   
   But I think it's a little dangerous to take such a low-level approach?  (Your `// nocommit` above).  Because that approach means anyone who calls determinize with a `0` work limit will quietly get an NFA back, not just `AutomatonQuery` who knows how to handle the NFA properly.  I.e. other places that call `determinize` expect to always get back a deterministic automaton?  Maybe, instead, we could instead just change `AutomatonQuery` to take an optional `determinizeUpFront` boolean, which would default to `true` (preserving current behavior)?
   
   Or, maybe keep the `determinizeWorkLimit==0` to mean "use NFA", but move that `if` up from way down in `determinize` to higher up in `AutomatonQuery`?

##########
File path: lucene/core/src/java/org/apache/lucene/util/automaton/TransitionAccessor.java
##########
@@ -0,0 +1,15 @@
+package org.apache.lucene.util.automaton;

Review comment:
       Copyright header here too?

##########
File path: lucene/core/src/java/org/apache/lucene/util/automaton/NFARunAutomaton.java
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.automaton;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.hppc.BitMixer;
+
+/**
+ * A RunAutomaton that does not require DFA, it will determinize and memorize the generated DFA
+ * state along with the run
+ *
+ * <p>implemented based on: https://swtch.com/~rsc/regexp/regexp1.html
+ */
+public class NFARunAutomaton {
+
+  /** state ordinal of "no such state" */
+  public static final int MISSING = -1;
+
+  private static final int NOT_COMPUTED = -2;
+
+  private final Automaton automaton;
+  private final int[] points;
+  private final Map<DState, Integer> dStateToOrd = new HashMap<>(); // could init lazily?
+  private DState[] dStates;
+  private final int alphabetSize;
+
+  /**
+   * Constructor, assuming alphabet size is the whole codepoint space
+   *
+   * @param automaton incoming automaton, should be NFA, for DFA please use {@link RunAutomaton} for
+   *     better efficiency
+   */
+  public NFARunAutomaton(Automaton automaton) {
+    this(automaton, Character.MAX_CODE_POINT);
+  }
+
+  /**
+   * Constructor
+   *
+   * @param automaton incoming automaton, should be NFA, for DFA please use {@link RunAutomaton} *
+   *     for better efficiency
+   * @param alphabetSize alphabet size
+   */
+  public NFARunAutomaton(Automaton automaton, int alphabetSize) {
+    this.automaton = automaton;
+    points = automaton.getStartPoints();
+    this.alphabetSize = alphabetSize;
+    dStates = new DState[10];
+    findDState(new DState(new int[] {0}));
+  }
+
+  /**
+   * For a given state and an incoming character (codepoint), return the next state
+   *
+   * @param state incoming state, should either be 0 or some state that is returned previously by
+   *     this function
+   * @param c codepoint
+   * @return the next state or {@link #MISSING} if the transition doesn't exist
+   */
+  public int step(int state, int c) {
+    assert dStates[state] != null;
+    return step(dStates[state], c);
+  }
+
+  /**
+   * Run through a given codepoint array, return accepted or not, should only be used in test
+   *
+   * @param s String represented by an int array
+   * @return accept or not
+   */
+  boolean run(int[] s) {

Review comment:
       OK, I agree.

##########
File path: lucene/core/src/java/org/apache/lucene/util/automaton/NFARunAutomaton.java
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.automaton;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.hppc.BitMixer;
+
+/**
+ * A RunAutomaton that does not require DFA, it will determinize and memorize the generated DFA
+ * state along with the run
+ *
+ * <p>implemented based on: https://swtch.com/~rsc/regexp/regexp1.html
+ */
+public class NFARunAutomaton {
+
+  /** state ordinal of "no such state" */
+  public static final int MISSING = -1;
+
+  private static final int NOT_COMPUTED = -2;
+
+  private final Automaton automaton;
+  private final int[] points;
+  private final Map<DState, Integer> dStateToOrd = new HashMap<>(); // could init lazily?
+  private DState[] dStates;
+  private final int alphabetSize;
+
+  /**
+   * Constructor, assuming alphabet size is the whole codepoint space
+   *
+   * @param automaton incoming automaton, should be NFA, for DFA please use {@link RunAutomaton} for
+   *     better efficiency
+   */
+  public NFARunAutomaton(Automaton automaton) {
+    this(automaton, Character.MAX_CODE_POINT);
+  }
+
+  /**
+   * Constructor
+   *
+   * @param automaton incoming automaton, should be NFA, for DFA please use {@link RunAutomaton} *
+   *     for better efficiency
+   * @param alphabetSize alphabet size
+   */
+  public NFARunAutomaton(Automaton automaton, int alphabetSize) {
+    this.automaton = automaton;
+    points = automaton.getStartPoints();
+    this.alphabetSize = alphabetSize;
+    dStates = new DState[10];
+    findDState(new DState(new int[] {0}));
+  }
+
+  /**
+   * For a given state and an incoming character (codepoint), return the next state
+   *
+   * @param state incoming state, should either be 0 or some state that is returned previously by
+   *     this function
+   * @param c codepoint
+   * @return the next state or {@link #MISSING} if the transition doesn't exist
+   */
+  public int step(int state, int c) {
+    assert dStates[state] != null;
+    return step(dStates[state], c);
+  }
+
+  /**
+   * Run through a given codepoint array, return accepted or not, should only be used in test
+   *
+   * @param s String represented by an int array
+   * @return accept or not
+   */
+  boolean run(int[] s) {
+    int p = 0;
+    for (int c : s) {
+      p = step(p, c);
+      if (p == MISSING) return false;
+    }
+    return dStates[p].isAccept;
+  }
+
+  /**
+   * From an existing DFA state, step to next DFA state given character c if the transition is
+   * previously tried then this operation will just use the cached result, otherwise it will call
+   * {@link #step(int[], int)} to get the next state and cache the result
+   */
+  private int step(DState dState, int c) {
+    int charClass = getCharClass(c);
+    if (dState.nextState(charClass) == NOT_COMPUTED) {
+      // the next dfa state has not been computed yet
+      dState.setNextState(charClass, findDState(step(dState.nfaStates, c)));
+    }
+    return dState.nextState(charClass);
+  }
+
+  /**
+   * given a list of NFA states and a character c, compute the output list of NFA state which is
+   * wrapped as a DFA state
+   */
+  private DState step(int[] nfaStates, int c) {
+    Transition transition = new Transition();
+    StateSet stateSet = new StateSet(5); // fork IntHashSet from hppc instead?
+    int numTransitions;
+    for (int nfaState : nfaStates) {
+      numTransitions = automaton.initTransition(nfaState, transition);
+      for (int i = 0; i < numTransitions; i++) {
+        automaton.getNextTransition(transition);
+        if (transition.min <= c && transition.max >= c) {
+          stateSet.incr(transition.dest);
+        }
+      }
+    }
+    if (stateSet.size() == 0) {
+      return null;
+    }
+    return new DState(stateSet.getArray());
+  }
+
+  /**
+   * return the ordinal of given DFA state, generate a new ordinal if the given DFA state is a new
+   * one
+   */
+  private int findDState(DState dState) {
+    if (dState == null) {
+      return MISSING;
+    }
+    int ord = dStateToOrd.getOrDefault(dState, -1);
+    if (ord >= 0) {
+      return ord;
+    }
+    ord = dStateToOrd.size();
+    dStateToOrd.put(dState, ord);
+    assert ord >= dStates.length || dStates[ord] == null;
+    if (ord >= dStates.length) {
+      dStates = ArrayUtil.grow(dStates, ord + 1);
+    }
+    dStates[ord] = dState;
+    return ord;
+  }
+
+  /** Gets character class of given codepoint */
+  final int getCharClass(int c) {
+    assert c < alphabetSize;
+    // binary search
+    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;
+  }
+
+  private class DState {

Review comment:
       OK, I think we can clean up the code duplication later -- it's not urgent, certainly not for first prototype PR.  Let's just focus on making NFA work well here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org