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/07/28 15:42:43 UTC

[GitHub] [lucene] mikemccand commented on a change in pull request #157: LUCENE-9963 Fix issue with FlattenGraphFilter throwing exceptions from holes

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



##########
File path: lucene/core/src/java/org/apache/lucene/analysis/AutomatonToTokenStream.java
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.analysis;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+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.Automaton;
+import org.apache.lucene.util.automaton.Operations;
+import org.apache.lucene.util.automaton.Transition;
+
+/** Converts an Automaton into a TokenStream. */
+public class AutomatonToTokenStream {

Review comment:
       Whoa, awesome!  This will be a really helpful infrastructure for future testing!  And of course now I really feel compelled to take a Lev1("lucene") and run it through here and watch what the resulting tokens are!

##########
File path: lucene/core/src/java/org/apache/lucene/analysis/AutomatonToTokenStream.java
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.analysis;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+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.Automaton;
+import org.apache.lucene.util.automaton.Operations;
+import org.apache.lucene.util.automaton.Transition;
+
+/** Converts an Automaton into a TokenStream. */
+public class AutomatonToTokenStream {
+
+  private AutomatonToTokenStream() {}
+
+  /**
+   * converts an automaton into a TokenStream. This is done by first Topo sorting the nodes in the
+   * Automaton. Nodes that have the same distance from the start are grouped together to form the
+   * position nodes for the TokenStream. The resulting TokenStream releases edges from the automaton
+   * as tokens in order from the position nodes. This requires the automaton be a finite DAG.
+   *
+   * @param automaton automaton to convert. Must be a finite DAG.
+   * @return TokenStream representation of automaton.
+   */
+  public static TokenStream toTokenStream(Automaton automaton) {
+    if (Operations.isFinite(automaton) == false) {
+      throw new IllegalArgumentException("Automaton must be finite");
+    }
+
+    List<List<Integer>> positionNodes = new ArrayList<>();
+
+    Transition[][] transitions = automaton.getSortedTransitions();
+
+    int[] indegree = new int[transitions.length];
+
+    for (int i = 0; i < transitions.length; i++) {
+      for (int edge = 0; edge < transitions[i].length; edge++) {
+        indegree[transitions[i][edge].dest] += 1;
+      }
+    }
+    if (indegree[0] != 0) {
+      throw new IllegalArgumentException("Start node has incoming edges, creating cycle");
+    }
+
+    LinkedList<RemapNode> noIncomingEdges = new LinkedList<>();
+    Map<Integer, Integer> idToPos = new HashMap<>();
+    noIncomingEdges.addLast(new RemapNode(0, 0));
+    while (noIncomingEdges.isEmpty() == false) {

Review comment:
       Whoa, is this toposort?  What algorithm :)  Maybe add a comment that that is what this code is doing, with which algo?
   

##########
File path: lucene/core/src/test/org/apache/lucene/analysis/TestAutomatonToTokenStream.java
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.analysis;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.DaciukMihovAutomatonBuilder;
+
+public class TestAutomatonToTokenStream extends BaseTokenStreamTestCase {
+
+  public void testSinglePath() throws IOException {
+    List<BytesRef> acceptStrings = new ArrayList<>();
+    acceptStrings.add(new BytesRef("abc"));
+
+    Automaton flatPathAutomaton = DaciukMihovAutomatonBuilder.build(acceptStrings);
+    TokenStream ts = AutomatonToTokenStream.toTokenStream(flatPathAutomaton);
+    assertTokenStreamContents(
+        ts,
+        new String[] {"a", "b", "c"},
+        new int[] {0, 1, 2},
+        new int[] {1, 2, 3},
+        new int[] {1, 1, 1},
+        new int[] {1, 1, 1},
+        3);
+  }
+
+  public void testParallelPaths() throws IOException {
+    List<BytesRef> acceptStrings = new ArrayList<>();
+    acceptStrings.add(new BytesRef("123"));
+    acceptStrings.add(new BytesRef("abc"));
+
+    Automaton flatPathAutomaton = DaciukMihovAutomatonBuilder.build(acceptStrings);
+    TokenStream ts = AutomatonToTokenStream.toTokenStream(flatPathAutomaton);
+    assertTokenStreamContents(

Review comment:
       So cool!!

##########
File path: lucene/core/src/java/org/apache/lucene/analysis/AutomatonToTokenStream.java
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.analysis;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+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.Automaton;
+import org.apache.lucene.util.automaton.Operations;
+import org.apache.lucene.util.automaton.Transition;
+
+/** Converts an Automaton into a TokenStream. */
+public class AutomatonToTokenStream {
+
+  private AutomatonToTokenStream() {}
+
+  /**
+   * converts an automaton into a TokenStream. This is done by first Topo sorting the nodes in the
+   * Automaton. Nodes that have the same distance from the start are grouped together to form the
+   * position nodes for the TokenStream. The resulting TokenStream releases edges from the automaton
+   * as tokens in order from the position nodes. This requires the automaton be a finite DAG.
+   *
+   * @param automaton automaton to convert. Must be a finite DAG.
+   * @return TokenStream representation of automaton.
+   */
+  public static TokenStream toTokenStream(Automaton automaton) {
+    if (Operations.isFinite(automaton) == false) {
+      throw new IllegalArgumentException("Automaton must be finite");
+    }
+
+    List<List<Integer>> positionNodes = new ArrayList<>();
+
+    Transition[][] transitions = automaton.getSortedTransitions();
+
+    int[] indegree = new int[transitions.length];
+
+    for (int i = 0; i < transitions.length; i++) {
+      for (int edge = 0; edge < transitions[i].length; edge++) {
+        indegree[transitions[i][edge].dest] += 1;
+      }
+    }
+    if (indegree[0] != 0) {
+      throw new IllegalArgumentException("Start node has incoming edges, creating cycle");
+    }
+
+    LinkedList<RemapNode> noIncomingEdges = new LinkedList<>();
+    Map<Integer, Integer> idToPos = new HashMap<>();
+    noIncomingEdges.addLast(new RemapNode(0, 0));
+    while (noIncomingEdges.isEmpty() == false) {
+      RemapNode currState = noIncomingEdges.removeFirst();
+      for (int i = 0; i < transitions[currState.id].length; i++) {
+        indegree[transitions[currState.id][i].dest] -= 1;
+        if (indegree[transitions[currState.id][i].dest] == 0) {
+          noIncomingEdges.addLast(
+              new RemapNode(transitions[currState.id][i].dest, currState.pos + 1));
+        }
+      }
+      if (positionNodes.size() == currState.pos) {
+        List<Integer> posIncs = new ArrayList<>();
+        posIncs.add(currState.id);
+        positionNodes.add(posIncs);
+      } else {
+        positionNodes.get(currState.pos).add(currState.id);
+      }
+      idToPos.put(currState.id, currState.pos);
+    }
+
+    for (int i = 0; i < indegree.length; i++) {
+      if (indegree[i] != 0) {
+        throw new IllegalArgumentException("Cycle found in automaton");
+      }
+    }
+
+    List<List<EdgeToken>> edgesByLayer = new ArrayList<>();
+    for (List<Integer> layer : positionNodes) {
+      List<EdgeToken> edges = new ArrayList<>();
+      for (int state : layer) {
+        for (Transition t : transitions[state]) {
+          // each edge in the token stream can only be on value, though a transition takes a range.
+          for (int val = t.min; val <= t.max; val++) {
+            int destLayer = idToPos.get(t.dest);
+            edges.add(new EdgeToken(destLayer, val));
+            // If there's an intermediate accept state, add an edge to the terminal state.
+            if (automaton.isAccept(t.dest) && destLayer != positionNodes.size() - 1) {
+              edges.add(new EdgeToken(positionNodes.size() - 1, val));

Review comment:
       Ahh, the token graph has only one "accept" state, the highest end position, so we have to move these accept transitions "out" to that end pos.  Sneaky!

##########
File path: lucene/analysis/common/src/java/org/apache/lucene/analysis/core/FlattenGraphFilter.java
##########
@@ -362,6 +415,55 @@ public boolean incrementToken() throws IOException {
     }
   }
 
+  private OutputNode recoverFromHole(InputNode src, int startOffset, int posinc) {
+    // This means the "from" node of this token was never seen as a "to" node,
+    // which should only happen if we just crossed a hole.  This is a challenging
+    // case for us because we normally rely on the full dependencies expressed
+    // by the arcs to assign outgoing node IDs.  It would be better if tokens
+    // were never dropped but instead just marked deleted with a new
+    // TermDeletedAttribute (boolean valued) ... but until that future, we have
+    // a hack here to forcefully jump the output node ID:
+    assert src.outputNode == -1;
+    src.node = inputFrom;
+
+    int outIndex;
+    int previousInputFrom = inputFrom - posinc;
+    if (previousInputFrom >= 0) {
+      InputNode offsetSrc = inputNodes.get(previousInputFrom);
+      /* Select output src node. Need to make sure the new output node isn't placed too far ahead.
+       * If a disconnected node is placed at the end of the output graph that may place it after output nodes that map to input nodes that are after src in the input.
+       * Since it is disconnected there is no path to it, and there could be holes after meaning no paths to following nodes. This "floating" edge will cause problems in FreeBefore.
+       * In the following section make sure the edge connects to something.
+       * Related test testLongHole testAltPathLastStepHoleFollowedByHole, testAltPathFirstStepHole, testShingledGapWithHoles
+       */
+      if (offsetSrc.minToNode < inputFrom) {
+        // There is a possible path to this node.
+        // place this node one position off from the possible path keeping a 1 inc gap.
+        // Can't be larger than 1 inc or risk getting disconnected.
+        outIndex = inputNodes.get(offsetSrc.minToNode).outputNode + 1;
+      } else {
+        // no information about how the current node was previously connected.
+        // Connect it to the end.

Review comment:
       This connects the orphan'd token to the "end" as far as we've consumed/produced so far (`outputNodes.getMaxPos()`) right?  This seems OK (not sure how else we could recover) but I wonder in practice what incoming hole configuration tickles this?

##########
File path: lucene/core/src/java/org/apache/lucene/analysis/AutomatonToTokenStream.java
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.analysis;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+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.Automaton;
+import org.apache.lucene.util.automaton.Operations;
+import org.apache.lucene.util.automaton.Transition;
+
+/** Converts an Automaton into a TokenStream. */
+public class AutomatonToTokenStream {
+
+  private AutomatonToTokenStream() {}
+
+  /**
+   * converts an automaton into a TokenStream. This is done by first Topo sorting the nodes in the
+   * Automaton. Nodes that have the same distance from the start are grouped together to form the
+   * position nodes for the TokenStream. The resulting TokenStream releases edges from the automaton
+   * as tokens in order from the position nodes. This requires the automaton be a finite DAG.
+   *
+   * @param automaton automaton to convert. Must be a finite DAG.
+   * @return TokenStream representation of automaton.
+   */
+  public static TokenStream toTokenStream(Automaton automaton) {
+    if (Operations.isFinite(automaton) == false) {
+      throw new IllegalArgumentException("Automaton must be finite");
+    }
+
+    List<List<Integer>> positionNodes = new ArrayList<>();
+
+    Transition[][] transitions = automaton.getSortedTransitions();
+
+    int[] indegree = new int[transitions.length];
+
+    for (int i = 0; i < transitions.length; i++) {
+      for (int edge = 0; edge < transitions[i].length; edge++) {
+        indegree[transitions[i][edge].dest] += 1;
+      }
+    }
+    if (indegree[0] != 0) {
+      throw new IllegalArgumentException("Start node has incoming edges, creating cycle");
+    }
+
+    LinkedList<RemapNode> noIncomingEdges = new LinkedList<>();
+    Map<Integer, Integer> idToPos = new HashMap<>();
+    noIncomingEdges.addLast(new RemapNode(0, 0));
+    while (noIncomingEdges.isEmpty() == false) {
+      RemapNode currState = noIncomingEdges.removeFirst();
+      for (int i = 0; i < transitions[currState.id].length; i++) {
+        indegree[transitions[currState.id][i].dest] -= 1;
+        if (indegree[transitions[currState.id][i].dest] == 0) {
+          noIncomingEdges.addLast(
+              new RemapNode(transitions[currState.id][i].dest, currState.pos + 1));
+        }
+      }
+      if (positionNodes.size() == currState.pos) {
+        List<Integer> posIncs = new ArrayList<>();
+        posIncs.add(currState.id);
+        positionNodes.add(posIncs);
+      } else {
+        positionNodes.get(currState.pos).add(currState.id);
+      }
+      idToPos.put(currState.id, currState.pos);

Review comment:
       It looks like we "eagerly" place all nodes onto the same position if the topo-sort allows it (i.e. those nodes are at the same "layer" after topo-sort)?  But doesn't that then over-generalize, causing the resulting token graph to accept more strings than the automaton originally did?

##########
File path: lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestFlattenGraphFilter.java
##########
@@ -544,8 +732,288 @@ protected TokenStreamComponents createComponents(String fieldName) {
     }
 
     String text = String.join(" ", stringTokens);
-    checkAnalysisConsistency(random, a, false, text);
+    // FlattenGraphFilter can create inconsistent offsets.
+    // If that is resolved we can check offsets
+    // Until then converting to automaton will pull text through and check if we hit asserts.
+    // checkAnalysisConsistency(random, withFlattenGraph, false, text);
+    TokenStreamToAutomaton tsta = new TokenStreamToAutomaton();
+    TokenStream flattenedTokenStream = withFlattenGraph.tokenStream("field", text);
+    assertFalse(Operations.hasDeadStates(tsta.toAutomaton(flattenedTokenStream)));
+    flattenedTokenStream.close();
+
+    /*
+       CheckGeneralization can get VERY slow as matching holes to tokens or other holes generates a lot of potentially valid paths.
+       Analyzer withoutFlattenGraph =
+           new Analyzer() {
+             @Override
+             protected TokenStreamComponents createComponents(String fieldName) {
+               Tokenizer in = new WhitespaceTokenizer();
+               TokenStream result = new SynonymGraphFilter(in, synMap, true);
+               result = new StopFilter(result, stopWords);
+               return new TokenStreamComponents(in, result);
+             }
+           };
+       checkGeneralization(
+           withFlattenGraph.tokenStream("field", text),
+           withoutFlattenGraph.tokenStream("field", text));
+
+    */
   }
 
+  /*
+   * Make some strings, make an automaton that accepts those strings, convert that automaton into a TokenStream,
+   * flatten it, back to an automaton, and see if the original strings are still accepted.
+   */
+  public void testPathsNotLost() throws IOException {
+    int wordCount = random().nextInt(5) + 5;
+    List<BytesRef> acceptStrings = new LinkedList<>();
+    for (int i = 0; i < wordCount; i++) {
+      int wordLen = random().nextInt(5) + 5;
+      BytesRef ref = new BytesRef(wordLen);
+      ref.length = wordLen;
+      ref.offset = 0;
+      for (int j = 0; j < wordLen; j++) {
+        ref.bytes[j] = (byte) (random().nextInt(5) + 65);
+      }
+      acceptStrings.add(ref);
+    }
+    acceptStrings.sort(Comparator.naturalOrder());
+
+    acceptStrings = acceptStrings.stream().limit(wordCount).collect(Collectors.toList());
+    Automaton nonFlattenedAutomaton = DaciukMihovAutomatonBuilder.build(acceptStrings);
+
+    TokenStream ts = AutomatonToTokenStream.toTokenStream(nonFlattenedAutomaton);
+    TokenStream flattenedTokenStream = new FlattenGraphFilter(ts);
+    TokenStreamToAutomaton tsta = new TokenStreamToAutomaton();
+    Automaton flattenedAutomaton = tsta.toAutomaton(flattenedTokenStream);
+
+    // TokenStreamToAutomaton adds position increment transitions into the automaton.
+    List<BytesRef> acceptStringsWithPosSep = createAcceptStringsWithPosSep(acceptStrings);
+
+    for (BytesRef acceptString : acceptStringsWithPosSep) {
+      assertTrue(
+          "string not accepted " + acceptString.utf8ToString(),
+          recursivelyValidate(acceptString, 0, 0, flattenedAutomaton));
+    }
+  }
+
+  /**
+   * adds POS_SEP bytes between bytes to match TokenStreamToAutomaton format.
+   *
+   * @param acceptStrings Byte refs of accepted strings. Each byte is a transition
+   * @return List of ByteRefs where each byte is separated by a POS_SEP byte.
+   */
+  private List<BytesRef> createAcceptStringsWithPosSep(List<BytesRef> acceptStrings) {
+    List<BytesRef> acceptStringsWithPosSep = new ArrayList<>();
+    for (BytesRef acceptString : acceptStrings) {
+      BytesRef withPosSep = new BytesRef(acceptString.length * 2 - 1);
+      withPosSep.length = acceptString.length * 2 - 1;
+      withPosSep.offset = 0;
+      for (int i = 0; i < acceptString.length; i++) {
+        withPosSep.bytes[i * 2] = acceptString.bytes[i];
+        if (i * 2 + 1 < withPosSep.length) {
+          withPosSep.bytes[i * 2 + 1] = TokenStreamToAutomaton.POS_SEP;
+        }
+      }
+      acceptStringsWithPosSep.add(withPosSep);
+    }
+    return acceptStringsWithPosSep;
+  }
+
+  /**
+   * Checks if acceptString is accepted by the automaton. Automaton may be an NFA.
+   *
+   * @param acceptString String to test
+   * @param acceptStringIndex current index into acceptString, initial value should be 0
+   * @param state state to transition from. initial value should be 0
+   * @param automaton Automaton to test
+   * @return true if acceptString is accepted by the automaton. otherwise false.
+   */
+  public boolean recursivelyValidate(
+      BytesRef acceptString, int acceptStringIndex, int state, Automaton automaton) {
+    if (acceptStringIndex == acceptString.length) {
+      return automaton.isAccept(state);
+    }
+
+    Transition transition = new Transition();
+    automaton.initTransition(state, transition);
+    int numTransitions = automaton.getNumTransitions(state);
+    boolean accept = false;
+    // Automaton can be NFA, so we need to check all matching transitions
+    for (int i = 0; i < numTransitions; i++) {
+      automaton.getTransition(state, i, transition);
+      if (transition.min <= acceptString.bytes[acceptStringIndex]
+          && transition.max >= acceptString.bytes[acceptStringIndex]) {
+        accept =
+            recursivelyValidate(acceptString, acceptStringIndex + 1, transition.dest, automaton);
+      }
+      if (accept == true) {
+        break;
+      }
+    }
+    return accept;
+  }
+
+  /**
+   * This method checks if strings that lead to the accept state of the not flattened TokenStream
+   * also lead to the accept state in the flattened TokenStream. This gets complicated when you
+   * factor in holes. The FlattenGraphFilter will remove alternate paths that are made entirely of
+   * holes. An alternate path of Holes is indistinguishable from a path that just has long
+   * lengths(ex: testStrangelyNumberedNodes). Also alternate paths that end in multiple holes could
+   * be interpreted as sequential holes after the branching has converged during flattening. This
+   * leads to a lot of weird logic about navigating around holes that may compromise the accuracy of
+   * this test.
+   *
+   * @param flattened flattened TokenStream
+   * @param notFlattened not flattened TokenStream
+   * @throws IOException on error creating Automata
+   */
+  /* private void checkGeneralization(TokenStream flattened, TokenStream notFlattened)

Review comment:
       Hmm why did you need to comment out all these methods?




-- 
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