Merge branch 'main' into speed_up_filtered_maxscore

This commit is contained in:
Adrien Grand 2024-12-03 17:44:49 +01:00
commit 9b314aa155
31 changed files with 555 additions and 216 deletions

View File

@ -19,7 +19,7 @@ Improvements
Optimizations Optimizations
--------------------- ---------------------
(No changes) * GITHUB#14011: Reduce allocation rate in HNSW concurrent merge. (Viliam Durina)
Bug Fixes Bug Fixes
--------------------- ---------------------
@ -41,6 +41,9 @@ API Changes
* GITHUB#13957: Removed LeafSimScorer class, to save its overhead. Scorers now * GITHUB#13957: Removed LeafSimScorer class, to save its overhead. Scorers now
compute scores directly from a SimScorer, postings and norms. (Adrien Grand) compute scores directly from a SimScorer, postings and norms. (Adrien Grand)
* GITHUB#13998: Add IndexInput::isLoaded to determine if the contents of an
input is resident in physical memory. (Chris Hegarty)
New Features New Features
--------------------- ---------------------
(No changes) (No changes)
@ -113,6 +116,15 @@ Optimizations
* GITHUB#14014: Filtered disjunctions now get executed via `MaxScoreBulkScorer`. * GITHUB#14014: Filtered disjunctions now get executed via `MaxScoreBulkScorer`.
(Adrien Grand) (Adrien Grand)
* GITHUB#14023: Make JVM inlining decisions more predictable in our main
queries. (Adrien Grand)
* GITHUB#14032: Speed up PostingsEnum when positions are requested.
(Adrien Grand)
* GITHUB#14031: Ensure Panama float vector distance impls inlinable.
(Robert Muir, Chris Hegarty)
Bug Fixes Bug Fixes
--------------------- ---------------------
* GITHUB#13832: Fixed an issue where the DefaultPassageFormatter.format method did not format passages as intended * GITHUB#13832: Fixed an issue where the DefaultPassageFormatter.format method did not format passages as intended
@ -143,6 +155,8 @@ Bug Fixes
* GITHUB#14008: Counts provided by taxonomy facets in addition to another aggregation are now returned together with * GITHUB#14008: Counts provided by taxonomy facets in addition to another aggregation are now returned together with
their corresponding ordinals. (Paul King) their corresponding ordinals. (Paul King)
* GITHUB#14027: Make SegmentInfos#readCommit(Directory, String, int) public (Luca Cavanna)
======================= Lucene 10.0.0 ======================= ======================= Lucene 10.0.0 =======================
API Changes API Changes

View File

