-
Notifications
You must be signed in to change notification settings - Fork 1.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Use IndexInput#prefetch
for terms dictionary lookups.
#13359
Changes from 1 commit
e880d88
127699c
082776b
36ffdc5
062b054
ea6ebc3
64c355f
2639e1b
373da0d
b0ab98f
e6d19a7
448472d
2fcd76a
4fb285c
a2b9a4d
c596da3
d9f0e2a
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,10 +17,12 @@ | |
package org.apache.lucene.index; | ||
|
||
import java.io.IOException; | ||
import java.io.UncheckedIOException; | ||
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; | ||
|
||
|
@@ -179,15 +181,39 @@ public void accumulateStatistics(final int docFreq, final long totalTermFreq) { | |
* @return the {@link TermState} for the given readers ord or <code>null</code> if no {@link | ||
* TermState} for the reader was registered | ||
*/ | ||
public TermState get(LeafReaderContext ctx) throws IOException { | ||
public Supplier<TermState> get(LeafReaderContext ctx) throws IOException { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 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! |
||
assert ctx.ord >= 0 && ctx.ord < states.length; | ||
if (term == null) return states[ctx.ord]; | ||
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(); | ||
final Terms terms = Terms.getTerms(ctx.reader(), term.field()); | ||
final TermsEnum termsEnum = terms.iterator(); | ||
termsEnum.prepareSeekExact(term.bytes()); | ||
return () -> { | ||
if (this.states[ctx.ord] == null) { | ||
try { | ||
TermState state = null; | ||
if (termsEnum.seekExact(term.bytes())) { | ||
state = termsEnum.termState(); | ||
} | ||
this.states[ctx.ord] = state == null ? EMPTY_TERMSTATE : state; | ||
} catch (IOException e) { | ||
throw new UncheckedIOException(e); | ||
} | ||
} | ||
TermState state = this.states[ctx.ord]; | ||
if (state == EMPTY_TERMSTATE) { | ||
return null; | ||
} | ||
return state; | ||
}; | ||
} | ||
if (this.states[ctx.ord] == EMPTY_TERMSTATE) return null; | ||
return this.states[ctx.ord]; | ||
return () -> { | ||
TermState state = this.states[ctx.ord]; | ||
if (state == EMPTY_TERMSTATE) { | ||
return null; | ||
} | ||
return state; | ||
}; | ||
} | ||
|
||
/** | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,6 +17,7 @@ | |
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; | ||
|
@@ -61,6 +62,15 @@ 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. | ||
*/ | ||
public void prepareSeekExact(BytesRef text) throws IOException {} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we look into subclasses such as |
||
|
||
/** | ||
* 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 | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,7 +17,9 @@ | |
package org.apache.lucene.search; | ||
|
||
import java.io.IOException; | ||
import java.io.UncheckedIOException; | ||
import java.util.Objects; | ||
import java.util.function.Supplier; | ||
import org.apache.lucene.index.IndexReaderContext; | ||
import org.apache.lucene.index.LeafReader; | ||
import org.apache.lucene.index.LeafReaderContext; | ||
|
@@ -119,18 +121,34 @@ public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOExcepti | |
: "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) { | ||
return null; | ||
} | ||
final int docFreq = termsEnum.docFreq(); | ||
final Supplier<TermState> stateSupplier = termStates.get(context); | ||
|
||
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) { | ||
// nocommit: should we start allowing ScorerSupplier#get to return null? | ||
return new ConstantScoreScorer( | ||
TermWeight.this, 0f, scoreMode, DocIdSetIterator.empty()); | ||
} | ||
|
||
LeafSimScorer scorer = | ||
new LeafSimScorer(simScorer, context.reader(), term.field(), scoreMode.needsScores()); | ||
if (scoreMode == ScoreMode.TOP_SCORES) { | ||
|
@@ -150,7 +168,12 @@ public Scorer get(long leadCost) throws IOException { | |
|
||
@Override | ||
public long cost() { | ||
return docFreq; | ||
try { | ||
TermsEnum te = getTermsEnum(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hmm this getter got more costly. It's too bad There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Note that we were already getting a Agreed that it's a pity to pull a terms enum only to get a cost, which is already encapsulated in the term state. Though I don't expect it to be a major issue in practice. |
||
return te == null ? 0 : te.docFreq(); | ||
} catch (IOException e) { | ||
throw new UncheckedIOException(e); | ||
} | ||
} | ||
|
||
@Override | ||
|
@@ -183,7 +206,7 @@ private TermsEnum getTermsEnum(LeafReaderContext context) throws IOException { | |
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 TermState state = termStates.get(context).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; | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I really don't like these calls to
seek()
just to prefetch data. Since it is just prefetching, I'd prefer if this "dance" was an impl detail, if needed.It would make the code simpler to just pass parameter to prefetch rather than do this.
Then it is clear that the default implementation won't cause harm (unnecessary io) for any directory subclasses
So I think prefetch should take location as argument? It is just a hint and not real i/o by the thread. It's intentionally not sequential and sequential API for it only hurts.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I opened #13363.