LUCENE-9237: Faster UniformSplit intersect TermsEnum.

Closes #1270
This commit is contained in:
Bruno Roustant 2020-02-28 11:17:24 +01:00
parent d10db0d98f
commit 99af698107
No known key found for this signature in database
GPG Key ID: CD28DABB95360525
8 changed files with 327 additions and 383 deletions

View File

@ -177,6 +177,8 @@ Improvements
* LUCENE-9245: Reduce AutomatonTermsEnum memory usage. (Bruno Roustant, Robert Muir)
* LUCENE-9237: Faster UniformSplit intersect TermsEnum. (Bruno Roustant)
Optimizations
---------------------

View File

@ -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);

View File

@ -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<Long> 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<Long> 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<Long> 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;
}
}

View File

@ -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;
}
/**

View File

@ -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.
* <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;
/**
* 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.
* <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 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.
*
* <p>
* 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.
*
* <p>
* This method traverses the DFA from the given position in the String,
* starting at the given state.
*
* <p>
* If this cannot satisfy the machine, returns false. This method will
* walk the minimal path, in lexicographic order, as long as possible.
*
* <p>
* 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 */
}
}
}
}

View File

@ -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;
}

View File

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

View File

@ -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);
}