LUCENE-3109: rename Fields/Producer/Consumer to InvertedFields*

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1310969 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2012-04-08 10:09:19 +00:00
parent 13b75650e9
commit c9031ca725
90 changed files with 299 additions and 320 deletions

View File

@ -259,6 +259,9 @@ Changes in backwards compatibility policy
* LUCENE-2000: clone() now returns covariant types where possible. (ryan)
* LUCENE-3109: Rename Fields/Producer/Consumer -> InvertedFields*
(Iulius Curt via Mike McCandless)
Changes in Runtime Behavior
* LUCENE-2846: omitNorms now behaves like omitTermFrequencyAndPositions, if you

View File

@ -33,7 +33,7 @@ import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
@ -70,7 +70,7 @@ public class TokenSources {
String field, Document doc, Analyzer analyzer) throws IOException {
TokenStream ts = null;
Fields vectors = reader.getTermVectors(docId);
InvertedFields vectors = reader.getTermVectors(docId);
if (vectors != null) {
Terms vector = vectors.terms(field);
if (vector != null) {
@ -102,7 +102,7 @@ public class TokenSources {
String field, Analyzer analyzer) throws IOException {
TokenStream ts = null;
Fields vectors = reader.getTermVectors(docId);
InvertedFields vectors = reader.getTermVectors(docId);
if (vectors != null) {
Terms vector = vectors.terms(field);
if (vector != null) {
@ -275,7 +275,7 @@ public class TokenSources {
public static TokenStream getTokenStream(IndexReader reader, int docId,
String field) throws IOException {
Fields vectors = reader.getTermVectors(docId);
InvertedFields vectors = reader.getTermVectors(docId);
if (vectors == null) {
throw new IllegalArgumentException(field + " in doc #" + docId
+ "does not have any term position data stored");

View File

@ -22,7 +22,7 @@ import java.util.LinkedList;
import java.util.Set;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
@ -78,7 +78,7 @@ public class FieldTermStack {
// just return to make null snippet if un-matched fieldName specified when fieldMatch == true
if( termSet == null ) return;
final Fields vectors = reader.getTermVectors(docId);
final InvertedFields vectors = reader.getTermVectors(docId);
if (vectors == null) {
// null snippet
return;

View File

@ -35,13 +35,13 @@ import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.Norm;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.OrdTermState;
import org.apache.lucene.index.StoredFieldVisitor;
@ -58,7 +58,6 @@ import org.apache.lucene.store.RAMDirectory; // for javadocs
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Constants; // for javadocs
import org.apache.lucene.util.RamUsageEstimator;
/**
@ -716,7 +715,7 @@ public class MemoryIndex {
return fieldInfos;
}
private class MemoryFields extends Fields {
private class MemoryFields extends InvertedFields {
@Override
public FieldsEnum iterator() {
return new FieldsEnum() {
@ -791,7 +790,7 @@ public class MemoryIndex {
}
@Override
public Fields fields() {
public InvertedFields fields() {
sortFields();
return new MemoryFields();
}
@ -1017,7 +1016,7 @@ public class MemoryIndex {
}
@Override
public Fields getTermVectors(int docID) {
public InvertedFields getTermVectors(int docID) {
if (docID == 0) {
return fields();
} else {

View File

@ -19,8 +19,8 @@ package org.apache.lucene.misc;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.Terms;
@ -115,7 +115,7 @@ public class HighFreqTerms {
TermStatsQueue tiq = null;
if (field != null) {
Fields fields = MultiFields.getFields(reader);
InvertedFields fields = MultiFields.getFields(reader);
if (fields == null) {
throw new RuntimeException("field " + field + " not found");
}
@ -126,7 +126,7 @@ public class HighFreqTerms {
fillQueue(termsEnum, tiq, field);
}
} else {
Fields fields = MultiFields.getFields(reader);
InvertedFields fields = MultiFields.getFields(reader);
if (fields == null) {
throw new RuntimeException("no fields found for this index");
}

View File

@ -56,7 +56,7 @@ import org.apache.lucene.util.DoubleBarrelLRUCache;
* implementation of the terms dict index.
* @lucene.experimental */
public class BlockTermsReader extends FieldsProducer {
public class BlockTermsReader extends InvertedFieldsProducer {
// Open input to the main terms dict file (_X.tis)
private final IndexInput in;

View File

@ -47,7 +47,7 @@ import org.apache.lucene.util.RamUsageEstimator;
* @lucene.experimental
*/
public class BlockTermsWriter extends FieldsConsumer {
public class BlockTermsWriter extends InvertedFieldsConsumer {
final static String CODEC_NAME = "BLOCK_TERMS_DICT";

View File

@ -85,7 +85,7 @@ import org.apache.lucene.util.fst.Util;
* @lucene.experimental
*/
public class BlockTreeTermsReader extends FieldsProducer {
public class BlockTreeTermsReader extends InvertedFieldsProducer {
// Open input to the main terms dict file (_X.tib)
private final IndexInput in;

View File

@ -83,7 +83,7 @@ import org.apache.lucene.util.fst.Util;
* @lucene.experimental
*/
public class BlockTreeTermsWriter extends FieldsConsumer {
public class BlockTreeTermsWriter extends InvertedFieldsConsumer {
public final static int DEFAULT_MIN_BLOCK_SIZE = 25;
public final static int DEFAULT_MAX_BLOCK_SIZE = 48;

View File

@ -21,7 +21,7 @@ import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.Terms;
@ -33,7 +33,7 @@ import org.apache.lucene.index.Terms;
*
* @lucene.experimental
*/
public abstract class FieldsConsumer implements Closeable {
public abstract class InvertedFieldsConsumer implements Closeable {
/** Add a new field */
public abstract TermsConsumer addField(FieldInfo field) throws IOException;
@ -41,7 +41,7 @@ public abstract class FieldsConsumer implements Closeable {
/** Called when we are done adding everything. */
public abstract void close() throws IOException;
public void merge(MergeState mergeState, Fields fields) throws IOException {
public void merge(MergeState mergeState, InvertedFields fields) throws IOException {
FieldsEnum fieldsEnum = fields.iterator();
assert fieldsEnum != null;
String field;

View File

@ -20,7 +20,7 @@ package org.apache.lucene.codecs;
import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
/** Abstract API that produces terms, doc, freq, prox and
* payloads postings.
@ -28,6 +28,6 @@ import org.apache.lucene.index.Fields;
* @lucene.experimental
*/
public abstract class FieldsProducer extends Fields implements Closeable {
public abstract class InvertedFieldsProducer extends InvertedFields implements Closeable {
public abstract void close() throws IOException;
}

View File

@ -49,12 +49,12 @@ public abstract class PostingsFormat implements NamedSPILoader.NamedSPI {
}
/** Writes a new segment */
public abstract FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException;
public abstract InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException;
/** Reads a segment. NOTE: by the time this call
* returns, it must hold open any files it will need to
* use; else, those files may be deleted. */
public abstract FieldsProducer fieldsProducer(SegmentReadState state) throws IOException;
public abstract InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException;
/**
* Gathers files associated with this segment

View File

@ -22,7 +22,7 @@ import java.io.IOException;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute; // javadocs
import org.apache.lucene.index.DocsAndPositionsEnum; // javadocs
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
/**
* Codec API for reading term vectors:
@ -35,7 +35,7 @@ public abstract class TermVectorsReader implements Cloneable,Closeable {
* term vectors were not indexed. If offsets are
* available they are in an {@link OffsetAttribute}
* available from the {@link DocsAndPositionsEnum}. */
public abstract Fields get(int doc) throws IOException;
public abstract InvertedFields get(int doc) throws IOException;
/** Create a clone that one caller at a time may use to
* read term vectors. */

View File

@ -24,7 +24,7 @@ import java.util.Comparator;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.Terms;
@ -153,7 +153,7 @@ public abstract class TermVectorsWriter implements Closeable {
}
// NOTE: it's very important to first assign to vectors then pass it to
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
Fields vectors = reader.reader.getTermVectors(docID);
InvertedFields vectors = reader.reader.getTermVectors(docID);
addAllDocVectors(vectors, mergeState.fieldInfos);
docCount++;
mergeState.checkAbort.work(300);
@ -166,9 +166,9 @@ public abstract class TermVectorsWriter implements Closeable {
/** Safe (but, slowish) default method to write every
* vector field in the document. This default
* implementation requires that the vectors implement
* both Fields.getUniqueFieldCount and
* both InvertedFields.getUniqueFieldCount and
* Terms.getUniqueTermCount. */
protected final void addAllDocVectors(Fields vectors, FieldInfos fieldInfos) throws IOException {
protected final void addAllDocVectors(InvertedFields vectors, FieldInfos fieldInfos) throws IOException {
if (vectors == null) {
startDocument(0);
return;

View File

@ -22,8 +22,8 @@ import java.util.Set;
import org.apache.lucene.codecs.BlockTreeTermsReader;
import org.apache.lucene.codecs.BlockTreeTermsWriter;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
@ -44,11 +44,11 @@ class AppendingPostingsFormat extends PostingsFormat {
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase docsWriter = new Lucene40PostingsWriter(state);
boolean success = false;
try {
FieldsConsumer ret = new AppendingTermsWriter(state, docsWriter, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
InvertedFieldsConsumer ret = new AppendingTermsWriter(state, docsWriter, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
success = true;
return ret;
} finally {
@ -59,12 +59,12 @@ class AppendingPostingsFormat extends PostingsFormat {
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix);
boolean success = false;
try {
FieldsProducer ret = new AppendingTermsReader(
InvertedFieldsProducer ret = new AppendingTermsReader(
state.dir,
state.fieldInfos,
state.segmentInfo.name,

View File

@ -25,7 +25,7 @@ import java.util.Iterator;
import java.util.Map;
import java.util.TreeMap;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo.IndexOptions;
@ -49,7 +49,7 @@ import org.apache.lucene.util.UnicodeUtil;
* @deprecated (4.0)
*/
@Deprecated
class Lucene3xFields extends FieldsProducer {
class Lucene3xFields extends InvertedFieldsProducer {
private static final boolean DEBUG_SURROGATES = false;
@ -59,18 +59,14 @@ class Lucene3xFields extends FieldsProducer {
public final IndexInput freqStream;
public final IndexInput proxStream;
final private FieldInfos fieldInfos;
private final SegmentInfo si;
final TreeMap<String,FieldInfo> fields = new TreeMap<String,FieldInfo>();
final Map<String,Terms> preTerms = new HashMap<String,Terms>();
private final Directory dir;
private final IOContext context;
private Directory cfsReader;
public Lucene3xFields(Directory dir, FieldInfos fieldInfos, SegmentInfo info, IOContext context, int indexDivisor)
throws IOException {
si = info;
// NOTE: we must always load terms index, even for
// "sequential" scan during merging, because what is
// sequential to merger may not be to TermInfosReader
@ -88,7 +84,6 @@ class Lucene3xFields extends FieldsProducer {
tisNoIndex = null;
tis = r;
}
this.context = context;
this.fieldInfos = fieldInfos;
// make sure that all index files have been read or are kept open

View File

@ -20,8 +20,8 @@ package org.apache.lucene.codecs.lucene3x;
import java.util.Set;
import java.io.IOException;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentWriteState;
@ -55,12 +55,12 @@ class Lucene3xPostingsFormat extends PostingsFormat {
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
return new Lucene3xFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor);
}

View File

@ -30,7 +30,7 @@ import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexFormatTooNewException;
@ -195,7 +195,7 @@ class Lucene3xTermVectorsReader extends TermVectorsReader {
return size;
}
private class TVFields extends Fields {
private class TVFields extends InvertedFields {
private final int[] fieldNumbers;
private final long[] fieldFPs;
private final Map<Integer,Integer> fieldNumberToIndex = new HashMap<Integer,Integer>();
@ -654,12 +654,12 @@ class Lucene3xTermVectorsReader extends TermVectorsReader {
}
@Override
public Fields get(int docID) throws IOException {
public InvertedFields get(int docID) throws IOException {
if (docID < 0 || docID >= numTotalDocs) {
throw new IllegalArgumentException("doID=" + docID + " is out of bounds [0.." + (numTotalDocs-1) + "]");
}
if (tvx != null) {
Fields fields = new TVFields(docID);
InvertedFields fields = new TVFields(docID);
if (fields.getUniqueFieldCount() == 0) {
// TODO: we can improve writer here, eg write 0 into
// tvx file, so we know on first read from tvx that

View File

@ -22,8 +22,8 @@ import java.util.Set;
import org.apache.lucene.codecs.BlockTreeTermsReader;
import org.apache.lucene.codecs.BlockTreeTermsWriter;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
@ -54,7 +54,7 @@ public class Lucene40PostingsFormat extends PostingsFormat {
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase docs = new Lucene40PostingsWriter(state);
// TODO: should we make the terms index more easily
@ -63,7 +63,7 @@ public class Lucene40PostingsFormat extends PostingsFormat {
// Or... you must make a new Codec for this?
boolean success = false;
try {
FieldsConsumer ret = new BlockTreeTermsWriter(state, docs, minBlockSize, maxBlockSize);
InvertedFieldsConsumer ret = new BlockTreeTermsWriter(state, docs, minBlockSize, maxBlockSize);
success = true;
return ret;
} finally {
@ -76,12 +76,12 @@ public class Lucene40PostingsFormat extends PostingsFormat {
public final static int TERMS_CACHE_SIZE = 1024;
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix);
boolean success = false;
try {
FieldsProducer ret = new BlockTreeTermsReader(
InvertedFieldsProducer ret = new BlockTreeTermsReader(
state.dir,
state.fieldInfos,
state.segmentInfo.name,

View File

@ -30,7 +30,7 @@ import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexFormatTooNewException;
@ -225,7 +225,7 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
return size;
}
private class TVFields extends Fields {
private class TVFields extends InvertedFields {
private final int[] fieldNumbers;
private final long[] fieldFPs;
private final Map<Integer,Integer> fieldNumberToIndex = new HashMap<Integer,Integer>();
@ -668,12 +668,12 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
}
@Override
public Fields get(int docID) throws IOException {
public InvertedFields get(int docID) throws IOException {
if (docID < 0 || docID >= numTotalDocs) {
throw new IllegalArgumentException("doID=" + docID + " is out of bounds [0.." + (numTotalDocs-1) + "]");
}
if (tvx != null) {
Fields fields = new TVFields(docID);
InvertedFields fields = new TVFields(docID);
if (fields.getUniqueFieldCount() == 0) {
// TODO: we can improve writer here, eg write 0 into
// tvx file, so we know on first read from tvx that

View File

@ -23,7 +23,7 @@ import java.util.Comparator;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.MergePolicy.MergeAbortedException;
import org.apache.lucene.index.MergeState;
@ -231,7 +231,7 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
}
@Override
public final int merge(MergeState mergeState) throws IOException {
public int merge(MergeState mergeState) throws IOException {
// Used for bulk-reading raw bytes for term vectors
int rawDocLengths[] = new int[MAX_RAW_MERGE_DOCS];
int rawDocLengths2[] = new int[MAX_RAW_MERGE_DOCS];
@ -309,7 +309,7 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
// NOTE: it's very important to first assign to vectors then pass it to
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
Fields vectors = reader.reader.getTermVectors(docNum);
InvertedFields vectors = reader.reader.getTermVectors(docNum);
addAllDocVectors(vectors, mergeState.fieldInfos);
totalNumDocs++;
mergeState.checkAbort.work(300);
@ -339,7 +339,7 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
for (int docNum = 0; docNum < maxDoc; docNum++) {
// NOTE: it's very important to first assign to vectors then pass it to
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
Fields vectors = reader.reader.getTermVectors(docNum);
InvertedFields vectors = reader.reader.getTermVectors(docNum);
addAllDocVectors(vectors, mergeState.fieldInfos);
mergeState.checkAbort.work(300);
}

View File

@ -24,8 +24,8 @@ import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.codecs.PostingsConsumer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.TermStats;
@ -283,12 +283,12 @@ public class MemoryPostingsFormat extends PostingsFormat {
private static String EXTENSION = "ram";
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
final String fileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, EXTENSION);
final IndexOutput out = state.directory.createOutput(fileName, state.context);
return new FieldsConsumer() {
return new InvertedFieldsConsumer() {
@Override
public TermsConsumer addField(FieldInfo field) {
//System.out.println("\naddField field=" + field.name);
@ -840,7 +840,7 @@ public class MemoryPostingsFormat extends PostingsFormat {
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, EXTENSION);
final IndexInput in = state.dir.openInput(fileName, IOContext.READONCE);
@ -860,7 +860,7 @@ public class MemoryPostingsFormat extends PostingsFormat {
in.close();
}
return new FieldsProducer() {
return new InvertedFieldsProducer() {
@Override
public FieldsEnum iterator() {
final Iterator<TermsReader> iter = fields.values().iterator();

View File

@ -28,8 +28,8 @@ import java.util.ServiceLoader; // javadocs
import java.util.Set;
import java.util.TreeMap;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.TermsConsumer;
import org.apache.lucene.index.FieldInfo;
@ -72,17 +72,17 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state)
public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state)
throws IOException {
return new FieldsWriter(state);
}
// NOTE: not private to avoid $accessN at runtime!!
static class FieldsConsumerAndID implements Closeable {
final FieldsConsumer fieldsConsumer;
final InvertedFieldsConsumer fieldsConsumer;
final String segmentSuffix;
public FieldsConsumerAndID(FieldsConsumer fieldsConsumer, String segmentSuffix) {
public FieldsConsumerAndID(InvertedFieldsConsumer fieldsConsumer, String segmentSuffix) {
this.fieldsConsumer = fieldsConsumer;
this.segmentSuffix = segmentSuffix;
}
@ -93,7 +93,7 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
}
};
private class FieldsWriter extends FieldsConsumer {
private class FieldsWriter extends InvertedFieldsConsumer {
private final Map<PostingsFormat,FieldsConsumerAndID> formats = new IdentityHashMap<PostingsFormat,FieldsConsumerAndID>();
@ -181,10 +181,10 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
}
}
private class FieldsReader extends FieldsProducer {
private class FieldsReader extends InvertedFieldsProducer {
private final Map<String,FieldsProducer> fields = new TreeMap<String,FieldsProducer>();
private final Map<PostingsFormat,FieldsProducer> formats = new IdentityHashMap<PostingsFormat,FieldsProducer>();
private final Map<String,InvertedFieldsProducer> fields = new TreeMap<String,InvertedFieldsProducer>();
private final Map<PostingsFormat,InvertedFieldsProducer> formats = new IdentityHashMap<PostingsFormat,InvertedFieldsProducer>();
public FieldsReader(final SegmentReadState readState) throws IOException {
@ -243,7 +243,7 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
@Override
public Terms terms(String field) throws IOException {
FieldsProducer fieldsProducer = fields.get(field);
InvertedFieldsProducer fieldsProducer = fields.get(field);
return fieldsProducer == null ? null : fieldsProducer.terms(field);
}
@ -259,7 +259,7 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state)
public InvertedFieldsProducer fieldsProducer(SegmentReadState state)
throws IOException {
return new FieldsReader(state);
}

View File

@ -22,8 +22,8 @@ import java.util.Set;
import org.apache.lucene.codecs.BlockTreeTermsReader;
import org.apache.lucene.codecs.BlockTreeTermsWriter;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.codecs.PostingsBaseFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
@ -66,7 +66,7 @@ public abstract class PulsingPostingsFormat extends PostingsFormat {
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase docsWriter = wrappedPostingsBaseFormat.postingsWriterBase(state);
// Terms that have <= freqCutoff number of docs are
@ -76,7 +76,7 @@ public abstract class PulsingPostingsFormat extends PostingsFormat {
// Terms dict
boolean success = false;
try {
FieldsConsumer ret = new BlockTreeTermsWriter(state, pulsingWriter, minBlockSize, maxBlockSize);
InvertedFieldsConsumer ret = new BlockTreeTermsWriter(state, pulsingWriter, minBlockSize, maxBlockSize);
success = true;
return ret;
} finally {
@ -87,14 +87,14 @@ public abstract class PulsingPostingsFormat extends PostingsFormat {
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase docsReader = wrappedPostingsBaseFormat.postingsReaderBase(state);
PostingsReaderBase pulsingReader = new PulsingPostingsReader(docsReader);
boolean success = false;
try {
FieldsProducer ret = new BlockTreeTermsReader(
InvertedFieldsProducer ret = new BlockTreeTermsReader(
state.dir, state.fieldInfos, state.segmentInfo.name,
pulsingReader,
state.context,

View File

@ -22,7 +22,7 @@ import java.util.Comparator;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo.IndexOptions;
@ -47,7 +47,7 @@ import org.apache.lucene.util.fst.PairOutputs;
import org.apache.lucene.util.fst.PositiveIntOutputs;
import org.apache.lucene.util.fst.Util;
class SimpleTextFieldsReader extends FieldsProducer {
class SimpleTextFieldsReader extends InvertedFieldsProducer {
private final IndexInput in;
private final FieldInfos fieldInfos;

View File

@ -18,7 +18,7 @@ package org.apache.lucene.codecs.simpletext;
*/
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.PostingsConsumer;
import org.apache.lucene.codecs.TermStats;
import org.apache.lucene.codecs.TermsConsumer;
@ -30,7 +30,7 @@ import org.apache.lucene.store.IndexOutput;
import java.io.IOException;
import java.util.Comparator;
class SimpleTextFieldsWriter extends FieldsConsumer {
class SimpleTextFieldsWriter extends InvertedFieldsConsumer {
private final IndexOutput out;
private final BytesRef scratch = new BytesRef(10);

View File

@ -20,8 +20,8 @@ package org.apache.lucene.codecs.simpletext;
import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentWriteState;
@ -43,12 +43,12 @@ public class SimpleTextPostingsFormat extends PostingsFormat {
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new SimpleTextFieldsWriter(state);
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
return new SimpleTextFieldsReader(state);
}

View File

@ -29,7 +29,7 @@ import java.util.TreeMap;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
@ -94,7 +94,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
}
@Override
public Fields get(int doc) throws IOException {
public InvertedFields get(int doc) throws IOException {
// TestTV tests for this in testBadParams... but is this
// really guaranteed by the API?
if (doc < 0 || doc >= offsets.size()) {
@ -221,7 +221,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
return scratchUTF16.toString();
}
private class SimpleTVFields extends Fields {
private class SimpleTVFields extends InvertedFields {
private final SortedMap<String,SimpleTVTerms> fields;
SimpleTVFields(SortedMap<String,SimpleTVTerms> fields) throws IOException {

View File

@ -19,10 +19,8 @@ package org.apache.lucene.index;
import java.io.IOException;
import org.apache.lucene.search.SearcherManager; // javadocs
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.ReaderUtil; // for javadocs
/** {@code AtomicReader} is an abstract class, providing an interface for accessing an
index. Search of an index is done entirely through this abstract interface,
@ -74,15 +72,15 @@ public abstract class AtomicReader extends IndexReader {
}
/**
* Returns {@link Fields} for this reader.
* Returns {@link InvertedFields} for this reader.
* This method may return null if the reader has no
* postings.
*/
public abstract Fields fields() throws IOException;
public abstract InvertedFields fields() throws IOException;
@Override
public final int docFreq(String field, BytesRef term) throws IOException {
final Fields fields = fields();
final InvertedFields fields = fields();
if (fields == null) {
return 0;
}
@ -104,7 +102,7 @@ public abstract class AtomicReader extends IndexReader {
* account deleted documents that have not yet been merged
* away. */
public final long totalTermFreq(String field, BytesRef term) throws IOException {
final Fields fields = fields();
final InvertedFields fields = fields();
if (fields == null) {
return 0;
}
@ -122,7 +120,7 @@ public abstract class AtomicReader extends IndexReader {
/** This may return null if the field does not exist.*/
public final Terms terms(String field) throws IOException {
final Fields fields = fields();
final InvertedFields fields = fields();
if (fields == null) {
return null;
}
@ -135,7 +133,7 @@ public abstract class AtomicReader extends IndexReader {
public final DocsEnum termDocsEnum(Bits liveDocs, String field, BytesRef term, boolean needsFreqs) throws IOException {
assert field != null;
assert term != null;
final Fields fields = fields();
final InvertedFields fields = fields();
if (fields != null) {
final Terms terms = fields.terms(field);
if (terms != null) {
@ -155,7 +153,7 @@ public abstract class AtomicReader extends IndexReader {
public final DocsAndPositionsEnum termPositionsEnum(Bits liveDocs, String field, BytesRef term, boolean needsOffsets) throws IOException {
assert field != null;
assert term != null;
final Fields fields = fields();
final InvertedFields fields = fields();
if (fields != null) {
final Terms terms = fields.terms(field);
if (terms != null) {
@ -176,7 +174,7 @@ public abstract class AtomicReader extends IndexReader {
public final DocsEnum termDocsEnum(Bits liveDocs, String field, BytesRef term, TermState state, boolean needsFreqs) throws IOException {
assert state != null;
assert field != null;
final Fields fields = fields();
final InvertedFields fields = fields();
if (fields != null) {
final Terms terms = fields.terms(field);
if (terms != null) {
@ -197,7 +195,7 @@ public abstract class AtomicReader extends IndexReader {
public final DocsAndPositionsEnum termPositionsEnum(Bits liveDocs, String field, BytesRef term, TermState state, boolean needsOffsets) throws IOException {
assert state != null;
assert field != null;
final Fields fields = fields();
final InvertedFields fields = fields();
if (fields != null) {
final Terms terms = fields.terms(field);
if (terms != null) {
@ -213,7 +211,7 @@ public abstract class AtomicReader extends IndexReader {
* in this reader.
*/
public final long getUniqueTermCount() throws IOException {
final Fields fields = fields();
final InvertedFields fields = fields();
if (fields == null) {
return 0;
}

View File

@ -83,7 +83,7 @@ public abstract class BaseCompositeReader<R extends IndexReader> extends Composi
}
@Override
public final Fields getTermVectors(int docID) throws IOException {
public final InvertedFields getTermVectors(int docID) throws IOException {
ensureOpen();
final int i = readerIndex(docID); // find subreader num
return subReaders[i].getTermVectors(docID - starts[i]); // dispatch to subreader

View File

@ -357,7 +357,7 @@ class BufferedDeletesStream {
// Delete by Term
private synchronized long applyTermDeletes(Iterable<Term> termsIter, ReadersAndLiveDocs rld, SegmentReader reader) throws IOException {
long delCount = 0;
Fields fields = reader.fields();
InvertedFields fields = reader.fields();
if (fields == null) {
// This reader has no postings
return 0;

View File

@ -701,11 +701,11 @@ public class CheckIndex {
}
/**
* checks Fields api is consistent with itself.
* checks InvertedFields api is consistent with itself.
* searcher is optional, to verify with queries. Can be null.
*/
// TODO: cutover term vectors to this!
private Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, IndexSearcher searcher) throws IOException {
private Status.TermIndexStatus checkFields(InvertedFields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, IndexSearcher searcher) throws IOException {
// TODO: we should probably return our own stats thing...?!
final Status.TermIndexStatus status = new Status.TermIndexStatus();
@ -1003,7 +1003,7 @@ public class CheckIndex {
// make sure TermsEnum is empty:
final Terms fieldTerms2 = fieldsEnum.terms();
if (fieldTerms2 != null && fieldTerms2.iterator(null).next() != null) {
throw new RuntimeException("Fields.terms(field=" + field + ") returned null yet the field appears to have terms");
throw new RuntimeException("InvertedFields.terms(field=" + field + ") returned null yet the field appears to have terms");
}
} else {
if (fieldTerms instanceof BlockTreeTermsReader.FieldReader) {
@ -1160,7 +1160,7 @@ public class CheckIndex {
infoStream.print(" test: terms, freq, prox...");
}
final Fields fields = reader.fields();
final InvertedFields fields = reader.fields();
status = checkFields(fields, liveDocs, maxDoc, fieldInfos, is);
if (liveDocs != null) {
if (infoStream != null) {
@ -1328,7 +1328,7 @@ public class CheckIndex {
}
}
msg("OK [" + status.docCount + " total doc Count; Num DocValues Fields "
msg("OK [" + status.docCount + " total doc Count; Num DocValues InvertedFields "
+ status.totalValueFields);
} catch (Throwable e) {
msg("ERROR [" + String.valueOf(e.getMessage()) + "]");
@ -1362,7 +1362,7 @@ public class CheckIndex {
final Bits liveDocs = reader.getLiveDocs();
final Fields postingsFields;
final InvertedFields postingsFields;
// TODO: testTermsIndex
if (crossCheckTermVectors) {
postingsFields = reader.fields();
@ -1377,7 +1377,7 @@ public class CheckIndex {
// Intentionally pull/visit (but don't count in
// stats) deleted documents to make sure they too
// are not corrupt:
Fields tfv = reader.getTermVectors(j);
InvertedFields tfv = reader.getTermVectors(j);
// TODO: can we make a IS(FIR) that searches just
// this term vector... to pass for searcher?

View File

@ -200,7 +200,7 @@ public class DocTermOrds {
}
if (indexedTermsArray == null) {
//System.out.println("GET normal enum");
final Fields fields = reader.fields();
final InvertedFields fields = reader.fields();
if (fields == null) {
return null;
}
@ -241,7 +241,7 @@ public class DocTermOrds {
final int[] lastTerm = new int[maxDoc]; // last term we saw for this document
final byte[][] bytes = new byte[maxDoc][]; // list of term numbers for the doc (delta encoded vInts)
final Fields fields = reader.fields();
final InvertedFields fields = reader.fields();
if (fields == null) {
// No terms
return;

View File

@ -74,7 +74,7 @@ public abstract class DocValues implements Closeable {
* <p>
* {@link Source} instances obtained from this method are closed / released
* from the cache once this {@link DocValues} instance is closed by the
* {@link IndexReader}, {@link Fields} or {@link FieldsEnum} the
* {@link IndexReader}, {@link InvertedFields} or {@link FieldsEnum} the
* {@link DocValues} was created from.
*/
public Source getSource() throws IOException {

View File

@ -36,12 +36,12 @@ import java.util.Comparator;
*/
public class FilterAtomicReader extends AtomicReader {
/** Base class for filtering {@link Fields}
/** Base class for filtering {@link InvertedFields}
* implementations. */
public static class FilterFields extends Fields {
protected final Fields in;
public static class FilterFields extends InvertedFields {
protected final InvertedFields in;
public FilterFields(Fields in) {
public FilterFields(InvertedFields in) {
this.in = in;
}
@ -329,7 +329,7 @@ public class FilterAtomicReader extends AtomicReader {
}
@Override
public Fields getTermVectors(int docID)
public InvertedFields getTermVectors(int docID)
throws IOException {
ensureOpen();
return in.getTermVectors(docID);
@ -365,7 +365,7 @@ public class FilterAtomicReader extends AtomicReader {
}
@Override
public Fields fields() throws IOException {
public InvertedFields fields() throws IOException {
ensureOpen();
return in.fields();
}

View File

@ -22,7 +22,7 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CollectionUtil;
@ -34,7 +34,7 @@ final class FreqProxTermsWriter extends TermsHashConsumer {
void abort() {}
// TODO: would be nice to factor out more of this, eg the
// FreqProxFieldMergeState, and code to visit all Fields
// FreqProxFieldMergeState, and code to visit all InvertedFields
// under the same FieldInfo together, up into TermsHash*.
// Other writers would presumably share alot of this...
@ -57,7 +57,7 @@ final class FreqProxTermsWriter extends TermsHashConsumer {
// Sort by field name
CollectionUtil.quickSort(allFields);
final FieldsConsumer consumer = state.codec.postingsFormat().fieldsConsumer(state);
final InvertedFieldsConsumer consumer = state.codec.postingsFormat().fieldsConsumer(state);
boolean success = false;
@ -66,7 +66,7 @@ final class FreqProxTermsWriter extends TermsHashConsumer {
/*
Current writer chain:
FieldsConsumer
InvertedFieldsConsumer
-> IMPL: FormatPostingsTermsDictWriter
-> TermsConsumer
-> IMPL: FormatPostingsTermsDictWriter.TermsWriter

View File

@ -23,7 +23,7 @@ import java.util.Map;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.PostingsConsumer;
import org.apache.lucene.codecs.TermStats;
import org.apache.lucene.codecs.TermsConsumer;
@ -316,7 +316,7 @@ final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implem
/* Walk through all unique text tokens (Posting
* instances) found in this field and serialize them
* into a single RAM segment. */
void flush(String fieldName, FieldsConsumer consumer, final SegmentWriteState state)
void flush(String fieldName, InvertedFieldsConsumer consumer, final SegmentWriteState state)
throws CorruptIndexException, IOException {
final TermsConsumer termsConsumer = consumer.addField(fieldInfo);

View File

@ -27,11 +27,9 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.DocumentStoredFieldVisitor;
import org.apache.lucene.search.SearcherManager; // javadocs
import org.apache.lucene.store.*;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.Bits; // javadocs
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.ReaderUtil; // for javadocs
/** IndexReader is an abstract class, providing an interface for accessing an
index. Search of an index is done entirely through this abstract interface,
@ -373,19 +371,19 @@ public abstract class IndexReader implements Closeable {
}
/** Retrieve term vectors for this document, or null if
* term vectors were not indexed. The returned Fields
* term vectors were not indexed. The returned InvertedFields
* instance acts like a single-document inverted index
* (the docID will be 0). */
public abstract Fields getTermVectors(int docID)
public abstract InvertedFields getTermVectors(int docID)
throws IOException;
/** Retrieve term vector for this document and field, or
* null if term vectors were not indexed. The returned
* Fields instance acts like a single-document inverted
* InvertedFields instance acts like a single-document inverted
* index (the docID will be 0). */
public final Terms getTermVector(int docID, String field)
throws IOException {
Fields vectors = getTermVectors(docID);
InvertedFields vectors = getTermVectors(docID);
if (vectors == null) {
return null;
}

View File

@ -22,7 +22,7 @@ import java.io.IOException;
/** Flex API for access to fields and terms
* @lucene.experimental */
public abstract class Fields {
public abstract class InvertedFields {
/** Returns an iterator that will step through all fields
* names. This will not return null. */
@ -64,5 +64,5 @@ public abstract class Fields {
return numTerms;
}
public final static Fields[] EMPTY_ARRAY = new Fields[0];
public final static InvertedFields[] EMPTY_ARRAY = new InvertedFields[0];
}

View File

@ -46,12 +46,12 @@ import org.apache.lucene.util.ReaderUtil;
* @lucene.experimental
*/
public final class MultiFields extends Fields {
private final Fields[] subs;
public final class MultiFields extends InvertedFields {
private final InvertedFields[] subs;
private final ReaderUtil.Slice[] subSlices;
private final Map<String,Terms> terms = new ConcurrentHashMap<String,Terms>();
/** Returns a single {@link Fields} instance for this
/** Returns a single {@link InvertedFields} instance for this
* reader, merging fields/terms/docs/positions on the
* fly. This method will return null if the reader
* has no postings.
@ -60,7 +60,7 @@ public final class MultiFields extends Fields {
* It's better to get the sub-readers (using {@link
* Gather}) and iterate through them
* yourself. */
public static Fields getFields(IndexReader r) throws IOException {
public static InvertedFields getFields(IndexReader r) throws IOException {
if (r instanceof AtomicReader) {
// already an atomic reader
return ((AtomicReader) r).fields();
@ -71,13 +71,13 @@ public final class MultiFields extends Fields {
// no fields
return null;
} else {
final List<Fields> fields = new ArrayList<Fields>();
final List<InvertedFields> fields = new ArrayList<InvertedFields>();
final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
new ReaderUtil.Gather(r) {
@Override
protected void add(int base, AtomicReader r) throws IOException {
final Fields f = r.fields();
final InvertedFields f = r.fields();
if (f != null) {
fields.add(f);
slices.add(new ReaderUtil.Slice(base, r.maxDoc(), fields.size()-1));
@ -90,7 +90,7 @@ public final class MultiFields extends Fields {
} else if (fields.size() == 1) {
return fields.get(0);
} else {
return new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
return new MultiFields(fields.toArray(InvertedFields.EMPTY_ARRAY),
slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY));
}
}
@ -130,7 +130,7 @@ public final class MultiFields extends Fields {
/** This method may return null if the field does not exist.*/
public static Terms getTerms(IndexReader r, String field) throws IOException {
final Fields fields = getFields(r);
final InvertedFields fields = getFields(r);
if (fields == null) {
return null;
} else {
@ -170,7 +170,7 @@ public final class MultiFields extends Fields {
return null;
}
public MultiFields(Fields[] subs, ReaderUtil.Slice[] subSlices) {
public MultiFields(InvertedFields[] subs, ReaderUtil.Slice[] subSlices) {
this.subs = subs;
this.subSlices = subSlices;
}

View File

@ -41,7 +41,7 @@ public final class MultiFieldsEnum extends FieldsEnum {
private int numTop;
private final Fields fields;
private final InvertedFields fields;
private String currentField;
@ -125,7 +125,7 @@ public final class MultiFieldsEnum extends FieldsEnum {
}
@Override
protected final boolean lessThan(FieldsEnumWithSlice fieldsA, FieldsEnumWithSlice fieldsB) {
protected boolean lessThan(FieldsEnumWithSlice fieldsA, FieldsEnumWithSlice fieldsB) {
// No need to break ties by field name: TermsEnum handles that
return fieldsA.current.compareTo(fieldsB.current) < 0;
}

View File

@ -114,9 +114,9 @@ public final class ParallelAtomicReader extends AtomicReader {
}
}
// build Fields instance
// build InvertedFields instance
for (final AtomicReader reader : this.parallelReaders) {
final Fields readerFields = reader.fields();
final InvertedFields readerFields = reader.fields();
if (readerFields != null) {
final FieldsEnum it = readerFields.iterator();
String name;
@ -176,7 +176,7 @@ public final class ParallelAtomicReader extends AtomicReader {
}
// Single instance of this, per ParallelReader instance
private final class ParallelFields extends Fields {
private final class ParallelFields extends InvertedFields {
final Map<String,Terms> fields = new TreeMap<String,Terms>();
ParallelFields() {
@ -214,7 +214,7 @@ public final class ParallelAtomicReader extends AtomicReader {
}
@Override
public Fields fields() {
public InvertedFields fields() {
ensureOpen();
return fields;
}
@ -246,7 +246,7 @@ public final class ParallelAtomicReader extends AtomicReader {
}
@Override
public Fields getTermVectors(int docID) throws IOException {
public InvertedFields getTermVectors(int docID) throws IOException {
ensureOpen();
ParallelFields fields = null;
for (Map.Entry<String,AtomicReader> ent : tvFieldToReader.entrySet()) {

View File

@ -24,7 +24,7 @@ import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.StoredFieldsReader;
@ -50,7 +50,7 @@ final class SegmentCoreReaders {
final FieldInfos fieldInfos;
final FieldsProducer fields;
final InvertedFieldsProducer fields;
final PerDocProducer perDocProducer;
final PerDocProducer norms;
@ -148,7 +148,7 @@ final class SegmentCoreReaders {
}
}
private final void notifyCoreClosedListeners() {
private void notifyCoreClosedListeners() {
synchronized(coreClosedListeners) {
for (CoreClosedListener listener : coreClosedListeners) {
listener.onClose(owner);

View File

@ -25,7 +25,7 @@ import java.util.Map;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.TermVectorsWriter;
@ -334,14 +334,14 @@ final class SegmentMerger {
private final void mergeTerms(SegmentWriteState segmentWriteState) throws CorruptIndexException, IOException {
final List<Fields> fields = new ArrayList<Fields>();
final List<InvertedFields> fields = new ArrayList<InvertedFields>();
final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
int docBase = 0;
for(int readerIndex=0;readerIndex<mergeState.readers.size();readerIndex++) {
final MergeState.IndexReaderAndLiveDocs r = mergeState.readers.get(readerIndex);
final Fields f = r.reader.fields();
final InvertedFields f = r.reader.fields();
final int maxDoc = r.reader.maxDoc();
if (f != null) {
slices.add(new ReaderUtil.Slice(docBase, maxDoc, readerIndex));
@ -350,11 +350,11 @@ final class SegmentMerger {
docBase += maxDoc;
}
final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState);
final InvertedFieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState);
boolean success = false;
try {
consumer.merge(mergeState,
new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
new MultiFields(fields.toArray(InvertedFields.EMPTY_ARRAY),
slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY)));
success = true;
} finally {

View File

@ -137,7 +137,7 @@ public final class SegmentReader extends AtomicReader {
}
@Override
public Fields fields() throws IOException {
public InvertedFields fields() throws IOException {
ensureOpen();
return core.fields;
}
@ -167,7 +167,7 @@ public final class SegmentReader extends AtomicReader {
* @throws IOException
*/
@Override
public Fields getTermVectors(int docID) throws IOException {
public InvertedFields getTermVectors(int docID) throws IOException {
TermVectorsReader termVectorsReader = getTermVectorsReader();
if (termVectorsReader == null) {
return null;

View File

@ -24,8 +24,6 @@ import java.util.Map;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ReaderUtil; // javadoc
import org.apache.lucene.index.DirectoryReader; // javadoc
import org.apache.lucene.index.MultiReader; // javadoc
/**
* This class forces a composite reader (eg a {@link
@ -47,7 +45,7 @@ public final class SlowCompositeReaderWrapper extends AtomicReader {
private final CompositeReader in;
private final Map<String, DocValues> normsCache = new HashMap<String, DocValues>();
private final Fields fields;
private final InvertedFields fields;
private final Bits liveDocs;
/** This method is sugar for getting an {@link AtomicReader} from
@ -77,7 +75,7 @@ public final class SlowCompositeReaderWrapper extends AtomicReader {
}
@Override
public Fields fields() throws IOException {
public InvertedFields fields() throws IOException {
ensureOpen();
return fields;
}
@ -100,7 +98,7 @@ public final class SlowCompositeReaderWrapper extends AtomicReader {
}
@Override
public Fields getTermVectors(int docID)
public InvertedFields getTermVectors(int docID)
throws IOException {
ensureOpen();
return in.getTermVectors(docID);

View File

@ -24,7 +24,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.automaton.CompiledAutomaton;
/**
* Access to the terms in a specific field. See {@link Fields}.
* Access to the terms in a specific field. See {@link InvertedFields}.
* @lucene.experimental
*/

View File

@ -21,7 +21,7 @@ import java.io.IOException;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
@ -85,7 +85,7 @@ public class MultiTermQueryWrapperFilter<Q extends MultiTermQuery> extends Filte
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
final AtomicReader reader = context.reader();
final Fields fields = reader.fields();
final InvertedFields fields = reader.fields();
if (fields == null) {
// reader has no fields
return DocIdSet.EMPTY_DOCIDSET;

View File

@ -21,7 +21,7 @@ import java.io.IOException;
import java.util.Comparator;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.Term;
@ -50,7 +50,7 @@ abstract class TermCollectingRewrite<Q extends Query> extends MultiTermQuery.Rew
Comparator<BytesRef> lastTermComp = null;
final AtomicReaderContext[] leaves = topReaderContext.leaves();
for (AtomicReaderContext context : leaves) {
final Fields fields = context.reader().fields();
final InvertedFields fields = context.reader().fields();
if (fields == null) {
// reader has no fields
continue;

View File

@ -18,7 +18,7 @@ package org.apache.lucene.search.spans;
*/
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.TermState;
@ -93,7 +93,7 @@ public class SpanTermQuery extends SpanQuery {
if (termContext == null) {
// this happens with span-not query, as it doesn't include the NOT side in extractTerms()
// so we seek to the term now in this segment..., this sucks because its ugly mostly!
final Fields fields = context.reader().fields();
final InvertedFields fields = context.reader().fields();
if (fields != null) {
final Terms terms = fields.terms(term.field());
if (terms != null) {

View File

@ -21,7 +21,7 @@ import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.Term;
@ -89,7 +89,7 @@ public final class TermContext {
//if (DEBUG) System.out.println("prts.build term=" + term);
for (int i = 0; i < leaves.length; i++) {
//if (DEBUG) System.out.println(" r=" + leaves[i].reader);
final Fields fields = leaves[i].reader().fields();
final InvertedFields fields = leaves[i].reader().fields();
if (fields != null) {
final Terms terms = fields.terms(field);
if (terms != null) {

View File

@ -21,12 +21,11 @@ import java.io.IOException;
import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.appending.AppendingCodec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
@ -129,7 +128,7 @@ public class TestAppendingCodec extends LuceneTestCase {
assertEquals(2, reader.numDocs());
Document doc2 = reader.document(0);
assertEquals(text, doc2.get("f"));
Fields fields = MultiFields.getFields(reader);
InvertedFields fields = MultiFields.getFields(reader);
Terms terms = fields.terms("f");
assertNotNull(terms);
TermsEnum te = terms.iterator(null);

View File

@ -21,10 +21,9 @@ import java.io.StringReader;
import org.apache.lucene.analysis.EmptyTokenizer;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
@ -342,7 +341,7 @@ public class TestDocument extends LuceneTestCase {
assertEquals(1, s.search(new TermQuery(new Term("tokenized_tokenstream", "xyz")), 1).totalHits);
for(String field : new String[] {"tv", "tv_pos", "tv_off", "tv_pos_off"}) {
Fields tvFields = r.getTermVectors(0);
InvertedFields tvFields = r.getTermVectors(0);
Terms tvs = tvFields.terms(field);
assertNotNull(tvs);
assertEquals(2, tvs.getUniqueTermCount());

View File

@ -23,8 +23,8 @@ import java.util.HashSet;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.codecs.PostingsConsumer;
import org.apache.lucene.codecs.TermStats;
import org.apache.lucene.codecs.TermsConsumer;
@ -106,7 +106,7 @@ public class TestCodecs extends LuceneTestCase {
return fieldInfo.name.compareTo(other.fieldInfo.name);
}
public void write(final FieldsConsumer consumer) throws Throwable {
public void write(final InvertedFieldsConsumer consumer) throws Throwable {
Arrays.sort(terms);
final TermsConsumer termsConsumer = consumer.addField(fieldInfo);
long sumTotalTermCount = 0;
@ -260,7 +260,7 @@ public class TestCodecs extends LuceneTestCase {
Codec codec = Codec.getDefault();
final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, codec, clonedFieldInfos);
final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random), DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR));
final InvertedFieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random), DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR));
final FieldsEnum fieldsEnum = reader.iterator();
assertNotNull(fieldsEnum.next());
@ -319,7 +319,7 @@ public class TestCodecs extends LuceneTestCase {
if (VERBOSE) {
System.out.println("TEST: now read postings");
}
final FieldsProducer terms = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random), DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR));
final InvertedFieldsProducer terms = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random), DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR));
final Verify[] threads = new Verify[NUM_TEST_THREADS-1];
for(int i=0;i<NUM_TEST_THREADS-1;i++) {
@ -398,12 +398,12 @@ public class TestCodecs extends LuceneTestCase {
}
private class Verify extends Thread {
final Fields termsDict;
final InvertedFields termsDict;
final FieldData[] fields;
final SegmentInfo si;
volatile boolean failed;
Verify(final SegmentInfo si, final FieldData[] fields, final Fields termsDict) {
Verify(final SegmentInfo si, final FieldData[] fields, final InvertedFields termsDict) {
this.fields = fields;
this.termsDict = termsDict;
this.si = si;
@ -549,18 +549,16 @@ public class TestCodecs extends LuceneTestCase {
term = field.terms[upto];
if (LuceneTestCase.random.nextInt(3) == 1) {
final DocsEnum docs;
final DocsEnum docsAndFreqs;
final DocsAndPositionsEnum postings;
if (!field.omitTF) {
postings = termsEnum.docsAndPositions(null, null, false);
if (postings != null) {
docs = docsAndFreqs = postings;
docs = postings;
} else {
docs = docsAndFreqs = _TestUtil.docs(random, termsEnum, null, null, true);
docs = _TestUtil.docs(random, termsEnum, null, null, true);
}
} else {
postings = null;
docsAndFreqs = null;
docs = _TestUtil.docs(random, termsEnum, null, null, false);
}
assertNotNull(docs);
@ -620,7 +618,7 @@ public class TestCodecs extends LuceneTestCase {
final Codec codec = Codec.getDefault();
final SegmentWriteState state = new SegmentWriteState(InfoStream.getDefault(), dir, SEGMENT, fieldInfos, 10000, termIndexInterval, codec, null, newIOContext(random));
final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(state);
final InvertedFieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(state);
Arrays.sort(fields);
for (final FieldData field : fields) {
if (!allowPreFlex && codec instanceof Lucene3xCodec) {

View File

@ -60,7 +60,7 @@ public class TestDocCount extends LuceneTestCase {
}
private void verifyCount(IndexReader ir) throws Exception {
Fields fields = MultiFields.getFields(ir);
InvertedFields fields = MultiFields.getFields(ir);
if (fields == null) {
return;
}

View File

@ -157,10 +157,10 @@ public class TestDuelingCodecs extends LuceneTestCase {
}
/**
* Fields api equivalency
* InvertedFields api equivalency
*/
public void assertFields(Fields leftFields, Fields rightFields, boolean deep) throws Exception {
// Fields could be null if there are no postings,
public void assertFields(InvertedFields leftFields, InvertedFields rightFields, boolean deep) throws Exception {
// InvertedFields could be null if there are no postings,
// but then it must be null for both
if (leftFields == null || rightFields == null) {
assertNull(info, leftFields);
@ -181,9 +181,9 @@ public class TestDuelingCodecs extends LuceneTestCase {
}
/**
* checks that top-level statistics on Fields are the same
* checks that top-level statistics on InvertedFields are the same
*/
public void assertFieldStatistics(Fields leftFields, Fields rightFields) throws Exception {
public void assertFieldStatistics(InvertedFields leftFields, InvertedFields rightFields) throws Exception {
if (leftFields.getUniqueFieldCount() != -1 && rightFields.getUniqueFieldCount() != -1) {
assertEquals(info, leftFields.getUniqueFieldCount(), rightFields.getUniqueFieldCount());
}
@ -448,9 +448,9 @@ public class TestDuelingCodecs extends LuceneTestCase {
* checks that norms are the same across all fields
*/
public void assertNorms(IndexReader leftReader, IndexReader rightReader) throws Exception {
Fields leftFields = MultiFields.getFields(leftReader);
Fields rightFields = MultiFields.getFields(rightReader);
// Fields could be null if there are no postings,
InvertedFields leftFields = MultiFields.getFields(leftReader);
InvertedFields rightFields = MultiFields.getFields(rightReader);
// InvertedFields could be null if there are no postings,
// but then it must be null for both
if (leftFields == null || rightFields == null) {
assertNull(info, leftFields);
@ -522,8 +522,8 @@ public class TestDuelingCodecs extends LuceneTestCase {
public void assertTermVectors(IndexReader leftReader, IndexReader rightReader) throws Exception {
assert leftReader.maxDoc() == rightReader.maxDoc();
for (int i = 0; i < leftReader.maxDoc(); i++) {
Fields leftFields = leftReader.getTermVectors(i);
Fields rightFields = rightReader.getTermVectors(i);
InvertedFields leftFields = leftReader.getTermVectors(i);
InvertedFields rightFields = rightReader.getTermVectors(i);
assertFields(leftFields, rightFields, rarely());
}
}

View File

@ -37,7 +37,7 @@ public class TestFilterAtomicReader extends LuceneTestCase {
/** Filter that only permits terms containing 'e'.*/
private static class TestFields extends FilterFields {
TestFields(Fields in) {
TestFields(InvertedFields in) {
super(in);
}
@Override
@ -117,7 +117,7 @@ public class TestFilterAtomicReader extends LuceneTestCase {
}
@Override
public Fields fields() throws IOException {
public InvertedFields fields() throws IOException {
return new TestFields(super.fields());
}
}

View File

@ -281,7 +281,7 @@ public class TestIndexableField extends LuceneTestCase {
// TODO: offsets
} else {
Fields vectors = r.getTermVectors(docID);
InvertedFields vectors = r.getTermVectors(docID);
assertTrue(vectors == null || vectors.terms(name) == null);
}

View File

@ -222,7 +222,7 @@ public class TestPerSegmentDeletes extends LuceneTestCase {
public static int[] toDocsArray(Term term, Bits bits, IndexReader reader)
throws IOException {
Fields fields = MultiFields.getFields(reader);
InvertedFields fields = MultiFields.getFields(reader);
Terms cterms = fields.terms(term.field);
TermsEnum ctermsEnum = cterms.iterator(null);
if (ctermsEnum.seekExact(new BytesRef(term.text()), false)) {

View File

@ -202,7 +202,7 @@ public class TestSegmentReader extends LuceneTestCase {
assertTrue(freq > 0);
}
Fields results = reader.getTermVectors(0);
InvertedFields results = reader.getTermVectors(0);
assertTrue(results != null);
assertEquals("We do not have 3 term freq vectors", 3, results.getUniqueFieldCount());
}

View File

@ -313,7 +313,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
int[] r2r1 = new int[r2.maxDoc()]; // r2 id to r1 id mapping
// create mapping from id2 space to id2 based on idField
final Fields f1 = MultiFields.getFields(r1);
final InvertedFields f1 = MultiFields.getFields(r1);
if (f1 == null) {
// make sure r2 is empty
assertNull(MultiFields.getFields(r2));
@ -330,7 +330,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
final Bits liveDocs1 = MultiFields.getLiveDocs(r1);
final Bits liveDocs2 = MultiFields.getLiveDocs(r2);
Fields fields = MultiFields.getFields(r2);
InvertedFields fields = MultiFields.getFields(r2);
if (fields == null) {
// make sure r1 is in fact empty (eg has only all
// deleted docs):
@ -393,7 +393,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
verifyEquals(r1.getTermVectors(id1), r2.getTermVectors(id2));
} catch (Throwable e) {
System.out.println("FAILED id=" + term + " id1=" + id1 + " id2=" + id2);
Fields tv1 = r1.getTermVectors(id1);
InvertedFields tv1 = r1.getTermVectors(id1);
System.out.println(" d1=" + tv1);
if (tv1 != null) {
FieldsEnum fieldsEnum = tv1.iterator();
@ -427,7 +427,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
}
}
Fields tv2 = r2.getTermVectors(id2);
InvertedFields tv2 = r2.getTermVectors(id2);
System.out.println(" d2=" + tv2);
if (tv2 != null) {
FieldsEnum fieldsEnum = tv2.iterator();
@ -598,7 +598,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
}
}
public static void verifyEquals(Fields d1, Fields d2) throws IOException {
public static void verifyEquals(InvertedFields d1, InvertedFields d2) throws IOException {
if (d1 == null) {
assertTrue(d2 == null || d2.getUniqueFieldCount() == 0);
return;

View File

@ -75,7 +75,7 @@ public class TestSumDocFreq extends LuceneTestCase {
private void assertSumDocFreq(IndexReader ir) throws Exception {
// compute sumDocFreq across all fields
Fields fields = MultiFields.getFields(ir);
InvertedFields fields = MultiFields.getFields(ir);
FieldsEnum fieldEnum = fields.iterator();
String f = null;
while ((f = fieldEnum.next()) != null) {

View File

@ -21,7 +21,7 @@ import java.io.IOException;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.*;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
@ -154,7 +154,6 @@ class MultiThreadTermVectorsReader implements Runnable {
catch (Exception e) {
e.printStackTrace();
}
return;
}
private void testTermVectors() throws Exception {
@ -163,7 +162,7 @@ class MultiThreadTermVectorsReader implements Runnable {
long start = 0L;
for (int docId = 0; docId < numDocs; docId++) {
start = System.currentTimeMillis();
Fields vectors = reader.getTermVectors(docId);
InvertedFields vectors = reader.getTermVectors(docId);
timeElapsed += System.currentTimeMillis()-start;
// verify vectors result
@ -177,7 +176,7 @@ class MultiThreadTermVectorsReader implements Runnable {
}
}
private void verifyVectors(Fields vectors, int num) throws IOException {
private void verifyVectors(InvertedFields vectors, int num) throws IOException {
FieldsEnum fieldsEnum = vectors.iterator();
while(fieldsEnum.next() != null) {
Terms terms = fieldsEnum.terms();

View File

@ -96,7 +96,7 @@ public class TestTermVectors extends LuceneTestCase {
assertEquals(100, hits.length);
for (int i = 0; i < hits.length; i++) {
Fields vectors = searcher.reader.getTermVectors(hits[i].doc);
InvertedFields vectors = searcher.reader.getTermVectors(hits[i].doc);
assertNotNull(vectors);
assertEquals("doc=" + hits[i].doc + " tv=" + vectors, 1, vectors.getUniqueFieldCount());
}
@ -120,7 +120,7 @@ public class TestTermVectors extends LuceneTestCase {
writer.addDocument(doc);
IndexReader reader = writer.getReader();
writer.close();
Fields v = reader.getTermVectors(0);
InvertedFields v = reader.getTermVectors(0);
assertEquals(4, v.getUniqueFieldCount());
String[] expectedFields = new String[]{"a", "b", "c", "x"};
int[] expectedPositions = new int[]{1, 2, 0};
@ -164,7 +164,7 @@ public class TestTermVectors extends LuceneTestCase {
DocsAndPositionsEnum dpEnum = null;
for (int i = 0; i < hits.length; i++) {
Fields vectors = searcher.reader.getTermVectors(hits[i].doc);
InvertedFields vectors = searcher.reader.getTermVectors(hits[i].doc);
assertNotNull(vectors);
assertEquals(1, vectors.getUniqueFieldCount());
@ -203,7 +203,7 @@ public class TestTermVectors extends LuceneTestCase {
assertEquals(100, hits.length);
for (int i = 0; i < hits.length; i++) {
Fields vectors = searcher.reader.getTermVectors(hits[i].doc);
InvertedFields vectors = searcher.reader.getTermVectors(hits[i].doc);
assertNotNull(vectors);
assertEquals(1, vectors.getUniqueFieldCount());
}
@ -369,7 +369,7 @@ public class TestTermVectors extends LuceneTestCase {
assertEquals(10, hits.length);
for (int i = 0; i < hits.length; i++) {
Fields vectors = searcher.reader.getTermVectors(hits[i].doc);
InvertedFields vectors = searcher.reader.getTermVectors(hits[i].doc);
assertNotNull(vectors);
assertEquals(1, vectors.getUniqueFieldCount());
}
@ -416,7 +416,7 @@ public class TestTermVectors extends LuceneTestCase {
ScoreDoc[] hits = searcher.search(query, null, 1000).scoreDocs;
assertEquals(1, hits.length);
Fields vectors = searcher.reader.getTermVectors(hits[0].doc);
InvertedFields vectors = searcher.reader.getTermVectors(hits[0].doc);
assertNotNull(vectors);
assertEquals(1, vectors.getUniqueFieldCount());
Terms vector = vectors.terms("field");

View File

@ -20,7 +20,7 @@ package org.apache.lucene.codecs.lucene3x;
import java.io.IOException;
import java.util.Comparator;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.PostingsConsumer;
import org.apache.lucene.codecs.TermStats;
import org.apache.lucene.codecs.TermsConsumer;
@ -34,7 +34,7 @@ import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
class PreFlexRWFieldsWriter extends FieldsConsumer {
class PreFlexRWFieldsWriter extends InvertedFieldsConsumer {
private final TermInfosWriter termsOut;
private final IndexOutput freqOut;

View File

@ -19,8 +19,8 @@ package org.apache.lucene.codecs.lucene3x;
import java.io.IOException;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.util.LuceneTestCase;
@ -38,12 +38,12 @@ class PreFlexRWPostingsFormat extends Lucene3xPostingsFormat {
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new PreFlexRWFieldsWriter(state);
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
// Whenever IW opens readers, eg for merging, we have to
// keep terms order in UTF16:

View File

@ -22,8 +22,8 @@ import java.util.Set;
import org.apache.lucene.codecs.BlockTermsReader;
import org.apache.lucene.codecs.BlockTermsWriter;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.codecs.FixedGapTermsIndexReader;
import org.apache.lucene.codecs.FixedGapTermsIndexWriter;
import org.apache.lucene.codecs.PostingsFormat;
@ -48,7 +48,7 @@ public class Lucene40WithOrds extends PostingsFormat {
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase docs = new Lucene40PostingsWriter(state);
// TODO: should we make the terms index more easily
@ -70,7 +70,7 @@ public class Lucene40WithOrds extends PostingsFormat {
try {
// Must use BlockTermsWriter (not BlockTree) because
// BlockTree doens't support ords (yet)...
FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, docs);
InvertedFieldsConsumer ret = new BlockTermsWriter(indexWriter, state, docs);
success = true;
return ret;
} finally {
@ -87,7 +87,7 @@ public class Lucene40WithOrds extends PostingsFormat {
public final static int TERMS_CACHE_SIZE = 1024;
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix);
TermsIndexReaderBase indexReader;
@ -108,7 +108,7 @@ public class Lucene40WithOrds extends PostingsFormat {
success = false;
try {
FieldsProducer ret = new BlockTermsReader(indexReader,
InvertedFieldsProducer ret = new BlockTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,

View File

@ -22,8 +22,8 @@ import java.util.Set;
import org.apache.lucene.codecs.BlockTermsReader;
import org.apache.lucene.codecs.BlockTermsWriter;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.codecs.FixedGapTermsIndexReader;
import org.apache.lucene.codecs.FixedGapTermsIndexWriter;
import org.apache.lucene.codecs.PostingsFormat;
@ -124,7 +124,7 @@ public class MockFixedIntBlockPostingsFormat extends PostingsFormat {
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new SepPostingsWriter(state, new MockIntFactory(blockSize));
boolean success = false;
@ -140,7 +140,7 @@ public class MockFixedIntBlockPostingsFormat extends PostingsFormat {
success = false;
try {
FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
InvertedFieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
success = true;
return ret;
} finally {
@ -155,7 +155,7 @@ public class MockFixedIntBlockPostingsFormat extends PostingsFormat {
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new SepPostingsReader(state.dir,
state.segmentInfo,
state.context,
@ -179,7 +179,7 @@ public class MockFixedIntBlockPostingsFormat extends PostingsFormat {
success = false;
try {
FieldsProducer ret = new BlockTermsReader(indexReader,
InvertedFieldsProducer ret = new BlockTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,

View File

@ -22,8 +22,8 @@ import java.util.Set;
import org.apache.lucene.codecs.BlockTermsReader;
import org.apache.lucene.codecs.BlockTermsWriter;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.codecs.FixedGapTermsIndexReader;
import org.apache.lucene.codecs.FixedGapTermsIndexWriter;
import org.apache.lucene.codecs.PostingsFormat;
@ -147,7 +147,7 @@ public class MockVariableIntBlockPostingsFormat extends PostingsFormat {
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new SepPostingsWriter(state, new MockIntFactory(baseBlockSize));
boolean success = false;
@ -163,7 +163,7 @@ public class MockVariableIntBlockPostingsFormat extends PostingsFormat {
success = false;
try {
FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
InvertedFieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
success = true;
return ret;
} finally {
@ -178,7 +178,7 @@ public class MockVariableIntBlockPostingsFormat extends PostingsFormat {
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new SepPostingsReader(state.dir,
state.segmentInfo,
state.context,
@ -202,7 +202,7 @@ public class MockVariableIntBlockPostingsFormat extends PostingsFormat {
success = false;
try {
FieldsProducer ret = new BlockTermsReader(indexReader,
InvertedFieldsProducer ret = new BlockTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,

View File

@ -28,8 +28,8 @@ import org.apache.lucene.codecs.BlockTermsReader;
import org.apache.lucene.codecs.BlockTermsWriter;
import org.apache.lucene.codecs.BlockTreeTermsReader;
import org.apache.lucene.codecs.BlockTreeTermsWriter;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.codecs.FixedGapTermsIndexReader;
import org.apache.lucene.codecs.FixedGapTermsIndexWriter;
import org.apache.lucene.codecs.PostingsFormat;
@ -126,7 +126,7 @@ public class MockRandomPostingsFormat extends PostingsFormat {
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
// we pull this before the seed intentionally: because its not consumed at runtime
// (the skipInterval is written into postings header)
int skipInterval = _TestUtil.nextInt(seedRandom, 2, 10);
@ -171,7 +171,7 @@ public class MockRandomPostingsFormat extends PostingsFormat {
postingsWriter = new PulsingPostingsWriter(totTFCutoff, postingsWriter);
}
final FieldsConsumer fields;
final InvertedFieldsConsumer fields;
if (random.nextBoolean()) {
// Use BlockTree terms dict
@ -270,7 +270,7 @@ public class MockRandomPostingsFormat extends PostingsFormat {
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
final String seedFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, SEED_EXT);
final IndexInput in = state.dir.openInput(seedFileName, state.context);
@ -310,7 +310,7 @@ public class MockRandomPostingsFormat extends PostingsFormat {
postingsReader = new PulsingPostingsReader(postingsReader);
}
final FieldsProducer fields;
final InvertedFieldsProducer fields;
if (random.nextBoolean()) {
// Use BlockTree terms dict

View File

@ -22,8 +22,8 @@ import java.util.Set;
import org.apache.lucene.codecs.BlockTermsReader;
import org.apache.lucene.codecs.BlockTermsWriter;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.codecs.FixedGapTermsIndexReader;
import org.apache.lucene.codecs.FixedGapTermsIndexWriter;
import org.apache.lucene.codecs.PostingsFormat;
@ -52,7 +52,7 @@ public class MockSepPostingsFormat extends PostingsFormat {
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new SepPostingsWriter(state, new MockSingleIntFactory());
@ -69,7 +69,7 @@ public class MockSepPostingsFormat extends PostingsFormat {
success = false;
try {
FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
InvertedFieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
success = true;
return ret;
} finally {
@ -84,7 +84,7 @@ public class MockSepPostingsFormat extends PostingsFormat {
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new SepPostingsReader(state.dir, state.segmentInfo,
state.context, new MockSingleIntFactory(), state.segmentSuffix);
@ -107,7 +107,7 @@ public class MockSepPostingsFormat extends PostingsFormat {
success = false;
try {
FieldsProducer ret = new BlockTermsReader(indexReader,
InvertedFieldsProducer ret = new BlockTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,

View File

@ -22,8 +22,8 @@ import java.util.Set;
import org.apache.lucene.codecs.BlockTreeTermsReader;
import org.apache.lucene.codecs.BlockTreeTermsWriter;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
@ -48,7 +48,7 @@ public class NestedPulsingPostingsFormat extends PostingsFormat {
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase docsWriter = new Lucene40PostingsWriter(state);
PostingsWriterBase pulsingWriterInner = new PulsingPostingsWriter(2, docsWriter);
@ -57,7 +57,7 @@ public class NestedPulsingPostingsFormat extends PostingsFormat {
// Terms dict
boolean success = false;
try {
FieldsConsumer ret = new BlockTreeTermsWriter(state, pulsingWriter,
InvertedFieldsConsumer ret = new BlockTreeTermsWriter(state, pulsingWriter,
BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
success = true;
return ret;
@ -69,13 +69,13 @@ public class NestedPulsingPostingsFormat extends PostingsFormat {
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase docsReader = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix);
PostingsReaderBase pulsingReaderInner = new PulsingPostingsReader(docsReader);
PostingsReaderBase pulsingReader = new PulsingPostingsReader(pulsingReaderInner);
boolean success = false;
try {
FieldsProducer ret = new BlockTreeTermsReader(
InvertedFieldsProducer ret = new BlockTreeTermsReader(
state.dir, state.fieldInfos, state.segmentInfo.name,
pulsingReader,
state.context,

View File

@ -29,8 +29,8 @@ import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.InvertedFieldsConsumer;
import org.apache.lucene.codecs.InvertedFieldsProducer;
import org.apache.lucene.codecs.PostingsConsumer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.TermStats;
@ -100,7 +100,7 @@ public class RAMOnlyPostingsFormat extends PostingsFormat {
}
// Postings state:
static class RAMPostings extends FieldsProducer {
static class RAMPostings extends InvertedFieldsProducer {
final Map<String,RAMField> fieldToTerms = new TreeMap<String,RAMField>();
@Override
@ -186,7 +186,7 @@ public class RAMOnlyPostingsFormat extends PostingsFormat {
}
// Classes for writing to the postings state
private static class RAMFieldsConsumer extends FieldsConsumer {
private static class RAMFieldsConsumer extends InvertedFieldsConsumer {
private final RAMPostings postings;
private final RAMTermsConsumer termsConsumer = new RAMTermsConsumer();
@ -534,7 +534,7 @@ public class RAMOnlyPostingsFormat extends PostingsFormat {
private static final String ID_EXTENSION = "id";
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState writeState) throws IOException {
public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState writeState) throws IOException {
final int id = nextID.getAndIncrement();
// TODO -- ok to do this up front instead of
@ -565,7 +565,7 @@ public class RAMOnlyPostingsFormat extends PostingsFormat {
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState readState)
public InvertedFieldsProducer fieldsProducer(SegmentReadState readState)
throws IOException {
// Load our ID:

View File

@ -20,8 +20,6 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.Set;
import org.apache.lucene.index.FilterAtomicReader;
public final class FieldFilterAtomicReader extends FilterAtomicReader {
private final Set<String> fields;
@ -50,8 +48,8 @@ public final class FieldFilterAtomicReader extends FilterAtomicReader {
}
@Override
public Fields getTermVectors(int docID) throws IOException {
Fields f = super.getTermVectors(docID);
public InvertedFields getTermVectors(int docID) throws IOException {
InvertedFields f = super.getTermVectors(docID);
if (f == null) {
return null;
}
@ -101,8 +99,8 @@ public final class FieldFilterAtomicReader extends FilterAtomicReader {
}
@Override
public Fields fields() throws IOException {
final Fields f = super.fields();
public InvertedFields fields() throws IOException {
final InvertedFields f = super.fields();
return (f == null) ? null : new FieldFilterFields(f);
}
@ -125,7 +123,7 @@ public final class FieldFilterAtomicReader extends FilterAtomicReader {
}
private class FieldFilterFields extends FilterFields {
public FieldFilterFields(Fields in) {
public FieldFilterFields(InvertedFields in) {
super(in);
}

View File

@ -340,7 +340,7 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
try {
if (s.getIndexReader().numDocs() > 0) {
smokeTestSearcher(s);
Fields fields = MultiFields.getFields(s.getIndexReader());
InvertedFields fields = MultiFields.getFields(s.getIndexReader());
if (fields == null) {
continue;
}

View File

@ -41,7 +41,7 @@ import org.apache.lucene.collation.CollationKeyAnalyzer;
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
@ -750,7 +750,7 @@ public class TestPerfTasksLogic extends BenchmarkTestCase {
writer.close();
Directory dir = benchmark.getRunData().getDirectory();
IndexReader reader = IndexReader.open(dir);
Fields tfv = reader.getTermVectors(0);
InvertedFields tfv = reader.getTermVectors(0);
assertNotNull(tfv);
assertTrue(tfv.getUniqueFieldCount() > 0);
reader.close();

View File

@ -57,7 +57,7 @@ public class TermsFilter extends Filter {
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
AtomicReader reader = context.reader();
FixedBitSet result = new FixedBitSet(reader.maxDoc());
Fields fields = reader.fields();
InvertedFields fields = reader.fields();
if (fields == null) {
return result;

View File

@ -18,13 +18,12 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.Terms;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.LongDocValues;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.util.BytesRef;
import java.io.IOException;
import java.util.Map;
@ -58,7 +57,7 @@ public class SumTotalTermFreqValueSource extends ValueSource {
public void createWeight(Map context, IndexSearcher searcher) throws IOException {
long sumTotalTermFreq = 0;
for (AtomicReaderContext readerContext : searcher.getTopReaderContext().leaves()) {
Fields fields = readerContext.reader().fields();
InvertedFields fields = readerContext.reader().fields();
if (fields == null) continue;
Terms terms = fields.terms(indexedField);
if (terms == null) continue;

View File

@ -40,7 +40,7 @@ public class TFValueSource extends TermFreqValueSource {
@Override
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
Fields fields = readerContext.reader().fields();
InvertedFields fields = readerContext.reader().fields();
final Terms terms = fields.terms(field);
IndexSearcher searcher = (IndexSearcher)context.get("searcher");
final TFIDFSimilarity similarity = IDFValueSource.asTFIDF(searcher.getSimilarity(), field);

View File

@ -38,7 +38,7 @@ public class TermFreqValueSource extends DocFreqValueSource {
@Override
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
Fields fields = readerContext.reader().fields();
InvertedFields fields = readerContext.reader().fields();
final Terms terms = fields.terms(field);
return new IntDocValues(this) {

View File

@ -22,7 +22,7 @@ import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.MultiFields;
@ -705,7 +705,7 @@ public final class MoreLikeThis {
public PriorityQueue<Object[]> retrieveTerms(int docNum) throws IOException {
Map<String, Int> termFreqMap = new HashMap<String, Int>();
for (String fieldName : fieldNames) {
final Fields vectors = ir.getTermVectors(docNum);
final InvertedFields vectors = ir.getTermVectors(docNum);
final Terms vector;
if (vectors != null) {
vector = vectors.terms(fieldName);

View File

@ -575,7 +575,7 @@ public class LukeRequestHandler extends RequestHandlerBase
final CharsRef spare = new CharsRef();
Fields fields = MultiFields.getFields(req.getSearcher().getIndexReader());
InvertedFields fields = MultiFields.getFields(req.getSearcher().getIndexReader());
if (fields == null) { // No indexed fields
return;

View File

@ -532,7 +532,7 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
//convert the ids to Lucene doc ids, the ordSet and termValues needs to be the same size as the number of elevation docs we have
ordSet.clear();
Fields fields = context.reader().fields();
InvertedFields fields = context.reader().fields();
if (fields == null) return this;
Terms terms = fields.terms(idField);
if (terms == null) return this;

View File

@ -10,7 +10,7 @@ import java.util.Map;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.StoredFieldVisitor;
@ -251,7 +251,7 @@ public class TermVectorComponent extends SearchComponent implements SolrCoreAwar
}
} else {
// extract all fields
final Fields vectors = reader.getTermVectors(docId);
final InvertedFields vectors = reader.getTermVectors(docId);
final FieldsEnum fieldsEnum = vectors.iterator();
String field;
while((field = fieldsEnum.next()) != null) {

View File

@ -118,7 +118,7 @@ public class TermsComponent extends SearchComponent {
final AtomicReader indexReader = rb.req.getSearcher().getAtomicReader();
Fields lfields = indexReader.fields();
InvertedFields lfields = indexReader.fields();
for (String field : fields) {
NamedList<Integer> fieldTerms = new NamedList<Integer>();

View File

@ -669,7 +669,7 @@ public class SimpleFacets {
startTermBytes = new BytesRef(indexedPrefix);
}
Fields fields = r.fields();
InvertedFields fields = r.fields();
Terms terms = fields==null ? null : fields.terms(field);
TermsEnum termsEnum = null;
SolrIndexSearcher.DocsEnumState deState = null;

View File

@ -262,8 +262,8 @@ class JoinQuery extends Query {
fastForRandomSet = new HashDocSet(sset.getDocs(), 0, sset.size());
}
Fields fromFields = fromSearcher.getAtomicReader().fields();
Fields toFields = fromSearcher==toSearcher ? fromFields : toSearcher.getAtomicReader().fields();
InvertedFields fromFields = fromSearcher.getAtomicReader().fields();
InvertedFields toFields = fromSearcher==toSearcher ? fromFields : toSearcher.getAtomicReader().fields();
if (fromFields == null) return DocSet.EMPTY;
Terms terms = fromFields.terms(fromField);
Terms toTerms = toFields.terms(toField);

View File

@ -34,14 +34,12 @@ import org.apache.lucene.document.LongField;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.*;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.search.*;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.OpenBitSet;
import org.apache.lucene.util.ReaderUtil;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
@ -584,7 +582,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
* @return the first document number containing the term
*/
public int getFirstMatch(Term t) throws IOException {
Fields fields = atomicReader.fields();
InvertedFields fields = atomicReader.fields();
if (fields == null) return -1;
Terms terms = fields.terms(t.field());
if (terms == null) return -1;
@ -612,7 +610,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
final AtomicReaderContext leaf = leaves[i];
final AtomicReader reader = leaf.reader();
final Fields fields = reader.fields();
final InvertedFields fields = reader.fields();
if (fields == null) continue;
final Bits liveDocs = reader.getLiveDocs();
@ -998,7 +996,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
final AtomicReaderContext leaf = leaves[i];
final AtomicReader reader = leaf.reader();
collector.setNextReader(leaf);
Fields fields = reader.fields();
InvertedFields fields = reader.fields();
Terms terms = fields.terms(t.field());
BytesRef termBytes = t.bytes();

View File

@ -23,8 +23,8 @@ import java.util.Random;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.InvertedFields;
import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.AtomicReaderContext;
@ -368,12 +368,12 @@ public class TestDocSet extends LuceneTestCase {
}
@Override
public Fields fields() {
public InvertedFields fields() {
return null;
}
@Override
public Fields getTermVectors(int doc) {
public InvertedFields getTermVectors(int doc) {
return null;
}

View File

@ -28,8 +28,6 @@ import org.apache.lucene.util.BytesRef;
import org.apache.noggit.ObjectBuilder;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.update.UpdateHandler;
import org.apache.solr.update.UpdateLog;
@ -1698,7 +1696,7 @@ public class TestRealTimeGet extends SolrTestCaseJ4 {
public int getFirstMatch(IndexReader r, Term t) throws IOException {
Fields fields = MultiFields.getFields(r);
InvertedFields fields = MultiFields.getFields(r);
if (fields == null) return -1;
Terms terms = fields.terms(t.field());
if (terms == null) return -1;