diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index fbfc88cdbd9..4ad15f5c4e1 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -177,6 +177,8 @@ Improvements * LUCENE-9245: Reduce AutomatonTermsEnum memory usage. (Bruno Roustant, Robert Muir) +* LUCENE-9237: Faster UniformSplit intersect TermsEnum. (Bruno Roustant) + Optimizations --------------------- diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/BlockReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/BlockReader.java index 8d4bfc04328..f510c5f9cac 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/BlockReader.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/BlockReader.java @@ -129,6 +129,7 @@ public class BlockReader extends BaseTermsEnum implements Accountable { // Scratch objects to avoid object reallocation. protected BytesRef scratchBlockBytes; protected final BlockTermState scratchTermState; + protected BlockLine scratchBlockLine; /** * @param dictionaryBrowserSupplier to load the {@link IndexDictionary.Browser} @@ -300,7 +301,7 @@ public class BlockReader extends BaseTermsEnum implements Accountable { } boolean isIncrementalEncodingSeed = lineIndexInBlock == 0 || lineIndexInBlock == blockHeader.getMiddleLineIndex(); lineIndexInBlock++; - return blockLine = blockLineReader.readLine(blockReadBuffer, isIncrementalEncodingSeed, blockLine); + return blockLine = blockLineReader.readLine(blockReadBuffer, isIncrementalEncodingSeed, scratchBlockLine); } /** @@ -413,6 +414,7 @@ public class BlockReader extends BaseTermsEnum implements Accountable { termStatesReadBuffer = new ByteArrayDataInput(); termStateSerializer = createDeltaBaseTermStateSerializer(); scratchBlockBytes = new BytesRef(); + scratchBlockLine = new BlockLine(new TermBytes(0, scratchBlockBytes), 0); } } @@ -559,7 +561,7 @@ public class BlockReader extends BaseTermsEnum implements Accountable { termStateForced = false; } - private CorruptIndexException newCorruptIndexException(String msg, Long fp) { + protected CorruptIndexException newCorruptIndexException(String msg, Long fp) { return new CorruptIndexException(msg + (fp == null ? "" : " at FP " + fp) + " for field \"" + fieldMetadata.getFieldInfo().name + "\"", blockInput); diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/FSTDictionary.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/FSTDictionary.java index fa0e5eaf9f2..08dbacfd08d 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/FSTDictionary.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/FSTDictionary.java @@ -25,10 +25,8 @@ import org.apache.lucene.store.DataInput; import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.IndexInput; import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.BytesRefBuilder; import org.apache.lucene.util.IntsRefBuilder; import org.apache.lucene.util.RamUsageEstimator; -import org.apache.lucene.util.StringHelper; import org.apache.lucene.util.fst.BytesRefFSTEnum; import org.apache.lucene.util.fst.FST; import org.apache.lucene.util.fst.FSTCompiler; @@ -113,81 +111,10 @@ public class FSTDictionary implements IndexDictionary { protected final BytesRefFSTEnum fstEnum = new BytesRefFSTEnum<>(dictionary); - protected static final int STATE_SEEK = 0, STATE_NEXT = 1, STATE_END = 2; - protected int state = STATE_SEEK; - - // Note: key and pointer are one position prior to the current fstEnum position, - // since we need need the fstEnum to be one ahead to calculate the prefix. - protected final BytesRefBuilder keyBuilder = new BytesRefBuilder(); - protected int blockPrefixLen = 0; - protected long blockFilePointer = -1; - @Override public long seekBlock(BytesRef term) throws IOException { - state = STATE_SEEK; BytesRefFSTEnum.InputOutput seekFloor = fstEnum.seekFloor(term); - if (seekFloor == null) { - blockFilePointer = -1; - } else { - blockFilePointer = seekFloor.output; - } - return blockFilePointer; - } - - @Override - public BytesRef nextKey() throws IOException { - if (state == STATE_END) { - // if fstEnum is at end, then that's it. - return null; - } - - if (state == STATE_SEEK && blockFilePointer == -1) { // see seekBlock - if (fstEnum.next() == null) { // advance. - state = STATE_END; // probably never happens (empty FST)? We code defensively. - return null; - } - } - keyBuilder.copyBytes(fstEnum.current().input); - blockFilePointer = fstEnum.current().output; - assert blockFilePointer >= 0; - - state = STATE_NEXT; - - BytesRef key = keyBuilder.get(); - - // advance fstEnum - BytesRefFSTEnum.InputOutput inputOutput = fstEnum.next(); - - // calc common prefix - if (inputOutput == null) { - state = STATE_END; // for *next* call; current state is good - blockPrefixLen = 0; - } else { - int sortKeyLength = StringHelper.sortKeyLength(key, inputOutput.input); - assert sortKeyLength >= 1; - blockPrefixLen = sortKeyLength - 1; - } - return key; - } - - @Override - public BytesRef peekKey() { - assert state != STATE_SEEK; - return (state == STATE_END) ? null : fstEnum.current().input; - } - - @Override - public int getBlockPrefixLen() { - assert state != STATE_SEEK; - assert blockPrefixLen >= 0; - return blockPrefixLen; - } - - @Override - public long getBlockFilePointer() { - assert state != STATE_SEEK; - assert blockFilePointer >= 0; - return blockFilePointer; + return seekFloor == null ? -1 : seekFloor.output; } } diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IndexDictionary.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IndexDictionary.java index 60a3405ef0f..e59748673d9 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IndexDictionary.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IndexDictionary.java @@ -102,31 +102,6 @@ public interface IndexDictionary extends Accountable { * term precedes alphabetically the first block key of the dictionary. */ long seekBlock(BytesRef term) throws IOException; - - /** - * Returns the next block key and positions the browser at this key. - * A key is a prefix of a term in the dictionary. - * If seekBlock was just called then this is the current block key. - */ - BytesRef nextKey() throws IOException; - - /** - * Returns the next key without advancing. - * Only call this after {@link #nextKey()} returns a non-null result. - */ - BytesRef peekKey() throws IOException; - - /** - * Returns the number of characters of this block's key that is in common with all terms in this block. - * Only call this after {@link #nextKey()} returns a non-null result. - */ - int getBlockPrefixLen() throws IOException; - - /** - * Returns the block file pointer associated with the key returned. - * Only call this after {@link #nextKey()} returns a non-null result. - */ - long getBlockFilePointer() throws IOException; } /** diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java index 0322ebfe65d..ab4af8609d2 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java @@ -18,260 +18,308 @@ 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. + *

+ * 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; + /** + * Block iteration order. Whether to move next block, jump to a block away, or end the iteration. + */ + protected enum BlockIteration {NEXT, SEEK, END} + + /** + * Threshold that controls when to attempt to jump to a block away. + *

+ * 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. + *

+ * 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 final Automaton automaton; protected final ByteRunAutomaton runAutomaton; - protected final BytesRef commonSuffixRef; // maybe null - protected final BytesRef commonPrefixRef; - protected final BytesRef startTerm; // maybe null + protected final boolean finite; + protected final BytesRef commonSuffix; // maybe null + protected final int minTermLength; + protected final AutomatonNextTermCalculator nextStringCalculator; - /** Set this when our current mode is seeking to this term. Set to null after. */ + /** + * Set this when our current mode is seeking to this term. Set to null after. + */ protected BytesRef seekTerm; - - protected int blockPrefixRunAutomatonState; - protected int blockPrefixLen; - /** - * Number of bytes accepted by the last call to {@link #runAutomatonForState}. + * Number of bytes accepted by the automaton when validating the current term. */ - protected int numBytesAccepted; + protected int numMatchedBytes; /** - * Whether the current term is beyond the automaton common prefix. - * If true this means the enumeration should stop immediately. + * Automaton states reached when validating the current term, from 0 to {@link #numMatchedBytes} - 1. */ - protected boolean beyondCommonPrefix; + protected int[] states; + /** + * Block iteration order determined when scanning the terms in the current block. + */ + protected BlockIteration blockIteration; + /** + * 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 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 + automaton = compiled.automaton; + runAutomaton = compiled.runAutomaton; + finite = compiled.finite; + commonSuffix = compiled.commonSuffixRef; + minTermLength = getMinTermLength(); + nextStringCalculator = new AutomatonNextTermCalculator(compiled); + seekTerm = startTerm; + } - this.startTerm = startTerm; - assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef); - // it is thus also true that startTerm >= commonPrefixRef - - this.seekTerm = startTerm != null ? startTerm : commonPrefixRef; + /** + * Computes the minimal length of the terms accepted by the automaton. + * This speeds up the term scanning for automatons accepting a finite language. + */ + 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; + } + // Since we are only dealing with finite language, there is no loop to detect. + int commonPrefixLength = 0; + int state = 0; + Transition t = null; + 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 (automaton.getNumTransitions(state) == 1) { + if (t == null) { + t = new Transition(); + } + automaton.getTransition(state, 0, t); + if (t.min == t.max) { + state = t.dest; + commonPrefixLength++; + continue; + } + } + break; + } + // Min term length is the sum of common prefix and common suffix lengths. + return commonPrefixLength + commonSuffixLength; } @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 + if (blockHeader == null) { + if (!seekFirstBlock()) { + return null; } + states = new int[32]; + blockIteration = BlockIteration.NEXT; } - + termState = null; do { - - // look in the rest of this block. BytesRef term = nextTermInBlockMatching(); if (term != null) { return term; } + } while (nextBlock()); + return null; + } - // next term dict matching prefix - } while (nextBlockMatchingPrefix()); - - return null; // EOF + 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 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). + * 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 boolean nextBlockMatchingPrefix() throws IOException { - if (beyondCommonPrefix) { - return false; // EOF + protected BytesRef nextTermInBlockMatching() throws IOException { + 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; } - - 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 + while (true) { + TermBytes lineTermBytes = blockLine.getTermBytes(); + BytesRef lineTerm = lineTermBytes.getTerm(); + assert lineTerm.offset == 0; + if (states.length <= lineTerm.length) { + states = ArrayUtil.growExact(states, ArrayUtil.oversize(lineTerm.length + 1, Byte.BYTES)); } - - 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 + // Since terms are 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 (lineTerm.length >= minTermLength && (commonSuffix == null || endsWithCommonSuffix(lineTerm.bytes, lineTerm.length))) { + int state = states[index]; + while (true) { + if (index == lineTerm.length) { + if (runAutomaton.isAccept(state)) { + // The automaton accepts the current term. Record the number of matched bytes and return the term. + assert runAutomaton.run(lineTerm.bytes, 0, lineTerm.length); + numMatchedBytes = index; + if (numConsecutivelyRejectedTerms > 0) { + numConsecutivelyRejectedTerms = 0; + } + assert blockIteration == BlockIteration.NEXT; + return lineTerm; + } + break; + } + state = runAutomaton.step(state, lineTerm.bytes[index] & 0xff); + if (state == -1) { + // The automaton rejects the current term. + break; + } + // Record the reached automaton state. + states[++index] = state; + } } - - if (blockPrefixRunAutomatonState >= 0) { - break; // a match + // 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(lineTerm.bytes, 0, lineTerm.length); + numMatchedBytes = index; + // If the number of consecutively rejected terms reaches the threshold, + // then determine whether it is worthwhile 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; + } + // It is worthwhile 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 scanning this block terms and set the iteration order to jump to a block away by seeking seekTerm. + blockIteration = BlockIteration.SEEK; + return null; + } + seekTerm = null; + // If it is not worthwhile to jump to a block away, do not attempt anymore for the current block. + numConsecutivelyRejectedTerms = Integer.MIN_VALUE; + } else if (readLineInBlock() == null) { + // No more terms in the block. The iteration order is to open the very next block. + assert blockIteration == BlockIteration.NEXT; + return null; } + } + } - // - // 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; + /** + * Indicates whether the given term ends with the automaton common suffix. + * This allows to quickly skip terms that the automaton would reject eventually. + */ + protected boolean endsWithCommonSuffix(byte[] termBytes, int termLength) { + byte[] suffixBytes = commonSuffix.bytes; + int suffixLength = commonSuffix.length; + int offset = termLength - suffixLength; + assert offset >= 0; // We already checked minTermLength. + for (int i = 0; i < suffixLength; i++) { + if (termBytes[offset + i] != suffixBytes[i]) { + 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? - } - - // 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 - } - 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()); - + } return true; } /** - * Find the next block line that matches, or null when at end of block. + * Opens the next block. + * Depending on the {@link #blockIteration} order, it may be the very next block, or a block away that may contain + * {@link #seekTerm}. + * + * @return true if the next block is opened; false if there is no blocks anymore and the iteration is over. */ - 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; - } - } - assert blockLine != null; - } else { - if (readLineInBlock() == null) { - 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) { - return null; - } - } - - } while (true); - } - - protected boolean runAutomatonFromPrefix(BytesRef term) { - int state = runAutomatonForState(term.bytes, term.offset + blockPrefixLen, term.length - blockPrefixLen, blockPrefixRunAutomatonState); - if (state >= 0 && runAutomaton.isAccept(state)) { - return true; - } - if (isBeyondCommonPrefix(term)) { - // If the automaton rejects early the term, before the common prefix length, - // and if the term rejected byte is lexicographically after the same byte in the common prefix, - // then it means the current term is beyond the common prefix. - // Exit immediately the enumeration. - beyondCommonPrefix = true; - } - return false; - } - - /** - * Run the automaton and return the final state (not necessary accepted). -1 signifies no state / no match. - * Sets {@link #numBytesAccepted} with the offset of the first byte rejected by the automaton; - * or (offset + length) if no byte is rejected. - */ - protected int runAutomatonForState(byte[] s, int offset, int length, int initialState) { - //see ByteRunAutomaton.run(); similar - int state = initialState; - int index = 0; - while (index < length) { - state = runAutomaton.step(state, s[index + offset] & 0xFF); - if (state == -1) { + protected boolean nextBlock() throws IOException { + long blockStartFP; + switch (blockIteration) { + case NEXT: + assert seekTerm == null; + blockStartFP = blockInput.getFilePointer(); break; - } - index++; + case SEEK: + assert seekTerm != null; + blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm); + if (isBeyondLastTerm(seekTerm, blockStartFP)) { + return false; + } + blockIteration = BlockIteration.NEXT; + break; + case END: + return false; + default: + throw new UnsupportedOperationException("Unsupported " + BlockIteration.class.getSimpleName()); } - numBytesAccepted = index; - return state; - } - - /** - * Determines if the provided {@link BytesRef} is beyond the automaton common prefix. - * This method must be called after a call to {@link #runAutomatonForState} because - * it uses {@link #numBytesAccepted} value. - */ - protected boolean isBeyondCommonPrefix(BytesRef bytesRef) { - // If the automaton rejects early the bytes, before the common prefix length, - // and if the rejected byte is lexicographically after the same byte in the common prefix, - // then it means the bytes are beyond the common prefix. - return numBytesAccepted < commonPrefixRef.length - && numBytesAccepted < bytesRef.length - && (bytesRef.bytes[numBytesAccepted + bytesRef.offset] & 0xFF) > (commonPrefixRef.bytes[numBytesAccepted + commonPrefixRef.offset] & 0xFF); + numMatchedBytes = 0; + numConsecutivelyRejectedTerms = 0; + initializeHeader(seekTerm, blockStartFP); + return blockHeader != null; } @Override @@ -284,65 +332,67 @@ public class IntersectBlockReader extends BlockReader { throw new UnsupportedOperationException(); } - @Override - public SeekStatus seekCeil(BytesRef text) { - throw new UnsupportedOperationException(); - } - @Override public void seekExact(BytesRef term, TermState state) { throw new UnsupportedOperationException(); } + @Override + public SeekStatus seekCeil(BytesRef text) { + throw new UnsupportedOperationException(); + } + /** - * This is a copy of AutomatonTermsEnum. Since it's an inner class, the outer class can + * This is mostly a copy of AutomatonTermsEnum. Since it's an inner class, the outer class can * call methods that ATE does not expose. It'd be nice if ATE's logic could be more extensible. */ - protected static class AutomatonNextTermCalculator { - // a tableized array-based form of the DFA - protected final ByteRunAutomaton runAutomaton; - // common suffix of the automaton - protected final BytesRef commonSuffixRef; - // true if the automaton accepts a finite language - protected final boolean finite; - // array of sorted transitions for each state, indexed by state number - protected final Automaton automaton; - // for path tracking: each long records gen when we last + protected class AutomatonNextTermCalculator { + // for path tracking: each short records gen when we last // visited the state; we use gens to avoid having to clear - protected final long[] visited; - protected long curGen; + protected final short[] visited; + protected short curGen; // the reference used for seeking forwards through the term dictionary protected final BytesRefBuilder seekBytesRef = new BytesRefBuilder(); // true if we are enumerating an infinite portion of the DFA. // in this case it is faster to drive the query based on the terms dictionary. // when this is true, linearUpperBound indicate the end of range // of terms where we should simply do sequential reads instead. - protected boolean linear = false; - protected final BytesRef linearUpperBound = new BytesRef(10); - protected Transition transition = new Transition(); + protected boolean linear; + protected final BytesRef linearUpperBound = new BytesRef(); + protected final Transition transition = new Transition(); protected final IntsRefBuilder savedStates = new IntsRefBuilder(); protected AutomatonNextTermCalculator(CompiledAutomaton compiled) { - if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) { - throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead"); - } - this.finite = compiled.finite; - this.runAutomaton = compiled.runAutomaton; - assert this.runAutomaton != null; - this.commonSuffixRef = compiled.commonSuffixRef; - this.automaton = compiled.automaton; - - // used for path tracking, where each bit is a numbered state. - visited = new long[runAutomaton.getSize()]; + visited = compiled.finite ? null : new short[runAutomaton.getSize()]; } - /** True if the current state of the automata is best iterated linearly (without seeking). */ + /** + * Records the given state has been visited. + */ + protected void setVisited(int state) { + if (!finite) { + visited[state] = curGen; + } + } + + /** + * Indicates whether the given state has been visited. + */ + protected boolean isVisited(int state) { + return !finite && visited[state] == curGen; + } + + /** + * True if the current state of the automata is best iterated linearly (without seeking). + */ protected boolean isLinearState(BytesRef term) { return linear && term.compareTo(linearUpperBound) < 0; } - /** @see org.apache.lucene.index.FilteredTermsEnum#nextSeekTerm(BytesRef) */ - protected BytesRef nextSeekTerm(final BytesRef term) throws IOException { + /** + * @see org.apache.lucene.index.FilteredTermsEnum#nextSeekTerm(BytesRef) + */ + protected BytesRef nextSeekTerm(final BytesRef term) { //System.out.println("ATE.nextSeekTerm term=" + term); if (term == null) { assert seekBytesRef.length() == 0; @@ -371,12 +421,11 @@ public class IntersectBlockReader extends BlockReader { assert linear == false; int state = 0; - assert state == 0; int maxInterval = 0xff; //System.out.println("setLinear pos=" + position + " seekbytesRef=" + seekBytesRef); for (int i = 0; i < position; i++) { state = runAutomaton.step(state, seekBytesRef.byteAt(i) & 0xff); - assert state >= 0: "state=" + state; + assert state >= 0 : "state=" + state; } final int numTransitions = automaton.getNumTransitions(state); automaton.initTransition(state, transition); @@ -392,8 +441,9 @@ public class IntersectBlockReader extends BlockReader { if (maxInterval != 0xff) maxInterval++; int length = position + 1; /* position + maxTransition */ - if (linearUpperBound.bytes.length < length) - linearUpperBound.bytes = new byte[length]; + if (linearUpperBound.bytes.length < length) { + linearUpperBound.bytes = new byte[ArrayUtil.oversize(length, Byte.BYTES)]; + } System.arraycopy(seekBytesRef.bytes(), 0, linearUpperBound.bytes, 0, position); linearUpperBound.bytes[position] = (byte) maxInterval; linearUpperBound.length = length; @@ -405,7 +455,7 @@ public class IntersectBlockReader extends BlockReader { * Increments the byte buffer to the next String in binary order after s that will not put * the machine into a reject state. If such a string does not exist, returns * false. - * + *

