mirror of https://github.com/apache/lucene.git
Use `IndexInput#prefetch` for terms dictionary lookups. (#13359)
This introduces `TermsEnum#prepareSeekExact`, which essentially calls `IndexInput#prefetch` at the right offset for the given term. Then it takes advantage of the fact that `BooleanQuery` already calls `Weight#scorerSupplier` on all clauses, before later calling `ScorerSupplier#get` on all clauses. So `TermQuery` now calls `TermsEnum#prepareSeekExact` on `Weight#scorerSupplier` (if scores are not needed), which in-turn means that the I/O all terms dictionary lookups get parallelized across all term queries of a `BooleanQuery` on a given segment (intra-segment parallelism).
This commit is contained in:
parent
da41215a67
commit
026d661e5f
|
@ -43,6 +43,7 @@ import org.apache.lucene.store.ChecksumIndexInput;
|
||||||
import org.apache.lucene.store.DataOutput;
|
import org.apache.lucene.store.DataOutput;
|
||||||
import org.apache.lucene.store.IndexOutput;
|
import org.apache.lucene.store.IndexOutput;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
import org.apache.lucene.util.IOBooleanSupplier;
|
||||||
import org.apache.lucene.util.IOUtils;
|
import org.apache.lucene.util.IOUtils;
|
||||||
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||||
|
|
||||||
|
@ -315,12 +316,21 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean seekExact(BytesRef text) throws IOException {
|
public IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException {
|
||||||
// The magical fail-fast speed up that is the entire point of all of
|
// The magical fail-fast speed up that is the entire point of all of
|
||||||
// this code - save a disk seek if there is a match on an in-memory
|
// this code - save a disk seek if there is a match on an in-memory
|
||||||
// structure
|
// structure
|
||||||
// that may occasionally give a false positive but guaranteed no false
|
// that may occasionally give a false positive but guaranteed no false
|
||||||
// negatives
|
// negatives
|
||||||
|
if (filter.contains(text) == ContainsResult.NO) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
return delegate().prepareSeekExact(text);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean seekExact(BytesRef text) throws IOException {
|
||||||
|
// See #prepareSeekExact
|
||||||
if (filter.contains(text) == ContainsResult.NO) {
|
if (filter.contains(text) == ContainsResult.NO) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,11 +20,11 @@ package org.apache.lucene.codecs.uniformsplit.sharedterms;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.RandomAccess;
|
import java.util.RandomAccess;
|
||||||
|
import org.apache.lucene.index.BaseTermsEnum;
|
||||||
import org.apache.lucene.index.ImpactsEnum;
|
import org.apache.lucene.index.ImpactsEnum;
|
||||||
import org.apache.lucene.index.MergeState;
|
import org.apache.lucene.index.MergeState;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
import org.apache.lucene.index.PostingsEnum;
|
||||||
import org.apache.lucene.index.TermState;
|
import org.apache.lucene.index.TermState;
|
||||||
import org.apache.lucene.index.TermsEnum;
|
|
||||||
import org.apache.lucene.util.AttributeSource;
|
import org.apache.lucene.util.AttributeSource;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
|
||||||
|
@ -34,7 +34,7 @@ import org.apache.lucene.util.BytesRef;
|
||||||
*
|
*
|
||||||
* @lucene.experimental
|
* @lucene.experimental
|
||||||
*/
|
*/
|
||||||
class STMergingTermsEnum extends TermsEnum {
|
class STMergingTermsEnum extends BaseTermsEnum {
|
||||||
|
|
||||||
protected final String fieldName;
|
protected final String fieldName;
|
||||||
protected final MultiSegmentsPostingsEnum multiPostingsEnum;
|
protected final MultiSegmentsPostingsEnum multiPostingsEnum;
|
||||||
|
@ -63,11 +63,6 @@ class STMergingTermsEnum extends TermsEnum {
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean seekExact(BytesRef text) throws IOException {
|
|
||||||
throw new UnsupportedOperationException();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public SeekStatus seekCeil(BytesRef text) {
|
public SeekStatus seekCeil(BytesRef text) {
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
|
|
|
@ -23,6 +23,7 @@ import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import org.apache.lucene.index.BaseTermsEnum;
|
||||||
import org.apache.lucene.index.BinaryDocValues;
|
import org.apache.lucene.index.BinaryDocValues;
|
||||||
import org.apache.lucene.index.DocIDMerger;
|
import org.apache.lucene.index.DocIDMerger;
|
||||||
import org.apache.lucene.index.DocValues;
|
import org.apache.lucene.index.DocValues;
|
||||||
|
@ -498,7 +499,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
||||||
* {@link SortedDocValues#lookupOrd(int)} or {@link SortedSetDocValues#lookupOrd(long)} on every
|
* {@link SortedDocValues#lookupOrd(int)} or {@link SortedSetDocValues#lookupOrd(long)} on every
|
||||||
* call to {@link TermsEnum#next()}.
|
* call to {@link TermsEnum#next()}.
|
||||||
*/
|
*/
|
||||||
private static class MergedTermsEnum extends TermsEnum {
|
private static class MergedTermsEnum extends BaseTermsEnum {
|
||||||
|
|
||||||
private final TermsEnum[] subs;
|
private final TermsEnum[] subs;
|
||||||
private final OrdinalMap ordinalMap;
|
private final OrdinalMap ordinalMap;
|
||||||
|
@ -542,11 +543,6 @@ public abstract class DocValuesConsumer implements Closeable {
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean seekExact(BytesRef text) throws IOException {
|
|
||||||
throw new UnsupportedOperationException();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public SeekStatus seekCeil(BytesRef text) throws IOException {
|
public SeekStatus seekCeil(BytesRef text) throws IOException {
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
|
@ -557,11 +553,6 @@ public abstract class DocValuesConsumer implements Closeable {
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void seekExact(BytesRef term, TermState state) throws IOException {
|
|
||||||
throw new UnsupportedOperationException();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int docFreq() throws IOException {
|
public int docFreq() throws IOException {
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.lucene.store.IndexInput;
|
||||||
import org.apache.lucene.util.ArrayUtil;
|
import org.apache.lucene.util.ArrayUtil;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.BytesRefBuilder;
|
import org.apache.lucene.util.BytesRefBuilder;
|
||||||
|
import org.apache.lucene.util.IOBooleanSupplier;
|
||||||
import org.apache.lucene.util.RamUsageEstimator;
|
import org.apache.lucene.util.RamUsageEstimator;
|
||||||
import org.apache.lucene.util.fst.FST;
|
import org.apache.lucene.util.fst.FST;
|
||||||
import org.apache.lucene.util.fst.Util;
|
import org.apache.lucene.util.fst.Util;
|
||||||
|
@ -307,15 +308,13 @@ final class SegmentTermsEnum extends BaseTermsEnum {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
private IOBooleanSupplier prepareSeekExact(BytesRef target, boolean prefetch) throws IOException {
|
||||||
public boolean seekExact(BytesRef target) throws IOException {
|
|
||||||
|
|
||||||
if (fr.index == null) {
|
if (fr.index == null) {
|
||||||
throw new IllegalStateException("terms index was not loaded");
|
throw new IllegalStateException("terms index was not loaded");
|
||||||
}
|
}
|
||||||
|
|
||||||
if (fr.size() > 0 && (target.compareTo(fr.getMin()) < 0 || target.compareTo(fr.getMax()) > 0)) {
|
if (fr.size() > 0 && (target.compareTo(fr.getMin()) < 0 || target.compareTo(fr.getMax()) > 0)) {
|
||||||
return false;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
term.grow(1 + target.length);
|
term.grow(1 + target.length);
|
||||||
|
@ -431,7 +430,7 @@ final class SegmentTermsEnum extends BaseTermsEnum {
|
||||||
// if (DEBUG) {
|
// if (DEBUG) {
|
||||||
// System.out.println(" target is same as current; return true");
|
// System.out.println(" target is same as current; return true");
|
||||||
// }
|
// }
|
||||||
return true;
|
return () -> true;
|
||||||
} else {
|
} else {
|
||||||
// if (DEBUG) {
|
// if (DEBUG) {
|
||||||
// System.out.println(" target is same as current but term doesn't exist");
|
// System.out.println(" target is same as current but term doesn't exist");
|
||||||
|
@ -501,24 +500,30 @@ final class SegmentTermsEnum extends BaseTermsEnum {
|
||||||
// if (DEBUG) {
|
// if (DEBUG) {
|
||||||
// System.out.println(" FAST NOT_FOUND term=" + ToStringUtils.bytesRefToString(term));
|
// System.out.println(" FAST NOT_FOUND term=" + ToStringUtils.bytesRefToString(term));
|
||||||
// }
|
// }
|
||||||
return false;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
currentFrame.loadBlock();
|
if (prefetch) {
|
||||||
|
currentFrame.prefetchBlock();
|
||||||
final SeekStatus result = currentFrame.scanToTerm(target, true);
|
|
||||||
if (result == SeekStatus.FOUND) {
|
|
||||||
// if (DEBUG) {
|
|
||||||
// System.out.println(" return FOUND term=" + term.utf8ToString() + " " + term);
|
|
||||||
// }
|
|
||||||
return true;
|
|
||||||
} else {
|
|
||||||
// if (DEBUG) {
|
|
||||||
// System.out.println(" got " + result + "; return NOT_FOUND term=" +
|
|
||||||
// ToStringUtils.bytesRefToString(term));
|
|
||||||
// }
|
|
||||||
return false;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
return () -> {
|
||||||
|
currentFrame.loadBlock();
|
||||||
|
|
||||||
|
final SeekStatus result = currentFrame.scanToTerm(target, true);
|
||||||
|
if (result == SeekStatus.FOUND) {
|
||||||
|
// if (DEBUG) {
|
||||||
|
// System.out.println(" return FOUND term=" + term.utf8ToString() + " " + term);
|
||||||
|
// }
|
||||||
|
return true;
|
||||||
|
} else {
|
||||||
|
// if (DEBUG) {
|
||||||
|
// System.out.println(" got " + result + "; return NOT_FOUND term=" +
|
||||||
|
// ToStringUtils.bytesRefToString(term));
|
||||||
|
// }
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
};
|
||||||
} else {
|
} else {
|
||||||
// Follow this arc
|
// Follow this arc
|
||||||
arc = nextArc;
|
arc = nextArc;
|
||||||
|
@ -556,25 +561,42 @@ final class SegmentTermsEnum extends BaseTermsEnum {
|
||||||
// if (DEBUG) {
|
// if (DEBUG) {
|
||||||
// System.out.println(" FAST NOT_FOUND term=" + ToStringUtils.bytesRefToString(term));
|
// System.out.println(" FAST NOT_FOUND term=" + ToStringUtils.bytesRefToString(term));
|
||||||
// }
|
// }
|
||||||
return false;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
currentFrame.loadBlock();
|
if (prefetch) {
|
||||||
|
currentFrame.prefetchBlock();
|
||||||
final SeekStatus result = currentFrame.scanToTerm(target, true);
|
|
||||||
if (result == SeekStatus.FOUND) {
|
|
||||||
// if (DEBUG) {
|
|
||||||
// System.out.println(" return FOUND term=" + term.utf8ToString() + " " + term);
|
|
||||||
// }
|
|
||||||
return true;
|
|
||||||
} else {
|
|
||||||
// if (DEBUG) {
|
|
||||||
// System.out.println(" got result " + result + "; return NOT_FOUND term=" +
|
|
||||||
// term.utf8ToString());
|
|
||||||
// }
|
|
||||||
|
|
||||||
return false;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
return () -> {
|
||||||
|
currentFrame.loadBlock();
|
||||||
|
|
||||||
|
final SeekStatus result = currentFrame.scanToTerm(target, true);
|
||||||
|
if (result == SeekStatus.FOUND) {
|
||||||
|
// if (DEBUG) {
|
||||||
|
// System.out.println(" return FOUND term=" + term.utf8ToString() + " " + term);
|
||||||
|
// }
|
||||||
|
return true;
|
||||||
|
} else {
|
||||||
|
// if (DEBUG) {
|
||||||
|
// System.out.println(" got result " + result + "; return NOT_FOUND term=" +
|
||||||
|
// term.utf8ToString());
|
||||||
|
// }
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOBooleanSupplier prepareSeekExact(BytesRef target) throws IOException {
|
||||||
|
return prepareSeekExact(target, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean seekExact(BytesRef target) throws IOException {
|
||||||
|
IOBooleanSupplier termExistsSupplier = prepareSeekExact(target, false);
|
||||||
|
return termExistsSupplier != null && termExistsSupplier.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -133,6 +133,21 @@ final class SegmentTermsEnumFrame {
|
||||||
loadBlock();
|
loadBlock();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void prefetchBlock() throws IOException {
|
||||||
|
if (nextEnt != -1) {
|
||||||
|
// Already loaded
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Clone the IndexInput lazily, so that consumers
|
||||||
|
// that just pull a TermsEnum to
|
||||||
|
// seekExact(TermState) don't pay this cost:
|
||||||
|
ste.initIndexInput();
|
||||||
|
|
||||||
|
// TODO: Could we know the number of bytes to prefetch?
|
||||||
|
ste.in.prefetch(fp, 1);
|
||||||
|
}
|
||||||
|
|
||||||
/* Does initial decode of next block of terms; this
|
/* Does initial decode of next block of terms; this
|
||||||
doesn't actually decode the docFreq, totalTermFreq,
|
doesn't actually decode the docFreq, totalTermFreq,
|
||||||
postings details (frq/prx offset, etc.) metadata;
|
postings details (frq/prx offset, etc.) metadata;
|
||||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.index;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import org.apache.lucene.util.AttributeSource;
|
import org.apache.lucene.util.AttributeSource;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
import org.apache.lucene.util.IOBooleanSupplier;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A base TermsEnum that adds default implementations for
|
* A base TermsEnum that adds default implementations for
|
||||||
|
@ -58,6 +59,11 @@ public abstract class BaseTermsEnum extends TermsEnum {
|
||||||
return seekCeil(text) == SeekStatus.FOUND;
|
return seekCeil(text) == SeekStatus.FOUND;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException {
|
||||||
|
return () -> seekExact(text);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void seekExact(BytesRef term, TermState state) throws IOException {
|
public void seekExact(BytesRef term, TermState state) throws IOException {
|
||||||
if (!seekExact(term)) {
|
if (!seekExact(term)) {
|
||||||
|
|
|
@ -79,6 +79,7 @@ import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.BytesRefBuilder;
|
import org.apache.lucene.util.BytesRefBuilder;
|
||||||
import org.apache.lucene.util.CommandLineUtil;
|
import org.apache.lucene.util.CommandLineUtil;
|
||||||
import org.apache.lucene.util.FixedBitSet;
|
import org.apache.lucene.util.FixedBitSet;
|
||||||
|
import org.apache.lucene.util.IOBooleanSupplier;
|
||||||
import org.apache.lucene.util.IOUtils;
|
import org.apache.lucene.util.IOUtils;
|
||||||
import org.apache.lucene.util.LongBitSet;
|
import org.apache.lucene.util.LongBitSet;
|
||||||
import org.apache.lucene.util.NamedThreadFactory;
|
import org.apache.lucene.util.NamedThreadFactory;
|
||||||
|
@ -3869,6 +3870,7 @@ public final class CheckIndex implements Closeable {
|
||||||
TermsEnum postingsTermsEnum = postingsTerms.iterator();
|
TermsEnum postingsTermsEnum = postingsTerms.iterator();
|
||||||
|
|
||||||
final boolean hasProx = terms.hasOffsets() || terms.hasPositions();
|
final boolean hasProx = terms.hasOffsets() || terms.hasPositions();
|
||||||
|
int seekExactCounter = 0;
|
||||||
BytesRef term;
|
BytesRef term;
|
||||||
while ((term = termsEnum.next()) != null) {
|
while ((term = termsEnum.next()) != null) {
|
||||||
|
|
||||||
|
@ -3876,7 +3878,14 @@ public final class CheckIndex implements Closeable {
|
||||||
postings = termsEnum.postings(postings, PostingsEnum.ALL);
|
postings = termsEnum.postings(postings, PostingsEnum.ALL);
|
||||||
assert postings != null;
|
assert postings != null;
|
||||||
|
|
||||||
if (postingsTermsEnum.seekExact(term) == false) {
|
boolean termExists;
|
||||||
|
if ((seekExactCounter++ & 0x01) == 0) {
|
||||||
|
termExists = postingsTermsEnum.seekExact(term);
|
||||||
|
} else {
|
||||||
|
IOBooleanSupplier termExistsSupplier = postingsTermsEnum.prepareSeekExact(term);
|
||||||
|
termExists = termExistsSupplier != null && termExistsSupplier.get();
|
||||||
|
}
|
||||||
|
if (termExists == false) {
|
||||||
throw new CheckIndexException(
|
throw new CheckIndexException(
|
||||||
"vector term="
|
"vector term="
|
||||||
+ term
|
+ term
|
||||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.lucene.search.KnnCollector;
|
||||||
import org.apache.lucene.util.AttributeSource;
|
import org.apache.lucene.util.AttributeSource;
|
||||||
import org.apache.lucene.util.Bits;
|
import org.apache.lucene.util.Bits;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
import org.apache.lucene.util.IOBooleanSupplier;
|
||||||
import org.apache.lucene.util.Unwrappable;
|
import org.apache.lucene.util.Unwrappable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -161,6 +162,7 @@ public abstract class FilterLeafReader extends LeafReader {
|
||||||
|
|
||||||
/** Base class for filtering {@link TermsEnum} implementations. */
|
/** Base class for filtering {@link TermsEnum} implementations. */
|
||||||
public abstract static class FilterTermsEnum extends TermsEnum {
|
public abstract static class FilterTermsEnum extends TermsEnum {
|
||||||
|
|
||||||
/** The underlying TermsEnum instance. */
|
/** The underlying TermsEnum instance. */
|
||||||
protected final TermsEnum in;
|
protected final TermsEnum in;
|
||||||
|
|
||||||
|
@ -236,6 +238,11 @@ public abstract class FilterLeafReader extends LeafReader {
|
||||||
in.seekExact(term, state);
|
in.seekExact(term, state);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException {
|
||||||
|
return in.prepareSeekExact(text);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public TermState termState() throws IOException {
|
public TermState termState() throws IOException {
|
||||||
return in.termState();
|
return in.termState();
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.index;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import org.apache.lucene.util.AttributeSource;
|
import org.apache.lucene.util.AttributeSource;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
import org.apache.lucene.util.IOBooleanSupplier;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Abstract class for enumerating a subset of all terms.
|
* Abstract class for enumerating a subset of all terms.
|
||||||
|
@ -155,6 +156,16 @@ public abstract class FilteredTermsEnum extends TermsEnum {
|
||||||
throw new UnsupportedOperationException(getClass().getName() + " does not support seeking");
|
throw new UnsupportedOperationException(getClass().getName() + " does not support seeking");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This enum does not support seeking!
|
||||||
|
*
|
||||||
|
* @throws UnsupportedOperationException In general, subclasses do not support seeking.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException {
|
||||||
|
throw new UnsupportedOperationException(getClass().getName() + " does not support seeking");
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This enum does not support seeking!
|
* This enum does not support seeking!
|
||||||
*
|
*
|
||||||
|
|
|
@ -17,12 +17,12 @@
|
||||||
package org.apache.lucene.index;
|
package org.apache.lucene.index;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.function.Supplier;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
import org.apache.lucene.search.IndexSearcher;
|
import org.apache.lucene.search.IndexSearcher;
|
||||||
import org.apache.lucene.search.TaskExecutor;
|
import org.apache.lucene.util.ArrayUtil;
|
||||||
|
import org.apache.lucene.util.IOBooleanSupplier;
|
||||||
|
import org.apache.lucene.util.IOSupplier;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Maintains a {@link IndexReader} {@link TermState} view over {@link IndexReader} instances
|
* Maintains a {@link IndexReader} {@link TermState} view over {@link IndexReader} instances
|
||||||
|
@ -80,6 +80,8 @@ public final class TermStates {
|
||||||
register(state, ord, docFreq, totalTermFreq);
|
register(state, ord, docFreq, totalTermFreq);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private record PendingTermLookup(TermsEnum termsEnum, IOBooleanSupplier supplier) {}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a {@link TermStates} from a top-level {@link IndexReaderContext} and the given {@link
|
* Creates a {@link TermStates} from a top-level {@link IndexReaderContext} and the given {@link
|
||||||
* Term}. This method will lookup the given term in all context's leaf readers and register each
|
* Term}. This method will lookup the given term in all context's leaf readers and register each
|
||||||
|
@ -97,42 +99,29 @@ public final class TermStates {
|
||||||
assert context != null;
|
assert context != null;
|
||||||
final TermStates perReaderTermState = new TermStates(needsStats ? null : term, context);
|
final TermStates perReaderTermState = new TermStates(needsStats ? null : term, context);
|
||||||
if (needsStats) {
|
if (needsStats) {
|
||||||
TaskExecutor taskExecutor = indexSearcher.getTaskExecutor();
|
PendingTermLookup[] pendingTermLookups = new PendingTermLookup[0];
|
||||||
// build the term states concurrently
|
|
||||||
List<Callable<TermStateInfo>> tasks = new ArrayList<>(context.leaves().size());
|
|
||||||
for (LeafReaderContext ctx : context.leaves()) {
|
for (LeafReaderContext ctx : context.leaves()) {
|
||||||
tasks.add(
|
Terms terms = Terms.getTerms(ctx.reader(), term.field());
|
||||||
() -> {
|
TermsEnum termsEnum = terms.iterator();
|
||||||
TermsEnum termsEnum = loadTermsEnum(ctx, term);
|
// Schedule the I/O in the terms dictionary in the background.
|
||||||
return termsEnum == null
|
IOBooleanSupplier termExistsSupplier = termsEnum.prepareSeekExact(term.bytes());
|
||||||
? null
|
if (termExistsSupplier != null) {
|
||||||
: new TermStateInfo(
|
pendingTermLookups = ArrayUtil.grow(pendingTermLookups, ctx.ord + 1);
|
||||||
termsEnum.termState(),
|
pendingTermLookups[ctx.ord] = new PendingTermLookup(termsEnum, termExistsSupplier);
|
||||||
ctx.ord,
|
}
|
||||||
termsEnum.docFreq(),
|
|
||||||
termsEnum.totalTermFreq());
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
List<TermStateInfo> resultInfos = taskExecutor.invokeAll(tasks);
|
for (int ord = 0; ord < pendingTermLookups.length; ++ord) {
|
||||||
for (TermStateInfo info : resultInfos) {
|
PendingTermLookup pendingTermLookup = pendingTermLookups[ord];
|
||||||
if (info != null) {
|
if (pendingTermLookup != null && pendingTermLookup.supplier.get()) {
|
||||||
|
TermsEnum termsEnum = pendingTermLookup.termsEnum();
|
||||||
perReaderTermState.register(
|
perReaderTermState.register(
|
||||||
info.getState(), info.getOrdinal(), info.getDocFreq(), info.getTotalTermFreq());
|
termsEnum.termState(), ord, termsEnum.docFreq(), termsEnum.totalTermFreq());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return perReaderTermState;
|
return perReaderTermState;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static TermsEnum loadTermsEnum(LeafReaderContext ctx, Term term) throws IOException {
|
|
||||||
final Terms terms = Terms.getTerms(ctx.reader(), term.field());
|
|
||||||
final TermsEnum termsEnum = terms.iterator();
|
|
||||||
if (termsEnum.seekExact(term.bytes())) {
|
|
||||||
return termsEnum;
|
|
||||||
}
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Clears the {@link TermStates} internal state and removes all registered {@link TermState}s */
|
/** Clears the {@link TermStates} internal state and removes all registered {@link TermState}s */
|
||||||
public void clear() {
|
public void clear() {
|
||||||
docFreq = 0;
|
docFreq = 0;
|
||||||
|
@ -172,22 +161,60 @@ public final class TermStates {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the {@link TermState} for a leaf reader context or <code>null</code> if no {@link
|
* Returns a {@link Supplier} for a {@link TermState} for the given {@link LeafReaderContext}.
|
||||||
* TermState} for the context was registered.
|
* This may return {@code null} if some cheap checks help figure out that this term doesn't exist
|
||||||
|
* in this leaf. The {@link Supplier} may then also return {@code null} if the term doesn't exist.
|
||||||
|
*
|
||||||
|
* <p>Calling this method typically schedules some I/O in the background, so it is recommended to
|
||||||
|
* retrieve {@link Supplier}s across all required terms first before calling {@link Supplier#get}
|
||||||
|
* on all {@link Supplier}s so that the I/O for these terms can be performed in parallel.
|
||||||
*
|
*
|
||||||
* @param ctx the {@link LeafReaderContext} to get the {@link TermState} for.
|
* @param ctx the {@link LeafReaderContext} to get the {@link TermState} for.
|
||||||
* @return the {@link TermState} for the given readers ord or <code>null</code> if no {@link
|
* @return a Supplier for a TermState.
|
||||||
* TermState} for the reader was registered
|
|
||||||
*/
|
*/
|
||||||
public TermState get(LeafReaderContext ctx) throws IOException {
|
public IOSupplier<TermState> get(LeafReaderContext ctx) throws IOException {
|
||||||
assert ctx.ord >= 0 && ctx.ord < states.length;
|
assert ctx.ord >= 0 && ctx.ord < states.length;
|
||||||
if (term == null) return states[ctx.ord];
|
if (term == null) {
|
||||||
if (this.states[ctx.ord] == null) {
|
if (states[ctx.ord] == null) {
|
||||||
TermsEnum te = loadTermsEnum(ctx, term);
|
return null;
|
||||||
this.states[ctx.ord] = te == null ? EMPTY_TERMSTATE : te.termState();
|
} else {
|
||||||
|
return () -> states[ctx.ord];
|
||||||
|
}
|
||||||
}
|
}
|
||||||
if (this.states[ctx.ord] == EMPTY_TERMSTATE) return null;
|
if (this.states[ctx.ord] == null) {
|
||||||
return this.states[ctx.ord];
|
final Terms terms = ctx.reader().terms(term.field());
|
||||||
|
if (terms == null) {
|
||||||
|
this.states[ctx.ord] = EMPTY_TERMSTATE;
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
final TermsEnum termsEnum = terms.iterator();
|
||||||
|
IOBooleanSupplier termExistsSupplier = termsEnum.prepareSeekExact(term.bytes());
|
||||||
|
if (termExistsSupplier == null) {
|
||||||
|
this.states[ctx.ord] = EMPTY_TERMSTATE;
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
return () -> {
|
||||||
|
if (this.states[ctx.ord] == null) {
|
||||||
|
TermState state = null;
|
||||||
|
if (termExistsSupplier.get()) {
|
||||||
|
state = termsEnum.termState();
|
||||||
|
this.states[ctx.ord] = state;
|
||||||
|
} else {
|
||||||
|
this.states[ctx.ord] = EMPTY_TERMSTATE;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
TermState state = this.states[ctx.ord];
|
||||||
|
if (state == EMPTY_TERMSTATE) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
return state;
|
||||||
|
};
|
||||||
|
}
|
||||||
|
TermState state = this.states[ctx.ord];
|
||||||
|
if (state == EMPTY_TERMSTATE) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
return () -> state;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -230,40 +257,4 @@ public final class TermStates {
|
||||||
|
|
||||||
return sb.toString();
|
return sb.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Wrapper over TermState, ordinal value, term doc frequency and total term frequency */
|
|
||||||
private static final class TermStateInfo {
|
|
||||||
private final TermState state;
|
|
||||||
private final int ordinal;
|
|
||||||
private final int docFreq;
|
|
||||||
private final long totalTermFreq;
|
|
||||||
|
|
||||||
/** Initialize TermStateInfo */
|
|
||||||
public TermStateInfo(TermState state, int ordinal, int docFreq, long totalTermFreq) {
|
|
||||||
this.state = state;
|
|
||||||
this.ordinal = ordinal;
|
|
||||||
this.docFreq = docFreq;
|
|
||||||
this.totalTermFreq = totalTermFreq;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Get term state */
|
|
||||||
public TermState getState() {
|
|
||||||
return state;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Get ordinal value */
|
|
||||||
public int getOrdinal() {
|
|
||||||
return ordinal;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Get term doc frequency */
|
|
||||||
public int getDocFreq() {
|
|
||||||
return docFreq;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Get total term frequency */
|
|
||||||
public long getTotalTermFreq() {
|
|
||||||
return totalTermFreq;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,9 +17,11 @@
|
||||||
package org.apache.lucene.index;
|
package org.apache.lucene.index;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import org.apache.lucene.store.IndexInput;
|
||||||
import org.apache.lucene.util.AttributeSource;
|
import org.apache.lucene.util.AttributeSource;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.BytesRefIterator;
|
import org.apache.lucene.util.BytesRefIterator;
|
||||||
|
import org.apache.lucene.util.IOBooleanSupplier;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Iterator to seek ({@link #seekCeil(BytesRef)}, {@link #seekExact(BytesRef)}) or step through
|
* Iterator to seek ({@link #seekCeil(BytesRef)}, {@link #seekExact(BytesRef)}) or step through
|
||||||
|
@ -61,6 +63,23 @@ public abstract class TermsEnum implements BytesRefIterator {
|
||||||
*/
|
*/
|
||||||
public abstract boolean seekExact(BytesRef text) throws IOException;
|
public abstract boolean seekExact(BytesRef text) throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Two-phase {@link #seekExact}. The first phase typically calls {@link IndexInput#prefetch} on
|
||||||
|
* the right range of bytes under the hood, while the second phase {@link IOBooleanSupplier#get()}
|
||||||
|
* actually seeks the term within these bytes. This can be used to parallelize I/O across multiple
|
||||||
|
* terms by calling {@link #prepareSeekExact} on multiple terms enums before calling {@link
|
||||||
|
* IOBooleanSupplier#get()}.
|
||||||
|
*
|
||||||
|
* <p><b>NOTE</b>: It is illegal to call other methods on this {@link TermsEnum} after calling
|
||||||
|
* this method until {@link IOBooleanSupplier#get()} is called.
|
||||||
|
*
|
||||||
|
* <p><b>NOTE</b>: This may return {@code null} if this {@link TermsEnum} can identify that the
|
||||||
|
* term may not exist without performing any I/O.
|
||||||
|
*
|
||||||
|
* <p><b>NOTE</b>: The returned {@link IOBooleanSupplier} must be consumed in the same thread.
|
||||||
|
*/
|
||||||
|
public abstract IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Seeks to the specified term, if it exists, or to the next (ceiling) term. Returns SeekStatus to
|
* Seeks to the specified term, if it exists, or to the next (ceiling) term. Returns SeekStatus to
|
||||||
* indicate whether exact term was found, a different term was found, or EOF was hit. The target
|
* indicate whether exact term was found, a different term was found, or EOF was hit. The target
|
||||||
|
@ -178,9 +197,7 @@ public abstract class TermsEnum implements BytesRefIterator {
|
||||||
* of unused Attributes does not matter.
|
* of unused Attributes does not matter.
|
||||||
*/
|
*/
|
||||||
public static final TermsEnum EMPTY =
|
public static final TermsEnum EMPTY =
|
||||||
new TermsEnum() {
|
new BaseTermsEnum() {
|
||||||
|
|
||||||
private AttributeSource atts = null;
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public SeekStatus seekCeil(BytesRef term) {
|
public SeekStatus seekCeil(BytesRef term) {
|
||||||
|
@ -225,19 +242,6 @@ public abstract class TermsEnum implements BytesRefIterator {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override // make it synchronized here, to prevent double lazy init
|
|
||||||
public synchronized AttributeSource attributes() {
|
|
||||||
if (atts == null) {
|
|
||||||
atts = new AttributeSource();
|
|
||||||
}
|
|
||||||
return atts;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean seekExact(BytesRef text) throws IOException {
|
|
||||||
return seekCeil(text) == SeekStatus.FOUND;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public TermState termState() {
|
public TermState termState() {
|
||||||
throw new IllegalStateException("this method should never be called");
|
throw new IllegalStateException("this method should never be called");
|
||||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.index.TermState;
|
||||||
import org.apache.lucene.index.TermStates;
|
import org.apache.lucene.index.TermStates;
|
||||||
import org.apache.lucene.search.BooleanClause.Occur;
|
import org.apache.lucene.search.BooleanClause.Occur;
|
||||||
import org.apache.lucene.util.ArrayUtil;
|
import org.apache.lucene.util.ArrayUtil;
|
||||||
|
import org.apache.lucene.util.IOSupplier;
|
||||||
import org.apache.lucene.util.InPlaceMergeSorter;
|
import org.apache.lucene.util.InPlaceMergeSorter;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -316,7 +317,11 @@ public final class BlendedTermQuery extends Query {
|
||||||
List<LeafReaderContext> leaves = readerContext.leaves();
|
List<LeafReaderContext> leaves = readerContext.leaves();
|
||||||
TermStates newCtx = new TermStates(readerContext);
|
TermStates newCtx = new TermStates(readerContext);
|
||||||
for (int i = 0; i < leaves.size(); ++i) {
|
for (int i = 0; i < leaves.size(); ++i) {
|
||||||
TermState termState = ctx.get(leaves.get(i));
|
IOSupplier<TermState> supplier = ctx.get(leaves.get(i));
|
||||||
|
if (supplier == null) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
TermState termState = supplier.get();
|
||||||
if (termState == null) {
|
if (termState == null) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.search;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Supplier;
|
||||||
|
import org.apache.lucene.index.BaseTermsEnum;
|
||||||
import org.apache.lucene.index.ImpactsEnum;
|
import org.apache.lucene.index.ImpactsEnum;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
import org.apache.lucene.index.PostingsEnum;
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
|
@ -30,6 +31,7 @@ import org.apache.lucene.util.AttributeReflector;
|
||||||
import org.apache.lucene.util.AttributeSource;
|
import org.apache.lucene.util.AttributeSource;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.BytesRefBuilder;
|
import org.apache.lucene.util.BytesRefBuilder;
|
||||||
|
import org.apache.lucene.util.IOBooleanSupplier;
|
||||||
import org.apache.lucene.util.UnicodeUtil;
|
import org.apache.lucene.util.UnicodeUtil;
|
||||||
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||||
|
|
||||||
|
@ -39,7 +41,7 @@ import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||||
* <p>Term enumerations are always ordered by {@link BytesRef#compareTo}. Each term in the
|
* <p>Term enumerations are always ordered by {@link BytesRef#compareTo}. Each term in the
|
||||||
* enumeration is greater than all that precede it.
|
* enumeration is greater than all that precede it.
|
||||||
*/
|
*/
|
||||||
public final class FuzzyTermsEnum extends TermsEnum {
|
public final class FuzzyTermsEnum extends BaseTermsEnum {
|
||||||
|
|
||||||
// NOTE: we can't subclass FilteredTermsEnum here because we need to sometimes change actualEnum:
|
// NOTE: we can't subclass FilteredTermsEnum here because we need to sometimes change actualEnum:
|
||||||
private TermsEnum actualEnum;
|
private TermsEnum actualEnum;
|
||||||
|
@ -324,6 +326,11 @@ public final class FuzzyTermsEnum extends TermsEnum {
|
||||||
return actualEnum.seekExact(text);
|
return actualEnum.seekExact(text);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException {
|
||||||
|
return actualEnum.prepareSeekExact(text);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public SeekStatus seekCeil(BytesRef text) throws IOException {
|
public SeekStatus seekCeil(BytesRef text) throws IOException {
|
||||||
return actualEnum.seekCeil(text);
|
return actualEnum.seekCeil(text);
|
||||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.lucene.search.similarities.Similarity;
|
||||||
import org.apache.lucene.search.similarities.Similarity.SimScorer;
|
import org.apache.lucene.search.similarities.Similarity.SimScorer;
|
||||||
import org.apache.lucene.util.ArrayUtil;
|
import org.apache.lucene.util.ArrayUtil;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
import org.apache.lucene.util.IOSupplier;
|
||||||
import org.apache.lucene.util.PriorityQueue;
|
import org.apache.lucene.util.PriorityQueue;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -271,7 +272,8 @@ public class MultiPhraseQuery extends Query {
|
||||||
List<PostingsEnum> postings = new ArrayList<>();
|
List<PostingsEnum> postings = new ArrayList<>();
|
||||||
|
|
||||||
for (Term term : terms) {
|
for (Term term : terms) {
|
||||||
TermState termState = termStates.get(term).get(context);
|
IOSupplier<TermState> supplier = termStates.get(term).get(context);
|
||||||
|
TermState termState = supplier == null ? null : supplier.get();
|
||||||
if (termState != null) {
|
if (termState != null) {
|
||||||
termsEnum.seekExact(term.bytes(), termState);
|
termsEnum.seekExact(term.bytes(), termState);
|
||||||
postings.add(
|
postings.add(
|
||||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.lucene.search.similarities.Similarity;
|
||||||
import org.apache.lucene.search.similarities.Similarity.SimScorer;
|
import org.apache.lucene.search.similarities.Similarity.SimScorer;
|
||||||
import org.apache.lucene.util.ArrayUtil;
|
import org.apache.lucene.util.ArrayUtil;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
import org.apache.lucene.util.IOSupplier;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A Query that matches documents containing a particular sequence of terms. A PhraseQuery is built
|
* A Query that matches documents containing a particular sequence of terms. A PhraseQuery is built
|
||||||
|
@ -498,7 +499,8 @@ public class PhraseQuery extends Query {
|
||||||
|
|
||||||
for (int i = 0; i < terms.length; i++) {
|
for (int i = 0; i < terms.length; i++) {
|
||||||
final Term t = terms[i];
|
final Term t = terms[i];
|
||||||
final TermState state = states[i].get(context);
|
final IOSupplier<TermState> supplier = states[i].get(context);
|
||||||
|
final TermState state = supplier == null ? null : supplier.get();
|
||||||
if (state == null) {
|
if (state == null) {
|
||||||
/* term doesnt exist in this segment */
|
/* term doesnt exist in this segment */
|
||||||
assert termNotInReader(reader, t) : "no termstate found but term exists in reader";
|
assert termNotInReader(reader, t) : "no termstate found but term exists in reader";
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
package org.apache.lucene.search;
|
package org.apache.lucene.search;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.UncheckedIOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
@ -38,6 +39,7 @@ import org.apache.lucene.index.Terms;
|
||||||
import org.apache.lucene.index.TermsEnum;
|
import org.apache.lucene.index.TermsEnum;
|
||||||
import org.apache.lucene.search.similarities.Similarity;
|
import org.apache.lucene.search.similarities.Similarity;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
import org.apache.lucene.util.IOSupplier;
|
||||||
import org.apache.lucene.util.PriorityQueue;
|
import org.apache.lucene.util.PriorityQueue;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -277,80 +279,120 @@ public final class SynonymQuery extends Query {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
|
public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
|
||||||
final Scorer synonymScorer;
|
@SuppressWarnings({"rawtypes", "unchecked"})
|
||||||
List<PostingsEnum> iterators = new ArrayList<>();
|
IOSupplier<TermState>[] termStateSuppliers = new IOSupplier[terms.length];
|
||||||
List<ImpactsEnum> impacts = new ArrayList<>();
|
|
||||||
List<Float> termBoosts = new ArrayList<>();
|
|
||||||
for (int i = 0; i < terms.length; i++) {
|
for (int i = 0; i < terms.length; i++) {
|
||||||
TermState state = termStates[i].get(context);
|
// schedule the I/O for terms dictionary lookups in the background
|
||||||
if (state != null) {
|
termStateSuppliers[i] = termStates[i].get(context);
|
||||||
TermsEnum termsEnum = context.reader().terms(field).iterator();
|
}
|
||||||
termsEnum.seekExact(terms[i].term, state);
|
|
||||||
if (scoreMode == ScoreMode.TOP_SCORES) {
|
return new ScorerSupplier() {
|
||||||
ImpactsEnum impactsEnum = termsEnum.impacts(PostingsEnum.FREQS);
|
|
||||||
iterators.add(impactsEnum);
|
List<PostingsEnum> iterators;
|
||||||
impacts.add(impactsEnum);
|
List<ImpactsEnum> impacts;
|
||||||
} else {
|
List<Float> termBoosts;
|
||||||
PostingsEnum postingsEnum = termsEnum.postings(null, PostingsEnum.FREQS);
|
long cost;
|
||||||
iterators.add(postingsEnum);
|
|
||||||
impacts.add(new SlowImpactsEnum(postingsEnum));
|
private void init() throws IOException {
|
||||||
|
if (iterators != null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
iterators = new ArrayList<>();
|
||||||
|
impacts = new ArrayList<>();
|
||||||
|
termBoosts = new ArrayList<>();
|
||||||
|
cost = 0L;
|
||||||
|
|
||||||
|
for (int i = 0; i < terms.length; i++) {
|
||||||
|
IOSupplier<TermState> supplier = termStateSuppliers[i];
|
||||||
|
TermState state = supplier == null ? null : supplier.get();
|
||||||
|
if (state != null) {
|
||||||
|
TermsEnum termsEnum = context.reader().terms(field).iterator();
|
||||||
|
termsEnum.seekExact(terms[i].term, state);
|
||||||
|
if (scoreMode == ScoreMode.TOP_SCORES) {
|
||||||
|
ImpactsEnum impactsEnum = termsEnum.impacts(PostingsEnum.FREQS);
|
||||||
|
iterators.add(impactsEnum);
|
||||||
|
impacts.add(impactsEnum);
|
||||||
|
} else {
|
||||||
|
PostingsEnum postingsEnum = termsEnum.postings(null, PostingsEnum.FREQS);
|
||||||
|
iterators.add(postingsEnum);
|
||||||
|
impacts.add(new SlowImpactsEnum(postingsEnum));
|
||||||
|
}
|
||||||
|
termBoosts.add(terms[i].boost);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
for (DocIdSetIterator iterator : iterators) {
|
||||||
|
cost += iterator.cost();
|
||||||
}
|
}
|
||||||
termBoosts.add(terms[i].boost);
|
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
if (iterators.isEmpty()) {
|
@Override
|
||||||
return null;
|
public Scorer get(long leadCost) throws IOException {
|
||||||
}
|
init();
|
||||||
|
|
||||||
LeafSimScorer simScorer = new LeafSimScorer(simWeight, context.reader(), field, true);
|
if (iterators.isEmpty()) {
|
||||||
|
return new ConstantScoreScorer(0f, scoreMode, DocIdSetIterator.empty());
|
||||||
|
}
|
||||||
|
|
||||||
// we must optimize this case (term not in segment), disjunctions require >= 2 subs
|
LeafSimScorer simScorer = new LeafSimScorer(simWeight, context.reader(), field, true);
|
||||||
if (iterators.size() == 1) {
|
|
||||||
final TermScorer scorer;
|
// we must optimize this case (term not in segment), disjunctions require >= 2 subs
|
||||||
if (scoreMode == ScoreMode.TOP_SCORES) {
|
if (iterators.size() == 1) {
|
||||||
scorer = new TermScorer(impacts.get(0), simScorer);
|
final TermScorer scorer;
|
||||||
} else {
|
if (scoreMode == ScoreMode.TOP_SCORES) {
|
||||||
scorer = new TermScorer(iterators.get(0), simScorer);
|
scorer = new TermScorer(impacts.get(0), simScorer);
|
||||||
}
|
} else {
|
||||||
float boost = termBoosts.get(0);
|
scorer = new TermScorer(iterators.get(0), simScorer);
|
||||||
synonymScorer =
|
}
|
||||||
scoreMode == ScoreMode.COMPLETE_NO_SCORES || boost == 1f
|
float boost = termBoosts.get(0);
|
||||||
|
return scoreMode == ScoreMode.COMPLETE_NO_SCORES || boost == 1f
|
||||||
? scorer
|
? scorer
|
||||||
: new FreqBoostTermScorer(boost, scorer, simScorer);
|
: new FreqBoostTermScorer(boost, scorer, simScorer);
|
||||||
} else {
|
} else {
|
||||||
|
|
||||||
// we use termscorers + disjunction as an impl detail
|
// we use termscorers + disjunction as an impl detail
|
||||||
DisiPriorityQueue queue = new DisiPriorityQueue(iterators.size());
|
DisiPriorityQueue queue = new DisiPriorityQueue(iterators.size());
|
||||||
for (int i = 0; i < iterators.size(); i++) {
|
for (int i = 0; i < iterators.size(); i++) {
|
||||||
PostingsEnum postings = iterators.get(i);
|
PostingsEnum postings = iterators.get(i);
|
||||||
final TermScorer termScorer = new TermScorer(postings, simScorer);
|
final TermScorer termScorer = new TermScorer(postings, simScorer);
|
||||||
float boost = termBoosts.get(i);
|
float boost = termBoosts.get(i);
|
||||||
final DisiWrapperFreq wrapper = new DisiWrapperFreq(termScorer, boost);
|
final DisiWrapperFreq wrapper = new DisiWrapperFreq(termScorer, boost);
|
||||||
queue.add(wrapper);
|
queue.add(wrapper);
|
||||||
}
|
}
|
||||||
// Even though it is called approximation, it is accurate since none of
|
// Even though it is called approximation, it is accurate since none of
|
||||||
// the sub iterators are two-phase iterators.
|
// the sub iterators are two-phase iterators.
|
||||||
DocIdSetIterator iterator = new DisjunctionDISIApproximation(queue);
|
DocIdSetIterator iterator = new DisjunctionDISIApproximation(queue);
|
||||||
|
|
||||||
float[] boosts = new float[impacts.size()];
|
float[] boosts = new float[impacts.size()];
|
||||||
for (int i = 0; i < boosts.length; i++) {
|
for (int i = 0; i < boosts.length; i++) {
|
||||||
boosts[i] = termBoosts.get(i);
|
boosts[i] = termBoosts.get(i);
|
||||||
}
|
}
|
||||||
ImpactsSource impactsSource = mergeImpacts(impacts.toArray(new ImpactsEnum[0]), boosts);
|
ImpactsSource impactsSource = mergeImpacts(impacts.toArray(new ImpactsEnum[0]), boosts);
|
||||||
MaxScoreCache maxScoreCache = new MaxScoreCache(impactsSource, simScorer.getSimScorer());
|
MaxScoreCache maxScoreCache =
|
||||||
ImpactsDISI impactsDisi = new ImpactsDISI(iterator, maxScoreCache);
|
new MaxScoreCache(impactsSource, simScorer.getSimScorer());
|
||||||
|
ImpactsDISI impactsDisi = new ImpactsDISI(iterator, maxScoreCache);
|
||||||
|
|
||||||
if (scoreMode == ScoreMode.TOP_SCORES) {
|
if (scoreMode == ScoreMode.TOP_SCORES) {
|
||||||
// TODO: only do this when this is the top-level scoring clause
|
// TODO: only do this when this is the top-level scoring clause
|
||||||
// (ScorerSupplier#setTopLevelScoringClause) to save the overhead of wrapping with
|
// (ScorerSupplier#setTopLevelScoringClause) to save the overhead of wrapping with
|
||||||
// ImpactsDISI when it would not help
|
// ImpactsDISI when it would not help
|
||||||
iterator = impactsDisi;
|
iterator = impactsDisi;
|
||||||
|
}
|
||||||
|
|
||||||
|
return new SynonymScorer(queue, iterator, impactsDisi, simScorer);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
synonymScorer = new SynonymScorer(queue, iterator, impactsDisi, simScorer);
|
@Override
|
||||||
}
|
public long cost() {
|
||||||
return new DefaultScorerSupplier(synonymScorer);
|
try {
|
||||||
|
init();
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new UncheckedIOException(e);
|
||||||
|
}
|
||||||
|
return cost;
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
package org.apache.lucene.search;
|
package org.apache.lucene.search;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.UncheckedIOException;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import org.apache.lucene.index.IndexReaderContext;
|
import org.apache.lucene.index.IndexReaderContext;
|
||||||
import org.apache.lucene.index.LeafReader;
|
import org.apache.lucene.index.LeafReader;
|
||||||
|
@ -28,6 +29,7 @@ import org.apache.lucene.index.TermState;
|
||||||
import org.apache.lucene.index.TermStates;
|
import org.apache.lucene.index.TermStates;
|
||||||
import org.apache.lucene.index.TermsEnum;
|
import org.apache.lucene.index.TermsEnum;
|
||||||
import org.apache.lucene.search.similarities.Similarity;
|
import org.apache.lucene.search.similarities.Similarity;
|
||||||
|
import org.apache.lucene.util.IOSupplier;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A Query that matches documents containing a term. This may be combined with other terms with a
|
* A Query that matches documents containing a term. This may be combined with other terms with a
|
||||||
|
@ -119,18 +121,35 @@ public class TermQuery extends Query {
|
||||||
: "The top-reader used to create Weight is not the same as the current reader's top-reader ("
|
: "The top-reader used to create Weight is not the same as the current reader's top-reader ("
|
||||||
+ ReaderUtil.getTopLevelContext(context);
|
+ ReaderUtil.getTopLevelContext(context);
|
||||||
|
|
||||||
final TermsEnum termsEnum = getTermsEnum(context);
|
final IOSupplier<TermState> stateSupplier = termStates.get(context);
|
||||||
if (termsEnum == null) {
|
if (stateSupplier == null) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
final int docFreq = termsEnum.docFreq();
|
|
||||||
|
|
||||||
return new ScorerSupplier() {
|
return new ScorerSupplier() {
|
||||||
|
|
||||||
|
private TermsEnum termsEnum;
|
||||||
private boolean topLevelScoringClause = false;
|
private boolean topLevelScoringClause = false;
|
||||||
|
|
||||||
|
private TermsEnum getTermsEnum() throws IOException {
|
||||||
|
if (termsEnum == null) {
|
||||||
|
TermState state = stateSupplier.get();
|
||||||
|
if (state == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
termsEnum = context.reader().terms(term.field()).iterator();
|
||||||
|
termsEnum.seekExact(term.bytes(), state);
|
||||||
|
}
|
||||||
|
return termsEnum;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Scorer get(long leadCost) throws IOException {
|
public Scorer get(long leadCost) throws IOException {
|
||||||
|
TermsEnum termsEnum = getTermsEnum();
|
||||||
|
if (termsEnum == null) {
|
||||||
|
return new ConstantScoreScorer(0f, scoreMode, DocIdSetIterator.empty());
|
||||||
|
}
|
||||||
|
|
||||||
LeafSimScorer scorer =
|
LeafSimScorer scorer =
|
||||||
new LeafSimScorer(simScorer, context.reader(), term.field(), scoreMode.needsScores());
|
new LeafSimScorer(simScorer, context.reader(), term.field(), scoreMode.needsScores());
|
||||||
if (scoreMode == ScoreMode.TOP_SCORES) {
|
if (scoreMode == ScoreMode.TOP_SCORES) {
|
||||||
|
@ -149,7 +168,12 @@ public class TermQuery extends Query {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long cost() {
|
public long cost() {
|
||||||
return docFreq;
|
try {
|
||||||
|
TermsEnum te = getTermsEnum();
|
||||||
|
return te == null ? 0 : te.docFreq();
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new UncheckedIOException(e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -173,7 +197,8 @@ public class TermQuery extends Query {
|
||||||
assert termStates.wasBuiltFor(ReaderUtil.getTopLevelContext(context))
|
assert termStates.wasBuiltFor(ReaderUtil.getTopLevelContext(context))
|
||||||
: "The top-reader used to create Weight is not the same as the current reader's top-reader ("
|
: "The top-reader used to create Weight is not the same as the current reader's top-reader ("
|
||||||
+ ReaderUtil.getTopLevelContext(context);
|
+ ReaderUtil.getTopLevelContext(context);
|
||||||
final TermState state = termStates.get(context);
|
final IOSupplier<TermState> supplier = termStates.get(context);
|
||||||
|
final TermState state = supplier == null ? null : supplier.get();
|
||||||
if (state == null) { // term is not present in that reader
|
if (state == null) { // term is not present in that reader
|
||||||
assert termNotInReader(context.reader(), term)
|
assert termNotInReader(context.reader(), term)
|
||||||
: "no termstate found but term exists in reader term=" + term;
|
: "no termstate found but term exists in reader term=" + term;
|
||||||
|
@ -193,11 +218,11 @@ public class TermQuery extends Query {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
|
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
|
||||||
TermScorer scorer = (TermScorer) scorer(context);
|
Scorer scorer = scorer(context);
|
||||||
if (scorer != null) {
|
if (scorer != null) {
|
||||||
int newDoc = scorer.iterator().advance(doc);
|
int newDoc = scorer.iterator().advance(doc);
|
||||||
if (newDoc == doc) {
|
if (newDoc == doc) {
|
||||||
float freq = scorer.freq();
|
float freq = ((TermScorer) scorer).freq();
|
||||||
LeafSimScorer docScorer =
|
LeafSimScorer docScorer =
|
||||||
new LeafSimScorer(simScorer, context.reader(), term.field(), true);
|
new LeafSimScorer(simScorer, context.reader(), term.field(), true);
|
||||||
Explanation freqExplanation =
|
Explanation freqExplanation =
|
||||||
|
|
|
@ -0,0 +1,37 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.lucene.util;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Boolean supplier that is allowed to throw an IOException.
|
||||||
|
*
|
||||||
|
* @see java.util.function.BooleanSupplier
|
||||||
|
*/
|
||||||
|
@FunctionalInterface
|
||||||
|
public interface IOBooleanSupplier {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets the boolean result.
|
||||||
|
*
|
||||||
|
* @return the result
|
||||||
|
* @throws IOException if supplying the result throws an {@link IOException}
|
||||||
|
*/
|
||||||
|
boolean get() throws IOException;
|
||||||
|
}
|
|
@ -92,7 +92,7 @@ public class TestBooleanRewrites extends LuceneTestCase {
|
||||||
// make sure to set score=0
|
// make sure to set score=0
|
||||||
BooleanQuery.Builder query2 = new BooleanQuery.Builder();
|
BooleanQuery.Builder query2 = new BooleanQuery.Builder();
|
||||||
query2.add(new TermQuery(new Term("field", "a")), Occur.FILTER);
|
query2.add(new TermQuery(new Term("field", "a")), Occur.FILTER);
|
||||||
query2.add(new TermQuery(new Term("field", "b")), Occur.SHOULD);
|
query2.add(new TermQuery(new Term("missing_field", "b")), Occur.SHOULD);
|
||||||
final Weight weight =
|
final Weight weight =
|
||||||
searcher.createWeight(searcher.rewrite(query2.build()), ScoreMode.COMPLETE, 1);
|
searcher.createWeight(searcher.rewrite(query2.build()), ScoreMode.COMPLETE, 1);
|
||||||
final Scorer scorer = weight.scorer(reader.leaves().get(0));
|
final Scorer scorer = weight.scorer(reader.leaves().get(0));
|
||||||
|
|
|
@ -180,7 +180,7 @@ public class TestBooleanScorer extends LuceneTestCase {
|
||||||
Query query =
|
Query query =
|
||||||
new BooleanQuery.Builder()
|
new BooleanQuery.Builder()
|
||||||
.add(new TermQuery(new Term("foo", "bar")), Occur.SHOULD) // existing term
|
.add(new TermQuery(new Term("foo", "bar")), Occur.SHOULD) // existing term
|
||||||
.add(new TermQuery(new Term("foo", "baz")), Occur.SHOULD) // missing term
|
.add(new TermQuery(new Term("missing_field", "baz")), Occur.SHOULD) // missing term
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
// no scores -> term scorer
|
// no scores -> term scorer
|
||||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.lucene.tests.search.QueryUtils;
|
||||||
import org.apache.lucene.tests.util.LuceneTestCase;
|
import org.apache.lucene.tests.util.LuceneTestCase;
|
||||||
import org.apache.lucene.tests.util.TestUtil;
|
import org.apache.lucene.tests.util.TestUtil;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
import org.apache.lucene.util.IOBooleanSupplier;
|
||||||
import org.apache.lucene.util.IOUtils;
|
import org.apache.lucene.util.IOUtils;
|
||||||
|
|
||||||
public class TestTermQuery extends LuceneTestCase {
|
public class TestTermQuery extends LuceneTestCase {
|
||||||
|
@ -259,6 +260,11 @@ public class TestTermQuery extends LuceneTestCase {
|
||||||
throw new AssertionError("no seek");
|
throw new AssertionError("no seek");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException {
|
||||||
|
throw new AssertionError("no seek");
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void seekExact(BytesRef term, TermState state) throws IOException {
|
public void seekExact(BytesRef term, TermState state) throws IOException {
|
||||||
throw new AssertionError("no seek");
|
throw new AssertionError("no seek");
|
||||||
|
|
|
@ -1380,7 +1380,7 @@ public class TestBlockJoin extends LuceneTestCase {
|
||||||
IndexSearcher searcher = newSearcher(r);
|
IndexSearcher searcher = newSearcher(r);
|
||||||
|
|
||||||
// never matches:
|
// never matches:
|
||||||
Query childQuery = new TermQuery(new Term("childText", "bogus"));
|
Query childQuery = new TermQuery(new Term("childBogusField", "bogus"));
|
||||||
BitSetProducer parentsFilter =
|
BitSetProducer parentsFilter =
|
||||||
new QueryBitSetProducer(new TermQuery(new Term("isParent", "yes")));
|
new QueryBitSetProducer(new TermQuery(new Term("isParent", "yes")));
|
||||||
CheckJoinIndex.check(r, parentsFilter);
|
CheckJoinIndex.check(r, parentsFilter);
|
||||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.lucene.index.TermsEnum;
|
||||||
import org.apache.lucene.search.IndexSearcher;
|
import org.apache.lucene.search.IndexSearcher;
|
||||||
import org.apache.lucene.search.QueryVisitor;
|
import org.apache.lucene.search.QueryVisitor;
|
||||||
import org.apache.lucene.search.ScoreMode;
|
import org.apache.lucene.search.ScoreMode;
|
||||||
|
import org.apache.lucene.util.IOSupplier;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Matches spans containing a term. This should not be used for terms that are indexed at position
|
* Matches spans containing a term. This should not be used for terms that are indexed at position
|
||||||
|
@ -135,7 +136,8 @@ public class SpanTermQuery extends SpanQuery {
|
||||||
: "The top-reader used to create Weight is not the same as the current reader's top-reader ("
|
: "The top-reader used to create Weight is not the same as the current reader's top-reader ("
|
||||||
+ ReaderUtil.getTopLevelContext(context);
|
+ ReaderUtil.getTopLevelContext(context);
|
||||||
|
|
||||||
final TermState state = termStates.get(context);
|
final IOSupplier<TermState> supplier = termStates.get(context);
|
||||||
|
final TermState state = supplier == null ? null : supplier.get();
|
||||||
if (state == null) { // term is not present in that reader
|
if (state == null) { // term is not present in that reader
|
||||||
assert context.reader().docFreq(term) == 0
|
assert context.reader().docFreq(term) == 0
|
||||||
: "no termstate found but term exists in reader term=" + term;
|
: "no termstate found but term exists in reader term=" + term;
|
||||||
|
|
|
@ -62,6 +62,7 @@ import org.apache.lucene.search.similarities.Similarity;
|
||||||
import org.apache.lucene.search.similarities.SimilarityBase;
|
import org.apache.lucene.search.similarities.SimilarityBase;
|
||||||
import org.apache.lucene.util.Accountable;
|
import org.apache.lucene.util.Accountable;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
import org.apache.lucene.util.IOSupplier;
|
||||||
import org.apache.lucene.util.RamUsageEstimator;
|
import org.apache.lucene.util.RamUsageEstimator;
|
||||||
import org.apache.lucene.util.SmallFloat;
|
import org.apache.lucene.util.SmallFloat;
|
||||||
|
|
||||||
|
@ -405,7 +406,8 @@ public final class CombinedFieldQuery extends Query implements Accountable {
|
||||||
List<PostingsEnum> iterators = new ArrayList<>();
|
List<PostingsEnum> iterators = new ArrayList<>();
|
||||||
List<FieldAndWeight> fields = new ArrayList<>();
|
List<FieldAndWeight> fields = new ArrayList<>();
|
||||||
for (int i = 0; i < fieldTerms.length; i++) {
|
for (int i = 0; i < fieldTerms.length; i++) {
|
||||||
TermState state = termStates[i].get(context);
|
IOSupplier<TermState> supplier = termStates[i].get(context);
|
||||||
|
TermState state = supplier == null ? null : supplier.get();
|
||||||
if (state != null) {
|
if (state != null) {
|
||||||
TermsEnum termsEnum = context.reader().terms(fieldTerms[i].field()).iterator();
|
TermsEnum termsEnum = context.reader().terms(fieldTerms[i].field()).iterator();
|
||||||
termsEnum.seekExact(fieldTerms[i].bytes(), state);
|
termsEnum.seekExact(fieldTerms[i].bytes(), state);
|
||||||
|
|
|
@ -56,6 +56,7 @@ import org.apache.lucene.search.Weight;
|
||||||
import org.apache.lucene.search.similarities.Similarity;
|
import org.apache.lucene.search.similarities.Similarity;
|
||||||
import org.apache.lucene.util.ArrayUtil;
|
import org.apache.lucene.util.ArrayUtil;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
import org.apache.lucene.util.IOSupplier;
|
||||||
import org.apache.lucene.util.mutable.MutableValueBool;
|
import org.apache.lucene.util.mutable.MutableValueBool;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -387,7 +388,8 @@ public class PhraseWildcardQuery extends Query {
|
||||||
Terms terms = leafReaderContext.reader().terms(term.field());
|
Terms terms = leafReaderContext.reader().terms(term.field());
|
||||||
if (terms != null) {
|
if (terms != null) {
|
||||||
checkTermsHavePositions(terms);
|
checkTermsHavePositions(terms);
|
||||||
TermState termState = termStates.get(leafReaderContext);
|
IOSupplier<TermState> supplier = termStates.get(leafReaderContext);
|
||||||
|
TermState termState = supplier == null ? null : supplier.get();
|
||||||
if (termState != null) {
|
if (termState != null) {
|
||||||
termMatchesInSegment = true;
|
termMatchesInSegment = true;
|
||||||
numMatches++;
|
numMatches++;
|
||||||
|
|
|
@ -50,6 +50,7 @@ import org.apache.lucene.search.Weight;
|
||||||
import org.apache.lucene.search.similarities.Similarity;
|
import org.apache.lucene.search.similarities.Similarity;
|
||||||
import org.apache.lucene.util.Accountable;
|
import org.apache.lucene.util.Accountable;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
import org.apache.lucene.util.IOSupplier;
|
||||||
import org.apache.lucene.util.IntsRef;
|
import org.apache.lucene.util.IntsRef;
|
||||||
import org.apache.lucene.util.RamUsageEstimator;
|
import org.apache.lucene.util.RamUsageEstimator;
|
||||||
import org.apache.lucene.util.automaton.Automaton;
|
import org.apache.lucene.util.automaton.Automaton;
|
||||||
|
@ -416,7 +417,8 @@ public class TermAutomatonQuery extends Query implements Accountable {
|
||||||
: "The top-reader used to create Weight is not the same as the current reader's top-reader ("
|
: "The top-reader used to create Weight is not the same as the current reader's top-reader ("
|
||||||
+ ReaderUtil.getTopLevelContext(context);
|
+ ReaderUtil.getTopLevelContext(context);
|
||||||
BytesRef term = idToTerm.get(ent.key);
|
BytesRef term = idToTerm.get(ent.key);
|
||||||
TermState state = termStates.get(context);
|
IOSupplier<TermState> supplier = termStates.get(context);
|
||||||
|
TermState state = supplier == null ? null : supplier.get();
|
||||||
if (state != null) {
|
if (state != null) {
|
||||||
TermsEnum termsEnum = context.reader().terms(field).iterator();
|
TermsEnum termsEnum = context.reader().terms(field).iterator();
|
||||||
termsEnum.seekExact(term, state);
|
termsEnum.seekExact(term, state);
|
||||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.lucene.internal.tests.TestSecrets;
|
||||||
import org.apache.lucene.search.DocIdSetIterator;
|
import org.apache.lucene.search.DocIdSetIterator;
|
||||||
import org.apache.lucene.util.Bits;
|
import org.apache.lucene.util.Bits;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
import org.apache.lucene.util.IOBooleanSupplier;
|
||||||
import org.apache.lucene.util.VirtualMethod;
|
import org.apache.lucene.util.VirtualMethod;
|
||||||
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||||
|
|
||||||
|
@ -267,7 +268,8 @@ public class AssertingLeafReader extends FilterLeafReader {
|
||||||
private enum State {
|
private enum State {
|
||||||
INITIAL,
|
INITIAL,
|
||||||
POSITIONED,
|
POSITIONED,
|
||||||
UNPOSITIONED
|
UNPOSITIONED,
|
||||||
|
TWO_PHASE_SEEKING;
|
||||||
};
|
};
|
||||||
|
|
||||||
private State state = State.INITIAL;
|
private State state = State.INITIAL;
|
||||||
|
@ -370,6 +372,7 @@ public class AssertingLeafReader extends FilterLeafReader {
|
||||||
@Override
|
@Override
|
||||||
public void seekExact(long ord) throws IOException {
|
public void seekExact(long ord) throws IOException {
|
||||||
assertThread("Terms enums", creationThread);
|
assertThread("Terms enums", creationThread);
|
||||||
|
assert state != State.TWO_PHASE_SEEKING : "Unfinished two-phase seeking";
|
||||||
super.seekExact(ord);
|
super.seekExact(ord);
|
||||||
state = State.POSITIONED;
|
state = State.POSITIONED;
|
||||||
}
|
}
|
||||||
|
@ -377,6 +380,7 @@ public class AssertingLeafReader extends FilterLeafReader {
|
||||||
@Override
|
@Override
|
||||||
public SeekStatus seekCeil(BytesRef term) throws IOException {
|
public SeekStatus seekCeil(BytesRef term) throws IOException {
|
||||||
assertThread("Terms enums", creationThread);
|
assertThread("Terms enums", creationThread);
|
||||||
|
assert state != State.TWO_PHASE_SEEKING : "Unfinished two-phase seeking";
|
||||||
assert term.isValid();
|
assert term.isValid();
|
||||||
SeekStatus result = super.seekCeil(term);
|
SeekStatus result = super.seekCeil(term);
|
||||||
if (result == SeekStatus.END) {
|
if (result == SeekStatus.END) {
|
||||||
|
@ -390,6 +394,7 @@ public class AssertingLeafReader extends FilterLeafReader {
|
||||||
@Override
|
@Override
|
||||||
public boolean seekExact(BytesRef text) throws IOException {
|
public boolean seekExact(BytesRef text) throws IOException {
|
||||||
assertThread("Terms enums", creationThread);
|
assertThread("Terms enums", creationThread);
|
||||||
|
assert state != State.TWO_PHASE_SEEKING : "Unfinished two-phase seeking";
|
||||||
assert text.isValid();
|
assert text.isValid();
|
||||||
boolean result;
|
boolean result;
|
||||||
if (delegateOverridesSeekExact) {
|
if (delegateOverridesSeekExact) {
|
||||||
|
@ -405,6 +410,27 @@ public class AssertingLeafReader extends FilterLeafReader {
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException {
|
||||||
|
assertThread("Terms enums", creationThread);
|
||||||
|
assert state != State.TWO_PHASE_SEEKING : "Unfinished two-phase seeking";
|
||||||
|
assert text.isValid();
|
||||||
|
IOBooleanSupplier in = this.in.prepareSeekExact(text);
|
||||||
|
if (in == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
state = State.TWO_PHASE_SEEKING;
|
||||||
|
return () -> {
|
||||||
|
boolean exists = in.get();
|
||||||
|
if (exists) {
|
||||||
|
state = State.POSITIONED;
|
||||||
|
} else {
|
||||||
|
state = State.UNPOSITIONED;
|
||||||
|
}
|
||||||
|
return exists;
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public TermState termState() throws IOException {
|
public TermState termState() throws IOException {
|
||||||
assertThread("Terms enums", creationThread);
|
assertThread("Terms enums", creationThread);
|
||||||
|
@ -415,6 +441,7 @@ public class AssertingLeafReader extends FilterLeafReader {
|
||||||
@Override
|
@Override
|
||||||
public void seekExact(BytesRef term, TermState state) throws IOException {
|
public void seekExact(BytesRef term, TermState state) throws IOException {
|
||||||
assertThread("Terms enums", creationThread);
|
assertThread("Terms enums", creationThread);
|
||||||
|
assert this.state != State.TWO_PHASE_SEEKING : "Unfinished two-phase seeking";
|
||||||
assert term.isValid();
|
assert term.isValid();
|
||||||
in.seekExact(term, state);
|
in.seekExact(term, state);
|
||||||
this.state = State.POSITIONED;
|
this.state = State.POSITIONED;
|
||||||
|
|
Loading…
Reference in New Issue