mikemccand commented on code in PR #13359: URL: https://github.com/apache/lucene/pull/13359#discussion_r1609713219
########## lucene/core/src/java/org/apache/lucene/index/TermsEnum.java: ########## @@ -61,6 +62,21 @@ public enum SeekStatus { */ public abstract boolean seekExact(BytesRef text) throws IOException; + /** + * Prepare a future call to {@link #seekExact}. This typically calls {@link IndexInput#prefetch} + * on the right range of bytes under the hood so that the next call to {@link #seekExact} is + * faster. This can be used to parallelize I/O across multiple terms by calling {@link + * #prepareSeekExact} on multiple terms enums before calling {@link #seekExact(BytesRef)} on the + * same {@link TermsEnum}s. + * + * <p><b>NOTE</b>: The terms enum is unpositioned after calling this method. + * + * <p><b>NOTE</b>: It is not necessary to call this method before calling {@link + * #seekExact(BytesRef, TermState)}. {@link TermsEnum} implementations are expected to implement + * this method in an I/O-free fashion. Review Comment: `this method` -> `that method`? Since I think you mean `seekExact(BytesRef, TermState)` when you say `this method` here (but the previous `this method` two lines up is referring to `prepareSeekExact`? Pronouns are hard! ########## lucene/core/src/java/org/apache/lucene/index/TermsEnum.java: ########## @@ -61,6 +62,21 @@ public enum SeekStatus { */ public abstract boolean seekExact(BytesRef text) throws IOException; + /** + * Prepare a future call to {@link #seekExact}. This typically calls {@link IndexInput#prefetch} + * on the right range of bytes under the hood so that the next call to {@link #seekExact} is + * faster. This can be used to parallelize I/O across multiple terms by calling {@link + * #prepareSeekExact} on multiple terms enums before calling {@link #seekExact(BytesRef)} on the + * same {@link TermsEnum}s. + * + * <p><b>NOTE</b>: The terms enum is unpositioned after calling this method. Review Comment: Hmm is it really that it is unpositioned, or, that this method does not alter the `TermsEnum`'s positioned state? I.e. if I position it to some term, then call this method, won't it still be positioned on that same (prior) term? Or are we trying to reserve the future right in the API to break the positioning, even though this first impl preserves it? ########## lucene/core/src/java/org/apache/lucene/search/TermQuery.java: ########## @@ -150,7 +170,12 @@ public Scorer get(long leadCost) throws IOException { @Override public long cost() { - return docFreq; + try { + TermsEnum te = getTermsEnum(); Review Comment: Hmm this getter got more costly. It's too bad `TermState` is so opaque -- under the hood it (`BlockTermState`) is already storing `docFreq`. ########## lucene/core/src/java/org/apache/lucene/search/BlendedTermQuery.java: ########## @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import java.util.function.Supplier; Review Comment: Should we open a spinoff issue to maybe add prefetch to `TermInSetQuery` too? ########## lucene/core/src/java/org/apache/lucene/index/TermStates.java: ########## @@ -172,22 +155,59 @@ public void accumulateStatistics(final int docFreq, final long totalTermFreq) { } /** - * Returns the {@link TermState} for a leaf reader context or <code>null</code> 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. + * + * <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. - * @return the {@link TermState} for the given readers ord or <code>null</code> if no {@link - * TermState} for the reader was registered + * @return a Supplier for a TermState. */ - public TermState get(LeafReaderContext ctx) throws IOException { + public Supplier<TermState> get(LeafReaderContext ctx) throws IOException { Review Comment: This sort of reminds me of two-phase commit, except at read-time not write-time: we now break up these IO heavy read APIs into two phases, now, where step 1 is the intention to get X soon (allowing prefetch to happen, especially concurrently not just in the background of the calling thread, but, across the N different Xs we want to retrieve). Step 2 is to then go and block on the IO to retrieve each of the N Xs. Two phased reads! ########## lucene/core/src/java/org/apache/lucene/index/CheckIndex.java: ########## @@ -3754,13 +3754,17 @@ public static Status.TermVectorStatus testTermVectors( TermsEnum postingsTermsEnum = postingsTerms.iterator(); final boolean hasProx = terms.hasOffsets() || terms.hasPositions(); + int seekExactCounter = 0; BytesRef term; while ((term = termsEnum.next()) != null) { // This is the term vectors: postings = termsEnum.postings(postings, PostingsEnum.ALL); assert postings != null; + if ((seekExactCounter++ & 0xFF) == 0) { + postingsTermsEnum.prepareSeekExact(term); Review Comment: Nice -- this is to make sure we are exercising the API? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org