* The correctness of this method depends upon the automaton being deterministic, * and having no transitions to dead states. * @@ -414,21 +464,24 @@ public class IntersectBlockReader extends BlockReader { protected boolean nextString() { int state; int pos = 0; - savedStates.grow(seekBytesRef.length()+1); + savedStates.grow(seekBytesRef.length() + 1); savedStates.setIntAt(0, 0); while (true) { - curGen++; + if (!finite && ++curGen == 0) { + // Clear the visited states every time curGen wraps (so very infrequently to not impact average perf). + Arrays.fill(visited, (short) -1); + } linear = false; // walk the automaton until a character is rejected. for (state = savedStates.intAt(pos); pos < seekBytesRef.length(); pos++) { - visited[state] = curGen; + setVisited(state); int nextState = runAutomaton.step(state, seekBytesRef.byteAt(pos) & 0xff); if (nextState == -1) break; - savedStates.setIntAt(pos+1, nextState); + savedStates.setIntAt(pos + 1, nextState); // we found a loop, record it for faster enumeration - if (!finite && !linear && visited[nextState] == curGen) { + if (!linear && isVisited(nextState)) { setLinear(pos); } state = nextState; @@ -446,7 +499,7 @@ public class IntersectBlockReader extends BlockReader { /* String is good to go as-is */ return true; /* else advance further */ - // TODO: paranoia? if we backtrack thru an infinite DFA, the loop detection is important! + // paranoia? if we backtrack thru an infinite DFA, the loop detection is important! // for now, restart from scratch for all infinite DFAs if (!finite) pos = 0; } @@ -456,20 +509,20 @@ public class IntersectBlockReader extends BlockReader { /** * Returns the next String in lexicographic order that will not put * the machine into a reject state. - * + *

* This method traverses the DFA from the given position in the String, * starting at the given state. - * + *

* If this cannot satisfy the machine, returns false. This method will * walk the minimal path, in lexicographic order, as long as possible. - * + *

* If this method returns false, then there might still be more solutions, * it is necessary to backtrack to find out. * - * @param state current non-reject state + * @param state current non-reject state * @param position useful portion of the string * @return true if more possible solutions exist for the DFA from this - * position + * position */ protected boolean nextString(int state, int position) { /* @@ -487,7 +540,7 @@ public class IntersectBlockReader extends BlockReader { } seekBytesRef.setLength(position); - visited[state] = curGen; + setVisited(state); final int numTransitions = automaton.getNumTransitions(state); automaton.initTransition(state, transition); @@ -505,8 +558,8 @@ public class IntersectBlockReader extends BlockReader { * as long as is possible, continue down the minimal path in * lexicographic order. if a loop or accept state is encountered, stop. */ - while (visited[state] != curGen && !runAutomaton.isAccept(state)) { - visited[state] = curGen; + while (!isVisited(state) && !runAutomaton.isAccept(state)) { + setVisited(state); /* * Note: we work with a DFA with no transitions to dead states. * so the below is ok, if it is not an accept state, @@ -521,8 +574,8 @@ public class IntersectBlockReader extends BlockReader { seekBytesRef.append((byte) transition.min); // we found a loop, record it for faster enumeration - if (!finite && !linear && visited[state] == curGen) { - setLinear(seekBytesRef.length()-1); + if (!linear && isVisited(state)) { + setLinear(seekBytesRef.length() - 1); } } return true; @@ -546,13 +599,11 @@ public class IntersectBlockReader extends BlockReader { // because there is no higher character in binary sort order. if (nextChar++ != 0xff) { seekBytesRef.setByteAt(position, (byte) nextChar); - seekBytesRef.setLength(position+1); + seekBytesRef.setLength(position + 1); return position; } } return -1; /* all solutions exhausted */ } } - -} - +} \ No newline at end of file diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STBlockReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STBlockReader.java index 6d7c79d6f65..8fef5202992 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STBlockReader.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STBlockReader.java @@ -48,18 +48,15 @@ public class STBlockReader extends BlockReader { @Override public BytesRef next() throws IOException { - BytesRef next = super.next(); - if (next == null) { - return null; - } - // Check if the term occurs for the searched field. - while (!termOccursInField()) { + BytesRef next; + do { next = super.next(); if (next == null) { - // No more term for any field. + // No more terms. return null; } - } + // Check if the term occurs for the searched field. + } while (!termOccursInField()); // The term occurs for the searched field. return next; } diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STIntersectBlockReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STIntersectBlockReader.java index 099b6c31cef..89159f84675 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STIntersectBlockReader.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STIntersectBlockReader.java @@ -64,18 +64,15 @@ public class STIntersectBlockReader extends IntersectBlockReader { @Override public BytesRef next() throws IOException { - BytesRef next = super.next(); - if (next == null) { - return null; - } - // Check if the term occurs for the searched field. - while (!termOccursInField()) { + BytesRef next; + do { next = super.next(); if (next == null) { - // No more term. + // No more terms. return null; } - } + // Check if the term occurs for the searched field. + } while (!termOccursInField()); // The term occurs for the searched field. return next; } @@ -85,22 +82,14 @@ public class STIntersectBlockReader extends IntersectBlockReader { return termState != null; } - @Override - protected boolean nextBlockMatchingPrefix() throws IOException { - // block header maybe null if we are positioned outside the field block range - return super.nextBlockMatchingPrefix() && blockHeader != null; - } - @Override protected STBlockLine.Serializer createBlockLineSerializer() { return new STBlockLine.Serializer(); } /** - * Reads the {@link BlockTermState} on the current line for the specific field - * corresponding this this reader. - * Changes the current {@link BlockTermState} to null if the term does not - * occur for the field. + * Reads the {@link BlockTermState} on the current line for the specific field corresponding to this reader. + * Returns null if the term does not occur for the field. */ @Override protected BlockTermState readTermState() throws IOException { diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STUniformSplitTerms.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STUniformSplitTerms.java index 6e11ae9ba08..01d374c364e 100755 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STUniformSplitTerms.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STUniformSplitTerms.java @@ -51,6 +51,7 @@ public class STUniformSplitTerms extends UniformSplitTerms { @Override public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException { + checkIntersectAutomatonType(compiled); return new STIntersectBlockReader(compiled, startTerm, dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder, fieldInfos); }