LUCENE-4230: allow consumer to specify that payloads are not require when pulling D&PEnum, so codec can optimize if possible

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1367186 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2012-07-30 17:31:34 +00:00
parent 8f20844ee9
commit 72f99b47b4
67 changed files with 361 additions and 399 deletions

View File

@ -34,6 +34,12 @@ New features
CharFilterFactories to the lucene-analysis module. The API is still
experimental. (Chris Male, Robert Muir, Uwe Schindler)
* LUCENE-4230: When pulling a DocsAndPositionsEnum you can now
specify whether or not you require payloads (in addition to
offsets); turning one or both off may allow some codec
implementations to optimize the enum implementation. (Robert Muir,
Mike McCandless)
API Changes
* LUCENE-4138: update of morfologik (Polish morphological analyzer) to 1.5.3.
@ -72,6 +78,11 @@ API Changes
* LUCENE-3884: Moved ElisionFilter out of org.apache.lucene.analysis.fr
package into org.apache.lucene.analysis.util. (Robert Muir)
* LUCENE-4230: When pulling a DocsAndPositionsEnum you now pass an int
flags instead of the previous boolean needOffsets. Currently
recognized flags are DocsAndPositionsEnum.FLAG_PAYLOADS and
DocsAndPositionsEnum.FLAG_OFFSETS (Robert Muir, Mike McCandless)
Optimizations
* LUCENE-4171: Performance improvements to Packed64.

View File

@ -57,30 +57,6 @@ enumeration APIs. Here are the major changes:
...
}
The bulk read API has also changed. Instead of this:
int[] docs = new int[256];
int[] freqs = new int[256];
while(true) {
int count = td.read(docs, freqs)
if (count == 0) {
break;
}
// use docs[i], freqs[i]
}
do this:
DocsEnum.BulkReadResult bulk = td.getBulkResult();
while(true) {
int count = td.read();
if (count == 0) {
break;
}
// use bulk.docs.ints[i] and bulk.freqs.ints[i]
}
* TermPositions is renamed to DocsAndPositionsEnum, and no longer
extends the docs only enumerator (DocsEnum).
@ -170,7 +146,7 @@ enumeration APIs. Here are the major changes:
Bits liveDocs = reader.getLiveDocs();
DocsEnum docsEnum = null;
docsEnum = termsEnum.docs(liveDocs, docsEnum);
docsEnum = termsEnum.docs(liveDocs, docsEnum, needsFreqs);
You can pass in a prior DocsEnum and it will be reused if possible.
@ -187,7 +163,7 @@ enumeration APIs. Here are the major changes:
String field;
BytesRef text;
DocsEnum docsEnum = reader.termDocsEnum(reader.getLiveDocs(), field, text);
DocsEnum docsEnum = reader.termDocsEnum(reader.getLiveDocs(), field, text, needsFreqs);
Likewise for DocsAndPositionsEnum.

View File

@ -285,8 +285,7 @@ public class TestClassicAnalyzer extends BaseTokenStreamTestCase {
DocsAndPositionsEnum tps = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
"content",
new BytesRef("another"),
false);
new BytesRef("another"));
assertTrue(tps.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(1, tps.freq());
assertEquals(3, tps.nextPosition());

View File

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

View File

@ -694,20 +694,14 @@ public class BlockTermsReader extends FieldsProducer {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
// Positions were not indexed:
return null;
}
if (needsOffsets &&
fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) {
// Offsets were not indexed:
return null;
}
decodeMetaData();
return postingsReader.docsAndPositions(fieldInfo, state, liveDocs, reuse, needsOffsets);
return postingsReader.docsAndPositions(fieldInfo, state, liveDocs, reuse, flags);
}
@Override

View File

@ -902,20 +902,14 @@ public class BlockTreeTermsReader extends FieldsProducer {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
// Positions were not indexed:
return null;
}
if (needsOffsets &&
fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) {
// Offsets were not indexed:
return null;
}
currentFrame.decodeMetaData();
return postingsReader.docsAndPositions(fieldInfo, currentFrame.termState, skipDocs, reuse, needsOffsets);
return postingsReader.docsAndPositions(fieldInfo, currentFrame.termState, skipDocs, reuse, flags);
}
private int getState() {
@ -2137,21 +2131,15 @@ public class BlockTreeTermsReader extends FieldsProducer {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
// Positions were not indexed:
return null;
}
if (needsOffsets &&
fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) {
// Offsets were not indexed:
return null;
}
assert !eof;
currentFrame.decodeMetaData();
return postingsReader.docsAndPositions(fieldInfo, currentFrame.state, skipDocs, reuse, needsOffsets);
return postingsReader.docsAndPositions(fieldInfo, currentFrame.state, skipDocs, reuse, flags);
}
@Override

View File

@ -56,7 +56,7 @@ public abstract class PostingsReaderBase implements Closeable {
/** Must fully consume state, since after this call that
* TermState may be reused. */
public abstract DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState state, Bits skipDocs, DocsAndPositionsEnum reuse,
boolean needsOffsets) throws IOException;
int flags) throws IOException;
public abstract void close() throws IOException;

View File

