From 026d661e5fd91f879de8a429687c70b6eb9b9f9d Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Wed, 10 Jul 2024 15:36:35 +0200 Subject: [PATCH] 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). --- .../bloom/BloomFilteringPostingsFormat.java | 12 +- .../sharedterms/STMergingTermsEnum.java | 9 +- .../lucene/codecs/DocValuesConsumer.java | 13 +- .../lucene90/blocktree/SegmentTermsEnum.java | 94 ++++++---- .../blocktree/SegmentTermsEnumFrame.java | 15 ++ .../apache/lucene/index/BaseTermsEnum.java | 6 + .../org/apache/lucene/index/CheckIndex.java | 11 +- .../apache/lucene/index/FilterLeafReader.java | 7 + .../lucene/index/FilteredTermsEnum.java | 11 ++ .../org/apache/lucene/index/TermStates.java | 147 ++++++++-------- .../org/apache/lucene/index/TermsEnum.java | 36 ++-- .../lucene/search/BlendedTermQuery.java | 7 +- .../apache/lucene/search/FuzzyTermsEnum.java | 9 +- .../lucene/search/MultiPhraseQuery.java | 4 +- .../org/apache/lucene/search/PhraseQuery.java | 4 +- .../apache/lucene/search/SynonymQuery.java | 164 +++++++++++------- .../org/apache/lucene/search/TermQuery.java | 39 ++++- .../apache/lucene/util/IOBooleanSupplier.java | 37 ++++ .../lucene/search/TestBooleanRewrites.java | 2 +- .../lucene/search/TestBooleanScorer.java | 2 +- .../apache/lucene/search/TestTermQuery.java | 6 + .../lucene/search/join/TestBlockJoin.java | 2 +- .../lucene/queries/spans/SpanTermQuery.java | 4 +- .../sandbox/search/CombinedFieldQuery.java | 4 +- .../sandbox/search/PhraseWildcardQuery.java | 4 +- .../sandbox/search/TermAutomatonQuery.java | 4 +- .../tests/index/AssertingLeafReader.java | 29 +++- 27 files changed, 452 insertions(+), 230 deletions(-) create mode 100644 lucene/core/src/java/org/apache/lucene/util/IOBooleanSupplier.java diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java index 2c908fcabe3..1daa1761fd8 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java @@ -43,6 +43,7 @@ import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOBooleanSupplier; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.automaton.CompiledAutomaton; @@ -315,12 +316,21 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat { } @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 // this code - save a disk seek if there is a match on an in-memory // structure // that may occasionally give a false positive but guaranteed no false // 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) { return false; } diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STMergingTermsEnum.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STMergingTermsEnum.java index 7a772c38908..95d63b14e3d 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STMergingTermsEnum.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STMergingTermsEnum.java @@ -20,11 +20,11 @@ package org.apache.lucene.codecs.uniformsplit.sharedterms; import java.io.IOException; import java.util.List; import java.util.RandomAccess; +import org.apache.lucene.index.BaseTermsEnum; import org.apache.lucene.index.ImpactsEnum; import org.apache.lucene.index.MergeState; import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.TermState; -import org.apache.lucene.index.TermsEnum; import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.BytesRef; @@ -34,7 +34,7 @@ import org.apache.lucene.util.BytesRef; * * @lucene.experimental */ -class STMergingTermsEnum extends TermsEnum { +class STMergingTermsEnum extends BaseTermsEnum { protected final String fieldName; protected final MultiSegmentsPostingsEnum multiPostingsEnum; @@ -63,11 +63,6 @@ class STMergingTermsEnum extends TermsEnum { throw new UnsupportedOperationException(); } - @Override - public boolean seekExact(BytesRef text) throws IOException { - throw new UnsupportedOperationException(); - } - @Override public SeekStatus seekCeil(BytesRef text) { throw new UnsupportedOperationException(); diff --git a/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java index 0d171812fae..cbb906788e5 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import org.apache.lucene.index.BaseTermsEnum; import org.apache.lucene.index.BinaryDocValues; import org.apache.lucene.index.DocIDMerger; 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 * call to {@link TermsEnum#next()}. */ - private static class MergedTermsEnum extends TermsEnum { + private static class MergedTermsEnum extends BaseTermsEnum { private final TermsEnum[] subs; private final OrdinalMap ordinalMap; @@ -542,11 +543,6 @@ public abstract class DocValuesConsumer implements Closeable { throw new UnsupportedOperationException(); } - @Override - public boolean seekExact(BytesRef text) throws IOException { - throw new UnsupportedOperationException(); - } - @Override public SeekStatus seekCeil(BytesRef text) throws IOException { throw new UnsupportedOperationException(); @@ -557,11 +553,6 @@ public abstract class DocValuesConsumer implements Closeable { throw new UnsupportedOperationException(); } - @Override - public void seekExact(BytesRef term, TermState state) throws IOException { - throw new UnsupportedOperationException(); - } - @Override public int docFreq() throws IOException { throw new UnsupportedOperationException(); diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnum.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnum.java index e3389931be7..91776585407 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnum.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnum.java @@ -30,6 +30,7 @@ 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.IOBooleanSupplier; import org.apache.lucene.util.RamUsageEstimator; import org.apache.lucene.util.fst.FST; import org.apache.lucene.util.fst.Util; @@ -307,15 +308,13 @@ final class SegmentTermsEnum extends BaseTermsEnum { return true; } - @Override - public boolean seekExact(BytesRef target) throws IOException { - + private IOBooleanSupplier prepareSeekExact(BytesRef target, boolean prefetch) throws IOException { if (fr.index == null) { throw new IllegalStateException("terms index was not loaded"); } if (fr.size() > 0 && (target.compareTo(fr.getMin()) < 0 || target.compareTo(fr.getMax()) > 0)) { - return false; + return null; } term.grow(1 + target.length); @@ -431,7 +430,7 @@ final class SegmentTermsEnum extends BaseTermsEnum { // if (DEBUG) { // System.out.println(" target is same as current; return true"); // } - return true; + return () -> true; } else { // if (DEBUG) { // System.out.println(" target is same as current but term doesn't exist"); @@ -501,24 +500,30 @@ final class SegmentTermsEnum extends BaseTermsEnum { // if (DEBUG) { // System.out.println(" FAST NOT_FOUND term=" + ToStringUtils.bytesRefToString(term)); // } - return false; + return null; } - 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; + if (prefetch) { + currentFrame.prefetchBlock(); } + + 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 { // Follow this arc arc = nextArc; @@ -556,25 +561,42 @@ final class SegmentTermsEnum extends BaseTermsEnum { // if (DEBUG) { // System.out.println(" FAST NOT_FOUND term=" + ToStringUtils.bytesRefToString(term)); // } - return false; + return null; } - 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; + if (prefetch) { + currentFrame.prefetchBlock(); } + + 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 diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java index c1552c8ce60..5ecbc3c173e 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java @@ -133,6 +133,21 @@ final class SegmentTermsEnumFrame { 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 doesn't actually decode the docFreq, totalTermFreq, postings details (frq/prx offset, etc.) metadata; diff --git a/lucene/core/src/java/org/apache/lucene/index/BaseTermsEnum.java b/lucene/core/src/java/org/apache/lucene/index/BaseTermsEnum.java index 37b8395aebb..6809a7a8682 100644 --- a/lucene/core/src/java/org/apache/lucene/index/BaseTermsEnum.java +++ b/lucene/core/src/java/org/apache/lucene/index/BaseTermsEnum.java @@ -20,6 +20,7 @@ package org.apache.lucene.index; import java.io.IOException; import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOBooleanSupplier; /** * A base TermsEnum that adds default implementations for @@ -58,6 +59,11 @@ public abstract class BaseTermsEnum extends TermsEnum { return seekCeil(text) == SeekStatus.FOUND; } + @Override + public IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException { + return () -> seekExact(text); + } + @Override public void seekExact(BytesRef term, TermState state) throws IOException { if (!seekExact(term)) { diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java index 0f6020f7873..aaa76f418a9 100644 --- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java +++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java @@ -79,6 +79,7 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefBuilder; import org.apache.lucene.util.CommandLineUtil; import org.apache.lucene.util.FixedBitSet; +import org.apache.lucene.util.IOBooleanSupplier; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.LongBitSet; import org.apache.lucene.util.NamedThreadFactory; @@ -3869,6 +3870,7 @@ public final class CheckIndex implements Closeable { TermsEnum postingsTermsEnum = postingsTerms.iterator(); final boolean hasProx = terms.hasOffsets() || terms.hasPositions(); + int seekExactCounter = 0; BytesRef term; while ((term = termsEnum.next()) != null) { @@ -3876,7 +3878,14 @@ public final class CheckIndex implements Closeable { postings = termsEnum.postings(postings, PostingsEnum.ALL); 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( "vector term=" + term diff --git a/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java b/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java index 4935237178a..87d62f22d04 100644 --- a/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java @@ -22,6 +22,7 @@ import org.apache.lucene.search.KnnCollector; import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOBooleanSupplier; import org.apache.lucene.util.Unwrappable; /** @@ -161,6 +162,7 @@ public abstract class FilterLeafReader extends LeafReader { /** Base class for filtering {@link TermsEnum} implementations. */ public abstract static class FilterTermsEnum extends TermsEnum { + /** The underlying TermsEnum instance. */ protected final TermsEnum in; @@ -236,6 +238,11 @@ public abstract class FilterLeafReader extends LeafReader { in.seekExact(term, state); } + @Override + public IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException { + return in.prepareSeekExact(text); + } + @Override public TermState termState() throws IOException { return in.termState(); diff --git a/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java b/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java index 5ee99878567..c8354cd881f 100644 --- a/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java +++ b/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java @@ -19,6 +19,7 @@ package org.apache.lucene.index; import java.io.IOException; import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOBooleanSupplier; /** * 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"); } + /** + * 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! * diff --git a/lucene/core/src/java/org/apache/lucene/index/TermStates.java b/lucene/core/src/java/org/apache/lucene/index/TermStates.java index 75c64a907b1..2321104d30f 100644 --- a/lucene/core/src/java/org/apache/lucene/index/TermStates.java +++ b/lucene/core/src/java/org/apache/lucene/index/TermStates.java @@ -17,12 +17,12 @@ package org.apache.lucene.index; import java.io.IOException; -import java.util.ArrayList; import java.util.Arrays; -import java.util.List; -import java.util.concurrent.Callable; +import java.util.function.Supplier; 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 @@ -80,6 +80,8 @@ public final class TermStates { 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 * 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; final TermStates perReaderTermState = new TermStates(needsStats ? null : term, context); if (needsStats) { - TaskExecutor taskExecutor = indexSearcher.getTaskExecutor(); - // build the term states concurrently - List> tasks = new ArrayList<>(context.leaves().size()); + PendingTermLookup[] pendingTermLookups = new PendingTermLookup[0]; for (LeafReaderContext ctx : context.leaves()) { - tasks.add( - () -> { - TermsEnum termsEnum = loadTermsEnum(ctx, term); - return termsEnum == null - ? null - : new TermStateInfo( - termsEnum.termState(), - ctx.ord, - termsEnum.docFreq(), - termsEnum.totalTermFreq()); - }); + Terms terms = Terms.getTerms(ctx.reader(), term.field()); + TermsEnum termsEnum = terms.iterator(); + // Schedule the I/O in the terms dictionary in the background. + IOBooleanSupplier termExistsSupplier = termsEnum.prepareSeekExact(term.bytes()); + if (termExistsSupplier != null) { + pendingTermLookups = ArrayUtil.grow(pendingTermLookups, ctx.ord + 1); + pendingTermLookups[ctx.ord] = new PendingTermLookup(termsEnum, termExistsSupplier); + } } - List resultInfos = taskExecutor.invokeAll(tasks); - for (TermStateInfo info : resultInfos) { - if (info != null) { + for (int ord = 0; ord < pendingTermLookups.length; ++ord) { + PendingTermLookup pendingTermLookup = pendingTermLookups[ord]; + if (pendingTermLookup != null && pendingTermLookup.supplier.get()) { + TermsEnum termsEnum = pendingTermLookup.termsEnum(); perReaderTermState.register( - info.getState(), info.getOrdinal(), info.getDocFreq(), info.getTotalTermFreq()); + termsEnum.termState(), ord, termsEnum.docFreq(), termsEnum.totalTermFreq()); } } } 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 */ public void clear() { docFreq = 0; @@ -172,22 +161,60 @@ public final class TermStates { } /** - * Returns the {@link TermState} for a leaf reader context or null if no {@link - * TermState} for the context was registered. + * Returns a {@link Supplier} for a {@link TermState} for the given {@link LeafReaderContext}. + * 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. + * + *

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. - * @return the {@link TermState} for the given readers ord or null if no {@link - * TermState} for the reader was registered + * @return a Supplier for a TermState. */ - public TermState get(LeafReaderContext ctx) throws IOException { + public IOSupplier get(LeafReaderContext ctx) throws IOException { assert ctx.ord >= 0 && ctx.ord < states.length; - if (term == null) return states[ctx.ord]; - if (this.states[ctx.ord] == null) { - TermsEnum te = loadTermsEnum(ctx, term); - this.states[ctx.ord] = te == null ? EMPTY_TERMSTATE : te.termState(); + if (term == null) { + if (states[ctx.ord] == null) { + return null; + } else { + return () -> states[ctx.ord]; + } } - if (this.states[ctx.ord] == EMPTY_TERMSTATE) return null; - return this.states[ctx.ord]; + if (this.states[ctx.ord] == null) { + 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(); } - - /** 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; - } - } } diff --git a/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java b/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java index 79e985c9204..2ff6a2719b6 100644 --- a/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java +++ b/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java @@ -17,9 +17,11 @@ package org.apache.lucene.index; import java.io.IOException; +import org.apache.lucene.store.IndexInput; import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; +import org.apache.lucene.util.IOBooleanSupplier; /** * 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; + /** + * 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()}. + * + *

NOTE: It is illegal to call other methods on this {@link TermsEnum} after calling + * this method until {@link IOBooleanSupplier#get()} is called. + * + *

NOTE: This may return {@code null} if this {@link TermsEnum} can identify that the + * term may not exist without performing any I/O. + * + *

NOTE: 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 * 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. */ public static final TermsEnum EMPTY = - new TermsEnum() { - - private AttributeSource atts = null; + new BaseTermsEnum() { @Override public SeekStatus seekCeil(BytesRef term) { @@ -225,19 +242,6 @@ public abstract class TermsEnum implements BytesRefIterator { 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 public TermState termState() { throw new IllegalStateException("this method should never be called"); diff --git a/lucene/core/src/java/org/apache/lucene/search/BlendedTermQuery.java b/lucene/core/src/java/org/apache/lucene/search/BlendedTermQuery.java index 8b7d3e80fcd..8dc66036969 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BlendedTermQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/BlendedTermQuery.java @@ -26,6 +26,7 @@ import org.apache.lucene.index.TermState; import org.apache.lucene.index.TermStates; import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.IOSupplier; import org.apache.lucene.util.InPlaceMergeSorter; /** @@ -316,7 +317,11 @@ public final class BlendedTermQuery extends Query { List leaves = readerContext.leaves(); TermStates newCtx = new TermStates(readerContext); for (int i = 0; i < leaves.size(); ++i) { - TermState termState = ctx.get(leaves.get(i)); + IOSupplier supplier = ctx.get(leaves.get(i)); + if (supplier == null) { + continue; + } + TermState termState = supplier.get(); if (termState == null) { continue; } diff --git a/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java b/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java index cfd6ed232de..bf9aa0f8f2c 100644 --- a/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java +++ b/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java @@ -18,6 +18,7 @@ package org.apache.lucene.search; import java.io.IOException; import java.util.function.Supplier; +import org.apache.lucene.index.BaseTermsEnum; import org.apache.lucene.index.ImpactsEnum; import org.apache.lucene.index.PostingsEnum; 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.BytesRef; import org.apache.lucene.util.BytesRefBuilder; +import org.apache.lucene.util.IOBooleanSupplier; import org.apache.lucene.util.UnicodeUtil; import org.apache.lucene.util.automaton.CompiledAutomaton; @@ -39,7 +41,7 @@ import org.apache.lucene.util.automaton.CompiledAutomaton; *

Term enumerations are always ordered by {@link BytesRef#compareTo}. Each term in the * 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: private TermsEnum actualEnum; @@ -324,6 +326,11 @@ public final class FuzzyTermsEnum extends TermsEnum { return actualEnum.seekExact(text); } + @Override + public IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException { + return actualEnum.prepareSeekExact(text); + } + @Override public SeekStatus seekCeil(BytesRef text) throws IOException { return actualEnum.seekCeil(text); diff --git a/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java b/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java index 315b94a2acd..83510c02991 100644 --- a/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java @@ -38,6 +38,7 @@ import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.search.similarities.Similarity.SimScorer; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOSupplier; import org.apache.lucene.util.PriorityQueue; /** @@ -271,7 +272,8 @@ public class MultiPhraseQuery extends Query { List postings = new ArrayList<>(); for (Term term : terms) { - TermState termState = termStates.get(term).get(context); + IOSupplier supplier = termStates.get(term).get(context); + TermState termState = supplier == null ? null : supplier.get(); if (termState != null) { termsEnum.seekExact(term.bytes(), termState); postings.add( diff --git a/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java b/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java index c5a5ee36fd4..55d0e228d1a 100644 --- a/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java @@ -38,6 +38,7 @@ import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.search.similarities.Similarity.SimScorer; import org.apache.lucene.util.ArrayUtil; 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 @@ -498,7 +499,8 @@ public class PhraseQuery extends Query { for (int i = 0; i < terms.length; i++) { final Term t = terms[i]; - final TermState state = states[i].get(context); + final IOSupplier supplier = states[i].get(context); + final TermState state = supplier == null ? null : supplier.get(); if (state == null) { /* term doesnt exist in this segment */ assert termNotInReader(reader, t) : "no termstate found but term exists in reader"; diff --git a/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java b/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java index 0500f4630dc..82a3b6e0148 100644 --- a/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java @@ -17,6 +17,7 @@ package org.apache.lucene.search; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -38,6 +39,7 @@ import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOSupplier; import org.apache.lucene.util.PriorityQueue; /** @@ -277,80 +279,120 @@ public final class SynonymQuery extends Query { @Override public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException { - final Scorer synonymScorer; - List iterators = new ArrayList<>(); - List impacts = new ArrayList<>(); - List termBoosts = new ArrayList<>(); + @SuppressWarnings({"rawtypes", "unchecked"}) + IOSupplier[] termStateSuppliers = new IOSupplier[terms.length]; for (int i = 0; i < terms.length; i++) { - TermState state = termStates[i].get(context); - 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)); + // schedule the I/O for terms dictionary lookups in the background + termStateSuppliers[i] = termStates[i].get(context); + } + + return new ScorerSupplier() { + + List iterators; + List impacts; + List termBoosts; + long cost; + + 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 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()) { - return null; - } + @Override + 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 - if (iterators.size() == 1) { - final TermScorer scorer; - if (scoreMode == ScoreMode.TOP_SCORES) { - scorer = new TermScorer(impacts.get(0), simScorer); - } else { - scorer = new TermScorer(iterators.get(0), simScorer); - } - float boost = termBoosts.get(0); - synonymScorer = - scoreMode == ScoreMode.COMPLETE_NO_SCORES || boost == 1f + LeafSimScorer simScorer = new LeafSimScorer(simWeight, context.reader(), field, true); + + // we must optimize this case (term not in segment), disjunctions require >= 2 subs + if (iterators.size() == 1) { + final TermScorer scorer; + if (scoreMode == ScoreMode.TOP_SCORES) { + scorer = new TermScorer(impacts.get(0), simScorer); + } else { + scorer = new TermScorer(iterators.get(0), simScorer); + } + float boost = termBoosts.get(0); + return scoreMode == ScoreMode.COMPLETE_NO_SCORES || boost == 1f ? scorer : new FreqBoostTermScorer(boost, scorer, simScorer); - } else { + } else { - // we use termscorers + disjunction as an impl detail - DisiPriorityQueue queue = new DisiPriorityQueue(iterators.size()); - for (int i = 0; i < iterators.size(); i++) { - PostingsEnum postings = iterators.get(i); - final TermScorer termScorer = new TermScorer(postings, simScorer); - float boost = termBoosts.get(i); - final DisiWrapperFreq wrapper = new DisiWrapperFreq(termScorer, boost); - queue.add(wrapper); - } - // Even though it is called approximation, it is accurate since none of - // the sub iterators are two-phase iterators. - DocIdSetIterator iterator = new DisjunctionDISIApproximation(queue); + // we use termscorers + disjunction as an impl detail + DisiPriorityQueue queue = new DisiPriorityQueue(iterators.size()); + for (int i = 0; i < iterators.size(); i++) { + PostingsEnum postings = iterators.get(i); + final TermScorer termScorer = new TermScorer(postings, simScorer); + float boost = termBoosts.get(i); + final DisiWrapperFreq wrapper = new DisiWrapperFreq(termScorer, boost); + queue.add(wrapper); + } + // Even though it is called approximation, it is accurate since none of + // the sub iterators are two-phase iterators. + DocIdSetIterator iterator = new DisjunctionDISIApproximation(queue); - float[] boosts = new float[impacts.size()]; - for (int i = 0; i < boosts.length; i++) { - boosts[i] = termBoosts.get(i); - } - ImpactsSource impactsSource = mergeImpacts(impacts.toArray(new ImpactsEnum[0]), boosts); - MaxScoreCache maxScoreCache = new MaxScoreCache(impactsSource, simScorer.getSimScorer()); - ImpactsDISI impactsDisi = new ImpactsDISI(iterator, maxScoreCache); + float[] boosts = new float[impacts.size()]; + for (int i = 0; i < boosts.length; i++) { + boosts[i] = termBoosts.get(i); + } + ImpactsSource impactsSource = mergeImpacts(impacts.toArray(new ImpactsEnum[0]), boosts); + MaxScoreCache maxScoreCache = + new MaxScoreCache(impactsSource, simScorer.getSimScorer()); + ImpactsDISI impactsDisi = new ImpactsDISI(iterator, maxScoreCache); - if (scoreMode == ScoreMode.TOP_SCORES) { - // TODO: only do this when this is the top-level scoring clause - // (ScorerSupplier#setTopLevelScoringClause) to save the overhead of wrapping with - // ImpactsDISI when it would not help - iterator = impactsDisi; + if (scoreMode == ScoreMode.TOP_SCORES) { + // TODO: only do this when this is the top-level scoring clause + // (ScorerSupplier#setTopLevelScoringClause) to save the overhead of wrapping with + // ImpactsDISI when it would not help + iterator = impactsDisi; + } + + return new SynonymScorer(queue, iterator, impactsDisi, simScorer); + } } - synonymScorer = new SynonymScorer(queue, iterator, impactsDisi, simScorer); - } - return new DefaultScorerSupplier(synonymScorer); + @Override + public long cost() { + try { + init(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return cost; + } + }; } @Override diff --git a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java index 84037acd0d4..3a843addcc3 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java @@ -17,6 +17,7 @@ package org.apache.lucene.search; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.Objects; import org.apache.lucene.index.IndexReaderContext; 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.TermsEnum; 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 @@ -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 (" + ReaderUtil.getTopLevelContext(context); - final TermsEnum termsEnum = getTermsEnum(context); - if (termsEnum == null) { + final IOSupplier stateSupplier = termStates.get(context); + if (stateSupplier == null) { return null; } - final int docFreq = termsEnum.docFreq(); return new ScorerSupplier() { + private TermsEnum termsEnum; 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 public Scorer get(long leadCost) throws IOException { + TermsEnum termsEnum = getTermsEnum(); + if (termsEnum == null) { + return new ConstantScoreScorer(0f, scoreMode, DocIdSetIterator.empty()); + } + LeafSimScorer scorer = new LeafSimScorer(simScorer, context.reader(), term.field(), scoreMode.needsScores()); if (scoreMode == ScoreMode.TOP_SCORES) { @@ -149,7 +168,12 @@ public class TermQuery extends Query { @Override public long cost() { - return docFreq; + try { + TermsEnum te = getTermsEnum(); + return te == null ? 0 : te.docFreq(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } } @Override @@ -173,7 +197,8 @@ public class TermQuery extends Query { assert termStates.wasBuiltFor(ReaderUtil.getTopLevelContext(context)) : "The top-reader used to create Weight is not the same as the current reader's top-reader (" + ReaderUtil.getTopLevelContext(context); - final TermState state = termStates.get(context); + final IOSupplier supplier = termStates.get(context); + final TermState state = supplier == null ? null : supplier.get(); if (state == null) { // term is not present in that reader assert termNotInReader(context.reader(), term) : "no termstate found but term exists in reader term=" + term; @@ -193,11 +218,11 @@ public class TermQuery extends Query { @Override public Explanation explain(LeafReaderContext context, int doc) throws IOException { - TermScorer scorer = (TermScorer) scorer(context); + Scorer scorer = scorer(context); if (scorer != null) { int newDoc = scorer.iterator().advance(doc); if (newDoc == doc) { - float freq = scorer.freq(); + float freq = ((TermScorer) scorer).freq(); LeafSimScorer docScorer = new LeafSimScorer(simScorer, context.reader(), term.field(), true); Explanation freqExplanation = diff --git a/lucene/core/src/java/org/apache/lucene/util/IOBooleanSupplier.java b/lucene/core/src/java/org/apache/lucene/util/IOBooleanSupplier.java new file mode 100644 index 00000000000..4100c6c53c5 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/util/IOBooleanSupplier.java @@ -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; +} diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanRewrites.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanRewrites.java index 87b8068d2f1..15f0c0d0c94 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanRewrites.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanRewrites.java @@ -92,7 +92,7 @@ public class TestBooleanRewrites extends LuceneTestCase { // make sure to set score=0 BooleanQuery.Builder query2 = new BooleanQuery.Builder(); 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 = searcher.createWeight(searcher.rewrite(query2.build()), ScoreMode.COMPLETE, 1); final Scorer scorer = weight.scorer(reader.leaves().get(0)); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java index fec70cb3dcc..1258dd8b10e 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java @@ -180,7 +180,7 @@ public class TestBooleanScorer extends LuceneTestCase { Query query = new BooleanQuery.Builder() .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(); // no scores -> term scorer diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTermQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestTermQuery.java index 3b35f67cad1..8911500546b 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTermQuery.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTermQuery.java @@ -43,6 +43,7 @@ import org.apache.lucene.tests.search.QueryUtils; import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.lucene.tests.util.TestUtil; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOBooleanSupplier; import org.apache.lucene.util.IOUtils; public class TestTermQuery extends LuceneTestCase { @@ -259,6 +260,11 @@ public class TestTermQuery extends LuceneTestCase { throw new AssertionError("no seek"); } + @Override + public IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException { + throw new AssertionError("no seek"); + } + @Override public void seekExact(BytesRef term, TermState state) throws IOException { throw new AssertionError("no seek"); diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java index 4e445d7eeb2..77f00818dec 100644 --- a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java +++ b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java @@ -1380,7 +1380,7 @@ public class TestBlockJoin extends LuceneTestCase { IndexSearcher searcher = newSearcher(r); // never matches: - Query childQuery = new TermQuery(new Term("childText", "bogus")); + Query childQuery = new TermQuery(new Term("childBogusField", "bogus")); BitSetProducer parentsFilter = new QueryBitSetProducer(new TermQuery(new Term("isParent", "yes"))); CheckJoinIndex.check(r, parentsFilter); diff --git a/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanTermQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanTermQuery.java index 39081eb48c1..359f6ae7e95 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanTermQuery.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanTermQuery.java @@ -32,6 +32,7 @@ import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.QueryVisitor; 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 @@ -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 (" + ReaderUtil.getTopLevelContext(context); - final TermState state = termStates.get(context); + final IOSupplier supplier = termStates.get(context); + final TermState state = supplier == null ? null : supplier.get(); if (state == null) { // term is not present in that reader assert context.reader().docFreq(term) == 0 : "no termstate found but term exists in reader term=" + term; diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/CombinedFieldQuery.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/CombinedFieldQuery.java index 9fe6b890044..35b9e8dc78d 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/CombinedFieldQuery.java +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/CombinedFieldQuery.java @@ -62,6 +62,7 @@ import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.search.similarities.SimilarityBase; import org.apache.lucene.util.Accountable; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOSupplier; import org.apache.lucene.util.RamUsageEstimator; import org.apache.lucene.util.SmallFloat; @@ -405,7 +406,8 @@ public final class CombinedFieldQuery extends Query implements Accountable { List iterators = new ArrayList<>(); List fields = new ArrayList<>(); for (int i = 0; i < fieldTerms.length; i++) { - TermState state = termStates[i].get(context); + IOSupplier supplier = termStates[i].get(context); + TermState state = supplier == null ? null : supplier.get(); if (state != null) { TermsEnum termsEnum = context.reader().terms(fieldTerms[i].field()).iterator(); termsEnum.seekExact(fieldTerms[i].bytes(), state); diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/PhraseWildcardQuery.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/PhraseWildcardQuery.java index 2ba011dbf3a..49d6ef4bd9a 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/PhraseWildcardQuery.java +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/PhraseWildcardQuery.java @@ -56,6 +56,7 @@ import org.apache.lucene.search.Weight; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOSupplier; import org.apache.lucene.util.mutable.MutableValueBool; /** @@ -387,7 +388,8 @@ public class PhraseWildcardQuery extends Query { Terms terms = leafReaderContext.reader().terms(term.field()); if (terms != null) { checkTermsHavePositions(terms); - TermState termState = termStates.get(leafReaderContext); + IOSupplier supplier = termStates.get(leafReaderContext); + TermState termState = supplier == null ? null : supplier.get(); if (termState != null) { termMatchesInSegment = true; numMatches++; diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/TermAutomatonQuery.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/TermAutomatonQuery.java index 1e8855dfda3..46386b52249 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/TermAutomatonQuery.java +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/TermAutomatonQuery.java @@ -50,6 +50,7 @@ import org.apache.lucene.search.Weight; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.util.Accountable; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOSupplier; import org.apache.lucene.util.IntsRef; import org.apache.lucene.util.RamUsageEstimator; 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 (" + ReaderUtil.getTopLevelContext(context); BytesRef term = idToTerm.get(ent.key); - TermState state = termStates.get(context); + IOSupplier supplier = termStates.get(context); + TermState state = supplier == null ? null : supplier.get(); if (state != null) { TermsEnum termsEnum = context.reader().terms(field).iterator(); termsEnum.seekExact(term, state); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/AssertingLeafReader.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/AssertingLeafReader.java index fc02ae82de2..3151754a075 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/AssertingLeafReader.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/AssertingLeafReader.java @@ -51,6 +51,7 @@ import org.apache.lucene.internal.tests.TestSecrets; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOBooleanSupplier; import org.apache.lucene.util.VirtualMethod; import org.apache.lucene.util.automaton.CompiledAutomaton; @@ -267,7 +268,8 @@ public class AssertingLeafReader extends FilterLeafReader { private enum State { INITIAL, POSITIONED, - UNPOSITIONED + UNPOSITIONED, + TWO_PHASE_SEEKING; }; private State state = State.INITIAL; @@ -370,6 +372,7 @@ public class AssertingLeafReader extends FilterLeafReader { @Override public void seekExact(long ord) throws IOException { assertThread("Terms enums", creationThread); + assert state != State.TWO_PHASE_SEEKING : "Unfinished two-phase seeking"; super.seekExact(ord); state = State.POSITIONED; } @@ -377,6 +380,7 @@ public class AssertingLeafReader extends FilterLeafReader { @Override public SeekStatus seekCeil(BytesRef term) throws IOException { assertThread("Terms enums", creationThread); + assert state != State.TWO_PHASE_SEEKING : "Unfinished two-phase seeking"; assert term.isValid(); SeekStatus result = super.seekCeil(term); if (result == SeekStatus.END) { @@ -390,6 +394,7 @@ public class AssertingLeafReader extends FilterLeafReader { @Override public boolean seekExact(BytesRef text) throws IOException { assertThread("Terms enums", creationThread); + assert state != State.TWO_PHASE_SEEKING : "Unfinished two-phase seeking"; assert text.isValid(); boolean result; if (delegateOverridesSeekExact) { @@ -405,6 +410,27 @@ public class AssertingLeafReader extends FilterLeafReader { 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 public TermState termState() throws IOException { assertThread("Terms enums", creationThread); @@ -415,6 +441,7 @@ public class AssertingLeafReader extends FilterLeafReader { @Override public void seekExact(BytesRef term, TermState state) throws IOException { assertThread("Terms enums", creationThread); + assert this.state != State.TWO_PHASE_SEEKING : "Unfinished two-phase seeking"; assert term.isValid(); in.seekExact(term, state); this.state = State.POSITIONED;