@ -638,9 +638,13 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
final boolean indexHasPayloads; final boolean indexHasPayloads;
final boolean indexHasOffsetsOrPayloads; final boolean indexHasOffsetsOrPayloads;
private int freq; // freq we last read private long freqFP; // offset of the freq block
private int position; // current position private int position; // current position
// value of docBufferUpto on the last doc ID when positions have been read
private int posDocBufferUpto;
// how many positions "behind" we are; nextPosition must // how many positions "behind" we are; nextPosition must
// skip these to "catch up": // skip these to "catch up":
private int posPendingCount; private int posPendingCount;
@ -662,6 +666,7 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
private boolean needsOffsets; // true if we actually need offsets private boolean needsOffsets; // true if we actually need offsets
private boolean needsPayloads; // true if we actually need payloads private boolean needsPayloads; // true if we actually need payloads
private boolean needsPayloadsOrOffsets;
public EverythingEnum(FieldInfo fieldInfo) throws IOException { public EverythingEnum(FieldInfo fieldInfo) throws IOException {
super(fieldInfo); super(fieldInfo);
@ -745,8 +750,11 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
lastPosBlockFP = posTermStartFP + termState.lastPosBlockOffset; lastPosBlockFP = posTermStartFP + termState.lastPosBlockOffset;
} }
this.needsOffsets = PostingsEnum.featureRequested(flags, PostingsEnum.OFFSETS); this.needsOffsets =
this.needsPayloads = PostingsEnum.featureRequested(flags, PostingsEnum.PAYLOADS); indexHasOffsets && PostingsEnum.featureRequested(flags, PostingsEnum.OFFSETS);
this.needsPayloads =
indexHasPayloads && PostingsEnum.featureRequested(flags, PostingsEnum.PAYLOADS);
this.needsPayloadsOrOffsets = this.needsPayloads || this.needsOffsets;
level1BlockPosUpto = 0; level1BlockPosUpto = 0;
level1BlockPayUpto = 0; level1BlockPayUpto = 0;
@ -758,8 +766,13 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
} }
@Override @Override
public int freq() { public int freq() throws IOException {
return freq; if (freqFP != -1) {
docIn.seek(freqFP);
pforUtil.decode(docInUtil, freqBuffer);
freqFP = -1;
}
return freqBuffer[docBufferUpto - 1];
} }
private void refillDocs() throws IOException { private void refillDocs() throws IOException {
@ -768,11 +781,13 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
if (left >= BLOCK_SIZE) { if (left >= BLOCK_SIZE) {
forDeltaUtil.decodeAndPrefixSum(docInUtil, prevDocID, docBuffer); forDeltaUtil.decodeAndPrefixSum(docInUtil, prevDocID, docBuffer);
pforUtil.decode(docInUtil, freqBuffer); freqFP = docIn.getFilePointer();
PForUtil.skip(docIn);
docCountUpto += BLOCK_SIZE; docCountUpto += BLOCK_SIZE;
} else if (docFreq == 1) { } else if (docFreq == 1) {
docBuffer[0] = singletonDocID; docBuffer[0] = singletonDocID;
freqBuffer[0] = (int) totalTermFreq; freqBuffer[0] = (int) totalTermFreq;
freqFP = -1;
docBuffer[1] = NO_MORE_DOCS; docBuffer[1] = NO_MORE_DOCS;
docCountUpto++; docCountUpto++;
docBufferSize = 1; docBufferSize = 1;
@ -781,11 +796,13 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
PostingsUtil.readVIntBlock(docIn, docBuffer, freqBuffer, left, indexHasFreq, true); PostingsUtil.readVIntBlock(docIn, docBuffer, freqBuffer, left, indexHasFreq, true);
prefixSum(docBuffer, left, prevDocID); prefixSum(docBuffer, left, prevDocID);
docBuffer[left] = NO_MORE_DOCS; docBuffer[left] = NO_MORE_DOCS;
freqFP = -1;
docCountUpto += left; docCountUpto += left;
docBufferSize = left; docBufferSize = left;
} }
prevDocID = docBuffer[BLOCK_SIZE - 1]; prevDocID = docBuffer[BLOCK_SIZE - 1];
docBufferUpto = 0; docBufferUpto = 0;
posDocBufferUpto = 0;
assert docBuffer[docBufferSize] == NO_MORE_DOCS; assert docBuffer[docBufferSize] == NO_MORE_DOCS;
} }
@ -846,6 +863,8 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
payloadByteUpto = level0BlockPayUpto; payloadByteUpto = level0BlockPayUpto;
} }
posBufferUpto = BLOCK_SIZE; posBufferUpto = BLOCK_SIZE;
} else {
posPendingCount += sumOverRange(freqBuffer, posDocBufferUpto, BLOCK_SIZE);
} }
if (docFreq - docCountUpto >= BLOCK_SIZE) { if (docFreq - docCountUpto >= BLOCK_SIZE) {
@ -875,34 +894,23 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
} }
this.doc = docBuffer[docBufferUpto]; this.doc = docBuffer[docBufferUpto];
this.freq = freqBuffer[docBufferUpto];
docBufferUpto++; docBufferUpto++;
posPendingCount += freq;
position = 0;
lastStartOffset = 0;
return doc; return doc;
} }
private void skipLevel0To(int target) throws IOException { private void skipLevel0To(int target) throws IOException {
long posFP;
int posUpto;
long payFP;
int payUpto;
while (true) { while (true) {
prevDocID = level0LastDocID; prevDocID = level0LastDocID;
// If nextBlockPosFP is less than the current FP, it means that the block of positions for posFP = level0PosEndFP;
// the first docs of the next block are already decoded. In this case we just accumulate posUpto = level0BlockPosUpto;
// frequencies into posPendingCount instead of seeking backwards and decoding the same pos payFP = level0PayEndFP;
// block again. payUpto = level0BlockPayUpto;
if (level0PosEndFP >= posIn.getFilePointer()) {
posIn.seek(level0PosEndFP);
posPendingCount = level0BlockPosUpto;
if (indexHasOffsetsOrPayloads) {
assert level0PayEndFP >= payIn.getFilePointer();
payIn.seek(level0PayEndFP);
payloadByteUpto = level0BlockPayUpto;
}
posBufferUpto = BLOCK_SIZE;
} else {
posPendingCount += sumOverRange(freqBuffer, docBufferUpto, BLOCK_SIZE);
}
if (docFreq - docCountUpto >= BLOCK_SIZE) { if (docFreq - docCountUpto >= BLOCK_SIZE) {
docIn.readVLong(); // skip0 num bytes docIn.readVLong(); // skip0 num bytes
@ -931,6 +939,23 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
break; break;
} }
} }
// If nextBlockPosFP is less than the current FP, it means that the block of positions for
// the first docs of the next block are already decoded. In this case we just accumulate
// frequencies into posPendingCount instead of seeking backwards and decoding the same pos
// block again.
if (posFP >= posIn.getFilePointer()) {
posIn.seek(posFP);
posPendingCount = posUpto;
if (indexHasOffsetsOrPayloads) {
assert level0PayEndFP >= payIn.getFilePointer();
payIn.seek(payFP);
payloadByteUpto = payUpto;
}
posBufferUpto = BLOCK_SIZE;
} else {
posPendingCount += sumOverRange(freqBuffer, posDocBufferUpto, BLOCK_SIZE);
}
} }
@Override @Override
@ -947,16 +972,12 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
} }
int next = VectorUtil.findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize); int next = VectorUtil.findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize);
posPendingCount += sumOverRange(freqBuffer, docBufferUpto, next + 1);
this.freq = freqBuffer[next];
this.docBufferUpto = next + 1; this.docBufferUpto = next + 1;
position = 0;
lastStartOffset = 0;
return this.doc = docBuffer[next]; return this.doc = docBuffer[next];
} }
private void skipPositions() throws IOException { private void skipPositions(int freq) throws IOException {
// Skip positions now: // Skip positions now:
int toSkip = posPendingCount - freq; int toSkip = posPendingCount - freq;
// if (DEBUG) { // if (DEBUG) {
@ -1003,41 +1024,45 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
lastStartOffset = 0; lastStartOffset = 0;
} }
private void refillLastPositionBlock() throws IOException {
final int count = (int) (totalTermFreq % BLOCK_SIZE);
int payloadLength = 0;
int offsetLength = 0;
payloadByteUpto = 0;
for (int i = 0; i < count; i++) {
int code = posIn.readVInt();
if (indexHasPayloads) {
if ((code & 1) != 0) {
payloadLength = posIn.readVInt();
}
payloadLengthBuffer[i] = payloadLength;
posDeltaBuffer[i] = code >>> 1;
if (payloadLength != 0) {
if (payloadByteUpto + payloadLength > payloadBytes.length) {
payloadBytes = ArrayUtil.grow(payloadBytes, payloadByteUpto + payloadLength);
}
posIn.readBytes(payloadBytes, payloadByteUpto, payloadLength);
payloadByteUpto += payloadLength;
}
} else {
posDeltaBuffer[i] = code;
}
if (indexHasOffsets) {
int deltaCode = posIn.readVInt();
if ((deltaCode & 1) != 0) {
offsetLength = posIn.readVInt();
}
offsetStartDeltaBuffer[i] = deltaCode >>> 1;
offsetLengthBuffer[i] = offsetLength;
}
}
payloadByteUpto = 0;
}
private void refillPositions() throws IOException { private void refillPositions() throws IOException {
if (posIn.getFilePointer() == lastPosBlockFP) { if (posIn.getFilePointer() == lastPosBlockFP) {
final int count = (int) (totalTermFreq % BLOCK_SIZE); refillLastPositionBlock();
int payloadLength = 0;
int offsetLength = 0;
payloadByteUpto = 0;
for (int i = 0; i < count; i++) {
int code = posIn.readVInt();
if (indexHasPayloads) {
if ((code & 1) != 0) {
payloadLength = posIn.readVInt();
}
payloadLengthBuffer[i] = payloadLength;
posDeltaBuffer[i] = code >>> 1;
if (payloadLength != 0) {
if (payloadByteUpto + payloadLength > payloadBytes.length) {
payloadBytes = ArrayUtil.grow(payloadBytes, payloadByteUpto + payloadLength);
}
posIn.readBytes(payloadBytes, payloadByteUpto, payloadLength);
payloadByteUpto += payloadLength;
}
} else {
posDeltaBuffer[i] = code;
}
if (indexHasOffsets) {
int deltaCode = posIn.readVInt();
if ((deltaCode & 1) != 0) {
offsetLength = posIn.readVInt();
}
offsetStartDeltaBuffer[i] = deltaCode >>> 1;
offsetLengthBuffer[i] = offsetLength;
}
}
payloadByteUpto = 0;
} else { } else {
pforUtil.decode(posInUtil, posDeltaBuffer); pforUtil.decode(posInUtil, posDeltaBuffer);
@ -1054,8 +1079,7 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
// this works, because when writing a vint block we always force the first length to be // this works, because when writing a vint block we always force the first length to be
// written // written
PForUtil.skip(payIn); // skip over lengths PForUtil.skip(payIn); // skip over lengths
int numBytes = payIn.readVInt(); // read length of payloadBytes payIn.skipBytes(payIn.readVInt()); // skip over payloadBytes
payIn.seek(payIn.getFilePointer() + numBytes); // skip over payloadBytes
} }
payloadByteUpto = 0; payloadByteUpto = 0;
} }
@ -1074,13 +1098,40 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
} }
} }
private void accumulatePayloadAndOffsets() {
if (needsPayloads) {
payloadLength = payloadLengthBuffer[posBufferUpto];
payload.bytes = payloadBytes;
payload.offset = payloadByteUpto;
payload.length = payloadLength;
payloadByteUpto += payloadLength;
}
if (needsOffsets) {
startOffset = lastStartOffset + offsetStartDeltaBuffer[posBufferUpto];
endOffset = startOffset + offsetLengthBuffer[posBufferUpto];
lastStartOffset = startOffset;
}
}
@Override @Override
public int nextPosition() throws IOException { public int nextPosition() throws IOException {
assert posPendingCount > 0; if (posDocBufferUpto != docBufferUpto) {
int freq = freq(); // triggers lazy decoding of freqs
if (posPendingCount > freq) { // First position that is being read on this doc.
skipPositions(); posPendingCount += sumOverRange(freqBuffer, posDocBufferUpto, docBufferUpto);
posPendingCount = freq; posDocBufferUpto = docBufferUpto;
assert posPendingCount > 0;
if (posPendingCount > freq) {
skipPositions(freq);
posPendingCount = freq;
}
position = 0;
lastStartOffset = 0;
} }
if (posBufferUpto == BLOCK_SIZE) { if (posBufferUpto == BLOCK_SIZE) {
@ -1089,18 +1140,8 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
} }
position += posDeltaBuffer[posBufferUpto]; position += posDeltaBuffer[posBufferUpto];
if (indexHasPayloads) { if (needsPayloadsOrOffsets) {
payloadLength = payloadLengthBuffer[posBufferUpto]; accumulatePayloadAndOffsets();
payload.bytes = payloadBytes;
payload.offset = payloadByteUpto;
payload.length = payloadLength;
payloadByteUpto += payloadLength;
}
if (indexHasOffsets) {
startOffset = lastStartOffset + offsetStartDeltaBuffer[posBufferUpto];
endOffset = startOffset + offsetLengthBuffer[posBufferUpto];
lastStartOffset = startOffset;
} }
posBufferUpto++; posBufferUpto++;
@ -1110,17 +1151,23 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
@Override @Override
public int startOffset() { public int startOffset() {
if (needsOffsets == false) {
return -1;
}
return startOffset; return startOffset;
} }
@Override @Override
public int endOffset() { public int endOffset() {
if (needsOffsets == false) {
return -1;
}
return endOffset; return endOffset;
} }
@Override @Override
public BytesRef getPayload() { public BytesRef getPayload() {
if (payloadLength == 0) { if (needsPayloads == false || payloadLength == 0) {
return null; return null;
} else { } else {
return payload; return payload;
@ -1466,9 +1513,13 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
final boolean indexHasPayloads; final boolean indexHasPayloads;
final boolean indexHasOffsetsOrPayloads; final boolean indexHasOffsetsOrPayloads;
private int freq; // freq we last read private long freqFP; // offset of the freq block
private int position; // current position private int position; // current position
// value of docBufferUpto on the last doc ID when positions have been read
private int posDocBufferUpto;
// how many positions "behind" we are; nextPosition must // how many positions "behind" we are; nextPosition must
// skip these to "catch up": // skip these to "catch up":
private int posPendingCount; private int posPendingCount;
@ -1516,8 +1567,13 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
} }
@Override @Override
public int freq() { public int freq() throws IOException {
return freq; if (freqFP != -1) {
docIn.seek(freqFP);
pforUtil.decode(docInUtil, freqBuffer);
freqFP = -1;
}
return freqBuffer[docBufferUpto - 1];
} }
private void refillDocs() throws IOException { private void refillDocs() throws IOException {
@ -1526,24 +1582,30 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
if (left >= BLOCK_SIZE) { if (left >= BLOCK_SIZE) {
forDeltaUtil.decodeAndPrefixSum(docInUtil, prevDocID, docBuffer); forDeltaUtil.decodeAndPrefixSum(docInUtil, prevDocID, docBuffer);
pforUtil.decode(docInUtil, freqBuffer); freqFP = docIn.getFilePointer();
PForUtil.skip(docIn);
docCountUpto += BLOCK_SIZE; docCountUpto += BLOCK_SIZE;
} else if (docFreq == 1) { } else if (docFreq == 1) {
docBuffer[0] = singletonDocID; docBuffer[0] = singletonDocID;
freqBuffer[0] = (int) totalTermFreq; freqBuffer[0] = (int) totalTermFreq;
freqFP = -1;
docBuffer[1] = NO_MORE_DOCS; docBuffer[1] = NO_MORE_DOCS;
docCountUpto++; docCountUpto++;
docBufferSize = 1; docBufferSize = 1;
} else { } else {
// Read vInts: // Read vInts:
PostingsUtil.readVIntBlock(docIn, docBuffer, freqBuffer, left, indexHasFreq, true); PostingsUtil.readVIntBlock(docIn, docBuffer, freqBuffer, left, indexHasFreq, true);
prefixSum(docBuffer, left, prevDocID); prefixSum(docBuffer, left, prevDocID);
docBuffer[left] = NO_MORE_DOCS; docBuffer[left] = NO_MORE_DOCS;
freqFP = -1;
docCountUpto += left; docCountUpto += left;
docBufferSize = left; docBufferSize = left;
freqFP = -1;
} }
prevDocID = docBuffer[BLOCK_SIZE - 1]; prevDocID = docBuffer[BLOCK_SIZE - 1];
docBufferUpto = 0; docBufferUpto = 0;
posDocBufferUpto = 0;
assert docBuffer[docBufferSize] == NO_MORE_DOCS; assert docBuffer[docBufferSize] == NO_MORE_DOCS;
} }
@ -1585,20 +1647,14 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
} }
private void skipLevel0To(int target) throws IOException { private void skipLevel0To(int target) throws IOException {
long posFP;
int posUpto;
while (true) { while (true) {
prevDocID = level0LastDocID; prevDocID = level0LastDocID;
// If nextBlockPosFP is less than the current FP, it means that the block of positions for posFP = level0PosEndFP;
// the first docs of the next block are already decoded. In this case we just accumulate posUpto = level0BlockPosUpto;
// frequencies into posPendingCount instead of seeking backwards and decoding the same pos
// block again.
if (level0PosEndFP >= posIn.getFilePointer()) {
posIn.seek(level0PosEndFP);
posPendingCount = level0BlockPosUpto;
posBufferUpto = BLOCK_SIZE;
} else {
posPendingCount += sumOverRange(freqBuffer, docBufferUpto, BLOCK_SIZE);
}
if (docFreq - docCountUpto >= BLOCK_SIZE) { if (docFreq - docCountUpto >= BLOCK_SIZE) {
docIn.readVLong(); // skip0 num bytes docIn.readVLong(); // skip0 num bytes
@ -1631,6 +1687,18 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
break; break;
} }
} }
// If nextBlockPosFP is less than the current FP, it means that the block of positions for
// the first docs of the next block are already decoded. In this case we just accumulate
// frequencies into posPendingCount instead of seeking backwards and decoding the same pos
// block again.
if (posFP >= posIn.getFilePointer()) {
posIn.seek(posFP);
posPendingCount = posUpto;
posBufferUpto = BLOCK_SIZE;
} else {
posPendingCount += sumOverRange(freqBuffer, posDocBufferUpto, BLOCK_SIZE);
}
} }
@Override @Override
@ -1660,30 +1728,25 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
} }
doc = docBuffer[docBufferUpto]; doc = docBuffer[docBufferUpto];
freq = freqBuffer[docBufferUpto];
posPendingCount += freq;
docBufferUpto++; docBufferUpto++;
position = 0;
return this.doc; return this.doc;
} }
@Override @Override
public int advance(int target) throws IOException { public int advance(int target) throws IOException {
advanceShallow(target); if (target > level0LastDocID || needsRefilling) {
if (needsRefilling) { advanceShallow(target);
assert needsRefilling;
refillDocs(); refillDocs();
needsRefilling = false; needsRefilling = false;
} }
int next = VectorUtil.findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize); int next = VectorUtil.findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize);
posPendingCount += sumOverRange(freqBuffer, docBufferUpto, next + 1);
freq = freqBuffer[next];
docBufferUpto = next + 1; docBufferUpto = next + 1;
position = 0;
return this.doc = docBuffer[next]; return this.doc = docBuffer[next];
} }
private void skipPositions() throws IOException { private void skipPositions(int freq) throws IOException {
// Skip positions now: // Skip positions now:
int toSkip = posPendingCount - freq; int toSkip = posPendingCount - freq;
// if (DEBUG) { // if (DEBUG) {
@ -1703,8 +1766,6 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
refillPositions(); refillPositions();
posBufferUpto = toSkip; posBufferUpto = toSkip;
} }
position = 0;
} }
private void refillPositions() throws IOException { private void refillPositions() throws IOException {
@ -1739,11 +1800,21 @@ public final class Lucene101PostingsReader extends PostingsReaderBase {
@Override @Override
public int nextPosition() throws IOException { public int nextPosition() throws IOException {
assert posPendingCount > 0; if (posDocBufferUpto != docBufferUpto) {
int freq = freq(); // triggers lazy decoding of freqs
if (posPendingCount > freq) { // First position that is being read on this doc.
skipPositions(); posPendingCount += sumOverRange(freqBuffer, posDocBufferUpto, docBufferUpto);
posPendingCount = freq; posDocBufferUpto = docBufferUpto;
assert posPendingCount > 0;
if (posPendingCount > freq) {
skipPositions(freq);
posPendingCount = freq;
}
position = 0;
} }
if (posBufferUpto == BLOCK_SIZE) { if (posBufferUpto == BLOCK_SIZE) {

View File

@ -284,7 +284,14 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
return readCommit(directory, segmentFileName, Version.MIN_SUPPORTED_MAJOR); return readCommit(directory, segmentFileName, Version.MIN_SUPPORTED_MAJOR);
} }
static final SegmentInfos readCommit( /**
* Read a particular segmentFileName, as long as the commit's {@link
* SegmentInfos#getIndexCreatedVersionMajor()} is strictly greater than the provided minimum
* supported major version. If the commit's version is older, an {@link
* IndexFormatTooOldException} will be thrown. Note that this may throw an IOException if a commit
* is in process.
*/
public static final SegmentInfos readCommit(
Directory directory, String segmentFileName, int minSupportedMajorVersion) Directory directory, String segmentFileName, int minSupportedMajorVersion)
throws IOException { throws IOException {
@ -307,7 +314,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
} }
/** Read the commit from the provided {@link ChecksumIndexInput}. */ /** Read the commit from the provided {@link ChecksumIndexInput}. */
static final SegmentInfos readCommit( public static final SegmentInfos readCommit(
Directory directory, ChecksumIndexInput input, long generation, int minSupportedMajorVersion) Directory directory, ChecksumIndexInput input, long generation, int minSupportedMajorVersion)
throws IOException { throws IOException {
Throwable priorE = null; Throwable priorE = null;

View File

@ -38,7 +38,7 @@ final class BlockMaxConjunctionBulkScorer extends BulkScorer {
private final Scorer[] scorers; private final Scorer[] scorers;
private final DocIdSetIterator[] iterators; private final DocIdSetIterator[] iterators;
private final DocIdSetIterator lead1, lead2; private final DocIdSetIterator lead1, lead2;
private final Scorer scorer1, scorer2; private final Scorable scorer1, scorer2;
private final DocAndScore scorable = new DocAndScore(); private final DocAndScore scorable = new DocAndScore();
private final double[] sumOfOtherClauses; private final double[] sumOfOtherClauses;
private final int maxDoc; private final int maxDoc;
@ -51,10 +51,10 @@ final class BlockMaxConjunctionBulkScorer extends BulkScorer {
Arrays.sort(this.scorers, Comparator.comparingLong(scorer -> scorer.iterator().cost())); Arrays.sort(this.scorers, Comparator.comparingLong(scorer -> scorer.iterator().cost()));
this.iterators = this.iterators =
Arrays.stream(this.scorers).map(Scorer::iterator).toArray(DocIdSetIterator[]::new); Arrays.stream(this.scorers).map(Scorer::iterator).toArray(DocIdSetIterator[]::new);
lead1 = iterators[0]; lead1 = ScorerUtil.likelyImpactsEnum(iterators[0]);
lead2 = iterators[1]; lead2 = ScorerUtil.likelyImpactsEnum(iterators[1]);
scorer1 = this.scorers[0]; scorer1 = ScorerUtil.likelyTermScorer(this.scorers[0]);
scorer2 = this.scorers[1]; scorer2 = ScorerUtil.likelyTermScorer(this.scorers[1]);
this.sumOfOtherClauses = new double[this.scorers.length]; this.sumOfOtherClauses = new double[this.scorers.length];
for (int i = 0; i < sumOfOtherClauses.length; i++) { for (int i = 0; i < sumOfOtherClauses.length; i++) {
sumOfOtherClauses[i] = Double.POSITIVE_INFINITY; sumOfOtherClauses[i] = Double.POSITIVE_INFINITY;

View File

@ -29,6 +29,7 @@ import java.util.List;
*/ */
final class BlockMaxConjunctionScorer extends Scorer { final class BlockMaxConjunctionScorer extends Scorer {
final Scorer[] scorers; final Scorer[] scorers;
final Scorable[] scorables;
final DocIdSetIterator[] approximations; final DocIdSetIterator[] approximations;
final TwoPhaseIterator[] twoPhases; final TwoPhaseIterator[] twoPhases;
float minScore; float minScore;
@ -38,6 +39,8 @@ final class BlockMaxConjunctionScorer extends Scorer {
this.scorers = scorersList.toArray(new Scorer[scorersList.size()]); this.scorers = scorersList.toArray(new Scorer[scorersList.size()]);
// Sort scorer by cost // Sort scorer by cost
Arrays.sort(this.scorers, Comparator.comparingLong(s -> s.iterator().cost())); Arrays.sort(this.scorers, Comparator.comparingLong(s -> s.iterator().cost()));
this.scorables =
Arrays.stream(scorers).map(ScorerUtil::likelyTermScorer).toArray(Scorable[]::new);
this.approximations = new DocIdSetIterator[scorers.length]; this.approximations = new DocIdSetIterator[scorers.length];
List<TwoPhaseIterator> twoPhaseList = new ArrayList<>(); List<TwoPhaseIterator> twoPhaseList = new ArrayList<>();
@ -50,6 +53,7 @@ final class BlockMaxConjunctionScorer extends Scorer {
} else { } else {
approximations[i] = scorer.iterator(); approximations[i] = scorer.iterator();
} }
approximations[i] = ScorerUtil.likelyImpactsEnum(approximations[i]);
scorer.advanceShallow(0); scorer.advanceShallow(0);
} }
this.twoPhases = twoPhaseList.toArray(new TwoPhaseIterator[twoPhaseList.size()]); this.twoPhases = twoPhaseList.toArray(new TwoPhaseIterator[twoPhaseList.size()]);
@ -207,7 +211,7 @@ final class BlockMaxConjunctionScorer extends Scorer {
@Override @Override
public float score() throws IOException { public float score() throws IOException {
double score = 0; double score = 0;
for (Scorer scorer : scorers) { for (Scorable scorer : scorables) {
score += scorer.score(); score += scorer.score();
} }
return (float) score; return (float) score;

View File

@ -604,23 +604,20 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
// Important(this can only be processed after nested clauses have been flattened) // Important(this can only be processed after nested clauses have been flattened)
{ {
final Collection<Query> shoulds = clauseSets.get(Occur.SHOULD); final Collection<Query> shoulds = clauseSets.get(Occur.SHOULD);
if (shoulds.size() > 0) { if (shoulds.size() < minimumNumberShouldMatch) {
if (shoulds.size() < minimumNumberShouldMatch) { return new MatchNoDocsQuery("SHOULD clause count less than minimumNumberShouldMatch");
return new MatchNoDocsQuery("SHOULD clause count less than minimumNumberShouldMatch"); }
} if (shoulds.size() > 0 && shoulds.size() == minimumNumberShouldMatch) {
BooleanQuery.Builder builder = new BooleanQuery.Builder();
if (shoulds.size() == minimumNumberShouldMatch) { for (BooleanClause clause : clauses) {
BooleanQuery.Builder builder = new BooleanQuery.Builder(); if (clause.occur() == Occur.SHOULD) {
for (BooleanClause clause : clauses) { builder.add(clause.query(), Occur.MUST);
if (clause.occur() == Occur.SHOULD) { } else {
builder.add(clause.query(), Occur.MUST); builder.add(clause);
} else {
builder.add(clause);
}
} }
return builder.build();
} }
return builder.build();
} }
} }

View File

@ -155,7 +155,7 @@ final class BooleanScorer extends BulkScorer {
this.needsScores = needsScores; this.needsScores = needsScores;
LongArrayList costs = new LongArrayList(scorers.size()); LongArrayList costs = new LongArrayList(scorers.size());
for (Scorer scorer : scorers) { for (Scorer scorer : scorers) {
DisiWrapper w = new DisiWrapper(scorer); DisiWrapper w = new DisiWrapper(scorer, false);
costs.add(w.cost); costs.add(w.cost);
final DisiWrapper evicted = tail.insertWithOverflow(w); final DisiWrapper evicted = tail.insertWithOverflow(w);
if (evicted != null) { if (evicted != null) {
@ -177,7 +177,7 @@ final class BooleanScorer extends BulkScorer {
Bucket[] buckets = BooleanScorer.this.buckets; Bucket[] buckets = BooleanScorer.this.buckets;
DocIdSetIterator it = w.iterator; DocIdSetIterator it = w.iterator;
Scorer scorer = w.scorer; Scorable scorer = w.scorable;
int doc = w.doc; int doc = w.doc;
if (doc < min) { if (doc < min) {
doc = it.advance(min); doc = it.advance(min);

View File

@ -30,7 +30,7 @@ import org.apache.lucene.util.Bits;
*/ */
final class ConjunctionBulkScorer extends BulkScorer { final class ConjunctionBulkScorer extends BulkScorer {
private final Scorer[] scoringScorers; private final Scorable[] scoringScorers;
private final DocIdSetIterator lead1, lead2; private final DocIdSetIterator lead1, lead2;
private final List<DocIdSetIterator> others; private final List<DocIdSetIterator> others;
private final Scorable scorable; private final Scorable scorable;
@ -45,7 +45,8 @@ final class ConjunctionBulkScorer extends BulkScorer {
allScorers.addAll(requiredScoring); allScorers.addAll(requiredScoring);
allScorers.addAll(requiredNoScoring); allScorers.addAll(requiredNoScoring);
this.scoringScorers = requiredScoring.toArray(Scorer[]::new); this.scoringScorers =
requiredScoring.stream().map(ScorerUtil::likelyTermScorer).toArray(Scorable[]::new);
List<DocIdSetIterator> iterators = new ArrayList<>(); List<DocIdSetIterator> iterators = new ArrayList<>();
for (Scorer scorer : allScorers) { for (Scorer scorer : allScorers) {
iterators.add(scorer.iterator()); iterators.add(scorer.iterator());
@ -59,7 +60,7 @@ final class ConjunctionBulkScorer extends BulkScorer {
@Override @Override
public float score() throws IOException { public float score() throws IOException {
double score = 0; double score = 0;
for (Scorer scorer : scoringScorers) { for (Scorable scorer : scoringScorers) {
score += scorer.score(); score += scorer.score();
} }
return (float) score; return (float) score;

View File

@ -16,6 +16,8 @@
*/ */
package org.apache.lucene.search; package org.apache.lucene.search;
import java.util.Objects;
/** /**
* Wrapper used in {@link DisiPriorityQueue}. * Wrapper used in {@link DisiPriorityQueue}.
* *
@ -24,6 +26,7 @@ package org.apache.lucene.search;
public class DisiWrapper { public class DisiWrapper {
public final DocIdSetIterator iterator; public final DocIdSetIterator iterator;
public final Scorer scorer; public final Scorer scorer;
public final Scorable scorable;
public final long cost; public final long cost;
public final float matchCost; // the match cost for two-phase iterators, 0 otherwise public final float matchCost; // the match cost for two-phase iterators, 0 otherwise
public int doc; // the current doc, used for comparison public int doc; // the current doc, used for comparison
@ -42,9 +45,14 @@ public class DisiWrapper {
// for MaxScoreBulkScorer // for MaxScoreBulkScorer
float maxWindowScore; float maxWindowScore;
public DisiWrapper(Scorer scorer) { public DisiWrapper(Scorer scorer, boolean impacts) {
this.scorer = scorer; this.scorer = Objects.requireNonNull(scorer);
this.iterator = scorer.iterator(); this.scorable = ScorerUtil.likelyTermScorer(scorer);
if (impacts) {
this.iterator = ScorerUtil.likelyImpactsEnum(scorer.iterator());
} else {
this.iterator = scorer.iterator();
}
this.cost = iterator.cost(); this.cost = iterator.cost();
this.doc = -1; this.doc = -1;
this.twoPhaseView = scorer.twoPhaseIterator(); this.twoPhaseView = scorer.twoPhaseIterator();

View File

@ -60,7 +60,7 @@ final class DisjunctionMaxScorer extends DisjunctionScorer {
float scoreMax = 0; float scoreMax = 0;
double otherScoreSum = 0; double otherScoreSum = 0;
for (DisiWrapper w = topList; w != null; w = w.next) { for (DisiWrapper w = topList; w != null; w = w.next) {
float subScore = w.scorer.score(); float subScore = w.scorable.score();
if (subScore >= scoreMax) { if (subScore >= scoreMax) {
otherScoreSum += scoreMax; otherScoreSum += scoreMax;
scoreMax = subScore; scoreMax = subScore;

View File

@ -37,7 +37,7 @@ abstract class DisjunctionScorer extends Scorer {
} }
this.subScorers = new DisiPriorityQueue(subScorers.size()); this.subScorers = new DisiPriorityQueue(subScorers.size());
for (Scorer scorer : subScorers) { for (Scorer scorer : subScorers) {
final DisiWrapper w = new DisiWrapper(scorer); final DisiWrapper w = new DisiWrapper(scorer, false);
this.subScorers.add(w); this.subScorers.add(w);
} }
this.needsScores = scoreMode != ScoreMode.COMPLETE_NO_SCORES; this.needsScores = scoreMode != ScoreMode.COMPLETE_NO_SCORES;

View File

@ -40,7 +40,7 @@ final class DisjunctionSumScorer extends DisjunctionScorer {
double score = 0; double score = 0;
for (DisiWrapper w = topList; w != null; w = w.next) { for (DisiWrapper w = topList; w != null; w = w.next) {
score += w.scorer.score(); score += w.scorable.score();
} }
return (float) score; return (float) score;
} }

View File

@ -0,0 +1,51 @@
/*
* 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.search;
import java.io.IOException;
/** Wrapper around a {@link DocIdSetIterator}. */
public class FilterDocIdSetIterator extends DocIdSetIterator {
/** Wrapped instance. */
protected final DocIdSetIterator in;
/** Sole constructor. */
public FilterDocIdSetIterator(DocIdSetIterator in) {
this.in = in;
}
@Override
public int docID() {
return in.docID();
}
@Override
public int nextDoc() throws IOException {
return in.nextDoc();
}
@Override
public int advance(int target) throws IOException {
return in.advance(target);
}
@Override
public long cost() {
return in.cost();
}
}

View File

@ -35,7 +35,7 @@ public abstract class IndriDisjunctionScorer extends IndriScorer {
this.subScorersList = subScorersList; this.subScorersList = subScorersList;
this.subScorers = new DisiPriorityQueue(subScorersList.size()); this.subScorers = new DisiPriorityQueue(subScorersList.size());
for (Scorer scorer : subScorersList) { for (Scorer scorer : subScorersList) {
final DisiWrapper w = new DisiWrapper(scorer); final DisiWrapper w = new DisiWrapper(scorer, false);
this.subScorers.add(w); this.subScorers.add(w);
} }
this.approximation = new DisjunctionDISIApproximation(this.subScorers); this.approximation = new DisjunctionDISIApproximation(this.subScorers);

View File

@ -58,7 +58,7 @@ final class MaxScoreBulkScorer extends BulkScorer {
this.filter = null; this.filter = null;
filterMatches = null; filterMatches = null;
} else { } else {
this.filter = new DisiWrapper(filter); this.filter = new DisiWrapper(filter, false);
filterMatches = new FixedBitSet(INNER_WINDOW_SIZE); filterMatches = new FixedBitSet(INNER_WINDOW_SIZE);
} }
allScorers = new DisiWrapper[scorers.size()]; allScorers = new DisiWrapper[scorers.size()];
@ -66,7 +66,7 @@ final class MaxScoreBulkScorer extends BulkScorer {
int i = 0; int i = 0;
long cost = 0; long cost = 0;
for (Scorer scorer : scorers) { for (Scorer scorer : scorers) {
DisiWrapper w = new DisiWrapper(scorer); DisiWrapper w = new DisiWrapper(scorer, true);
cost += w.cost; cost += w.cost;
allScorers[i++] = w; allScorers[i++] = w;
} }
@ -256,7 +256,7 @@ final class MaxScoreBulkScorer extends BulkScorer {
if (acceptDocs != null && acceptDocs.get(doc) == false) { if (acceptDocs != null && acceptDocs.get(doc) == false) {
continue; continue;
} }
scoreNonEssentialClauses(collector, doc, top.scorer.score(), firstEssentialScorer); scoreNonEssentialClauses(collector, doc, top.scorable.score(), firstEssentialScorer);
} }
top.doc = top.iterator.docID(); top.doc = top.iterator.docID();
essentialQueue.updateTop(); essentialQueue.updateTop();
@ -284,7 +284,7 @@ final class MaxScoreBulkScorer extends BulkScorer {
continue; continue;
} }
double score = lead1.scorer.score(); double score = lead1.scorable.score();
// We specialize handling the second best scorer, which seems to help a bit with performance. // We specialize handling the second best scorer, which seems to help a bit with performance.
// But this is the exact same logic as in the below for loop. // But this is the exact same logic as in the below for loop.
@ -303,7 +303,7 @@ final class MaxScoreBulkScorer extends BulkScorer {
continue; continue;
} }
score += lead2.scorer.score(); score += lead2.scorable.score();
for (int i = allScorers.length - 3; i >= firstRequiredScorer; --i) { for (int i = allScorers.length - 3; i >= firstRequiredScorer; --i) {
if ((float) MathUtil.sumUpperBound(score + maxScoreSums[i], allScorers.length) if ((float) MathUtil.sumUpperBound(score + maxScoreSums[i], allScorers.length)
@ -321,7 +321,7 @@ final class MaxScoreBulkScorer extends BulkScorer {
lead1.doc = lead1.iterator.advance(Math.min(w.doc, max)); lead1.doc = lead1.iterator.advance(Math.min(w.doc, max));
continue outer; continue outer;
} }
score += w.scorer.score(); score += w.scorable.score();
} }
scoreNonEssentialClauses(collector, lead1.doc, score, firstRequiredScorer); scoreNonEssentialClauses(collector, lead1.doc, score, firstRequiredScorer);
@ -342,7 +342,7 @@ final class MaxScoreBulkScorer extends BulkScorer {
if (acceptDocs == null || acceptDocs.get(doc)) { if (acceptDocs == null || acceptDocs.get(doc)) {
final int i = doc - innerWindowMin; final int i = doc - innerWindowMin;
windowMatches[i >>> 6] |= 1L << i; windowMatches[i >>> 6] |= 1L << i;
windowScores[i] += top.scorer.score(); windowScores[i] += top.scorable.score();
} }
} }
top.doc = top.iterator.docID(); top.doc = top.iterator.docID();
@ -439,7 +439,7 @@ final class MaxScoreBulkScorer extends BulkScorer {
scorer.doc = scorer.iterator.advance(doc); scorer.doc = scorer.iterator.advance(doc);
} }
if (scorer.doc == doc) { if (scorer.doc == doc) {
score += scorer.scorer.score(); score += scorer.scorable.score();
} }
} }

View File

@ -113,10 +113,10 @@ final class MultiTermQueryConstantScoreBlendedWrapper<Q extends MultiTermQuery>
DisiPriorityQueue subs = new DisiPriorityQueue(highFrequencyTerms.size() + 1); DisiPriorityQueue subs = new DisiPriorityQueue(highFrequencyTerms.size() + 1);
for (DocIdSetIterator disi : highFrequencyTerms) { for (DocIdSetIterator disi : highFrequencyTerms) {
Scorer s = wrapWithDummyScorer(this, disi); Scorer s = wrapWithDummyScorer(this, disi);
subs.add(new DisiWrapper(s)); subs.add(new DisiWrapper(s, false));
} }
Scorer s = wrapWithDummyScorer(this, otherTerms.build().iterator()); Scorer s = wrapWithDummyScorer(this, otherTerms.build().iterator());
subs.add(new DisiWrapper(s)); subs.add(new DisiWrapper(s, false));
return new WeightOrDocIdSetIterator(new DisjunctionDISIApproximation(subs)); return new WeightOrDocIdSetIterator(new DisjunctionDISIApproximation(subs));
} }

View File

@ -16,12 +16,48 @@
*/ */
package org.apache.lucene.search; package org.apache.lucene.search;
import java.io.IOException;
import java.util.stream.LongStream; import java.util.stream.LongStream;
import java.util.stream.StreamSupport; import java.util.stream.StreamSupport;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.FeatureField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.ImpactsEnum;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ByteBuffersDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.PriorityQueue; import org.apache.lucene.util.PriorityQueue;
/** Util class for Scorer related methods */ /** Util class for Scorer related methods */
class ScorerUtil { class ScorerUtil {
private static final Class<?> DEFAULT_IMPACTS_ENUM_CLASS;
static {
try (Directory dir = new ByteBuffersDirectory();
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig())) {
Document doc = new Document();
doc.add(new FeatureField("field", "value", 1f));
w.addDocument(doc);
try (DirectoryReader reader = DirectoryReader.open(w)) {
LeafReader leafReader = reader.leaves().get(0).reader();
TermsEnum te = leafReader.terms("field").iterator();
if (te.seekExact(new BytesRef("value")) == false) {
throw new Error();
}
ImpactsEnum ie = te.impacts(PostingsEnum.FREQS);
DEFAULT_IMPACTS_ENUM_CLASS = ie.getClass();
}
} catch (IOException e) {
throw new Error(e);
}
}
static long costWithMinShouldMatch(LongStream costs, int numScorers, int minShouldMatch) { static long costWithMinShouldMatch(LongStream costs, int numScorers, int minShouldMatch) {
// the idea here is the following: a boolean query c1,c2,...cn with minShouldMatch=m // the idea here is the following: a boolean query c1,c2,...cn with minShouldMatch=m
// could be rewritten to: // could be rewritten to:
@ -46,4 +82,30 @@ class ScorerUtil {
costs.forEach(pq::insertWithOverflow); costs.forEach(pq::insertWithOverflow);
return StreamSupport.stream(pq.spliterator(), false).mapToLong(Number::longValue).sum(); return StreamSupport.stream(pq.spliterator(), false).mapToLong(Number::longValue).sum();
} }
/**
* Optimize a {@link DocIdSetIterator} for the case when it is likely implemented via an {@link
* ImpactsEnum}. This return method only has 2 possible return types, which helps make sure that
* calls to {@link DocIdSetIterator#nextDoc()} and {@link DocIdSetIterator#advance(int)} are
* bimorphic at most and candidate for inlining.
*/
static DocIdSetIterator likelyImpactsEnum(DocIdSetIterator it) {
if (it.getClass() != DEFAULT_IMPACTS_ENUM_CLASS
&& it.getClass() != FilterDocIdSetIterator.class) {
it = new FilterDocIdSetIterator(it);
}
return it;
}
/**
* Optimize a {@link Scorable} for the case when it is likely implemented via a {@link
* TermScorer}. This return method only has 2 possible return types, which helps make sure that
* calls to {@link Scorable#score()} are bimorphic at most and candidate for inlining.
*/
static Scorable likelyTermScorer(Scorable scorable) {
if (scorable.getClass() != TermScorer.class && scorable.getClass() != FilterScorable.class) {
scorable = new FilterScorable(scorable);
}
return scorable;
}
} }

View File

@ -646,7 +646,7 @@ public final class SynonymQuery extends Query {
final float boost; final float boost;
DisiWrapperFreq(Scorer scorer, float boost) { DisiWrapperFreq(Scorer scorer, float boost) {
super(scorer); super(scorer, false);
this.pe = (PostingsEnum) scorer.iterator(); this.pe = (PostingsEnum) scorer.iterator();
this.boost = boost; this.boost = boost;
} }

View File

@ -196,7 +196,12 @@ final class WANDScorer extends Scorer {
} }
for (Scorer scorer : scorers) { for (Scorer scorer : scorers) {
addUnpositionedLead(new DisiWrapper(scorer)); // Ideally we would pass true when scoreMode == TOP_SCORES and false otherwise, but this would
// break the optimization as there could then be 3 different impls of DocIdSetIterator
// (ImpactsEnum, PostingsEnum and <Else>). So we pass true to favor disjunctions sorted by
// descending score as opposed to non-scoring disjunctions whose minShouldMatch is greater
// than 1.
addUnpositionedLead(new DisiWrapper(scorer, true));
} }
this.cost = this.cost =
@ -221,7 +226,7 @@ final class WANDScorer extends Scorer {
List<Float> leadScores = new ArrayList<>(); List<Float> leadScores = new ArrayList<>();
for (DisiWrapper w = lead; w != null; w = w.next) { for (DisiWrapper w = lead; w != null; w = w.next) {
assert w.doc == doc; assert w.doc == doc;
leadScores.add(w.scorer.score()); leadScores.add(w.scorable.score());
} }
// Make sure to recompute the sum in the same order to get the same floating point rounding // Make sure to recompute the sum in the same order to get the same floating point rounding
// errors. // errors.
@ -370,7 +375,7 @@ final class WANDScorer extends Scorer {
this.lead = lead; this.lead = lead;
freq += 1; freq += 1;
if (scoreMode == ScoreMode.TOP_SCORES) { if (scoreMode == ScoreMode.TOP_SCORES) {
leadScore += lead.scorer.score(); leadScore += lead.scorable.score();
} }
} }
@ -522,7 +527,7 @@ final class WANDScorer extends Scorer {
lead.next = null; lead.next = null;
freq = 1; freq = 1;
if (scoreMode == ScoreMode.TOP_SCORES) { if (scoreMode == ScoreMode.TOP_SCORES) {
leadScore = lead.scorer.score(); leadScore = lead.scorable.score();
} }
while (head.size() > 0 && head.top().doc == doc) { while (head.size() > 0 && head.top().doc == doc) {
addLead(head.pop()); addLead(head.pop());
@ -553,7 +558,7 @@ final class WANDScorer extends Scorer {
if (scoreMode != ScoreMode.TOP_SCORES) { if (scoreMode != ScoreMode.TOP_SCORES) {
// With TOP_SCORES, the score was already computed on the fly. // With TOP_SCORES, the score was already computed on the fly.
for (DisiWrapper s = lead; s != null; s = s.next) { for (DisiWrapper s = lead; s != null; s = s.next) {
leadScore += s.scorer.score(); leadScore += s.scorable.score();
} }
} }
return (float) leadScore; return (float) leadScore;

View File

@ -18,6 +18,7 @@ package org.apache.lucene.store;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.util.Optional;
import org.apache.lucene.codecs.CompoundFormat; import org.apache.lucene.codecs.CompoundFormat;
/** /**
@ -234,4 +235,19 @@ public abstract class IndexInput extends DataInput implements Closeable {
* <p>The default implementation is a no-op. * <p>The default implementation is a no-op.
*/ */
public void updateReadAdvice(ReadAdvice readAdvice) throws IOException {} public void updateReadAdvice(ReadAdvice readAdvice) throws IOException {}
/**
* Returns a hint whether all the contents of this input are resident in physical memory. It's a
* hint because the operating system may have paged out some of the data by the time this method
* returns. If the optional is true, then it's likely that the contents of this input are resident
* in physical memory. A value of false does not imply that the contents are not resident in
* physical memory. An empty optional is returned if it is not possible to determine.
*
* <p>This runs in linear time with the {@link #length()} of this input / page size.
*
* <p>The default implementation returns an empty optional.
*/
public Optional<Boolean> isLoaded() {
return Optional.empty();
}
} }

View File

@ -17,6 +17,7 @@
package org.apache.lucene.store; package org.apache.lucene.store;
import java.io.IOException; import java.io.IOException;
import java.util.Optional;
import org.apache.lucene.util.BitUtil; // javadocs import org.apache.lucene.util.BitUtil; // javadocs
/** /**
@ -77,4 +78,13 @@ public interface RandomAccessInput {
* @see IndexInput#prefetch * @see IndexInput#prefetch
*/ */
default void prefetch(long offset, long length) throws IOException {} default void prefetch(long offset, long length) throws IOException {}
/**
* Returns a hint whether all the contents of this input are resident in physical memory.
*
* @see IndexInput#isLoaded()
*/
default Optional<Boolean> isLoaded() {
return Optional.empty();
}
} }

View File

@ -30,6 +30,7 @@ import static org.apache.lucene.util.fst.FST.NON_FINAL_END_NODE;
import static org.apache.lucene.util.fst.FST.getNumPresenceBytes; import static org.apache.lucene.util.fst.FST.getNumPresenceBytes;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays;
import java.util.Objects; import java.util.Objects;
import org.apache.lucene.store.ByteArrayDataOutput; import org.apache.lucene.store.ByteArrayDataOutput;
import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.DataOutput;
@ -869,14 +870,14 @@ public class FSTCompiler<T> {
} }
// compare shared prefix length // compare shared prefix length
int pos1 = 0; int pos = 0;
int pos2 = input.offset; if (lastInput.length() > 0) {
final int pos1Stop = Math.min(lastInput.length(), input.length); int mismatch =
while (pos1 < pos1Stop && lastInput.intAt(pos1) == input.ints[pos2]) { Arrays.mismatch(
pos1++; lastInput.ints(), 0, lastInput.length(), input.ints, input.offset, input.length);
pos2++; pos += mismatch == -1 ? lastInput.length() : mismatch;
} }
final int prefixLenPlus1 = pos1 + 1; final int prefixLenPlus1 = pos + 1;
if (frontier.length < input.length + 1) { if (frontier.length < input.length + 1) {
final UnCompiledNode<T>[] next = ArrayUtil.grow(frontier, input.length + 1); final UnCompiledNode<T>[] next = ArrayUtil.grow(frontier, input.length + 1);

View File

@ -75,6 +75,9 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport {
} }
} }
// cached vector sizes for smaller method bodies
private static final int FLOAT_SPECIES_LENGTH = FLOAT_SPECIES.length();
// the way FMA should work! if available use it, otherwise fall back to mul/add // the way FMA should work! if available use it, otherwise fall back to mul/add
private static FloatVector fma(FloatVector a, FloatVector b, FloatVector c) { private static FloatVector fma(FloatVector a, FloatVector b, FloatVector c) {
if (Constants.HAS_FAST_VECTOR_FMA) { if (Constants.HAS_FAST_VECTOR_FMA) {
@ -99,7 +102,7 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport {
float res = 0; float res = 0;
// if the array size is large (> 2x platform vector size), its worth the overhead to vectorize // if the array size is large (> 2x platform vector size), its worth the overhead to vectorize
if (a.length > 2 * FLOAT_SPECIES.length()) { if (a.length > 2 * FLOAT_SPECIES_LENGTH) {
i += FLOAT_SPECIES.loopBound(a.length); i += FLOAT_SPECIES.loopBound(a.length);
res += dotProductBody(a, b, i); res += dotProductBody(a, b, i);
} }
@ -120,30 +123,33 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport {
FloatVector acc2 = FloatVector.zero(FLOAT_SPECIES); FloatVector acc2 = FloatVector.zero(FLOAT_SPECIES);
FloatVector acc3 = FloatVector.zero(FLOAT_SPECIES); FloatVector acc3 = FloatVector.zero(FLOAT_SPECIES);
FloatVector acc4 = FloatVector.zero(FLOAT_SPECIES); FloatVector acc4 = FloatVector.zero(FLOAT_SPECIES);
int unrolledLimit = limit - 3 * FLOAT_SPECIES.length(); final int unrolledLimit = limit - 3 * FLOAT_SPECIES_LENGTH;
for (; i < unrolledLimit; i += 4 * FLOAT_SPECIES.length()) { for (; i < unrolledLimit; i += 4 * FLOAT_SPECIES_LENGTH) {
// one // one
FloatVector va = FloatVector.fromArray(FLOAT_SPECIES, a, i); FloatVector va = FloatVector.fromArray(FLOAT_SPECIES, a, i);
FloatVector vb = FloatVector.fromArray(FLOAT_SPECIES, b, i); FloatVector vb = FloatVector.fromArray(FLOAT_SPECIES, b, i);
acc1 = fma(va, vb, acc1); acc1 = fma(va, vb, acc1);
// two // two
FloatVector vc = FloatVector.fromArray(FLOAT_SPECIES, a, i + FLOAT_SPECIES.length()); final int i2 = i + FLOAT_SPECIES_LENGTH;
FloatVector vd = FloatVector.fromArray(FLOAT_SPECIES, b, i + FLOAT_SPECIES.length()); FloatVector vc = FloatVector.fromArray(FLOAT_SPECIES, a, i2);
FloatVector vd = FloatVector.fromArray(FLOAT_SPECIES, b, i2);
acc2 = fma(vc, vd, acc2); acc2 = fma(vc, vd, acc2);
// three // three
FloatVector ve = FloatVector.fromArray(FLOAT_SPECIES, a, i + 2 * FLOAT_SPECIES.length()); final int i3 = i2 + FLOAT_SPECIES_LENGTH;
FloatVector vf = FloatVector.fromArray(FLOAT_SPECIES, b, i + 2 * FLOAT_SPECIES.length()); FloatVector ve = FloatVector.fromArray(FLOAT_SPECIES, a, i3);
FloatVector vf = FloatVector.fromArray(FLOAT_SPECIES, b, i3);
acc3 = fma(ve, vf, acc3); acc3 = fma(ve, vf, acc3);
// four // four
FloatVector vg = FloatVector.fromArray(FLOAT_SPECIES, a, i + 3 * FLOAT_SPECIES.length()); final int i4 = i3 + FLOAT_SPECIES_LENGTH;
FloatVector vh = FloatVector.fromArray(FLOAT_SPECIES, b, i + 3 * FLOAT_SPECIES.length()); FloatVector vg = FloatVector.fromArray(FLOAT_SPECIES, a, i4);
FloatVector vh = FloatVector.fromArray(FLOAT_SPECIES, b, i4);
acc4 = fma(vg, vh, acc4); acc4 = fma(vg, vh, acc4);
} }
// vector tail: less scalar computations for unaligned sizes, esp with big vector sizes // vector tail: less scalar computations for unaligned sizes, esp with big vector sizes
for (; i < limit; i += FLOAT_SPECIES.length()) { for (; i < limit; i += FLOAT_SPECIES_LENGTH) {
FloatVector va = FloatVector.fromArray(FLOAT_SPECIES, a, i); FloatVector va = FloatVector.fromArray(FLOAT_SPECIES, a, i);
FloatVector vb = FloatVector.fromArray(FLOAT_SPECIES, b, i); FloatVector vb = FloatVector.fromArray(FLOAT_SPECIES, b, i);
acc1 = fma(va, vb, acc1); acc1 = fma(va, vb, acc1);
@ -162,7 +168,7 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport {
float norm2 = 0; float norm2 = 0;
// if the array size is large (> 2x platform vector size), its worth the overhead to vectorize // if the array size is large (> 2x platform vector size), its worth the overhead to vectorize
if (a.length > 2 * FLOAT_SPECIES.length()) { if (a.length > 2 * FLOAT_SPECIES_LENGTH) {
i += FLOAT_SPECIES.loopBound(a.length); i += FLOAT_SPECIES.loopBound(a.length);
float[] ret = cosineBody(a, b, i); float[] ret = cosineBody(a, b, i);
sum += ret[0]; sum += ret[0];
@ -190,8 +196,8 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport {
FloatVector norm1_2 = FloatVector.zero(FLOAT_SPECIES); FloatVector norm1_2 = FloatVector.zero(FLOAT_SPECIES);
FloatVector norm2_1 = FloatVector.zero(FLOAT_SPECIES); FloatVector norm2_1 = FloatVector.zero(FLOAT_SPECIES);
FloatVector norm2_2 = FloatVector.zero(FLOAT_SPECIES); FloatVector norm2_2 = FloatVector.zero(FLOAT_SPECIES);
int unrolledLimit = limit - FLOAT_SPECIES.length(); final int unrolledLimit = limit - FLOAT_SPECIES_LENGTH;
for (; i < unrolledLimit; i += 2 * FLOAT_SPECIES.length()) { for (; i < unrolledLimit; i += 2 * FLOAT_SPECIES_LENGTH) {
// one // one
FloatVector va = FloatVector.fromArray(FLOAT_SPECIES, a, i); FloatVector va = FloatVector.fromArray(FLOAT_SPECIES, a, i);
FloatVector vb = FloatVector.fromArray(FLOAT_SPECIES, b, i); FloatVector vb = FloatVector.fromArray(FLOAT_SPECIES, b, i);
@ -200,14 +206,15 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport {
norm2_1 = fma(vb, vb, norm2_1); norm2_1 = fma(vb, vb, norm2_1);
// two // two
FloatVector vc = FloatVector.fromArray(FLOAT_SPECIES, a, i + FLOAT_SPECIES.length()); final int i2 = i + FLOAT_SPECIES_LENGTH;
FloatVector vd = FloatVector.fromArray(FLOAT_SPECIES, b, i + FLOAT_SPECIES.length()); FloatVector vc = FloatVector.fromArray(FLOAT_SPECIES, a, i2);
FloatVector vd = FloatVector.fromArray(FLOAT_SPECIES, b, i2);
sum2 = fma(vc, vd, sum2); sum2 = fma(vc, vd, sum2);
norm1_2 = fma(vc, vc, norm1_2); norm1_2 = fma(vc, vc, norm1_2);
norm2_2 = fma(vd, vd, norm2_2); norm2_2 = fma(vd, vd, norm2_2);
} }
// vector tail: less scalar computations for unaligned sizes, esp with big vector sizes // vector tail: less scalar computations for unaligned sizes, esp with big vector sizes
for (; i < limit; i += FLOAT_SPECIES.length()) { for (; i < limit; i += FLOAT_SPECIES_LENGTH) {
FloatVector va = FloatVector.fromArray(FLOAT_SPECIES, a, i); FloatVector va = FloatVector.fromArray(FLOAT_SPECIES, a, i);
FloatVector vb = FloatVector.fromArray(FLOAT_SPECIES, b, i); FloatVector vb = FloatVector.fromArray(FLOAT_SPECIES, b, i);
sum1 = fma(va, vb, sum1); sum1 = fma(va, vb, sum1);
@ -227,7 +234,7 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport {
float res = 0; float res = 0;
// if the array size is large (> 2x platform vector size), its worth the overhead to vectorize // if the array size is large (> 2x platform vector size), its worth the overhead to vectorize
if (a.length > 2 * FLOAT_SPECIES.length()) { if (a.length > 2 * FLOAT_SPECIES_LENGTH) {
i += FLOAT_SPECIES.loopBound(a.length); i += FLOAT_SPECIES.loopBound(a.length);
res += squareDistanceBody(a, b, i); res += squareDistanceBody(a, b, i);
} }
@ -240,6 +247,12 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport {
return res; return res;
} }
/** helper: returns fma(a.sub(b), a.sub(b), c) */
private static FloatVector square(FloatVector a, FloatVector b, FloatVector c) {
FloatVector diff = a.sub(b);
return fma(diff, diff, c);
}
/** vectorized square distance body */ /** vectorized square distance body */
private float squareDistanceBody(float[] a, float[] b, int limit) { private float squareDistanceBody(float[] a, float[] b, int limit) {
int i = 0; int i = 0;
@ -249,38 +262,36 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport {
FloatVector acc2 = FloatVector.zero(FLOAT_SPECIES); FloatVector acc2 = FloatVector.zero(FLOAT_SPECIES);
FloatVector acc3 = FloatVector.zero(FLOAT_SPECIES); FloatVector acc3 = FloatVector.zero(FLOAT_SPECIES);
FloatVector acc4 = FloatVector.zero(FLOAT_SPECIES); FloatVector acc4 = FloatVector.zero(FLOAT_SPECIES);
int unrolledLimit = limit - 3 * FLOAT_SPECIES.length(); final int unrolledLimit = limit - 3 * FLOAT_SPECIES_LENGTH;
for (; i < unrolledLimit; i += 4 * FLOAT_SPECIES.length()) { for (; i < unrolledLimit; i += 4 * FLOAT_SPECIES_LENGTH) {
// one // one
FloatVector va = FloatVector.fromArray(FLOAT_SPECIES, a, i); FloatVector va = FloatVector.fromArray(FLOAT_SPECIES, a, i);
FloatVector vb = FloatVector.fromArray(FLOAT_SPECIES, b, i); FloatVector vb = FloatVector.fromArray(FLOAT_SPECIES, b, i);
FloatVector diff1 = va.sub(vb); acc1 = square(va, vb, acc1);
acc1 = fma(diff1, diff1, acc1);
// two // two
FloatVector vc = FloatVector.fromArray(FLOAT_SPECIES, a, i + FLOAT_SPECIES.length()); final int i2 = i + FLOAT_SPECIES_LENGTH;
FloatVector vd = FloatVector.fromArray(FLOAT_SPECIES, b, i + FLOAT_SPECIES.length()); FloatVector vc = FloatVector.fromArray(FLOAT_SPECIES, a, i2);
FloatVector diff2 = vc.sub(vd); FloatVector vd = FloatVector.fromArray(FLOAT_SPECIES, b, i2);
acc2 = fma(diff2, diff2, acc2); acc2 = square(vc, vd, acc2);
// three // three
FloatVector ve = FloatVector.fromArray(FLOAT_SPECIES, a, i + 2 * FLOAT_SPECIES.length()); final int i3 = i2 + FLOAT_SPECIES_LENGTH;
FloatVector vf = FloatVector.fromArray(FLOAT_SPECIES, b, i + 2 * FLOAT_SPECIES.length()); FloatVector ve = FloatVector.fromArray(FLOAT_SPECIES, a, i3);
FloatVector diff3 = ve.sub(vf); FloatVector vf = FloatVector.fromArray(FLOAT_SPECIES, b, i3);
acc3 = fma(diff3, diff3, acc3); acc3 = square(ve, vf, acc3);
// four // four
FloatVector vg = FloatVector.fromArray(FLOAT_SPECIES, a, i + 3 * FLOAT_SPECIES.length()); final int i4 = i3 + FLOAT_SPECIES_LENGTH;
FloatVector vh = FloatVector.fromArray(FLOAT_SPECIES, b, i + 3 * FLOAT_SPECIES.length()); FloatVector vg = FloatVector.fromArray(FLOAT_SPECIES, a, i4);
FloatVector diff4 = vg.sub(vh); FloatVector vh = FloatVector.fromArray(FLOAT_SPECIES, b, i4);
acc4 = fma(diff4, diff4, acc4); acc4 = square(vg, vh, acc4);
} }
// vector tail: less scalar computations for unaligned sizes, esp with big vector sizes // vector tail: less scalar computations for unaligned sizes, esp with big vector sizes
for (; i < limit; i += FLOAT_SPECIES.length()) { for (; i < limit; i += FLOAT_SPECIES_LENGTH) {
FloatVector va = FloatVector.fromArray(FLOAT_SPECIES, a, i); FloatVector va = FloatVector.fromArray(FLOAT_SPECIES, a, i);
FloatVector vb = FloatVector.fromArray(FLOAT_SPECIES, b, i); FloatVector vb = FloatVector.fromArray(FLOAT_SPECIES, b, i);
FloatVector diff = va.sub(vb); acc1 = square(va, vb, acc1);
acc1 = fma(diff, diff, acc1);
} }
// reduce // reduce
FloatVector res1 = acc1.add(acc2); FloatVector res1 = acc1.add(acc2);

View File

@ -420,6 +420,16 @@ abstract class MemorySegmentIndexInput extends IndexInput
} }
} }
@Override
public Optional<Boolean> isLoaded() {
for (MemorySegment seg : segments) {
if (seg.isLoaded() == false) {
return Optional.of(Boolean.FALSE);
}
}
return Optional.of(Boolean.TRUE);
}
@Override @Override
public byte readByte(long pos) throws IOException { public byte readByte(long pos) throws IOException {
try { try {

View File

@ -401,14 +401,12 @@ public class TestBooleanRewrites extends LuceneTestCase {
bq = bq =
new BooleanQuery.Builder() new BooleanQuery.Builder()
.setMinimumNumberShouldMatch(random().nextInt(5))
.add(new TermQuery(new Term("foo", "bar")), Occur.MUST) .add(new TermQuery(new Term("foo", "bar")), Occur.MUST)
.add(new TermQuery(new Term("foo", "baz")), Occur.MUST) .add(new TermQuery(new Term("foo", "baz")), Occur.MUST)
.add(new MatchAllDocsQuery(), Occur.FILTER) .add(new MatchAllDocsQuery(), Occur.FILTER)
.build(); .build();
Query expected = Query expected =
new BooleanQuery.Builder() new BooleanQuery.Builder()
.setMinimumNumberShouldMatch(bq.getMinimumNumberShouldMatch())
.add(new TermQuery(new Term("foo", "bar")), Occur.MUST) .add(new TermQuery(new Term("foo", "bar")), Occur.MUST)
.add(new TermQuery(new Term("foo", "baz")), Occur.MUST) .add(new TermQuery(new Term("foo", "baz")), Occur.MUST)
.build(); .build();
@ -476,7 +474,22 @@ public class TestBooleanRewrites extends LuceneTestCase {
Query query = randomBooleanQuery(random()); Query query = randomBooleanQuery(random());
final TopDocs td1 = searcher1.search(query, 100); final TopDocs td1 = searcher1.search(query, 100);
final TopDocs td2 = searcher2.search(query, 100); final TopDocs td2 = searcher2.search(query, 100);
assertEquals(td1, td2); try {
assertEquals(td1, td2);
} catch (AssertionError e) {
System.out.println(query);
Query rewritten = query;
do {
query = rewritten;
rewritten = query.rewrite(searcher1);
System.out.println(rewritten);
TopDocs tdx = searcher2.search(rewritten, 100);
if (td2.totalHits.value() != tdx.totalHits.value()) {
System.out.println("Bad");
}
} while (query != rewritten);
throw e;
}
} }
searcher1.getIndexReader().close(); searcher1.getIndexReader().close();

View File

@ -70,7 +70,7 @@ public class TestDisiPriorityQueue extends LuceneTestCase {
private static DisiWrapper wrapper(DocIdSetIterator iterator) throws IOException { private static DisiWrapper wrapper(DocIdSetIterator iterator) throws IOException {
Query q = new DummyQuery(iterator); Query q = new DummyQuery(iterator);
Scorer s = q.createWeight(null, ScoreMode.COMPLETE_NO_SCORES, 1.0f).scorer(null); Scorer s = q.createWeight(null, ScoreMode.COMPLETE_NO_SCORES, 1.0f).scorer(null);
return new DisiWrapper(s); return new DisiWrapper(s, random().nextBoolean());
} }
private static DocIdSetIterator randomDisi(Random r) { private static DocIdSetIterator randomDisi(Random r) {

View File

@ -422,15 +422,17 @@ public final class CombinedFieldQuery extends Query implements Accountable {
} }
private static class WeightedDisiWrapper extends DisiWrapper { private static class WeightedDisiWrapper extends DisiWrapper {
final PostingsEnum postingsEnum;
final float weight; final float weight;
WeightedDisiWrapper(Scorer scorer, float weight) { WeightedDisiWrapper(Scorer scorer, float weight) {
super(scorer); super(scorer, false);
this.weight = weight; this.weight = weight;
this.postingsEnum = (PostingsEnum) scorer.iterator();
} }
float freq() throws IOException { float freq() throws IOException {
return weight * ((PostingsEnum) iterator).freq(); return weight * postingsEnum.freq();
} }
} }

View File

@ -54,7 +54,7 @@ final class CoveringScorer extends Scorer {
subScorers = new DisiPriorityQueue(scorers.size()); subScorers = new DisiPriorityQueue(scorers.size());
for (Scorer scorer : scorers) { for (Scorer scorer : scorers) {
subScorers.add(new DisiWrapper(scorer)); subScorers.add(new DisiWrapper(scorer, false));
} }
this.cost = scorers.stream().map(Scorer::iterator).mapToLong(DocIdSetIterator::cost).sum(); this.cost = scorers.stream().map(Scorer::iterator).mapToLong(DocIdSetIterator::cost).sum();
@ -210,7 +210,7 @@ final class CoveringScorer extends Scorer {
setTopListAndFreqIfNecessary(); setTopListAndFreqIfNecessary();
double score = 0; double score = 0;
for (DisiWrapper w = topList; w != null; w = w.next) { for (DisiWrapper w = topList; w != null; w = w.next) {
score += w.scorer.score(); score += w.scorable.score();
} }
return (float) score; return (float) score;
} }

View File

@ -51,9 +51,11 @@ import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.FilterDirectory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.MMapDirectory;
import org.apache.lucene.store.RandomAccessInput; import org.apache.lucene.store.RandomAccessInput;
import org.apache.lucene.store.ReadAdvice; import org.apache.lucene.store.ReadAdvice;
import org.apache.lucene.tests.mockfile.ExtrasFS; import org.apache.lucene.tests.mockfile.ExtrasFS;
@ -1636,4 +1638,44 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
} }
} }
} }
public void testIsLoaded() throws IOException {
testIsLoaded(0);
}
public void testIsLoadedOnSlice() throws IOException {
testIsLoaded(TestUtil.nextInt(random(), 1, 1024));
}
private void testIsLoaded(int startOffset) throws IOException {
try (Directory dir = getDirectory(createTempDir())) {
if (FilterDirectory.unwrap(dir) instanceof MMapDirectory mMapDirectory) {
mMapDirectory.setPreload(MMapDirectory.ALL_FILES);
}
final int totalLength = startOffset + TestUtil.nextInt(random(), 16384, 65536);
byte[] arr = new byte[totalLength];
random().nextBytes(arr);
try (IndexOutput out = dir.createOutput("temp.bin", IOContext.DEFAULT)) {
out.writeBytes(arr, arr.length);
}
try (IndexInput orig = dir.openInput("temp.bin", IOContext.DEFAULT)) {
IndexInput in;
if (startOffset == 0) {
in = orig.clone();
} else {
in = orig.slice("slice", startOffset, totalLength - startOffset);
}
var loaded = in.isLoaded();
if (FilterDirectory.unwrap(dir) instanceof MMapDirectory
// direct IO wraps MMap but does not support isLoaded
&& !(dir.getClass().getName().contains("DirectIO"))) {
assertTrue(loaded.isPresent());
assertTrue(loaded.get());
} else {
assertFalse(loaded.isPresent());
}
}
}
}
} }

View File

@ -19,6 +19,7 @@ package org.apache.lucene.tests.store;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.util.Map; import java.util.Map;
import java.util.Optional;
import java.util.Set; import java.util.Set;
import org.apache.lucene.internal.tests.TestSecrets; import org.apache.lucene.internal.tests.TestSecrets;
import org.apache.lucene.store.FilterIndexInput; import org.apache.lucene.store.FilterIndexInput;
@ -184,6 +185,13 @@ public class MockIndexInputWrapper extends FilterIndexInput {
in.prefetch(offset, length); in.prefetch(offset, length);
} }
@Override
public Optional<Boolean> isLoaded() {
ensureOpen();
ensureAccessible();
return in.isLoaded();
}
@Override @Override
public void updateReadAdvice(ReadAdvice readAdvice) throws IOException { public void updateReadAdvice(ReadAdvice readAdvice) throws IOException {
ensureOpen(); ensureOpen();

View File

@ -17,6 +17,7 @@
package org.apache.lucene.tests.store; package org.apache.lucene.tests.store;
import java.io.IOException; import java.io.IOException;
import java.util.Optional;
import java.util.concurrent.atomic.LongAdder; import java.util.concurrent.atomic.LongAdder;
import org.apache.lucene.internal.hppc.LongHashSet; import org.apache.lucene.internal.hppc.LongHashSet;
import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.ChecksumIndexInput;
@ -206,5 +207,10 @@ public class SerialIOCountingDirectory extends FilterDirectory {
IndexInput clone = in.clone(); IndexInput clone = in.clone();
return new SerializedIOCountingIndexInput(clone, readAdvice, sliceOffset, sliceLength); return new SerializedIOCountingIndexInput(clone, readAdvice, sliceOffset, sliceLength);
} }
@Override
public Optional<Boolean> isLoaded() {
return in.isLoaded();
}
} }
} }