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 2020/02/20 22:19:42 UTC

[GitHub] [lucene-solr] bruno-roustant opened a new pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

bruno-roustant opened a new pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270
 
 
   New version of TermsEnum intersect for UniformSplit. It is 70% more efficient than the previous version for FuzzyQuery.
   
   Compared to BlockTree IntersectTermsEnum:
   - It is still slower for FuzzyQuery (-40%) but it is faster than the previous version (which was -65%).
   - It is roughly same speed for WildcardQuery (-5%).
   - It is slightly faster for PrefixQuery (+5%). Sometimes benchmarks show more improvement (I've seen up to +17% a third of the time).

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r383527391
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java
 ##########
 @@ -18,260 +18,337 @@
 package org.apache.lucene.codecs.uniformsplit;
 
 import java.io.IOException;
-import java.util.Objects;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
  * The "intersect" {@link TermsEnum} response to {@link UniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
  * intersecting the terms with an automaton.
+ * <p>
+ * By design of the UniformSplit block keys, it is less efficient than
+ * {@code org.apache.lucene.codecs.blocktree.IntersectTermsEnum} for {@link org.apache.lucene.search.FuzzyQuery} (-37%).
+ * It is slightly slower for {@link org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for
+ * {@link org.apache.lucene.search.PrefixQuery} (+5%).
+ *
+ * @lucene.experimental
  */
 public class IntersectBlockReader extends BlockReader {
 
-  protected final AutomatonNextTermCalculator nextStringCalculator;
-  protected final ByteRunAutomaton runAutomaton;
-  protected final BytesRef commonSuffixRef; // maybe null
-  protected final BytesRef commonPrefixRef;
-  protected final BytesRef startTerm; // maybe null
+  /**
+   * Block iteration order. Whether to move next block, jump to a block away, or end the iteration.
+   */
+  protected enum BlockIteration {NEXT, SEEK, END}
 
-  /** Set this when our current mode is seeking to this term.  Set to null after. */
-  protected BytesRef seekTerm;
+  /**
+   * Threshold that controls when to attempt to jump to a block away.
+   * <p>
+   * This counter is 0 when entering a block. It is incremented each time a term is rejected by the automaton.
+   * When the counter is greater than or equal to this threshold, then we compute the next term accepted by
+   * the automaton, with {@link AutomatonNextTermCalculator}, and we jump to a block away if the next term
+   * accepted is greater than the immediate next term in the block.
+   * <p>
+   * A low value, for example 1, improves the performance of automatons requiring many jumps, for example
+   * {@link org.apache.lucene.search.FuzzyQuery} and most {@link org.apache.lucene.search.WildcardQuery}.
+   * A higher value improves the performance of automatons with less or no jump, for example
+   * {@link org.apache.lucene.search.PrefixQuery}.
+   * A threshold of 4 seems to be a good balance.
+   */
+  protected final int NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD = 4;
 
-  protected int blockPrefixRunAutomatonState;
-  protected int blockPrefixLen;
+  protected final Automaton automaton;
+  protected final ByteRunAutomaton runAutomaton;
+  protected final boolean finite;
+  protected final BytesRef commonSuffix; // maybe null
+  protected final int minTermLength;
+  protected final AutomatonNextTermCalculator nextStringCalculator;
 
   /**
-   * Number of bytes accepted by the last call to {@link #runAutomatonForState}.
+   * Set this when our current mode is seeking to this term.  Set to null after.
+   */
+  protected BytesRef seekTerm;
+  /**
+   * Number of bytes accepted by the automaton when validating the current term.
+   */
+  protected int numMatchedBytes;
+  /**
+   * Automaton states reached when validating the current term, from 0 to {@link #numMatchedBytes} - 1.
+   */
+  protected int[] states;
+  /**
+   * Block iteration order determined when scanning the terms in the current block.
    */
-  protected int numBytesAccepted;
+  protected BlockIteration blockIteration;
   /**
-   * Whether the current term is beyond the automaton common prefix.
-   * If true this means the enumeration should stop immediately.
+   * Counter of the number of consecutively rejected terms.
+   * Depending on {@link #NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD}, this may trigger a jump to a block away.
    */
-  protected boolean beyondCommonPrefix;
+  protected int numConsecutivelyRejectedTerms;
 
-  public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
-                              IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
-                              FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
+  protected IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
+                                 IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput,
+                                 PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
+                                 BlockDecoder blockDecoder) throws IOException {
     super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
-    this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);
-    Automaton automaton = Objects.requireNonNull(compiled.automaton);
-    this.runAutomaton = Objects.requireNonNull(compiled.runAutomaton);
-    this.commonSuffixRef = compiled.commonSuffixRef; // maybe null
-    this.commonPrefixRef = Operations.getCommonPrefixBytesRef(automaton); // never null
-
-    this.startTerm = startTerm;
-    assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef);
-    // it is thus also true that startTerm >= commonPrefixRef
-
-    this.seekTerm = startTerm != null ? startTerm : commonPrefixRef;
+    checkIntersectAutomatonType(compiled);
+    automaton = compiled.automaton;
+    runAutomaton = compiled.runAutomaton;
+    finite = compiled.finite;
+    commonSuffix = compiled.commonSuffixRef;
+    minTermLength = getMinTermLength();
+    nextStringCalculator = new AutomatonNextTermCalculator(compiled);
+    seekTerm = startTerm;
   }
 
-  @Override
-  public BytesRef next() throws IOException {
-    clearTermState();
-
-    if (blockHeader == null) { // initial state
-      // note: don't call super.seekCeil here; we have our own logic
-
-      // Set the browser position to the block having the seek term.
-      // Even if -1, it's okay since we'll soon call nextKey().
-      long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
-      if (isBeyondLastTerm(seekTerm, blockStartFP)) {
-        return null; // EOF
-      }
-
-      // Starting at this block find and load the next matching block.
-      //   note: Since seekBlock was just called, we actually consider the current block as "next".
-      if (nextBlockMatchingPrefix() == false) { // note: starts at seek'ed block, which may have a match
-        return null; // EOF
-      }
+  protected void checkIntersectAutomatonType(CompiledAutomaton compiled) {
+    if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+      throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
     }
-
-    do {
-
-      // look in the rest of this block.
-      BytesRef term = nextTermInBlockMatching();
-      if (term != null) {
-        return term;
-      }
-
-      // next term dict matching prefix
-    } while (nextBlockMatchingPrefix());
-
-    return null; // EOF
   }
 
   /**
-   * Find the next block that appears to contain terms that could match the automata.
-   * The prefix is the primary clue.  Returns true if at one, or false for no more (EOF).
+   * Computes the minimal length of the terms accepted by the automaton.
+   * This speeds up the term scanning for automatons accepting a finite language.
    */
-  protected boolean nextBlockMatchingPrefix() throws IOException {
-    if (beyondCommonPrefix) {
-      return false; // EOF
+  protected int getMinTermLength() {
+    // Automatons accepting infinite language (e.g. PrefixQuery and WildcardQuery) do not benefit much from
+    // min term length while it takes time to compute it. More precisely, by skipping this computation PrefixQuery
+    // is significantly boosted while WildcardQuery might be slightly degraded on average. This min term length
+    // mainly boosts FuzzyQuery.
+    int commonSuffixLength = commonSuffix == null ? 0 : commonSuffix.length;
+    if (!finite) {
+      return commonSuffixLength;
     }
-
-    IndexDictionary.Browser browser = getOrCreateDictionaryBrowser();
-
-    do {
-
-      // Get next block key (becomes in effect the current blockKey)
-      BytesRef blockKey = browser.nextKey();
-      if (blockKey == null) {
-        return false; // EOF
-      }
-
-      blockPrefixLen = browser.getBlockPrefixLen();
-      blockPrefixRunAutomatonState = runAutomatonForState(blockKey.bytes, blockKey.offset, blockPrefixLen, 0);
-
-      // We may have passed commonPrefix  (a short-circuit optimization).
-      if (isBeyondCommonPrefix(blockKey)) {
-        return false; // EOF
+    int commonPrefixLength = 0;
+    int state = 0;
+    Transition t = null;
+    int[] visited = null;
+    int visitedLength = 0;
+    while (true) {
+      if (runAutomaton.isAccept(state)) {
+        // The common prefix reaches a final state. So common prefix and common suffix overlap.
+        // Min term length is the max between common prefix and common suffix lengths.
+        return Math.max(commonPrefixLength, commonSuffixLength);
       }
-
-      if (blockPrefixRunAutomatonState >= 0) {
-        break; // a match
+      if (automaton.getNumTransitions(state) == 1) {
+        if (t == null) {
+          t = new Transition();
+        }
+        automaton.getTransition(state, 0, t);
+        if (t.min == t.max) {
+          state = t.dest;
+          if (visited == null) {
+            visited = new int[8];
+            visited[visitedLength++] = 0;
+          }
+          if (!contains(visited, visitedLength, state)) {
+            visited = ArrayUtil.grow(visited, ++visitedLength);
+            visited[visitedLength - 1] = state;
+            commonPrefixLength++;
+            continue;
+          }
+        }
       }
+      break;
+    }
+    // Min term length is the sum of common prefix and common suffix lengths.
+    return commonPrefixLength + commonSuffixLength;
+  }
 
-      //
-      // This block doesn't match.
-      //
-
-      seekTerm = null; // we're moving on to another block, and seekTerm is before it.
-
-      // Should we simply get the next key (linear mode) or try to seek?
-      if (nextStringCalculator.isLinearState(blockKey)) {
-        continue;
+  private static boolean contains(int[] ints, int length, int value) {
+    assert length <= ints.length;
+    for (int i = 0; i < length; i++) {
+      if (ints[i] == value) {
+        return true;
       }
+    }
+    return false;
+  }
 
-      // Maybe the next block's key matches?  We have to check this before calling nextStringCalculator.
-      BytesRef peekKey = browser.peekKey();
-      if (peekKey == null) {
-        return false; // EOF
-      }
-      if (runAutomatonForState(peekKey.bytes, peekKey.offset, peekKey.length, 0) >= 0) {
-        continue; // yay; it matched.  Continue to actually advance to it.  This is rare?
+  @Override
+  public BytesRef next() throws IOException {
+    if (blockHeader == null) {
+      if (!seekFirstBlock()) {
+        return null;
       }
-
-      // Seek to a block by calculating the next term to match the automata *following* peekKey.
-      this.seekTerm = nextStringCalculator.nextSeekTerm(browser.peekKey());
-      if (seekTerm == null) {
-        return false; // EOF
+      states = new int[32];
+      blockIteration = BlockIteration.NEXT;
+    }
+    termState = null;
+    do {
+      BytesRef term = nextTermInBlockMatching();
+      if (term != null) {
+        return term;
       }
-      browser.seekBlock(seekTerm);
-      //continue
-
-    } while (true); // while not a match
-
-    // A match!
-
-    //NOTE: we could determine if this automata has a prefix for this specific block (longer than the commonPrefix).
-    //  If we see it, we could set it as the seekTerm and we could also exit the block early if we get past this prefix
-    //  and runAutomatonFromPrefix would start from this prefix.  Smiley tried but benchmarks were not favorable to it.
-
-    initializeHeader(null, browser.getBlockFilePointer());
+    } while (nextBlock());
+    return null;
+  }
 
-    return true;
+  protected boolean seekFirstBlock() throws IOException {
+    seekTerm = nextStringCalculator.nextSeekTerm(seekTerm);
+    if (seekTerm == null) {
+      return false;
+    }
+    long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
+    if (blockStartFP == -1) {
+      blockStartFP = fieldMetadata.getFirstBlockStartFP();
+    } else if (isBeyondLastTerm(seekTerm, blockStartFP)) {
+      return false;
+    }
+    initializeHeader(seekTerm, blockStartFP);
+    return blockHeader != null;
   }
 
   /**
-   * Find the next block line that matches, or null when at end of block.
+   * Finds the next block line that matches (accepted by the automaton), or null when at end of block.
+   *
+   * @return The next term in the current block that is accepted by the automaton; or null if none.
    */
   protected BytesRef nextTermInBlockMatching() throws IOException {
-    do {
-      // if seekTerm is set, then we seek into this block instead of starting with the first blindly.
-      if (seekTerm != null) {
-        assert blockLine == null;
-        boolean moveBeyondIfFound = seekTerm == startTerm; // for startTerm, we want to get the following term
-        SeekStatus seekStatus = seekInBlock(seekTerm);
-        seekTerm = null;// reset.
-        if (seekStatus == SeekStatus.END) {
-          return null;
-        } else if (seekStatus == SeekStatus.FOUND && moveBeyondIfFound) {
-          if (readLineInBlock() == null) {
-            return null;
+    if (seekTerm == null) {
+      if (readLineInBlock() == null) {
+        return null;
+      }
+    } else {
+      SeekStatus seekStatus = seekInBlock(seekTerm);
+      seekTerm = null;
+      if (seekStatus == SeekStatus.END) {
+        return null;
+      }
+      assert numMatchedBytes == 0;
+      assert numConsecutivelyRejectedTerms == 0;
+    }
+    int[] states = this.states;
+    while (true) {
+      TermBytes lineTermBytes = blockLine.getTermBytes();
+      BytesRef lineTerm = lineTermBytes.getTerm();
+      assert lineTerm.offset == 0;
+      byte[] termBytes = lineTerm.bytes;
+      int termLength = lineTerm.length;
+      if (states.length <= termLength) {
+        states = this.states = ArrayUtil.growExact(states, ArrayUtil.oversize(termLength + 1, Byte.BYTES));
+      }
+      // Since terms a delta encoded, we may start the automaton steps from the last state reached by the previous term.
+      int index = Math.min(lineTermBytes.getSuffixOffset(), numMatchedBytes);
+      // Skip this term early if it is shorter than the min term length, or if it does not end with the common suffix
+      // accepted by the automaton.
+      if (termLength >= minTermLength && (commonSuffix == null || endsWithCommonSuffix(termBytes, termLength))) {
+        int state = states[index];
+        while (true) {
+          if (index == termLength) {
+            if (runAutomaton.isAccept(state)) {
+              // The automaton accepts the current term. Record the number of matched bytes and return the term.
+              assert runAutomaton.run(termBytes, 0, termLength);
+              numMatchedBytes = index;
+              if (numConsecutivelyRejectedTerms > 0) {
+                numConsecutivelyRejectedTerms = 0;
+              }
+              assert blockIteration == BlockIteration.NEXT;
+              return lineTerm;
+            }
+            break;
+          }
+          state = runAutomaton.step(state, termBytes[index] & 0xff);
+          if (state == -1) {
+            // The automaton rejects the current term.
+            break;
           }
+          // Record the reached automaton state.
+          states[++index] = state;
         }
-        assert blockLine != null;
-      } else {
-        if (readLineInBlock() == null) {
+      }
+      // The current term is not accepted by the automaton.
+      // Still record the reached automaton state to start the next term steps from there.
+      assert !runAutomaton.run(termBytes, 0, termLength);
+      numMatchedBytes = index;
+      // If the number of consecutively rejected terms reaches the threshold, then determine whether it is worth to
+      // jump to a block away.
+      if (++numConsecutivelyRejectedTerms >= NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD
+          && lineIndexInBlock < blockHeader.getLinesCount() - 1
+          && !nextStringCalculator.isLinearState(lineTerm)) {
+        // Compute the next term accepted by the automaton after the current term.
+        if ((seekTerm = nextStringCalculator.nextSeekTerm(lineTerm)) == null) {
+          blockIteration = BlockIteration.END;
           return null;
         }
-      }
-
-      TermBytes lineTermBytes = blockLine.getTermBytes();
-      BytesRef lineTerm = lineTermBytes.getTerm();
-
-      if (commonSuffixRef == null || StringHelper.endsWith(lineTerm, commonSuffixRef)) {
-        if (runAutomatonFromPrefix(lineTerm)) {
-          return lineTerm;
-        } else if (beyondCommonPrefix) {
+        // It is worth to jump to a block away if the next term accepted is after the next term in the block.
+        // Actually the block away may be the current block, but this is a good heuristic.
+        readLineInBlock();
+        if (seekTerm.compareTo(blockLine.getTermBytes().getTerm()) > 0) {
+          // Stop this block terms a set the iteration order to jump to a block away by seeking seekTerm.
+          blockIteration = BlockIteration.SEEK;
           return null;
         }
+        seekTerm = null;
+        // If it is not worth to jump to a block away, do not attempt anymore for the current block.
 
 Review comment:
   Reword "worth" -> either "worth it" or "worthwhile"

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] bruno-roustant commented on issue #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
bruno-roustant commented on issue #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#issuecomment-590972922
 
 
   I put my comment in the JIRA description.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r383523994
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java
 ##########
 @@ -18,260 +18,337 @@
 package org.apache.lucene.codecs.uniformsplit;
 
 import java.io.IOException;
-import java.util.Objects;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
  * The "intersect" {@link TermsEnum} response to {@link UniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
  * intersecting the terms with an automaton.
+ * <p>
+ * By design of the UniformSplit block keys, it is less efficient than
+ * {@code org.apache.lucene.codecs.blocktree.IntersectTermsEnum} for {@link org.apache.lucene.search.FuzzyQuery} (-37%).
+ * It is slightly slower for {@link org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for
+ * {@link org.apache.lucene.search.PrefixQuery} (+5%).
+ *
+ * @lucene.experimental
  */
 public class IntersectBlockReader extends BlockReader {
 
-  protected final AutomatonNextTermCalculator nextStringCalculator;
-  protected final ByteRunAutomaton runAutomaton;
-  protected final BytesRef commonSuffixRef; // maybe null
-  protected final BytesRef commonPrefixRef;
-  protected final BytesRef startTerm; // maybe null
+  /**
+   * Block iteration order. Whether to move next block, jump to a block away, or end the iteration.
+   */
+  protected enum BlockIteration {NEXT, SEEK, END}
 
-  /** Set this when our current mode is seeking to this term.  Set to null after. */
-  protected BytesRef seekTerm;
+  /**
+   * Threshold that controls when to attempt to jump to a block away.
+   * <p>
+   * This counter is 0 when entering a block. It is incremented each time a term is rejected by the automaton.
+   * When the counter is greater than or equal to this threshold, then we compute the next term accepted by
+   * the automaton, with {@link AutomatonNextTermCalculator}, and we jump to a block away if the next term
+   * accepted is greater than the immediate next term in the block.
+   * <p>
+   * A low value, for example 1, improves the performance of automatons requiring many jumps, for example
+   * {@link org.apache.lucene.search.FuzzyQuery} and most {@link org.apache.lucene.search.WildcardQuery}.
+   * A higher value improves the performance of automatons with less or no jump, for example
+   * {@link org.apache.lucene.search.PrefixQuery}.
+   * A threshold of 4 seems to be a good balance.
+   */
+  protected final int NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD = 4;
 
-  protected int blockPrefixRunAutomatonState;
-  protected int blockPrefixLen;
+  protected final Automaton automaton;
+  protected final ByteRunAutomaton runAutomaton;
+  protected final boolean finite;
+  protected final BytesRef commonSuffix; // maybe null
+  protected final int minTermLength;
+  protected final AutomatonNextTermCalculator nextStringCalculator;
 
   /**
-   * Number of bytes accepted by the last call to {@link #runAutomatonForState}.
+   * Set this when our current mode is seeking to this term.  Set to null after.
+   */
+  protected BytesRef seekTerm;
+  /**
+   * Number of bytes accepted by the automaton when validating the current term.
+   */
+  protected int numMatchedBytes;
+  /**
+   * Automaton states reached when validating the current term, from 0 to {@link #numMatchedBytes} - 1.
+   */
+  protected int[] states;
+  /**
+   * Block iteration order determined when scanning the terms in the current block.
    */
-  protected int numBytesAccepted;
+  protected BlockIteration blockIteration;
   /**
-   * Whether the current term is beyond the automaton common prefix.
-   * If true this means the enumeration should stop immediately.
+   * Counter of the number of consecutively rejected terms.
+   * Depending on {@link #NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD}, this may trigger a jump to a block away.
    */
-  protected boolean beyondCommonPrefix;
+  protected int numConsecutivelyRejectedTerms;
 
-  public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
-                              IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
-                              FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
+  protected IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
+                                 IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput,
+                                 PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
+                                 BlockDecoder blockDecoder) throws IOException {
     super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
-    this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);
-    Automaton automaton = Objects.requireNonNull(compiled.automaton);
-    this.runAutomaton = Objects.requireNonNull(compiled.runAutomaton);
-    this.commonSuffixRef = compiled.commonSuffixRef; // maybe null
-    this.commonPrefixRef = Operations.getCommonPrefixBytesRef(automaton); // never null
-
-    this.startTerm = startTerm;
-    assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef);
-    // it is thus also true that startTerm >= commonPrefixRef
-
-    this.seekTerm = startTerm != null ? startTerm : commonPrefixRef;
+    checkIntersectAutomatonType(compiled);
+    automaton = compiled.automaton;
+    runAutomaton = compiled.runAutomaton;
+    finite = compiled.finite;
+    commonSuffix = compiled.commonSuffixRef;
+    minTermLength = getMinTermLength();
+    nextStringCalculator = new AutomatonNextTermCalculator(compiled);
+    seekTerm = startTerm;
   }
 
-  @Override
-  public BytesRef next() throws IOException {
-    clearTermState();
-
-    if (blockHeader == null) { // initial state
-      // note: don't call super.seekCeil here; we have our own logic
-
-      // Set the browser position to the block having the seek term.
-      // Even if -1, it's okay since we'll soon call nextKey().
-      long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
-      if (isBeyondLastTerm(seekTerm, blockStartFP)) {
-        return null; // EOF
-      }
-
-      // Starting at this block find and load the next matching block.
-      //   note: Since seekBlock was just called, we actually consider the current block as "next".
-      if (nextBlockMatchingPrefix() == false) { // note: starts at seek'ed block, which may have a match
-        return null; // EOF
-      }
+  protected void checkIntersectAutomatonType(CompiledAutomaton compiled) {
+    if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+      throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
     }
-
-    do {
-
-      // look in the rest of this block.
-      BytesRef term = nextTermInBlockMatching();
-      if (term != null) {
-        return term;
-      }
-
-      // next term dict matching prefix
-    } while (nextBlockMatchingPrefix());
-
-    return null; // EOF
   }
 
   /**
-   * Find the next block that appears to contain terms that could match the automata.
-   * The prefix is the primary clue.  Returns true if at one, or false for no more (EOF).
+   * Computes the minimal length of the terms accepted by the automaton.
+   * This speeds up the term scanning for automatons accepting a finite language.
    */
-  protected boolean nextBlockMatchingPrefix() throws IOException {
-    if (beyondCommonPrefix) {
-      return false; // EOF
+  protected int getMinTermLength() {
+    // Automatons accepting infinite language (e.g. PrefixQuery and WildcardQuery) do not benefit much from
+    // min term length while it takes time to compute it. More precisely, by skipping this computation PrefixQuery
+    // is significantly boosted while WildcardQuery might be slightly degraded on average. This min term length
+    // mainly boosts FuzzyQuery.
+    int commonSuffixLength = commonSuffix == null ? 0 : commonSuffix.length;
+    if (!finite) {
+      return commonSuffixLength;
     }
-
-    IndexDictionary.Browser browser = getOrCreateDictionaryBrowser();
-
-    do {
-
-      // Get next block key (becomes in effect the current blockKey)
-      BytesRef blockKey = browser.nextKey();
-      if (blockKey == null) {
-        return false; // EOF
-      }
-
-      blockPrefixLen = browser.getBlockPrefixLen();
-      blockPrefixRunAutomatonState = runAutomatonForState(blockKey.bytes, blockKey.offset, blockPrefixLen, 0);
-
-      // We may have passed commonPrefix  (a short-circuit optimization).
-      if (isBeyondCommonPrefix(blockKey)) {
-        return false; // EOF
+    int commonPrefixLength = 0;
+    int state = 0;
+    Transition t = null;
+    int[] visited = null;
+    int visitedLength = 0;
+    while (true) {
+      if (runAutomaton.isAccept(state)) {
+        // The common prefix reaches a final state. So common prefix and common suffix overlap.
+        // Min term length is the max between common prefix and common suffix lengths.
+        return Math.max(commonPrefixLength, commonSuffixLength);
       }
-
-      if (blockPrefixRunAutomatonState >= 0) {
-        break; // a match
+      if (automaton.getNumTransitions(state) == 1) {
+        if (t == null) {
+          t = new Transition();
+        }
+        automaton.getTransition(state, 0, t);
+        if (t.min == t.max) {
+          state = t.dest;
+          if (visited == null) {
+            visited = new int[8];
+            visited[visitedLength++] = 0;
+          }
+          if (!contains(visited, visitedLength, state)) {
+            visited = ArrayUtil.grow(visited, ++visitedLength);
+            visited[visitedLength - 1] = state;
+            commonPrefixLength++;
+            continue;
+          }
+        }
       }
+      break;
+    }
+    // Min term length is the sum of common prefix and common suffix lengths.
+    return commonPrefixLength + commonSuffixLength;
+  }
 
-      //
-      // This block doesn't match.
-      //
-
-      seekTerm = null; // we're moving on to another block, and seekTerm is before it.
-
-      // Should we simply get the next key (linear mode) or try to seek?
-      if (nextStringCalculator.isLinearState(blockKey)) {
-        continue;
+  private static boolean contains(int[] ints, int length, int value) {
+    assert length <= ints.length;
+    for (int i = 0; i < length; i++) {
+      if (ints[i] == value) {
+        return true;
       }
+    }
+    return false;
+  }
 
-      // Maybe the next block's key matches?  We have to check this before calling nextStringCalculator.
-      BytesRef peekKey = browser.peekKey();
-      if (peekKey == null) {
-        return false; // EOF
-      }
-      if (runAutomatonForState(peekKey.bytes, peekKey.offset, peekKey.length, 0) >= 0) {
-        continue; // yay; it matched.  Continue to actually advance to it.  This is rare?
+  @Override
+  public BytesRef next() throws IOException {
+    if (blockHeader == null) {
+      if (!seekFirstBlock()) {
+        return null;
       }
-
-      // Seek to a block by calculating the next term to match the automata *following* peekKey.
-      this.seekTerm = nextStringCalculator.nextSeekTerm(browser.peekKey());
-      if (seekTerm == null) {
-        return false; // EOF
+      states = new int[32];
+      blockIteration = BlockIteration.NEXT;
+    }
+    termState = null;
+    do {
+      BytesRef term = nextTermInBlockMatching();
+      if (term != null) {
+        return term;
       }
-      browser.seekBlock(seekTerm);
-      //continue
-
-    } while (true); // while not a match
-
-    // A match!
-
-    //NOTE: we could determine if this automata has a prefix for this specific block (longer than the commonPrefix).
-    //  If we see it, we could set it as the seekTerm and we could also exit the block early if we get past this prefix
-    //  and runAutomatonFromPrefix would start from this prefix.  Smiley tried but benchmarks were not favorable to it.
-
-    initializeHeader(null, browser.getBlockFilePointer());
+    } while (nextBlock());
+    return null;
+  }
 
-    return true;
+  protected boolean seekFirstBlock() throws IOException {
+    seekTerm = nextStringCalculator.nextSeekTerm(seekTerm);
+    if (seekTerm == null) {
+      return false;
+    }
+    long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
+    if (blockStartFP == -1) {
+      blockStartFP = fieldMetadata.getFirstBlockStartFP();
+    } else if (isBeyondLastTerm(seekTerm, blockStartFP)) {
+      return false;
+    }
+    initializeHeader(seekTerm, blockStartFP);
+    return blockHeader != null;
   }
 
   /**
-   * Find the next block line that matches, or null when at end of block.
+   * Finds the next block line that matches (accepted by the automaton), or null when at end of block.
+   *
+   * @return The next term in the current block that is accepted by the automaton; or null if none.
    */
   protected BytesRef nextTermInBlockMatching() throws IOException {
-    do {
-      // if seekTerm is set, then we seek into this block instead of starting with the first blindly.
-      if (seekTerm != null) {
-        assert blockLine == null;
-        boolean moveBeyondIfFound = seekTerm == startTerm; // for startTerm, we want to get the following term
-        SeekStatus seekStatus = seekInBlock(seekTerm);
-        seekTerm = null;// reset.
-        if (seekStatus == SeekStatus.END) {
-          return null;
-        } else if (seekStatus == SeekStatus.FOUND && moveBeyondIfFound) {
-          if (readLineInBlock() == null) {
-            return null;
+    if (seekTerm == null) {
+      if (readLineInBlock() == null) {
+        return null;
+      }
+    } else {
+      SeekStatus seekStatus = seekInBlock(seekTerm);
+      seekTerm = null;
+      if (seekStatus == SeekStatus.END) {
+        return null;
+      }
+      assert numMatchedBytes == 0;
+      assert numConsecutivelyRejectedTerms == 0;
+    }
+    int[] states = this.states;
+    while (true) {
+      TermBytes lineTermBytes = blockLine.getTermBytes();
+      BytesRef lineTerm = lineTermBytes.getTerm();
+      assert lineTerm.offset == 0;
+      byte[] termBytes = lineTerm.bytes;
+      int termLength = lineTerm.length;
+      if (states.length <= termLength) {
+        states = this.states = ArrayUtil.growExact(states, ArrayUtil.oversize(termLength + 1, Byte.BYTES));
+      }
+      // Since terms a delta encoded, we may start the automaton steps from the last state reached by the previous term.
+      int index = Math.min(lineTermBytes.getSuffixOffset(), numMatchedBytes);
+      // Skip this term early if it is shorter than the min term length, or if it does not end with the common suffix
+      // accepted by the automaton.
+      if (termLength >= minTermLength && (commonSuffix == null || endsWithCommonSuffix(termBytes, termLength))) {
+        int state = states[index];
+        while (true) {
+          if (index == termLength) {
+            if (runAutomaton.isAccept(state)) {
+              // The automaton accepts the current term. Record the number of matched bytes and return the term.
+              assert runAutomaton.run(termBytes, 0, termLength);
+              numMatchedBytes = index;
+              if (numConsecutivelyRejectedTerms > 0) {
 
 Review comment:
   Can't we just always set to 0?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r383526667
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java
 ##########
 @@ -18,260 +18,337 @@
 package org.apache.lucene.codecs.uniformsplit;
 
 import java.io.IOException;
-import java.util.Objects;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
  * The "intersect" {@link TermsEnum} response to {@link UniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
  * intersecting the terms with an automaton.
+ * <p>
+ * By design of the UniformSplit block keys, it is less efficient than
+ * {@code org.apache.lucene.codecs.blocktree.IntersectTermsEnum} for {@link org.apache.lucene.search.FuzzyQuery} (-37%).
+ * It is slightly slower for {@link org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for
+ * {@link org.apache.lucene.search.PrefixQuery} (+5%).
+ *
+ * @lucene.experimental
  */
 public class IntersectBlockReader extends BlockReader {
 
-  protected final AutomatonNextTermCalculator nextStringCalculator;
-  protected final ByteRunAutomaton runAutomaton;
-  protected final BytesRef commonSuffixRef; // maybe null
-  protected final BytesRef commonPrefixRef;
-  protected final BytesRef startTerm; // maybe null
+  /**
+   * Block iteration order. Whether to move next block, jump to a block away, or end the iteration.
+   */
+  protected enum BlockIteration {NEXT, SEEK, END}
 
-  /** Set this when our current mode is seeking to this term.  Set to null after. */
-  protected BytesRef seekTerm;
+  /**
+   * Threshold that controls when to attempt to jump to a block away.
+   * <p>
+   * This counter is 0 when entering a block. It is incremented each time a term is rejected by the automaton.
+   * When the counter is greater than or equal to this threshold, then we compute the next term accepted by
+   * the automaton, with {@link AutomatonNextTermCalculator}, and we jump to a block away if the next term
+   * accepted is greater than the immediate next term in the block.
+   * <p>
+   * A low value, for example 1, improves the performance of automatons requiring many jumps, for example
+   * {@link org.apache.lucene.search.FuzzyQuery} and most {@link org.apache.lucene.search.WildcardQuery}.
+   * A higher value improves the performance of automatons with less or no jump, for example
+   * {@link org.apache.lucene.search.PrefixQuery}.
+   * A threshold of 4 seems to be a good balance.
+   */
+  protected final int NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD = 4;
 
-  protected int blockPrefixRunAutomatonState;
-  protected int blockPrefixLen;
+  protected final Automaton automaton;
+  protected final ByteRunAutomaton runAutomaton;
+  protected final boolean finite;
+  protected final BytesRef commonSuffix; // maybe null
+  protected final int minTermLength;
+  protected final AutomatonNextTermCalculator nextStringCalculator;
 
   /**
-   * Number of bytes accepted by the last call to {@link #runAutomatonForState}.
+   * Set this when our current mode is seeking to this term.  Set to null after.
+   */
+  protected BytesRef seekTerm;
+  /**
+   * Number of bytes accepted by the automaton when validating the current term.
+   */
+  protected int numMatchedBytes;
+  /**
+   * Automaton states reached when validating the current term, from 0 to {@link #numMatchedBytes} - 1.
+   */
+  protected int[] states;
+  /**
+   * Block iteration order determined when scanning the terms in the current block.
    */
-  protected int numBytesAccepted;
+  protected BlockIteration blockIteration;
   /**
-   * Whether the current term is beyond the automaton common prefix.
-   * If true this means the enumeration should stop immediately.
+   * Counter of the number of consecutively rejected terms.
+   * Depending on {@link #NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD}, this may trigger a jump to a block away.
    */
-  protected boolean beyondCommonPrefix;
+  protected int numConsecutivelyRejectedTerms;
 
-  public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
-                              IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
-                              FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
+  protected IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
+                                 IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput,
+                                 PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
+                                 BlockDecoder blockDecoder) throws IOException {
     super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
-    this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);
-    Automaton automaton = Objects.requireNonNull(compiled.automaton);
-    this.runAutomaton = Objects.requireNonNull(compiled.runAutomaton);
-    this.commonSuffixRef = compiled.commonSuffixRef; // maybe null
-    this.commonPrefixRef = Operations.getCommonPrefixBytesRef(automaton); // never null
-
-    this.startTerm = startTerm;
-    assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef);
-    // it is thus also true that startTerm >= commonPrefixRef
-
-    this.seekTerm = startTerm != null ? startTerm : commonPrefixRef;
+    checkIntersectAutomatonType(compiled);
+    automaton = compiled.automaton;
+    runAutomaton = compiled.runAutomaton;
+    finite = compiled.finite;
+    commonSuffix = compiled.commonSuffixRef;
+    minTermLength = getMinTermLength();
+    nextStringCalculator = new AutomatonNextTermCalculator(compiled);
+    seekTerm = startTerm;
   }
 
-  @Override
-  public BytesRef next() throws IOException {
-    clearTermState();
-
-    if (blockHeader == null) { // initial state
-      // note: don't call super.seekCeil here; we have our own logic
-
-      // Set the browser position to the block having the seek term.
-      // Even if -1, it's okay since we'll soon call nextKey().
-      long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
-      if (isBeyondLastTerm(seekTerm, blockStartFP)) {
-        return null; // EOF
-      }
-
-      // Starting at this block find and load the next matching block.
-      //   note: Since seekBlock was just called, we actually consider the current block as "next".
-      if (nextBlockMatchingPrefix() == false) { // note: starts at seek'ed block, which may have a match
-        return null; // EOF
-      }
+  protected void checkIntersectAutomatonType(CompiledAutomaton compiled) {
+    if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+      throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
     }
-
-    do {
-
-      // look in the rest of this block.
-      BytesRef term = nextTermInBlockMatching();
-      if (term != null) {
-        return term;
-      }
-
-      // next term dict matching prefix
-    } while (nextBlockMatchingPrefix());
-
-    return null; // EOF
   }
 
   /**
-   * Find the next block that appears to contain terms that could match the automata.
-   * The prefix is the primary clue.  Returns true if at one, or false for no more (EOF).
+   * Computes the minimal length of the terms accepted by the automaton.
+   * This speeds up the term scanning for automatons accepting a finite language.
    */
-  protected boolean nextBlockMatchingPrefix() throws IOException {
-    if (beyondCommonPrefix) {
-      return false; // EOF
+  protected int getMinTermLength() {
+    // Automatons accepting infinite language (e.g. PrefixQuery and WildcardQuery) do not benefit much from
+    // min term length while it takes time to compute it. More precisely, by skipping this computation PrefixQuery
+    // is significantly boosted while WildcardQuery might be slightly degraded on average. This min term length
+    // mainly boosts FuzzyQuery.
+    int commonSuffixLength = commonSuffix == null ? 0 : commonSuffix.length;
+    if (!finite) {
+      return commonSuffixLength;
     }
-
-    IndexDictionary.Browser browser = getOrCreateDictionaryBrowser();
-
-    do {
-
-      // Get next block key (becomes in effect the current blockKey)
-      BytesRef blockKey = browser.nextKey();
-      if (blockKey == null) {
-        return false; // EOF
-      }
-
-      blockPrefixLen = browser.getBlockPrefixLen();
-      blockPrefixRunAutomatonState = runAutomatonForState(blockKey.bytes, blockKey.offset, blockPrefixLen, 0);
-
-      // We may have passed commonPrefix  (a short-circuit optimization).
-      if (isBeyondCommonPrefix(blockKey)) {
-        return false; // EOF
+    int commonPrefixLength = 0;
+    int state = 0;
+    Transition t = null;
+    int[] visited = null;
+    int visitedLength = 0;
+    while (true) {
+      if (runAutomaton.isAccept(state)) {
+        // The common prefix reaches a final state. So common prefix and common suffix overlap.
+        // Min term length is the max between common prefix and common suffix lengths.
+        return Math.max(commonPrefixLength, commonSuffixLength);
       }
-
-      if (blockPrefixRunAutomatonState >= 0) {
-        break; // a match
+      if (automaton.getNumTransitions(state) == 1) {
+        if (t == null) {
+          t = new Transition();
+        }
+        automaton.getTransition(state, 0, t);
+        if (t.min == t.max) {
+          state = t.dest;
+          if (visited == null) {
+            visited = new int[8];
+            visited[visitedLength++] = 0;
+          }
+          if (!contains(visited, visitedLength, state)) {
+            visited = ArrayUtil.grow(visited, ++visitedLength);
+            visited[visitedLength - 1] = state;
+            commonPrefixLength++;
+            continue;
+          }
+        }
       }
+      break;
+    }
+    // Min term length is the sum of common prefix and common suffix lengths.
+    return commonPrefixLength + commonSuffixLength;
+  }
 
-      //
-      // This block doesn't match.
-      //
-
-      seekTerm = null; // we're moving on to another block, and seekTerm is before it.
-
-      // Should we simply get the next key (linear mode) or try to seek?
-      if (nextStringCalculator.isLinearState(blockKey)) {
-        continue;
+  private static boolean contains(int[] ints, int length, int value) {
+    assert length <= ints.length;
+    for (int i = 0; i < length; i++) {
+      if (ints[i] == value) {
+        return true;
       }
+    }
+    return false;
+  }
 
-      // Maybe the next block's key matches?  We have to check this before calling nextStringCalculator.
-      BytesRef peekKey = browser.peekKey();
-      if (peekKey == null) {
-        return false; // EOF
-      }
-      if (runAutomatonForState(peekKey.bytes, peekKey.offset, peekKey.length, 0) >= 0) {
-        continue; // yay; it matched.  Continue to actually advance to it.  This is rare?
+  @Override
+  public BytesRef next() throws IOException {
+    if (blockHeader == null) {
+      if (!seekFirstBlock()) {
+        return null;
       }
-
-      // Seek to a block by calculating the next term to match the automata *following* peekKey.
-      this.seekTerm = nextStringCalculator.nextSeekTerm(browser.peekKey());
-      if (seekTerm == null) {
-        return false; // EOF
+      states = new int[32];
+      blockIteration = BlockIteration.NEXT;
+    }
+    termState = null;
+    do {
+      BytesRef term = nextTermInBlockMatching();
+      if (term != null) {
+        return term;
       }
-      browser.seekBlock(seekTerm);
-      //continue
-
-    } while (true); // while not a match
-
-    // A match!
-
-    //NOTE: we could determine if this automata has a prefix for this specific block (longer than the commonPrefix).
-    //  If we see it, we could set it as the seekTerm and we could also exit the block early if we get past this prefix
-    //  and runAutomatonFromPrefix would start from this prefix.  Smiley tried but benchmarks were not favorable to it.
-
-    initializeHeader(null, browser.getBlockFilePointer());
+    } while (nextBlock());
+    return null;
+  }
 
-    return true;
+  protected boolean seekFirstBlock() throws IOException {
+    seekTerm = nextStringCalculator.nextSeekTerm(seekTerm);
+    if (seekTerm == null) {
+      return false;
+    }
+    long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
+    if (blockStartFP == -1) {
+      blockStartFP = fieldMetadata.getFirstBlockStartFP();
+    } else if (isBeyondLastTerm(seekTerm, blockStartFP)) {
+      return false;
+    }
+    initializeHeader(seekTerm, blockStartFP);
+    return blockHeader != null;
   }
 
   /**
-   * Find the next block line that matches, or null when at end of block.
+   * Finds the next block line that matches (accepted by the automaton), or null when at end of block.
+   *
+   * @return The next term in the current block that is accepted by the automaton; or null if none.
    */
   protected BytesRef nextTermInBlockMatching() throws IOException {
-    do {
-      // if seekTerm is set, then we seek into this block instead of starting with the first blindly.
-      if (seekTerm != null) {
-        assert blockLine == null;
-        boolean moveBeyondIfFound = seekTerm == startTerm; // for startTerm, we want to get the following term
-        SeekStatus seekStatus = seekInBlock(seekTerm);
-        seekTerm = null;// reset.
-        if (seekStatus == SeekStatus.END) {
-          return null;
-        } else if (seekStatus == SeekStatus.FOUND && moveBeyondIfFound) {
-          if (readLineInBlock() == null) {
-            return null;
+    if (seekTerm == null) {
+      if (readLineInBlock() == null) {
+        return null;
+      }
+    } else {
+      SeekStatus seekStatus = seekInBlock(seekTerm);
+      seekTerm = null;
+      if (seekStatus == SeekStatus.END) {
+        return null;
+      }
+      assert numMatchedBytes == 0;
+      assert numConsecutivelyRejectedTerms == 0;
+    }
+    int[] states = this.states;
+    while (true) {
+      TermBytes lineTermBytes = blockLine.getTermBytes();
+      BytesRef lineTerm = lineTermBytes.getTerm();
+      assert lineTerm.offset == 0;
+      byte[] termBytes = lineTerm.bytes;
+      int termLength = lineTerm.length;
+      if (states.length <= termLength) {
+        states = this.states = ArrayUtil.growExact(states, ArrayUtil.oversize(termLength + 1, Byte.BYTES));
+      }
+      // Since terms a delta encoded, we may start the automaton steps from the last state reached by the previous term.
+      int index = Math.min(lineTermBytes.getSuffixOffset(), numMatchedBytes);
+      // Skip this term early if it is shorter than the min term length, or if it does not end with the common suffix
+      // accepted by the automaton.
+      if (termLength >= minTermLength && (commonSuffix == null || endsWithCommonSuffix(termBytes, termLength))) {
+        int state = states[index];
+        while (true) {
+          if (index == termLength) {
+            if (runAutomaton.isAccept(state)) {
+              // The automaton accepts the current term. Record the number of matched bytes and return the term.
+              assert runAutomaton.run(termBytes, 0, termLength);
+              numMatchedBytes = index;
+              if (numConsecutivelyRejectedTerms > 0) {
+                numConsecutivelyRejectedTerms = 0;
+              }
+              assert blockIteration == BlockIteration.NEXT;
+              return lineTerm;
+            }
+            break;
+          }
+          state = runAutomaton.step(state, termBytes[index] & 0xff);
+          if (state == -1) {
+            // The automaton rejects the current term.
+            break;
           }
+          // Record the reached automaton state.
+          states[++index] = state;
         }
-        assert blockLine != null;
-      } else {
-        if (readLineInBlock() == null) {
+      }
+      // The current term is not accepted by the automaton.
+      // Still record the reached automaton state to start the next term steps from there.
+      assert !runAutomaton.run(termBytes, 0, termLength);
+      numMatchedBytes = index;
+      // If the number of consecutively rejected terms reaches the threshold, then determine whether it is worth to
+      // jump to a block away.
+      if (++numConsecutivelyRejectedTerms >= NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD
+          && lineIndexInBlock < blockHeader.getLinesCount() - 1
+          && !nextStringCalculator.isLinearState(lineTerm)) {
+        // Compute the next term accepted by the automaton after the current term.
+        if ((seekTerm = nextStringCalculator.nextSeekTerm(lineTerm)) == null) {
+          blockIteration = BlockIteration.END;
           return null;
         }
-      }
-
-      TermBytes lineTermBytes = blockLine.getTermBytes();
-      BytesRef lineTerm = lineTermBytes.getTerm();
-
-      if (commonSuffixRef == null || StringHelper.endsWith(lineTerm, commonSuffixRef)) {
-        if (runAutomatonFromPrefix(lineTerm)) {
-          return lineTerm;
-        } else if (beyondCommonPrefix) {
+        // It is worth to jump to a block away if the next term accepted is after the next term in the block.
 
 Review comment:
   Reword "worth" -> either "worth it" or "worthwhile"

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r383536370
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java
 ##########
 @@ -285,64 +362,66 @@ public void seekExact(long ord) {
   }
 
   @Override
-  public SeekStatus seekCeil(BytesRef text) {
+  public void seekExact(BytesRef term, TermState state) {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public void seekExact(BytesRef term, TermState state) {
+  public SeekStatus seekCeil(BytesRef text) {
     throw new UnsupportedOperationException();
   }
 
   /**
    * This is a copy of AutomatonTermsEnum.  Since it's an inner class, the outer class can
 
 Review comment:
   Well; it's _mostly_ a copy of AutomatonTermsEnum now :-/  The duplication is a shame.  Just insert the word "_mostly_" and it satisfies me.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] dsmiley removed a comment on issue #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
dsmiley removed a comment on issue #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#issuecomment-590966953
 
 
   Your explanation really helps; perhaps it should be posted on the JIRA as it really summarizes what's going on.
   
   > For the proposal of the block prefix in the BlockHeader, does that mean that we have to open the block to get the prefix? Because the speed for FuzzyQuery highly depends on how many block we don't open.
   
   It would mean opening the block, admittedly, though it would allow short-circuiting reading the 4 blocklines before your code here gives up on the block.  It's a shame we can't get better block common prefixes that allow short-circuiting fuzzy queries, at least an edit distance 1.  I think it could help if the blockline chunking algorithm could consider the prefix in its algorithm so as to increase the probability of getting better block prefixes.  It'd never be a guarantee since we want reasonable sized blocks.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] bruno-roustant commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
bruno-roustant commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r382682649
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java
 ##########
 @@ -1091,25 +1091,33 @@ public static String getCommonPrefix(Automaton a) {
    * @return common prefix, which can be an empty (length 0) BytesRef (never null)
    */
   public static BytesRef getCommonPrefixBytesRef(Automaton a) {
 
 Review comment:
   Some optimizations in this method (always called to build the commonSuffix for CompiledAutomaton for infinite language, e.g. PrefixQuery and WildcardQuery). Lazy create structures (WildcardQuery may not have a first single-path state), and only one visited.add() call. This speeds up mainly WildcardQuery (a couple % in my luceneutil measures).

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] bruno-roustant edited a comment on issue #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
bruno-roustant edited a comment on issue #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#issuecomment-590780144
 
 
   When I debugged thoroughly to understand what was the limitation of the approach we had (to compute the common prefix between two consecutive block keys in the FST), I saw that actually for all FuzzyQuery the common prefix matched so we entered all blocks.
   I realized that the FuzzyQuery automaton accepts many variations for the prefix, and the common prefix was not long enough to allow us to filter correctly.
   
   I looked at what VarGapFixedInterval did. It jumped all the time after each term to find the next target term accepted by the automaton. And this was sufficiently efficient thanks to a vital optimization that compared the target term to the immediate following term, to actually not jump most of the time.
   
   So I applied the same idea to compute the next accepted term and jump, but now with a first condition based on the number of consecutively rejected terms, and by anticipating the comparison of the accepted term with the immediate term. This is the main factor of the improvement. We leverage also optimizations that speed up the automaton validation of terms in the block.
   
   For the proposal of the block prefix in the BlockHeader, does that mean that we have to open the block to get the prefix? Because the speed for FuzzyQuery highly depends on how many block we *don't* open.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r383522035
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java
 ##########
 @@ -18,260 +18,337 @@
 package org.apache.lucene.codecs.uniformsplit;
 
 import java.io.IOException;
-import java.util.Objects;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
  * The "intersect" {@link TermsEnum} response to {@link UniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
  * intersecting the terms with an automaton.
+ * <p>
+ * By design of the UniformSplit block keys, it is less efficient than
+ * {@code org.apache.lucene.codecs.blocktree.IntersectTermsEnum} for {@link org.apache.lucene.search.FuzzyQuery} (-37%).
+ * It is slightly slower for {@link org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for
+ * {@link org.apache.lucene.search.PrefixQuery} (+5%).
+ *
+ * @lucene.experimental
  */
 public class IntersectBlockReader extends BlockReader {
 
-  protected final AutomatonNextTermCalculator nextStringCalculator;
-  protected final ByteRunAutomaton runAutomaton;
-  protected final BytesRef commonSuffixRef; // maybe null
-  protected final BytesRef commonPrefixRef;
-  protected final BytesRef startTerm; // maybe null
+  /**
+   * Block iteration order. Whether to move next block, jump to a block away, or end the iteration.
+   */
+  protected enum BlockIteration {NEXT, SEEK, END}
 
-  /** Set this when our current mode is seeking to this term.  Set to null after. */
-  protected BytesRef seekTerm;
+  /**
+   * Threshold that controls when to attempt to jump to a block away.
+   * <p>
+   * This counter is 0 when entering a block. It is incremented each time a term is rejected by the automaton.
+   * When the counter is greater than or equal to this threshold, then we compute the next term accepted by
+   * the automaton, with {@link AutomatonNextTermCalculator}, and we jump to a block away if the next term
+   * accepted is greater than the immediate next term in the block.
+   * <p>
+   * A low value, for example 1, improves the performance of automatons requiring many jumps, for example
+   * {@link org.apache.lucene.search.FuzzyQuery} and most {@link org.apache.lucene.search.WildcardQuery}.
+   * A higher value improves the performance of automatons with less or no jump, for example
+   * {@link org.apache.lucene.search.PrefixQuery}.
+   * A threshold of 4 seems to be a good balance.
+   */
+  protected final int NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD = 4;
 
-  protected int blockPrefixRunAutomatonState;
-  protected int blockPrefixLen;
+  protected final Automaton automaton;
+  protected final ByteRunAutomaton runAutomaton;
+  protected final boolean finite;
+  protected final BytesRef commonSuffix; // maybe null
+  protected final int minTermLength;
+  protected final AutomatonNextTermCalculator nextStringCalculator;
 
   /**
-   * Number of bytes accepted by the last call to {@link #runAutomatonForState}.
+   * Set this when our current mode is seeking to this term.  Set to null after.
+   */
+  protected BytesRef seekTerm;
+  /**
+   * Number of bytes accepted by the automaton when validating the current term.
+   */
+  protected int numMatchedBytes;
+  /**
+   * Automaton states reached when validating the current term, from 0 to {@link #numMatchedBytes} - 1.
+   */
+  protected int[] states;
+  /**
+   * Block iteration order determined when scanning the terms in the current block.
    */
-  protected int numBytesAccepted;
+  protected BlockIteration blockIteration;
   /**
-   * Whether the current term is beyond the automaton common prefix.
-   * If true this means the enumeration should stop immediately.
+   * Counter of the number of consecutively rejected terms.
+   * Depending on {@link #NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD}, this may trigger a jump to a block away.
    */
-  protected boolean beyondCommonPrefix;
+  protected int numConsecutivelyRejectedTerms;
 
-  public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
-                              IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
-                              FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
+  protected IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
+                                 IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput,
+                                 PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
+                                 BlockDecoder blockDecoder) throws IOException {
     super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
-    this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);
-    Automaton automaton = Objects.requireNonNull(compiled.automaton);
-    this.runAutomaton = Objects.requireNonNull(compiled.runAutomaton);
-    this.commonSuffixRef = compiled.commonSuffixRef; // maybe null
-    this.commonPrefixRef = Operations.getCommonPrefixBytesRef(automaton); // never null
-
-    this.startTerm = startTerm;
-    assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef);
-    // it is thus also true that startTerm >= commonPrefixRef
-
-    this.seekTerm = startTerm != null ? startTerm : commonPrefixRef;
+    checkIntersectAutomatonType(compiled);
+    automaton = compiled.automaton;
+    runAutomaton = compiled.runAutomaton;
+    finite = compiled.finite;
+    commonSuffix = compiled.commonSuffixRef;
+    minTermLength = getMinTermLength();
+    nextStringCalculator = new AutomatonNextTermCalculator(compiled);
+    seekTerm = startTerm;
   }
 
-  @Override
-  public BytesRef next() throws IOException {
-    clearTermState();
-
-    if (blockHeader == null) { // initial state
-      // note: don't call super.seekCeil here; we have our own logic
-
-      // Set the browser position to the block having the seek term.
-      // Even if -1, it's okay since we'll soon call nextKey().
-      long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
-      if (isBeyondLastTerm(seekTerm, blockStartFP)) {
-        return null; // EOF
-      }
-
-      // Starting at this block find and load the next matching block.
-      //   note: Since seekBlock was just called, we actually consider the current block as "next".
-      if (nextBlockMatchingPrefix() == false) { // note: starts at seek'ed block, which may have a match
-        return null; // EOF
-      }
+  protected void checkIntersectAutomatonType(CompiledAutomaton compiled) {
+    if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+      throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
     }
-
-    do {
-
-      // look in the rest of this block.
-      BytesRef term = nextTermInBlockMatching();
-      if (term != null) {
-        return term;
-      }
-
-      // next term dict matching prefix
-    } while (nextBlockMatchingPrefix());
-
-    return null; // EOF
   }
 
   /**
-   * Find the next block that appears to contain terms that could match the automata.
-   * The prefix is the primary clue.  Returns true if at one, or false for no more (EOF).
+   * Computes the minimal length of the terms accepted by the automaton.
+   * This speeds up the term scanning for automatons accepting a finite language.
    */
-  protected boolean nextBlockMatchingPrefix() throws IOException {
-    if (beyondCommonPrefix) {
-      return false; // EOF
+  protected int getMinTermLength() {
+    // Automatons accepting infinite language (e.g. PrefixQuery and WildcardQuery) do not benefit much from
+    // min term length while it takes time to compute it. More precisely, by skipping this computation PrefixQuery
+    // is significantly boosted while WildcardQuery might be slightly degraded on average. This min term length
+    // mainly boosts FuzzyQuery.
+    int commonSuffixLength = commonSuffix == null ? 0 : commonSuffix.length;
+    if (!finite) {
+      return commonSuffixLength;
     }
-
-    IndexDictionary.Browser browser = getOrCreateDictionaryBrowser();
-
-    do {
-
-      // Get next block key (becomes in effect the current blockKey)
-      BytesRef blockKey = browser.nextKey();
-      if (blockKey == null) {
-        return false; // EOF
-      }
-
-      blockPrefixLen = browser.getBlockPrefixLen();
-      blockPrefixRunAutomatonState = runAutomatonForState(blockKey.bytes, blockKey.offset, blockPrefixLen, 0);
-
-      // We may have passed commonPrefix  (a short-circuit optimization).
-      if (isBeyondCommonPrefix(blockKey)) {
-        return false; // EOF
+    int commonPrefixLength = 0;
+    int state = 0;
+    Transition t = null;
+    int[] visited = null;
+    int visitedLength = 0;
+    while (true) {
+      if (runAutomaton.isAccept(state)) {
+        // The common prefix reaches a final state. So common prefix and common suffix overlap.
+        // Min term length is the max between common prefix and common suffix lengths.
+        return Math.max(commonPrefixLength, commonSuffixLength);
       }
-
-      if (blockPrefixRunAutomatonState >= 0) {
-        break; // a match
+      if (automaton.getNumTransitions(state) == 1) {
+        if (t == null) {
+          t = new Transition();
+        }
+        automaton.getTransition(state, 0, t);
+        if (t.min == t.max) {
+          state = t.dest;
+          if (visited == null) {
+            visited = new int[8];
+            visited[visitedLength++] = 0;
+          }
+          if (!contains(visited, visitedLength, state)) {
+            visited = ArrayUtil.grow(visited, ++visitedLength);
+            visited[visitedLength - 1] = state;
+            commonPrefixLength++;
+            continue;
+          }
+        }
       }
+      break;
+    }
+    // Min term length is the sum of common prefix and common suffix lengths.
+    return commonPrefixLength + commonSuffixLength;
+  }
 
-      //
-      // This block doesn't match.
-      //
-
-      seekTerm = null; // we're moving on to another block, and seekTerm is before it.
-
-      // Should we simply get the next key (linear mode) or try to seek?
-      if (nextStringCalculator.isLinearState(blockKey)) {
-        continue;
+  private static boolean contains(int[] ints, int length, int value) {
+    assert length <= ints.length;
+    for (int i = 0; i < length; i++) {
+      if (ints[i] == value) {
+        return true;
       }
+    }
+    return false;
+  }
 
-      // Maybe the next block's key matches?  We have to check this before calling nextStringCalculator.
-      BytesRef peekKey = browser.peekKey();
-      if (peekKey == null) {
-        return false; // EOF
-      }
-      if (runAutomatonForState(peekKey.bytes, peekKey.offset, peekKey.length, 0) >= 0) {
-        continue; // yay; it matched.  Continue to actually advance to it.  This is rare?
+  @Override
+  public BytesRef next() throws IOException {
+    if (blockHeader == null) {
+      if (!seekFirstBlock()) {
+        return null;
       }
-
-      // Seek to a block by calculating the next term to match the automata *following* peekKey.
-      this.seekTerm = nextStringCalculator.nextSeekTerm(browser.peekKey());
-      if (seekTerm == null) {
-        return false; // EOF
+      states = new int[32];
+      blockIteration = BlockIteration.NEXT;
+    }
+    termState = null;
+    do {
+      BytesRef term = nextTermInBlockMatching();
+      if (term != null) {
+        return term;
       }
-      browser.seekBlock(seekTerm);
-      //continue
-
-    } while (true); // while not a match
-
-    // A match!
-
-    //NOTE: we could determine if this automata has a prefix for this specific block (longer than the commonPrefix).
-    //  If we see it, we could set it as the seekTerm and we could also exit the block early if we get past this prefix
-    //  and runAutomatonFromPrefix would start from this prefix.  Smiley tried but benchmarks were not favorable to it.
-
-    initializeHeader(null, browser.getBlockFilePointer());
+    } while (nextBlock());
+    return null;
+  }
 
-    return true;
+  protected boolean seekFirstBlock() throws IOException {
+    seekTerm = nextStringCalculator.nextSeekTerm(seekTerm);
+    if (seekTerm == null) {
+      return false;
+    }
+    long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
+    if (blockStartFP == -1) {
+      blockStartFP = fieldMetadata.getFirstBlockStartFP();
+    } else if (isBeyondLastTerm(seekTerm, blockStartFP)) {
+      return false;
+    }
+    initializeHeader(seekTerm, blockStartFP);
+    return blockHeader != null;
   }
 
   /**
-   * Find the next block line that matches, or null when at end of block.
+   * Finds the next block line that matches (accepted by the automaton), or null when at end of block.
+   *
+   * @return The next term in the current block that is accepted by the automaton; or null if none.
    */
   protected BytesRef nextTermInBlockMatching() throws IOException {
-    do {
-      // if seekTerm is set, then we seek into this block instead of starting with the first blindly.
-      if (seekTerm != null) {
-        assert blockLine == null;
-        boolean moveBeyondIfFound = seekTerm == startTerm; // for startTerm, we want to get the following term
-        SeekStatus seekStatus = seekInBlock(seekTerm);
-        seekTerm = null;// reset.
-        if (seekStatus == SeekStatus.END) {
-          return null;
-        } else if (seekStatus == SeekStatus.FOUND && moveBeyondIfFound) {
-          if (readLineInBlock() == null) {
-            return null;
+    if (seekTerm == null) {
+      if (readLineInBlock() == null) {
+        return null;
+      }
+    } else {
+      SeekStatus seekStatus = seekInBlock(seekTerm);
+      seekTerm = null;
+      if (seekStatus == SeekStatus.END) {
+        return null;
+      }
+      assert numMatchedBytes == 0;
+      assert numConsecutivelyRejectedTerms == 0;
+    }
+    int[] states = this.states;
+    while (true) {
+      TermBytes lineTermBytes = blockLine.getTermBytes();
+      BytesRef lineTerm = lineTermBytes.getTerm();
+      assert lineTerm.offset == 0;
+      byte[] termBytes = lineTerm.bytes;
+      int termLength = lineTerm.length;
+      if (states.length <= termLength) {
+        states = this.states = ArrayUtil.growExact(states, ArrayUtil.oversize(termLength + 1, Byte.BYTES));
+      }
+      // Since terms a delta encoded, we may start the automaton steps from the last state reached by the previous term.
 
 Review comment:
   a -> are

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] bruno-roustant edited a comment on issue #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
bruno-roustant edited a comment on issue #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#issuecomment-590780144
 
 
   When I debugged thoroughly to understand what was the limitation of the approach we had (to compute the common prefix between two consecutive block keys in the FST), I saw that actually for all FuzzyQuery the common prefix matched so we entered all blocks.
   I realized that the FuzzyQuery automaton accepts many variations for the prefix, and the common prefix was not long enough to allow us to filter correctly.
   
   I looked at what VarGapFixedInterval did. It jumped all the time after each term to find the next target term accepted by the automaton. And this was sufficiently efficient thanks to a vital optimization that compared the target term to the immediate following term, to actually not jump most of the time.
   
   So I applied the same idea to compute the next accepted term and jump, but now with a first condition based on the number of consecutively rejected terms, and by anticipating the comparison of the accepted term with the immediate term. This is the main factor of the improvement. We leverage also optimization that speed up the automaton validation of terms in the block.
   
   For the proposal of the block prefix in the BlockHeader, does that mean that we have to open the block to get the prefix? Because the speed for FuzzyQuery highly depends on how many block we *don't* open.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r383994833
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java
 ##########
 @@ -18,260 +18,337 @@
 package org.apache.lucene.codecs.uniformsplit;
 
 import java.io.IOException;
-import java.util.Objects;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
  * The "intersect" {@link TermsEnum} response to {@link UniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
  * intersecting the terms with an automaton.
+ * <p>
+ * By design of the UniformSplit block keys, it is less efficient than
+ * {@code org.apache.lucene.codecs.blocktree.IntersectTermsEnum} for {@link org.apache.lucene.search.FuzzyQuery} (-37%).
+ * It is slightly slower for {@link org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for
+ * {@link org.apache.lucene.search.PrefixQuery} (+5%).
+ *
+ * @lucene.experimental
  */
 public class IntersectBlockReader extends BlockReader {
 
-  protected final AutomatonNextTermCalculator nextStringCalculator;
-  protected final ByteRunAutomaton runAutomaton;
-  protected final BytesRef commonSuffixRef; // maybe null
-  protected final BytesRef commonPrefixRef;
-  protected final BytesRef startTerm; // maybe null
+  /**
+   * Block iteration order. Whether to move next block, jump to a block away, or end the iteration.
+   */
+  protected enum BlockIteration {NEXT, SEEK, END}
 
-  /** Set this when our current mode is seeking to this term.  Set to null after. */
-  protected BytesRef seekTerm;
+  /**
+   * Threshold that controls when to attempt to jump to a block away.
+   * <p>
+   * This counter is 0 when entering a block. It is incremented each time a term is rejected by the automaton.
+   * When the counter is greater than or equal to this threshold, then we compute the next term accepted by
+   * the automaton, with {@link AutomatonNextTermCalculator}, and we jump to a block away if the next term
+   * accepted is greater than the immediate next term in the block.
+   * <p>
+   * A low value, for example 1, improves the performance of automatons requiring many jumps, for example
+   * {@link org.apache.lucene.search.FuzzyQuery} and most {@link org.apache.lucene.search.WildcardQuery}.
+   * A higher value improves the performance of automatons with less or no jump, for example
+   * {@link org.apache.lucene.search.PrefixQuery}.
+   * A threshold of 4 seems to be a good balance.
+   */
+  protected final int NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD = 4;
 
-  protected int blockPrefixRunAutomatonState;
-  protected int blockPrefixLen;
+  protected final Automaton automaton;
+  protected final ByteRunAutomaton runAutomaton;
+  protected final boolean finite;
+  protected final BytesRef commonSuffix; // maybe null
+  protected final int minTermLength;
+  protected final AutomatonNextTermCalculator nextStringCalculator;
 
   /**
-   * Number of bytes accepted by the last call to {@link #runAutomatonForState}.
+   * Set this when our current mode is seeking to this term.  Set to null after.
+   */
+  protected BytesRef seekTerm;
+  /**
+   * Number of bytes accepted by the automaton when validating the current term.
+   */
+  protected int numMatchedBytes;
+  /**
+   * Automaton states reached when validating the current term, from 0 to {@link #numMatchedBytes} - 1.
+   */
+  protected int[] states;
+  /**
+   * Block iteration order determined when scanning the terms in the current block.
    */
-  protected int numBytesAccepted;
+  protected BlockIteration blockIteration;
   /**
-   * Whether the current term is beyond the automaton common prefix.
-   * If true this means the enumeration should stop immediately.
+   * Counter of the number of consecutively rejected terms.
+   * Depending on {@link #NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD}, this may trigger a jump to a block away.
    */
-  protected boolean beyondCommonPrefix;
+  protected int numConsecutivelyRejectedTerms;
 
-  public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
-                              IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
-                              FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
+  protected IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
+                                 IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput,
+                                 PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
+                                 BlockDecoder blockDecoder) throws IOException {
     super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
-    this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);
-    Automaton automaton = Objects.requireNonNull(compiled.automaton);
-    this.runAutomaton = Objects.requireNonNull(compiled.runAutomaton);
-    this.commonSuffixRef = compiled.commonSuffixRef; // maybe null
-    this.commonPrefixRef = Operations.getCommonPrefixBytesRef(automaton); // never null
-
-    this.startTerm = startTerm;
-    assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef);
-    // it is thus also true that startTerm >= commonPrefixRef
-
-    this.seekTerm = startTerm != null ? startTerm : commonPrefixRef;
+    checkIntersectAutomatonType(compiled);
+    automaton = compiled.automaton;
+    runAutomaton = compiled.runAutomaton;
+    finite = compiled.finite;
+    commonSuffix = compiled.commonSuffixRef;
+    minTermLength = getMinTermLength();
+    nextStringCalculator = new AutomatonNextTermCalculator(compiled);
+    seekTerm = startTerm;
   }
 
-  @Override
-  public BytesRef next() throws IOException {
-    clearTermState();
-
-    if (blockHeader == null) { // initial state
-      // note: don't call super.seekCeil here; we have our own logic
-
-      // Set the browser position to the block having the seek term.
-      // Even if -1, it's okay since we'll soon call nextKey().
-      long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
-      if (isBeyondLastTerm(seekTerm, blockStartFP)) {
-        return null; // EOF
-      }
-
-      // Starting at this block find and load the next matching block.
-      //   note: Since seekBlock was just called, we actually consider the current block as "next".
-      if (nextBlockMatchingPrefix() == false) { // note: starts at seek'ed block, which may have a match
-        return null; // EOF
-      }
+  protected void checkIntersectAutomatonType(CompiledAutomaton compiled) {
+    if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+      throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
     }
-
-    do {
-
-      // look in the rest of this block.
-      BytesRef term = nextTermInBlockMatching();
-      if (term != null) {
-        return term;
-      }
-
-      // next term dict matching prefix
-    } while (nextBlockMatchingPrefix());
-
-    return null; // EOF
   }
 
   /**
-   * Find the next block that appears to contain terms that could match the automata.
-   * The prefix is the primary clue.  Returns true if at one, or false for no more (EOF).
+   * Computes the minimal length of the terms accepted by the automaton.
+   * This speeds up the term scanning for automatons accepting a finite language.
    */
-  protected boolean nextBlockMatchingPrefix() throws IOException {
-    if (beyondCommonPrefix) {
-      return false; // EOF
+  protected int getMinTermLength() {
+    // Automatons accepting infinite language (e.g. PrefixQuery and WildcardQuery) do not benefit much from
+    // min term length while it takes time to compute it. More precisely, by skipping this computation PrefixQuery
+    // is significantly boosted while WildcardQuery might be slightly degraded on average. This min term length
+    // mainly boosts FuzzyQuery.
+    int commonSuffixLength = commonSuffix == null ? 0 : commonSuffix.length;
+    if (!finite) {
+      return commonSuffixLength;
     }
-
-    IndexDictionary.Browser browser = getOrCreateDictionaryBrowser();
-
-    do {
-
-      // Get next block key (becomes in effect the current blockKey)
-      BytesRef blockKey = browser.nextKey();
-      if (blockKey == null) {
-        return false; // EOF
-      }
-
-      blockPrefixLen = browser.getBlockPrefixLen();
-      blockPrefixRunAutomatonState = runAutomatonForState(blockKey.bytes, blockKey.offset, blockPrefixLen, 0);
-
-      // We may have passed commonPrefix  (a short-circuit optimization).
-      if (isBeyondCommonPrefix(blockKey)) {
-        return false; // EOF
+    int commonPrefixLength = 0;
+    int state = 0;
+    Transition t = null;
+    int[] visited = null;
+    int visitedLength = 0;
+    while (true) {
+      if (runAutomaton.isAccept(state)) {
+        // The common prefix reaches a final state. So common prefix and common suffix overlap.
+        // Min term length is the max between common prefix and common suffix lengths.
+        return Math.max(commonPrefixLength, commonSuffixLength);
       }
-
-      if (blockPrefixRunAutomatonState >= 0) {
-        break; // a match
+      if (automaton.getNumTransitions(state) == 1) {
+        if (t == null) {
+          t = new Transition();
+        }
+        automaton.getTransition(state, 0, t);
+        if (t.min == t.max) {
+          state = t.dest;
+          if (visited == null) {
+            visited = new int[8];
+            visited[visitedLength++] = 0;
+          }
+          if (!contains(visited, visitedLength, state)) {
+            visited = ArrayUtil.grow(visited, ++visitedLength);
+            visited[visitedLength - 1] = state;
+            commonPrefixLength++;
+            continue;
+          }
+        }
       }
+      break;
+    }
+    // Min term length is the sum of common prefix and common suffix lengths.
+    return commonPrefixLength + commonSuffixLength;
+  }
 
-      //
-      // This block doesn't match.
-      //
-
-      seekTerm = null; // we're moving on to another block, and seekTerm is before it.
-
-      // Should we simply get the next key (linear mode) or try to seek?
-      if (nextStringCalculator.isLinearState(blockKey)) {
-        continue;
+  private static boolean contains(int[] ints, int length, int value) {
+    assert length <= ints.length;
+    for (int i = 0; i < length; i++) {
+      if (ints[i] == value) {
+        return true;
       }
+    }
+    return false;
+  }
 
-      // Maybe the next block's key matches?  We have to check this before calling nextStringCalculator.
-      BytesRef peekKey = browser.peekKey();
-      if (peekKey == null) {
-        return false; // EOF
-      }
-      if (runAutomatonForState(peekKey.bytes, peekKey.offset, peekKey.length, 0) >= 0) {
-        continue; // yay; it matched.  Continue to actually advance to it.  This is rare?
+  @Override
+  public BytesRef next() throws IOException {
+    if (blockHeader == null) {
+      if (!seekFirstBlock()) {
+        return null;
       }
-
-      // Seek to a block by calculating the next term to match the automata *following* peekKey.
-      this.seekTerm = nextStringCalculator.nextSeekTerm(browser.peekKey());
-      if (seekTerm == null) {
-        return false; // EOF
+      states = new int[32];
+      blockIteration = BlockIteration.NEXT;
+    }
+    termState = null;
+    do {
+      BytesRef term = nextTermInBlockMatching();
+      if (term != null) {
+        return term;
       }
-      browser.seekBlock(seekTerm);
-      //continue
-
-    } while (true); // while not a match
-
-    // A match!
-
-    //NOTE: we could determine if this automata has a prefix for this specific block (longer than the commonPrefix).
-    //  If we see it, we could set it as the seekTerm and we could also exit the block early if we get past this prefix
-    //  and runAutomatonFromPrefix would start from this prefix.  Smiley tried but benchmarks were not favorable to it.
-
-    initializeHeader(null, browser.getBlockFilePointer());
+    } while (nextBlock());
+    return null;
+  }
 
-    return true;
+  protected boolean seekFirstBlock() throws IOException {
+    seekTerm = nextStringCalculator.nextSeekTerm(seekTerm);
+    if (seekTerm == null) {
+      return false;
+    }
+    long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
+    if (blockStartFP == -1) {
+      blockStartFP = fieldMetadata.getFirstBlockStartFP();
+    } else if (isBeyondLastTerm(seekTerm, blockStartFP)) {
+      return false;
+    }
+    initializeHeader(seekTerm, blockStartFP);
+    return blockHeader != null;
   }
 
   /**
-   * Find the next block line that matches, or null when at end of block.
+   * Finds the next block line that matches (accepted by the automaton), or null when at end of block.
+   *
+   * @return The next term in the current block that is accepted by the automaton; or null if none.
    */
   protected BytesRef nextTermInBlockMatching() throws IOException {
-    do {
-      // if seekTerm is set, then we seek into this block instead of starting with the first blindly.
-      if (seekTerm != null) {
-        assert blockLine == null;
-        boolean moveBeyondIfFound = seekTerm == startTerm; // for startTerm, we want to get the following term
-        SeekStatus seekStatus = seekInBlock(seekTerm);
-        seekTerm = null;// reset.
-        if (seekStatus == SeekStatus.END) {
-          return null;
-        } else if (seekStatus == SeekStatus.FOUND && moveBeyondIfFound) {
-          if (readLineInBlock() == null) {
-            return null;
+    if (seekTerm == null) {
+      if (readLineInBlock() == null) {
+        return null;
+      }
+    } else {
+      SeekStatus seekStatus = seekInBlock(seekTerm);
+      seekTerm = null;
+      if (seekStatus == SeekStatus.END) {
+        return null;
+      }
+      assert numMatchedBytes == 0;
+      assert numConsecutivelyRejectedTerms == 0;
+    }
+    int[] states = this.states;
+    while (true) {
+      TermBytes lineTermBytes = blockLine.getTermBytes();
+      BytesRef lineTerm = lineTermBytes.getTerm();
+      assert lineTerm.offset == 0;
+      byte[] termBytes = lineTerm.bytes;
+      int termLength = lineTerm.length;
 
 Review comment:
   Oh that's why; I thought it was readability.  I'm fine either way.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r383990057
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java
 ##########
 @@ -18,260 +18,337 @@
 package org.apache.lucene.codecs.uniformsplit;
 
 import java.io.IOException;
-import java.util.Objects;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
  * The "intersect" {@link TermsEnum} response to {@link UniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
  * intersecting the terms with an automaton.
+ * <p>
+ * By design of the UniformSplit block keys, it is less efficient than
+ * {@code org.apache.lucene.codecs.blocktree.IntersectTermsEnum} for {@link org.apache.lucene.search.FuzzyQuery} (-37%).
+ * It is slightly slower for {@link org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for
+ * {@link org.apache.lucene.search.PrefixQuery} (+5%).
+ *
+ * @lucene.experimental
  */
 public class IntersectBlockReader extends BlockReader {
 
-  protected final AutomatonNextTermCalculator nextStringCalculator;
-  protected final ByteRunAutomaton runAutomaton;
-  protected final BytesRef commonSuffixRef; // maybe null
-  protected final BytesRef commonPrefixRef;
-  protected final BytesRef startTerm; // maybe null
+  /**
+   * Block iteration order. Whether to move next block, jump to a block away, or end the iteration.
+   */
+  protected enum BlockIteration {NEXT, SEEK, END}
 
-  /** Set this when our current mode is seeking to this term.  Set to null after. */
-  protected BytesRef seekTerm;
+  /**
+   * Threshold that controls when to attempt to jump to a block away.
+   * <p>
+   * This counter is 0 when entering a block. It is incremented each time a term is rejected by the automaton.
+   * When the counter is greater than or equal to this threshold, then we compute the next term accepted by
+   * the automaton, with {@link AutomatonNextTermCalculator}, and we jump to a block away if the next term
+   * accepted is greater than the immediate next term in the block.
+   * <p>
+   * A low value, for example 1, improves the performance of automatons requiring many jumps, for example
+   * {@link org.apache.lucene.search.FuzzyQuery} and most {@link org.apache.lucene.search.WildcardQuery}.
+   * A higher value improves the performance of automatons with less or no jump, for example
+   * {@link org.apache.lucene.search.PrefixQuery}.
+   * A threshold of 4 seems to be a good balance.
+   */
+  protected final int NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD = 4;
 
-  protected int blockPrefixRunAutomatonState;
-  protected int blockPrefixLen;
+  protected final Automaton automaton;
+  protected final ByteRunAutomaton runAutomaton;
+  protected final boolean finite;
+  protected final BytesRef commonSuffix; // maybe null
+  protected final int minTermLength;
+  protected final AutomatonNextTermCalculator nextStringCalculator;
 
   /**
-   * Number of bytes accepted by the last call to {@link #runAutomatonForState}.
+   * Set this when our current mode is seeking to this term.  Set to null after.
+   */
+  protected BytesRef seekTerm;
+  /**
+   * Number of bytes accepted by the automaton when validating the current term.
+   */
+  protected int numMatchedBytes;
+  /**
+   * Automaton states reached when validating the current term, from 0 to {@link #numMatchedBytes} - 1.
+   */
+  protected int[] states;
+  /**
+   * Block iteration order determined when scanning the terms in the current block.
    */
-  protected int numBytesAccepted;
+  protected BlockIteration blockIteration;
   /**
-   * Whether the current term is beyond the automaton common prefix.
-   * If true this means the enumeration should stop immediately.
+   * Counter of the number of consecutively rejected terms.
+   * Depending on {@link #NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD}, this may trigger a jump to a block away.
    */
-  protected boolean beyondCommonPrefix;
+  protected int numConsecutivelyRejectedTerms;
 
-  public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
-                              IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
-                              FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
+  protected IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
+                                 IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput,
+                                 PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
+                                 BlockDecoder blockDecoder) throws IOException {
     super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
-    this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);
-    Automaton automaton = Objects.requireNonNull(compiled.automaton);
-    this.runAutomaton = Objects.requireNonNull(compiled.runAutomaton);
-    this.commonSuffixRef = compiled.commonSuffixRef; // maybe null
-    this.commonPrefixRef = Operations.getCommonPrefixBytesRef(automaton); // never null
-
-    this.startTerm = startTerm;
-    assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef);
-    // it is thus also true that startTerm >= commonPrefixRef
-
-    this.seekTerm = startTerm != null ? startTerm : commonPrefixRef;
+    checkIntersectAutomatonType(compiled);
+    automaton = compiled.automaton;
+    runAutomaton = compiled.runAutomaton;
+    finite = compiled.finite;
+    commonSuffix = compiled.commonSuffixRef;
+    minTermLength = getMinTermLength();
+    nextStringCalculator = new AutomatonNextTermCalculator(compiled);
+    seekTerm = startTerm;
   }
 
-  @Override
-  public BytesRef next() throws IOException {
-    clearTermState();
-
-    if (blockHeader == null) { // initial state
-      // note: don't call super.seekCeil here; we have our own logic
-
-      // Set the browser position to the block having the seek term.
-      // Even if -1, it's okay since we'll soon call nextKey().
-      long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
-      if (isBeyondLastTerm(seekTerm, blockStartFP)) {
-        return null; // EOF
-      }
-
-      // Starting at this block find and load the next matching block.
-      //   note: Since seekBlock was just called, we actually consider the current block as "next".
-      if (nextBlockMatchingPrefix() == false) { // note: starts at seek'ed block, which may have a match
-        return null; // EOF
-      }
+  protected void checkIntersectAutomatonType(CompiledAutomaton compiled) {
+    if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+      throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
     }
-
-    do {
-
-      // look in the rest of this block.
-      BytesRef term = nextTermInBlockMatching();
-      if (term != null) {
-        return term;
-      }
-
-      // next term dict matching prefix
-    } while (nextBlockMatchingPrefix());
-
-    return null; // EOF
   }
 
   /**
-   * Find the next block that appears to contain terms that could match the automata.
-   * The prefix is the primary clue.  Returns true if at one, or false for no more (EOF).
+   * Computes the minimal length of the terms accepted by the automaton.
+   * This speeds up the term scanning for automatons accepting a finite language.
    */
-  protected boolean nextBlockMatchingPrefix() throws IOException {
-    if (beyondCommonPrefix) {
-      return false; // EOF
+  protected int getMinTermLength() {
 
 Review comment:
   BlockTree is not the only consumer of automatons.  I've used them in the UnifiedHighligher as well for highlighting automata/MTQs in a place that processes a TokenStream to see which tokens match the query.  Perhaps a quick minimum length check there would be beneficial.  I don't want to suggest any integration work here for you, only the idea that this utility method might have value elsewhere.  On the other hand, perhaps it should not be moved to a general place until there is another use.  
   Also, maybe this method ought to have a dedicated test?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] bruno-roustant commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
bruno-roustant commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r383748557
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java
 ##########
 @@ -18,260 +18,337 @@
 package org.apache.lucene.codecs.uniformsplit;
 
 import java.io.IOException;
-import java.util.Objects;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
  * The "intersect" {@link TermsEnum} response to {@link UniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
  * intersecting the terms with an automaton.
+ * <p>
+ * By design of the UniformSplit block keys, it is less efficient than
+ * {@code org.apache.lucene.codecs.blocktree.IntersectTermsEnum} for {@link org.apache.lucene.search.FuzzyQuery} (-37%).
+ * It is slightly slower for {@link org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for
+ * {@link org.apache.lucene.search.PrefixQuery} (+5%).
+ *
+ * @lucene.experimental
  */
 public class IntersectBlockReader extends BlockReader {
 
-  protected final AutomatonNextTermCalculator nextStringCalculator;
-  protected final ByteRunAutomaton runAutomaton;
-  protected final BytesRef commonSuffixRef; // maybe null
-  protected final BytesRef commonPrefixRef;
-  protected final BytesRef startTerm; // maybe null
+  /**
+   * Block iteration order. Whether to move next block, jump to a block away, or end the iteration.
+   */
+  protected enum BlockIteration {NEXT, SEEK, END}
 
-  /** Set this when our current mode is seeking to this term.  Set to null after. */
-  protected BytesRef seekTerm;
+  /**
+   * Threshold that controls when to attempt to jump to a block away.
+   * <p>
+   * This counter is 0 when entering a block. It is incremented each time a term is rejected by the automaton.
+   * When the counter is greater than or equal to this threshold, then we compute the next term accepted by
+   * the automaton, with {@link AutomatonNextTermCalculator}, and we jump to a block away if the next term
+   * accepted is greater than the immediate next term in the block.
+   * <p>
+   * A low value, for example 1, improves the performance of automatons requiring many jumps, for example
+   * {@link org.apache.lucene.search.FuzzyQuery} and most {@link org.apache.lucene.search.WildcardQuery}.
+   * A higher value improves the performance of automatons with less or no jump, for example
+   * {@link org.apache.lucene.search.PrefixQuery}.
+   * A threshold of 4 seems to be a good balance.
+   */
+  protected final int NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD = 4;
 
-  protected int blockPrefixRunAutomatonState;
-  protected int blockPrefixLen;
+  protected final Automaton automaton;
+  protected final ByteRunAutomaton runAutomaton;
+  protected final boolean finite;
+  protected final BytesRef commonSuffix; // maybe null
+  protected final int minTermLength;
+  protected final AutomatonNextTermCalculator nextStringCalculator;
 
   /**
-   * Number of bytes accepted by the last call to {@link #runAutomatonForState}.
+   * Set this when our current mode is seeking to this term.  Set to null after.
+   */
+  protected BytesRef seekTerm;
+  /**
+   * Number of bytes accepted by the automaton when validating the current term.
+   */
+  protected int numMatchedBytes;
+  /**
+   * Automaton states reached when validating the current term, from 0 to {@link #numMatchedBytes} - 1.
+   */
+  protected int[] states;
+  /**
+   * Block iteration order determined when scanning the terms in the current block.
    */
-  protected int numBytesAccepted;
+  protected BlockIteration blockIteration;
   /**
-   * Whether the current term is beyond the automaton common prefix.
-   * If true this means the enumeration should stop immediately.
+   * Counter of the number of consecutively rejected terms.
+   * Depending on {@link #NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD}, this may trigger a jump to a block away.
    */
-  protected boolean beyondCommonPrefix;
+  protected int numConsecutivelyRejectedTerms;
 
-  public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
-                              IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
-                              FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
+  protected IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
+                                 IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput,
+                                 PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
+                                 BlockDecoder blockDecoder) throws IOException {
     super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
-    this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);
-    Automaton automaton = Objects.requireNonNull(compiled.automaton);
-    this.runAutomaton = Objects.requireNonNull(compiled.runAutomaton);
-    this.commonSuffixRef = compiled.commonSuffixRef; // maybe null
-    this.commonPrefixRef = Operations.getCommonPrefixBytesRef(automaton); // never null
-
-    this.startTerm = startTerm;
-    assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef);
-    // it is thus also true that startTerm >= commonPrefixRef
-
-    this.seekTerm = startTerm != null ? startTerm : commonPrefixRef;
+    checkIntersectAutomatonType(compiled);
+    automaton = compiled.automaton;
+    runAutomaton = compiled.runAutomaton;
+    finite = compiled.finite;
+    commonSuffix = compiled.commonSuffixRef;
+    minTermLength = getMinTermLength();
+    nextStringCalculator = new AutomatonNextTermCalculator(compiled);
+    seekTerm = startTerm;
   }
 
-  @Override
-  public BytesRef next() throws IOException {
-    clearTermState();
-
-    if (blockHeader == null) { // initial state
-      // note: don't call super.seekCeil here; we have our own logic
-
-      // Set the browser position to the block having the seek term.
-      // Even if -1, it's okay since we'll soon call nextKey().
-      long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
-      if (isBeyondLastTerm(seekTerm, blockStartFP)) {
-        return null; // EOF
-      }
-
-      // Starting at this block find and load the next matching block.
-      //   note: Since seekBlock was just called, we actually consider the current block as "next".
-      if (nextBlockMatchingPrefix() == false) { // note: starts at seek'ed block, which may have a match
-        return null; // EOF
-      }
+  protected void checkIntersectAutomatonType(CompiledAutomaton compiled) {
+    if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+      throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
     }
-
-    do {
-
-      // look in the rest of this block.
-      BytesRef term = nextTermInBlockMatching();
-      if (term != null) {
-        return term;
-      }
-
-      // next term dict matching prefix
-    } while (nextBlockMatchingPrefix());
-
-    return null; // EOF
   }
 
   /**
-   * Find the next block that appears to contain terms that could match the automata.
-   * The prefix is the primary clue.  Returns true if at one, or false for no more (EOF).
+   * Computes the minimal length of the terms accepted by the automaton.
+   * This speeds up the term scanning for automatons accepting a finite language.
    */
-  protected boolean nextBlockMatchingPrefix() throws IOException {
-    if (beyondCommonPrefix) {
-      return false; // EOF
+  protected int getMinTermLength() {
+    // Automatons accepting infinite language (e.g. PrefixQuery and WildcardQuery) do not benefit much from
+    // min term length while it takes time to compute it. More precisely, by skipping this computation PrefixQuery
+    // is significantly boosted while WildcardQuery might be slightly degraded on average. This min term length
+    // mainly boosts FuzzyQuery.
+    int commonSuffixLength = commonSuffix == null ? 0 : commonSuffix.length;
+    if (!finite) {
+      return commonSuffixLength;
     }
-
-    IndexDictionary.Browser browser = getOrCreateDictionaryBrowser();
-
-    do {
-
-      // Get next block key (becomes in effect the current blockKey)
-      BytesRef blockKey = browser.nextKey();
-      if (blockKey == null) {
-        return false; // EOF
-      }
-
-      blockPrefixLen = browser.getBlockPrefixLen();
-      blockPrefixRunAutomatonState = runAutomatonForState(blockKey.bytes, blockKey.offset, blockPrefixLen, 0);
-
-      // We may have passed commonPrefix  (a short-circuit optimization).
-      if (isBeyondCommonPrefix(blockKey)) {
-        return false; // EOF
+    int commonPrefixLength = 0;
+    int state = 0;
+    Transition t = null;
+    int[] visited = null;
+    int visitedLength = 0;
+    while (true) {
+      if (runAutomaton.isAccept(state)) {
+        // The common prefix reaches a final state. So common prefix and common suffix overlap.
+        // Min term length is the max between common prefix and common suffix lengths.
+        return Math.max(commonPrefixLength, commonSuffixLength);
       }
-
-      if (blockPrefixRunAutomatonState >= 0) {
-        break; // a match
+      if (automaton.getNumTransitions(state) == 1) {
+        if (t == null) {
+          t = new Transition();
+        }
+        automaton.getTransition(state, 0, t);
+        if (t.min == t.max) {
+          state = t.dest;
+          if (visited == null) {
+            visited = new int[8];
+            visited[visitedLength++] = 0;
+          }
+          if (!contains(visited, visitedLength, state)) {
+            visited = ArrayUtil.grow(visited, ++visitedLength);
+            visited[visitedLength - 1] = state;
+            commonPrefixLength++;
+            continue;
+          }
+        }
       }
+      break;
+    }
+    // Min term length is the sum of common prefix and common suffix lengths.
+    return commonPrefixLength + commonSuffixLength;
+  }
 
-      //
-      // This block doesn't match.
-      //
-
-      seekTerm = null; // we're moving on to another block, and seekTerm is before it.
-
-      // Should we simply get the next key (linear mode) or try to seek?
-      if (nextStringCalculator.isLinearState(blockKey)) {
-        continue;
+  private static boolean contains(int[] ints, int length, int value) {
+    assert length <= ints.length;
+    for (int i = 0; i < length; i++) {
+      if (ints[i] == value) {
+        return true;
       }
+    }
+    return false;
+  }
 
-      // Maybe the next block's key matches?  We have to check this before calling nextStringCalculator.
-      BytesRef peekKey = browser.peekKey();
-      if (peekKey == null) {
-        return false; // EOF
-      }
-      if (runAutomatonForState(peekKey.bytes, peekKey.offset, peekKey.length, 0) >= 0) {
-        continue; // yay; it matched.  Continue to actually advance to it.  This is rare?
+  @Override
+  public BytesRef next() throws IOException {
+    if (blockHeader == null) {
+      if (!seekFirstBlock()) {
+        return null;
       }
-
-      // Seek to a block by calculating the next term to match the automata *following* peekKey.
-      this.seekTerm = nextStringCalculator.nextSeekTerm(browser.peekKey());
-      if (seekTerm == null) {
-        return false; // EOF
+      states = new int[32];
+      blockIteration = BlockIteration.NEXT;
+    }
+    termState = null;
+    do {
+      BytesRef term = nextTermInBlockMatching();
+      if (term != null) {
+        return term;
       }
-      browser.seekBlock(seekTerm);
-      //continue
-
-    } while (true); // while not a match
-
-    // A match!
-
-    //NOTE: we could determine if this automata has a prefix for this specific block (longer than the commonPrefix).
-    //  If we see it, we could set it as the seekTerm and we could also exit the block early if we get past this prefix
-    //  and runAutomatonFromPrefix would start from this prefix.  Smiley tried but benchmarks were not favorable to it.
-
-    initializeHeader(null, browser.getBlockFilePointer());
+    } while (nextBlock());
+    return null;
+  }
 
-    return true;
+  protected boolean seekFirstBlock() throws IOException {
+    seekTerm = nextStringCalculator.nextSeekTerm(seekTerm);
+    if (seekTerm == null) {
+      return false;
+    }
+    long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
+    if (blockStartFP == -1) {
+      blockStartFP = fieldMetadata.getFirstBlockStartFP();
+    } else if (isBeyondLastTerm(seekTerm, blockStartFP)) {
+      return false;
+    }
+    initializeHeader(seekTerm, blockStartFP);
+    return blockHeader != null;
   }
 
   /**
-   * Find the next block line that matches, or null when at end of block.
+   * Finds the next block line that matches (accepted by the automaton), or null when at end of block.
+   *
+   * @return The next term in the current block that is accepted by the automaton; or null if none.
    */
   protected BytesRef nextTermInBlockMatching() throws IOException {
-    do {
-      // if seekTerm is set, then we seek into this block instead of starting with the first blindly.
-      if (seekTerm != null) {
-        assert blockLine == null;
-        boolean moveBeyondIfFound = seekTerm == startTerm; // for startTerm, we want to get the following term
-        SeekStatus seekStatus = seekInBlock(seekTerm);
-        seekTerm = null;// reset.
-        if (seekStatus == SeekStatus.END) {
-          return null;
-        } else if (seekStatus == SeekStatus.FOUND && moveBeyondIfFound) {
-          if (readLineInBlock() == null) {
-            return null;
+    if (seekTerm == null) {
+      if (readLineInBlock() == null) {
+        return null;
+      }
+    } else {
+      SeekStatus seekStatus = seekInBlock(seekTerm);
+      seekTerm = null;
+      if (seekStatus == SeekStatus.END) {
+        return null;
+      }
+      assert numMatchedBytes == 0;
+      assert numConsecutivelyRejectedTerms == 0;
+    }
+    int[] states = this.states;
+    while (true) {
+      TermBytes lineTermBytes = blockLine.getTermBytes();
+      BytesRef lineTerm = lineTermBytes.getTerm();
+      assert lineTerm.offset == 0;
+      byte[] termBytes = lineTerm.bytes;
+      int termLength = lineTerm.length;
 
 Review comment:
   Since those fields are not final, it is (slightly) faster to have them a local variables when they are manipulated in a highly repeating loop. But I agree this is over-optimization. +1 I'll remove them.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r383546038
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java
 ##########
 @@ -18,260 +18,337 @@
 package org.apache.lucene.codecs.uniformsplit;
 
 import java.io.IOException;
-import java.util.Objects;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
  * The "intersect" {@link TermsEnum} response to {@link UniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
  * intersecting the terms with an automaton.
+ * <p>
+ * By design of the UniformSplit block keys, it is less efficient than
+ * {@code org.apache.lucene.codecs.blocktree.IntersectTermsEnum} for {@link org.apache.lucene.search.FuzzyQuery} (-37%).
+ * It is slightly slower for {@link org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for
+ * {@link org.apache.lucene.search.PrefixQuery} (+5%).
+ *
+ * @lucene.experimental
  */
 public class IntersectBlockReader extends BlockReader {
 
-  protected final AutomatonNextTermCalculator nextStringCalculator;
-  protected final ByteRunAutomaton runAutomaton;
-  protected final BytesRef commonSuffixRef; // maybe null
-  protected final BytesRef commonPrefixRef;
-  protected final BytesRef startTerm; // maybe null
+  /**
+   * Block iteration order. Whether to move next block, jump to a block away, or end the iteration.
+   */
+  protected enum BlockIteration {NEXT, SEEK, END}
 
-  /** Set this when our current mode is seeking to this term.  Set to null after. */
-  protected BytesRef seekTerm;
+  /**
+   * Threshold that controls when to attempt to jump to a block away.
+   * <p>
+   * This counter is 0 when entering a block. It is incremented each time a term is rejected by the automaton.
+   * When the counter is greater than or equal to this threshold, then we compute the next term accepted by
+   * the automaton, with {@link AutomatonNextTermCalculator}, and we jump to a block away if the next term
+   * accepted is greater than the immediate next term in the block.
+   * <p>
+   * A low value, for example 1, improves the performance of automatons requiring many jumps, for example
+   * {@link org.apache.lucene.search.FuzzyQuery} and most {@link org.apache.lucene.search.WildcardQuery}.
+   * A higher value improves the performance of automatons with less or no jump, for example
+   * {@link org.apache.lucene.search.PrefixQuery}.
+   * A threshold of 4 seems to be a good balance.
+   */
+  protected final int NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD = 4;
 
-  protected int blockPrefixRunAutomatonState;
-  protected int blockPrefixLen;
+  protected final Automaton automaton;
+  protected final ByteRunAutomaton runAutomaton;
+  protected final boolean finite;
+  protected final BytesRef commonSuffix; // maybe null
+  protected final int minTermLength;
+  protected final AutomatonNextTermCalculator nextStringCalculator;
 
   /**
-   * Number of bytes accepted by the last call to {@link #runAutomatonForState}.
+   * Set this when our current mode is seeking to this term.  Set to null after.
+   */
+  protected BytesRef seekTerm;
+  /**
+   * Number of bytes accepted by the automaton when validating the current term.
+   */
+  protected int numMatchedBytes;
+  /**
+   * Automaton states reached when validating the current term, from 0 to {@link #numMatchedBytes} - 1.
+   */
+  protected int[] states;
+  /**
+   * Block iteration order determined when scanning the terms in the current block.
    */
-  protected int numBytesAccepted;
+  protected BlockIteration blockIteration;
   /**
-   * Whether the current term is beyond the automaton common prefix.
-   * If true this means the enumeration should stop immediately.
+   * Counter of the number of consecutively rejected terms.
+   * Depending on {@link #NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD}, this may trigger a jump to a block away.
    */
-  protected boolean beyondCommonPrefix;
+  protected int numConsecutivelyRejectedTerms;
 
-  public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
-                              IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
-                              FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
+  protected IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
+                                 IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput,
+                                 PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
+                                 BlockDecoder blockDecoder) throws IOException {
     super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
-    this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);
-    Automaton automaton = Objects.requireNonNull(compiled.automaton);
-    this.runAutomaton = Objects.requireNonNull(compiled.runAutomaton);
-    this.commonSuffixRef = compiled.commonSuffixRef; // maybe null
-    this.commonPrefixRef = Operations.getCommonPrefixBytesRef(automaton); // never null
-
-    this.startTerm = startTerm;
-    assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef);
-    // it is thus also true that startTerm >= commonPrefixRef
-
-    this.seekTerm = startTerm != null ? startTerm : commonPrefixRef;
+    checkIntersectAutomatonType(compiled);
+    automaton = compiled.automaton;
+    runAutomaton = compiled.runAutomaton;
+    finite = compiled.finite;
+    commonSuffix = compiled.commonSuffixRef;
+    minTermLength = getMinTermLength();
+    nextStringCalculator = new AutomatonNextTermCalculator(compiled);
+    seekTerm = startTerm;
   }
 
-  @Override
-  public BytesRef next() throws IOException {
-    clearTermState();
-
-    if (blockHeader == null) { // initial state
-      // note: don't call super.seekCeil here; we have our own logic
-
-      // Set the browser position to the block having the seek term.
-      // Even if -1, it's okay since we'll soon call nextKey().
-      long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
-      if (isBeyondLastTerm(seekTerm, blockStartFP)) {
-        return null; // EOF
-      }
-
-      // Starting at this block find and load the next matching block.
-      //   note: Since seekBlock was just called, we actually consider the current block as "next".
-      if (nextBlockMatchingPrefix() == false) { // note: starts at seek'ed block, which may have a match
-        return null; // EOF
-      }
+  protected void checkIntersectAutomatonType(CompiledAutomaton compiled) {
+    if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+      throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
     }
-
-    do {
-
-      // look in the rest of this block.
-      BytesRef term = nextTermInBlockMatching();
-      if (term != null) {
-        return term;
-      }
-
-      // next term dict matching prefix
-    } while (nextBlockMatchingPrefix());
-
-    return null; // EOF
   }
 
   /**
-   * Find the next block that appears to contain terms that could match the automata.
-   * The prefix is the primary clue.  Returns true if at one, or false for no more (EOF).
+   * Computes the minimal length of the terms accepted by the automaton.
+   * This speeds up the term scanning for automatons accepting a finite language.
    */
-  protected boolean nextBlockMatchingPrefix() throws IOException {
-    if (beyondCommonPrefix) {
-      return false; // EOF
+  protected int getMinTermLength() {
+    // Automatons accepting infinite language (e.g. PrefixQuery and WildcardQuery) do not benefit much from
+    // min term length while it takes time to compute it. More precisely, by skipping this computation PrefixQuery
+    // is significantly boosted while WildcardQuery might be slightly degraded on average. This min term length
+    // mainly boosts FuzzyQuery.
+    int commonSuffixLength = commonSuffix == null ? 0 : commonSuffix.length;
+    if (!finite) {
+      return commonSuffixLength;
     }
-
-    IndexDictionary.Browser browser = getOrCreateDictionaryBrowser();
-
-    do {
-
-      // Get next block key (becomes in effect the current blockKey)
-      BytesRef blockKey = browser.nextKey();
-      if (blockKey == null) {
-        return false; // EOF
-      }
-
-      blockPrefixLen = browser.getBlockPrefixLen();
-      blockPrefixRunAutomatonState = runAutomatonForState(blockKey.bytes, blockKey.offset, blockPrefixLen, 0);
-
-      // We may have passed commonPrefix  (a short-circuit optimization).
-      if (isBeyondCommonPrefix(blockKey)) {
-        return false; // EOF
+    int commonPrefixLength = 0;
+    int state = 0;
+    Transition t = null;
+    int[] visited = null;
+    int visitedLength = 0;
+    while (true) {
+      if (runAutomaton.isAccept(state)) {
+        // The common prefix reaches a final state. So common prefix and common suffix overlap.
+        // Min term length is the max between common prefix and common suffix lengths.
+        return Math.max(commonPrefixLength, commonSuffixLength);
       }
-
-      if (blockPrefixRunAutomatonState >= 0) {
-        break; // a match
+      if (automaton.getNumTransitions(state) == 1) {
+        if (t == null) {
+          t = new Transition();
+        }
+        automaton.getTransition(state, 0, t);
+        if (t.min == t.max) {
+          state = t.dest;
+          if (visited == null) {
+            visited = new int[8];
+            visited[visitedLength++] = 0;
+          }
+          if (!contains(visited, visitedLength, state)) {
+            visited = ArrayUtil.grow(visited, ++visitedLength);
+            visited[visitedLength - 1] = state;
+            commonPrefixLength++;
+            continue;
+          }
+        }
       }
+      break;
+    }
+    // Min term length is the sum of common prefix and common suffix lengths.
+    return commonPrefixLength + commonSuffixLength;
+  }
 
-      //
-      // This block doesn't match.
-      //
-
-      seekTerm = null; // we're moving on to another block, and seekTerm is before it.
-
-      // Should we simply get the next key (linear mode) or try to seek?
-      if (nextStringCalculator.isLinearState(blockKey)) {
-        continue;
+  private static boolean contains(int[] ints, int length, int value) {
+    assert length <= ints.length;
+    for (int i = 0; i < length; i++) {
+      if (ints[i] == value) {
+        return true;
       }
+    }
+    return false;
+  }
 
-      // Maybe the next block's key matches?  We have to check this before calling nextStringCalculator.
-      BytesRef peekKey = browser.peekKey();
-      if (peekKey == null) {
-        return false; // EOF
-      }
-      if (runAutomatonForState(peekKey.bytes, peekKey.offset, peekKey.length, 0) >= 0) {
-        continue; // yay; it matched.  Continue to actually advance to it.  This is rare?
+  @Override
+  public BytesRef next() throws IOException {
+    if (blockHeader == null) {
+      if (!seekFirstBlock()) {
+        return null;
       }
-
-      // Seek to a block by calculating the next term to match the automata *following* peekKey.
-      this.seekTerm = nextStringCalculator.nextSeekTerm(browser.peekKey());
-      if (seekTerm == null) {
-        return false; // EOF
+      states = new int[32];
+      blockIteration = BlockIteration.NEXT;
+    }
+    termState = null;
+    do {
+      BytesRef term = nextTermInBlockMatching();
+      if (term != null) {
+        return term;
       }
-      browser.seekBlock(seekTerm);
-      //continue
-
-    } while (true); // while not a match
-
-    // A match!
-
-    //NOTE: we could determine if this automata has a prefix for this specific block (longer than the commonPrefix).
-    //  If we see it, we could set it as the seekTerm and we could also exit the block early if we get past this prefix
-    //  and runAutomatonFromPrefix would start from this prefix.  Smiley tried but benchmarks were not favorable to it.
-
-    initializeHeader(null, browser.getBlockFilePointer());
+    } while (nextBlock());
+    return null;
+  }
 
-    return true;
+  protected boolean seekFirstBlock() throws IOException {
+    seekTerm = nextStringCalculator.nextSeekTerm(seekTerm);
+    if (seekTerm == null) {
+      return false;
+    }
+    long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
+    if (blockStartFP == -1) {
+      blockStartFP = fieldMetadata.getFirstBlockStartFP();
+    } else if (isBeyondLastTerm(seekTerm, blockStartFP)) {
+      return false;
+    }
+    initializeHeader(seekTerm, blockStartFP);
+    return blockHeader != null;
   }
 
   /**
-   * Find the next block line that matches, or null when at end of block.
+   * Finds the next block line that matches (accepted by the automaton), or null when at end of block.
+   *
+   * @return The next term in the current block that is accepted by the automaton; or null if none.
    */
   protected BytesRef nextTermInBlockMatching() throws IOException {
-    do {
-      // if seekTerm is set, then we seek into this block instead of starting with the first blindly.
-      if (seekTerm != null) {
-        assert blockLine == null;
-        boolean moveBeyondIfFound = seekTerm == startTerm; // for startTerm, we want to get the following term
-        SeekStatus seekStatus = seekInBlock(seekTerm);
-        seekTerm = null;// reset.
-        if (seekStatus == SeekStatus.END) {
-          return null;
-        } else if (seekStatus == SeekStatus.FOUND && moveBeyondIfFound) {
-          if (readLineInBlock() == null) {
-            return null;
+    if (seekTerm == null) {
+      if (readLineInBlock() == null) {
+        return null;
+      }
+    } else {
+      SeekStatus seekStatus = seekInBlock(seekTerm);
+      seekTerm = null;
+      if (seekStatus == SeekStatus.END) {
+        return null;
+      }
+      assert numMatchedBytes == 0;
+      assert numConsecutivelyRejectedTerms == 0;
+    }
+    int[] states = this.states;
+    while (true) {
+      TermBytes lineTermBytes = blockLine.getTermBytes();
+      BytesRef lineTerm = lineTermBytes.getTerm();
+      assert lineTerm.offset == 0;
+      byte[] termBytes = lineTerm.bytes;
+      int termLength = lineTerm.length;
 
 Review comment:
   I see you are locally declaring variables that are already fields of the parent class.  IMO these don't seem to add readability clarity: states, termBytes, termLength

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r383517461
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java
 ##########
 @@ -18,260 +18,337 @@
 package org.apache.lucene.codecs.uniformsplit;
 
 import java.io.IOException;
-import java.util.Objects;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
  * The "intersect" {@link TermsEnum} response to {@link UniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
  * intersecting the terms with an automaton.
+ * <p>
+ * By design of the UniformSplit block keys, it is less efficient than
+ * {@code org.apache.lucene.codecs.blocktree.IntersectTermsEnum} for {@link org.apache.lucene.search.FuzzyQuery} (-37%).
+ * It is slightly slower for {@link org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for
+ * {@link org.apache.lucene.search.PrefixQuery} (+5%).
+ *
+ * @lucene.experimental
  */
 public class IntersectBlockReader extends BlockReader {
 
-  protected final AutomatonNextTermCalculator nextStringCalculator;
-  protected final ByteRunAutomaton runAutomaton;
-  protected final BytesRef commonSuffixRef; // maybe null
-  protected final BytesRef commonPrefixRef;
-  protected final BytesRef startTerm; // maybe null
+  /**
+   * Block iteration order. Whether to move next block, jump to a block away, or end the iteration.
+   */
+  protected enum BlockIteration {NEXT, SEEK, END}
 
-  /** Set this when our current mode is seeking to this term.  Set to null after. */
-  protected BytesRef seekTerm;
+  /**
+   * Threshold that controls when to attempt to jump to a block away.
+   * <p>
+   * This counter is 0 when entering a block. It is incremented each time a term is rejected by the automaton.
+   * When the counter is greater than or equal to this threshold, then we compute the next term accepted by
+   * the automaton, with {@link AutomatonNextTermCalculator}, and we jump to a block away if the next term
+   * accepted is greater than the immediate next term in the block.
+   * <p>
+   * A low value, for example 1, improves the performance of automatons requiring many jumps, for example
+   * {@link org.apache.lucene.search.FuzzyQuery} and most {@link org.apache.lucene.search.WildcardQuery}.
+   * A higher value improves the performance of automatons with less or no jump, for example
+   * {@link org.apache.lucene.search.PrefixQuery}.
+   * A threshold of 4 seems to be a good balance.
+   */
+  protected final int NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD = 4;
 
-  protected int blockPrefixRunAutomatonState;
-  protected int blockPrefixLen;
+  protected final Automaton automaton;
+  protected final ByteRunAutomaton runAutomaton;
+  protected final boolean finite;
+  protected final BytesRef commonSuffix; // maybe null
+  protected final int minTermLength;
+  protected final AutomatonNextTermCalculator nextStringCalculator;
 
   /**
-   * Number of bytes accepted by the last call to {@link #runAutomatonForState}.
+   * Set this when our current mode is seeking to this term.  Set to null after.
+   */
+  protected BytesRef seekTerm;
+  /**
+   * Number of bytes accepted by the automaton when validating the current term.
+   */
+  protected int numMatchedBytes;
+  /**
+   * Automaton states reached when validating the current term, from 0 to {@link #numMatchedBytes} - 1.
+   */
+  protected int[] states;
+  /**
+   * Block iteration order determined when scanning the terms in the current block.
    */
-  protected int numBytesAccepted;
+  protected BlockIteration blockIteration;
   /**
-   * Whether the current term is beyond the automaton common prefix.
-   * If true this means the enumeration should stop immediately.
+   * Counter of the number of consecutively rejected terms.
+   * Depending on {@link #NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD}, this may trigger a jump to a block away.
    */
-  protected boolean beyondCommonPrefix;
+  protected int numConsecutivelyRejectedTerms;
 
-  public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
-                              IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
-                              FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
+  protected IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
+                                 IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput,
+                                 PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
+                                 BlockDecoder blockDecoder) throws IOException {
     super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
-    this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);
-    Automaton automaton = Objects.requireNonNull(compiled.automaton);
-    this.runAutomaton = Objects.requireNonNull(compiled.runAutomaton);
-    this.commonSuffixRef = compiled.commonSuffixRef; // maybe null
-    this.commonPrefixRef = Operations.getCommonPrefixBytesRef(automaton); // never null
-
-    this.startTerm = startTerm;
-    assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef);
-    // it is thus also true that startTerm >= commonPrefixRef
-
-    this.seekTerm = startTerm != null ? startTerm : commonPrefixRef;
+    checkIntersectAutomatonType(compiled);
+    automaton = compiled.automaton;
+    runAutomaton = compiled.runAutomaton;
+    finite = compiled.finite;
+    commonSuffix = compiled.commonSuffixRef;
+    minTermLength = getMinTermLength();
+    nextStringCalculator = new AutomatonNextTermCalculator(compiled);
+    seekTerm = startTerm;
   }
 
-  @Override
-  public BytesRef next() throws IOException {
-    clearTermState();
-
-    if (blockHeader == null) { // initial state
-      // note: don't call super.seekCeil here; we have our own logic
-
-      // Set the browser position to the block having the seek term.
-      // Even if -1, it's okay since we'll soon call nextKey().
-      long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
-      if (isBeyondLastTerm(seekTerm, blockStartFP)) {
-        return null; // EOF
-      }
-
-      // Starting at this block find and load the next matching block.
-      //   note: Since seekBlock was just called, we actually consider the current block as "next".
-      if (nextBlockMatchingPrefix() == false) { // note: starts at seek'ed block, which may have a match
-        return null; // EOF
-      }
+  protected void checkIntersectAutomatonType(CompiledAutomaton compiled) {
+    if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+      throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
     }
-
-    do {
-
-      // look in the rest of this block.
-      BytesRef term = nextTermInBlockMatching();
-      if (term != null) {
-        return term;
-      }
-
-      // next term dict matching prefix
-    } while (nextBlockMatchingPrefix());
-
-    return null; // EOF
   }
 
   /**
-   * Find the next block that appears to contain terms that could match the automata.
-   * The prefix is the primary clue.  Returns true if at one, or false for no more (EOF).
+   * Computes the minimal length of the terms accepted by the automaton.
+   * This speeds up the term scanning for automatons accepting a finite language.
    */
-  protected boolean nextBlockMatchingPrefix() throws IOException {
-    if (beyondCommonPrefix) {
-      return false; // EOF
+  protected int getMinTermLength() {
 
 Review comment:
   Cool!  Computing the minimum matching term length looks like something generally useful for any CompiledAutomaton to quickly short-circuit matching.  After all, there are similar precomputed matching hints in CompiledAutomaton like computing the common suffix bytes.  So, maybe this belongs on CompiledAutomaton?
   WDYT @mikemccand ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r383499305
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java
 ##########
 @@ -18,260 +18,337 @@
 package org.apache.lucene.codecs.uniformsplit;
 
 import java.io.IOException;
-import java.util.Objects;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
  * The "intersect" {@link TermsEnum} response to {@link UniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
  * intersecting the terms with an automaton.
+ * <p>
+ * By design of the UniformSplit block keys, it is less efficient than
+ * {@code org.apache.lucene.codecs.blocktree.IntersectTermsEnum} for {@link org.apache.lucene.search.FuzzyQuery} (-37%).
+ * It is slightly slower for {@link org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for
+ * {@link org.apache.lucene.search.PrefixQuery} (+5%).
+ *
+ * @lucene.experimental
  */
 public class IntersectBlockReader extends BlockReader {
 
-  protected final AutomatonNextTermCalculator nextStringCalculator;
-  protected final ByteRunAutomaton runAutomaton;
-  protected final BytesRef commonSuffixRef; // maybe null
-  protected final BytesRef commonPrefixRef;
-  protected final BytesRef startTerm; // maybe null
+  /**
+   * Block iteration order. Whether to move next block, jump to a block away, or end the iteration.
+   */
+  protected enum BlockIteration {NEXT, SEEK, END}
 
-  /** Set this when our current mode is seeking to this term.  Set to null after. */
-  protected BytesRef seekTerm;
+  /**
+   * Threshold that controls when to attempt to jump to a block away.
+   * <p>
+   * This counter is 0 when entering a block. It is incremented each time a term is rejected by the automaton.
+   * When the counter is greater than or equal to this threshold, then we compute the next term accepted by
+   * the automaton, with {@link AutomatonNextTermCalculator}, and we jump to a block away if the next term
+   * accepted is greater than the immediate next term in the block.
+   * <p>
+   * A low value, for example 1, improves the performance of automatons requiring many jumps, for example
+   * {@link org.apache.lucene.search.FuzzyQuery} and most {@link org.apache.lucene.search.WildcardQuery}.
+   * A higher value improves the performance of automatons with less or no jump, for example
+   * {@link org.apache.lucene.search.PrefixQuery}.
+   * A threshold of 4 seems to be a good balance.
+   */
+  protected final int NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD = 4;
 
-  protected int blockPrefixRunAutomatonState;
-  protected int blockPrefixLen;
+  protected final Automaton automaton;
+  protected final ByteRunAutomaton runAutomaton;
+  protected final boolean finite;
+  protected final BytesRef commonSuffix; // maybe null
+  protected final int minTermLength;
+  protected final AutomatonNextTermCalculator nextStringCalculator;
 
   /**
-   * Number of bytes accepted by the last call to {@link #runAutomatonForState}.
+   * Set this when our current mode is seeking to this term.  Set to null after.
+   */
+  protected BytesRef seekTerm;
+  /**
+   * Number of bytes accepted by the automaton when validating the current term.
+   */
+  protected int numMatchedBytes;
+  /**
+   * Automaton states reached when validating the current term, from 0 to {@link #numMatchedBytes} - 1.
+   */
+  protected int[] states;
+  /**
+   * Block iteration order determined when scanning the terms in the current block.
    */
-  protected int numBytesAccepted;
+  protected BlockIteration blockIteration;
   /**
-   * Whether the current term is beyond the automaton common prefix.
-   * If true this means the enumeration should stop immediately.
+   * Counter of the number of consecutively rejected terms.
+   * Depending on {@link #NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD}, this may trigger a jump to a block away.
    */
-  protected boolean beyondCommonPrefix;
+  protected int numConsecutivelyRejectedTerms;
 
-  public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
-                              IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
-                              FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
+  protected IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
+                                 IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput,
+                                 PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
+                                 BlockDecoder blockDecoder) throws IOException {
     super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
-    this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);
-    Automaton automaton = Objects.requireNonNull(compiled.automaton);
-    this.runAutomaton = Objects.requireNonNull(compiled.runAutomaton);
-    this.commonSuffixRef = compiled.commonSuffixRef; // maybe null
-    this.commonPrefixRef = Operations.getCommonPrefixBytesRef(automaton); // never null
-
-    this.startTerm = startTerm;
-    assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef);
-    // it is thus also true that startTerm >= commonPrefixRef
-
-    this.seekTerm = startTerm != null ? startTerm : commonPrefixRef;
+    checkIntersectAutomatonType(compiled);
+    automaton = compiled.automaton;
+    runAutomaton = compiled.runAutomaton;
+    finite = compiled.finite;
+    commonSuffix = compiled.commonSuffixRef;
+    minTermLength = getMinTermLength();
+    nextStringCalculator = new AutomatonNextTermCalculator(compiled);
+    seekTerm = startTerm;
   }
 
-  @Override
-  public BytesRef next() throws IOException {
-    clearTermState();
-
-    if (blockHeader == null) { // initial state
-      // note: don't call super.seekCeil here; we have our own logic
-
-      // Set the browser position to the block having the seek term.
-      // Even if -1, it's okay since we'll soon call nextKey().
-      long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
-      if (isBeyondLastTerm(seekTerm, blockStartFP)) {
-        return null; // EOF
-      }
-
-      // Starting at this block find and load the next matching block.
-      //   note: Since seekBlock was just called, we actually consider the current block as "next".
-      if (nextBlockMatchingPrefix() == false) { // note: starts at seek'ed block, which may have a match
-        return null; // EOF
-      }
+  protected void checkIntersectAutomatonType(CompiledAutomaton compiled) {
 
 Review comment:
   I see you moved this here from the Terms impl.  However notice that all the other formats have it on their Terms impl.  Lets be consistent, ehh?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] bruno-roustant commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
bruno-roustant commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r383748557
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java
 ##########
 @@ -18,260 +18,337 @@
 package org.apache.lucene.codecs.uniformsplit;
 
 import java.io.IOException;
-import java.util.Objects;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
  * The "intersect" {@link TermsEnum} response to {@link UniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
  * intersecting the terms with an automaton.
+ * <p>
+ * By design of the UniformSplit block keys, it is less efficient than
+ * {@code org.apache.lucene.codecs.blocktree.IntersectTermsEnum} for {@link org.apache.lucene.search.FuzzyQuery} (-37%).
+ * It is slightly slower for {@link org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for
+ * {@link org.apache.lucene.search.PrefixQuery} (+5%).
+ *
+ * @lucene.experimental
  */
 public class IntersectBlockReader extends BlockReader {
 
-  protected final AutomatonNextTermCalculator nextStringCalculator;
-  protected final ByteRunAutomaton runAutomaton;
-  protected final BytesRef commonSuffixRef; // maybe null
-  protected final BytesRef commonPrefixRef;
-  protected final BytesRef startTerm; // maybe null
+  /**
+   * Block iteration order. Whether to move next block, jump to a block away, or end the iteration.
+   */
+  protected enum BlockIteration {NEXT, SEEK, END}
 
-  /** Set this when our current mode is seeking to this term.  Set to null after. */
-  protected BytesRef seekTerm;
+  /**
+   * Threshold that controls when to attempt to jump to a block away.
+   * <p>
+   * This counter is 0 when entering a block. It is incremented each time a term is rejected by the automaton.
+   * When the counter is greater than or equal to this threshold, then we compute the next term accepted by
+   * the automaton, with {@link AutomatonNextTermCalculator}, and we jump to a block away if the next term
+   * accepted is greater than the immediate next term in the block.
+   * <p>
+   * A low value, for example 1, improves the performance of automatons requiring many jumps, for example
+   * {@link org.apache.lucene.search.FuzzyQuery} and most {@link org.apache.lucene.search.WildcardQuery}.
+   * A higher value improves the performance of automatons with less or no jump, for example
+   * {@link org.apache.lucene.search.PrefixQuery}.
+   * A threshold of 4 seems to be a good balance.
+   */
+  protected final int NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD = 4;
 
-  protected int blockPrefixRunAutomatonState;
-  protected int blockPrefixLen;
+  protected final Automaton automaton;
+  protected final ByteRunAutomaton runAutomaton;
+  protected final boolean finite;
+  protected final BytesRef commonSuffix; // maybe null
+  protected final int minTermLength;
+  protected final AutomatonNextTermCalculator nextStringCalculator;
 
   /**
-   * Number of bytes accepted by the last call to {@link #runAutomatonForState}.
+   * Set this when our current mode is seeking to this term.  Set to null after.
+   */
+  protected BytesRef seekTerm;
+  /**
+   * Number of bytes accepted by the automaton when validating the current term.
+   */
+  protected int numMatchedBytes;
+  /**
+   * Automaton states reached when validating the current term, from 0 to {@link #numMatchedBytes} - 1.
+   */
+  protected int[] states;
+  /**
+   * Block iteration order determined when scanning the terms in the current block.
    */
-  protected int numBytesAccepted;
+  protected BlockIteration blockIteration;
   /**
-   * Whether the current term is beyond the automaton common prefix.
-   * If true this means the enumeration should stop immediately.
+   * Counter of the number of consecutively rejected terms.
+   * Depending on {@link #NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD}, this may trigger a jump to a block away.
    */
-  protected boolean beyondCommonPrefix;
+  protected int numConsecutivelyRejectedTerms;
 
-  public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
-                              IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
-                              FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
+  protected IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
+                                 IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput,
+                                 PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
+                                 BlockDecoder blockDecoder) throws IOException {
     super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
-    this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);
-    Automaton automaton = Objects.requireNonNull(compiled.automaton);
-    this.runAutomaton = Objects.requireNonNull(compiled.runAutomaton);
-    this.commonSuffixRef = compiled.commonSuffixRef; // maybe null
-    this.commonPrefixRef = Operations.getCommonPrefixBytesRef(automaton); // never null
-
-    this.startTerm = startTerm;
-    assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef);
-    // it is thus also true that startTerm >= commonPrefixRef
-
-    this.seekTerm = startTerm != null ? startTerm : commonPrefixRef;
+    checkIntersectAutomatonType(compiled);
+    automaton = compiled.automaton;
+    runAutomaton = compiled.runAutomaton;
+    finite = compiled.finite;
+    commonSuffix = compiled.commonSuffixRef;
+    minTermLength = getMinTermLength();
+    nextStringCalculator = new AutomatonNextTermCalculator(compiled);
+    seekTerm = startTerm;
   }
 
-  @Override
-  public BytesRef next() throws IOException {
-    clearTermState();
-
-    if (blockHeader == null) { // initial state
-      // note: don't call super.seekCeil here; we have our own logic
-
-      // Set the browser position to the block having the seek term.
-      // Even if -1, it's okay since we'll soon call nextKey().
-      long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
-      if (isBeyondLastTerm(seekTerm, blockStartFP)) {
-        return null; // EOF
-      }
-
-      // Starting at this block find and load the next matching block.
-      //   note: Since seekBlock was just called, we actually consider the current block as "next".
-      if (nextBlockMatchingPrefix() == false) { // note: starts at seek'ed block, which may have a match
-        return null; // EOF
-      }
+  protected void checkIntersectAutomatonType(CompiledAutomaton compiled) {
+    if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+      throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
     }
-
-    do {
-
-      // look in the rest of this block.
-      BytesRef term = nextTermInBlockMatching();
-      if (term != null) {
-        return term;
-      }
-
-      // next term dict matching prefix
-    } while (nextBlockMatchingPrefix());
-
-    return null; // EOF
   }
 
   /**
-   * Find the next block that appears to contain terms that could match the automata.
-   * The prefix is the primary clue.  Returns true if at one, or false for no more (EOF).
+   * Computes the minimal length of the terms accepted by the automaton.
+   * This speeds up the term scanning for automatons accepting a finite language.
    */
-  protected boolean nextBlockMatchingPrefix() throws IOException {
-    if (beyondCommonPrefix) {
-      return false; // EOF
+  protected int getMinTermLength() {
+    // Automatons accepting infinite language (e.g. PrefixQuery and WildcardQuery) do not benefit much from
+    // min term length while it takes time to compute it. More precisely, by skipping this computation PrefixQuery
+    // is significantly boosted while WildcardQuery might be slightly degraded on average. This min term length
+    // mainly boosts FuzzyQuery.
+    int commonSuffixLength = commonSuffix == null ? 0 : commonSuffix.length;
+    if (!finite) {
+      return commonSuffixLength;
     }
-
-    IndexDictionary.Browser browser = getOrCreateDictionaryBrowser();
-
-    do {
-
-      // Get next block key (becomes in effect the current blockKey)
-      BytesRef blockKey = browser.nextKey();
-      if (blockKey == null) {
-        return false; // EOF
-      }
-
-      blockPrefixLen = browser.getBlockPrefixLen();
-      blockPrefixRunAutomatonState = runAutomatonForState(blockKey.bytes, blockKey.offset, blockPrefixLen, 0);
-
-      // We may have passed commonPrefix  (a short-circuit optimization).
-      if (isBeyondCommonPrefix(blockKey)) {
-        return false; // EOF
+    int commonPrefixLength = 0;
+    int state = 0;
+    Transition t = null;
+    int[] visited = null;
+    int visitedLength = 0;
+    while (true) {
+      if (runAutomaton.isAccept(state)) {
+        // The common prefix reaches a final state. So common prefix and common suffix overlap.
+        // Min term length is the max between common prefix and common suffix lengths.
+        return Math.max(commonPrefixLength, commonSuffixLength);
       }
-
-      if (blockPrefixRunAutomatonState >= 0) {
-        break; // a match
+      if (automaton.getNumTransitions(state) == 1) {
+        if (t == null) {
+          t = new Transition();
+        }
+        automaton.getTransition(state, 0, t);
+        if (t.min == t.max) {
+          state = t.dest;
+          if (visited == null) {
+            visited = new int[8];
+            visited[visitedLength++] = 0;
+          }
+          if (!contains(visited, visitedLength, state)) {
+            visited = ArrayUtil.grow(visited, ++visitedLength);
+            visited[visitedLength - 1] = state;
+            commonPrefixLength++;
+            continue;
+          }
+        }
       }
+      break;
+    }
+    // Min term length is the sum of common prefix and common suffix lengths.
+    return commonPrefixLength + commonSuffixLength;
+  }
 
-      //
-      // This block doesn't match.
-      //
-
-      seekTerm = null; // we're moving on to another block, and seekTerm is before it.
-
-      // Should we simply get the next key (linear mode) or try to seek?
-      if (nextStringCalculator.isLinearState(blockKey)) {
-        continue;
+  private static boolean contains(int[] ints, int length, int value) {
+    assert length <= ints.length;
+    for (int i = 0; i < length; i++) {
+      if (ints[i] == value) {
+        return true;
       }
+    }
+    return false;
+  }
 
-      // Maybe the next block's key matches?  We have to check this before calling nextStringCalculator.
-      BytesRef peekKey = browser.peekKey();
-      if (peekKey == null) {
-        return false; // EOF
-      }
-      if (runAutomatonForState(peekKey.bytes, peekKey.offset, peekKey.length, 0) >= 0) {
-        continue; // yay; it matched.  Continue to actually advance to it.  This is rare?
+  @Override
+  public BytesRef next() throws IOException {
+    if (blockHeader == null) {
+      if (!seekFirstBlock()) {
+        return null;
       }
-
-      // Seek to a block by calculating the next term to match the automata *following* peekKey.
-      this.seekTerm = nextStringCalculator.nextSeekTerm(browser.peekKey());
-      if (seekTerm == null) {
-        return false; // EOF
+      states = new int[32];
+      blockIteration = BlockIteration.NEXT;
+    }
+    termState = null;
+    do {
+      BytesRef term = nextTermInBlockMatching();
+      if (term != null) {
+        return term;
       }
-      browser.seekBlock(seekTerm);
-      //continue
-
-    } while (true); // while not a match
-
-    // A match!
-
-    //NOTE: we could determine if this automata has a prefix for this specific block (longer than the commonPrefix).
-    //  If we see it, we could set it as the seekTerm and we could also exit the block early if we get past this prefix
-    //  and runAutomatonFromPrefix would start from this prefix.  Smiley tried but benchmarks were not favorable to it.
-
-    initializeHeader(null, browser.getBlockFilePointer());
+    } while (nextBlock());
+    return null;
+  }
 
-    return true;
+  protected boolean seekFirstBlock() throws IOException {
+    seekTerm = nextStringCalculator.nextSeekTerm(seekTerm);
+    if (seekTerm == null) {
+      return false;
+    }
+    long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
+    if (blockStartFP == -1) {
+      blockStartFP = fieldMetadata.getFirstBlockStartFP();
+    } else if (isBeyondLastTerm(seekTerm, blockStartFP)) {
+      return false;
+    }
+    initializeHeader(seekTerm, blockStartFP);
+    return blockHeader != null;
   }
 
   /**
-   * Find the next block line that matches, or null when at end of block.
+   * Finds the next block line that matches (accepted by the automaton), or null when at end of block.
+   *
+   * @return The next term in the current block that is accepted by the automaton; or null if none.
    */
   protected BytesRef nextTermInBlockMatching() throws IOException {
-    do {
-      // if seekTerm is set, then we seek into this block instead of starting with the first blindly.
-      if (seekTerm != null) {
-        assert blockLine == null;
-        boolean moveBeyondIfFound = seekTerm == startTerm; // for startTerm, we want to get the following term
-        SeekStatus seekStatus = seekInBlock(seekTerm);
-        seekTerm = null;// reset.
-        if (seekStatus == SeekStatus.END) {
-          return null;
-        } else if (seekStatus == SeekStatus.FOUND && moveBeyondIfFound) {
-          if (readLineInBlock() == null) {
-            return null;
+    if (seekTerm == null) {
+      if (readLineInBlock() == null) {
+        return null;
+      }
+    } else {
+      SeekStatus seekStatus = seekInBlock(seekTerm);
+      seekTerm = null;
+      if (seekStatus == SeekStatus.END) {
+        return null;
+      }
+      assert numMatchedBytes == 0;
+      assert numConsecutivelyRejectedTerms == 0;
+    }
+    int[] states = this.states;
+    while (true) {
+      TermBytes lineTermBytes = blockLine.getTermBytes();
+      BytesRef lineTerm = lineTermBytes.getTerm();
+      assert lineTerm.offset == 0;
+      byte[] termBytes = lineTerm.bytes;
+      int termLength = lineTerm.length;
 
 Review comment:
   Since those fields are not final, it is (slightly) faster to have them as local variables when they are manipulated in a highly repeating loop. But I agree this is over-optimization. +1 I'll remove them.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] bruno-roustant commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
bruno-roustant commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r383756413
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java
 ##########
 @@ -18,260 +18,337 @@
 package org.apache.lucene.codecs.uniformsplit;
 
 import java.io.IOException;
-import java.util.Objects;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
  * The "intersect" {@link TermsEnum} response to {@link UniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
  * intersecting the terms with an automaton.
+ * <p>
+ * By design of the UniformSplit block keys, it is less efficient than
+ * {@code org.apache.lucene.codecs.blocktree.IntersectTermsEnum} for {@link org.apache.lucene.search.FuzzyQuery} (-37%).
+ * It is slightly slower for {@link org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for
+ * {@link org.apache.lucene.search.PrefixQuery} (+5%).
+ *
+ * @lucene.experimental
  */
 public class IntersectBlockReader extends BlockReader {
 
-  protected final AutomatonNextTermCalculator nextStringCalculator;
-  protected final ByteRunAutomaton runAutomaton;
-  protected final BytesRef commonSuffixRef; // maybe null
-  protected final BytesRef commonPrefixRef;
-  protected final BytesRef startTerm; // maybe null
+  /**
+   * Block iteration order. Whether to move next block, jump to a block away, or end the iteration.
+   */
+  protected enum BlockIteration {NEXT, SEEK, END}
 
-  /** Set this when our current mode is seeking to this term.  Set to null after. */
-  protected BytesRef seekTerm;
+  /**
+   * Threshold that controls when to attempt to jump to a block away.
+   * <p>
+   * This counter is 0 when entering a block. It is incremented each time a term is rejected by the automaton.
+   * When the counter is greater than or equal to this threshold, then we compute the next term accepted by
+   * the automaton, with {@link AutomatonNextTermCalculator}, and we jump to a block away if the next term
+   * accepted is greater than the immediate next term in the block.
+   * <p>
+   * A low value, for example 1, improves the performance of automatons requiring many jumps, for example
+   * {@link org.apache.lucene.search.FuzzyQuery} and most {@link org.apache.lucene.search.WildcardQuery}.
+   * A higher value improves the performance of automatons with less or no jump, for example
+   * {@link org.apache.lucene.search.PrefixQuery}.
+   * A threshold of 4 seems to be a good balance.
+   */
+  protected final int NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD = 4;
 
-  protected int blockPrefixRunAutomatonState;
-  protected int blockPrefixLen;
+  protected final Automaton automaton;
+  protected final ByteRunAutomaton runAutomaton;
+  protected final boolean finite;
+  protected final BytesRef commonSuffix; // maybe null
+  protected final int minTermLength;
+  protected final AutomatonNextTermCalculator nextStringCalculator;
 
   /**
-   * Number of bytes accepted by the last call to {@link #runAutomatonForState}.
+   * Set this when our current mode is seeking to this term.  Set to null after.
+   */
+  protected BytesRef seekTerm;
+  /**
+   * Number of bytes accepted by the automaton when validating the current term.
+   */
+  protected int numMatchedBytes;
+  /**
+   * Automaton states reached when validating the current term, from 0 to {@link #numMatchedBytes} - 1.
+   */
+  protected int[] states;
+  /**
+   * Block iteration order determined when scanning the terms in the current block.
    */
-  protected int numBytesAccepted;
+  protected BlockIteration blockIteration;
   /**
-   * Whether the current term is beyond the automaton common prefix.
-   * If true this means the enumeration should stop immediately.
+   * Counter of the number of consecutively rejected terms.
+   * Depending on {@link #NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD}, this may trigger a jump to a block away.
    */
-  protected boolean beyondCommonPrefix;
+  protected int numConsecutivelyRejectedTerms;
 
-  public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
-                              IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
-                              FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
+  protected IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
+                                 IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput,
+                                 PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
+                                 BlockDecoder blockDecoder) throws IOException {
     super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
-    this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);
-    Automaton automaton = Objects.requireNonNull(compiled.automaton);
-    this.runAutomaton = Objects.requireNonNull(compiled.runAutomaton);
-    this.commonSuffixRef = compiled.commonSuffixRef; // maybe null
-    this.commonPrefixRef = Operations.getCommonPrefixBytesRef(automaton); // never null
-
-    this.startTerm = startTerm;
-    assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef);
-    // it is thus also true that startTerm >= commonPrefixRef
-
-    this.seekTerm = startTerm != null ? startTerm : commonPrefixRef;
+    checkIntersectAutomatonType(compiled);
+    automaton = compiled.automaton;
+    runAutomaton = compiled.runAutomaton;
+    finite = compiled.finite;
+    commonSuffix = compiled.commonSuffixRef;
+    minTermLength = getMinTermLength();
+    nextStringCalculator = new AutomatonNextTermCalculator(compiled);
+    seekTerm = startTerm;
   }
 
-  @Override
-  public BytesRef next() throws IOException {
-    clearTermState();
-
-    if (blockHeader == null) { // initial state
-      // note: don't call super.seekCeil here; we have our own logic
-
-      // Set the browser position to the block having the seek term.
-      // Even if -1, it's okay since we'll soon call nextKey().
-      long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
-      if (isBeyondLastTerm(seekTerm, blockStartFP)) {
-        return null; // EOF
-      }
-
-      // Starting at this block find and load the next matching block.
-      //   note: Since seekBlock was just called, we actually consider the current block as "next".
-      if (nextBlockMatchingPrefix() == false) { // note: starts at seek'ed block, which may have a match
-        return null; // EOF
-      }
+  protected void checkIntersectAutomatonType(CompiledAutomaton compiled) {
+    if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+      throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
     }
-
-    do {
-
-      // look in the rest of this block.
-      BytesRef term = nextTermInBlockMatching();
-      if (term != null) {
-        return term;
-      }
-
-      // next term dict matching prefix
-    } while (nextBlockMatchingPrefix());
-
-    return null; // EOF
   }
 
   /**
-   * Find the next block that appears to contain terms that could match the automata.
-   * The prefix is the primary clue.  Returns true if at one, or false for no more (EOF).
+   * Computes the minimal length of the terms accepted by the automaton.
+   * This speeds up the term scanning for automatons accepting a finite language.
    */
-  protected boolean nextBlockMatchingPrefix() throws IOException {
-    if (beyondCommonPrefix) {
-      return false; // EOF
+  protected int getMinTermLength() {
+    // Automatons accepting infinite language (e.g. PrefixQuery and WildcardQuery) do not benefit much from
+    // min term length while it takes time to compute it. More precisely, by skipping this computation PrefixQuery
+    // is significantly boosted while WildcardQuery might be slightly degraded on average. This min term length
+    // mainly boosts FuzzyQuery.
+    int commonSuffixLength = commonSuffix == null ? 0 : commonSuffix.length;
+    if (!finite) {
+      return commonSuffixLength;
     }
-
-    IndexDictionary.Browser browser = getOrCreateDictionaryBrowser();
-
-    do {
-
-      // Get next block key (becomes in effect the current blockKey)
-      BytesRef blockKey = browser.nextKey();
-      if (blockKey == null) {
-        return false; // EOF
-      }
-
-      blockPrefixLen = browser.getBlockPrefixLen();
-      blockPrefixRunAutomatonState = runAutomatonForState(blockKey.bytes, blockKey.offset, blockPrefixLen, 0);
-
-      // We may have passed commonPrefix  (a short-circuit optimization).
-      if (isBeyondCommonPrefix(blockKey)) {
-        return false; // EOF
+    int commonPrefixLength = 0;
+    int state = 0;
+    Transition t = null;
+    int[] visited = null;
+    int visitedLength = 0;
+    while (true) {
+      if (runAutomaton.isAccept(state)) {
+        // The common prefix reaches a final state. So common prefix and common suffix overlap.
+        // Min term length is the max between common prefix and common suffix lengths.
+        return Math.max(commonPrefixLength, commonSuffixLength);
       }
-
-      if (blockPrefixRunAutomatonState >= 0) {
-        break; // a match
+      if (automaton.getNumTransitions(state) == 1) {
+        if (t == null) {
+          t = new Transition();
+        }
+        automaton.getTransition(state, 0, t);
+        if (t.min == t.max) {
+          state = t.dest;
+          if (visited == null) {
+            visited = new int[8];
+            visited[visitedLength++] = 0;
+          }
+          if (!contains(visited, visitedLength, state)) {
+            visited = ArrayUtil.grow(visited, ++visitedLength);
+            visited[visitedLength - 1] = state;
+            commonPrefixLength++;
+            continue;
+          }
+        }
       }
+      break;
+    }
+    // Min term length is the sum of common prefix and common suffix lengths.
+    return commonPrefixLength + commonSuffixLength;
+  }
 
-      //
-      // This block doesn't match.
-      //
-
-      seekTerm = null; // we're moving on to another block, and seekTerm is before it.
-
-      // Should we simply get the next key (linear mode) or try to seek?
-      if (nextStringCalculator.isLinearState(blockKey)) {
-        continue;
+  private static boolean contains(int[] ints, int length, int value) {
+    assert length <= ints.length;
+    for (int i = 0; i < length; i++) {
+      if (ints[i] == value) {
+        return true;
       }
+    }
+    return false;
+  }
 
-      // Maybe the next block's key matches?  We have to check this before calling nextStringCalculator.
-      BytesRef peekKey = browser.peekKey();
-      if (peekKey == null) {
-        return false; // EOF
-      }
-      if (runAutomatonForState(peekKey.bytes, peekKey.offset, peekKey.length, 0) >= 0) {
-        continue; // yay; it matched.  Continue to actually advance to it.  This is rare?
+  @Override
+  public BytesRef next() throws IOException {
+    if (blockHeader == null) {
+      if (!seekFirstBlock()) {
+        return null;
       }
-
-      // Seek to a block by calculating the next term to match the automata *following* peekKey.
-      this.seekTerm = nextStringCalculator.nextSeekTerm(browser.peekKey());
-      if (seekTerm == null) {
-        return false; // EOF
+      states = new int[32];
+      blockIteration = BlockIteration.NEXT;
+    }
+    termState = null;
+    do {
+      BytesRef term = nextTermInBlockMatching();
+      if (term != null) {
+        return term;
       }
-      browser.seekBlock(seekTerm);
-      //continue
-
-    } while (true); // while not a match
-
-    // A match!
-
-    //NOTE: we could determine if this automata has a prefix for this specific block (longer than the commonPrefix).
-    //  If we see it, we could set it as the seekTerm and we could also exit the block early if we get past this prefix
-    //  and runAutomatonFromPrefix would start from this prefix.  Smiley tried but benchmarks were not favorable to it.
-
-    initializeHeader(null, browser.getBlockFilePointer());
+    } while (nextBlock());
+    return null;
+  }
 
-    return true;
+  protected boolean seekFirstBlock() throws IOException {
+    seekTerm = nextStringCalculator.nextSeekTerm(seekTerm);
+    if (seekTerm == null) {
+      return false;
+    }
+    long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
+    if (blockStartFP == -1) {
+      blockStartFP = fieldMetadata.getFirstBlockStartFP();
+    } else if (isBeyondLastTerm(seekTerm, blockStartFP)) {
+      return false;
+    }
+    initializeHeader(seekTerm, blockStartFP);
+    return blockHeader != null;
   }
 
   /**
-   * Find the next block line that matches, or null when at end of block.
+   * Finds the next block line that matches (accepted by the automaton), or null when at end of block.
+   *
+   * @return The next term in the current block that is accepted by the automaton; or null if none.
    */
   protected BytesRef nextTermInBlockMatching() throws IOException {
-    do {
-      // if seekTerm is set, then we seek into this block instead of starting with the first blindly.
-      if (seekTerm != null) {
-        assert blockLine == null;
-        boolean moveBeyondIfFound = seekTerm == startTerm; // for startTerm, we want to get the following term
-        SeekStatus seekStatus = seekInBlock(seekTerm);
-        seekTerm = null;// reset.
-        if (seekStatus == SeekStatus.END) {
-          return null;
-        } else if (seekStatus == SeekStatus.FOUND && moveBeyondIfFound) {
-          if (readLineInBlock() == null) {
-            return null;
+    if (seekTerm == null) {
+      if (readLineInBlock() == null) {
+        return null;
+      }
+    } else {
+      SeekStatus seekStatus = seekInBlock(seekTerm);
+      seekTerm = null;
+      if (seekStatus == SeekStatus.END) {
+        return null;
+      }
+      assert numMatchedBytes == 0;
+      assert numConsecutivelyRejectedTerms == 0;
+    }
+    int[] states = this.states;
+    while (true) {
+      TermBytes lineTermBytes = blockLine.getTermBytes();
+      BytesRef lineTerm = lineTermBytes.getTerm();
+      assert lineTerm.offset == 0;
+      byte[] termBytes = lineTerm.bytes;
+      int termLength = lineTerm.length;
+      if (states.length <= termLength) {
+        states = this.states = ArrayUtil.growExact(states, ArrayUtil.oversize(termLength + 1, Byte.BYTES));
+      }
+      // Since terms a delta encoded, we may start the automaton steps from the last state reached by the previous term.
+      int index = Math.min(lineTermBytes.getSuffixOffset(), numMatchedBytes);
+      // Skip this term early if it is shorter than the min term length, or if it does not end with the common suffix
+      // accepted by the automaton.
+      if (termLength >= minTermLength && (commonSuffix == null || endsWithCommonSuffix(termBytes, termLength))) {
+        int state = states[index];
+        while (true) {
+          if (index == termLength) {
+            if (runAutomaton.isAccept(state)) {
+              // The automaton accepts the current term. Record the number of matched bytes and return the term.
+              assert runAutomaton.run(termBytes, 0, termLength);
+              numMatchedBytes = index;
+              if (numConsecutivelyRejectedTerms > 0) {
 
 Review comment:
   Some lines below there is a subtlety that sets numConsecutivelyRejectedTerms = Integer.MIN_VALUE to ensure we will not attempt to jump for the current block anymore. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] bruno-roustant commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
bruno-roustant commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r385573506
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java
 ##########
 @@ -18,260 +18,337 @@
 package org.apache.lucene.codecs.uniformsplit;
 
 import java.io.IOException;
-import java.util.Objects;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
  * The "intersect" {@link TermsEnum} response to {@link UniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
  * intersecting the terms with an automaton.
+ * <p>
+ * By design of the UniformSplit block keys, it is less efficient than
+ * {@code org.apache.lucene.codecs.blocktree.IntersectTermsEnum} for {@link org.apache.lucene.search.FuzzyQuery} (-37%).
+ * It is slightly slower for {@link org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for
+ * {@link org.apache.lucene.search.PrefixQuery} (+5%).
+ *
+ * @lucene.experimental
  */
 public class IntersectBlockReader extends BlockReader {
 
-  protected final AutomatonNextTermCalculator nextStringCalculator;
-  protected final ByteRunAutomaton runAutomaton;
-  protected final BytesRef commonSuffixRef; // maybe null
-  protected final BytesRef commonPrefixRef;
-  protected final BytesRef startTerm; // maybe null
+  /**
+   * Block iteration order. Whether to move next block, jump to a block away, or end the iteration.
+   */
+  protected enum BlockIteration {NEXT, SEEK, END}
 
-  /** Set this when our current mode is seeking to this term.  Set to null after. */
-  protected BytesRef seekTerm;
+  /**
+   * Threshold that controls when to attempt to jump to a block away.
+   * <p>
+   * This counter is 0 when entering a block. It is incremented each time a term is rejected by the automaton.
+   * When the counter is greater than or equal to this threshold, then we compute the next term accepted by
+   * the automaton, with {@link AutomatonNextTermCalculator}, and we jump to a block away if the next term
+   * accepted is greater than the immediate next term in the block.
+   * <p>
+   * A low value, for example 1, improves the performance of automatons requiring many jumps, for example
+   * {@link org.apache.lucene.search.FuzzyQuery} and most {@link org.apache.lucene.search.WildcardQuery}.
+   * A higher value improves the performance of automatons with less or no jump, for example
+   * {@link org.apache.lucene.search.PrefixQuery}.
+   * A threshold of 4 seems to be a good balance.
+   */
+  protected final int NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD = 4;
 
-  protected int blockPrefixRunAutomatonState;
-  protected int blockPrefixLen;
+  protected final Automaton automaton;
+  protected final ByteRunAutomaton runAutomaton;
+  protected final boolean finite;
+  protected final BytesRef commonSuffix; // maybe null
+  protected final int minTermLength;
+  protected final AutomatonNextTermCalculator nextStringCalculator;
 
   /**
-   * Number of bytes accepted by the last call to {@link #runAutomatonForState}.
+   * Set this when our current mode is seeking to this term.  Set to null after.
+   */
+  protected BytesRef seekTerm;
+  /**
+   * Number of bytes accepted by the automaton when validating the current term.
+   */
+  protected int numMatchedBytes;
+  /**
+   * Automaton states reached when validating the current term, from 0 to {@link #numMatchedBytes} - 1.
+   */
+  protected int[] states;
+  /**
+   * Block iteration order determined when scanning the terms in the current block.
    */
-  protected int numBytesAccepted;
+  protected BlockIteration blockIteration;
   /**
-   * Whether the current term is beyond the automaton common prefix.
-   * If true this means the enumeration should stop immediately.
+   * Counter of the number of consecutively rejected terms.
+   * Depending on {@link #NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD}, this may trigger a jump to a block away.
    */
-  protected boolean beyondCommonPrefix;
+  protected int numConsecutivelyRejectedTerms;
 
-  public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
-                              IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
-                              FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
+  protected IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
+                                 IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput,
+                                 PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
+                                 BlockDecoder blockDecoder) throws IOException {
     super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
-    this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);
-    Automaton automaton = Objects.requireNonNull(compiled.automaton);
-    this.runAutomaton = Objects.requireNonNull(compiled.runAutomaton);
-    this.commonSuffixRef = compiled.commonSuffixRef; // maybe null
-    this.commonPrefixRef = Operations.getCommonPrefixBytesRef(automaton); // never null
-
-    this.startTerm = startTerm;
-    assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef);
-    // it is thus also true that startTerm >= commonPrefixRef
-
-    this.seekTerm = startTerm != null ? startTerm : commonPrefixRef;
+    checkIntersectAutomatonType(compiled);
+    automaton = compiled.automaton;
+    runAutomaton = compiled.runAutomaton;
+    finite = compiled.finite;
+    commonSuffix = compiled.commonSuffixRef;
+    minTermLength = getMinTermLength();
+    nextStringCalculator = new AutomatonNextTermCalculator(compiled);
+    seekTerm = startTerm;
   }
 
-  @Override
-  public BytesRef next() throws IOException {
-    clearTermState();
-
-    if (blockHeader == null) { // initial state
-      // note: don't call super.seekCeil here; we have our own logic
-
-      // Set the browser position to the block having the seek term.
-      // Even if -1, it's okay since we'll soon call nextKey().
-      long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
-      if (isBeyondLastTerm(seekTerm, blockStartFP)) {
-        return null; // EOF
-      }
-
-      // Starting at this block find and load the next matching block.
-      //   note: Since seekBlock was just called, we actually consider the current block as "next".
-      if (nextBlockMatchingPrefix() == false) { // note: starts at seek'ed block, which may have a match
-        return null; // EOF
-      }
+  protected void checkIntersectAutomatonType(CompiledAutomaton compiled) {
+    if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+      throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
     }
-
-    do {
-
-      // look in the rest of this block.
-      BytesRef term = nextTermInBlockMatching();
-      if (term != null) {
-        return term;
-      }
-
-      // next term dict matching prefix
-    } while (nextBlockMatchingPrefix());
-
-    return null; // EOF
   }
 
   /**
-   * Find the next block that appears to contain terms that could match the automata.
-   * The prefix is the primary clue.  Returns true if at one, or false for no more (EOF).
+   * Computes the minimal length of the terms accepted by the automaton.
+   * This speeds up the term scanning for automatons accepting a finite language.
    */
-  protected boolean nextBlockMatchingPrefix() throws IOException {
-    if (beyondCommonPrefix) {
-      return false; // EOF
+  protected int getMinTermLength() {
 
 Review comment:
   Actually a test specific to this method is not easy at all. I don't see a way to randomize and assert the min length without using a code similar to the code itself.
   I'll let it as it is today: I leverage the multiple random tests for automatons, regex and fuzzy queries, and I verify that the min term length works (no false negative) with the assertions already present in the nextTermInBlockMatching() method. For each term, we verify with runAutomaton.run() that the acceptance/rejection is correct.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] asfgit closed pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] bruno-roustant edited a comment on issue #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
bruno-roustant edited a comment on issue #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#issuecomment-590780144
 
 
   When I debugged thoroughly to understand what was the limitation of the approach we had (to compute the common prefix between two consecutive block keys in the FST), I saw that actually for all FuzzyQuery the common prefix matched so we entered all blocks.
   I realized that the FuzzyQuery automaton accepts many variations for the prefix, and the common prefix was not long enough to allow us to filter correctly.
   
   I looked at what VarGapFixedInterval did. It jumped all the time after each term to find the next target term accepted by the automaton. And this was sufficiently efficient thanks to a vital optimization that compared the target term to the immediate following term, to actually not jump most of the time.
   
   So I applied the same idea to compute the next accepted term and jump, but now with a first condition based on the number of consecutively rejected terms, and by anticipating the comparison of the accepted term with the immediate next term. This is the main factor of the improvement. We leverage also optimizations that speed up the automaton validation of terms in the block.
   
   For the proposal of the block prefix in the BlockHeader, does that mean that we have to open the block to get the prefix? Because the speed for FuzzyQuery highly depends on how many block we *don't* open.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] bruno-roustant edited a comment on issue #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
bruno-roustant edited a comment on issue #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#issuecomment-590780144
 
 
   When I debugged thoroughly to understand what was the limitation of the approach we had (to compute the common prefix between two consecutive block keys in the FST), I saw that actually for all FuzzyQuery the common prefix matched so we entered all blocks.
   I realized that the FuzzyQuery automaton accepts many variations for the prefix, and the common prefix was not long enough to allow us to filter correctly.
   
   I looked at what VarGapFixedInterval did. It jumped all the time after each term to find the next target term accepted by the automaton. And this was sufficiently efficient thanks to a vital optimization that compared the target term to the immediate following term, to actually not jump most of the time.
   
   So I applied the same idea to compute the next accepted term and jump, but now with a first condition based on the number of consecutively rejected terms, and by anticipating the comparison of the accepted term with the immediate next term. This is the main factor of the improvement. We leverage also optimizations that speed up the automaton validation of terms in the block.
   
   For the proposal of the block prefix in the BlockHeader, does that mean that we have to open the block to get the prefix? Because the speed for FuzzyQuery highly depends on how many blocks we *don't* open.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] bruno-roustant commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
bruno-roustant commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r383740837
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java
 ##########
 @@ -18,260 +18,337 @@
 package org.apache.lucene.codecs.uniformsplit;
 
 import java.io.IOException;
-import java.util.Objects;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
  * The "intersect" {@link TermsEnum} response to {@link UniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
  * intersecting the terms with an automaton.
+ * <p>
+ * By design of the UniformSplit block keys, it is less efficient than
+ * {@code org.apache.lucene.codecs.blocktree.IntersectTermsEnum} for {@link org.apache.lucene.search.FuzzyQuery} (-37%).
+ * It is slightly slower for {@link org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for
+ * {@link org.apache.lucene.search.PrefixQuery} (+5%).
+ *
+ * @lucene.experimental
  */
 public class IntersectBlockReader extends BlockReader {
 
-  protected final AutomatonNextTermCalculator nextStringCalculator;
-  protected final ByteRunAutomaton runAutomaton;
-  protected final BytesRef commonSuffixRef; // maybe null
-  protected final BytesRef commonPrefixRef;
-  protected final BytesRef startTerm; // maybe null
+  /**
+   * Block iteration order. Whether to move next block, jump to a block away, or end the iteration.
+   */
+  protected enum BlockIteration {NEXT, SEEK, END}
 
-  /** Set this when our current mode is seeking to this term.  Set to null after. */
-  protected BytesRef seekTerm;
+  /**
+   * Threshold that controls when to attempt to jump to a block away.
+   * <p>
+   * This counter is 0 when entering a block. It is incremented each time a term is rejected by the automaton.
+   * When the counter is greater than or equal to this threshold, then we compute the next term accepted by
+   * the automaton, with {@link AutomatonNextTermCalculator}, and we jump to a block away if the next term
+   * accepted is greater than the immediate next term in the block.
+   * <p>
+   * A low value, for example 1, improves the performance of automatons requiring many jumps, for example
+   * {@link org.apache.lucene.search.FuzzyQuery} and most {@link org.apache.lucene.search.WildcardQuery}.
+   * A higher value improves the performance of automatons with less or no jump, for example
+   * {@link org.apache.lucene.search.PrefixQuery}.
+   * A threshold of 4 seems to be a good balance.
+   */
+  protected final int NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD = 4;
 
-  protected int blockPrefixRunAutomatonState;
-  protected int blockPrefixLen;
+  protected final Automaton automaton;
+  protected final ByteRunAutomaton runAutomaton;
+  protected final boolean finite;
+  protected final BytesRef commonSuffix; // maybe null
+  protected final int minTermLength;
+  protected final AutomatonNextTermCalculator nextStringCalculator;
 
   /**
-   * Number of bytes accepted by the last call to {@link #runAutomatonForState}.
+   * Set this when our current mode is seeking to this term.  Set to null after.
+   */
+  protected BytesRef seekTerm;
+  /**
+   * Number of bytes accepted by the automaton when validating the current term.
+   */
+  protected int numMatchedBytes;
+  /**
+   * Automaton states reached when validating the current term, from 0 to {@link #numMatchedBytes} - 1.
+   */
+  protected int[] states;
+  /**
+   * Block iteration order determined when scanning the terms in the current block.
    */
-  protected int numBytesAccepted;
+  protected BlockIteration blockIteration;
   /**
-   * Whether the current term is beyond the automaton common prefix.
-   * If true this means the enumeration should stop immediately.
+   * Counter of the number of consecutively rejected terms.
+   * Depending on {@link #NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD}, this may trigger a jump to a block away.
    */
-  protected boolean beyondCommonPrefix;
+  protected int numConsecutivelyRejectedTerms;
 
-  public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
-                              IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
-                              FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
+  protected IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
+                                 IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput,
+                                 PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
+                                 BlockDecoder blockDecoder) throws IOException {
     super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
-    this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);
-    Automaton automaton = Objects.requireNonNull(compiled.automaton);
-    this.runAutomaton = Objects.requireNonNull(compiled.runAutomaton);
-    this.commonSuffixRef = compiled.commonSuffixRef; // maybe null
-    this.commonPrefixRef = Operations.getCommonPrefixBytesRef(automaton); // never null
-
-    this.startTerm = startTerm;
-    assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef);
-    // it is thus also true that startTerm >= commonPrefixRef
-
-    this.seekTerm = startTerm != null ? startTerm : commonPrefixRef;
+    checkIntersectAutomatonType(compiled);
+    automaton = compiled.automaton;
+    runAutomaton = compiled.runAutomaton;
+    finite = compiled.finite;
+    commonSuffix = compiled.commonSuffixRef;
+    minTermLength = getMinTermLength();
+    nextStringCalculator = new AutomatonNextTermCalculator(compiled);
+    seekTerm = startTerm;
   }
 
-  @Override
-  public BytesRef next() throws IOException {
-    clearTermState();
-
-    if (blockHeader == null) { // initial state
-      // note: don't call super.seekCeil here; we have our own logic
-
-      // Set the browser position to the block having the seek term.
-      // Even if -1, it's okay since we'll soon call nextKey().
-      long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
-      if (isBeyondLastTerm(seekTerm, blockStartFP)) {
-        return null; // EOF
-      }
-
-      // Starting at this block find and load the next matching block.
-      //   note: Since seekBlock was just called, we actually consider the current block as "next".
-      if (nextBlockMatchingPrefix() == false) { // note: starts at seek'ed block, which may have a match
-        return null; // EOF
-      }
+  protected void checkIntersectAutomatonType(CompiledAutomaton compiled) {
 
 Review comment:
   Hum, now I see why we did that way the first time. Ok I'll revert that, then I'll duplicate in SharedTermsUniformSplit Terms because it was missing.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] bruno-roustant commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
bruno-roustant commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r383746326
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java
 ##########
 @@ -18,260 +18,337 @@
 package org.apache.lucene.codecs.uniformsplit;
 
 import java.io.IOException;
-import java.util.Objects;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
  * The "intersect" {@link TermsEnum} response to {@link UniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
  * intersecting the terms with an automaton.
+ * <p>
+ * By design of the UniformSplit block keys, it is less efficient than
+ * {@code org.apache.lucene.codecs.blocktree.IntersectTermsEnum} for {@link org.apache.lucene.search.FuzzyQuery} (-37%).
+ * It is slightly slower for {@link org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for
+ * {@link org.apache.lucene.search.PrefixQuery} (+5%).
+ *
+ * @lucene.experimental
  */
 public class IntersectBlockReader extends BlockReader {
 
-  protected final AutomatonNextTermCalculator nextStringCalculator;
-  protected final ByteRunAutomaton runAutomaton;
-  protected final BytesRef commonSuffixRef; // maybe null
-  protected final BytesRef commonPrefixRef;
-  protected final BytesRef startTerm; // maybe null
+  /**
+   * Block iteration order. Whether to move next block, jump to a block away, or end the iteration.
+   */
+  protected enum BlockIteration {NEXT, SEEK, END}
 
-  /** Set this when our current mode is seeking to this term.  Set to null after. */
-  protected BytesRef seekTerm;
+  /**
+   * Threshold that controls when to attempt to jump to a block away.
+   * <p>
+   * This counter is 0 when entering a block. It is incremented each time a term is rejected by the automaton.
+   * When the counter is greater than or equal to this threshold, then we compute the next term accepted by
+   * the automaton, with {@link AutomatonNextTermCalculator}, and we jump to a block away if the next term
+   * accepted is greater than the immediate next term in the block.
+   * <p>
+   * A low value, for example 1, improves the performance of automatons requiring many jumps, for example
+   * {@link org.apache.lucene.search.FuzzyQuery} and most {@link org.apache.lucene.search.WildcardQuery}.
+   * A higher value improves the performance of automatons with less or no jump, for example
+   * {@link org.apache.lucene.search.PrefixQuery}.
+   * A threshold of 4 seems to be a good balance.
+   */
+  protected final int NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD = 4;
 
-  protected int blockPrefixRunAutomatonState;
-  protected int blockPrefixLen;
+  protected final Automaton automaton;
+  protected final ByteRunAutomaton runAutomaton;
+  protected final boolean finite;
+  protected final BytesRef commonSuffix; // maybe null
+  protected final int minTermLength;
+  protected final AutomatonNextTermCalculator nextStringCalculator;
 
   /**
-   * Number of bytes accepted by the last call to {@link #runAutomatonForState}.
+   * Set this when our current mode is seeking to this term.  Set to null after.
+   */
+  protected BytesRef seekTerm;
+  /**
+   * Number of bytes accepted by the automaton when validating the current term.
+   */
+  protected int numMatchedBytes;
+  /**
+   * Automaton states reached when validating the current term, from 0 to {@link #numMatchedBytes} - 1.
+   */
+  protected int[] states;
+  /**
+   * Block iteration order determined when scanning the terms in the current block.
    */
-  protected int numBytesAccepted;
+  protected BlockIteration blockIteration;
   /**
-   * Whether the current term is beyond the automaton common prefix.
-   * If true this means the enumeration should stop immediately.
+   * Counter of the number of consecutively rejected terms.
+   * Depending on {@link #NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD}, this may trigger a jump to a block away.
    */
-  protected boolean beyondCommonPrefix;
+  protected int numConsecutivelyRejectedTerms;
 
-  public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
-                              IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
-                              FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
+  protected IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
+                                 IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput,
+                                 PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
+                                 BlockDecoder blockDecoder) throws IOException {
     super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
-    this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);
-    Automaton automaton = Objects.requireNonNull(compiled.automaton);
-    this.runAutomaton = Objects.requireNonNull(compiled.runAutomaton);
-    this.commonSuffixRef = compiled.commonSuffixRef; // maybe null
-    this.commonPrefixRef = Operations.getCommonPrefixBytesRef(automaton); // never null
-
-    this.startTerm = startTerm;
-    assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef);
-    // it is thus also true that startTerm >= commonPrefixRef
-
-    this.seekTerm = startTerm != null ? startTerm : commonPrefixRef;
+    checkIntersectAutomatonType(compiled);
+    automaton = compiled.automaton;
+    runAutomaton = compiled.runAutomaton;
+    finite = compiled.finite;
+    commonSuffix = compiled.commonSuffixRef;
+    minTermLength = getMinTermLength();
+    nextStringCalculator = new AutomatonNextTermCalculator(compiled);
+    seekTerm = startTerm;
   }
 
-  @Override
-  public BytesRef next() throws IOException {
-    clearTermState();
-
-    if (blockHeader == null) { // initial state
-      // note: don't call super.seekCeil here; we have our own logic
-
-      // Set the browser position to the block having the seek term.
-      // Even if -1, it's okay since we'll soon call nextKey().
-      long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
-      if (isBeyondLastTerm(seekTerm, blockStartFP)) {
-        return null; // EOF
-      }
-
-      // Starting at this block find and load the next matching block.
-      //   note: Since seekBlock was just called, we actually consider the current block as "next".
-      if (nextBlockMatchingPrefix() == false) { // note: starts at seek'ed block, which may have a match
-        return null; // EOF
-      }
+  protected void checkIntersectAutomatonType(CompiledAutomaton compiled) {
+    if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+      throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
     }
-
-    do {
-
-      // look in the rest of this block.
-      BytesRef term = nextTermInBlockMatching();
-      if (term != null) {
-        return term;
-      }
-
-      // next term dict matching prefix
-    } while (nextBlockMatchingPrefix());
-
-    return null; // EOF
   }
 
   /**
-   * Find the next block that appears to contain terms that could match the automata.
-   * The prefix is the primary clue.  Returns true if at one, or false for no more (EOF).
+   * Computes the minimal length of the terms accepted by the automaton.
+   * This speeds up the term scanning for automatons accepting a finite language.
    */
-  protected boolean nextBlockMatchingPrefix() throws IOException {
-    if (beyondCommonPrefix) {
-      return false; // EOF
+  protected int getMinTermLength() {
 
 Review comment:
   I don't think it will bring value to BlockTree IntersectTermsEnum because by design it already browses the automaton transitions to go to the blocks that have the corresponding prefixes. So this method is probably better here as a specificity of the UniformSplit intersect.
   As opposed to common suffix which is generally useful.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] bruno-roustant commented on issue #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
bruno-roustant commented on issue #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#issuecomment-590780144
 
 
   When a debugged thoroughly to understand what was the limitation of the approach we had (to compute the common prefix between two consecutive block keys in the FST), I saw that actually for all FuzzyQuery the common prefix matched so we entered all blocks.
   I realized that the FuzzyQuery automaton accepts many variations for the prefix, and the common prefix was not long enough to allow us to filter correctly.
   
   I looked at what VarGapFixedInterval did. It jumped all the time after each term to find the next target term accepted by the automaton. And this was sufficiently efficient thanks to a vital optimization that compared the target term to the immediate following term, to actually not jump most of the time.
   
   So I applied the same idea to compute the next accepted term and jump, but now with a first condition based on the number of consecutively rejected terms, and by anticipating the comparison of the accepted term with the immediate term. This is the main factor of the improvement. We leverage also optimization that speed up the automaton validation of terms in the block.
   
   For the proposal of the block prefix in the BlockHeader, does that mean that we have to open the block to get the prefix? Because the speed for FuzzyQuery highly depends on how many block we *don't* open.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] bruno-roustant commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.

Posted by GitBox <gi...@apache.org>.
bruno-roustant commented on a change in pull request #1270: LUCENE-9237: Faster UniformSplit IntersectBlockReader.
URL: https://github.com/apache/lucene-solr/pull/1270#discussion_r384014576
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java
 ##########
 @@ -18,260 +18,337 @@
 package org.apache.lucene.codecs.uniformsplit;
 
 import java.io.IOException;
-import java.util.Objects;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
  * The "intersect" {@link TermsEnum} response to {@link UniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
  * intersecting the terms with an automaton.
+ * <p>
+ * By design of the UniformSplit block keys, it is less efficient than
+ * {@code org.apache.lucene.codecs.blocktree.IntersectTermsEnum} for {@link org.apache.lucene.search.FuzzyQuery} (-37%).
+ * It is slightly slower for {@link org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for
+ * {@link org.apache.lucene.search.PrefixQuery} (+5%).
+ *
+ * @lucene.experimental
  */
 public class IntersectBlockReader extends BlockReader {
 
-  protected final AutomatonNextTermCalculator nextStringCalculator;
-  protected final ByteRunAutomaton runAutomaton;
-  protected final BytesRef commonSuffixRef; // maybe null
-  protected final BytesRef commonPrefixRef;
-  protected final BytesRef startTerm; // maybe null
+  /**
+   * Block iteration order. Whether to move next block, jump to a block away, or end the iteration.
+   */
+  protected enum BlockIteration {NEXT, SEEK, END}
 
-  /** Set this when our current mode is seeking to this term.  Set to null after. */
-  protected BytesRef seekTerm;
+  /**
+   * Threshold that controls when to attempt to jump to a block away.
+   * <p>
+   * This counter is 0 when entering a block. It is incremented each time a term is rejected by the automaton.
+   * When the counter is greater than or equal to this threshold, then we compute the next term accepted by
+   * the automaton, with {@link AutomatonNextTermCalculator}, and we jump to a block away if the next term
+   * accepted is greater than the immediate next term in the block.
+   * <p>
+   * A low value, for example 1, improves the performance of automatons requiring many jumps, for example
+   * {@link org.apache.lucene.search.FuzzyQuery} and most {@link org.apache.lucene.search.WildcardQuery}.
+   * A higher value improves the performance of automatons with less or no jump, for example
+   * {@link org.apache.lucene.search.PrefixQuery}.
+   * A threshold of 4 seems to be a good balance.
+   */
+  protected final int NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD = 4;
 
-  protected int blockPrefixRunAutomatonState;
-  protected int blockPrefixLen;
+  protected final Automaton automaton;
+  protected final ByteRunAutomaton runAutomaton;
+  protected final boolean finite;
+  protected final BytesRef commonSuffix; // maybe null
+  protected final int minTermLength;
+  protected final AutomatonNextTermCalculator nextStringCalculator;
 
   /**
-   * Number of bytes accepted by the last call to {@link #runAutomatonForState}.
+   * Set this when our current mode is seeking to this term.  Set to null after.
+   */
+  protected BytesRef seekTerm;
+  /**
+   * Number of bytes accepted by the automaton when validating the current term.
+   */
+  protected int numMatchedBytes;
+  /**
+   * Automaton states reached when validating the current term, from 0 to {@link #numMatchedBytes} - 1.
+   */
+  protected int[] states;
+  /**
+   * Block iteration order determined when scanning the terms in the current block.
    */
-  protected int numBytesAccepted;
+  protected BlockIteration blockIteration;
   /**
-   * Whether the current term is beyond the automaton common prefix.
-   * If true this means the enumeration should stop immediately.
+   * Counter of the number of consecutively rejected terms.
+   * Depending on {@link #NUM_CONSECUTIVELY_REJECTED_TERMS_THRESHOLD}, this may trigger a jump to a block away.
    */
-  protected boolean beyondCommonPrefix;
+  protected int numConsecutivelyRejectedTerms;
 
-  public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
-                              IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
-                              FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
+  protected IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
+                                 IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput,
+                                 PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
+                                 BlockDecoder blockDecoder) throws IOException {
     super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
-    this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);
-    Automaton automaton = Objects.requireNonNull(compiled.automaton);
-    this.runAutomaton = Objects.requireNonNull(compiled.runAutomaton);
-    this.commonSuffixRef = compiled.commonSuffixRef; // maybe null
-    this.commonPrefixRef = Operations.getCommonPrefixBytesRef(automaton); // never null
-
-    this.startTerm = startTerm;
-    assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef);
-    // it is thus also true that startTerm >= commonPrefixRef
-
-    this.seekTerm = startTerm != null ? startTerm : commonPrefixRef;
+    checkIntersectAutomatonType(compiled);
+    automaton = compiled.automaton;
+    runAutomaton = compiled.runAutomaton;
+    finite = compiled.finite;
+    commonSuffix = compiled.commonSuffixRef;
+    minTermLength = getMinTermLength();
+    nextStringCalculator = new AutomatonNextTermCalculator(compiled);
+    seekTerm = startTerm;
   }
 
-  @Override
-  public BytesRef next() throws IOException {
-    clearTermState();
-
-    if (blockHeader == null) { // initial state
-      // note: don't call super.seekCeil here; we have our own logic
-
-      // Set the browser position to the block having the seek term.
-      // Even if -1, it's okay since we'll soon call nextKey().
-      long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
-      if (isBeyondLastTerm(seekTerm, blockStartFP)) {
-        return null; // EOF
-      }
-
-      // Starting at this block find and load the next matching block.
-      //   note: Since seekBlock was just called, we actually consider the current block as "next".
-      if (nextBlockMatchingPrefix() == false) { // note: starts at seek'ed block, which may have a match
-        return null; // EOF
-      }
+  protected void checkIntersectAutomatonType(CompiledAutomaton compiled) {
+    if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+      throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
     }
-
-    do {
-
-      // look in the rest of this block.
-      BytesRef term = nextTermInBlockMatching();
-      if (term != null) {
-        return term;
-      }
-
-      // next term dict matching prefix
-    } while (nextBlockMatchingPrefix());
-
-    return null; // EOF
   }
 
   /**
-   * Find the next block that appears to contain terms that could match the automata.
-   * The prefix is the primary clue.  Returns true if at one, or false for no more (EOF).
+   * Computes the minimal length of the terms accepted by the automaton.
+   * This speeds up the term scanning for automatons accepting a finite language.
    */
-  protected boolean nextBlockMatchingPrefix() throws IOException {
-    if (beyondCommonPrefix) {
-      return false; // EOF
+  protected int getMinTermLength() {
 
 Review comment:
   Good point for the test. I'll try to craft one, when I have some time.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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