LUCENE-6553: Simplify how live docs are applied.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1687524 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Adrien Grand 2015-06-25 13:59:19 +00:00
parent f11fe3513d
commit d671dd8d89
219 changed files with 1434 additions and 1838 deletions

View File

@ -42,6 +42,7 @@ API Changes
BooleanQuery with one MUST clause for the query, and one FILTER clause for
the filter. (Adrien Grand)
======================= Lucene 5.3.0 =======================
New Features
@ -132,6 +133,10 @@ API Changes
be replaced with a BooleanQuery which handle the query as a MUST clause and
the filter as a FILTER clause. (Adrien Grand)
* LUCENE-6553: The postings, spans and scorer APIs no longer take an acceptDocs
parameter. Live docs are now always checked on top of these APIs.
(Adrien Grand)
Bug fixes
* LUCENE-6500: ParallelCompositeReader did not always call
@ -225,6 +230,12 @@ Build
* LUCENE-6567: Simplify payload checking in SpanPayloadCheckQuery (Alan
Woodward)
Changes in Backwards Compatibility Policy
* LUCENE-6553: The iterator returned by the LeafReader.postings method now
always includes deleted docs, so you have to check for deleted documents on
top of the iterator. (Adrien Grand)
======================= Lucene 5.2.1 =======================
Bug Fixes

View File