@ -224,16 +224,9 @@ public abstract class TermVectorsWriter implements Closeable {
// TODO: we need a "query" API where we can ask (via
// flex API) what this term was indexed with...
// Both positions & offsets:
docsAndPositionsEnum = termsEnum.docsAndPositions(null, null, true);
final boolean hasOffsets;
docsAndPositionsEnum = termsEnum.docsAndPositions(null, null);
boolean hasOffsets = false;
boolean hasPositions = false;
if (docsAndPositionsEnum == null) {
// Fallback: no offsets
docsAndPositionsEnum = termsEnum.docsAndPositions(null, null, false);
hasOffsets = false;
} else {
hasOffsets = true;
}
if (docsAndPositionsEnum != null) {
final int docID = docsAndPositionsEnum.nextDoc();
@ -242,23 +235,19 @@ public abstract class TermVectorsWriter implements Closeable {
for(int posUpto=0; posUpto<freq; posUpto++) {
final int pos = docsAndPositionsEnum.nextPosition();
final int startOffset = docsAndPositionsEnum.startOffset();
final int endOffset = docsAndPositionsEnum.endOffset();
if (!startedField) {
assert numTerms > 0;
hasPositions = pos != -1;
hasOffsets = startOffset != -1;
startField(fieldInfo, numTerms, hasPositions, hasOffsets);
startTerm(termsEnum.term(), freq);
startedField = true;
}
final int startOffset;
final int endOffset;
if (hasOffsets) {
startOffset = docsAndPositionsEnum.startOffset();
endOffset = docsAndPositionsEnum.endOffset();
assert startOffset != -1;
assert endOffset != -1;
} else {
startOffset = -1;
endOffset = -1;
}
assert !hasPositions || pos >= 0;
addPosition(pos, startOffset, endOffset);

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import java.util.Comparator;
import org.apache.lucene.index.FieldInfo; // javadocs
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.TermsEnum;
@ -150,7 +151,7 @@ public abstract class TermsConsumer {
while((term = termsEnum.next()) != null) {
// We can pass null for liveDocs, because the
// mapping enum will skip the non-live docs:
postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn, false);
postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn, DocsAndPositionsEnum.FLAG_PAYLOADS);
assert postingsEnumIn != null;
postingsEnum.reset(postingsEnumIn);
// set PayloadProcessor
@ -184,7 +185,7 @@ public abstract class TermsConsumer {
while((term = termsEnum.next()) != null) {
// We can pass null for liveDocs, because the
// mapping enum will skip the non-live docs:
postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn, true);
postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn);
assert postingsEnumIn != null;
postingsEnum.reset(postingsEnumIn);
// set PayloadProcessor

View File

@ -250,11 +250,14 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
@Override
public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState termState, Bits liveDocs,
DocsAndPositionsEnum reuse, boolean needsOffsets)
DocsAndPositionsEnum reuse, int flags)
throws IOException {
boolean hasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
// TODO: can we optimize if FLAG_PAYLOADS / FLAG_OFFSETS
// isn't passed?
// TODO: refactor
if (fieldInfo.hasPayloads() || hasOffsets) {
SegmentFullPositionsEnum docsEnum;

View File

@ -490,10 +490,7 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
if (needsOffsets && !storeOffsets) {
return null;
}
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
if (!storePositions && !storeOffsets) {
return null;
@ -640,14 +637,20 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
@Override
public int startOffset() {
assert startOffsets != null;
return startOffsets[nextPos-1];
if (startOffsets == null) {
return -1;
} else {
return startOffsets[nextPos-1];
}
}
@Override
public int endOffset() {
assert endOffsets != null;
return endOffsets[nextPos-1];
if (endOffsets == null) {
return -1;
} else {
return endOffsets[nextPos-1];
}
}
}

View File

@ -314,7 +314,7 @@ public class DirectPostingsFormat extends PostingsFormat {
termOffsets[count+1] = termOffset;
if (hasPos) {
docsAndPositionsEnum = termsEnum.docsAndPositions(null, docsAndPositionsEnum, hasOffsets);
docsAndPositionsEnum = termsEnum.docsAndPositions(null, docsAndPositionsEnum);
} else {
docsEnum = termsEnum.docs(null, docsEnum, hasFreq);
}
@ -858,13 +858,10 @@ public class DirectPostingsFormat extends PostingsFormat {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) {
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) {
if (!hasPos) {
return null;
}
if (needsOffsets && !hasOffsets) {
return null;
}
// TODO: implement reuse, something like Pulsing:
// it's hairy!
@ -1420,13 +1417,10 @@ public class DirectPostingsFormat extends PostingsFormat {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) {
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) {
if (!hasPos) {
return null;
}
if (needsOffsets && !hasOffsets) {
return null;
}
// TODO: implement reuse, something like Pulsing:
// it's hairy!

View File

@ -714,13 +714,9 @@ public class MemoryPostingsFormat extends PostingsFormat {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) {
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) {
boolean hasOffsets = field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
if (needsOffsets && !hasOffsets) {
return null; // not available
}
if (field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
return null;
}

View File

@ -213,7 +213,7 @@ public class PulsingPostingsReader extends PostingsReaderBase {
@Override
public DocsAndPositionsEnum docsAndPositions(FieldInfo field, BlockTermState _termState, Bits liveDocs, DocsAndPositionsEnum reuse,
boolean needsOffsets) throws IOException {
int flags) throws IOException {
final PulsingTermState termState = (PulsingTermState) _termState;
@ -240,11 +240,11 @@ public class PulsingPostingsReader extends PostingsReaderBase {
} else {
if (reuse instanceof PulsingDocsAndPositionsEnum) {
DocsAndPositionsEnum wrapped = wrappedPostingsReader.docsAndPositions(field, termState.wrappedTermState, liveDocs, (DocsAndPositionsEnum) getOther(reuse),
needsOffsets);
flags);
setOther(wrapped, reuse); // wrapped.other = reuse
return wrapped;
} else {
return wrappedPostingsReader.docsAndPositions(field, termState.wrappedTermState, liveDocs, reuse, needsOffsets);
return wrappedPostingsReader.docsAndPositions(field, termState.wrappedTermState, liveDocs, reuse, flags);
}
}
}

View File

@ -278,13 +278,9 @@ public class SepPostingsReader extends PostingsReaderBase {
@Override
public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState _termState, Bits liveDocs,
DocsAndPositionsEnum reuse, boolean needsOffsets)
DocsAndPositionsEnum reuse, int flags)
throws IOException {
if (needsOffsets) {
return null;
}
assert fieldInfo.getIndexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
final SepTermState termState = (SepTermState) _termState;
SepDocsAndPositionsEnum postingsEnum;

View File

@ -208,19 +208,13 @@ class SimpleTextFieldsReader extends FieldsProducer {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
// Positions were not indexed
return null;
}
if (needsOffsets &&
indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) {
// Offsets were not indexed
return null;
}
SimpleTextDocsAndPositionsEnum docsAndPositionsEnum;
if (reuse != null && reuse instanceof SimpleTextDocsAndPositionsEnum && ((SimpleTextDocsAndPositionsEnum) reuse).canReuse(SimpleTextFieldsReader.this.in)) {
docsAndPositionsEnum = (SimpleTextDocsAndPositionsEnum) reuse;

View File

@ -365,14 +365,11 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
SimpleTVPostings postings = current.getValue();
if (postings.positions == null && postings.startOffsets == null) {
return null;
}
if (needsOffsets && (postings.startOffsets == null || postings.endOffsets == null)) {
return null;
}
// TODO: reuse
SimpleTVDocsAndPositionsEnum e = new SimpleTVDocsAndPositionsEnum();
e.reset(liveDocs, postings.positions, postings.startOffsets, postings.endOffsets);
@ -507,12 +504,20 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
@Override
public int startOffset() {
return startOffsets[nextPos-1];
if (startOffsets == null) {
return -1;
} else {
return startOffsets[nextPos-1];
}
}
@Override
public int endOffset() {
return endOffsets[nextPos-1];
if (endOffsets == null) {
return -1;
} else {
return endOffsets[nextPos-1];
}
}
}
}

View File

@ -133,12 +133,24 @@ public abstract class AtomicReader extends IndexReader {
}
return null;
}
/** Returns {@link DocsAndPositionsEnum} for the specified
* field & term. This will return null if either the
* field or term does not exist or positions weren't indexed.
* @see #termPositionsEnum(Bits, String, BytesRef, int) */
public final DocsAndPositionsEnum termPositionsEnum(Bits liveDocs, String field, BytesRef term) throws IOException {
return termPositionsEnum(liveDocs, field, term, DocsAndPositionsEnum.FLAG_OFFSETS | DocsAndPositionsEnum.FLAG_PAYLOADS);
}
/** Returns {@link DocsAndPositionsEnum} for the specified
* field & term. This may return null, if either the
* field or term does not exist, or needsOffsets is
* true but offsets were not indexed for this field. */
public final DocsAndPositionsEnum termPositionsEnum(Bits liveDocs, String field, BytesRef term, boolean needsOffsets) throws IOException {
* field & term, with control over whether offsets and payloads are
* required. Some codecs may be able to optimize their
* implementation when offsets and/or payloads are not required.
* This will return null, if either the field or term
* does not exist or positions weren't indexed. See
* {@link TermsEnum#docsAndPositions(Bits,DocsAndPositionsEnum,int)}. */
public final DocsAndPositionsEnum termPositionsEnum(Bits liveDocs, String field, BytesRef term, int flags) throws IOException {
assert field != null;
assert term != null;
final Fields fields = fields();
@ -147,7 +159,7 @@ public abstract class AtomicReader extends IndexReader {
if (terms != null) {
final TermsEnum termsEnum = terms.iterator(null);
if (termsEnum.seekExact(term, true)) {
return termsEnum.docsAndPositions(liveDocs, null, needsOffsets);
return termsEnum.docsAndPositions(liveDocs, null, flags);
}
}
}

View File

@ -34,6 +34,7 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldType; // for javadocs
import org.apache.lucene.index.DocValues.SortedSource;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
@ -682,7 +683,6 @@ public class CheckIndex {
DocsEnum docs = null;
DocsEnum docsAndFreqs = null;
DocsAndPositionsEnum postings = null;
DocsAndPositionsEnum offsets = null;
String lastField = null;
final FieldsEnum fieldsEnum = fields.iterator();
@ -699,11 +699,11 @@ public class CheckIndex {
// check that the field is in fieldinfos, and is indexed.
// TODO: add a separate test to check this for different reader impls
FieldInfo fi = fieldInfos.fieldInfo(field);
if (fi == null) {
FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
if (fieldInfo == null) {
throw new RuntimeException("fieldsEnum inconsistent with fieldInfos, no fieldInfos for: " + field);
}
if (!fi.isIndexed()) {
if (!fieldInfo.isIndexed()) {
throw new RuntimeException("fieldsEnum inconsistent with fieldInfos, isIndexed == false for: " + field);
}
@ -757,8 +757,7 @@ public class CheckIndex {
docs = termsEnum.docs(liveDocs, docs, false);
docsAndFreqs = termsEnum.docs(liveDocs, docsAndFreqs, true);
postings = termsEnum.docsAndPositions(liveDocs, postings, false);
offsets = termsEnum.docsAndPositions(liveDocs, offsets, true);
postings = termsEnum.docsAndPositions(liveDocs, postings);
if (hasOrd) {
long ord = -1;
@ -782,29 +781,20 @@ public class CheckIndex {
final DocsEnum docsAndFreqs2;
final boolean hasPositions;
final boolean hasFreqs;
final boolean hasOffsets;
if (offsets != null) {
docs2 = postings = offsets;
docsAndFreqs2 = postings = offsets;
hasOffsets = true;
hasPositions = true;
hasFreqs = true;
} else if (postings != null) {
final boolean hasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
if (postings != null) {
docs2 = postings;
docsAndFreqs2 = postings;
hasOffsets = false;
hasPositions = true;
hasFreqs = true;
} else if (docsAndFreqs != null) {
docs2 = docsAndFreqs;
docsAndFreqs2 = docsAndFreqs;
hasOffsets = false;
hasPositions = false;
hasFreqs = true;
} else {
docs2 = docs;
docsAndFreqs2 = null;
hasOffsets = false;
hasPositions = false;
hasFreqs = false;
}
@ -927,7 +917,7 @@ public class CheckIndex {
if (hasPositions) {
for(int idx=0;idx<7;idx++) {
final int skipDocID = (int) (((idx+1)*(long) maxDoc)/8);
postings = termsEnum.docsAndPositions(liveDocs, postings, hasOffsets);
postings = termsEnum.docsAndPositions(liveDocs, postings);
final int docID = postings.advance(skipDocID);
if (docID == DocIdSetIterator.NO_MORE_DOCS) {
break;
@ -1460,44 +1450,35 @@ public class CheckIndex {
BytesRef term = null;
while ((term = termsEnum.next()) != null) {
final boolean hasPositions;
final boolean hasOffsets;
final boolean hasProx;
final boolean hasFreqs;
// TODO: really we need a reflection/query
// API so we can just ask what was indexed
// instead of "probing"...
// TODO: cleanup:
// Try offsets:
postings = termsEnum.docsAndPositions(null, postings, true);
postings = termsEnum.docsAndPositions(null, postings);
if (postings == null) {
hasOffsets = false;
// Try only positions:
postings = termsEnum.docsAndPositions(null, postings, false);
if (postings == null) {
hasPositions = false;
// Try docIDs & freqs:
docs = termsEnum.docs(null, docs, true);
if (docs == null) {
// OK, only docIDs:
hasFreqs = false;
docs = termsEnum.docs(null, docs, false);
} else {
hasFreqs = true;
}
hasProx = false;
// Try docIDs & freqs:
docs = termsEnum.docs(null, docs, true);
if (docs == null) {
// OK, only docIDs:
hasFreqs = false;
docs = termsEnum.docs(null, docs, false);
} else {
hasPositions = true;
hasFreqs = true;
hasFreqs = true;
}
} else {
hasOffsets = true;
// NOTE: may be a lie... but we accept -1
hasPositions = true;
hasProx = true;
hasFreqs = true;
}
final DocsEnum docs2;
if (hasPositions || hasOffsets) {
if (hasProx) {
assert postings != null;
docs2 = postings;
} else {
@ -1510,23 +1491,18 @@ public class CheckIndex {
if (!postingsTermsEnum.seekExact(term, true)) {
throw new RuntimeException("vector term=" + term + " field=" + field + " does not exist in postings; doc=" + j);
}
postingsPostings = postingsTermsEnum.docsAndPositions(null, postingsPostings, true);
postingsPostings = postingsTermsEnum.docsAndPositions(null, postingsPostings);
if (postingsPostings == null) {
// Term vectors were indexed w/ offsets but postings were not
postingsPostings = postingsTermsEnum.docsAndPositions(null, postingsPostings, false);
if (postingsPostings == null) {
postingsDocs = postingsTermsEnum.docs(null, postingsDocs, true);
// Term vectors were indexed w/ pos but postings were not
postingsDocs = postingsTermsEnum.docs(null, postingsDocs, true);
if (postingsDocs == null) {
postingsHasFreq = false;
postingsDocs = postingsTermsEnum.docs(null, postingsDocs, false);
if (postingsDocs == null) {
postingsHasFreq = false;
postingsDocs = postingsTermsEnum.docs(null, postingsDocs, false);
if (postingsDocs == null) {
throw new RuntimeException("vector term=" + term + " field=" + field + " does not exist in postings; doc=" + j);
}
} else {
postingsHasFreq = true;
throw new RuntimeException("vector term=" + term + " field=" + field + " does not exist in postings; doc=" + j);
}
} else {
postingsHasFreq = true;
postingsHasFreq = true;
}
} else {
postingsHasFreq = true;
@ -1555,7 +1531,7 @@ public class CheckIndex {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": freq=" + tf + " differs from postings freq=" + postingsDocs2.freq());
}
if (hasPositions || hasOffsets) {
if (hasProx) {
for (int i = 0; i < tf; i++) {
int pos = postings.nextPosition();
if (postingsPostings != null) {
@ -1565,32 +1541,30 @@ public class CheckIndex {
}
}
if (hasOffsets) {
// Call the methods to at least make
// sure they don't throw exc:
final int startOffset = postings.startOffset();
final int endOffset = postings.endOffset();
// TODO: these are too anal...?
/*
if (endOffset < startOffset) {
throw new RuntimeException("vector startOffset=" + startOffset + " is > endOffset=" + endOffset);
}
if (startOffset < lastStartOffset) {
throw new RuntimeException("vector startOffset=" + startOffset + " is < prior startOffset=" + lastStartOffset);
}
lastStartOffset = startOffset;
*/
// Call the methods to at least make
// sure they don't throw exc:
final int startOffset = postings.startOffset();
final int endOffset = postings.endOffset();
// TODO: these are too anal...?
/*
if (endOffset < startOffset) {
throw new RuntimeException("vector startOffset=" + startOffset + " is > endOffset=" + endOffset);
}
if (startOffset < lastStartOffset) {
throw new RuntimeException("vector startOffset=" + startOffset + " is < prior startOffset=" + lastStartOffset);
}
lastStartOffset = startOffset;
*/
if (postingsPostings != null) {
final int postingsStartOffset = postingsPostings.startOffset();
if (postingsPostings != null) {
final int postingsStartOffset = postingsPostings.startOffset();
final int postingsEndOffset = postingsPostings.endOffset();
if (startOffset != -1 && postingsStartOffset != -1 && startOffset != postingsStartOffset) {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": startOffset=" + startOffset + " differs from postings startOffset=" + postingsStartOffset);
}
if (endOffset != -1 && postingsEndOffset != -1 && endOffset != postingsEndOffset) {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": endOffset=" + endOffset + " differs from postings endOffset=" + postingsEndOffset);
}
final int postingsEndOffset = postingsPostings.endOffset();
if (startOffset != -1 && postingsStartOffset != -1 && startOffset != postingsStartOffset) {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": startOffset=" + startOffset + " differs from postings startOffset=" + postingsStartOffset);
}
if (endOffset != -1 && postingsEndOffset != -1 && endOffset != postingsEndOffset) {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": endOffset=" + endOffset + " differs from postings endOffset=" + postingsEndOffset);
}
}
}

View File

@ -673,8 +673,8 @@ public class DocTermOrds {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
return termsEnum.docsAndPositions(liveDocs, reuse, needsOffsets);
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
return termsEnum.docsAndPositions(liveDocs, reuse, flags);
}
@Override

View File

@ -23,7 +23,14 @@ import org.apache.lucene.util.BytesRef;
/** Also iterates through positions. */
public abstract class DocsAndPositionsEnum extends DocsEnum {
/** Flag to pass to {@link TermsEnum#docsAndPositions(Bits,DocsAndPositionsEnum,int)}
* if you require offsets in the returned enum. */
public static final int FLAG_OFFSETS = 0x1;
/** Flag to pass to {@link TermsEnum#docsAndPositions(Bits,DocsAndPositionsEnum,int)}
* if you require payloads in the returned enum. */
public static final int FLAG_PAYLOADS = 0x2;
/** Returns the next position. You should only call this
* up to {@link DocsEnum#freq()} times else
* the behavior is not defined. If positions were not

View File

@ -186,8 +186,8 @@ public class FilterAtomicReader extends AtomicReader {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
return in.docsAndPositions(liveDocs, reuse, needsOffsets);
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
return in.docsAndPositions(liveDocs, reuse, flags);
}
@Override

View File

@ -171,8 +171,8 @@ public abstract class FilteredTermsEnum extends TermsEnum {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits bits, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
return tenum.docsAndPositions(bits, reuse, needsOffsets);
public DocsAndPositionsEnum docsAndPositions(Bits bits, DocsAndPositionsEnum reuse, int flags) throws IOException {
return tenum.docsAndPositions(bits, reuse, flags);
}
/** This enum does not support seeking!

View File

@ -139,15 +139,27 @@ public final class MultiFields extends Fields {
/** Returns {@link DocsAndPositionsEnum} for the specified
* field & term. This may return null if the term does
* not exist or positions were not indexed. */
public static DocsAndPositionsEnum getTermPositionsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term, boolean needsOffsets) throws IOException {
* not exist or positions were not indexed.
* @see #getTermPositionsEnum(IndexReader, Bits, String, BytesRef, int) */
public static DocsAndPositionsEnum getTermPositionsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term) throws IOException {
return getTermPositionsEnum(r, liveDocs, field, term, DocsAndPositionsEnum.FLAG_OFFSETS | DocsAndPositionsEnum.FLAG_PAYLOADS);
}
/** Returns {@link DocsAndPositionsEnum} for the specified
* field & term, with control over whether offsets and payloads are
* required. Some codecs may be able to optimize
* their implementation when offsets and/or payloads are not
* required. This will return null if the term does not
* exist or positions were not indexed. See {@link
* TermsEnum#docsAndPositions(Bits,DocsAndPositionsEnum,int)}. */
public static DocsAndPositionsEnum getTermPositionsEnum(IndexReader r, Bits liveDocs, 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(null);
if (termsEnum.seekExact(term, true)) {
return termsEnum.docsAndPositions(liveDocs, null, needsOffsets);
return termsEnum.docsAndPositions(liveDocs, null, flags);
}
}
return null;

View File

@ -415,7 +415,7 @@ public final class MultiTermsEnum extends TermsEnum {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
MultiDocsAndPositionsEnum docsAndPositionsEnum;
// Can only reuse if incoming enum is also a MultiDocsAndPositionsEnum
if (reuse != null && reuse instanceof MultiDocsAndPositionsEnum) {
@ -466,7 +466,7 @@ public final class MultiTermsEnum extends TermsEnum {
}
assert entry.index < docsAndPositionsEnum.subDocsAndPositionsEnum.length: entry.index + " vs " + docsAndPositionsEnum.subDocsAndPositionsEnum.length + "; " + subs.length;
final DocsAndPositionsEnum subPostings = entry.terms.docsAndPositions(b, docsAndPositionsEnum.subDocsAndPositionsEnum[entry.index], needsOffsets);
final DocsAndPositionsEnum subPostings = entry.terms.docsAndPositions(b, docsAndPositionsEnum.subDocsAndPositionsEnum[entry.index], flags);
if (subPostings != null) {
docsAndPositionsEnum.subDocsAndPositionsEnum[entry.index] = subPostings;

View File

@ -138,6 +138,8 @@ public abstract class TermsEnum implements BytesRefIterator {
* deleted documents into account. */
public abstract long totalTermFreq() throws IOException;
// TODO: cutover to flags / make needsFreqs "a hint" / add
// default w/ needsFreqs=true
/** Get {@link DocsEnum} for the current term. Do not
* call this when the enum is unpositioned. This method
* may return null (if needsFreqs is true but freqs were
@ -152,14 +154,32 @@ public abstract class TermsEnum implements BytesRefIterator {
public abstract DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException;
/** Get {@link DocsAndPositionsEnum} for the current term.
* Do not call this when the enum is unpositioned.
* This method will only return null if needsOffsets is
* true but offsets were not indexed.
* Do not call this when the enum is unpositioned. This
* method will return null if positions were not
* indexed.
*
* @param liveDocs unset bits are documents that should not
* be returned
* @param reuse pass a prior DocsAndPositionsEnum for possible reuse
* @param needsOffsets true if offsets are required */
public abstract DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException;
* @see #docsAndPositions(Bits, DocsAndPositionsEnum, int) */
public final DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse) throws IOException {
return docsAndPositions(liveDocs, reuse, DocsAndPositionsEnum.FLAG_OFFSETS | DocsAndPositionsEnum.FLAG_PAYLOADS);
}
/** Get {@link DocsAndPositionsEnum} for the current term,
* with control over whether offsets and payloads are
* required. Some codecs may be able to optimize their
* implementation when offsets and/or payloads are not required.
* Do not call this when the enum is unpositioned. This
* will return null if positions were not indexed.
* @param liveDocs unset bits are documents that should not
* be returned
* @param reuse pass a prior DocsAndPositionsEnum for possible reuse
* @param flags specifies which optional per-position values you
* require; see {@link DocsAndPositionsEnum#FLAG_OFFSETS} and
* {@link DocsAndPositionsEnum#FLAG_PAYLOADS}. */
public abstract DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException;
/**
* Expert: Returns the TermsEnums internal state to position the TermsEnum
@ -225,7 +245,7 @@ public abstract class TermsEnum implements BytesRefIterator {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) {
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) {
throw new IllegalStateException("this method should never be called");
}

View File

@ -1047,7 +1047,7 @@ class FieldCacheImpl implements FieldCache {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
throw new UnsupportedOperationException();
}

View File

@ -278,8 +278,8 @@ public class FuzzyTermsEnum extends TermsEnum {
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs,
DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
return actualEnum.docsAndPositions(liveDocs, reuse, needsOffsets);
DocsAndPositionsEnum reuse, int flags) throws IOException {
return actualEnum.docsAndPositions(liveDocs, reuse, flags);
}
@Override

View File

@ -226,7 +226,7 @@ public class MultiPhraseQuery extends Query {
return null;
}
termsEnum.seekExact(term.bytes(), termState);
postingsEnum = termsEnum.docsAndPositions(liveDocs, null, false);
postingsEnum = termsEnum.docsAndPositions(liveDocs, null, 0);
if (postingsEnum == null) {
// term does exist, but has no positions
@ -484,7 +484,7 @@ class UnionDocsAndPositionsEnum extends DocsAndPositionsEnum {
continue;
}
termsEnum.seekExact(term.bytes(), termState);
DocsAndPositionsEnum postings = termsEnum.docsAndPositions(liveDocs, null, false);
DocsAndPositionsEnum postings = termsEnum.docsAndPositions(liveDocs, null, 0);
if (postings == null) {
// term does exist, but has no positions
throw new IllegalStateException("field \"" + term.field() + "\" was indexed without position data; cannot run PhraseQuery (term=" + term.text() + ")");

View File

@ -264,7 +264,7 @@ public class PhraseQuery extends Query {
return null;
}
te.seekExact(t.bytes(), state);
DocsAndPositionsEnum postingsEnum = te.docsAndPositions(liveDocs, null, false);
DocsAndPositionsEnum postingsEnum = te.docsAndPositions(liveDocs, null, 0);
// PhraseQuery on a field that did not index
// positions.

View File

@ -120,7 +120,7 @@ public class SpanTermQuery extends SpanQuery {
final TermsEnum termsEnum = context.reader().terms(term.field()).iterator(null);
termsEnum.seekExact(term.bytes(), state);
final DocsAndPositionsEnum postings = termsEnum.docsAndPositions(acceptDocs, null, false);
final DocsAndPositionsEnum postings = termsEnum.docsAndPositions(acceptDocs, null, DocsAndPositionsEnum.FLAG_PAYLOADS);
if (postings != null) {
return new TermSpans(postings, term);

View File

@ -75,8 +75,7 @@ public class TestCachingTokenFilter extends BaseTokenStreamTestCase {
DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
"preanalyzed",
new BytesRef("term1"),
false);
new BytesRef("term1"));
assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(1, termPositions.freq());
assertEquals(0, termPositions.nextPosition());
@ -84,8 +83,7 @@ public class TestCachingTokenFilter extends BaseTokenStreamTestCase {
termPositions = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
"preanalyzed",
new BytesRef("term2"),
false);
new BytesRef("term2"));
assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(2, termPositions.freq());
assertEquals(1, termPositions.nextPosition());
@ -94,8 +92,7 @@ public class TestCachingTokenFilter extends BaseTokenStreamTestCase {
termPositions = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
"preanalyzed",
new BytesRef("term3"),
false);
new BytesRef("term3"));
assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(1, termPositions.freq());
assertEquals(2, termPositions.nextPosition());

View File

@ -35,7 +35,6 @@ import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.BaseDirectoryWrapper;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
@ -71,7 +70,7 @@ public class TestPulsingReuse extends LuceneTestCase {
DocsAndPositionsEnum posReuse = null;
te = segment.terms("foo").iterator(null);
while (te.next() != null) {
posReuse = te.docsAndPositions(null, posReuse, false);
posReuse = te.docsAndPositions(null, posReuse);
allEnums.put(posReuse, true);
}
@ -113,7 +112,7 @@ public class TestPulsingReuse extends LuceneTestCase {
DocsAndPositionsEnum posReuse = null;
te = segment.terms("foo").iterator(null);
while (te.next() != null) {
posReuse = te.docsAndPositions(null, posReuse, false);
posReuse = te.docsAndPositions(null, posReuse);
allEnums.put(posReuse, true);
}

View File

@ -474,7 +474,7 @@ public class TestCodecs extends LuceneTestCase {
if (field.omitTF) {
this.verifyDocs(term.docs, term.positions, _TestUtil.docs(random(), termsEnum, null, null, false), false);
} else {
this.verifyDocs(term.docs, term.positions, termsEnum.docsAndPositions(null, null, false), true);
this.verifyDocs(term.docs, term.positions, termsEnum.docsAndPositions(null, null), true);
}
// Test random seek by ord:
@ -494,7 +494,7 @@ public class TestCodecs extends LuceneTestCase {
if (field.omitTF) {
this.verifyDocs(term.docs, term.positions, _TestUtil.docs(random(), termsEnum, null, null, false), false);
} else {
this.verifyDocs(term.docs, term.positions, termsEnum.docsAndPositions(null, null, false), true);
this.verifyDocs(term.docs, term.positions, termsEnum.docsAndPositions(null, null), true);
}
}
@ -546,7 +546,7 @@ public class TestCodecs extends LuceneTestCase {
final DocsEnum docsAndFreqs;
final DocsAndPositionsEnum postings;
if (!field.omitTF) {
postings = termsEnum.docsAndPositions(null, null, false);
postings = termsEnum.docsAndPositions(null, null);
if (postings != null) {
docs = docsAndFreqs = postings;
} else {

View File

@ -631,8 +631,8 @@ public void testFilesOpenClose() throws IOException {
while(enum1.next() != null) {
assertEquals("Different terms", enum1.term(), enum2.next());
DocsAndPositionsEnum tp1 = enum1.docsAndPositions(liveDocs, null, false);
DocsAndPositionsEnum tp2 = enum2.docsAndPositions(liveDocs, null, false);
DocsAndPositionsEnum tp1 = enum1.docsAndPositions(liveDocs, null);
DocsAndPositionsEnum tp2 = enum2.docsAndPositions(liveDocs, null);
while(tp1.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
assertTrue(tp2.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);

View File

@ -247,7 +247,7 @@ public class TestDoc extends LuceneTestCase {
out.print(" term=" + field + ":" + tis.term());
out.println(" DF=" + tis.docFreq());
DocsAndPositionsEnum positions = tis.docsAndPositions(reader.getLiveDocs(), null, false);
DocsAndPositionsEnum positions = tis.docsAndPositions(reader.getLiveDocs(), null);
while (positions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
out.print(" doc=" + positions.docID());

View File

@ -92,7 +92,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
public DocsAndPositionsEnum getDocsAndPositions(AtomicReader reader,
BytesRef bytes, Bits liveDocs) throws IOException {
return reader.termPositionsEnum(null, fieldName, bytes, false);
return reader.termPositionsEnum(null, fieldName, bytes);
}
/**
@ -352,7 +352,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
writer.addDocument(doc);
DirectoryReader reader = writer.getReader();
AtomicReader r = getOnlySegmentReader(reader);
DocsAndPositionsEnum disi = r.termPositionsEnum(null, "foo", new BytesRef("bar"), false);
DocsAndPositionsEnum disi = r.termPositionsEnum(null, "foo", new BytesRef("bar"));
int docid = disi.docID();
assertTrue(docid == -1 || docid == DocIdSetIterator.NO_MORE_DOCS);
assertTrue(disi.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@ -360,7 +360,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
// now reuse and check again
TermsEnum te = r.terms("foo").iterator(null);
assertTrue(te.seekExact(new BytesRef("bar"), true));
disi = te.docsAndPositions(null, disi, false);
disi = te.docsAndPositions(null, disi);
docid = disi.docID();
assertTrue(docid == -1 || docid == DocIdSetIterator.NO_MORE_DOCS);
assertTrue(disi.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);

View File

@ -129,7 +129,7 @@ public class TestDocumentWriter extends LuceneTestCase {
SegmentReader reader = new SegmentReader(info, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(reader, MultiFields.getLiveDocs(reader),
"repeated", new BytesRef("repeated"), false);
"repeated", new BytesRef("repeated"));
assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
int freq = termPositions.freq();
assertEquals(2, freq);
@ -200,7 +200,7 @@ public class TestDocumentWriter extends LuceneTestCase {
writer.close();
SegmentReader reader = new SegmentReader(info, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(reader, reader.getLiveDocs(), "f1", new BytesRef("a"), false);
DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(reader, reader.getLiveDocs(), "f1", new BytesRef("a"));
assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
int freq = termPositions.freq();
assertEquals(3, freq);
@ -243,18 +243,18 @@ public class TestDocumentWriter extends LuceneTestCase {
writer.close();
SegmentReader reader = new SegmentReader(info, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
DocsAndPositionsEnum termPositions = reader.termPositionsEnum(reader.getLiveDocs(), "preanalyzed", new BytesRef("term1"), false);
DocsAndPositionsEnum termPositions = reader.termPositionsEnum(reader.getLiveDocs(), "preanalyzed", new BytesRef("term1"));
assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(1, termPositions.freq());
assertEquals(0, termPositions.nextPosition());
termPositions = reader.termPositionsEnum(reader.getLiveDocs(), "preanalyzed", new BytesRef("term2"), false);
termPositions = reader.termPositionsEnum(reader.getLiveDocs(), "preanalyzed", new BytesRef("term2"));
assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(2, termPositions.freq());
assertEquals(1, termPositions.nextPosition());
assertEquals(3, termPositions.nextPosition());
termPositions = reader.termPositionsEnum(reader.getLiveDocs(), "preanalyzed", new BytesRef("term3"), false);
termPositions = reader.termPositionsEnum(reader.getLiveDocs(), "preanalyzed", new BytesRef("term3"));
assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(1, termPositions.freq());
assertEquals(2, termPositions.nextPosition());

View File

@ -329,17 +329,17 @@ public class TestDuelingCodecs extends LuceneTestCase {
assertEquals(info, term, rightTermsEnum.next());
assertTermStats(leftTermsEnum, rightTermsEnum);
if (deep) {
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, false),
rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, false));
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, false),
rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, false));
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions),
rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions));
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions),
rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, false),
rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, false));
leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions),
rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, false),
rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, false));
leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions),
rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions));
// with freqs:
assertDocsEnum(leftDocs = leftTermsEnum.docs(null, leftDocs, true),

View File

@ -89,8 +89,8 @@ public class TestFilterAtomicReader extends LuceneTestCase {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
return new TestPositions(super.docsAndPositions(liveDocs, reuse == null ? null : ((FilterDocsAndPositionsEnum) reuse).in, needsOffsets));
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
return new TestPositions(super.docsAndPositions(liveDocs, reuse == null ? null : ((FilterDocsAndPositionsEnum) reuse).in, flags));
}
}
@ -164,8 +164,7 @@ public class TestFilterAtomicReader extends LuceneTestCase {
assertEquals(TermsEnum.SeekStatus.FOUND, terms.seekCeil(new BytesRef("one")));
DocsAndPositionsEnum positions = terms.docsAndPositions(MultiFields.getLiveDocs(reader),
null, false);
DocsAndPositionsEnum positions = terms.docsAndPositions(MultiFields.getLiveDocs(reader), null);
while (positions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
assertTrue((positions.docID() % 2) == 1);
}

View File

@ -940,14 +940,14 @@ public class TestIndexWriter extends LuceneTestCase {
Terms tpv = r.getTermVectors(0).terms("field");
TermsEnum termsEnum = tpv.iterator(null);
assertNotNull(termsEnum.next());
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null, false);
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null);
assertNotNull(dpEnum);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(1, dpEnum.freq());
assertEquals(100, dpEnum.nextPosition());
assertNotNull(termsEnum.next());
dpEnum = termsEnum.docsAndPositions(null, dpEnum, false);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
assertNotNull(dpEnum);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(1, dpEnum.freq());
@ -1588,7 +1588,7 @@ public class TestIndexWriter extends LuceneTestCase {
// Make sure position is still incremented when
// massive term is skipped:
DocsAndPositionsEnum tps = MultiFields.getTermPositionsEnum(reader, null, "content", new BytesRef("another"), false);
DocsAndPositionsEnum tps = MultiFields.getTermPositionsEnum(reader, null, "content", new BytesRef("another"));
assertEquals(0, tps.nextDoc());
assertEquals(1, tps.freq());
assertEquals(3, tps.nextPosition());

View File

@ -264,14 +264,14 @@ public class TestIndexableField extends LuceneTestCase {
TermsEnum termsEnum = tfv.iterator(null);
assertEquals(new BytesRef(""+counter), termsEnum.next());
assertEquals(1, termsEnum.totalTermFreq());
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null, false);
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null);
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.docsAndPositions(null, dpEnum, false);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(1, dpEnum.freq());
assertEquals(0, dpEnum.nextPosition());

View File

@ -157,8 +157,7 @@ public class TestLazyProxSkipping extends LuceneTestCase {
DocsAndPositionsEnum tp = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
this.field,
new BytesRef("b"),
false);
new BytesRef("b"));
for (int i = 0; i < 10; i++) {
tp.nextDoc();
@ -169,8 +168,7 @@ public class TestLazyProxSkipping extends LuceneTestCase {
tp = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
this.field,
new BytesRef("a"),
false);
new BytesRef("a"));
for (int i = 0; i < 10; i++) {
tp.nextDoc();

View File

@ -170,7 +170,7 @@ public class TestLongPostings extends LuceneTestCase {
System.out.println("\nTEST: iter=" + iter + " doS1=" + doS1);
}
final DocsAndPositionsEnum postings = MultiFields.getTermPositionsEnum(r, null, "field", new BytesRef(term), false);
final DocsAndPositionsEnum postings = MultiFields.getTermPositionsEnum(r, null, "field", new BytesRef(term));
int docID = -1;
while(docID < DocIdSetIterator.NO_MORE_DOCS) {

View File

@ -86,8 +86,7 @@ public class TestMultiLevelSkipList extends LuceneTestCase {
counter = 0;
DocsAndPositionsEnum tp = reader.termPositionsEnum(reader.getLiveDocs(),
term.field(),
new BytesRef(term.text()),
false);
new BytesRef(term.text()));
checkSkipTo(tp, 14, 185); // no skips
checkSkipTo(tp, 17, 190); // one skip on level 0

View File

@ -51,7 +51,7 @@ public class TestOmitPositions extends LuceneTestCase {
IndexReader reader = w.getReader();
w.close();
assertNull(MultiFields.getTermPositionsEnum(reader, null, "foo", new BytesRef("test"), false));
assertNull(MultiFields.getTermPositionsEnum(reader, null, "foo", new BytesRef("test")));
DocsEnum de = _TestUtil.docs(random(), reader, "foo", new BytesRef("test"), null, null, true);
while (de.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {

View File

@ -160,7 +160,7 @@ public class TestPayloadProcessorProvider extends LuceneTestCase {
IndexReader reader = DirectoryReader.open(dir);
try {
int numPayloads = 0;
DocsAndPositionsEnum tpe = MultiFields.getTermPositionsEnum(reader, null, field, text, false);
DocsAndPositionsEnum tpe = MultiFields.getTermPositionsEnum(reader, null, field, text);
while (tpe.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
tpe.nextPosition();
if (tpe.hasPayload()) {

View File

@ -188,8 +188,7 @@ public class TestPayloads extends LuceneTestCase {
tps[i] = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
terms[i].field(),
new BytesRef(terms[i].text()),
false);
new BytesRef(terms[i].text()));
}
while (tps[0].nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
@ -226,8 +225,7 @@ public class TestPayloads extends LuceneTestCase {
DocsAndPositionsEnum tp = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
terms[0].field(),
new BytesRef(terms[0].text()),
false);
new BytesRef(terms[0].text()));
tp.nextDoc();
tp.nextPosition();
// NOTE: prior rev of this test was failing to first
@ -255,8 +253,7 @@ public class TestPayloads extends LuceneTestCase {
tp = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
terms[1].field(),
new BytesRef(terms[1].text()),
false);
new BytesRef(terms[1].text()));
tp.nextDoc();
tp.nextPosition();
assertEquals("Wrong payload length.", 1, tp.getPayload().length);
@ -299,8 +296,7 @@ public class TestPayloads extends LuceneTestCase {
tp = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
fieldName,
new BytesRef(singleTerm),
false);
new BytesRef(singleTerm));
tp.nextDoc();
tp.nextPosition();
@ -501,7 +497,7 @@ public class TestPayloads extends LuceneTestCase {
DocsAndPositionsEnum tp = null;
while (terms.next() != null) {
String termText = terms.term().utf8ToString();
tp = terms.docsAndPositions(liveDocs, tp, false);
tp = terms.docsAndPositions(liveDocs, tp);
while(tp.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
int freq = tp.freq();
for (int i = 0; i < freq; i++) {

View File

@ -518,7 +518,7 @@ public class TestPostingsFormat extends LuceneTestCase {
maxIndexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
boolean doCheckOffsets = allowOffsets && random().nextInt(3) <= 2;
boolean doCheckPayloads = options.contains(Option.PAYLOADS) && allowPositions && fieldInfo.hasPayloads();
boolean doCheckPayloads = options.contains(Option.PAYLOADS) && allowPositions && fieldInfo.hasPayloads() && random().nextInt(3) <= 2;;
DocsEnum prevDocsEnum = null;
@ -528,15 +528,24 @@ public class TestPostingsFormat extends LuceneTestCase {
if (!doCheckPositions) {
if (allowPositions && random().nextInt(10) == 7) {
// 10% of the time, even though we will not check positions, pull a DocsAndPositions enum
if (VERBOSE) {
System.out.println(" get DocsAndPositionsEnum (but we won't check positions)");
}
if (options.contains(Option.REUSE_ENUMS) && random().nextInt(10) < 9) {
prevDocsEnum = threadState.reuseDocsAndPositionsEnum;
}
threadState.reuseDocsAndPositionsEnum = termsEnum.docsAndPositions(liveDocs, (DocsAndPositionsEnum) prevDocsEnum, false);
int flags = 0;
if (random().nextBoolean()) {
flags |= DocsAndPositionsEnum.FLAG_OFFSETS;
}
if (random().nextBoolean()) {
flags |= DocsAndPositionsEnum.FLAG_PAYLOADS;
}
if (VERBOSE) {
System.out.println(" get DocsAndPositionsEnum (but we won't check positions) flags=" + flags);
}
threadState.reuseDocsAndPositionsEnum = termsEnum.docsAndPositions(liveDocs, (DocsAndPositionsEnum) prevDocsEnum, flags);
docsEnum = threadState.reuseDocsAndPositionsEnum;
docsAndPositionsEnum = threadState.reuseDocsAndPositionsEnum;
} else {
@ -551,13 +560,23 @@ public class TestPostingsFormat extends LuceneTestCase {
docsAndPositionsEnum = null;
}
} else {
if (VERBOSE) {
System.out.println(" get DocsAndPositionsEnum");
}
if (options.contains(Option.REUSE_ENUMS) && random().nextInt(10) < 9) {
prevDocsEnum = threadState.reuseDocsAndPositionsEnum;
}
threadState.reuseDocsAndPositionsEnum = termsEnum.docsAndPositions(liveDocs, (DocsAndPositionsEnum) prevDocsEnum, doCheckOffsets);
int flags = 0;
if (doCheckOffsets || random().nextInt(3) == 1) {
flags |= DocsAndPositionsEnum.FLAG_OFFSETS;
}
if (doCheckPayloads|| random().nextInt(3) == 1) {
flags |= DocsAndPositionsEnum.FLAG_PAYLOADS;
}
if (VERBOSE) {
System.out.println(" get DocsAndPositionsEnum flags=" + flags);
}
threadState.reuseDocsAndPositionsEnum = termsEnum.docsAndPositions(liveDocs, (DocsAndPositionsEnum) prevDocsEnum, flags);
docsEnum = threadState.reuseDocsAndPositionsEnum;
docsAndPositionsEnum = threadState.reuseDocsAndPositionsEnum;
}

View File

@ -83,7 +83,7 @@ public class TestPostingsOffsets extends LuceneTestCase {
IndexReader r = w.getReader();
w.close();
DocsAndPositionsEnum dp = MultiFields.getTermPositionsEnum(r, null, "content", new BytesRef("a"), true);
DocsAndPositionsEnum dp = MultiFields.getTermPositionsEnum(r, null, "content", new BytesRef("a"));
assertNotNull(dp);
assertEquals(0, dp.nextDoc());
assertEquals(2, dp.freq());
@ -95,7 +95,7 @@ public class TestPostingsOffsets extends LuceneTestCase {
assertEquals(17, dp.endOffset());
assertEquals(DocIdSetIterator.NO_MORE_DOCS, dp.nextDoc());
dp = MultiFields.getTermPositionsEnum(r, null, "content", new BytesRef("b"), true);
dp = MultiFields.getTermPositionsEnum(r, null, "content", new BytesRef("b"));
assertNotNull(dp);
assertEquals(0, dp.nextDoc());
assertEquals(1, dp.freq());
@ -104,7 +104,7 @@ public class TestPostingsOffsets extends LuceneTestCase {
assertEquals(9, dp.endOffset());
assertEquals(DocIdSetIterator.NO_MORE_DOCS, dp.nextDoc());
dp = MultiFields.getTermPositionsEnum(r, null, "content", new BytesRef("c"), true);
dp = MultiFields.getTermPositionsEnum(r, null, "content", new BytesRef("c"));
assertNotNull(dp);
assertEquals(0, dp.nextDoc());
assertEquals(1, dp.freq());
@ -155,7 +155,7 @@ public class TestPostingsOffsets extends LuceneTestCase {
String terms[] = { "one", "two", "three", "four", "five", "six", "seven", "eight", "nine", "ten", "hundred" };
for (String term : terms) {
DocsAndPositionsEnum dp = MultiFields.getTermPositionsEnum(reader, null, "numbers", new BytesRef(term), true);
DocsAndPositionsEnum dp = MultiFields.getTermPositionsEnum(reader, null, "numbers", new BytesRef(term));
int doc;
while((doc = dp.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
String storedNumbers = reader.document(doc).get("numbers");
@ -183,7 +183,7 @@ public class TestPostingsOffsets extends LuceneTestCase {
for (int j = 0; j < numSkippingTests; j++) {
int num = _TestUtil.nextInt(random(), 100, Math.min(numDocs-1, 999));
DocsAndPositionsEnum dp = MultiFields.getTermPositionsEnum(reader, null, "numbers", new BytesRef("hundred"), true);
DocsAndPositionsEnum dp = MultiFields.getTermPositionsEnum(reader, null, "numbers", new BytesRef("hundred"));
int doc = dp.advance(num);
assertEquals(num, doc);
int freq = dp.freq();
@ -312,7 +312,8 @@ public class TestPostingsOffsets extends LuceneTestCase {
assertEquals(expected.size(), docs.freq());
}
docsAndPositions = termsEnum.docsAndPositions(null, docsAndPositions, false);
// explicitly exclude offsets here
docsAndPositions = termsEnum.docsAndPositions(null, docsAndPositions, DocsAndPositionsEnum.FLAG_PAYLOADS);
assertNotNull(docsAndPositions);
//System.out.println(" doc/freq/pos");
while((doc = docsAndPositions.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
@ -327,7 +328,7 @@ public class TestPostingsOffsets extends LuceneTestCase {
}
}
docsAndPositionsAndOffsets = termsEnum.docsAndPositions(null, docsAndPositions, true);
docsAndPositionsAndOffsets = termsEnum.docsAndPositions(null, docsAndPositions);
assertNotNull(docsAndPositionsAndOffsets);
//System.out.println(" doc/freq/pos/offs");
while((doc = docsAndPositionsAndOffsets.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {

View File

@ -149,8 +149,7 @@ public class TestSegmentReader extends LuceneTestCase {
DocsAndPositionsEnum positions = MultiFields.getTermPositionsEnum(reader,
MultiFields.getLiveDocs(reader),
DocHelper.TEXT_FIELD_1_KEY,
new BytesRef("field"),
false);
new BytesRef("field"));
// NOTE: prior rev of this test was failing to first
// call next here:
assertTrue(positions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);

View File

@ -408,7 +408,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
BytesRef term2;
while((term2 = termsEnum3.next()) != null) {
System.out.println(" " + term2.utf8ToString() + ": freq=" + termsEnum3.totalTermFreq());
dpEnum = termsEnum3.docsAndPositions(null, dpEnum, false);
dpEnum = termsEnum3.docsAndPositions(null, dpEnum);
if (dpEnum != null) {
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
final int freq = dpEnum.freq();
@ -442,7 +442,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
BytesRef term2;
while((term2 = termsEnum3.next()) != null) {
System.out.println(" " + term2.utf8ToString() + ": freq=" + termsEnum3.totalTermFreq());
dpEnum = termsEnum3.docsAndPositions(null, dpEnum, false);
dpEnum = termsEnum3.docsAndPositions(null, dpEnum);
if (dpEnum != null) {
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
final int freq = dpEnum.freq();
@ -632,8 +632,8 @@ public class TestStressIndexing2 extends LuceneTestCase {
assertEquals(termsEnum1.totalTermFreq(),
termsEnum2.totalTermFreq());
dpEnum1 = termsEnum1.docsAndPositions(null, dpEnum1, false);
dpEnum2 = termsEnum2.docsAndPositions(null, dpEnum2, false);
dpEnum1 = termsEnum1.docsAndPositions(null, dpEnum1);
dpEnum2 = termsEnum2.docsAndPositions(null, dpEnum2);
if (dpEnum1 != null) {
assertNotNull(dpEnum2);
int docID1 = dpEnum1.nextDoc();

View File

@ -252,7 +252,7 @@ public class TestTermVectorsReader extends LuceneTestCase {
//System.out.println("Term: " + term);
assertEquals(testTerms[i], term);
dpEnum = termsEnum.docsAndPositions(null, dpEnum, false);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
assertNotNull(dpEnum);
int doc = dpEnum.docID();
assertTrue(doc == -1 || doc == DocIdSetIterator.NO_MORE_DOCS);
@ -263,7 +263,7 @@ public class TestTermVectorsReader extends LuceneTestCase {
}
assertEquals(DocIdSetIterator.NO_MORE_DOCS, dpEnum.nextDoc());
dpEnum = termsEnum.docsAndPositions(null, dpEnum, true);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
doc = dpEnum.docID();
assertTrue(doc == -1 || doc == DocIdSetIterator.NO_MORE_DOCS);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@ -306,7 +306,7 @@ public class TestTermVectorsReader extends LuceneTestCase {
String term = text.utf8ToString();
assertEquals(testTerms[i], term);
dpEnum = termsEnum.docsAndPositions(null, dpEnum, false);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
assertNotNull(dpEnum);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(dpEnum.freq(), positions[i].length);
@ -315,7 +315,7 @@ public class TestTermVectorsReader extends LuceneTestCase {
}
assertEquals(DocIdSetIterator.NO_MORE_DOCS, dpEnum.nextDoc());
dpEnum = termsEnum.docsAndPositions(null, dpEnum, true);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertNotNull(dpEnum);
assertEquals(dpEnum.freq(), positions[i].length);

View File

@ -69,7 +69,7 @@ public class TestTermVectorsWriter extends LuceneTestCase {
// Token "" occurred once
assertEquals(1, termsEnum.totalTermFreq());
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null, true);
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
dpEnum.nextPosition();
assertEquals(8, dpEnum.startOffset());
@ -78,7 +78,7 @@ public class TestTermVectorsWriter extends LuceneTestCase {
// Token "abcd" occurred three times
assertEquals(new BytesRef("abcd"), termsEnum.next());
dpEnum = termsEnum.docsAndPositions(null, dpEnum, true);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
assertEquals(3, termsEnum.totalTermFreq());
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@ -118,7 +118,7 @@ public class TestTermVectorsWriter extends LuceneTestCase {
IndexReader r = DirectoryReader.open(dir);
TermsEnum termsEnum = r.getTermVectors(0).terms("field").iterator(null);
assertNotNull(termsEnum.next());
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null, true);
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null);
assertEquals(2, termsEnum.totalTermFreq());
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@ -153,7 +153,7 @@ public class TestTermVectorsWriter extends LuceneTestCase {
IndexReader r = DirectoryReader.open(dir);
TermsEnum termsEnum = r.getTermVectors(0).terms("field").iterator(null);
assertNotNull(termsEnum.next());
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null, true);
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null);
assertEquals(2, termsEnum.totalTermFreq());
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@ -192,7 +192,7 @@ public class TestTermVectorsWriter extends LuceneTestCase {
IndexReader r = DirectoryReader.open(dir);
TermsEnum termsEnum = r.getTermVectors(0).terms("field").iterator(null);
assertNotNull(termsEnum.next());
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null, true);
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null);
assertEquals(2, termsEnum.totalTermFreq());
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@ -228,7 +228,7 @@ public class TestTermVectorsWriter extends LuceneTestCase {
IndexReader r = DirectoryReader.open(dir);
TermsEnum termsEnum = r.getTermVectors(0).terms("field").iterator(null);
assertNotNull(termsEnum.next());
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null, true);
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null);
assertEquals(2, termsEnum.totalTermFreq());
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@ -265,7 +265,7 @@ public class TestTermVectorsWriter extends LuceneTestCase {
IndexReader r = DirectoryReader.open(dir);
TermsEnum termsEnum = r.getTermVectors(0).terms("field").iterator(null);
assertNotNull(termsEnum.next());
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null, true);
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
dpEnum.nextPosition();
@ -273,14 +273,14 @@ public class TestTermVectorsWriter extends LuceneTestCase {
assertEquals(4, dpEnum.endOffset());
assertNotNull(termsEnum.next());
dpEnum = termsEnum.docsAndPositions(null, dpEnum, true);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
dpEnum.nextPosition();
assertEquals(11, dpEnum.startOffset());
assertEquals(17, dpEnum.endOffset());
assertNotNull(termsEnum.next());
dpEnum = termsEnum.docsAndPositions(null, dpEnum, true);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
dpEnum.nextPosition();
assertEquals(18, dpEnum.startOffset());
@ -310,7 +310,7 @@ public class TestTermVectorsWriter extends LuceneTestCase {
IndexReader r = DirectoryReader.open(dir);
TermsEnum termsEnum = r.getTermVectors(0).terms("field").iterator(null);
assertNotNull(termsEnum.next());
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null, true);
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null);
assertEquals(1, (int) termsEnum.totalTermFreq());
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@ -319,7 +319,7 @@ public class TestTermVectorsWriter extends LuceneTestCase {
assertEquals(7, dpEnum.endOffset());
assertNotNull(termsEnum.next());
dpEnum = termsEnum.docsAndPositions(null, dpEnum, true);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
dpEnum.nextPosition();
assertEquals(8, dpEnum.startOffset());
@ -353,7 +353,7 @@ public class TestTermVectorsWriter extends LuceneTestCase {
IndexReader r = DirectoryReader.open(dir);
TermsEnum termsEnum = r.getTermVectors(0).terms("field").iterator(null);
assertNotNull(termsEnum.next());
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null, true);
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null);
assertEquals(1, (int) termsEnum.totalTermFreq());
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@ -362,7 +362,7 @@ public class TestTermVectorsWriter extends LuceneTestCase {
assertEquals(4, dpEnum.endOffset());
assertNotNull(termsEnum.next());
dpEnum = termsEnum.docsAndPositions(null, dpEnum, true);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
dpEnum.nextPosition();
assertEquals(6, dpEnum.startOffset());

View File

@ -103,8 +103,7 @@ public class TestPositionIncrement extends LuceneTestCase {
DocsAndPositionsEnum pos = MultiFields.getTermPositionsEnum(searcher.getIndexReader(),
MultiFields.getLiveDocs(searcher.getIndexReader()),
"field",
new BytesRef("1"),
false);
new BytesRef("1"));
pos.nextDoc();
// first token should be at position 0
assertEquals(0, pos.nextPosition());
@ -112,8 +111,7 @@ public class TestPositionIncrement extends LuceneTestCase {
pos = MultiFields.getTermPositionsEnum(searcher.getIndexReader(),
MultiFields.getLiveDocs(searcher.getIndexReader()),
"field",
new BytesRef("2"),
false);
new BytesRef("2"));
pos.nextDoc();
// second token should be at position 2
assertEquals(2, pos.nextPosition());
@ -216,8 +214,7 @@ public class TestPositionIncrement extends LuceneTestCase {
DocsAndPositionsEnum tp = r.termPositionsEnum(r.getLiveDocs(),
"content",
new BytesRef("a"),
false);
new BytesRef("a"));
int count = 0;
assertTrue(tp.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);

View File

@ -134,19 +134,19 @@ public class TestTermVectors extends LuceneTestCase {
assertNotNull(terms);
TermsEnum termsEnum = terms.iterator(null);
assertEquals("content", termsEnum.next().utf8ToString());
dpEnum = termsEnum.docsAndPositions(null, dpEnum, false);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(1, dpEnum.freq());
assertEquals(expectedPositions[0], dpEnum.nextPosition());
assertEquals("here", termsEnum.next().utf8ToString());
dpEnum = termsEnum.docsAndPositions(null, dpEnum, false);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(1, dpEnum.freq());
assertEquals(expectedPositions[1], dpEnum.nextPosition());
assertEquals("some", termsEnum.next().utf8ToString());
dpEnum = termsEnum.docsAndPositions(null, dpEnum, false);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(1, dpEnum.freq());
assertEquals(expectedPositions[2], dpEnum.nextPosition());
@ -176,7 +176,7 @@ public class TestTermVectors extends LuceneTestCase {
if (shouldBePosVector || shouldBeOffVector) {
while(true) {
dpEnum = termsEnum.docsAndPositions(null, dpEnum, shouldBeOffVector);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
assertNotNull(dpEnum);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@ -426,7 +426,7 @@ public class TestTermVectors extends LuceneTestCase {
assertNotNull(termsEnum.next());
assertEquals("one", termsEnum.term().utf8ToString());
assertEquals(5, termsEnum.totalTermFreq());
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null, false);
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null);
assertNotNull(dpEnum);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(5, dpEnum.freq());
@ -434,7 +434,7 @@ public class TestTermVectors extends LuceneTestCase {
assertEquals(i, dpEnum.nextPosition());
}
dpEnum = termsEnum.docsAndPositions(null, dpEnum, true);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
assertNotNull(dpEnum);
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(5, dpEnum.freq());

View File

@ -53,7 +53,7 @@ public class PayloadIterator {
this.buffer = buffer;
// TODO (Facet): avoid Multi*?
Bits liveDocs = MultiFields.getLiveDocs(indexReader);
this.tp = MultiFields.getTermPositionsEnum(indexReader, liveDocs, term.field(), term.bytes(), false);
this.tp = MultiFields.getTermPositionsEnum(indexReader, liveDocs, term.field(), term.bytes(), DocsAndPositionsEnum.FLAG_PAYLOADS);
}
/**

View File

@ -106,7 +106,7 @@ class ParentArray {
Bits liveDocs = MultiFields.getLiveDocs(indexReader);
DocsAndPositionsEnum positions = MultiFields.getTermPositionsEnum(indexReader, liveDocs,
Consts.FIELD_PAYLOADS, new BytesRef(Consts.PAYLOAD_PARENT),
false);
DocsAndPositionsEnum.FLAG_PAYLOADS);
if ((positions == null || positions.advance(first) == DocIdSetIterator.NO_MORE_DOCS) && first < num) {
throw new CorruptIndexException("Missing parent data for category " + first);
}

View File

@ -127,7 +127,7 @@ public class TokenSources {
private static boolean hasPositions(Terms vector) throws IOException {
final TermsEnum termsEnum = vector.iterator(null);
if (termsEnum.next() != null) {
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null, false);
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null, DocsAndPositionsEnum.FLAG_PAYLOADS);
if (dpEnum != null) {
int doc = dpEnum.nextDoc();
assert doc >= 0 && doc != DocIdSetIterator.NO_MORE_DOCS;
@ -222,18 +222,21 @@ public class TokenSources {
DocsAndPositionsEnum dpEnum = null;
while ((text = termsEnum.next()) != null) {
dpEnum = termsEnum.docsAndPositions(null, dpEnum, true);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
if (dpEnum == null) {
throw new IllegalArgumentException(
"Required TermVector Offset information was not found");
}
final String term = text.utf8ToString();
dpEnum.nextDoc();
final int freq = dpEnum.freq();
for(int posUpto=0;posUpto<freq;posUpto++) {
final int pos = dpEnum.nextPosition();
if (dpEnum.startOffset() < 0) {
throw new IllegalArgumentException(
"Required TermVector Offset information was not found");
}
final Token token = new Token(term,
dpEnum.startOffset(),
dpEnum.endOffset());

View File

@ -60,18 +60,14 @@ public final class TokenStreamFromTermPositionVector extends TokenStream {
BytesRef text;
DocsAndPositionsEnum dpEnum = null;
while((text = termsEnum.next()) != null) {
dpEnum = termsEnum.docsAndPositions(null, dpEnum, true);
final boolean hasOffsets;
if (dpEnum == null) {
hasOffsets = false;
dpEnum = termsEnum.docsAndPositions(null, dpEnum, false);
} else {
hasOffsets = true;
}
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
assert dpEnum != null; // presumably checked by TokenSources.hasPositions earlier
boolean hasOffsets = true;
dpEnum.nextDoc();
final int freq = dpEnum.freq();
for (int j = 0; j < freq; j++) {
int pos = dpEnum.nextPosition();
hasOffsets &= dpEnum.startOffset() >= 0;
Token token;
if (hasOffsets) {
token = new Token(text.utf8ToString(),

View File

@ -104,7 +104,7 @@ public class FieldTermStack {
if (!termSet.contains(term)) {
continue;
}
dpEnum = termsEnum.docsAndPositions(null, dpEnum, true);
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
if (dpEnum == null) {
// null snippet
return;
@ -118,6 +118,9 @@ public class FieldTermStack {
int pos = dpEnum.nextPosition();
// For weight look here: http://lucene.apache.org/core/3_6_0/api/core/org/apache/lucene/search/DefaultSimilarity.html
weight = ( float ) ( Math.log( numDocs / ( double ) ( reader.docFreq( fieldName, text ) + 1 ) ) + 1.0 );
if (dpEnum.startOffset() < 0) {
return; // no offsets, null snippet
}
termList.add( new TermInfo( term, dpEnum.startOffset(), dpEnum.endOffset(), pos, weight ) );
}
}

View File

@ -880,10 +880,7 @@ public class MemoryIndex {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) {
if (needsOffsets) {
return null;
}
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) {
if (reuse == null || !(reuse instanceof MemoryDocsAndPositionsEnum)) {
reuse = new MemoryDocsAndPositionsEnum();
}

View File

@ -225,7 +225,7 @@ public class MemoryIndexTest extends BaseTokenStreamTestCase {
MemoryIndex memory = new MemoryIndex(true);
memory.addField("foo", "bar", analyzer);
AtomicReader reader = (AtomicReader) memory.createSearcher().getIndexReader();
DocsAndPositionsEnum disi = reader.termPositionsEnum(null, "foo", new BytesRef("bar"), false);
DocsAndPositionsEnum disi = reader.termPositionsEnum(null, "foo", new BytesRef("bar"));
int docid = disi.docID();
assertTrue(docid == -1 || docid == DocIdSetIterator.NO_MORE_DOCS);
assertTrue(disi.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@ -236,7 +236,7 @@ public class MemoryIndexTest extends BaseTokenStreamTestCase {
// now reuse and check again
TermsEnum te = reader.terms("foo").iterator(null);
assertTrue(te.seekExact(new BytesRef("bar"), true));
disi = te.docsAndPositions(null, disi, false);
disi = te.docsAndPositions(null, disi);
docid = disi.docID();
assertTrue(docid == -1 || docid == DocIdSetIterator.NO_MORE_DOCS);
assertTrue(disi.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);

View File

@ -391,10 +391,7 @@ public class RAMOnlyPostingsFormat extends PostingsFormat {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) {
if (needsOffsets) {
return null;
}
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) {
return new RAMDocsAndPositionsEnum(ramField.termToDocs.get(current), liveDocs);
}
}

View File

@ -138,7 +138,7 @@ public class AssertingAtomicReader extends FilterAtomicReader {
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
assert state == State.POSITIONED: "docsAndPositions(...) called on unpositioned TermsEnum";
// TODO: should we give this thing a random to be super-evil,
@ -146,7 +146,7 @@ public class AssertingAtomicReader extends FilterAtomicReader {
if (reuse instanceof AssertingDocsAndPositionsEnum) {
reuse = ((AssertingDocsAndPositionsEnum) reuse).in;
}
DocsAndPositionsEnum docs = super.docsAndPositions(liveDocs, reuse, needsOffsets);
DocsAndPositionsEnum docs = super.docsAndPositions(liveDocs, reuse, flags);
return docs == null ? null : new AssertingDocsAndPositionsEnum(docs);
}

View File

@ -866,13 +866,23 @@ public class _TestUtil {
if (!termsEnum.seekExact(term, random.nextBoolean())) {
return null;
}
return docs(random, termsEnum, liveDocs, reuse, needsFreqs);
}
// Returns a DocsEnum from a positioned TermsEnum, but
// randomly sometimes uses a DocsAndFreqsEnum, DocsAndPositionsEnum.
public static DocsEnum docs(Random random, TermsEnum termsEnum, Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
if (random.nextBoolean()) {
if (random.nextBoolean()) {
// TODO: cast re-use to D&PE if we can...?
DocsAndPositionsEnum docsAndPositions = termsEnum.docsAndPositions(liveDocs, null, true);
if (docsAndPositions == null) {
docsAndPositions = termsEnum.docsAndPositions(liveDocs, null, false);
final int flags;
switch (random.nextInt(4)) {
case 0: flags = 0; break;
case 1: flags = DocsAndPositionsEnum.FLAG_OFFSETS; break;
case 2: flags = DocsAndPositionsEnum.FLAG_PAYLOADS; break;
default: flags = DocsAndPositionsEnum.FLAG_OFFSETS | DocsAndPositionsEnum.FLAG_PAYLOADS; break;
}
// TODO: cast to DocsAndPositionsEnum?
DocsAndPositionsEnum docsAndPositions = termsEnum.docsAndPositions(liveDocs, null, flags);
if (docsAndPositions != null) {
return docsAndPositions;
}
@ -884,28 +894,6 @@ public class _TestUtil {
}
return termsEnum.docs(liveDocs, reuse, needsFreqs);
}
// Returns a DocsEnum from a positioned TermsEnum, but
// randomly sometimes uses a DocsAndFreqsEnum, DocsAndPositionsEnum.
public static DocsEnum docs(Random random, TermsEnum termsEnum, Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
if (random.nextBoolean()) {
if (random.nextBoolean()) {
// TODO: cast re-use to D&PE if we can...?
DocsAndPositionsEnum docsAndPositions = termsEnum.docsAndPositions(liveDocs, null, true);
if (docsAndPositions == null) {
docsAndPositions = termsEnum.docsAndPositions(liveDocs, null, false);
}
if (docsAndPositions != null) {
return docsAndPositions;
}
}
final DocsEnum docsAndFreqs = termsEnum.docs(liveDocs, null, true);
if (docsAndFreqs != null) {
return docsAndFreqs;
}
}
return termsEnum.docs(liveDocs, null, needsFreqs);
}
public static CharSequence stringToCharSequence(String string, Random random) {
return bytesToCharSequence(new BytesRef(string), random);

View File

@ -329,28 +329,19 @@ public class TermVectorComponent extends SearchComponent implements SolrCoreAwar
termInfo.add("tf", freq);
}
dpEnum = termsEnum.docsAndPositions(null, dpEnum, fieldOptions.offsets);
boolean useOffsets = fieldOptions.offsets;
if (dpEnum == null) {
useOffsets = false;
dpEnum = termsEnum.docsAndPositions(null, dpEnum, false);
}
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
boolean useOffsets = false;
boolean usePositions = false;
if (dpEnum != null) {
dpEnum.nextDoc();
usePositions = fieldOptions.positions;
}
NamedList<Number> theOffsets = null;
if (useOffsets) {
theOffsets = new NamedList<Number>();
termInfo.add("offsets", theOffsets);
useOffsets = fieldOptions.offsets;
}
NamedList<Integer> positionsNL = null;
NamedList<Number> theOffsets = null;
if (usePositions || theOffsets != null) {
if (usePositions || useOffsets) {
for (int i = 0; i < freq; i++) {
final int pos = dpEnum.nextPosition();
if (usePositions && pos >= 0) {
@ -361,6 +352,15 @@ public class TermVectorComponent extends SearchComponent implements SolrCoreAwar
positionsNL.add("position", pos);
}
if (useOffsets && theOffsets == null) {
if (dpEnum.startOffset() == -1) {
useOffsets = false;
} else {
theOffsets = new NamedList<Number>();
termInfo.add("offsets", theOffsets);
}
}
if (theOffsets != null) {
theOffsets.add("start", dpEnum.startOffset());
theOffsets.add("end", dpEnum.endOffset());