@ -101,7 +101,6 @@ public class TestKeywordAnalyzer extends BaseTokenStreamTestCase {
reader,
"partnum",
new BytesRef("Q36"),
MultiFields.getLiveDocs(reader),
null,
0);
assertTrue(td.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@ -109,7 +108,6 @@ public class TestKeywordAnalyzer extends BaseTokenStreamTestCase {
reader,
"partnum",
new BytesRef("Q37"),
MultiFields.getLiveDocs(reader),
null,
0);
assertTrue(td.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);

View File

@ -111,7 +111,7 @@ public class TestTeeSinkTokenFilter extends BaseTokenStreamTestCase {
TermsEnum termsEnum = vector.iterator();
termsEnum.next();
assertEquals(2, termsEnum.totalTermFreq());
PostingsEnum positions = termsEnum.postings(null, null, PostingsEnum.ALL);
PostingsEnum positions = termsEnum.postings(null, PostingsEnum.ALL);
assertTrue(positions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(2, positions.freq());
positions.nextPosition();

View File

@ -300,7 +300,6 @@ public class TestClassicAnalyzer extends BaseTokenStreamTestCase {
// Make sure position is still incremented when
// massive term is skipped:
PostingsEnum tps = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
"content",
new BytesRef("another"));
assertTrue(tps.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);

View File

@ -991,7 +991,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
// should be found exactly
assertEquals(TermsEnum.SeekStatus.FOUND,
terms.seekCeil(aaaTerm));
assertEquals(35, countDocs(TestUtil.docs(random(), terms, null, null, PostingsEnum.NONE)));
assertEquals(35, countDocs(TestUtil.docs(random(), terms, null, PostingsEnum.NONE)));
assertNull(terms.next());
// should hit end of field
@ -1003,12 +1003,12 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
assertEquals(TermsEnum.SeekStatus.NOT_FOUND,
terms.seekCeil(new BytesRef("a")));
assertTrue(terms.term().bytesEquals(aaaTerm));
assertEquals(35, countDocs(TestUtil.docs(random(), terms, null, null, PostingsEnum.NONE)));
assertEquals(35, countDocs(TestUtil.docs(random(), terms, null, PostingsEnum.NONE)));
assertNull(terms.next());
assertEquals(TermsEnum.SeekStatus.FOUND,
terms.seekCeil(aaaTerm));
assertEquals(35, countDocs(TestUtil.docs(random(), terms, null, null, PostingsEnum.NONE)));
assertEquals(35, countDocs(TestUtil.docs(random(), terms, null, PostingsEnum.NONE)));
assertNull(terms.next());
r.close();

View File

@ -499,7 +499,7 @@ public class TestPerfTasksLogic extends BenchmarkTestCase {
TermsEnum termsEnum = terms.iterator();
PostingsEnum docs = null;
while(termsEnum.next() != null) {
docs = TestUtil.docs(random(), termsEnum, MultiFields.getLiveDocs(reader), docs, PostingsEnum.FREQS);
docs = TestUtil.docs(random(), termsEnum, docs, PostingsEnum.FREQS);
while(docs.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
totalTokenCount2 += docs.freq();
}

View File

@ -651,11 +651,11 @@ public class BlockTermsReader extends FieldsProducer {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
//System.out.println("BTR.docs this=" + this);
decodeMetaData();
//System.out.println("BTR.docs: state.docFreq=" + state.docFreq);
return postingsReader.postings(fieldInfo, state, liveDocs, reuse, flags);
return postingsReader.postings(fieldInfo, state, reuse, flags);
}
@Override

View File

@ -202,9 +202,9 @@ final class OrdsIntersectTermsEnum extends TermsEnum {
}
@Override
public PostingsEnum postings(Bits skipDocs, PostingsEnum reuse, int flags) throws IOException {
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
currentFrame.decodeMetaData();
return fr.parent.postingsReader.postings(fr.fieldInfo, currentFrame.termState, skipDocs, reuse, flags);
return fr.parent.postingsReader.postings(fr.fieldInfo, currentFrame.termState, reuse, flags);
}
private int getState() {

View File

@ -923,7 +923,7 @@ public final class OrdsSegmentTermsEnum extends TermsEnum {
}
@Override
public PostingsEnum postings(Bits skipDocs, PostingsEnum reuse, int flags) throws IOException {
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
assert !eof;
//if (DEBUG) {
//System.out.println("BTTR.docs seg=" + segment);
@ -932,7 +932,7 @@ public final class OrdsSegmentTermsEnum extends TermsEnum {
//if (DEBUG) {
//System.out.println(" state=" + currentFrame.state);
//}
return fr.parent.postingsReader.postings(fr.fieldInfo, currentFrame.state, skipDocs, reuse, flags);
return fr.parent.postingsReader.postings(fr.fieldInfo, currentFrame.state, reuse, flags);
}
@Override

View File

@ -369,9 +369,9 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags)
public PostingsEnum postings(PostingsEnum reuse, int flags)
throws IOException {
return delegate().postings(liveDocs, reuse, flags);
return delegate().postings(reuse, flags);
}
}
@ -455,7 +455,7 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
bloomFilters.put(fieldInfo, bloomFilter);
}
// Make sure there's at least one doc for this term:
postingsEnum = termsEnum.postings(null, postingsEnum, 0);
postingsEnum = termsEnum.postings(postingsEnum, 0);
if (postingsEnum.nextDoc() != PostingsEnum.NO_MORE_DOCS) {
bloomFilter.addValue(term);
}

View File

@ -43,7 +43,6 @@ import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.automaton.CompiledAutomaton;
@ -357,9 +356,9 @@ public final class DirectPostingsFormat extends PostingsFormat {
termOffsets[count+1] = termOffset;
if (hasPos) {
docsAndPositionsEnum = termsEnum.postings(null, docsAndPositionsEnum, PostingsEnum.ALL);
docsAndPositionsEnum = termsEnum.postings(docsAndPositionsEnum, PostingsEnum.ALL);
} else {
postingsEnum = termsEnum.postings(null, postingsEnum);
postingsEnum = termsEnum.postings(postingsEnum);
}
final TermAndSkip ent;
@ -848,7 +847,7 @@ public final class DirectPostingsFormat extends PostingsFormat {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
// TODO: implement reuse
// it's hairy!
@ -862,11 +861,8 @@ public final class DirectPostingsFormat extends PostingsFormat {
LowFreqDocsEnumNoTF docsEnum;
if (reuse instanceof LowFreqDocsEnumNoTF) {
docsEnum = (LowFreqDocsEnumNoTF) reuse;
if (!docsEnum.canReuse(liveDocs)) {
docsEnum = new LowFreqDocsEnumNoTF(liveDocs);
}
} else {
docsEnum = new LowFreqDocsEnumNoTF(liveDocs);
docsEnum = new LowFreqDocsEnumNoTF();
}
return docsEnum.reset(postings);
@ -875,23 +871,20 @@ public final class DirectPostingsFormat extends PostingsFormat {
LowFreqDocsEnumNoPos docsEnum;
if (reuse instanceof LowFreqDocsEnumNoPos) {
docsEnum = (LowFreqDocsEnumNoPos) reuse;
if (!docsEnum.canReuse(liveDocs)) {
docsEnum = new LowFreqDocsEnumNoPos(liveDocs);
}
} else {
docsEnum = new LowFreqDocsEnumNoPos(liveDocs);
docsEnum = new LowFreqDocsEnumNoPos();
}
return docsEnum.reset(postings);
}
final byte[] payloads = term.payloads;
return new LowFreqPostingsEnum(liveDocs, hasOffsets, hasPayloads).reset(postings, payloads);
return new LowFreqPostingsEnum(hasOffsets, hasPayloads).reset(postings, payloads);
} else {
final HighFreqTerm term = (HighFreqTerm) terms[termOrd];
if (hasPos == false) {
return new HighFreqDocsEnum(liveDocs).reset(term.docIDs, term.freqs);
return new HighFreqDocsEnum().reset(term.docIDs, term.freqs);
} else {
return new HighFreqPostingsEnum(liveDocs, hasOffsets).reset(term.docIDs, term.freqs, term.positions, term.payloads);
return new HighFreqPostingsEnum(hasOffsets).reset(term.docIDs, term.freqs, term.positions, term.payloads);
}
}
}
@ -912,11 +905,8 @@ public final class DirectPostingsFormat extends PostingsFormat {
LowFreqDocsEnum docsEnum;
if (reuse instanceof LowFreqDocsEnum) {
docsEnum = (LowFreqDocsEnum) reuse;
if (!docsEnum.canReuse(liveDocs, posLen)) {
docsEnum = new LowFreqDocsEnum(liveDocs, posLen);
}
} else {
docsEnum = new LowFreqDocsEnum(liveDocs, posLen);
docsEnum = new LowFreqDocsEnum( posLen);
}
return docsEnum.reset(postings);
@ -924,11 +914,8 @@ public final class DirectPostingsFormat extends PostingsFormat {
LowFreqDocsEnumNoPos docsEnum;
if (reuse instanceof LowFreqDocsEnumNoPos) {
docsEnum = (LowFreqDocsEnumNoPos) reuse;
if (!docsEnum.canReuse(liveDocs)) {
docsEnum = new LowFreqDocsEnumNoPos(liveDocs);
}
} else {
docsEnum = new LowFreqDocsEnumNoPos(liveDocs);
docsEnum = new LowFreqDocsEnumNoPos();
}
return docsEnum.reset(postings);
@ -937,11 +924,8 @@ public final class DirectPostingsFormat extends PostingsFormat {
LowFreqDocsEnumNoTF docsEnum;
if (reuse instanceof LowFreqDocsEnumNoTF) {
docsEnum = (LowFreqDocsEnumNoTF) reuse;
if (!docsEnum.canReuse(liveDocs)) {
docsEnum = new LowFreqDocsEnumNoTF(liveDocs);
}
} else {
docsEnum = new LowFreqDocsEnumNoTF(liveDocs);
docsEnum = new LowFreqDocsEnumNoTF();
}
return docsEnum.reset(postings);
@ -952,11 +936,8 @@ public final class DirectPostingsFormat extends PostingsFormat {
HighFreqDocsEnum docsEnum;
if (reuse instanceof HighFreqDocsEnum) {
docsEnum = (HighFreqDocsEnum) reuse;
if (!docsEnum.canReuse(liveDocs)) {
docsEnum = new HighFreqDocsEnum(liveDocs);
}
} else {
docsEnum = new HighFreqDocsEnum(liveDocs);
docsEnum = new HighFreqDocsEnum();
}
//System.out.println(" DE for term=" + new BytesRef(terms[termOrd].term).utf8ToString() + ": " + term.docIDs.length + " docs");
@ -1471,7 +1452,7 @@ public final class DirectPostingsFormat extends PostingsFormat {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) {
public PostingsEnum postings(PostingsEnum reuse, int flags) {
// TODO: implement reuse
// it's hairy!
@ -1481,10 +1462,10 @@ public final class DirectPostingsFormat extends PostingsFormat {
final LowFreqTerm term = ((LowFreqTerm) terms[termOrd]);
final int[] postings = term.postings;
final byte[] payloads = term.payloads;
return new LowFreqPostingsEnum(liveDocs, hasOffsets, hasPayloads).reset(postings, payloads);
return new LowFreqPostingsEnum(hasOffsets, hasPayloads).reset(postings, payloads);
} else {
final HighFreqTerm term = (HighFreqTerm) terms[termOrd];
return new HighFreqPostingsEnum(liveDocs, hasOffsets).reset(term.docIDs, term.freqs, term.positions, term.payloads);
return new HighFreqPostingsEnum(hasOffsets).reset(term.docIDs, term.freqs, term.positions, term.payloads);
}
}
@ -1501,17 +1482,17 @@ public final class DirectPostingsFormat extends PostingsFormat {
if (hasPayloads) {
posLen++;
}
return new LowFreqDocsEnum(liveDocs, posLen).reset(postings);
return new LowFreqDocsEnum(posLen).reset(postings);
} else {
return new LowFreqDocsEnumNoPos(liveDocs).reset(postings);
return new LowFreqDocsEnumNoPos().reset(postings);
}
} else {
return new LowFreqDocsEnumNoTF(liveDocs).reset(postings);
return new LowFreqDocsEnumNoTF().reset(postings);
}
} else {
final HighFreqTerm term = (HighFreqTerm) terms[termOrd];
// System.out.println("DE for term=" + new BytesRef(terms[termOrd].term).utf8ToString() + ": " + term.docIDs.length + " docs");
return new HighFreqDocsEnum(liveDocs).reset(term.docIDs, term.freqs);
return new HighFreqDocsEnum().reset(term.docIDs, term.freqs);
}
}
@ -1530,17 +1511,8 @@ public final class DirectPostingsFormat extends PostingsFormat {
// Docs only:
private final static class LowFreqDocsEnumNoTF extends PostingsEnum {
private int[] postings;
private final Bits liveDocs;
private int upto;
public LowFreqDocsEnumNoTF(Bits liveDocs) {
this.liveDocs = liveDocs;
}
public boolean canReuse(Bits liveDocs) {
return liveDocs == this.liveDocs;
}
public PostingsEnum reset(int[] postings) {
this.postings = postings;
upto = -1;
@ -1552,17 +1524,8 @@ public final class DirectPostingsFormat extends PostingsFormat {
@Override
public int nextDoc() {
upto++;
if (liveDocs == null) {
if (upto < postings.length) {
return postings[upto];
}
} else {
while (upto < postings.length) {
if (liveDocs.get(postings[upto])) {
return postings[upto];
}
upto++;
}
if (upto < postings.length) {
return postings[upto];
}
return NO_MORE_DOCS;
}
@ -1619,16 +1582,9 @@ public final class DirectPostingsFormat extends PostingsFormat {
// Docs + freqs:
private final static class LowFreqDocsEnumNoPos extends PostingsEnum {
private int[] postings;
private final Bits liveDocs;
private int upto;
public LowFreqDocsEnumNoPos(Bits liveDocs) {
this.liveDocs = liveDocs;
}
public boolean canReuse(Bits liveDocs) {
return liveDocs == this.liveDocs;
}
public LowFreqDocsEnumNoPos() {}
public PostingsEnum reset(int[] postings) {
this.postings = postings;
@ -1640,17 +1596,8 @@ public final class DirectPostingsFormat extends PostingsFormat {
@Override
public int nextDoc() {
upto += 2;
if (liveDocs == null) {
if (upto < postings.length) {
return postings[upto];
}
} else {
while (upto < postings.length) {
if (liveDocs.get(postings[upto])) {
return postings[upto];
}
upto += 2;
}
if (upto < postings.length) {
return postings[upto];
}
return NO_MORE_DOCS;
}
@ -1707,23 +1654,17 @@ public final class DirectPostingsFormat extends PostingsFormat {
// Docs + freqs + positions/offets:
private final static class LowFreqDocsEnum extends PostingsEnum {
private int[] postings;
private final Bits liveDocs;
private final int posMult;
private int upto;
private int freq;
public LowFreqDocsEnum(Bits liveDocs, int posMult) {
this.liveDocs = liveDocs;
public LowFreqDocsEnum(int posMult) {
this.posMult = posMult;
// if (DEBUG) {
// System.out.println("LowFreqDE: posMult=" + posMult);
// }
}
public boolean canReuse(Bits liveDocs, int posMult) {
return liveDocs == this.liveDocs && posMult == this.posMult;
}
public PostingsEnum reset(int[] postings) {
this.postings = postings;
upto = -2;
@ -1738,21 +1679,10 @@ public final class DirectPostingsFormat extends PostingsFormat {
// if (DEBUG) {
// System.out.println(" nextDoc freq=" + freq + " upto=" + upto + " vs " + postings.length);
// }
if (liveDocs == null) {
if (upto < postings.length) {
freq = postings[upto+1];
assert freq > 0;
return postings[upto];
}
} else {
while (upto < postings.length) {
freq = postings[upto+1];
assert freq > 0;
if (liveDocs.get(postings[upto])) {
return postings[upto];
}
upto += 2 + freq*posMult;
}
if (upto < postings.length) {
freq = postings[upto+1];
assert freq > 0;
return postings[upto];
}
return NO_MORE_DOCS;
}
@ -1811,7 +1741,6 @@ public final class DirectPostingsFormat extends PostingsFormat {
private final static class LowFreqPostingsEnum extends PostingsEnum {
private int[] postings;
private final Bits liveDocs;
private final int posMult;
private final boolean hasOffsets;
private final boolean hasPayloads;
@ -1828,8 +1757,7 @@ public final class DirectPostingsFormat extends PostingsFormat {
private int payloadLength;
private byte[] payloadBytes;
public LowFreqPostingsEnum(Bits liveDocs, boolean hasOffsets, boolean hasPayloads) {
this.liveDocs = liveDocs;
public LowFreqPostingsEnum(boolean hasOffsets, boolean hasPayloads) {
this.hasOffsets = hasOffsets;
this.hasPayloads = hasPayloads;
if (hasOffsets) {
@ -1873,33 +1801,11 @@ public final class DirectPostingsFormat extends PostingsFormat {
upto += posMult * skipPositions;
}
if (liveDocs == null) {
if (upto < postings.length) {
docID = postings[upto++];
freq = postings[upto++];
skipPositions = freq;
return docID;
}
} else {
while(upto < postings.length) {
docID = postings[upto++];
freq = postings[upto++];
if (liveDocs.get(docID)) {
skipPositions = freq;
return docID;
}
if (hasPayloads) {
for(int i=0;i<freq;i++) {
upto++;
if (hasOffsets) {
upto += 2;
}
payloadOffset += postings[upto++];
}
} else {
upto += posMult * freq;
}
}
if (upto < postings.length) {
docID = postings[upto++];
freq = postings[upto++];
skipPositions = freq;
return docID;
}
return docID = NO_MORE_DOCS;
@ -1970,17 +1876,10 @@ public final class DirectPostingsFormat extends PostingsFormat {
private final static class HighFreqDocsEnum extends PostingsEnum {
private int[] docIDs;
private int[] freqs;
private final Bits liveDocs;
private int upto;
private int docID = -1;
public HighFreqDocsEnum(Bits liveDocs) {
this.liveDocs = liveDocs;
}
public boolean canReuse(Bits liveDocs) {
return liveDocs == this.liveDocs;
}
public HighFreqDocsEnum() {}
public int[] getDocIDs() {
return docIDs;
@ -2000,18 +1899,9 @@ public final class DirectPostingsFormat extends PostingsFormat {
@Override
public int nextDoc() {
upto++;
if (liveDocs == null) {
try {
return docID = docIDs[upto];
} catch (ArrayIndexOutOfBoundsException e) {
}
} else {
while (upto < docIDs.length) {
if (liveDocs.get(docIDs[upto])) {
return docID = docIDs[upto];
}
upto++;
}
try {
return docID = docIDs[upto];
} catch (ArrayIndexOutOfBoundsException e) {
}
return docID = NO_MORE_DOCS;
}
@ -2121,14 +2011,6 @@ public final class DirectPostingsFormat extends PostingsFormat {
//System.out.println(" end upto=" + upto + " docID=" + (upto >= docIDs.length ? NO_MORE_DOCS : docIDs[upto]));
if (liveDocs != null) {
while (upto < docIDs.length) {
if (liveDocs.get(docIDs[upto])) {
break;
}
upto++;
}
}
if (upto == docIDs.length) {
//System.out.println(" return END");
return docID = NO_MORE_DOCS;
@ -2170,7 +2052,6 @@ public final class DirectPostingsFormat extends PostingsFormat {
private int[] freqs;
private int[][] positions;
private byte[][][] payloads;
private final Bits liveDocs;
private final boolean hasOffsets;
private final int posJump;
private int upto;
@ -2178,8 +2059,7 @@ public final class DirectPostingsFormat extends PostingsFormat {
private int posUpto;
private int[] curPositions;
public HighFreqPostingsEnum(Bits liveDocs, boolean hasOffsets) {
this.liveDocs = liveDocs;
public HighFreqPostingsEnum(boolean hasOffsets) {
this.hasOffsets = hasOffsets;
posJump = hasOffsets ? 3 : 1;
}
@ -2196,10 +2076,6 @@ public final class DirectPostingsFormat extends PostingsFormat {
return posJump;
}
public Bits getLiveDocs() {
return liveDocs;
}
public PostingsEnum reset(int[] docIDs, int[] freqs, int[][] positions, byte[][][] payloads) {
this.docIDs = docIDs;
this.freqs = freqs;
@ -2212,21 +2088,10 @@ public final class DirectPostingsFormat extends PostingsFormat {
@Override
public int nextDoc() {
upto++;
if (liveDocs == null) {
if (upto < docIDs.length) {
posUpto = -posJump;
curPositions = positions[upto];
return docID = docIDs[upto];
}
} else {
while (upto < docIDs.length) {
if (liveDocs.get(docIDs[upto])) {
posUpto = -posJump;
curPositions = positions[upto];
return docID = docIDs[upto];
}
upto++;
}
if (upto < docIDs.length) {
posUpto = -posJump;
curPositions = positions[upto];
return docID = docIDs[upto];
}
return docID = NO_MORE_DOCS;
@ -2360,14 +2225,6 @@ public final class DirectPostingsFormat extends PostingsFormat {
//System.out.println(" end upto=" + upto + " docID=" + (upto >= docIDs.length ? NO_MORE_DOCS : docIDs[upto]));
if (liveDocs != null) {
while (upto < docIDs.length) {
if (liveDocs.get(docIDs[upto])) {
break;
}
upto++;
}
}
if (upto == docIDs.length) {
//System.out.println(" return END");
return docID = NO_MORE_DOCS;

View File

@ -427,9 +427,9 @@ public class FSTOrdTermsReader extends FieldsProducer {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
decodeMetaData();
return postingsReader.postings(fieldInfo, state, liveDocs, reuse, flags);
return postingsReader.postings(fieldInfo, state, reuse, flags);
}
// TODO: this can be achieved by making use of Util.getByOutput()

View File

@ -46,7 +46,6 @@ import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.IOUtils;
@ -290,9 +289,9 @@ public class FSTTermsReader extends FieldsProducer {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
decodeMetaData();
return postingsReader.postings(fieldInfo, state, liveDocs, reuse, flags);
return postingsReader.postings(fieldInfo, state, reuse, flags);
}
@Override

View File

@ -892,7 +892,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
throw new UnsupportedOperationException();
}

View File

@ -353,10 +353,10 @@ public final class MemoryPostingsFormat extends PostingsFormat {
termsWriter.postingsWriter.reset();
if (writePositions) {
posEnum = termsEnum.postings(null, posEnum, enumFlags);
posEnum = termsEnum.postings(posEnum, enumFlags);
postingsEnum = posEnum;
} else {
postingsEnum = termsEnum.postings(null, postingsEnum, enumFlags);
postingsEnum = termsEnum.postings(postingsEnum, enumFlags);
posEnum = null;
}
@ -433,7 +433,6 @@ public final class MemoryPostingsFormat extends PostingsFormat {
private byte[] buffer = new byte[16];
private final ByteArrayDataInput in = new ByteArrayDataInput(buffer);
private Bits liveDocs;
private int docUpto;
private int docID = -1;
private int accum;
@ -450,14 +449,13 @@ public final class MemoryPostingsFormat extends PostingsFormat {
return indexOptions == this.indexOptions && storePayloads == this.storePayloads;
}
public FSTDocsEnum reset(BytesRef bufferIn, Bits liveDocs, int numDocs) {
public FSTDocsEnum reset(BytesRef bufferIn, int numDocs) {
assert numDocs > 0;
if (buffer.length < bufferIn.length) {
buffer = ArrayUtil.grow(buffer, bufferIn.length);
}
in.reset(buffer, 0, bufferIn.length);
System.arraycopy(bufferIn.bytes, bufferIn.offset, buffer, 0, bufferIn.length);
this.liveDocs = liveDocs;
docID = -1;
accum = 0;
docUpto = 0;
@ -469,62 +467,58 @@ public final class MemoryPostingsFormat extends PostingsFormat {
@Override
public int nextDoc() {
while(true) {
//System.out.println(" nextDoc cycle docUpto=" + docUpto + " numDocs=" + numDocs + " fp=" + in.getPosition() + " this=" + this);
if (docUpto == numDocs) {
// System.out.println(" END");
return docID = NO_MORE_DOCS;
}
docUpto++;
if (indexOptions == IndexOptions.DOCS) {
accum += in.readVInt();
//System.out.println(" nextDoc cycle docUpto=" + docUpto + " numDocs=" + numDocs + " fp=" + in.getPosition() + " this=" + this);
if (docUpto == numDocs) {
// System.out.println(" END");
return docID = NO_MORE_DOCS;
}
docUpto++;
if (indexOptions == IndexOptions.DOCS) {
accum += in.readVInt();
} else {
final int code = in.readVInt();
accum += code >>> 1;
//System.out.println(" docID=" + accum + " code=" + code);
if ((code & 1) != 0) {
freq = 1;
} else {
final int code = in.readVInt();
accum += code >>> 1;
//System.out.println(" docID=" + accum + " code=" + code);
if ((code & 1) != 0) {
freq = 1;
} else {
freq = in.readVInt();
assert freq > 0;
}
freq = in.readVInt();
assert freq > 0;
}
if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
// Skip positions/payloads
for(int posUpto=0;posUpto<freq;posUpto++) {
if (!storePayloads) {
in.readVInt();
} else {
final int posCode = in.readVInt();
if ((posCode & 1) != 0) {
payloadLen = in.readVInt();
}
in.skipBytes(payloadLen);
}
}
} else if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) {
// Skip positions/offsets/payloads
for(int posUpto=0;posUpto<freq;posUpto++) {
int posCode = in.readVInt();
if (storePayloads && ((posCode & 1) != 0)) {
if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
// Skip positions/payloads
for(int posUpto=0;posUpto<freq;posUpto++) {
if (!storePayloads) {
in.readVInt();
} else {
final int posCode = in.readVInt();
if ((posCode & 1) != 0) {
payloadLen = in.readVInt();
}
if ((in.readVInt() & 1) != 0) {
// new offset length
in.readVInt();
}
if (storePayloads) {
in.skipBytes(payloadLen);
}
in.skipBytes(payloadLen);
}
}
} else if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) {
// Skip positions/offsets/payloads
for(int posUpto=0;posUpto<freq;posUpto++) {
int posCode = in.readVInt();
if (storePayloads && ((posCode & 1) != 0)) {
payloadLen = in.readVInt();
}
if ((in.readVInt() & 1) != 0) {
// new offset length
in.readVInt();
}
if (storePayloads) {
in.skipBytes(payloadLen);
}
}
}
if (liveDocs == null || liveDocs.get(accum)) {
//System.out.println(" return docID=" + accum + " freq=" + freq);
return (docID = accum);
}
}
//System.out.println(" return docID=" + accum + " freq=" + freq);
return (docID = accum);
}
@Override
@ -577,7 +571,6 @@ public final class MemoryPostingsFormat extends PostingsFormat {
private byte[] buffer = new byte[16];
private final ByteArrayDataInput in = new ByteArrayDataInput(buffer);
private Bits liveDocs;
private int docUpto;
private int docID = -1;
private int accum;
@ -601,7 +594,7 @@ public final class MemoryPostingsFormat extends PostingsFormat {
return storePayloads == this.storePayloads && storeOffsets == this.storeOffsets;
}
public FSTPostingsEnum reset(BytesRef bufferIn, Bits liveDocs, int numDocs) {
public FSTPostingsEnum reset(BytesRef bufferIn, int numDocs) {
assert numDocs > 0;
// System.out.println("D&P reset bytes this=" + this);
@ -614,7 +607,6 @@ public final class MemoryPostingsFormat extends PostingsFormat {
}
in.reset(buffer, 0, bufferIn.length - bufferIn.offset);
System.arraycopy(bufferIn.bytes, bufferIn.offset, buffer, 0, bufferIn.length);
this.liveDocs = liveDocs;
docID = -1;
accum = 0;
docUpto = 0;
@ -649,37 +641,11 @@ public final class MemoryPostingsFormat extends PostingsFormat {
assert freq > 0;
}
if (liveDocs == null || liveDocs.get(accum)) {
pos = 0;
startOffset = storeOffsets ? 0 : -1;
posPending = freq;
//System.out.println(" return docID=" + accum + " freq=" + freq);
return (docID = accum);
}
// Skip positions
for(int posUpto=0;posUpto<freq;posUpto++) {
if (!storePayloads) {
in.readVInt();
} else {
final int skipCode = in.readVInt();
if ((skipCode & 1) != 0) {
payloadLength = in.readVInt();
//System.out.println(" new payloadLen=" + payloadLength);
}
}
if (storeOffsets) {
if ((in.readVInt() & 1) != 0) {
// new offset length
offsetLength = in.readVInt();
}
}
if (storePayloads) {
in.skipBytes(payloadLength);
}
}
pos = 0;
startOffset = storeOffsets ? 0 : -1;
posPending = freq;
//System.out.println(" return docID=" + accum + " freq=" + freq);
return (docID = accum);
}
}
@ -827,7 +793,7 @@ public final class MemoryPostingsFormat extends PostingsFormat {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) {
public PostingsEnum postings(PostingsEnum reuse, int flags) {
// TODO: the logic of which enum impl to choose should be refactored to be simpler...
boolean hasPositions = field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
@ -844,7 +810,7 @@ public final class MemoryPostingsFormat extends PostingsFormat {
}
}
//System.out.println("D&P reset this=" + this);
return docsAndPositionsEnum.reset(postingsSpare, liveDocs, docFreq);
return docsAndPositionsEnum.reset(postingsSpare, docFreq);
}
decodeMetaData();
@ -858,7 +824,7 @@ public final class MemoryPostingsFormat extends PostingsFormat {
docsEnum = new FSTDocsEnum(field.getIndexOptions(), field.hasPayloads());
}
}
return docsEnum.reset(this.postingsSpare, liveDocs, docFreq);
return docsEnum.reset(this.postingsSpare, docFreq);
}
@Override

View File

@ -207,7 +207,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
boolean hasPositions = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
if (hasPositions && PostingsEnum.featureRequested(flags, PostingsEnum.POSITIONS)) {
@ -218,7 +218,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
} else {
docsAndPositionsEnum = new SimpleTextPostingsEnum();
}
return docsAndPositionsEnum.reset(docsStart, liveDocs, indexOptions, docFreq);
return docsAndPositionsEnum.reset(docsStart, indexOptions, docFreq);
}
@ -228,7 +228,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
} else {
docsEnum = new SimpleTextDocsEnum();
}
return docsEnum.reset(docsStart, liveDocs, indexOptions == IndexOptions.DOCS, docFreq);
return docsEnum.reset(docsStart, indexOptions == IndexOptions.DOCS, docFreq);
}
}
@ -239,7 +239,6 @@ class SimpleTextFieldsReader extends FieldsProducer {
private boolean omitTF;
private int docID = -1;
private int tf;
private Bits liveDocs;
private final BytesRefBuilder scratch = new BytesRefBuilder();
private final CharsRefBuilder scratchUTF16 = new CharsRefBuilder();
private int cost;
@ -253,8 +252,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
return in == inStart;
}
public SimpleTextDocsEnum reset(long fp, Bits liveDocs, boolean omitTF, int docFreq) throws IOException {
this.liveDocs = liveDocs;
public SimpleTextDocsEnum reset(long fp, boolean omitTF, int docFreq) throws IOException {
in.seek(fp);
this.omitTF = omitTF;
docID = -1;
@ -304,7 +302,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
final long lineStart = in.getFilePointer();
SimpleTextUtil.readLine(in, scratch);
if (StringHelper.startsWith(scratch.get(), DOC)) {
if (!first && (liveDocs == null || liveDocs.get(docID))) {
if (!first) {
in.seek(lineStart);
if (!omitTF) {
tf = termFreq;
@ -328,7 +326,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
// skip
} else {
assert StringHelper.startsWith(scratch.get(), TERM) || StringHelper.startsWith(scratch.get(), FIELD) || StringHelper.startsWith(scratch.get(), END): "scratch=" + scratch.get().utf8ToString();
if (!first && (liveDocs == null || liveDocs.get(docID))) {
if (!first) {
in.seek(lineStart);
if (!omitTF) {
tf = termFreq;
@ -357,7 +355,6 @@ class SimpleTextFieldsReader extends FieldsProducer {
private final IndexInput in;
private int docID = -1;
private int tf;
private Bits liveDocs;
private final BytesRefBuilder scratch = new BytesRefBuilder();
private final BytesRefBuilder scratch2 = new BytesRefBuilder();
private final CharsRefBuilder scratchUTF16 = new CharsRefBuilder();
@ -380,8 +377,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
return in == inStart;
}
public SimpleTextPostingsEnum reset(long fp, Bits liveDocs, IndexOptions indexOptions, int docFreq) {
this.liveDocs = liveDocs;
public SimpleTextPostingsEnum reset(long fp, IndexOptions indexOptions, int docFreq) {
nextDocStart = fp;
docID = -1;
readPositions = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
@ -414,7 +410,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
SimpleTextUtil.readLine(in, scratch);
//System.out.println("NEXT DOC: " + scratch.utf8ToString());
if (StringHelper.startsWith(scratch.get(), DOC)) {
if (!first && (liveDocs == null || liveDocs.get(docID))) {
if (!first) {
nextDocStart = lineStart;
in.seek(posStart);
return docID;
@ -437,7 +433,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
// skip
} else {
assert StringHelper.startsWith(scratch.get(), TERM) || StringHelper.startsWith(scratch.get(), FIELD) || StringHelper.startsWith(scratch.get(), END);
if (!first && (liveDocs == null || liveDocs.get(docID))) {
if (!first) {
nextDocStart = lineStart;
in.seek(posStart);
return docID;

View File

@ -101,7 +101,7 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
break;
}
postingsEnum = termsEnum.postings(null, postingsEnum, flags);
postingsEnum = termsEnum.postings(postingsEnum, flags);
assert postingsEnum != null: "termsEnum=" + termsEnum + " hasPos=" + hasPositions + " flags=" + flags;

View File

@ -387,21 +387,21 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
if (PostingsEnum.featureRequested(flags, PostingsEnum.POSITIONS)) {
SimpleTVPostings postings = current.getValue();
if (postings.positions != null || postings.startOffsets != null) {
// TODO: reuse
SimpleTVPostingsEnum e = new SimpleTVPostingsEnum();
e.reset(liveDocs, postings.positions, postings.startOffsets, postings.endOffsets, postings.payloads);
e.reset(postings.positions, postings.startOffsets, postings.endOffsets, postings.payloads);
return e;
}
}
// TODO: reuse
SimpleTVDocsEnum e = new SimpleTVDocsEnum();
e.reset(liveDocs, PostingsEnum.featureRequested(flags, PostingsEnum.FREQS) == false ? 1 : current.getValue().freq);
e.reset(PostingsEnum.featureRequested(flags, PostingsEnum.FREQS) == false ? 1 : current.getValue().freq);
return e;
}
@ -412,7 +412,6 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
private boolean didNext;
private int doc = -1;
private int freq;
private Bits liveDocs;
@Override
public int freq() throws IOException {
@ -447,7 +446,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
@Override
public int nextDoc() {
if (!didNext && (liveDocs == null || liveDocs.get(0))) {
if (!didNext) {
didNext = true;
return (doc = 0);
} else {
@ -460,8 +459,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
return slowAdvance(target);
}
public void reset(Bits liveDocs, int freq) {
this.liveDocs = liveDocs;
public void reset(int freq) {
this.freq = freq;
this.doc = -1;
didNext = false;
@ -477,7 +475,6 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
private boolean didNext;
private int doc = -1;
private int nextPos;
private Bits liveDocs;
private int[] positions;
private BytesRef[] payloads;
private int[] startOffsets;
@ -500,7 +497,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
@Override
public int nextDoc() {
if (!didNext && (liveDocs == null || liveDocs.get(0))) {
if (!didNext) {
didNext = true;
return (doc = 0);
} else {
@ -513,8 +510,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
return slowAdvance(target);
}
public void reset(Bits liveDocs, int[] positions, int[] startOffsets, int[] endOffsets, BytesRef payloads[]) {
this.liveDocs = liveDocs;
public void reset(int[] positions, int[] startOffsets, int[] endOffsets, BytesRef payloads[]) {
this.positions = positions;
this.startOffsets = startOffsets;
this.endOffsets = endOffsets;

View File

@ -55,7 +55,6 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MultiTermQuery;
import org.apache.lucene.search.PrefixQuery;
import org.apache.lucene.search.TermRangeQuery;
import org.apache.lucene.store.Directory;
@ -169,7 +168,7 @@ public class TestAutoPrefixTerms extends LuceneTestCase {
System.out.println(" got term=" + te.term().utf8ToString());
}
verifier.sawTerm(te.term());
postingsEnum = te.postings(null, postingsEnum);
postingsEnum = te.postings(postingsEnum);
int docID;
while ((docID = postingsEnum.nextDoc()) != PostingsEnum.NO_MORE_DOCS) {
long v = docValues.get(docID);
@ -296,7 +295,7 @@ public class TestAutoPrefixTerms extends LuceneTestCase {
System.out.println(" got term=" + te.term() + " docFreq=" + te.docFreq());
}
verifier.sawTerm(te.term());
postingsEnum = te.postings(null, postingsEnum);
postingsEnum = te.postings(postingsEnum);
int docID;
while ((docID = postingsEnum.nextDoc()) != PostingsEnum.NO_MORE_DOCS) {
long v = docValues.get(docID);
@ -415,7 +414,7 @@ public class TestAutoPrefixTerms extends LuceneTestCase {
System.out.println("TEST: got term=" + te.term().utf8ToString() + " docFreq=" + te.docFreq());
}
verifier.sawTerm(te.term());
postingsEnum = te.postings(null, postingsEnum);
postingsEnum = te.postings(postingsEnum);
int docID;
while ((docID = postingsEnum.nextDoc()) != PostingsEnum.NO_MORE_DOCS) {
assertTrue("prefixBR=" + prefixBR + " docBR=" + docValues.get(docID), StringHelper.startsWith(docValues.get(docID), prefixBR));
@ -491,7 +490,7 @@ public class TestAutoPrefixTerms extends LuceneTestCase {
//TermsEnum te = terms.intersect(new CompiledAutomaton(a, true, false), null);
while (te.next() != null) {
verifier.sawTerm(te.term());
postingsEnum = te.postings(null, postingsEnum);
postingsEnum = te.postings(postingsEnum);
int docID;
while ((docID = postingsEnum.nextDoc()) != PostingsEnum.NO_MORE_DOCS) {
// The auto-prefix terms should never "overlap" one another, so we should only ever see a given docID one time:

View File

@ -26,7 +26,6 @@ import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
/** The core terms dictionaries (BlockTermsReader,
* BlockTreeTermsReader) interact with a single instance
@ -65,7 +64,7 @@ public abstract class PostingsReaderBase implements Closeable, Accountable {
/** Must fully consume state, since after this call that
* TermState may be reused. */
public abstract PostingsEnum postings(FieldInfo fieldInfo, BlockTermState state, Bits skipDocs, PostingsEnum reuse, int flags) throws IOException;
public abstract PostingsEnum postings(FieldInfo fieldInfo, BlockTermState state, PostingsEnum reuse, int flags) throws IOException;
/**
* Checks consistency of this reader.

View File

@ -119,7 +119,7 @@ public abstract class PushPostingsWriterBase extends PostingsWriterBase {
@Override
public final BlockTermState writeTerm(BytesRef term, TermsEnum termsEnum, FixedBitSet docsSeen) throws IOException {
startTerm();
postingsEnum = termsEnum.postings(null, postingsEnum, enumFlags);
postingsEnum = termsEnum.postings(postingsEnum, enumFlags);
assert postingsEnum != null;
int docFreq = 0;

View File

@ -267,7 +267,7 @@ public abstract class TermVectorsWriter implements Closeable {
startTerm(termsEnum.term(), freq);
if (hasPositions || hasOffsets) {
docsAndPositionsEnum = termsEnum.postings(null, docsAndPositionsEnum, PostingsEnum.OFFSETS | PostingsEnum.PAYLOADS);
docsAndPositionsEnum = termsEnum.postings(docsAndPositionsEnum, PostingsEnum.OFFSETS | PostingsEnum.PAYLOADS);
assert docsAndPositionsEnum != null;
final int docID = docsAndPositionsEnum.nextDoc();

View File

@ -21,7 +21,6 @@ import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.BitSet;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
/** Silly stub class, used only when writing an auto-prefix
@ -73,14 +72,11 @@ class BitSetTermsEnum extends TermsEnum {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) {
public PostingsEnum postings(PostingsEnum reuse, int flags) {
if (flags != PostingsEnum.NONE) {
// We only work with DOCS_ONLY fields
return null;
}
if (liveDocs != null) {
throw new IllegalArgumentException("cannot handle live docs");
}
postingsEnum.reset();
return postingsEnum;
}

View File

@ -474,7 +474,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
while (prefixTermsEnum.next() != null) {
//System.out.println(" got term=" + prefixTermsEnum.term().utf8ToString());
//termCount++;
prefixDocsEnum = prefixTermsEnum.postings(null, prefixDocsEnum, 0);
prefixDocsEnum = prefixTermsEnum.postings(prefixDocsEnum, 0);
//System.out.println(" " + prefixDocsEnum + " doc=" + prefixDocsEnum.docID());
prefixDocs.or(prefixDocsEnum);
}

View File

@ -25,7 +25,6 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.StringHelper;
@ -231,9 +230,9 @@ final class IntersectTermsEnum extends TermsEnum {
}
@Override
public PostingsEnum postings(Bits skipDocs, PostingsEnum reuse, int flags) throws IOException {
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
currentFrame.decodeMetaData();
return fr.parent.postingsReader.postings(fr.fieldInfo, currentFrame.termState, skipDocs, reuse, flags);
return fr.parent.postingsReader.postings(fr.fieldInfo, currentFrame.termState, reuse, flags);
}
private int getState() {

View File

@ -27,7 +27,6 @@ import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.RamUsageEstimator;
@ -993,7 +992,7 @@ final class SegmentTermsEnum extends TermsEnum {
}
@Override
public PostingsEnum postings(Bits skipDocs, PostingsEnum reuse, int flags) throws IOException {
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
assert !eof;
//if (DEBUG) {
//System.out.println("BTTR.docs seg=" + segment);
@ -1002,7 +1001,7 @@ final class SegmentTermsEnum extends TermsEnum {
//if (DEBUG) {
//System.out.println(" state=" + currentFrame.state);
//}
return fr.parent.postingsReader.postings(fr.fieldInfo, currentFrame.state, skipDocs, reuse, flags);
return fr.parent.postingsReader.postings(fr.fieldInfo, currentFrame.state, reuse, flags);
}
@Override

View File

@ -44,7 +44,6 @@ import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LongsRef;
@ -930,7 +929,7 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
}
@Override
public final PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
public final PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
final TVPostingsEnum docsEnum;
if (reuse != null && reuse instanceof TVPostingsEnum) {
docsEnum = (TVPostingsEnum) reuse;
@ -938,7 +937,7 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
docsEnum = new TVPostingsEnum();
}
docsEnum.reset(liveDocs, termFreqs[ord], positionIndex[ord], positions, startOffsets, lengths, payloads, payloadIndex);
docsEnum.reset(termFreqs[ord], positionIndex[ord], positions, startOffsets, lengths, payloads, payloadIndex);
return docsEnum;
}
@ -946,7 +945,6 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
private static class TVPostingsEnum extends PostingsEnum {
private Bits liveDocs;
private int doc = -1;
private int termFreq;
private int positionIndex;
@ -962,10 +960,9 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
payload = new BytesRef();
}
public void reset(Bits liveDocs, int freq, int positionIndex, int[] positions,
public void reset(int freq, int positionIndex, int[] positions,
int[] startOffsets, int[] lengths, BytesRef payloads,
int[] payloadIndex) {
this.liveDocs = liveDocs;
this.termFreq = freq;
this.positionIndex = positionIndex;
this.positions = positions;
@ -1061,7 +1058,7 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
@Override
public int nextDoc() throws IOException {
if (doc == -1 && (liveDocs == null || liveDocs.get(0))) {
if (doc == -1) {
return (doc = 0);
} else {
return (doc = NO_MORE_DOCS);

View File

@ -1122,7 +1122,7 @@ class Lucene50DocValuesProducer extends DocValuesProducer implements Closeable {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
throw new UnsupportedOperationException();
}

View File

@ -32,7 +32,6 @@ import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
@ -192,7 +191,7 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
}
@Override
public PostingsEnum postings(FieldInfo fieldInfo, BlockTermState termState, Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
public PostingsEnum postings(FieldInfo fieldInfo, BlockTermState termState, PostingsEnum reuse, int flags) throws IOException {
boolean indexHasPositions = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
boolean indexHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
@ -208,7 +207,7 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
} else {
docsEnum = new BlockDocsEnum(fieldInfo);
}
return docsEnum.reset(liveDocs, (IntBlockTermState) termState, flags);
return docsEnum.reset((IntBlockTermState) termState, flags);
} else if ((indexHasOffsets == false || PostingsEnum.featureRequested(flags, PostingsEnum.OFFSETS) == false) &&
(indexHasPayloads == false || PostingsEnum.featureRequested(flags, PostingsEnum.PAYLOADS) == false)) {
BlockPostingsEnum docsAndPositionsEnum;
@ -220,7 +219,7 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
} else {
docsAndPositionsEnum = new BlockPostingsEnum(fieldInfo);
}
return docsAndPositionsEnum.reset(liveDocs, (IntBlockTermState) termState);
return docsAndPositionsEnum.reset((IntBlockTermState) termState);
} else {
EverythingEnum everythingEnum;
if (reuse instanceof EverythingEnum) {
@ -231,7 +230,7 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
} else {
everythingEnum = new EverythingEnum(fieldInfo);
}
return everythingEnum.reset(liveDocs, (IntBlockTermState) termState, flags);
return everythingEnum.reset((IntBlockTermState) termState, flags);
}
}
@ -272,8 +271,6 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
// docID for next skip point, we won't use skipper if
// target docID is not larger than this
private int nextSkipDoc;
private Bits liveDocs;
private boolean needsFreq; // true if the caller actually needs frequencies
private int singletonDocID; // docid when there is a single pulsed posting, otherwise -1
@ -295,9 +292,7 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
indexHasPayloads == fieldInfo.hasPayloads();
}
public PostingsEnum reset(Bits liveDocs, IntBlockTermState termState, int flags) throws IOException {
this.liveDocs = liveDocs;
public PostingsEnum reset(IntBlockTermState termState, int flags) throws IOException {
docFreq = termState.docFreq;
totalTermFreq = indexHasFreq ? termState.totalTermFreq : docFreq;
docTermStartFP = termState.docStartFP;
@ -380,26 +375,20 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
@Override
public int nextDoc() throws IOException {
while (true) {
if (docUpto == docFreq) {
return doc = NO_MORE_DOCS;
}
if (docBufferUpto == BLOCK_SIZE) {
refillDocs();
}
accum += docDeltaBuffer[docBufferUpto];
docUpto++;
if (liveDocs == null || liveDocs.get(accum)) {
doc = accum;
freq = freqBuffer[docBufferUpto];
docBufferUpto++;
return doc;
}
docBufferUpto++;
if (docUpto == docFreq) {
return doc = NO_MORE_DOCS;
}
if (docBufferUpto == BLOCK_SIZE) {
refillDocs();
}
accum += docDeltaBuffer[docBufferUpto];
docUpto++;
doc = accum;
freq = freqBuffer[docBufferUpto];
docBufferUpto++;
return doc;
}
@Override
@ -467,14 +456,9 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
}
}
if (liveDocs == null || liveDocs.get(accum)) {
freq = freqBuffer[docBufferUpto];
docBufferUpto++;
return doc = accum;
} else {
docBufferUpto++;
return nextDoc();
}
freq = freqBuffer[docBufferUpto];
docBufferUpto++;
return doc = accum;
}
@Override
@ -544,7 +528,6 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
private int nextSkipDoc;
private Bits liveDocs;
private int singletonDocID; // docid when there is a single pulsed posting, otherwise -1
public BlockPostingsEnum(FieldInfo fieldInfo) throws IOException {
@ -562,9 +545,7 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
indexHasPayloads == fieldInfo.hasPayloads();
}
public PostingsEnum reset(Bits liveDocs, IntBlockTermState termState) throws IOException {
this.liveDocs = liveDocs;
public PostingsEnum reset(IntBlockTermState termState) throws IOException {
docFreq = termState.docFreq;
docTermStartFP = termState.docStartFP;
posTermStartFP = termState.posStartFP;
@ -660,26 +641,22 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
@Override
public int nextDoc() throws IOException {
while (true) {
if (docUpto == docFreq) {
return doc = NO_MORE_DOCS;
}
if (docBufferUpto == BLOCK_SIZE) {
refillDocs();
}
accum += docDeltaBuffer[docBufferUpto];
freq = freqBuffer[docBufferUpto];
posPendingCount += freq;
docBufferUpto++;
docUpto++;
if (liveDocs == null || liveDocs.get(accum)) {
doc = accum;
position = 0;
return doc;
}
if (docUpto == docFreq) {
return doc = NO_MORE_DOCS;
}
if (docBufferUpto == BLOCK_SIZE) {
refillDocs();
}
accum += docDeltaBuffer[docBufferUpto];
freq = freqBuffer[docBufferUpto];
posPendingCount += freq;
docBufferUpto++;
docUpto++;
doc = accum;
position = 0;
return doc;
}
@Override
@ -745,12 +722,8 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
}
}
if (liveDocs == null || liveDocs.get(accum)) {
position = 0;
return doc = accum;
} else {
return nextDoc();
}
position = 0;
return doc = accum;
}
// TODO: in theory we could avoid loading frq block
@ -904,8 +877,6 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
private long skipOffset;
private int nextSkipDoc;
private Bits liveDocs;
private boolean needsOffsets; // true if we actually need offsets
private boolean needsPayloads; // true if we actually need payloads
@ -946,9 +917,7 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
indexHasPayloads == fieldInfo.hasPayloads();
}
public EverythingEnum reset(Bits liveDocs, IntBlockTermState termState, int flags) throws IOException {
this.liveDocs = liveDocs;
public EverythingEnum reset(IntBlockTermState termState, int flags) throws IOException {
docFreq = termState.docFreq;
docTermStartFP = termState.docStartFP;
posTermStartFP = termState.posStartFP;
@ -1087,27 +1056,23 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
@Override
public int nextDoc() throws IOException {
while (true) {
if (docUpto == docFreq) {
return doc = NO_MORE_DOCS;
}
if (docBufferUpto == BLOCK_SIZE) {
refillDocs();
}
accum += docDeltaBuffer[docBufferUpto];
freq = freqBuffer[docBufferUpto];
posPendingCount += freq;
docBufferUpto++;
docUpto++;
if (liveDocs == null || liveDocs.get(accum)) {
doc = accum;
position = 0;
lastStartOffset = 0;
return doc;
}
if (docUpto == docFreq) {
return doc = NO_MORE_DOCS;
}
if (docBufferUpto == BLOCK_SIZE) {
refillDocs();
}
accum += docDeltaBuffer[docBufferUpto];
freq = freqBuffer[docBufferUpto];
posPendingCount += freq;
docBufferUpto++;
docUpto++;
doc = accum;
position = 0;
lastStartOffset = 0;
return doc;
}
@Override
@ -1174,13 +1139,9 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
}
}
if (liveDocs == null || liveDocs.get(accum)) {
position = 0;
lastStartOffset = 0;
return doc = accum;
} else {
return nextDoc();
}
position = 0;
lastStartOffset = 0;
return doc = accum;
}
// TODO: in theory we could avoid loading frq block

View File

@ -31,9 +31,9 @@ import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.InfoStream;
@ -550,7 +550,8 @@ class BufferedUpdatesStream implements Accountable {
if (state.delGen < delGen) {
// we don't need term frequencies for this
state.postingsEnum = state.termsEnum.postings(state.rld.getLiveDocs(), state.postingsEnum, PostingsEnum.NONE);
final Bits acceptDocs = state.rld.getLiveDocs();
state.postingsEnum = state.termsEnum.postings(state.postingsEnum, PostingsEnum.NONE);
assert state.postingsEnum != null;
@ -559,6 +560,9 @@ class BufferedUpdatesStream implements Accountable {
if (docID == DocIdSetIterator.NO_MORE_DOCS) {
break;
}
if (acceptDocs != null && acceptDocs.get(docID) == false) {
continue;
}
if (!state.any) {
state.rld.initWritableLiveDocs();
state.any = true;
@ -646,7 +650,8 @@ class BufferedUpdatesStream implements Accountable {
if (termsEnum.seekExact(term.bytes())) {
// we don't need term frequencies for this
postingsEnum = termsEnum.postings(segState.rld.getLiveDocs(), postingsEnum, PostingsEnum.NONE);
final Bits acceptDocs = segState.rld.getLiveDocs();
postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
DocValuesFieldUpdates dvUpdates = dvUpdatesContainer.getUpdates(update.field, update.type);
if (dvUpdates == null) {
@ -657,6 +662,9 @@ class BufferedUpdatesStream implements Accountable {
if (doc >= limit) {
break; // no more docs that can be updated for this term
}
if (acceptDocs != null && acceptDocs.get(doc) == false) {
continue;
}
dvUpdates.add(doc, update.value);
}
}

View File

@ -528,9 +528,8 @@ public class CheckIndex implements Closeable {
result.cantOpenSegments = true;
return result;
}
int format = 0;
try {
format = input.readInt();
/*int format =*/ input.readInt();
} catch (Throwable t) {
if (failFast) {
IOUtils.reThrow(t);
@ -959,7 +958,7 @@ public class CheckIndex implements Closeable {
}
}
postingsEnum = termsEnum.postings(null, postingsEnum, 0);
postingsEnum = termsEnum.postings(postingsEnum, 0);
int lastDoc = -1;
while (true) {
@ -1203,7 +1202,6 @@ public class CheckIndex implements Closeable {
long sumTotalTermFreq = 0;
long sumDocFreq = 0;
long upto = 0;
FixedBitSet visitedDocs = new FixedBitSet(maxDoc);
while(true) {
@ -1249,7 +1247,7 @@ public class CheckIndex implements Closeable {
}
sumDocFreq += docFreq;
postings = termsEnum.postings(liveDocs, postings, PostingsEnum.ALL);
postings = termsEnum.postings(postings, PostingsEnum.ALL);
if (hasFreqs == false) {
if (termsEnum.totalTermFreq() != -1) {
@ -1275,13 +1273,13 @@ public class CheckIndex implements Closeable {
int lastDoc = -1;
int docCount = 0;
boolean hasNonDeletedDocs = false;
long totalTermFreq = 0;
while(true) {
final int doc = postings.nextDoc();
if (doc == DocIdSetIterator.NO_MORE_DOCS) {
break;
}
status.totFreq++;
visitedDocs.set(doc);
int freq = -1;
if (hasFreqs) {
@ -1289,7 +1287,6 @@ public class CheckIndex implements Closeable {
if (freq <= 0) {
throw new RuntimeException("term " + term + ": doc " + doc + ": freq " + freq + " is out of bounds");
}
status.totPos += freq;
totalTermFreq += freq;
} else {
// When a field didn't index freq, it must
@ -1299,6 +1296,13 @@ public class CheckIndex implements Closeable {
throw new RuntimeException("term " + term + ": doc " + doc + ": freq " + freq + " != 1 when Terms.hasFreqs() is false");
}
}
if (liveDocs == null || liveDocs.get(doc)) {
hasNonDeletedDocs = true;
status.totFreq++;
if (freq >= 0) {
status.totPos += freq;
}
}
docCount++;
if (doc <= lastDoc) {
@ -1358,7 +1362,7 @@ public class CheckIndex implements Closeable {
}
}
if (docCount != 0) {
if (hasNonDeletedDocs) {
status.termCount++;
} else {
status.delTermCount++;
@ -1367,28 +1371,6 @@ public class CheckIndex implements Closeable {
final long totalTermFreq2 = termsEnum.totalTermFreq();
final boolean hasTotalTermFreq = hasFreqs && totalTermFreq2 != -1;
// Re-count if there are deleted docs:
if (liveDocs != null) {
if (hasFreqs) {
postings = termsEnum.postings(null, postings);
docCount = 0;
totalTermFreq = 0;
while(postings.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
visitedDocs.set(postings.docID());
docCount++;
totalTermFreq += postings.freq();
}
} else {
postings = termsEnum.postings(null, postings, PostingsEnum.NONE);
docCount = 0;
totalTermFreq = -1;
while(postings.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
visitedDocs.set(postings.docID());
docCount++;
}
}
}
if (docCount != docFreq) {
throw new RuntimeException("term " + term + " docFreq=" + docFreq + " != tot docs w/o deletions " + docCount);
}
@ -1406,7 +1388,7 @@ public class CheckIndex implements Closeable {
if (hasPositions) {
for(int idx=0;idx<7;idx++) {
final int skipDocID = (int) (((idx+1)*(long) maxDoc)/8);
postings = termsEnum.postings(liveDocs, postings, PostingsEnum.ALL);
postings = termsEnum.postings(postings, PostingsEnum.ALL);
final int docID = postings.advance(skipDocID);
if (docID == DocIdSetIterator.NO_MORE_DOCS) {
break;
@ -1470,7 +1452,7 @@ public class CheckIndex implements Closeable {
} else {
for(int idx=0;idx<7;idx++) {
final int skipDocID = (int) (((idx+1)*(long) maxDoc)/8);
postings = termsEnum.postings(liveDocs, postings, PostingsEnum.NONE);
postings = termsEnum.postings(postings, PostingsEnum.NONE);
final int docID = postings.advance(skipDocID);
if (docID == DocIdSetIterator.NO_MORE_DOCS) {
break;
@ -1552,7 +1534,7 @@ public class CheckIndex implements Closeable {
}
int expectedDocFreq = termsEnum.docFreq();
PostingsEnum d = termsEnum.postings(null, null, PostingsEnum.NONE);
PostingsEnum d = termsEnum.postings(null, PostingsEnum.NONE);
int docFreq = 0;
while (d.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
docFreq++;
@ -1591,7 +1573,6 @@ public class CheckIndex implements Closeable {
}
// Seek by term
long totDocCount = 0;
for(int i=seekCount-1;i>=0;i--) {
if (termsEnum.seekCeil(seekTerms[i]) != TermsEnum.SeekStatus.FOUND) {
throw new RuntimeException("seek to existing term " + seekTerms[i] + " failed");
@ -1600,40 +1581,10 @@ public class CheckIndex implements Closeable {
throw new RuntimeException("seek to existing term " + seekTerms[i] + " returned FOUND but seeked to the wrong term " + termsEnum.term());
}
postings = termsEnum.postings(liveDocs, postings, PostingsEnum.NONE);
postings = termsEnum.postings(postings, PostingsEnum.NONE);
if (postings == null) {
throw new RuntimeException("null DocsEnum from to existing term " + seekTerms[i]);
}
while (postings.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
totDocCount++;
}
}
long totDocCountNoDeletes = 0;
long totDocFreq = 0;
for(int i=0;i<seekCount;i++) {
if (!termsEnum.seekExact(seekTerms[i])) {
throw new RuntimeException("seek to existing term " + seekTerms[i] + " failed");
}
totDocFreq += termsEnum.docFreq();
postings = termsEnum.postings(null, postings, PostingsEnum.NONE);
if (postings == null) {
throw new RuntimeException("null DocsEnum from to existing term " + seekTerms[i]);
}
while(postings.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
totDocCountNoDeletes++;
}
}
if (totDocCount > totDocCountNoDeletes) {
throw new RuntimeException("more postings with deletes=" + totDocCount + " than without=" + totDocCountNoDeletes);
}
if (totDocCountNoDeletes != totDocFreq) {
throw new RuntimeException("docfreqs=" + totDocFreq + " != recomputed docfreqs=" + totDocCountNoDeletes);
}
}
}
@ -1685,7 +1636,6 @@ public class CheckIndex implements Closeable {
Status.TermIndexStatus status;
final int maxDoc = reader.maxDoc();
final Bits liveDocs = reader.getLiveDocs();
try {
if (infoStream != null) {
@ -1694,13 +1644,7 @@ public class CheckIndex implements Closeable {
final Fields fields = reader.getPostingsReader().getMergeInstance();
final FieldInfos fieldInfos = reader.getFieldInfos();
status = checkFields(fields, liveDocs, maxDoc, fieldInfos, true, false, infoStream, verbose);
if (liveDocs != null) {
if (infoStream != null) {
infoStream.print(" test (ignoring deletes): terms, freq, prox...");
}
checkFields(fields, null, maxDoc, fieldInfos, true, false, infoStream, verbose);
}
status = checkFields(fields, reader.getLiveDocs(), maxDoc, fieldInfos, true, false, infoStream, verbose);
} catch (Throwable e) {
if (failFast) {
IOUtils.reThrow(e);
@ -2015,7 +1959,6 @@ public class CheckIndex implements Closeable {
long startNS = System.nanoTime();
final Status.TermVectorStatus status = new Status.TermVectorStatus();
final FieldInfos fieldInfos = reader.getFieldInfos();
final Bits onlyDocIsDeleted = new FixedBitSet(1);
try {
if (infoStream != null) {
@ -2026,7 +1969,6 @@ public class CheckIndex implements Closeable {
// Only used if crossCheckTermVectors is true:
PostingsEnum postingsDocs = null;
PostingsEnum postingsDocs2 = null;
final Bits liveDocs = reader.getLiveDocs();
@ -2055,12 +1997,6 @@ public class CheckIndex implements Closeable {
// First run with no deletions:
checkFields(tfv, null, 1, fieldInfos, false, true, infoStream, verbose);
if (j == 0) {
// Also test with the 1 doc deleted; we only do this for first doc because this really is just looking for a [slightly] buggy
// TermVectors impl that fails to respect the incoming live docs:
checkFields(tfv, onlyDocIsDeleted, 1, fieldInfos, false, true, infoStream, verbose);
}
// Only agg stats if the doc is live:
final boolean doStats = liveDocs == null || liveDocs.get(j);
@ -2097,7 +2033,7 @@ public class CheckIndex implements Closeable {
while ((term = termsEnum.next()) != null) {
// This is the term vectors:
postings = termsEnum.postings(null, postings, PostingsEnum.ALL);
postings = termsEnum.postings(postings, PostingsEnum.ALL);
assert postings != null;
if (!postingsTermsEnum.seekExact(term)) {
@ -2105,11 +2041,11 @@ public class CheckIndex implements Closeable {
}
// This is the inverted index ("real" postings):
postingsDocs2 = postingsTermsEnum.postings(null, postingsDocs2, PostingsEnum.ALL);
assert postingsDocs2 != null;
postingsDocs = postingsTermsEnum.postings(postingsDocs, PostingsEnum.ALL);
assert postingsDocs != null;
final int advanceDoc = postingsDocs2.advance(j);
final int advanceDoc = postingsDocs.advance(j);
if (advanceDoc != j) {
throw new RuntimeException("vector term=" + term + " field=" + field + ": doc=" + j + " was not found in postings (got: " + advanceDoc + ")");
}
@ -2122,8 +2058,8 @@ public class CheckIndex implements Closeable {
if (postingsHasFreq) {
final int tf = postings.freq();
if (postingsHasFreq && postingsDocs2.freq() != tf) {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": freq=" + tf + " differs from postings freq=" + postingsDocs2.freq());
if (postingsHasFreq && postingsDocs.freq() != tf) {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": freq=" + tf + " differs from postings freq=" + postingsDocs.freq());
}
// Term vectors has prox?
@ -2131,7 +2067,7 @@ public class CheckIndex implements Closeable {
for (int i = 0; i < tf; i++) {
int pos = postings.nextPosition();
if (postingsTerms.hasPositions()) {
int postingsPos = postingsDocs2.nextPosition();
int postingsPos = postingsDocs.nextPosition();
if (terms.hasPositions() && pos != postingsPos) {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": pos=" + pos + " differs from postings pos=" + postingsPos);
}
@ -2153,8 +2089,8 @@ public class CheckIndex implements Closeable {
*/
if (startOffset != -1 && endOffset != -1 && postingsTerms.hasOffsets()) {
int postingsStartOffset = postingsDocs2.startOffset();
int postingsEndOffset = postingsDocs2.endOffset();
int postingsStartOffset = postingsDocs.startOffset();
int postingsEndOffset = postingsDocs.endOffset();
if (startOffset != postingsStartOffset) {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": startOffset=" + startOffset + " differs from postings startOffset=" + postingsStartOffset);
}
@ -2174,16 +2110,16 @@ public class CheckIndex implements Closeable {
if (payload == null) {
// we have payloads, but not at this position.
// postings has payloads too, it should not have one at this position
if (postingsDocs2.getPayload() != null) {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + " has no payload but postings does: " + postingsDocs2.getPayload());
if (postingsDocs.getPayload() != null) {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + " has no payload but postings does: " + postingsDocs.getPayload());
}
} else {
// we have payloads, and one at this position
// postings should also have one at this position, with the same bytes.
if (postingsDocs2.getPayload() == null) {
if (postingsDocs.getPayload() == null) {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + " has payload=" + payload + " but postings does not.");
}
BytesRef postingsPayload = postingsDocs2.getPayload();
BytesRef postingsPayload = postingsDocs.getPayload();
if (!payload.equals(postingsPayload)) {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + " has payload=" + payload + " but differs from postings payload=" + postingsPayload);
}

View File

@ -216,8 +216,8 @@ public class FilterLeafReader extends LeafReader {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
return in.postings(liveDocs, reuse, flags);
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
return in.postings(reuse, flags);
}
}

View File

@ -21,7 +21,6 @@ import java.io.IOException;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.Bits;
/**
* Abstract class for enumerating a subset of all terms.
@ -179,8 +178,8 @@ public abstract class FilteredTermsEnum extends TermsEnum {
}
@Override
public PostingsEnum postings(Bits bits, PostingsEnum reuse, int flags) throws IOException {
return tenum.postings(bits, reuse, flags);
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
return tenum.postings(reuse, flags);
}
/** This enum does not support seeking!

View File

@ -25,7 +25,6 @@ import java.util.Map;
import org.apache.lucene.index.FreqProxTermsWriterPerField.FreqProxPostingsArray;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@ -226,11 +225,7 @@ class FreqProxFields extends Fields {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) {
if (liveDocs != null) {
throw new IllegalArgumentException("liveDocs must be null");
}
public PostingsEnum postings(PostingsEnum reuse, int flags) {
if (PostingsEnum.featureRequested(flags, PostingsEnum.POSITIONS)) {
FreqProxPostingsEnum posEnum;

View File

@ -56,7 +56,7 @@ final class FreqProxTermsWriter extends TermsHash {
}
if (termsEnum != null && termsEnum.seekExact(deleteTerm.bytes())) {
postingsEnum = termsEnum.postings(null, postingsEnum, 0);
postingsEnum = termsEnum.postings(postingsEnum, 0);
int delDocLimit = segDeletes.get(deleteTerm);
assert delDocLimit < PostingsEnum.NO_MORE_DOCS;
while (true) {

View File

@ -210,7 +210,8 @@ public abstract class LeafReader extends IndexReader {
/** Returns {@link PostingsEnum} for the specified term.
* This will return null if either the field or
* term does not exist.
* @see TermsEnum#postings(Bits, PostingsEnum) */
* <p><b>NOTE:</b> The returned {@link PostingsEnum} may contain deleted docs.
* @see TermsEnum#postings(PostingsEnum) */
public final PostingsEnum postings(Term term, int flags) throws IOException {
assert term.field() != null;
assert term.bytes() != null;
@ -218,7 +219,7 @@ public abstract class LeafReader extends IndexReader {
if (terms != null) {
final TermsEnum termsEnum = terms.iterator();
if (termsEnum.seekExact(term.bytes())) {
return termsEnum.postings(getLiveDocs(), null, flags);
return termsEnum.postings(null, flags);
}
}
return null;
@ -231,6 +232,7 @@ public abstract class LeafReader extends IndexReader {
* and do not need any proximity data.
* This method is equivalent to
* {@link #postings(Term, int) postings(term, PostingsEnum.FREQS)}
* <p><b>NOTE:</b> The returned {@link PostingsEnum} may contain deleted docs.
* @see #postings(Term, int)
*/
public final PostingsEnum postings(Term term) throws IOException {

View File

@ -19,8 +19,6 @@ package org.apache.lucene.index;
import java.io.IOException;
import org.apache.lucene.util.Bits;
import static org.apache.lucene.index.FilterLeafReader.FilterFields;
import static org.apache.lucene.index.FilterLeafReader.FilterTerms;
import static org.apache.lucene.index.FilterLeafReader.FilterTermsEnum;
@ -107,11 +105,7 @@ public class MappedMultiFields extends FilterFields {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
if (liveDocs != null) {
throw new IllegalArgumentException("liveDocs must be null");
}
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
MappingMultiPostingsEnum mappingDocsAndPositionsEnum;
if (reuse instanceof MappingMultiPostingsEnum) {
MappingMultiPostingsEnum postings = (MappingMultiPostingsEnum) reuse;
@ -124,7 +118,7 @@ public class MappedMultiFields extends FilterFields {
mappingDocsAndPositionsEnum = new MappingMultiPostingsEnum(field, mergeState);
}
MultiPostingsEnum docsAndPositionsEnum = (MultiPostingsEnum) in.postings(liveDocs, mappingDocsAndPositionsEnum.multiDocsAndPositionsEnum, flags);
MultiPostingsEnum docsAndPositionsEnum = (MultiPostingsEnum) in.postings(mappingDocsAndPositionsEnum.multiDocsAndPositionsEnum, flags);
mappingDocsAndPositionsEnum.reset(docsAndPositionsEnum);
return mappingDocsAndPositionsEnum;
}

View File

@ -123,8 +123,8 @@ public final class MultiFields extends Fields {
/** Returns {@link PostingsEnum} for the specified field and
* term. This will return null if the field or term does
* not exist. */
public static PostingsEnum getTermDocsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term) throws IOException {
return getTermDocsEnum(r, liveDocs, field, term, PostingsEnum.FREQS);
public static PostingsEnum getTermDocsEnum(IndexReader r, String field, BytesRef term) throws IOException {
return getTermDocsEnum(r, field, term, PostingsEnum.FREQS);
}
/** Returns {@link PostingsEnum} for the specified field and
@ -132,15 +132,15 @@ public final class MultiFields extends Fields {
* Some codecs may be able to optimize their
* implementation when freqs are not required. This will
* return null if the field or term does not exist. See {@link
* TermsEnum#postings(Bits, PostingsEnum,int)}.*/
public static PostingsEnum getTermDocsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term, int flags) throws IOException {
* TermsEnum#postings(PostingsEnum,int)}.*/
public static PostingsEnum getTermDocsEnum(IndexReader r, String field, BytesRef term, int flags) throws IOException {
assert field != null;
assert term != null;
final Terms terms = getTerms(r, field);
if (terms != null) {
final TermsEnum termsEnum = terms.iterator();
if (termsEnum.seekExact(term)) {
return termsEnum.postings(liveDocs, null, flags);
return termsEnum.postings(null, flags);
}
}
return null;
@ -149,9 +149,9 @@ public final class MultiFields extends Fields {
/** Returns {@link PostingsEnum} for the specified
* field and term. This will return null if the field or
* term does not exist or positions were not indexed.
* @see #getTermPositionsEnum(IndexReader, Bits, String, BytesRef, int) */
public static PostingsEnum getTermPositionsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term) throws IOException {
return getTermPositionsEnum(r, liveDocs, field, term, PostingsEnum.ALL);
* @see #getTermPositionsEnum(IndexReader, String, BytesRef, int) */
public static PostingsEnum getTermPositionsEnum(IndexReader r, String field, BytesRef term) throws IOException {
return getTermPositionsEnum(r, field, term, PostingsEnum.ALL);
}
/** Returns {@link PostingsEnum} for the specified
@ -159,15 +159,15 @@ public final class MultiFields extends Fields {
* required. Some codecs may be able to optimize
* their implementation when offsets and/or payloads are not
* required. This will return null if the field or term does not
* exist. See {@link TermsEnum#postings(Bits, PostingsEnum,int)}. */
public static PostingsEnum getTermPositionsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term, int flags) throws IOException {
* exist. See {@link TermsEnum#postings(PostingsEnum,int)}. */
public static PostingsEnum getTermPositionsEnum(IndexReader r, String field, BytesRef term, int flags) throws IOException {
assert field != null;
assert term != null;
final Terms terms = getTerms(r, field);
if (terms != null) {
final TermsEnum termsEnum = terms.iterator();
if (termsEnum.seekExact(term)) {
return termsEnum.postings(liveDocs, null, flags);
return termsEnum.postings(null, flags);
}
}
return null;

View File

@ -20,7 +20,6 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.PriorityQueue;
@ -327,7 +326,7 @@ public final class MultiTermsEnum extends TermsEnum {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
MultiPostingsEnum docsEnum;
// Can only reuse if incoming enum is also a MultiDocsEnum
@ -340,13 +339,6 @@ public final class MultiTermsEnum extends TermsEnum {
} else {
docsEnum = new MultiPostingsEnum(this, subs.length);
}
final MultiBits multiLiveDocs;
if (liveDocs instanceof MultiBits) {
multiLiveDocs = (MultiBits) liveDocs;
} else {
multiLiveDocs = null;
}
int upto = 0;
@ -354,31 +346,8 @@ public final class MultiTermsEnum extends TermsEnum {
final TermsEnumWithSlice entry = top[i];
final Bits b;
if (multiLiveDocs != null) {
// optimize for common case: requested skip docs is a
// congruent sub-slice of MultiBits: in this case, we
// just pull the liveDocs from the sub reader, rather
// than making the inefficient
// Slice(Multi(sub-readers)):
final MultiBits.SubResult sub = multiLiveDocs.getMatchingSub(entry.subSlice);
if (sub.matches) {
b = sub.result;
} else {
// custom case: requested skip docs is foreign:
// must slice it on every access
b = new BitsSlice(liveDocs, entry.subSlice);
}
} else if (liveDocs != null) {
b = new BitsSlice(liveDocs, entry.subSlice);
} else {
// no deletions
b = null;
}
assert entry.index < docsEnum.subPostingsEnums.length: entry.index + " vs " + docsEnum.subPostingsEnums.length + "; " + subs.length;
final PostingsEnum subPostingsEnum = entry.terms.postings(b, docsEnum.subPostingsEnums[entry.index], flags);
final PostingsEnum subPostingsEnum = entry.terms.postings(docsEnum.subPostingsEnums[entry.index], flags);
assert subPostingsEnum != null;
docsEnum.subPostingsEnums[entry.index] = subPostingsEnum;
subDocs[upto].postingsEnum = subPostingsEnum;

View File

@ -21,7 +21,6 @@ import java.io.IOException;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
/** Iterates through the postings.
@ -30,29 +29,29 @@ import org.apache.lucene.util.BytesRef;
public abstract class PostingsEnum extends DocIdSetIterator {
/**
* Flag to pass to {@link TermsEnum#postings(Bits, PostingsEnum, int)} if you don't
* Flag to pass to {@link TermsEnum#postings(PostingsEnum, int)} if you don't
* require per-document postings in the returned enum.
*/
public static final short NONE = 0;
/** Flag to pass to {@link TermsEnum#postings(Bits, PostingsEnum, int)}
/** Flag to pass to {@link TermsEnum#postings(PostingsEnum, int)}
* if you require term frequencies in the returned enum. */
public static final short FREQS = 1 << 3;
/** Flag to pass to {@link TermsEnum#postings(Bits, PostingsEnum, int)}
/** Flag to pass to {@link TermsEnum#postings(PostingsEnum, int)}
* if you require term positions in the returned enum. */
public static final short POSITIONS = FREQS | 1 << 4;
/** Flag to pass to {@link TermsEnum#postings(Bits, PostingsEnum, int)}
/** Flag to pass to {@link TermsEnum#postings(PostingsEnum, int)}
* if you require offsets in the returned enum. */
public static final short OFFSETS = POSITIONS | 1 << 5;
/** Flag to pass to {@link TermsEnum#postings(Bits, PostingsEnum, int)}
/** Flag to pass to {@link TermsEnum#postings(PostingsEnum, int)}
* if you require payloads in the returned enum. */
public static final short PAYLOADS = POSITIONS | 1 << 6;
/**
* Flag to pass to {@link TermsEnum#postings(Bits, PostingsEnum, int)}
* Flag to pass to {@link TermsEnum#postings(PostingsEnum, int)}
* to get positions, payloads and offsets in the returned enum
*/
public static final short ALL = OFFSETS | PAYLOADS;

View File

@ -19,7 +19,6 @@ package org.apache.lucene.index;
import java.io.IOException;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@ -106,7 +105,7 @@ class SortedDocValuesTermsEnum extends TermsEnum {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
throw new UnsupportedOperationException();
}

View File

@ -17,7 +17,6 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@ -106,7 +105,7 @@ class SortedSetDocValuesTermsEnum extends TermsEnum {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
throw new UnsupportedOperationException();
}

View File

@ -20,7 +20,6 @@ package org.apache.lucene.index;
import java.io.IOException;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefIterator;
@ -142,18 +141,19 @@ public abstract class TermsEnum implements BytesRefIterator {
* call this when the enum is unpositioned. This method
* will not return null.
* <p>
* <b>NOTE</b>: the returned iterator may return deleted documents, so
* deleted documents have to be checked on top of the {@link PostingsEnum}.
* <p>
* Use this method if you only require documents and frequencies,
* and do not need any proximity data.
* This method is equivalent to
* {@link #postings(Bits, PostingsEnum, int) postings(liveDocs, reuse, PostingsEnum.FREQS)}
*
* @param liveDocs unset bits are documents that should not
* be returned
* {@link #postings(PostingsEnum, int) postings(reuse, PostingsEnum.FREQS)}
*
* @param reuse pass a prior PostingsEnum for possible reuse
* @see #postings(Bits, PostingsEnum, int)
* @see #postings(PostingsEnum, int)
*/
public final PostingsEnum postings(Bits liveDocs, PostingsEnum reuse) throws IOException {
return postings(liveDocs, reuse, PostingsEnum.FREQS);
public final PostingsEnum postings(PostingsEnum reuse) throws IOException {
return postings(reuse, PostingsEnum.FREQS);
}
/** Get {@link PostingsEnum} for the current term, with
@ -161,14 +161,15 @@ public abstract class TermsEnum implements BytesRefIterator {
* are required. Do not call this when the enum is
* unpositioned. This method may return null if the postings
* information required is not available from the index
*
* @param liveDocs unset bits are documents that should not
* be returned
* <p>
* <b>NOTE</b>: the returned iterator may return deleted documents, so
* deleted documents have to be checked on top of the {@link PostingsEnum}.
*
* @param reuse pass a prior PostingsEnum for possible reuse
* @param flags specifies which optional per-document values
* you require; see {@link PostingsEnum#FREQS}
*/
public abstract PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException;
public abstract PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException;
/**
* Expert: Returns the TermsEnums internal state to position the TermsEnum
@ -225,7 +226,7 @@ public abstract class TermsEnum implements BytesRefIterator {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) {
public PostingsEnum postings(PostingsEnum reuse, int flags) {
throw new IllegalStateException("this method should never be called");
}

View File

@ -21,7 +21,7 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import org.apache.lucene.search.BooleanWeight;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.PriorityQueue;
/**
@ -41,7 +41,7 @@ final class BooleanScorer extends BulkScorer {
return new BulkScorer() {
@Override
public int score(final LeafCollector collector, int min, int max) throws IOException {
public int score(final LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
final LeafCollector noScoreCollector = new LeafCollector() {
FakeScorer fake = new FakeScorer();
@ -56,7 +56,7 @@ final class BooleanScorer extends BulkScorer {
collector.collect(doc);
}
};
return scorer.score(noScoreCollector, min, max);
return scorer.score(noScoreCollector, acceptDocs, min, max);
}
@Override
@ -83,11 +83,11 @@ final class BooleanScorer extends BulkScorer {
}
void advance(int min) throws IOException {
score(min, min);
score(null, min, min);
}
void score(int min, int max) throws IOException {
next = scorer.score(orCollector, min, max);
void score(Bits acceptDocs, int min, int max) throws IOException {
next = scorer.score(orCollector, acceptDocs, min, max);
}
}
@ -237,12 +237,12 @@ final class BooleanScorer extends BulkScorer {
}
}
private void scoreWindow(LeafCollector collector, int base, int min, int max,
private void scoreWindow(LeafCollector collector, Bits acceptDocs, int base, int min, int max,
BulkScorerAndDoc[] scorers, int numScorers) throws IOException {
for (int i = 0; i < numScorers; ++i) {
final BulkScorerAndDoc scorer = scorers[i];
assert scorer.next < max;
scorer.score(min, max);
scorer.score(acceptDocs, min, max);
}
scoreMatches(collector, base);
@ -270,7 +270,7 @@ final class BooleanScorer extends BulkScorer {
return headTop;
}
private void scoreWindow(LeafCollector collector, int windowBase, int windowMin, int windowMax) throws IOException {
private void scoreWindow(LeafCollector collector, Bits acceptDocs, int windowBase, int windowMin, int windowMax) throws IOException {
// Fill 'leads' with all scorers from 'head' that are in the right window
leads[0] = head.pop();
int maxFreq = 1;
@ -296,7 +296,7 @@ final class BooleanScorer extends BulkScorer {
}
tail.clear();
scoreWindow(collector, windowBase, windowMin, windowMax, leads, maxFreq);
scoreWindow(collector, acceptDocs, windowBase, windowMin, windowMax, leads, maxFreq);
}
// Push back scorers into head and tail
@ -309,7 +309,7 @@ final class BooleanScorer extends BulkScorer {
}
@Override
public int score(LeafCollector collector, int min, int max) throws IOException {
public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
fakeScorer.doc = -1;
collector.setScorer(fakeScorer);
@ -321,7 +321,7 @@ final class BooleanScorer extends BulkScorer {
final int windowMax = Math.min(max, windowBase + SIZE);
// general case
scoreWindow(collector, windowBase, windowMin, windowMax);
scoreWindow(collector, acceptDocs, windowBase, windowMin, windowMax);
top = head.top();
}

View File

@ -28,7 +28,6 @@ import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.util.Bits;
/**
* Expert: the Weight for BooleanQuery, used to
@ -194,12 +193,12 @@ final class BooleanWeight extends Weight {
/** Try to build a boolean scorer for this weight. Returns null if {@link BooleanScorer}
* cannot be used. */
// pkg-private for forcing use of BooleanScorer in tests
BooleanScorer booleanScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
BooleanScorer booleanScorer(LeafReaderContext context) throws IOException {
List<BulkScorer> optional = new ArrayList<BulkScorer>();
Iterator<BooleanClause> cIter = query.iterator();
for (Weight w : weights) {
BooleanClause c = cIter.next();
BulkScorer subScorer = w.bulkScorer(context, acceptDocs);
BulkScorer subScorer = w.bulkScorer(context);
if (subScorer == null) {
if (c.isRequired()) {
@ -230,8 +229,8 @@ final class BooleanWeight extends Weight {
}
@Override
public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
final BooleanScorer bulkScorer = booleanScorer(context, acceptDocs);
public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
final BooleanScorer bulkScorer = booleanScorer(context);
if (bulkScorer != null) { // BooleanScorer is applicable
// TODO: what is the right heuristic here?
final long costThreshold;
@ -254,11 +253,11 @@ final class BooleanWeight extends Weight {
return bulkScorer;
}
}
return super.bulkScorer(context, acceptDocs);
return super.bulkScorer(context);
}
@Override
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
public Scorer scorer(LeafReaderContext context) throws IOException {
// initially the user provided value,
// but if minNrShouldMatch == optional.size(),
// we will optimize and move these to required, making this 0
@ -272,7 +271,7 @@ final class BooleanWeight extends Weight {
Iterator<BooleanClause> cIter = query.iterator();
for (Weight w : weights) {
BooleanClause c = cIter.next();
Scorer subScorer = w.scorer(context, acceptDocs);
Scorer subScorer = w.scorer(context);
if (subScorer == null) {
if (c.isRequired()) {
return null;

View File

@ -19,6 +19,8 @@ package org.apache.lucene.search;
import java.io.IOException;
import org.apache.lucene.util.Bits;
/** This class is used to score a range of documents at
* once, and is returned by {@link Weight#bulkScorer}. Only
* queries that have a more optimized means of scoring
@ -30,9 +32,11 @@ public abstract class BulkScorer {
/** Scores and collects all matching documents.
* @param collector The collector to which all matching documents are passed.
* @param acceptDocs {@link Bits} that represents the allowed documents to match, or
* {@code null} if they are all allowed to match.
*/
public void score(LeafCollector collector) throws IOException {
final int next = score(collector, 0, DocIdSetIterator.NO_MORE_DOCS);
public void score(LeafCollector collector, Bits acceptDocs) throws IOException {
final int next = score(collector, acceptDocs, 0, DocIdSetIterator.NO_MORE_DOCS);
assert next == DocIdSetIterator.NO_MORE_DOCS;
}
@ -54,14 +58,16 @@ public abstract class BulkScorer {
* <pre class="prettyprint">
* private final Scorer scorer; // set via constructor
*
* public int score(LeafCollector collector, int min, int max) throws IOException {
* public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
* collector.setScorer(scorer);
* int doc = scorer.docID();
* if (doc &lt; min) {
* doc = scorer.advance(min);
* }
* while (doc &lt; max) {
* collector.collect(doc);
* if (acceptDocs == null || acceptDocs.get(doc)) {
* collector.collect(doc);
* }
* doc = scorer.nextDoc();
* }
* return doc;
@ -69,11 +75,13 @@ public abstract class BulkScorer {
* </pre>
*
* @param collector The collector to which all matching documents are passed.
* @param acceptDocs {@link Bits} that represents the allowed documents to match, or
* {@code null} if they are all allowed to match.
* @param min Score starting at, including, this document
* @param max Score up to, but not including, this doc
* @return an under-estimation of the next matching doc after max
*/
public abstract int score(LeafCollector collector, int min, int max) throws IOException;
public abstract int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException;
/**
* Same as {@link Scorer#cost()} for bulk scorers.

View File

@ -33,7 +33,6 @@ import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.RoaringDocIdSet;
/**
@ -124,7 +123,7 @@ public class CachingWrapperQuery extends Query implements Accountable {
}
@Override
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
public Scorer scorer(LeafReaderContext context) throws IOException {
final LeafReader reader = context.reader();
final Object key = reader.getCoreCacheKey();
@ -133,7 +132,7 @@ public class CachingWrapperQuery extends Query implements Accountable {
hitCount++;
} else if (policy.shouldCache(query, context)) {
missCount++;
final Scorer scorer = weight.scorer(context, null);
final Scorer scorer = weight.scorer(context);
if (scorer == null) {
docIdSet = DocIdSet.EMPTY;
} else {
@ -141,7 +140,7 @@ public class CachingWrapperQuery extends Query implements Accountable {
}
cache.put(key, docIdSet);
} else {
return weight.scorer(context, acceptDocs);
return weight.scorer(context);
}
assert docIdSet != null;
@ -153,21 +152,7 @@ public class CachingWrapperQuery extends Query implements Accountable {
return null;
}
// We apply acceptDocs as an approximation
if (acceptDocs == null) {
return new ConstantScoreScorer(this, 0f, disi);
} else {
final TwoPhaseIterator twoPhaseView = new TwoPhaseIterator(disi) {
@Override
public boolean matches() throws IOException {
final int doc = approximation.docID();
return acceptDocs.get(doc);
}
};
return new ConstantScoreScorer(this, 0f, twoPhaseView);
}
return new ConstantScoreScorer(this, 0f, disi);
}
};
}

View File

@ -83,8 +83,8 @@ public class ConstantScoreQuery extends Query {
}
@Override
public int score(LeafCollector collector, int min, int max) throws IOException {
return bulkScorer.score(wrapCollector(collector), min, max);
public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
return bulkScorer.score(wrapCollector(collector), acceptDocs, min, max);
}
private LeafCollector wrapCollector(LeafCollector collector) {
@ -119,8 +119,8 @@ public class ConstantScoreQuery extends Query {
return new ConstantScoreWeight(this) {
@Override
public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
final BulkScorer innerScorer = innerWeight.bulkScorer(context, acceptDocs);
public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
final BulkScorer innerScorer = innerWeight.bulkScorer(context);
if (innerScorer == null) {
return null;
}
@ -128,8 +128,8 @@ public class ConstantScoreQuery extends Query {
}
@Override
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
final Scorer innerScorer = innerWeight.scorer(context, acceptDocs);
public Scorer scorer(LeafReaderContext context) throws IOException {
final Scorer innerScorer = innerWeight.scorer(context);
if (innerScorer == null) {
return null;
}

View File

@ -66,7 +66,7 @@ public abstract class ConstantScoreWeight extends Weight {
@Override
public final Explanation explain(LeafReaderContext context, int doc) throws IOException {
final Scorer s = scorer(context, context.reader().getLiveDocs());
final Scorer s = scorer(context);
final boolean exists;
if (s == null) {
exists = false;

View File

@ -27,7 +27,6 @@ import java.util.Set;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.Bits;
/**
* A query that generates the union of documents produced by its subqueries, and that scores each document with the maximum
@ -161,11 +160,11 @@ public class DisjunctionMaxQuery extends Query implements Iterable<Query> {
/** Create the scorer used to score our associated DisjunctionMaxQuery */
@Override
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
public Scorer scorer(LeafReaderContext context) throws IOException {
List<Scorer> scorers = new ArrayList<>();
for (Weight w : weights) {
// we will advance() subscorers
Scorer subScorer = w.scorer(context, acceptDocs);
Scorer subScorer = w.scorer(context);
if (subScorer != null) {
scorers.add(subScorer);
}

View File

@ -96,7 +96,7 @@ public abstract class Filter extends Query {
@Override
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
final Scorer scorer = scorer(context, context.reader().getLiveDocs());
final Scorer scorer = scorer(context);
final boolean match = (scorer != null && scorer.advance(doc) == doc);
if (match) {
assert scorer.score() == 0f;
@ -107,8 +107,8 @@ public abstract class Filter extends Query {
}
@Override
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
final DocIdSet set = getDocIdSet(context, acceptDocs);
public Scorer scorer(LeafReaderContext context) throws IOException {
final DocIdSet set = getDocIdSet(context, null);
if (set == null) {
return null;
}

View File

@ -31,7 +31,6 @@ import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.Attribute;
import org.apache.lucene.util.AttributeImpl;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.UnicodeUtil;
@ -265,8 +264,8 @@ public class FuzzyTermsEnum extends TermsEnum {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
return actualEnum.postings(liveDocs, reuse, flags);
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
return actualEnum.postings(reuse, flags);
}
@Override

View File

@ -46,6 +46,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.search.similarities.DefaultSimilarity;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.store.NIOFSDirectory; // javadoc
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.ThreadInterruptedException;
@ -614,10 +615,10 @@ public class IndexSearcher {
// continue with the following leaf
continue;
}
BulkScorer scorer = weight.bulkScorer(ctx, ctx.reader().getLiveDocs());
BulkScorer scorer = weight.bulkScorer(ctx);
if (scorer != null) {
try {
scorer.score(leafCollector);
scorer.score(leafCollector, ctx.reader().getLiveDocs());
} catch (CollectionTerminatedException e) {
// collection was terminated prematurely
// continue with the following leaf
@ -667,7 +668,10 @@ public class IndexSearcher {
int n = ReaderUtil.subIndex(doc, leafContexts);
final LeafReaderContext ctx = leafContexts.get(n);
int deBasedDoc = doc - ctx.docBase;
final Bits liveDocs = ctx.reader().getLiveDocs();
if (liveDocs != null && liveDocs.get(deBasedDoc) == false) {
return Explanation.noMatch("Document " + doc + " is deleted");
}
return weight.explain(ctx, deBasedDoc);
}

View File

@ -36,7 +36,6 @@ import org.apache.lucene.index.ReaderUtil;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.RoaringDocIdSet;
@ -566,7 +565,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
}
@Override
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
public Scorer scorer(LeafReaderContext context) throws IOException {
if (context.ord == 0) {
policy.onUse(getQuery());
}
@ -574,7 +573,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
if (docIdSet == null) {
if (cacheEntryHasReasonableWorstCaseSize(ReaderUtil.getTopLevelContext(context).reader().maxDoc())
&& policy.shouldCache(in.getQuery(), context)) {
final Scorer scorer = in.scorer(context, null);
final Scorer scorer = in.scorer(context);
if (scorer == null) {
docIdSet = DocIdSet.EMPTY;
} else {
@ -582,7 +581,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
}
putIfAbsent(in.getQuery(), context, docIdSet);
} else {
return in.scorer(context, acceptDocs);
return in.scorer(context);
}
}
@ -595,19 +594,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
return null;
}
// we apply acceptDocs as an approximation
if (acceptDocs == null) {
return new ConstantScoreScorer(this, 0f, disi);
} else {
final TwoPhaseIterator twoPhaseView = new TwoPhaseIterator(disi) {
@Override
public boolean matches() throws IOException {
final int doc = approximation.docID();
return acceptDocs.get(doc);
}
};
return new ConstantScoreScorer(this, 0f, twoPhaseView);
}
return new ConstantScoreScorer(this, 0f, disi);
}
}

View File

@ -33,7 +33,6 @@ import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.similarities.Similarity.SimScorer;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.PriorityQueue;
import org.apache.lucene.util.ToStringUtils;
@ -173,10 +172,9 @@ public class MultiPhraseQuery extends Query {
}
@Override
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
public Scorer scorer(LeafReaderContext context) throws IOException {
assert !termArrays.isEmpty();
final LeafReader reader = context.reader();
final Bits liveDocs = acceptDocs;
PhraseQuery.PostingsAndFreq[] postingsFreqs = new PhraseQuery.PostingsAndFreq[termArrays.size()];
@ -201,7 +199,7 @@ public class MultiPhraseQuery extends Query {
TermState termState = termContexts.get(term).get(context.ord);
if (termState != null) {
termsEnum.seekExact(term.bytes(), termState);
postings.add(termsEnum.postings(liveDocs, null, PostingsEnum.POSITIONS));
postings.add(termsEnum.postings(null, PostingsEnum.POSITIONS));
}
}
@ -233,7 +231,7 @@ public class MultiPhraseQuery extends Query {
@Override
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
Scorer scorer = scorer(context, context.reader().getLiveDocs());
Scorer scorer = scorer(context);
if (scorer != null) {
int newDoc = scorer.advance(doc);
if (newDoc == doc) {

View File

@ -31,7 +31,6 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.util.BitDocIdSet;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
/**
@ -136,7 +135,7 @@ final class MultiTermQueryConstantScoreWrapper<Q extends MultiTermQuery> extends
* On the given leaf context, try to either rewrite to a disjunction if
* there are few terms, or build a bitset containing matching docs.
*/
private WeightOrBitSet rewrite(LeafReaderContext context, Bits acceptDocs) throws IOException {
private WeightOrBitSet rewrite(LeafReaderContext context) throws IOException {
final Terms terms = context.reader().terms(query.field);
if (terms == null) {
// field does not exist
@ -168,14 +167,14 @@ final class MultiTermQueryConstantScoreWrapper<Q extends MultiTermQuery> extends
TermsEnum termsEnum2 = terms.iterator();
for (TermAndState t : collectedTerms) {
termsEnum2.seekExact(t.term, t.state);
docs = termsEnum2.postings(acceptDocs, docs, PostingsEnum.NONE);
docs = termsEnum2.postings(docs, PostingsEnum.NONE);
builder.or(docs);
}
}
// Then keep filling the bit set with remaining terms
do {
docs = termsEnum.postings(acceptDocs, docs, PostingsEnum.NONE);
docs = termsEnum.postings(docs, PostingsEnum.NONE);
builder.or(docs);
} while (termsEnum.next() != null);
@ -194,10 +193,10 @@ final class MultiTermQueryConstantScoreWrapper<Q extends MultiTermQuery> extends
}
@Override
public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
final WeightOrBitSet weightOrBitSet = rewrite(context, acceptDocs);
public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
final WeightOrBitSet weightOrBitSet = rewrite(context);
if (weightOrBitSet.weight != null) {
return weightOrBitSet.weight.bulkScorer(context, acceptDocs);
return weightOrBitSet.weight.bulkScorer(context);
} else {
final Scorer scorer = scorer(weightOrBitSet.bitset);
if (scorer == null) {
@ -208,10 +207,10 @@ final class MultiTermQueryConstantScoreWrapper<Q extends MultiTermQuery> extends
}
@Override
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
final WeightOrBitSet weightOrBitSet = rewrite(context, acceptDocs);
public Scorer scorer(LeafReaderContext context) throws IOException {
final WeightOrBitSet weightOrBitSet = rewrite(context);
if (weightOrBitSet.weight != null) {
return weightOrBitSet.weight.scorer(context, acceptDocs);
return weightOrBitSet.weight.scorer(context);
} else {
return scorer(weightOrBitSet.bitset);
}

View File

@ -37,7 +37,6 @@ import org.apache.lucene.index.TermsEnum;
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.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.ToStringUtils;
@ -396,10 +395,9 @@ public class PhraseQuery extends Query {
}
@Override
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
public Scorer scorer(LeafReaderContext context) throws IOException {
assert terms.length > 0;
final LeafReader reader = context.reader();
final Bits liveDocs = acceptDocs;
PostingsAndFreq[] postingsFreqs = new PostingsAndFreq[terms.length];
final Terms fieldTerms = reader.terms(field);
@ -422,7 +420,7 @@ public class PhraseQuery extends Query {
return null;
}
te.seekExact(t.bytes(), state);
PostingsEnum postingsEnum = te.postings(liveDocs, null, PostingsEnum.POSITIONS);
PostingsEnum postingsEnum = te.postings(null, PostingsEnum.POSITIONS);
postingsFreqs[i] = new PostingsAndFreq(postingsEnum, positions[i], t);
}
@ -445,7 +443,7 @@ public class PhraseQuery extends Query {
@Override
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
Scorer scorer = scorer(context, context.reader().getLiveDocs());
Scorer scorer = scorer(context);
if (scorer != null) {
int newDoc = scorer.advance(doc);
if (newDoc == doc) {

View File

@ -82,7 +82,7 @@ public abstract class QueryRescorer extends Rescorer {
if (readerContext != null) {
// We advanced to another segment:
docBase = readerContext.docBase;
scorer = weight.scorer(readerContext, null);
scorer = weight.scorer(readerContext);
}
if(scorer != null) {

View File

@ -61,10 +61,11 @@ public class QueryWrapperFilter extends Filter {
// get a private context that is used to rewrite, createWeight and score eventually
final LeafReaderContext privateContext = context.reader().getContext();
final Weight weight = new IndexSearcher(privateContext).createNormalizedWeight(query, false);
return new DocIdSet() {
DocIdSet set = new DocIdSet() {
@Override
public DocIdSetIterator iterator() throws IOException {
return weight.scorer(privateContext, acceptDocs);
return weight.scorer(privateContext);
}
@Override
@ -72,6 +73,7 @@ public class QueryWrapperFilter extends Filter {
return 0L;
}
};
return BitsFilteredDocIdSet.wrap(set, acceptDocs);
}
@Override

View File

@ -48,7 +48,7 @@ public abstract class RandomAccessWeight extends ConstantScoreWeight {
protected abstract Bits getMatchingDocs(LeafReaderContext context) throws IOException;
@Override
public final Scorer scorer(LeafReaderContext context, final Bits acceptDocs) throws IOException {
public final Scorer scorer(LeafReaderContext context) throws IOException {
final Bits matchingDocs = getMatchingDocs(context);
if (matchingDocs == null || matchingDocs instanceof MatchNoBits) {
return null;
@ -60,10 +60,6 @@ public abstract class RandomAccessWeight extends ConstantScoreWeight {
public boolean matches() throws IOException {
final int doc = approximation.docID();
if (acceptDocs != null && acceptDocs.get(doc) == false) {
return false;
}
return matchingDocs.get(doc);
}
};

View File

@ -32,7 +32,6 @@ import org.apache.lucene.index.TermState;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.search.similarities.Similarity.SimScorer;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ToStringUtils;
/**
@ -98,13 +97,13 @@ public class TermQuery extends Query {
}
@Override
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
public Scorer scorer(LeafReaderContext context) throws IOException {
assert termStates.topReaderContext == ReaderUtil.getTopLevelContext(context) : "The top-reader used to create Weight (" + termStates.topReaderContext + ") is not the same as the current reader's top-reader (" + ReaderUtil.getTopLevelContext(context);
final TermsEnum termsEnum = getTermsEnum(context);
if (termsEnum == null) {
return null;
}
PostingsEnum docs = termsEnum.postings(acceptDocs, null, needsScores ? PostingsEnum.FREQS : PostingsEnum.NONE);
PostingsEnum docs = termsEnum.postings(null, needsScores ? PostingsEnum.FREQS : PostingsEnum.NONE);
assert docs != null;
return new TermScorer(this, docs, similarity.simScorer(stats, context));
}
@ -136,7 +135,7 @@ public class TermQuery extends Query {
@Override
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
Scorer scorer = scorer(context, context.reader().getLiveDocs());
Scorer scorer = scorer(context);
if (scorer != null) {
int newDoc = scorer.advance(doc);
if (newDoc == doc) {

View File

@ -36,7 +36,7 @@ import org.apache.lucene.util.Bits;
* {@link org.apache.lucene.index.LeafReader} dependent state should reside in the {@link Scorer}.
* <p>
* Since {@link Weight} creates {@link Scorer} instances for a given
* {@link org.apache.lucene.index.LeafReaderContext} ({@link #scorer(org.apache.lucene.index.LeafReaderContext, Bits)})
* {@link org.apache.lucene.index.LeafReaderContext} ({@link #scorer(org.apache.lucene.index.LeafReaderContext)})
* callers must maintain the relationship between the searcher's top-level
* {@link IndexReaderContext} and the context used to create a {@link Scorer}.
* <p>
@ -51,7 +51,7 @@ import org.apache.lucene.util.Bits;
* <li>The query normalization factor is passed to {@link #normalize(float, float)}. At
* this point the weighting is complete.
* <li>A <code>Scorer</code> is constructed by
* {@link #scorer(org.apache.lucene.index.LeafReaderContext, Bits)}.
* {@link #scorer(org.apache.lucene.index.LeafReaderContext)}.
* </ol>
*
* @since 2.9
@ -105,14 +105,11 @@ public abstract class Weight {
*
* @param context
* the {@link org.apache.lucene.index.LeafReaderContext} for which to return the {@link Scorer}.
* @param acceptDocs
* Bits that represent the allowable docs to match (typically deleted docs
* but possibly filtering other documents)
*
* @return a {@link Scorer} which scores documents in/out-of order.
* @throws IOException if there is a low-level I/O error
*/
public abstract Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException;
public abstract Scorer scorer(LeafReaderContext context) throws IOException;
/**
* Optional method, to return a {@link BulkScorer} to
@ -124,17 +121,14 @@ public abstract class Weight {
*
* @param context
* the {@link org.apache.lucene.index.LeafReaderContext} for which to return the {@link Scorer}.
* @param acceptDocs
* Bits that represent the allowable docs to match (typically deleted docs
* but possibly filtering other documents)
*
* @return a {@link BulkScorer} which scores documents and
* passes them to a collector.
* @throws IOException if there is a low-level I/O error
*/
public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
Scorer scorer = scorer(context, acceptDocs);
Scorer scorer = scorer(context);
if (scorer == null) {
// No docs match
return null;
@ -164,23 +158,22 @@ public abstract class Weight {
}
@Override
public int score(LeafCollector collector, int min, int max) throws IOException {
// TODO: this may be sort of weird, when we are
// embedded in a BooleanScorer, because we are
// called for every chunk of 2048 documents. But,
// then, scorer is a FakeScorer in that case, so any
// Collector doing something "interesting" in
// setScorer will be forced to use BS2 anyways:
public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
collector.setScorer(scorer);
final TwoPhaseIterator twoPhase = scorer.asTwoPhaseIterator();
if (scorer.docID() == -1 && min == 0 && max == DocIdSetIterator.NO_MORE_DOCS) {
scoreAll(collector, scorer);
scoreAll(collector, scorer, twoPhase, acceptDocs);
return DocIdSetIterator.NO_MORE_DOCS;
} else {
int doc = scorer.docID();
if (doc < min) {
doc = scorer.advance(min);
if (twoPhase == null) {
doc = scorer.advance(min);
} else {
doc = twoPhase.approximation().advance(min);
}
}
return scoreRange(collector, scorer, doc, max);
return scoreRange(collector, scorer, twoPhase, acceptDocs, doc, max);
}
}
@ -188,21 +181,47 @@ public abstract class Weight {
* separate this from {@link #scoreAll} to help out
* hotspot.
* See <a href="https://issues.apache.org/jira/browse/LUCENE-5487">LUCENE-5487</a> */
static int scoreRange(LeafCollector collector, Scorer scorer, int currentDoc, int end) throws IOException {
while (currentDoc < end) {
collector.collect(currentDoc);
currentDoc = scorer.nextDoc();
static int scoreRange(LeafCollector collector, Scorer scorer, TwoPhaseIterator twoPhase,
Bits acceptDocs, int currentDoc, int end) throws IOException {
if (twoPhase == null) {
while (currentDoc < end) {
if (acceptDocs == null || acceptDocs.get(currentDoc)) {
collector.collect(currentDoc);
}
currentDoc = scorer.nextDoc();
}
return currentDoc;
} else {
final DocIdSetIterator approximation = twoPhase.approximation();
while (currentDoc < end) {
if ((acceptDocs == null || acceptDocs.get(currentDoc)) && twoPhase.matches()) {
collector.collect(currentDoc);
}
currentDoc = approximation.nextDoc();
}
return currentDoc;
}
return currentDoc;
}
/** Specialized method to bulk-score all hits; we
* separate this from {@link #scoreRange} to help out
* hotspot.
* See <a href="https://issues.apache.org/jira/browse/LUCENE-5487">LUCENE-5487</a> */
static void scoreAll(LeafCollector collector, Scorer scorer) throws IOException {
for (int doc = scorer.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = scorer.nextDoc()) {
collector.collect(doc);
static void scoreAll(LeafCollector collector, Scorer scorer, TwoPhaseIterator twoPhase, Bits acceptDocs) throws IOException {
if (twoPhase == null) {
for (int doc = scorer.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = scorer.nextDoc()) {
if (acceptDocs == null || acceptDocs.get(doc)) {
collector.collect(doc);
}
}
} else {
// The scorer has an approximation, so run the approximation first, then check acceptDocs, then confirm
final DocIdSetIterator approximation = twoPhase.approximation();
for (int doc = approximation.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = approximation.nextDoc()) {
if ((acceptDocs == null || acceptDocs.get(doc)) && twoPhase.matches()) {
collector.collect(doc);
}
}
}
}
}

View File

@ -470,7 +470,7 @@
* abstract method:
* <ol>
* <li>
* {@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.LeafCollector,int,int) score(LeafCollector,int,int)} &mdash;
* {@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.LeafCollector,org.apache.lucene.util.Bits,int,int) score(LeafCollector,Bits,int,int)} &mdash;
* Score all documents up to but not including the specified max document.
* </li>
* </ol>
@ -522,7 +522,7 @@
* <p>If a Filter is being used, some initial setup is done to determine which docs to include.
* Otherwise, we ask the Weight for a {@link org.apache.lucene.search.Scorer Scorer} for each
* {@link org.apache.lucene.index.IndexReader IndexReader} segment and proceed by calling
* {@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.LeafCollector) BulkScorer.score(LeafCollector)}.
* {@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.LeafCollector,org.apache.lucene.util.Bits) BulkScorer.score(LeafCollector,Bits)}.
* <p>At last, we are actually going to score some documents. The score method takes in the Collector
* (most likely the TopScoreDocCollector or TopFieldCollector) and does its business.Of course, here
* is where things get involved. The {@link org.apache.lucene.search.Scorer Scorer} that is returned

View File

@ -144,15 +144,15 @@ public class PayloadNearQuery extends SpanNearQuery {
}
@Override
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
Spans spans = super.getSpans(context, acceptDocs, Postings.PAYLOADS);
public Scorer scorer(LeafReaderContext context) throws IOException {
Spans spans = super.getSpans(context, Postings.PAYLOADS);
Similarity.SimScorer simScorer = simWeight == null ? null : similarity.simScorer(simWeight, context);
return (spans == null) ? null : new PayloadNearSpanScorer(spans, this, simScorer);
}
@Override
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
PayloadNearSpanScorer scorer = (PayloadNearSpanScorer) scorer(context, context.reader().getLiveDocs());
PayloadNearSpanScorer scorer = (PayloadNearSpanScorer) scorer(context);
if (scorer != null) {
int newDoc = scorer.advance(doc);
if (newDoc == doc) {

View File

@ -179,7 +179,7 @@ public class PayloadSpanUtil {
PayloadSpanCollector collector = new PayloadSpanCollector();
for (LeafReaderContext leafReaderContext : context.leaves()) {
final Spans spans = w.getSpans(leafReaderContext, leafReaderContext.reader().getLiveDocs(), SpanWeight.Postings.PAYLOADS);
final Spans spans = w.getSpans(leafReaderContext, SpanWeight.Postings.PAYLOADS);
if (spans != null) {
while (spans.nextDoc() != Spans.NO_MORE_DOCS) {
while (spans.nextStartPosition() != Spans.NO_MORE_POSITIONS) {

View File

@ -32,7 +32,6 @@ import org.apache.lucene.search.spans.SpanScorer;
import org.apache.lucene.search.spans.SpanTermQuery;
import org.apache.lucene.search.spans.SpanWeight;
import org.apache.lucene.search.spans.Spans;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import java.io.IOException;
@ -98,8 +97,8 @@ public class PayloadTermQuery extends SpanTermQuery {
}
@Override
public PayloadTermSpanScorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
Spans spans = super.getSpans(context, acceptDocs, Postings.PAYLOADS);
public PayloadTermSpanScorer scorer(LeafReaderContext context) throws IOException {
Spans spans = super.getSpans(context, Postings.PAYLOADS);
Similarity.SimScorer simScorer = simWeight == null ? null : similarity.simScorer(simWeight, context);
return (spans == null) ? null : new PayloadTermSpanScorer(spans, this, simScorer);
}
@ -184,7 +183,7 @@ public class PayloadTermQuery extends SpanTermQuery {
@Override
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
PayloadTermSpanScorer scorer = scorer(context, context.reader().getLiveDocs());
PayloadTermSpanScorer scorer = scorer(context);
if (scorer != null) {
int newDoc = scorer.advance(doc);
if (newDoc == doc) {

View File

@ -36,7 +36,6 @@ import org.apache.lucene.search.spans.SpanQuery;
import org.apache.lucene.search.spans.SpanScorer;
import org.apache.lucene.search.spans.SpanWeight;
import org.apache.lucene.search.spans.Spans;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ToStringUtils;
/**
@ -90,9 +89,9 @@ public class SpanPayloadCheckQuery extends SpanQuery {
}
@Override
public Spans getSpans(final LeafReaderContext context, Bits acceptDocs, Postings requiredPostings) throws IOException {
public Spans getSpans(final LeafReaderContext context, Postings requiredPostings) throws IOException {
final PayloadSpanCollector collector = new PayloadSpanCollector();
Spans matchSpans = matchWeight.getSpans(context, acceptDocs, requiredPostings.atLeast(Postings.PAYLOADS));
Spans matchSpans = matchWeight.getSpans(context, requiredPostings.atLeast(Postings.PAYLOADS));
return (matchSpans == null) ? null : new FilterSpans(matchSpans) {
@Override
protected AcceptStatus accept(Spans candidate) throws IOException {
@ -104,7 +103,7 @@ public class SpanPayloadCheckQuery extends SpanQuery {
}
@Override
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
public Scorer scorer(LeafReaderContext context) throws IOException {
if (field == null)
return null;
@ -113,7 +112,7 @@ public class SpanPayloadCheckQuery extends SpanQuery {
throw new IllegalStateException("field \"" + field + "\" was indexed without position data; cannot run SpanQuery (query=" + parentQuery + ")");
}
Spans spans = getSpans(context, acceptDocs, Postings.PAYLOADS);
Spans spans = getSpans(context, Postings.PAYLOADS);
Similarity.SimScorer simScorer = simWeight == null ? null : similarity.simScorer(simWeight, context);
return (spans == null) ? null : new SpanScorer(spans, this, simScorer);
}

View File

@ -23,7 +23,6 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.index.TermContext;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.Bits;
import java.io.IOException;
import java.util.ArrayList;
@ -71,12 +70,12 @@ abstract class SpanContainQuery extends SpanQuery implements Cloneable {
littleWeight.extractTerms(terms);
}
ArrayList<Spans> prepareConjunction(final LeafReaderContext context, final Bits acceptDocs, Postings postings) throws IOException {
Spans bigSpans = bigWeight.getSpans(context, acceptDocs, postings);
ArrayList<Spans> prepareConjunction(final LeafReaderContext context, Postings postings) throws IOException {
Spans bigSpans = bigWeight.getSpans(context, postings);
if (bigSpans == null) {
return null;
}
Spans littleSpans = littleWeight.getSpans(context, acceptDocs, postings);
Spans littleSpans = littleWeight.getSpans(context, postings);
if (littleSpans == null) {
return null;
}

View File

@ -70,8 +70,8 @@ public class SpanContainingQuery extends SpanContainQuery {
* The payload is from the spans of <code>big</code>.
*/
@Override
public Spans getSpans(final LeafReaderContext context, final Bits acceptDocs, Postings requiredPostings) throws IOException {
ArrayList<Spans> containerContained = prepareConjunction(context, acceptDocs, requiredPostings);
public Spans getSpans(final LeafReaderContext context, Postings requiredPostings) throws IOException {
ArrayList<Spans> containerContained = prepareConjunction(context, requiredPostings);
if (containerContained == null) {
return null;
}

View File

@ -31,7 +31,6 @@ import org.apache.lucene.index.TermContext;
import org.apache.lucene.index.Terms;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ToStringUtils;
/** Matches spans which are near one another. One can specify <i>slop</i>, the
@ -131,7 +130,7 @@ public class SpanNearQuery extends SpanQuery implements Cloneable {
}
@Override
public Spans getSpans(final LeafReaderContext context, Bits acceptDocs, Postings requiredPostings) throws IOException {
public Spans getSpans(final LeafReaderContext context, Postings requiredPostings) throws IOException {
Terms terms = context.reader().terms(field);
if (terms == null) {
@ -140,7 +139,7 @@ public class SpanNearQuery extends SpanQuery implements Cloneable {
ArrayList<Spans> subSpans = new ArrayList<>(clauses.size());
for (SpanWeight w : subWeights) {
Spans subSpan = w.getSpans(context, acceptDocs, requiredPostings);
Spans subSpan = w.getSpans(context, requiredPostings);
if (subSpan != null) {
subSpans.add(subSpan);
} else {

View File

@ -25,7 +25,6 @@ import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TwoPhaseIterator;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ToStringUtils;
import java.io.IOException;
@ -128,13 +127,13 @@ public class SpanNotQuery extends SpanQuery implements Cloneable {
}
@Override
public Spans getSpans(final LeafReaderContext context, final Bits acceptDocs, Postings requiredPostings) throws IOException {
Spans includeSpans = includeWeight.getSpans(context, acceptDocs, requiredPostings);
public Spans getSpans(final LeafReaderContext context, Postings requiredPostings) throws IOException {
Spans includeSpans = includeWeight.getSpans(context, requiredPostings);
if (includeSpans == null) {
return null;
}
Spans excludeSpans = excludeWeight.getSpans(context, acceptDocs, requiredPostings);
Spans excludeSpans = excludeWeight.getSpans(context, requiredPostings);
if (excludeSpans == null) {
return includeSpans;
}

View File

@ -27,7 +27,6 @@ import org.apache.lucene.search.DisjunctionDISIApproximation;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TwoPhaseIterator;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ToStringUtils;
import java.io.IOException;
@ -170,13 +169,13 @@ public class SpanOrQuery extends SpanQuery implements Cloneable {
}
@Override
public Spans getSpans(final LeafReaderContext context, final Bits acceptDocs, Postings requiredPostings)
public Spans getSpans(final LeafReaderContext context, Postings requiredPostings)
throws IOException {
ArrayList<Spans> subSpans = new ArrayList<>(clauses.size());
for (SpanWeight w : subWeights) {
Spans spans = w.getSpans(context, acceptDocs, requiredPostings);
Spans spans = w.getSpans(context, requiredPostings);
if (spans != null) {
subSpans.add(spans);
}

View File

@ -24,7 +24,6 @@ import org.apache.lucene.index.TermContext;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.spans.FilterSpans.AcceptStatus;
import org.apache.lucene.util.Bits;
import java.io.IOException;
import java.util.Map;
@ -93,8 +92,8 @@ public abstract class SpanPositionCheckQuery extends SpanQuery implements Clonea
}
@Override
public Spans getSpans(final LeafReaderContext context, Bits acceptDocs, Postings requiredPostings) throws IOException {
Spans matchSpans = matchWeight.getSpans(context, acceptDocs, requiredPostings);
public Spans getSpans(final LeafReaderContext context, Postings requiredPostings) throws IOException {
Spans matchSpans = matchWeight.getSpans(context, requiredPostings);
return (matchSpans == null) ? null : new FilterSpans(matchSpans) {
@Override
protected AcceptStatus accept(Spans candidate) throws IOException {

View File

@ -27,7 +27,6 @@ import org.apache.lucene.index.TermState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ToStringUtils;
import java.io.IOException;
@ -99,7 +98,7 @@ public class SpanTermQuery extends SpanQuery {
}
@Override
public Spans getSpans(final LeafReaderContext context, Bits acceptDocs, Postings requiredPostings) throws IOException {
public Spans getSpans(final LeafReaderContext context, Postings requiredPostings) throws IOException {
assert termContext.topReaderContext == ReaderUtil.getTopLevelContext(context) : "The top-reader used to create Weight (" + termContext.topReaderContext + ") is not the same as the current reader's top-reader (" + ReaderUtil.getTopLevelContext(context);
@ -118,7 +117,7 @@ public class SpanTermQuery extends SpanQuery {
final TermsEnum termsEnum = terms.iterator();
termsEnum.seekExact(term.bytes(), state);
final PostingsEnum postings = termsEnum.postings(acceptDocs, null, requiredPostings.getRequiredPostings());
final PostingsEnum postings = termsEnum.postings(null, requiredPostings.getRequiredPostings());
return new TermSpans(postings, term);
}
}

View File

@ -114,11 +114,10 @@ public abstract class SpanWeight extends Weight {
/**
* Expert: Return a Spans object iterating over matches from this Weight
* @param ctx a LeafReaderContext for this Spans
* @param acceptDocs a bitset of documents to check
* @return a Spans
* @throws IOException on error
*/
public abstract Spans getSpans(LeafReaderContext ctx, Bits acceptDocs, Postings requiredPostings) throws IOException;
public abstract Spans getSpans(LeafReaderContext ctx, Postings requiredPostings) throws IOException;
@Override
public float getValueForNormalization() throws IOException {
@ -133,7 +132,7 @@ public abstract class SpanWeight extends Weight {
}
@Override
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
public Scorer scorer(LeafReaderContext context) throws IOException {
if (field == null) {
return null;
}
@ -141,14 +140,14 @@ public abstract class SpanWeight extends Weight {
if (terms != null && terms.hasPositions() == false) {
throw new IllegalStateException("field \"" + field + "\" was indexed without position data; cannot run SpanQuery (query=" + parentQuery + ")");
}
Spans spans = getSpans(context, acceptDocs, Postings.POSITIONS);
Spans spans = getSpans(context, Postings.POSITIONS);
Similarity.SimScorer simScorer = simWeight == null ? null : similarity.simScorer(simWeight, context);
return (spans == null) ? null : new SpanScorer(spans, this, simScorer);
}
@Override
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
SpanScorer scorer = (SpanScorer) scorer(context, context.reader().getLiveDocs());
SpanScorer scorer = (SpanScorer) scorer(context);
if (scorer != null) {
int newDoc = scorer.advance(doc);
if (newDoc == doc) {

View File

@ -21,7 +21,6 @@ import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TermContext;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.util.Bits;
import java.io.IOException;
import java.util.ArrayList;
@ -71,8 +70,8 @@ public class SpanWithinQuery extends SpanContainQuery {
* The payload is from the spans of <code>little</code>.
*/
@Override
public Spans getSpans(final LeafReaderContext context, final Bits acceptDocs, Postings requiredPostings) throws IOException {
ArrayList<Spans> containerContained = prepareConjunction(context, acceptDocs, requiredPostings);
public Spans getSpans(final LeafReaderContext context, Postings requiredPostings) throws IOException {
ArrayList<Spans> containerContained = prepareConjunction(context, requiredPostings);
if (containerContained == null) {
return null;
}

View File

@ -85,7 +85,6 @@ public class TestCachingTokenFilter extends BaseTokenStreamTestCase {
IndexReader reader = writer.getReader();
PostingsEnum termPositions = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
"preanalyzed",
new BytesRef("term1"));
assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@ -93,7 +92,6 @@ public class TestCachingTokenFilter extends BaseTokenStreamTestCase {
assertEquals(0, termPositions.nextPosition());
termPositions = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
"preanalyzed",
new BytesRef("term2"));
assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@ -102,7 +100,6 @@ public class TestCachingTokenFilter extends BaseTokenStreamTestCase {
assertEquals(3, termPositions.nextPosition());
termPositions = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
"preanalyzed",
new BytesRef("term3"));
assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);

View File

@ -283,7 +283,6 @@ public class TestBlockPostingsFormat3 extends LuceneTestCase {
*/
public void assertTermsEnum(TermsEnum leftTermsEnum, TermsEnum rightTermsEnum, boolean deep, boolean hasPositions) throws Exception {
BytesRef term;
Bits randomBits = new RandomBits(MAXDOC, random().nextDouble(), random());
PostingsEnum leftPositions = null;
PostingsEnum rightPositions = null;
PostingsEnum leftDocs = null;
@ -295,84 +294,54 @@ public class TestBlockPostingsFormat3 extends LuceneTestCase {
if (deep) {
if (hasPositions) {
// with payloads + off
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.postings(null, leftPositions, PostingsEnum.ALL),
rightPositions = rightTermsEnum.postings(null, rightPositions, PostingsEnum.ALL));
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.postings(randomBits, leftPositions, PostingsEnum.ALL),
rightPositions = rightTermsEnum.postings(randomBits, rightPositions, PostingsEnum.ALL));
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.postings(leftPositions, PostingsEnum.ALL),
rightPositions = rightTermsEnum.postings(rightPositions, PostingsEnum.ALL));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.postings(null, leftPositions, PostingsEnum.ALL),
rightPositions = rightTermsEnum.postings(null, rightPositions, PostingsEnum.ALL));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.postings(randomBits, leftPositions, PostingsEnum.ALL),
rightPositions = rightTermsEnum.postings(randomBits, rightPositions, PostingsEnum.ALL));
leftPositions = leftTermsEnum.postings(leftPositions, PostingsEnum.ALL),
rightPositions = rightTermsEnum.postings(rightPositions, PostingsEnum.ALL));
// with payloads only
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.postings(null, leftPositions, PostingsEnum.PAYLOADS),
rightPositions = rightTermsEnum.postings(null, rightPositions, PostingsEnum.PAYLOADS));
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.postings(randomBits, leftPositions, PostingsEnum.PAYLOADS),
rightPositions = rightTermsEnum.postings(randomBits, rightPositions, PostingsEnum.PAYLOADS));
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.postings(leftPositions, PostingsEnum.PAYLOADS),
rightPositions = rightTermsEnum.postings(rightPositions, PostingsEnum.PAYLOADS));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.postings(null, leftPositions, PostingsEnum.PAYLOADS),
rightPositions = rightTermsEnum.postings(null, rightPositions, PostingsEnum.PAYLOADS));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.postings(randomBits, leftPositions, PostingsEnum.PAYLOADS),
rightPositions = rightTermsEnum.postings(randomBits, rightPositions, PostingsEnum.PAYLOADS));
leftPositions = leftTermsEnum.postings(leftPositions, PostingsEnum.PAYLOADS),
rightPositions = rightTermsEnum.postings(rightPositions, PostingsEnum.PAYLOADS));
// with offsets only
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.postings(null, leftPositions, PostingsEnum.OFFSETS),
rightPositions = rightTermsEnum.postings(null, rightPositions, PostingsEnum.OFFSETS));
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.postings(randomBits, leftPositions, PostingsEnum.OFFSETS),
rightPositions = rightTermsEnum.postings(randomBits, rightPositions, PostingsEnum.OFFSETS));
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.postings(leftPositions, PostingsEnum.OFFSETS),
rightPositions = rightTermsEnum.postings(rightPositions, PostingsEnum.OFFSETS));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.postings(null, leftPositions, PostingsEnum.OFFSETS),
rightPositions = rightTermsEnum.postings(null, rightPositions, PostingsEnum.OFFSETS));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.postings(randomBits, leftPositions, PostingsEnum.OFFSETS),
rightPositions = rightTermsEnum.postings(randomBits, rightPositions, PostingsEnum.OFFSETS));
leftPositions = leftTermsEnum.postings(leftPositions, PostingsEnum.OFFSETS),
rightPositions = rightTermsEnum.postings(rightPositions, PostingsEnum.OFFSETS));
// with positions only
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.postings(null, leftPositions, PostingsEnum.POSITIONS),
rightPositions = rightTermsEnum.postings(null, rightPositions, PostingsEnum.POSITIONS));
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.postings(randomBits, leftPositions, PostingsEnum.POSITIONS),
rightPositions = rightTermsEnum.postings(randomBits, rightPositions, PostingsEnum.POSITIONS));
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.postings(leftPositions, PostingsEnum.POSITIONS),
rightPositions = rightTermsEnum.postings(rightPositions, PostingsEnum.POSITIONS));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.postings(null, leftPositions, PostingsEnum.POSITIONS),
rightPositions = rightTermsEnum.postings(null, rightPositions, PostingsEnum.POSITIONS));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.postings(randomBits, leftPositions, PostingsEnum.POSITIONS),
rightPositions = rightTermsEnum.postings(randomBits, rightPositions, PostingsEnum.POSITIONS));
leftPositions = leftTermsEnum.postings(leftPositions, PostingsEnum.POSITIONS),
rightPositions = rightTermsEnum.postings(rightPositions, PostingsEnum.POSITIONS));
}
// with freqs:
assertDocsEnum(leftDocs = leftTermsEnum.postings(null, leftDocs),
rightDocs = rightTermsEnum.postings(null, rightDocs));
assertDocsEnum(leftDocs = leftTermsEnum.postings(randomBits, leftDocs),
rightDocs = rightTermsEnum.postings(randomBits, rightDocs));
assertDocsEnum(leftDocs = leftTermsEnum.postings(leftDocs),
rightDocs = rightTermsEnum.postings(rightDocs));
// w/o freqs:
assertDocsEnum(leftDocs = leftTermsEnum.postings(null, leftDocs, PostingsEnum.NONE),
rightDocs = rightTermsEnum.postings(null, rightDocs, PostingsEnum.NONE));
assertDocsEnum(leftDocs = leftTermsEnum.postings(randomBits, leftDocs, PostingsEnum.NONE),
rightDocs = rightTermsEnum.postings(randomBits, rightDocs, PostingsEnum.NONE));
assertDocsEnum(leftDocs = leftTermsEnum.postings(leftDocs, PostingsEnum.NONE),
rightDocs = rightTermsEnum.postings(rightDocs, PostingsEnum.NONE));
// with freqs:
assertDocsSkipping(leftTermsEnum.docFreq(),
leftDocs = leftTermsEnum.postings(null, leftDocs),
rightDocs = rightTermsEnum.postings(null, rightDocs));
assertDocsSkipping(leftTermsEnum.docFreq(),
leftDocs = leftTermsEnum.postings(randomBits, leftDocs),
rightDocs = rightTermsEnum.postings(randomBits, rightDocs));
leftDocs = leftTermsEnum.postings(leftDocs),
rightDocs = rightTermsEnum.postings(rightDocs));
// w/o freqs:
assertDocsSkipping(leftTermsEnum.docFreq(),
leftDocs = leftTermsEnum.postings(null, leftDocs, PostingsEnum.NONE),
rightDocs = rightTermsEnum.postings(null, rightDocs, PostingsEnum.NONE));
assertDocsSkipping(leftTermsEnum.docFreq(),
leftDocs = leftTermsEnum.postings(randomBits, leftDocs, PostingsEnum.NONE),
rightDocs = rightTermsEnum.postings(randomBits, rightDocs, PostingsEnum.NONE));
leftDocs = leftTermsEnum.postings(leftDocs, PostingsEnum.NONE),
rightDocs = rightTermsEnum.postings(rightDocs, PostingsEnum.NONE));
}
}
assertNull(rightTermsEnum.next());

View File

@ -532,7 +532,7 @@ public class TestAddIndexes extends LuceneTestCase {
private void verifyTermDocs(Directory dir, Term term, int numDocs)
throws IOException {
IndexReader reader = DirectoryReader.open(dir);
PostingsEnum postingsEnum = TestUtil.docs(random(), reader, term.field, term.bytes, null, null, PostingsEnum.NONE);
PostingsEnum postingsEnum = TestUtil.docs(random(), reader, term.field, term.bytes, null, PostingsEnum.NONE);
int count = 0;
while (postingsEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS)
count++;

View File

@ -245,7 +245,7 @@ public class TestCodecs extends LuceneTestCase {
// make sure it properly fully resets (rewinds) its
// internal state:
for(int iter=0;iter<2;iter++) {
postingsEnum = TestUtil.docs(random(), termsEnum, null, postingsEnum, PostingsEnum.NONE);
postingsEnum = TestUtil.docs(random(), termsEnum, postingsEnum, PostingsEnum.NONE);
assertEquals(terms[i].docs[0], postingsEnum.nextDoc());
assertEquals(DocIdSetIterator.NO_MORE_DOCS, postingsEnum.nextDoc());
}
@ -382,9 +382,9 @@ public class TestCodecs extends LuceneTestCase {
assertEquals(status, TermsEnum.SeekStatus.FOUND);
assertEquals(term.docs.length, termsEnum.docFreq());
if (field.omitTF) {
this.verifyDocs(term.docs, term.positions, TestUtil.docs(random(), termsEnum, null, null, PostingsEnum.NONE), false);
this.verifyDocs(term.docs, term.positions, TestUtil.docs(random(), termsEnum, null, PostingsEnum.NONE), false);
} else {
this.verifyDocs(term.docs, term.positions, termsEnum.postings(null, null, PostingsEnum.ALL), true);
this.verifyDocs(term.docs, term.positions, termsEnum.postings(null, PostingsEnum.ALL), true);
}
// Test random seek by ord:
@ -402,9 +402,9 @@ public class TestCodecs extends LuceneTestCase {
assertTrue(termsEnum.term().bytesEquals(new BytesRef(term.text2)));
assertEquals(term.docs.length, termsEnum.docFreq());
if (field.omitTF) {
this.verifyDocs(term.docs, term.positions, TestUtil.docs(random(), termsEnum, null, null, PostingsEnum.NONE), false);
this.verifyDocs(term.docs, term.positions, TestUtil.docs(random(), termsEnum, null, PostingsEnum.NONE), false);
} else {
this.verifyDocs(term.docs, term.positions, termsEnum.postings(null, null, PostingsEnum.ALL), true);
this.verifyDocs(term.docs, term.positions, termsEnum.postings(null, PostingsEnum.ALL), true);
}
}
@ -456,9 +456,9 @@ public class TestCodecs extends LuceneTestCase {
if (!field.omitTF) {
// TODO: we should randomize which postings features are available, but
// need to coordinate this with the checks below that rely on such features
postings = termsEnum.postings(null, null, PostingsEnum.ALL);
postings = termsEnum.postings(null, PostingsEnum.ALL);
} else {
postings = TestUtil.docs(random(), termsEnum, null, null, PostingsEnum.FREQS);
postings = TestUtil.docs(random(), termsEnum, null, PostingsEnum.FREQS);
}
assertNotNull(postings);
int upto2 = -1;
@ -677,8 +677,7 @@ public class TestCodecs extends LuceneTestCase {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) {
assert liveDocs == null;
public PostingsEnum postings(PostingsEnum reuse, int flags) {
return new DataPostingsEnum(fieldData.terms[upto]);
}

View File

@ -96,13 +96,12 @@ public class TestDirectoryReader extends LuceneTestCase {
PostingsEnum td = TestUtil.docs(random(), mr2,
"body",
te2.term(),
MultiFields.getLiveDocs(mr2),
null,
0);
TermsEnum te3 = MultiFields.getTerms(mr3, "body").iterator();
te3.seekCeil(new BytesRef("wow"));
td = TestUtil.docs(random(), te3, MultiFields.getLiveDocs(mr3),
td = TestUtil.docs(random(), te3,
td,
0);
@ -351,7 +350,6 @@ void assertTermDocsCount(String msg,
PostingsEnum tdocs = TestUtil.docs(random(), reader,
term.field(),
new BytesRef(term.text()),
MultiFields.getLiveDocs(reader),
null,
0);
int count = 0;
@ -617,7 +615,6 @@ public void testFilesOpenClose() throws IOException {
Fields fields1 = MultiFields.getFields(index1);
Fields fields2 = MultiFields.getFields(index2);
Iterator<String> fenum2 = fields2.iterator();
Bits liveDocs = MultiFields.getLiveDocs(index1);
for (String field1 : fields1) {
assertEquals("Different fields", field1, fenum2.next());
Terms terms1 = fields1.terms(field1);
@ -633,8 +630,8 @@ public void testFilesOpenClose() throws IOException {
while(enum1.next() != null) {
assertEquals("Different terms", enum1.term(), enum2.next());
PostingsEnum tp1 = enum1.postings(liveDocs, null, PostingsEnum.ALL);
PostingsEnum tp2 = enum2.postings(liveDocs, null, PostingsEnum.ALL);
PostingsEnum tp1 = enum1.postings(null, PostingsEnum.ALL);
PostingsEnum tp2 = enum2.postings(null, PostingsEnum.ALL);
while(tp1.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
assertTrue(tp2.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);

View File

@ -44,6 +44,7 @@ import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.MergeInfo;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.TrackingDirectoryWrapper;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.StringHelper;
@ -262,9 +263,13 @@ public class TestDoc extends LuceneTestCase {
out.print(" term=" + field + ":" + tis.term());
out.println(" DF=" + tis.docFreq());
PostingsEnum positions = tis.postings(reader.getLiveDocs(), null, PostingsEnum.POSITIONS);
PostingsEnum positions = tis.postings(null, PostingsEnum.POSITIONS);
final Bits liveDocs = reader.getLiveDocs();
while (positions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
if (liveDocs != null && liveDocs.get(positions.docID()) == false) {
continue;
}
out.print(" doc=" + positions.docID());
out.print(" TF=" + positions.freq());
out.print(" pos=");

View File

@ -67,7 +67,7 @@ public class TestDocCount extends LuceneTestCase {
FixedBitSet visited = new FixedBitSet(ir.maxDoc());
TermsEnum te = terms.iterator();
while (te.next() != null) {
PostingsEnum de = TestUtil.docs(random(), te, null, null, PostingsEnum.NONE);
PostingsEnum de = TestUtil.docs(random(), te, null, PostingsEnum.NONE);
while (de.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
visited.set(de.docID());
}

View File

@ -66,7 +66,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
IndexReaderContext topReaderContext = reader.getContext();
for (LeafReaderContext leafReaderContext : topReaderContext.leaves()) {
PostingsEnum docsAndPosEnum = getDocsAndPositions(
leafReaderContext.reader(), bytes, null);
leafReaderContext.reader(), bytes);
assertNotNull(docsAndPosEnum);
if (leafReaderContext.reader().maxDoc() == 0) {
continue;
@ -91,12 +91,12 @@ public class TestDocsAndPositions extends LuceneTestCase {
}
public PostingsEnum getDocsAndPositions(LeafReader reader,
BytesRef bytes, Bits liveDocs) throws IOException {
BytesRef bytes) throws IOException {
Terms terms = reader.terms(fieldName);
if (terms != null) {
TermsEnum te = terms.iterator();
if (te.seekExact(bytes)) {
return te.postings(liveDocs, null, PostingsEnum.ALL);
return te.postings(null, PostingsEnum.ALL);
}
}
return null;
@ -149,7 +149,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
IndexReaderContext topReaderContext = reader.getContext();
for (LeafReaderContext leafReaderContext : topReaderContext.leaves()) {
PostingsEnum docsAndPosEnum = getDocsAndPositions(
leafReaderContext.reader(), bytes, null);
leafReaderContext.reader(), bytes);
assertNotNull(docsAndPosEnum);
int initDoc = 0;
int maxDoc = leafReaderContext.reader().maxDoc();
@ -226,7 +226,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
IndexReaderContext topReaderContext = reader.getContext();
for (LeafReaderContext context : topReaderContext.leaves()) {
int maxDoc = context.reader().maxDoc();
PostingsEnum postingsEnum = TestUtil.docs(random(), context.reader(), fieldName, bytes, null, null, PostingsEnum.FREQS);
PostingsEnum postingsEnum = TestUtil.docs(random(), context.reader(), fieldName, bytes, null, PostingsEnum.FREQS);
if (findNext(freqInDoc, context.docBase, context.docBase + maxDoc) == Integer.MAX_VALUE) {
assertNull(postingsEnum);
continue;
@ -304,7 +304,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
IndexReaderContext topReaderContext = reader.getContext();
for (LeafReaderContext leafReaderContext : topReaderContext.leaves()) {
PostingsEnum docsAndPosEnum = getDocsAndPositions(
leafReaderContext.reader(), bytes, null);
leafReaderContext.reader(), bytes);
assertNotNull(docsAndPosEnum);
int initDoc = 0;
@ -336,7 +336,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
writer.addDocument(doc);
DirectoryReader reader = writer.getReader();
LeafReader r = getOnlySegmentReader(reader);
PostingsEnum disi = TestUtil.docs(random(), r, "foo", new BytesRef("bar"), null, null, PostingsEnum.NONE);
PostingsEnum disi = TestUtil.docs(random(), r, "foo", new BytesRef("bar"), null, PostingsEnum.NONE);
int docid = disi.docID();
assertEquals(-1, docid);
assertTrue(disi.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@ -344,7 +344,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
// now reuse and check again
TermsEnum te = r.terms("foo").iterator();
assertTrue(te.seekExact(new BytesRef("bar")));
disi = TestUtil.docs(random(), te, null, disi, PostingsEnum.NONE);
disi = TestUtil.docs(random(), te, disi, PostingsEnum.NONE);
docid = disi.docID();
assertEquals(-1, docid);
assertTrue(disi.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@ -369,7 +369,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
// now reuse and check again
TermsEnum te = r.terms("foo").iterator();
assertTrue(te.seekExact(new BytesRef("bar")));
disi = te.postings(null, disi, PostingsEnum.ALL);
disi = te.postings(disi, PostingsEnum.ALL);
docid = disi.docID();
assertEquals(-1, docid);
assertTrue(disi.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);

View File

@ -125,8 +125,7 @@ public class TestDocumentWriter extends LuceneTestCase {
writer.close();
SegmentReader reader = new SegmentReader(info, newIOContext(random()));
PostingsEnum termPositions = MultiFields.getTermPositionsEnum(reader, MultiFields.getLiveDocs(reader),
"repeated", new BytesRef("repeated"));
PostingsEnum termPositions = MultiFields.getTermPositionsEnum(reader, "repeated", new BytesRef("repeated"));
assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
int freq = termPositions.freq();
assertEquals(2, freq);
@ -197,7 +196,7 @@ public class TestDocumentWriter extends LuceneTestCase {
writer.close();
SegmentReader reader = new SegmentReader(info, newIOContext(random()));
PostingsEnum termPositions = MultiFields.getTermPositionsEnum(reader, reader.getLiveDocs(), "f1", new BytesRef("a"));
PostingsEnum termPositions = MultiFields.getTermPositionsEnum(reader, "f1", new BytesRef("a"));
assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
int freq = termPositions.freq();
assertEquals(3, freq);

View File

@ -28,7 +28,6 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.BaseDirectoryWrapper;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
@ -76,8 +75,8 @@ public class TestFilterLeafReader extends LuceneTestCase {
}
@Override
public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
return new TestPositions(super.postings(liveDocs, reuse == null ? null : ((FilterPostingsEnum) reuse).in, flags));
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
return new TestPositions(super.postings(reuse == null ? null : ((FilterPostingsEnum) reuse).in, flags));
}
}
@ -151,7 +150,7 @@ public class TestFilterLeafReader extends LuceneTestCase {
assertEquals(TermsEnum.SeekStatus.FOUND, terms.seekCeil(new BytesRef("one")));
PostingsEnum positions = terms.postings(MultiFields.getLiveDocs(reader), null, PostingsEnum.ALL);
PostingsEnum positions = terms.postings(null, PostingsEnum.ALL);
while (positions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
assertTrue((positions.docID() % 2) == 1);
}

View File

@ -498,7 +498,6 @@ public class TestIndexWriter extends LuceneTestCase {
PostingsEnum td = TestUtil.docs(random(), reader,
"field",
new BytesRef("a"),
MultiFields.getLiveDocs(reader),
null,
PostingsEnum.FREQS);
td.nextDoc();
@ -826,14 +825,14 @@ public class TestIndexWriter extends LuceneTestCase {
Terms tpv = r.getTermVectors(0).terms("field");
TermsEnum termsEnum = tpv.iterator();
assertNotNull(termsEnum.next());
PostingsEnum dpEnum = termsEnum.postings(null, null, PostingsEnum.ALL);
PostingsEnum dpEnum = termsEnum.postings(null, PostingsEnum.ALL);
assertNotNull(dpEnum);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(1, dpEnum.freq());
assertEquals(100, dpEnum.nextPosition());
assertNotNull(termsEnum.next());
dpEnum = termsEnum.postings(null, dpEnum, PostingsEnum.ALL);
dpEnum = termsEnum.postings(dpEnum, PostingsEnum.ALL);
assertNotNull(dpEnum);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(1, dpEnum.freq());
@ -1232,12 +1231,12 @@ public class TestIndexWriter extends LuceneTestCase {
// test that the terms were indexed.
assertTrue(TestUtil.docs(random(), ir, "binary", new BytesRef("doc1field1"), null, null, PostingsEnum.NONE).nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertTrue(TestUtil.docs(random(), ir, "binary", new BytesRef("doc2field1"), null, null, PostingsEnum.NONE).nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertTrue(TestUtil.docs(random(), ir, "binary", new BytesRef("doc3field1"), null, null, PostingsEnum.NONE).nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertTrue(TestUtil.docs(random(), ir, "string", new BytesRef("doc1field2"), null, null, PostingsEnum.NONE).nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertTrue(TestUtil.docs(random(), ir, "string", new BytesRef("doc2field2"), null, null, PostingsEnum.NONE).nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertTrue(TestUtil.docs(random(), ir, "string", new BytesRef("doc3field2"), null, null, PostingsEnum.NONE).nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertTrue(TestUtil.docs(random(), ir, "binary", new BytesRef("doc1field1"), null, PostingsEnum.NONE).nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertTrue(TestUtil.docs(random(), ir, "binary", new BytesRef("doc2field1"), null, PostingsEnum.NONE).nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertTrue(TestUtil.docs(random(), ir, "binary", new BytesRef("doc3field1"), null, PostingsEnum.NONE).nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertTrue(TestUtil.docs(random(), ir, "string", new BytesRef("doc1field2"), null, PostingsEnum.NONE).nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertTrue(TestUtil.docs(random(), ir, "string", new BytesRef("doc2field2"), null, PostingsEnum.NONE).nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertTrue(TestUtil.docs(random(), ir, "string", new BytesRef("doc3field2"), null, PostingsEnum.NONE).nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
ir.close();
dir.close();

View File

@ -531,13 +531,15 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
PostingsEnum tdocs = TestUtil.docs(random(), reader,
t.field(),
new BytesRef(t.text()),
MultiFields.getLiveDocs(reader),
null,
0);
final Bits liveDocs = MultiFields.getLiveDocs(reader);
int count = 0;
while(tdocs.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
count++;
if (liveDocs == null || liveDocs.get(tdocs.docID())) {
count++;
}
}
assertEquals(2, count);

View File

@ -38,6 +38,7 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.store.MockDirectoryWrapper.FakeIOException;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.LuceneTestCase;
@ -55,14 +56,16 @@ public class TestIndexWriterReader extends LuceneTestCase {
int count = 0;
PostingsEnum td = TestUtil.docs(random(), r,
t.field(), new BytesRef(t.text()),
MultiFields.getLiveDocs(r),
null,
0);
if (td != null) {
final Bits liveDocs = MultiFields.getLiveDocs(r);
while (td.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
td.docID();
count++;
if (liveDocs == null || liveDocs.get(td.docID())) {
count++;
}
}
}
return count;

View File

@ -247,7 +247,6 @@ public class TestIndexWriterWithThreads extends LuceneTestCase {
PostingsEnum tdocs = TestUtil.docs(random(), reader,
"field",
new BytesRef("aaa"),
MultiFields.getLiveDocs(reader),
null,
0);
int count = 0;

View File

@ -332,14 +332,14 @@ public class TestIndexableField extends LuceneTestCase {
TermsEnum termsEnum = tfv.iterator();
assertEquals(new BytesRef(""+counter), termsEnum.next());
assertEquals(1, termsEnum.totalTermFreq());
PostingsEnum dpEnum = termsEnum.postings(null, null, PostingsEnum.ALL);
PostingsEnum dpEnum = termsEnum.postings(null, PostingsEnum.ALL);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(1, dpEnum.freq());
assertEquals(1, dpEnum.nextPosition());
assertEquals(new BytesRef("text"), termsEnum.next());
assertEquals(1, termsEnum.totalTermFreq());
dpEnum = termsEnum.postings(null, dpEnum, PostingsEnum.ALL);
dpEnum = termsEnum.postings(dpEnum, PostingsEnum.ALL);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(1, dpEnum.freq());
assertEquals(0, dpEnum.nextPosition());

View File

@ -153,7 +153,6 @@ public class TestLazyProxSkipping extends LuceneTestCase {
IndexReader reader = DirectoryReader.open(directory);
PostingsEnum tp = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
this.field,
new BytesRef("b"));
@ -164,7 +163,6 @@ public class TestLazyProxSkipping extends LuceneTestCase {
}
tp = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
this.field,
new BytesRef("a"));

View File

@ -167,7 +167,7 @@ public class TestLongPostings extends LuceneTestCase {
System.out.println("\nTEST: iter=" + iter + " doS1=" + doS1);
}
final PostingsEnum postings = MultiFields.getTermPositionsEnum(r, null, "field", new BytesRef(term));
final PostingsEnum postings = MultiFields.getTermPositionsEnum(r, "field", new BytesRef(term));
int docID = -1;
while(docID < DocIdSetIterator.NO_MORE_DOCS) {
@ -374,10 +374,10 @@ public class TestLongPostings extends LuceneTestCase {
final PostingsEnum postings;
if (options == IndexOptions.DOCS) {
docs = TestUtil.docs(random(), r, "field", new BytesRef(term), null, null, PostingsEnum.NONE);
docs = TestUtil.docs(random(), r, "field", new BytesRef(term), null, PostingsEnum.NONE);
postings = null;
} else {
docs = postings = TestUtil.docs(random(), r, "field", new BytesRef(term), null, null, PostingsEnum.FREQS);
docs = postings = TestUtil.docs(random(), r, "field", new BytesRef(term), null, PostingsEnum.FREQS);
assert postings != null;
}
assert docs != null;

View File

@ -81,7 +81,7 @@ public class TestMaxPosition extends LuceneTestCase {
// Document should be visible:
IndexReader r = DirectoryReader.open(iw, true);
assertEquals(1, r.numDocs());
PostingsEnum postings = MultiFields.getTermPositionsEnum(r, null, "foo", new BytesRef("foo"));
PostingsEnum postings = MultiFields.getTermPositionsEnum(r, "foo", new BytesRef("foo"));
// "foo" appears in docID=0
assertEquals(0, postings.nextDoc());

View File

@ -135,13 +135,11 @@ public class TestMultiFields extends LuceneTestCase {
System.out.println("TEST: seek term="+ UnicodeUtil.toHexString(term.utf8ToString()) + " " + term);
}
PostingsEnum postingsEnum = TestUtil.docs(random(), reader, "field", term, liveDocs, null, PostingsEnum.NONE);
PostingsEnum postingsEnum = TestUtil.docs(random(), reader, "field", term, null, PostingsEnum.NONE);
assertNotNull(postingsEnum);
for(int docID : docs.get(term)) {
if (!deleted.contains(docID)) {
assertEquals(docID, postingsEnum.nextDoc());
}
assertEquals(docID, postingsEnum.nextDoc());
}
assertEquals(DocIdSetIterator.NO_MORE_DOCS, postingsEnum.nextDoc());
}
@ -176,8 +174,8 @@ public class TestMultiFields extends LuceneTestCase {
w.addDocument(d);
IndexReader r = w.getReader();
w.close();
PostingsEnum d1 = TestUtil.docs(random(), r, "f", new BytesRef("j"), null, null, PostingsEnum.NONE);
PostingsEnum d2 = TestUtil.docs(random(), r, "f", new BytesRef("j"), null, null, PostingsEnum.NONE);
PostingsEnum d1 = TestUtil.docs(random(), r, "f", new BytesRef("j"), null, PostingsEnum.NONE);
PostingsEnum d2 = TestUtil.docs(random(), r, "f", new BytesRef("j"), null, PostingsEnum.NONE);
assertEquals(0, d1.nextDoc());
assertEquals(0, d2.nextDoc());
r.close();
@ -194,7 +192,7 @@ public class TestMultiFields extends LuceneTestCase {
w.addDocument(d);
IndexReader r = w.getReader();
w.close();
PostingsEnum de = MultiFields.getTermDocsEnum(r, null, "f", new BytesRef("j"));
PostingsEnum de = MultiFields.getTermDocsEnum(r, "f", new BytesRef("j"));
assertEquals(0, de.nextDoc());
assertEquals(1, de.nextDoc());
assertEquals(DocIdSetIterator.NO_MORE_DOCS, de.nextDoc());

View File

@ -51,9 +51,9 @@ public class TestOmitPositions extends LuceneTestCase {
IndexReader reader = w.getReader();
w.close();
assertNotNull(MultiFields.getTermPositionsEnum(reader, null, "foo", new BytesRef("test")));
assertNotNull(MultiFields.getTermPositionsEnum(reader, "foo", new BytesRef("test")));
PostingsEnum de = TestUtil.docs(random(), reader, "foo", new BytesRef("test"), null, null, PostingsEnum.FREQS);
PostingsEnum de = TestUtil.docs(random(), reader, "foo", new BytesRef("test"), null, PostingsEnum.FREQS);
while (de.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
assertEquals(2, de.freq());
}

View File

@ -72,7 +72,7 @@ public class TestParallelTermEnum extends LuceneTestCase {
super.tearDown();
}
private void checkTerms(Terms terms, Bits liveDocs, String... termsList) throws IOException {
private void checkTerms(Terms terms, String... termsList) throws IOException {
assertNotNull(terms);
final TermsEnum te = terms.iterator();
@ -80,7 +80,7 @@ public class TestParallelTermEnum extends LuceneTestCase {
BytesRef b = te.next();
assertNotNull(b);
assertEquals(t, b.utf8ToString());
PostingsEnum td = TestUtil.docs(random(), te, liveDocs, null, PostingsEnum.NONE);
PostingsEnum td = TestUtil.docs(random(), te, null, PostingsEnum.NONE);
assertTrue(td.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(0, td.docID());
assertEquals(td.nextDoc(), DocIdSetIterator.NO_MORE_DOCS);
@ -91,22 +91,20 @@ public class TestParallelTermEnum extends LuceneTestCase {
public void test1() throws IOException {
ParallelLeafReader pr = new ParallelLeafReader(ir1, ir2);
Bits liveDocs = pr.getLiveDocs();
Fields fields = pr.fields();
Iterator<String> fe = fields.iterator();
String f = fe.next();
assertEquals("field1", f);
checkTerms(fields.terms(f), liveDocs, "brown", "fox", "jumps", "quick", "the");
checkTerms(fields.terms(f), "brown", "fox", "jumps", "quick", "the");
f = fe.next();
assertEquals("field2", f);
checkTerms(fields.terms(f), liveDocs, "brown", "fox", "jumps", "quick", "the");
checkTerms(fields.terms(f), "brown", "fox", "jumps", "quick", "the");
f = fe.next();
assertEquals("field3", f);
checkTerms(fields.terms(f), liveDocs, "dog", "fox", "jumps", "lazy", "over", "the");
checkTerms(fields.terms(f), "dog", "fox", "jumps", "lazy", "over", "the");
assertFalse(fe.hasNext());
}

View File

@ -193,7 +193,6 @@ public class TestPayloads extends LuceneTestCase {
PostingsEnum[] tps = new PostingsEnum[numTerms];
for (int i = 0; i < numTerms; i++) {
tps[i] = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
terms[i].field(),
new BytesRef(terms[i].text()));
}
@ -222,7 +221,6 @@ public class TestPayloads extends LuceneTestCase {
* test lazy skipping
*/
PostingsEnum tp = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
terms[0].field(),
new BytesRef(terms[0].text()));
tp.nextDoc();
@ -249,7 +247,6 @@ public class TestPayloads extends LuceneTestCase {
* Test different lengths at skip points
*/
tp = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
terms[1].field(),
new BytesRef(terms[1].text()));
tp.nextDoc();
@ -287,7 +284,6 @@ public class TestPayloads extends LuceneTestCase {
reader = DirectoryReader.open(dir);
tp = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
fieldName,
new BytesRef(singleTerm));
tp.nextDoc();
@ -485,11 +481,10 @@ public class TestPayloads extends LuceneTestCase {
writer.close();
IndexReader reader = DirectoryReader.open(dir);
TermsEnum terms = MultiFields.getFields(reader).terms(field).iterator();
Bits liveDocs = MultiFields.getLiveDocs(reader);
PostingsEnum tp = null;
while (terms.next() != null) {
String termText = terms.term().utf8ToString();
tp = terms.postings(liveDocs, tp, PostingsEnum.PAYLOADS);
tp = terms.postings(tp, PostingsEnum.PAYLOADS);
while(tp.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
int freq = tp.freq();
for (int i = 0; i < freq; i++) {

View File

@ -72,7 +72,7 @@ public class TestPayloadsOnVectors extends LuceneTestCase {
assert terms != null;
TermsEnum termsEnum = terms.iterator();
assertTrue(termsEnum.seekExact(new BytesRef("withPayload")));
PostingsEnum de = termsEnum.postings(null, null, PostingsEnum.ALL);
PostingsEnum de = termsEnum.postings(null, PostingsEnum.ALL);
assertEquals(0, de.nextDoc());
assertEquals(0, de.nextPosition());
assertEquals(new BytesRef("test"), de.getPayload());
@ -114,7 +114,7 @@ public class TestPayloadsOnVectors extends LuceneTestCase {
assert terms != null;
TermsEnum termsEnum = terms.iterator();
assertTrue(termsEnum.seekExact(new BytesRef("withPayload")));
PostingsEnum de = termsEnum.postings(null, null, PostingsEnum.ALL);
PostingsEnum de = termsEnum.postings(null, PostingsEnum.ALL);
assertEquals(0, de.nextDoc());
assertEquals(3, de.nextPosition());
assertEquals(new BytesRef("test"), de.getPayload());

View File

@ -226,7 +226,7 @@ public class TestPerSegmentDeletes extends LuceneTestCase {
Terms cterms = fields.terms(term.field);
TermsEnum ctermsEnum = cterms.iterator();
if (ctermsEnum.seekExact(new BytesRef(term.text()))) {
PostingsEnum postingsEnum = TestUtil.docs(random(), ctermsEnum, bits, null, PostingsEnum.NONE);
PostingsEnum postingsEnum = TestUtil.docs(random(), ctermsEnum, null, PostingsEnum.NONE);
return toArray(postingsEnum);
}
return null;

Some files were not shown because too many files have changed in this diff Show More