Index: lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java =================================================================== --- lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java (revision 1310827) +++ lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java (working copy) @@ -32,11 +32,8 @@ 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.IndexReader; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.*; +import org.apache.lucene.index.InvertedFields; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; @@ -70,7 +67,7 @@ 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 +99,7 @@ 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 +272,7 @@ 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"); Index: lucene/contrib/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java =================================================================== --- lucene/contrib/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java (revision 1310827) +++ lucene/contrib/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java (working copy) @@ -22,7 +22,7 @@ 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 @@ // 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; Index: lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java =================================================================== --- lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (revision 1310827) +++ lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (working copy) @@ -33,21 +33,8 @@ import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute; 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.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; -import org.apache.lucene.index.TermState; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.*; +import org.apache.lucene.index.InvertedFields; import org.apache.lucene.index.memory.MemoryIndexNormDocValues.SingleValueSource; import org.apache.lucene.search.Collector; import org.apache.lucene.search.IndexSearcher; @@ -58,7 +45,6 @@ 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 +702,7 @@ return fieldInfos; } - private class MemoryFields extends Fields { + private class MemoryFields extends InvertedFields { @Override public FieldsEnum iterator() { return new FieldsEnum() { @@ -791,7 +777,7 @@ } @Override - public Fields fields() { + public InvertedFields fields() { sortFields(); return new MemoryFields(); } @@ -1017,7 +1003,7 @@ } @Override - public Fields getTermVectors(int docID) { + public InvertedFields getTermVectors(int docID) { if (docID == 0) { return fields(); } else { Index: lucene/contrib/misc/src/java/org/apache/lucene/misc/HighFreqTerms.java =================================================================== --- lucene/contrib/misc/src/java/org/apache/lucene/misc/HighFreqTerms.java (revision 1310827) +++ lucene/contrib/misc/src/java/org/apache/lucene/misc/HighFreqTerms.java (working copy) @@ -17,14 +17,8 @@ * limitations under the License. */ -import org.apache.lucene.index.AtomicReader; -import org.apache.lucene.index.IndexReader; -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; -import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.*; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.store.FSDirectory; import org.apache.lucene.util.PriorityQueue; @@ -115,7 +109,7 @@ TermStatsQueue tiq = null; if (field != null) { - Fields fields = MultiFields.getFields(reader); + InvertedFields fields = MultiInvertedFields.getFields(reader); if (fields == null) { throw new RuntimeException("field " + field + " not found"); } @@ -126,7 +120,7 @@ fillQueue(termsEnum, tiq, field); } } else { - Fields fields = MultiFields.getFields(reader); + InvertedFields fields = MultiInvertedFields.getFields(reader); if (fields == null) { throw new RuntimeException("no fields found for this index"); } Index: lucene/contrib/misc/src/test/org/apache/lucene/index/TestMultiPassIndexSplitter.java =================================================================== --- lucene/contrib/misc/src/test/org/apache/lucene/index/TestMultiPassIndexSplitter.java (revision 1310827) +++ lucene/contrib/misc/src/test/org/apache/lucene/index/TestMultiPassIndexSplitter.java (working copy) @@ -71,7 +71,7 @@ assertTrue(ir.numDocs() - NUM_DOCS / 3 <= 1); // rounding error Document doc = ir.document(0); assertEquals("0", doc.get("id")); - TermsEnum te = MultiFields.getTerms(ir, "id").iterator(null); + TermsEnum te = MultiInvertedFields.getTerms(ir, "id").iterator(null); assertEquals(TermsEnum.SeekStatus.NOT_FOUND, te.seekCeil(new BytesRef("1"))); assertNotSame("1", te.term().utf8ToString()); ir.close(); @@ -79,7 +79,7 @@ assertTrue(ir.numDocs() - NUM_DOCS / 3 <= 1); doc = ir.document(0); assertEquals("1", doc.get("id")); - te = MultiFields.getTerms(ir, "id").iterator(null); + te = MultiInvertedFields.getTerms(ir, "id").iterator(null); assertEquals(TermsEnum.SeekStatus.NOT_FOUND, te.seekCeil(new BytesRef("0"))); assertNotSame("0", te.term().utf8ToString()); @@ -89,7 +89,7 @@ doc = ir.document(0); assertEquals("2", doc.get("id")); - te = MultiFields.getTerms(ir, "id").iterator(null); + te = MultiInvertedFields.getTerms(ir, "id").iterator(null); assertEquals(TermsEnum.SeekStatus.NOT_FOUND, te.seekCeil(new BytesRef("1"))); assertNotSame("1", te.term()); @@ -129,7 +129,7 @@ doc = ir.document(0); assertEquals(start + "", doc.get("id")); // make sure the deleted doc is not here - TermsEnum te = MultiFields.getTerms(ir, "id").iterator(null); + TermsEnum te = MultiInvertedFields.getTerms(ir, "id").iterator(null); Term t = new Term("id", (NUM_DOCS - 1) + ""); assertEquals(TermsEnum.SeekStatus.NOT_FOUND, te.seekCeil(new BytesRef(t.text()))); assertNotSame(t.text(), te.term().utf8ToString()); Index: lucene/contrib/misc/src/test/org/apache/lucene/index/TestPKIndexSplitter.java =================================================================== --- lucene/contrib/misc/src/test/org/apache/lucene/index/TestPKIndexSplitter.java (revision 1310827) +++ lucene/contrib/misc/src/test/org/apache/lucene/index/TestPKIndexSplitter.java (working copy) @@ -91,7 +91,7 @@ } private void checkContents(IndexReader ir, String indexname) throws Exception { - final Bits liveDocs = MultiFields.getLiveDocs(ir); + final Bits liveDocs = MultiInvertedFields.getLiveDocs(ir); for (int i = 0; i < ir.maxDoc(); i++) { if (liveDocs == null || liveDocs.get(i)) { assertEquals(indexname, ir.document(i).get("indexname")); Index: lucene/contrib/sandbox/src/java/org/apache/lucene/sandbox/queries/FuzzyLikeThisQuery.java =================================================================== --- lucene/contrib/sandbox/src/java/org/apache/lucene/sandbox/queries/FuzzyLikeThisQuery.java (revision 1310827) +++ lucene/contrib/sandbox/src/java/org/apache/lucene/sandbox/queries/FuzzyLikeThisQuery.java (working copy) @@ -28,7 +28,7 @@ import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.Term; import org.apache.lucene.search.*; import org.apache.lucene.search.similarities.TFIDFSimilarity; @@ -211,7 +211,7 @@ AttributeSource atts = new AttributeSource(); MaxNonCompetitiveBoostAttribute maxBoostAtt = atts.addAttribute(MaxNonCompetitiveBoostAttribute.class); - FuzzyTermsEnum fe = new FuzzyTermsEnum(MultiFields.getTerms(reader, startTerm.field()), atts, startTerm, f.minSimilarity, f.prefixLength, false); + FuzzyTermsEnum fe = new FuzzyTermsEnum(MultiInvertedFields.getTerms(reader, startTerm.field()), atts, startTerm, f.minSimilarity, f.prefixLength, false); //store the df so all variants use same idf int df = reader.docFreq(startTerm); int numVariants=0; Index: lucene/contrib/sandbox/src/test/org/apache/lucene/sandbox/queries/DuplicateFilterTest.java =================================================================== --- lucene/contrib/sandbox/src/test/org/apache/lucene/sandbox/queries/DuplicateFilterTest.java (revision 1310827) +++ lucene/contrib/sandbox/src/test/org/apache/lucene/sandbox/queries/DuplicateFilterTest.java (working copy) @@ -138,7 +138,7 @@ DocsEnum td = _TestUtil.docs(random, reader, KEY_FIELD, new BytesRef(url), - MultiFields.getLiveDocs(reader), + MultiInvertedFields.getLiveDocs(reader), null, false); @@ -162,7 +162,7 @@ DocsEnum td = _TestUtil.docs(random, reader, KEY_FIELD, new BytesRef(url), - MultiFields.getLiveDocs(reader), + MultiInvertedFields.getLiveDocs(reader), null, false); Index: lucene/contrib/sandbox/src/test/org/apache/lucene/sandbox/queries/regex/TestRegexQuery.java =================================================================== --- lucene/contrib/sandbox/src/test/org/apache/lucene/sandbox/queries/regex/TestRegexQuery.java (revision 1310827) +++ lucene/contrib/sandbox/src/test/org/apache/lucene/sandbox/queries/regex/TestRegexQuery.java (working copy) @@ -17,17 +17,13 @@ * limitations under the License. */ +import org.apache.lucene.index.*; import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper; import org.apache.lucene.store.Directory; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; -import org.apache.lucene.index.RandomIndexWriter; -import org.apache.lucene.index.Term; -import org.apache.lucene.index.Terms; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.document.Document; import org.apache.lucene.document.TextField; import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.spans.SpanNearQuery; import org.apache.lucene.search.spans.SpanQuery; @@ -81,7 +77,7 @@ } public void testMatchAll() throws Exception { - Terms terms = MultiFields.getTerms(searcher.getIndexReader(), FN); + Terms terms = MultiInvertedFields.getTerms(searcher.getIndexReader(), FN); TermsEnum te = new RegexQuery(new Term(FN, "jum.")).getTermsEnum(terms, new AttributeSource() /*dummy*/); // no term should match assertNull(te.next()); Index: lucene/core/src/java/org/apache/lucene/codecs/appending/AppendingPostingsFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/appending/AppendingPostingsFormat.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/appending/AppendingPostingsFormat.java (working copy) @@ -20,13 +20,8 @@ import java.io.IOException; 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.PostingsFormat; -import org.apache.lucene.codecs.PostingsReaderBase; -import org.apache.lucene.codecs.PostingsWriterBase; +import org.apache.lucene.codecs.*; +import org.apache.lucene.codecs.InvertedFieldsProducer; import org.apache.lucene.codecs.lucene40.Lucene40PostingsReader; import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter; import org.apache.lucene.index.SegmentInfo; @@ -44,11 +39,11 @@ } @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 +54,12 @@ } @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, Index: lucene/core/src/java/org/apache/lucene/codecs/BlockTermsReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/BlockTermsReader.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/BlockTermsReader.java (working copy) @@ -56,7 +56,7 @@ * 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; Index: lucene/core/src/java/org/apache/lucene/codecs/BlockTermsWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/BlockTermsWriter.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/BlockTermsWriter.java (working copy) @@ -47,7 +47,7 @@ * @lucene.experimental */ -public class BlockTermsWriter extends FieldsConsumer { +public class BlockTermsWriter extends InvertedFieldsConsumer { final static String CODEC_NAME = "BLOCK_TERMS_DICT"; Index: lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java (working copy) @@ -85,7 +85,7 @@ * @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; Index: lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java (working copy) @@ -83,7 +83,7 @@ * @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; Index: lucene/core/src/java/org/apache/lucene/codecs/InvertedFieldsConsumer.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/InvertedFieldsConsumer.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/codecs/InvertedFieldsConsumer.java (working copy) @@ -0,0 +1,58 @@ +package org.apache.lucene.codecs; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.Closeable; +import java.io.IOException; + +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.InvertedFields; +import org.apache.lucene.index.FieldsEnum; +import org.apache.lucene.index.MergeState; +import org.apache.lucene.index.Terms; + +/** Abstract API that consumes terms, doc, freq, prox, offset and + * payloads postings. Concrete implementations of this + * actually do "something" with the postings (write it into + * the index in a specific format). + * + * @lucene.experimental + */ +public abstract class InvertedFieldsConsumer implements Closeable { + + /** Add a new field */ + public abstract TermsConsumer addField(FieldInfo field) throws IOException; + + /** Called when we are done adding everything. */ + public abstract void close() throws IOException; + + public void merge(MergeState mergeState, InvertedFields fields) throws IOException { + FieldsEnum fieldsEnum = fields.iterator(); + assert fieldsEnum != null; + String field; + while((field = fieldsEnum.next()) != null) { + mergeState.fieldInfo = mergeState.fieldInfos.fieldInfo(field); + assert mergeState.fieldInfo != null : "FieldInfo for field is null: "+ field; + Terms terms = fieldsEnum.terms(); + if (terms != null) { + final TermsConsumer termsConsumer = addField(mergeState.fieldInfo); + termsConsumer.merge(mergeState, terms.iterator(null)); + } + } + } +} Index: lucene/core/src/java/org/apache/lucene/codecs/InvertedFieldsProducer.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/InvertedFieldsProducer.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/codecs/InvertedFieldsProducer.java (working copy) @@ -0,0 +1,33 @@ +package org.apache.lucene.codecs; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.Closeable; +import java.io.IOException; + +import org.apache.lucene.index.InvertedFields; + +/** Abstract API that produces terms, doc, freq, prox and + * payloads postings. + * + * @lucene.experimental + */ + +public abstract class InvertedFieldsProducer extends InvertedFields implements Closeable { + public abstract void close() throws IOException; +} Index: lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java (working copy) @@ -25,7 +25,7 @@ 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 @@ * @deprecated (4.0) */ @Deprecated -class Lucene3xFields extends FieldsProducer { +class Lucene3xFields extends InvertedFieldsProducer { private static final boolean DEBUG_SURROGATES = false; Index: lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xPostingsFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xPostingsFormat.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xPostingsFormat.java (working copy) @@ -20,8 +20,8 @@ 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 @@ } @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); } Index: lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java (working copy) @@ -30,7 +30,7 @@ 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 @@ return size; } - private class TVFields extends Fields { + private class TVFields extends InvertedFields { private final int[] fieldNumbers; private final long[] fieldFPs; private final Map fieldNumberToIndex = new HashMap(); @@ -654,12 +654,12 @@ } @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 Index: lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java (working copy) @@ -20,13 +20,8 @@ import java.io.IOException; 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.PostingsFormat; -import org.apache.lucene.codecs.PostingsReaderBase; -import org.apache.lucene.codecs.PostingsWriterBase; +import org.apache.lucene.codecs.*; +import org.apache.lucene.codecs.InvertedFieldsProducer; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; @@ -54,7 +49,7 @@ } @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 +58,7 @@ // 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 +71,12 @@ 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, Index: lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java (working copy) @@ -118,7 +118,7 @@ /** * Closes the underlying {@link org.apache.lucene.store.IndexInput} streams. - * This means that the Fields values will not be accessible. + * This means that the InvertedFields values will not be accessible. * * @throws IOException */ Index: lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java (working copy) @@ -25,19 +25,8 @@ import java.util.Set; import org.apache.lucene.codecs.TermVectorsReader; -import org.apache.lucene.index.CorruptIndexException; -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.FieldsEnum; -import org.apache.lucene.index.IndexFileNames; -import org.apache.lucene.index.IndexFormatTooNewException; -import org.apache.lucene.index.IndexFormatTooOldException; -import org.apache.lucene.index.SegmentInfo; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.*; +import org.apache.lucene.index.InvertedFields; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; @@ -225,7 +214,7 @@ return size; } - private class TVFields extends Fields { + private class TVFields extends InvertedFields { private final int[] fieldNumbers; private final long[] fieldFPs; private final Map fieldNumberToIndex = new HashMap(); @@ -668,12 +657,12 @@ } @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 Index: lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java (working copy) @@ -22,12 +22,9 @@ 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.IndexFileNames; +import org.apache.lucene.index.*; +import org.apache.lucene.index.InvertedFields; import org.apache.lucene.index.MergePolicy.MergeAbortedException; -import org.apache.lucene.index.MergeState; -import org.apache.lucene.index.SegmentReader; import org.apache.lucene.store.DataInput; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; @@ -309,7 +306,7 @@ // 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 +336,7 @@ 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); } Index: lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java (working copy) @@ -24,12 +24,8 @@ 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.PostingsConsumer; -import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.TermStats; -import org.apache.lucene.codecs.TermsConsumer; +import org.apache.lucene.codecs.*; +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; @@ -283,12 +279,12 @@ 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 +836,7 @@ } @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 +856,7 @@ in.close(); } - return new FieldsProducer() { + return new InvertedFieldsProducer() { @Override public FieldsEnum iterator() { final Iterator iter = fields.values().iterator(); Index: lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java (working copy) @@ -28,8 +28,8 @@ 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 @@ } @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 @@ } }; - private class FieldsWriter extends FieldsConsumer { + private class FieldsWriter extends InvertedFieldsConsumer { private final Map formats = new IdentityHashMap(); @@ -181,10 +181,10 @@ } } - private class FieldsReader extends FieldsProducer { + private class FieldsReader extends InvertedFieldsProducer { - private final Map fields = new TreeMap(); - private final Map formats = new IdentityHashMap(); + private final Map fields = new TreeMap(); + private final Map formats = new IdentityHashMap(); public FieldsReader(final SegmentReadState readState) throws IOException { @@ -243,7 +243,7 @@ @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 @@ } @Override - public FieldsProducer fieldsProducer(SegmentReadState state) + public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException { return new FieldsReader(state); } Index: lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java (working copy) @@ -49,12 +49,12 @@ } /** 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 Index: lucene/core/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java (working copy) @@ -20,14 +20,8 @@ import java.io.IOException; 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.PostingsBaseFormat; -import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.PostingsReaderBase; -import org.apache.lucene.codecs.PostingsWriterBase; +import org.apache.lucene.codecs.*; +import org.apache.lucene.codecs.InvertedFieldsConsumer; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; @@ -66,7 +60,7 @@ } @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 +70,7 @@ // 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 +81,14 @@ } @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, Index: lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java (working copy) @@ -22,7 +22,7 @@ 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.PositiveIntOutputs; import org.apache.lucene.util.fst.Util; -class SimpleTextFieldsReader extends FieldsProducer { +class SimpleTextFieldsReader extends InvertedFieldsProducer { private final IndexInput in; private final FieldInfos fieldInfos; Index: lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java (working copy) @@ -17,8 +17,8 @@ * limitations under the License. */ +import org.apache.lucene.codecs.InvertedFieldsConsumer; import org.apache.lucene.util.BytesRef; -import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.PostingsConsumer; import org.apache.lucene.codecs.TermStats; import org.apache.lucene.codecs.TermsConsumer; @@ -30,7 +30,7 @@ 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); Index: lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java (working copy) @@ -20,8 +20,8 @@ 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 @@ } @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); } Index: lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (working copy) @@ -27,14 +27,8 @@ 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.FieldsEnum; -import org.apache.lucene.index.IndexFileNames; -import org.apache.lucene.index.SegmentInfo; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.*; +import org.apache.lucene.index.InvertedFields; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; @@ -94,7 +88,7 @@ } @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 +215,7 @@ return scratchUTF16.toString(); } - private class SimpleTVFields extends Fields { + private class SimpleTVFields extends InvertedFields { private final SortedMap fields; SimpleTVFields(SortedMap fields) throws IOException { Index: lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java (working copy) @@ -22,7 +22,7 @@ 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 @@ * 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. */ Index: lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java (working copy) @@ -21,14 +21,8 @@ import java.io.IOException; 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.FieldsEnum; -import org.apache.lucene.index.MergeState; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.*; +import org.apache.lucene.index.InvertedFields; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.store.DataInput; import org.apache.lucene.util.Bits; @@ -153,7 +147,7 @@ } // 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 +160,9 @@ /** 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; Index: lucene/core/src/java/org/apache/lucene/index/AtomicReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (working copy) @@ -19,10 +19,8 @@ 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 @@ } /** - * 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 @@ * 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 @@ /** 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 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 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 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 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 @@ * in this reader. */ public final long getUniqueTermCount() throws IOException { - final Fields fields = fields(); + final InvertedFields fields = fields(); if (fields == null) { return 0; } Index: lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java (working copy) @@ -83,7 +83,7 @@ } @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 Index: lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java (working copy) @@ -357,7 +357,7 @@ // Delete by Term private synchronized long applyTermDeletes(Iterable 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; Index: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (working copy) @@ -701,11 +701,11 @@ } /** - * 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 @@ // 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 @@ 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 @@ } } - 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 @@ final Bits liveDocs = reader.getLiveDocs(); - final Fields postingsFields; + final InvertedFields postingsFields; // TODO: testTermsIndex if (crossCheckTermVectors) { postingsFields = reader.fields(); @@ -1377,7 +1377,7 @@ // 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? Index: lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java (working copy) @@ -200,7 +200,7 @@ } 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 @@ 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; Index: lucene/core/src/java/org/apache/lucene/index/DocValues.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DocValues.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/index/DocValues.java (working copy) @@ -74,7 +74,7 @@ *

* {@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 { Index: lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (working copy) @@ -36,12 +36,12 @@ */ 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 @@ } @Override - public Fields getTermVectors(int docID) + public InvertedFields getTermVectors(int docID) throws IOException { ensureOpen(); return in.getTermVectors(docID); @@ -365,7 +365,7 @@ } @Override - public Fields fields() throws IOException { + public InvertedFields fields() throws IOException { ensureOpen(); return in.fields(); } Index: lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (working copy) @@ -22,7 +22,7 @@ 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 @@ 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 @@ // 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 @@ /* Current writer chain: - FieldsConsumer + InvertedFieldsConsumer -> IMPL: FormatPostingsTermsDictWriter -> TermsConsumer -> IMPL: FormatPostingsTermsDictWriter.TermsWriter Index: lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (working copy) @@ -23,7 +23,7 @@ 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 @@ /* 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); Index: lucene/core/src/java/org/apache/lucene/index/IndexReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/IndexReader.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/index/IndexReader.java (working copy) @@ -27,11 +27,9 @@ 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.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 @@ } /** 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; } @@ -422,7 +420,7 @@ * requested document is deleted, and therefore asking for a deleted document * may yield unspecified results. Usually this is not required, however you * can test if the doc is deleted by checking the {@link - * Bits} returned from {@link MultiFields#getLiveDocs}. + * Bits} returned from {@link MultiInvertedFields#getLiveDocs}. * * NOTE: only the content of a field is returned, * if that field was stored during indexing. Metadata Index: lucene/core/src/java/org/apache/lucene/index/InvertedFields.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/InvertedFields.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/index/InvertedFields.java (working copy) @@ -0,0 +1,68 @@ +package org.apache.lucene.index; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; + +/** Flex API for access to fields and terms + * @lucene.experimental */ + +public abstract class InvertedFields { + + /** Returns an iterator that will step through all fields + * names. This will not return null. */ + public abstract FieldsEnum iterator() throws IOException; + + /** Get the {@link Terms} for this field. This will return + * null if the field does not exist. */ + public abstract Terms terms(String field) throws IOException; + + /** Returns the number of terms for all fields, or -1 if this + * measure isn't stored by the codec. Note that, just like + * other term measures, this measure does not take deleted + * documents into account. */ + public abstract int getUniqueFieldCount() throws IOException; + + /** Returns the number of terms for all fields, or -1 if this + * measure isn't stored by the codec. Note that, just like + * other term measures, this measure does not take deleted + * documents into account. */ + // TODO: deprecate? + public long getUniqueTermCount() throws IOException { + long numTerms = 0; + FieldsEnum it = iterator(); + while(true) { + String field = it.next(); + if (field == null) { + break; + } + Terms terms = terms(field); + if (terms != null) { + final long termCount = terms.getUniqueTermCount(); + if (termCount == -1) { + return -1; + } + + numTerms += termCount; + } + } + return numTerms; + } + + public final static InvertedFields[] EMPTY_ARRAY = new InvertedFields[0]; +} Index: lucene/core/src/java/org/apache/lucene/index/MultiFieldsEnum.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/MultiFieldsEnum.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/index/MultiFieldsEnum.java (working copy) @@ -41,13 +41,13 @@ private int numTop; - private final Fields fields; + private final InvertedFields fields; private String currentField; /** The subs array must be newly initialized FieldsEnum * (ie, {@link FieldsEnum#next} has not been called. */ - public MultiFieldsEnum(MultiFields fields, FieldsEnum[] subs, ReaderUtil.Slice[] subSlices) throws IOException { + public MultiFieldsEnum(MultiInvertedFields fields, FieldsEnum[] subs, ReaderUtil.Slice[] subSlices) throws IOException { this.fields = fields; queue = new FieldMergeQueue(subs.length); top = new FieldsEnumWithSlice[subs.length]; @@ -100,7 +100,7 @@ @Override public Terms terms() throws IOException { - // Ask our parent MultiFields: + // Ask our parent MultiInvertedFields: return fields.terms(currentField); } Index: lucene/core/src/java/org/apache/lucene/index/MultiInvertedFields.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/MultiInvertedFields.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/index/MultiInvertedFields.java (working copy) @@ -0,0 +1,268 @@ +package org.apache.lucene.index; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Collection; +import java.util.HashSet; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.MultiBits; +import org.apache.lucene.util.ReaderUtil.Gather; // for javadocs +import org.apache.lucene.util.ReaderUtil; + +/** + * Exposes flex API, merged from flex API of sub-segments. + * This is useful when you're interacting with an {@link + * IndexReader} implementation that consists of sequential + * sub-readers (eg {@link DirectoryReader} or {@link + * MultiReader}). + * + *

NOTE: for multi readers, you'll get better + * performance by gathering the sub readers using {@link + * ReaderUtil#gatherSubReaders} and then operate per-reader, + * instead of using this class. + * + * @lucene.experimental + */ + +public final class MultiInvertedFields extends InvertedFields { + private final InvertedFields[] subs; + private final ReaderUtil.Slice[] subSlices; + private final Map terms = new ConcurrentHashMap(); + + /** 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. + * + *

NOTE: this is a slow way to access postings. + * It's better to get the sub-readers (using {@link + * Gather}) and iterate through them + * yourself. */ + public static InvertedFields getFields(IndexReader r) throws IOException { + if (r instanceof AtomicReader) { + // already an atomic reader + return ((AtomicReader) r).fields(); + } + assert r instanceof CompositeReader; + final IndexReader[] subs = ((CompositeReader) r).getSequentialSubReaders(); + if (subs.length == 0) { + // no fields + return null; + } else { + final List fields = new ArrayList(); + final List slices = new ArrayList(); + + new ReaderUtil.Gather(r) { + @Override + protected void add(int base, AtomicReader r) throws IOException { + final InvertedFields f = r.fields(); + if (f != null) { + fields.add(f); + slices.add(new ReaderUtil.Slice(base, r.maxDoc(), fields.size()-1)); + } + } + }.run(); + + if (fields.isEmpty()) { + return null; + } else if (fields.size() == 1) { + return fields.get(0); + } else { + return new MultiInvertedFields(fields.toArray(InvertedFields.EMPTY_ARRAY), + slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY)); + } + } + } + + public static Bits getLiveDocs(IndexReader r) { + if (r.hasDeletions()) { + final List liveDocs = new ArrayList(); + final List starts = new ArrayList(); + + try { + final int maxDoc = new ReaderUtil.Gather(r) { + @Override + protected void add(int base, AtomicReader r) throws IOException { + // record all liveDocs, even if they are null + liveDocs.add(r.getLiveDocs()); + starts.add(base); + } + }.run(); + starts.add(maxDoc); + } catch (IOException ioe) { + // should not happen + throw new RuntimeException(ioe); + } + + assert liveDocs.size() > 0; + if (liveDocs.size() == 1) { + // Only one actual sub reader -- optimize this case + return liveDocs.get(0); + } else { + return new MultiBits(liveDocs, starts, true); + } + } else { + return null; + } + } + + /** This method may return null if the field does not exist.*/ + public static Terms getTerms(IndexReader r, String field) throws IOException { + final InvertedFields fields = getFields(r); + if (fields == null) { + return null; + } else { + return fields.terms(field); + } + } + + /** Returns {@link DocsEnum} for the specified field & + * term. This may return null if the term does not + * exist. */ + public static DocsEnum getTermDocsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term, boolean needsFreqs) throws IOException { + assert field != null; + assert term != null; + final Terms terms = getTerms(r, field); + if (terms != null) { + final TermsEnum termsEnum = terms.iterator(null); + if (termsEnum.seekExact(term, true)) { + return termsEnum.docs(liveDocs, null, needsFreqs); + } + } + return null; + } + + /** Returns {@link DocsAndPositionsEnum} for the specified + * field & term. This may return null if the term does + * not exist or positions were not indexed. */ + public static DocsAndPositionsEnum getTermPositionsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term, boolean needsOffsets) throws IOException { + assert field != null; + assert term != null; + final Terms terms = getTerms(r, field); + if (terms != null) { + final TermsEnum termsEnum = terms.iterator(null); + if (termsEnum.seekExact(term, true)) { + return termsEnum.docsAndPositions(liveDocs, null, needsOffsets); + } + } + return null; + } + + public MultiInvertedFields(InvertedFields[] subs, ReaderUtil.Slice[] subSlices) { + this.subs = subs; + this.subSlices = subSlices; + } + + @Override + public FieldsEnum iterator() throws IOException { + + final List fieldsEnums = new ArrayList(); + final List fieldsSlices = new ArrayList(); + for(int i=0;i subs2 = new ArrayList(); + final List slices2 = new ArrayList(); + + // Gather all sub-readers that share this field + for(int i=0;i subReaders = new ArrayList(); + ReaderUtil.gatherSubReaders(subReaders, reader); + final FieldInfos fieldInfos = new FieldInfos(); + for(AtomicReader subReader : subReaders) { + fieldInfos.add(subReader.getFieldInfos()); + } + return fieldInfos; + } + + public static Collection getIndexedFields(IndexReader reader) { + final Collection fields = new HashSet(); + for(FieldInfo fieldInfo : getMergedFieldInfos(reader)) { + if (fieldInfo.isIndexed) { + fields.add(fieldInfo.name); + } + } + return fields; + } +} + Index: lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java (working copy) @@ -49,7 +49,7 @@ */ public final class ParallelAtomicReader extends AtomicReader { private final FieldInfos fieldInfos = new FieldInfos(); - private final ParallelFields fields = new ParallelFields(); + private final ParallelInvertedFields fields = new ParallelInvertedFields(); private final AtomicReader[] parallelReaders, storedFieldsReaders; private final Set completeReaderSet = Collections.newSetFromMap(new IdentityHashMap()); @@ -114,9 +114,9 @@ } } - // 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; @@ -151,9 +151,9 @@ private final class ParallelFieldsEnum extends FieldsEnum { private String currentField; private final Iterator keys; - private final ParallelFields fields; + private final ParallelInvertedFields fields; - ParallelFieldsEnum(ParallelFields fields) { + ParallelFieldsEnum(ParallelInvertedFields fields) { this.fields = fields; keys = fields.fields.keySet().iterator(); } @@ -176,10 +176,10 @@ } // Single instance of this, per ParallelReader instance - private final class ParallelFields extends Fields { + private final class ParallelInvertedFields extends InvertedFields { final Map fields = new TreeMap(); - ParallelFields() { + ParallelInvertedFields() { } void addField(String fieldName, Terms terms) throws IOException { @@ -214,7 +214,7 @@ } @Override - public Fields fields() { + public InvertedFields fields() { ensureOpen(); return fields; } @@ -246,15 +246,15 @@ } @Override - public Fields getTermVectors(int docID) throws IOException { + public InvertedFields getTermVectors(int docID) throws IOException { ensureOpen(); - ParallelFields fields = null; + ParallelInvertedFields fields = null; for (Map.Entry ent : tvFieldToReader.entrySet()) { String fieldName = ent.getKey(); Terms vector = ent.getValue().getTermVector(docID, fieldName); if (vector != null) { if (fields == null) { - fields = new ParallelFields(); + fields = new ParallelInvertedFields(); } fields.addField(fieldName, vector); } Index: lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (working copy) @@ -23,12 +23,8 @@ 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.PerDocProducer; -import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.StoredFieldsReader; -import org.apache.lucene.codecs.TermVectorsReader; +import org.apache.lucene.codecs.*; +import org.apache.lucene.codecs.InvertedFieldsProducer; import org.apache.lucene.index.SegmentReader.CoreClosedListener; import org.apache.lucene.store.CompoundFileDirectory; import org.apache.lucene.store.Directory; @@ -50,7 +46,7 @@ final FieldInfos fieldInfos; - final FieldsProducer fields; + final InvertedFieldsProducer fields; final PerDocProducer perDocProducer; final PerDocProducer norms; Index: lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (working copy) @@ -23,12 +23,8 @@ import java.util.List; 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.PerDocConsumer; -import org.apache.lucene.codecs.StoredFieldsWriter; -import org.apache.lucene.codecs.TermVectorsWriter; +import org.apache.lucene.codecs.*; +import org.apache.lucene.codecs.InvertedFieldsConsumer; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.util.Bits; @@ -334,14 +330,14 @@ private final void mergeTerms(SegmentWriteState segmentWriteState) throws CorruptIndexException, IOException { - final List fields = new ArrayList(); + final List fields = new ArrayList(); final List slices = new ArrayList(); int docBase = 0; for(int readerIndex=0;readerIndex normsCache = new HashMap(); - private final Fields fields; + private final InvertedFields fields; private final Bits liveDocs; /** This method is sugar for getting an {@link AtomicReader} from @@ -66,8 +64,8 @@ public SlowCompositeReaderWrapper(CompositeReader reader) throws IOException { super(); in = reader; - fields = MultiFields.getFields(in); - liveDocs = MultiFields.getLiveDocs(in); + fields = MultiInvertedFields.getFields(in); + liveDocs = MultiInvertedFields.getLiveDocs(in); in.registerParentReader(this); } @@ -77,7 +75,7 @@ } @Override - public Fields fields() throws IOException { + public InvertedFields fields() throws IOException { ensureOpen(); return fields; } @@ -100,7 +98,7 @@ } @Override - public Fields getTermVectors(int docID) + public InvertedFields getTermVectors(int docID) throws IOException { ensureOpen(); return in.getTermVectors(docID); @@ -133,7 +131,7 @@ @Override public FieldInfos getFieldInfos() { ensureOpen(); - return MultiFields.getMergedFieldInfos(in); + return MultiInvertedFields.getMergedFieldInfos(in); } @Override Index: lucene/core/src/java/org/apache/lucene/index/Terms.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/Terms.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/index/Terms.java (working copy) @@ -24,7 +24,7 @@ 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 */ Index: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java =================================================================== --- lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java (working copy) @@ -35,7 +35,7 @@ import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.DirectoryReader; // javadocs import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.IndexReaderContext; import org.apache.lucene.index.StoredFieldVisitor; import org.apache.lucene.index.Term; @@ -816,7 +816,7 @@ assert field != null; - Terms terms = MultiFields.getTerms(reader, field); + Terms terms = MultiInvertedFields.getTerms(reader, field); if (terms == null) { docCount = 0; sumTotalTermFreq = 0; Index: lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (working copy) @@ -19,12 +19,8 @@ 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.AtomicReader; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.*; +import org.apache.lucene.index.InvertedFields; import org.apache.lucene.util.FixedBitSet; import org.apache.lucene.util.Bits; @@ -85,7 +81,7 @@ @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; Index: lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java =================================================================== --- lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java (working copy) @@ -17,13 +17,8 @@ * limitations under the License. */ -import org.apache.lucene.index.AtomicReaderContext; -import org.apache.lucene.index.Fields; -import org.apache.lucene.index.Term; -import org.apache.lucene.index.DocsAndPositionsEnum; -import org.apache.lucene.index.TermState; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.*; +import org.apache.lucene.index.InvertedFields; import org.apache.lucene.util.Bits; import org.apache.lucene.util.TermContext; import org.apache.lucene.util.ToStringUtils; @@ -93,7 +88,7 @@ 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) { Index: lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java =================================================================== --- lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java (working copy) @@ -20,13 +20,8 @@ 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.IndexReader; -import org.apache.lucene.index.IndexReaderContext; -import org.apache.lucene.index.Term; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.*; +import org.apache.lucene.index.InvertedFields; import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.TermContext; @@ -50,7 +45,7 @@ Comparator 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; Index: lucene/core/src/java/org/apache/lucene/util/TermContext.java =================================================================== --- lucene/core/src/java/org/apache/lucene/util/TermContext.java (revision 1310827) +++ lucene/core/src/java/org/apache/lucene/util/TermContext.java (working copy) @@ -20,14 +20,8 @@ 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.IndexReader; -import org.apache.lucene.index.IndexReaderContext; -import org.apache.lucene.index.Term; -import org.apache.lucene.index.TermState; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.*; +import org.apache.lucene.index.InvertedFields; /** * Maintains a {@link IndexReader} {@link TermState} view over @@ -89,7 +83,7 @@ //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) { Index: lucene/core/src/test/org/apache/lucene/analysis/TestCachingTokenFilter.java =================================================================== --- lucene/core/src/test/org/apache/lucene/analysis/TestCachingTokenFilter.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/analysis/TestCachingTokenFilter.java (working copy) @@ -25,7 +25,7 @@ import org.apache.lucene.document.Document; import org.apache.lucene.document.TextField; import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.DocsAndPositionsEnum; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.search.DocIdSetIterator; @@ -72,30 +72,30 @@ writer.addDocument(doc); IndexReader reader = writer.getReader(); - DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(reader, - MultiFields.getLiveDocs(reader), - "preanalyzed", - new BytesRef("term1"), - false); + DocsAndPositionsEnum termPositions = MultiInvertedFields.getTermPositionsEnum(reader, + MultiInvertedFields.getLiveDocs(reader), + "preanalyzed", + new BytesRef("term1"), + false); assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS); assertEquals(1, termPositions.freq()); assertEquals(0, termPositions.nextPosition()); - termPositions = MultiFields.getTermPositionsEnum(reader, - MultiFields.getLiveDocs(reader), - "preanalyzed", - new BytesRef("term2"), - false); + termPositions = MultiInvertedFields.getTermPositionsEnum(reader, + MultiInvertedFields.getLiveDocs(reader), + "preanalyzed", + new BytesRef("term2"), + false); assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS); assertEquals(2, termPositions.freq()); assertEquals(1, termPositions.nextPosition()); assertEquals(3, termPositions.nextPosition()); - termPositions = MultiFields.getTermPositionsEnum(reader, - MultiFields.getLiveDocs(reader), - "preanalyzed", - new BytesRef("term3"), - false); + termPositions = MultiInvertedFields.getTermPositionsEnum(reader, + MultiInvertedFields.getLiveDocs(reader), + "preanalyzed", + new BytesRef("term3"), + false); assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS); assertEquals(1, termPositions.freq()); assertEquals(2, termPositions.nextPosition()); Index: lucene/core/src/test/org/apache/lucene/codecs/appending/TestAppendingCodec.java =================================================================== --- lucene/core/src/test/org/apache/lucene/codecs/appending/TestAppendingCodec.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/codecs/appending/TestAppendingCodec.java (working copy) @@ -21,16 +21,15 @@ 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; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum.SeekStatus; import org.apache.lucene.index.TermsEnum; @@ -129,7 +128,7 @@ assertEquals(2, reader.numDocs()); Document doc2 = reader.document(0); assertEquals(text, doc2.get("f")); - Fields fields = MultiFields.getFields(reader); + InvertedFields fields = MultiInvertedFields.getFields(reader); Terms terms = fields.terms("f"); assertNotNull(terms); TermsEnum te = terms.iterator(null); Index: lucene/core/src/test/org/apache/lucene/codecs/lucene3x/TestSurrogates.java =================================================================== --- lucene/core/src/test/org/apache/lucene/codecs/lucene3x/TestSurrogates.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/codecs/lucene3x/TestSurrogates.java (working copy) @@ -18,7 +18,6 @@ */ import org.apache.lucene.store.*; -import org.apache.lucene.codecs.lucene3x.PreFlexRWCodec; import org.apache.lucene.document.*; import org.apache.lucene.analysis.*; import org.apache.lucene.index.*; @@ -101,7 +100,7 @@ if (VERBOSE) { System.out.println("\nTEST: top now enum reader=" + reader); } - FieldsEnum fieldsEnum = MultiFields.getFields(reader).iterator(); + FieldsEnum fieldsEnum = MultiInvertedFields.getFields(reader).iterator(); { // Test straight enum: @@ -164,7 +163,7 @@ // seek to it TermsEnum te = tes.get(field); if (te == null) { - te = MultiFields.getTerms(reader, field).iterator(null); + te = MultiInvertedFields.getTerms(reader, field).iterator(null); tes.put(field, te); } @@ -229,7 +228,7 @@ // term does not exist: TermsEnum te = tes.get(field); if (te == null) { - te = MultiFields.getTerms(reader, field).iterator(null); + te = MultiInvertedFields.getTerms(reader, field).iterator(null); tes.put(field, te); } Index: lucene/core/src/test/org/apache/lucene/codecs/lucene3x/TestTermInfosReaderIndex.java =================================================================== --- lucene/core/src/test/org/apache/lucene/codecs/lucene3x/TestTermInfosReaderIndex.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/codecs/lucene3x/TestTermInfosReaderIndex.java (working copy) @@ -36,7 +36,7 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.LogMergePolicy; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.SegmentReader; import org.apache.lucene.index.Term; @@ -154,7 +154,7 @@ private static List sample(IndexReader reader, int size) throws IOException { List sample = new ArrayList(); Random random = new Random(); - FieldsEnum fieldsEnum = MultiFields.getFields(reader).iterator(); + FieldsEnum fieldsEnum = MultiInvertedFields.getFields(reader).iterator(); String field; while((field = fieldsEnum.next()) != null) { Terms terms = fieldsEnum.terms(); Index: lucene/core/src/test/org/apache/lucene/codecs/pulsing/Test10KPulsings.java =================================================================== --- lucene/core/src/test/org/apache/lucene/codecs/pulsing/Test10KPulsings.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/codecs/pulsing/Test10KPulsings.java (working copy) @@ -25,17 +25,13 @@ import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.codecs.Codec; -import org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.FieldType; import org.apache.lucene.document.TextField; -import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.*; import org.apache.lucene.index.FieldInfo.IndexOptions; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; -import org.apache.lucene.index.RandomIndexWriter; -import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.store.MockDirectoryWrapper; import org.apache.lucene.util.LuceneTestCase; @@ -81,7 +77,7 @@ IndexReader ir = iw.getReader(); iw.close(); - TermsEnum te = MultiFields.getTerms(ir, "field").iterator(null); + TermsEnum te = MultiInvertedFields.getTerms(ir, "field").iterator(null); DocsEnum de = null; for (int i = 0; i < 10050; i++) { @@ -139,7 +135,7 @@ IndexReader ir = iw.getReader(); iw.close(); - TermsEnum te = MultiFields.getTerms(ir, "field").iterator(null); + TermsEnum te = MultiInvertedFields.getTerms(ir, "field").iterator(null); DocsEnum de = null; for (int i = 0; i < 10050; i++) { Index: lucene/core/src/test/org/apache/lucene/document/TestDocument.java =================================================================== --- lucene/core/src/test/org/apache/lucene/document/TestDocument.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/document/TestDocument.java (working copy) @@ -21,10 +21,9 @@ 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 @@ 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()); Index: lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java (working copy) @@ -31,7 +31,6 @@ import java.util.Collections; import java.util.List; import java.util.Random; -import org.junit.Ignore; // NOTE: this test will fail w/ PreFlexRW codec! (Because // this test uses full binary term space, but PreFlex cannot @@ -225,7 +224,7 @@ private List findTerms(IndexReader r) throws IOException { System.out.println("TEST: findTerms"); - final TermsEnum termsEnum = MultiFields.getTerms(r, "field").iterator(null); + final TermsEnum termsEnum = MultiInvertedFields.getTerms(r, "field").iterator(null); final List savedTerms = new ArrayList(); int nextSave = _TestUtil.nextInt(random, 500000, 1000000); BytesRef term; @@ -243,7 +242,7 @@ System.out.println("TEST: run " + terms.size() + " terms on reader=" + r); IndexSearcher s = new IndexSearcher(r); Collections.shuffle(terms); - TermsEnum termsEnum = MultiFields.getTerms(r, "field").iterator(null); + TermsEnum termsEnum = MultiInvertedFields.getTerms(r, "field").iterator(null); boolean failed = false; for(int iter=0;iter<10*terms.size();iter++) { final BytesRef term = terms.get(random.nextInt(terms.size())); Index: lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (working copy) @@ -295,7 +295,7 @@ _TestUtil.checkIndex(dir); - final Bits liveDocs = MultiFields.getLiveDocs(reader); + final Bits liveDocs = MultiInvertedFields.getLiveDocs(reader); for(int i=0;i<35;i++) { if (liveDocs.get(i)) { @@ -584,7 +584,7 @@ for (String name : oldNames) { Directory dir = oldIndexDirs.get(name); IndexReader r = IndexReader.open(dir); - TermsEnum terms = MultiFields.getFields(r).terms("content").iterator(null); + TermsEnum terms = MultiInvertedFields.getFields(r).terms("content").iterator(null); BytesRef t = terms.next(); assertNotNull(t); Index: lucene/core/src/test/org/apache/lucene/index/TestCodecs.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestCodecs.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/index/TestCodecs.java (working copy) @@ -22,12 +22,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.PostingsConsumer; -import org.apache.lucene.codecs.TermStats; -import org.apache.lucene.codecs.TermsConsumer; +import org.apache.lucene.codecs.*; +import org.apache.lucene.codecs.InvertedFieldsProducer; import org.apache.lucene.codecs.lucene3x.Lucene3xCodec; import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat; import org.apache.lucene.document.Document; @@ -106,7 +102,7 @@ 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 +256,7 @@ 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 +315,7 @@ 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 allFieldNames = new HashSet(); Collection indexedFieldNames = new HashSet(); @@ -354,7 +354,7 @@ DocsEnum tdocs = _TestUtil.docs(random, reader, term.field(), new BytesRef(term.text()), - MultiFields.getLiveDocs(reader), + MultiInvertedFields.getLiveDocs(reader), null, false); int count = 0; @@ -553,8 +553,8 @@ assertEquals("Single segment test differs.", index1.getSequentialSubReaders().length == 1, index2.getSequentialSubReaders().length == 1); // check field names - FieldInfos fieldInfos1 = MultiFields.getMergedFieldInfos(index1); - FieldInfos fieldInfos2 = MultiFields.getMergedFieldInfos(index2); + FieldInfos fieldInfos1 = MultiInvertedFields.getMergedFieldInfos(index1); + FieldInfos fieldInfos2 = MultiInvertedFields.getMergedFieldInfos(index2); assertEquals("IndexReaders have different numbers of fields.", fieldInfos1.size(), fieldInfos2.size()); final int numFields = fieldInfos1.size(); for(int fieldID=0;fieldID getDVFields(IndexReader reader) { Set fields = new HashSet(); - for(FieldInfo fi : MultiFields.getMergedFieldInfos(reader)) { + for(FieldInfo fi : MultiInvertedFields.getMergedFieldInfos(reader)) { if (fi.hasDocValues()) { fields.add(fi.name); } @@ -611,8 +611,8 @@ // TODO: this is kinda stupid, we don't delete documents in the test. public void assertDeletedDocs(IndexReader leftReader, IndexReader rightReader) throws Exception { assert leftReader.numDeletedDocs() == rightReader.numDeletedDocs(); - Bits leftBits = MultiFields.getLiveDocs(leftReader); - Bits rightBits = MultiFields.getLiveDocs(rightReader); + Bits leftBits = MultiInvertedFields.getLiveDocs(leftReader); + Bits rightBits = MultiInvertedFields.getLiveDocs(rightReader); if (leftBits == null || rightBits == null) { assertNull(info, leftBits); @@ -628,8 +628,8 @@ } public void assertFieldInfos(IndexReader leftReader, IndexReader rightReader) throws Exception { - FieldInfos leftInfos = MultiFields.getMergedFieldInfos(leftReader); - FieldInfos rightInfos = MultiFields.getMergedFieldInfos(rightReader); + FieldInfos leftInfos = MultiInvertedFields.getMergedFieldInfos(leftReader); + FieldInfos rightInfos = MultiInvertedFields.getMergedFieldInfos(rightReader); // TODO: would be great to verify more than just the names of the fields! TreeSet left = new TreeSet(); Index: lucene/core/src/test/org/apache/lucene/index/TestFilterAtomicReader.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestFilterAtomicReader.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/index/TestFilterAtomicReader.java (working copy) @@ -37,7 +37,7 @@ /** 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 @@ } @Override - public Fields fields() throws IOException { + public InvertedFields fields() throws IOException { return new TestFields(super.fields()); } } @@ -157,14 +157,14 @@ reader.close(); reader = IndexReader.open(target); - TermsEnum terms = MultiFields.getTerms(reader, "default").iterator(null); + TermsEnum terms = MultiInvertedFields.getTerms(reader, "default").iterator(null); while (terms.next() != null) { assertTrue(terms.term().utf8ToString().indexOf('e') != -1); } assertEquals(TermsEnum.SeekStatus.FOUND, terms.seekCeil(new BytesRef("one"))); - DocsAndPositionsEnum positions = terms.docsAndPositions(MultiFields.getLiveDocs(reader), + DocsAndPositionsEnum positions = terms.docsAndPositions(MultiInvertedFields.getLiveDocs(reader), null, false); while (positions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { assertTrue((positions.docID() % 2) == 1); Index: lucene/core/src/test/org/apache/lucene/index/TestFlex.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestFlex.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/index/TestFlex.java (working copy) @@ -53,7 +53,7 @@ IndexReader r = w.getReader(); - TermsEnum terms = MultiFields.getTerms(r, "field3").iterator(null); + TermsEnum terms = MultiInvertedFields.getTerms(r, "field3").iterator(null); assertEquals(TermsEnum.SeekStatus.END, terms.seekCeil(new BytesRef("abc"))); r.close(); } Index: lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java (working copy) @@ -281,7 +281,7 @@ // TODO: offsets } else { - Fields vectors = r.getTermVectors(docID); + InvertedFields vectors = r.getTermVectors(docID); assertTrue(vectors == null || vectors.terms(name) == null); } Index: lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (working copy) @@ -543,7 +543,7 @@ DocsEnum td = _TestUtil.docs(random, reader, "field", new BytesRef("a"), - MultiFields.getLiveDocs(reader), + MultiInvertedFields.getLiveDocs(reader), null, true); td.nextDoc(); @@ -898,11 +898,11 @@ hits = s.search(q, null, 1000).scoreDocs; assertEquals(1, hits.length); - DocsAndPositionsEnum tps = MultiFields.getTermPositionsEnum(s.getIndexReader(), - MultiFields.getLiveDocs(s.getIndexReader()), - "field", - new BytesRef("a"), - false); + DocsAndPositionsEnum tps = MultiInvertedFields.getTermPositionsEnum(s.getIndexReader(), + MultiInvertedFields.getLiveDocs(s.getIndexReader()), + "field", + new BytesRef("a"), + false); assertTrue(tps.nextDoc() != DocIdSetIterator.NO_MORE_DOCS); assertEquals(1, tps.freq()); @@ -1638,7 +1638,7 @@ // Make sure position is still incremented when // massive term is skipped: - DocsAndPositionsEnum tps = MultiFields.getTermPositionsEnum(reader, null, "content", new BytesRef("another"), false); + DocsAndPositionsEnum tps = MultiInvertedFields.getTermPositionsEnum(reader, null, "content", new BytesRef("another"), false); assertEquals(0, tps.nextDoc()); assertEquals(1, tps.freq()); assertEquals(3, tps.nextPosition()); Index: lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (working copy) @@ -504,7 +504,7 @@ DocsEnum tdocs = _TestUtil.docs(random, reader, t.field(), new BytesRef(t.text()), - MultiFields.getLiveDocs(reader), + MultiInvertedFields.getLiveDocs(reader), null, false); @@ -638,7 +638,7 @@ assertEquals(expected, reader.docFreq(new Term("contents", "here"))); assertEquals(expected, reader.maxDoc()); int numDel = 0; - final Bits liveDocs = MultiFields.getLiveDocs(reader); + final Bits liveDocs = MultiInvertedFields.getLiveDocs(reader); assertNotNull(liveDocs); for(int j=0;j allTerms, boolean isTop) throws IOException { - TermsEnum terms = MultiFields.getFields(r).terms("f").iterator(null); + TermsEnum terms = MultiInvertedFields.getFields(r).terms("f").iterator(null); BytesRef last = new BytesRef(); Index: lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java (working copy) @@ -214,7 +214,7 @@ DocsEnum tdocs = _TestUtil.docs(random, reader, "field", new BytesRef("aaa"), - MultiFields.getLiveDocs(reader), + MultiInvertedFields.getLiveDocs(reader), null, false); int count = 0; @@ -278,7 +278,7 @@ if (success) { IndexReader reader = IndexReader.open(dir); - final Bits delDocs = MultiFields.getLiveDocs(reader); + final Bits delDocs = MultiInvertedFields.getLiveDocs(reader); for(int j=0;j 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()); } Index: lucene/core/src/test/org/apache/lucene/index/TestSegmentTermDocs.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestSegmentTermDocs.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/index/TestSegmentTermDocs.java (working copy) @@ -136,7 +136,7 @@ DocsEnum tdocs = _TestUtil.docs(random, reader, ta.field(), new BytesRef(ta.text()), - MultiFields.getLiveDocs(reader), + MultiInvertedFields.getLiveDocs(reader), null, true); @@ -161,7 +161,7 @@ tdocs = _TestUtil.docs(random, reader, ta.field(), new BytesRef(ta.text()), - MultiFields.getLiveDocs(reader), + MultiInvertedFields.getLiveDocs(reader), null, false); @@ -179,7 +179,7 @@ tdocs = _TestUtil.docs(random, reader, tb.field(), new BytesRef(tb.text()), - MultiFields.getLiveDocs(reader), + MultiInvertedFields.getLiveDocs(reader), null, true); @@ -203,7 +203,7 @@ tdocs = _TestUtil.docs(random, reader, tb.field(), new BytesRef(tb.text()), - MultiFields.getLiveDocs(reader), + MultiInvertedFields.getLiveDocs(reader), null, true); @@ -223,7 +223,7 @@ tdocs = _TestUtil.docs(random, reader, tc.field(), new BytesRef(tc.text()), - MultiFields.getLiveDocs(reader), + MultiInvertedFields.getLiveDocs(reader), null, true); @@ -249,7 +249,7 @@ tdocs = _TestUtil.docs(random, reader, tc.field(), new BytesRef(tc.text()), - MultiFields.getLiveDocs(reader), + MultiInvertedFields.getLiveDocs(reader), null, false); assertTrue(tdocs.advance(5) != DocIdSetIterator.NO_MORE_DOCS); Index: lucene/core/src/test/org/apache/lucene/index/TestSegmentTermEnum.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestSegmentTermEnum.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/index/TestSegmentTermEnum.java (working copy) @@ -103,7 +103,7 @@ throws IOException { IndexReader reader = IndexReader.open(dir); - TermsEnum termEnum = MultiFields.getTerms(reader, "content").iterator(null); + TermsEnum termEnum = MultiInvertedFields.getTerms(reader, "content").iterator(null); // create enumeration of all terms // go to the first term (aaa) Index: lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java (working copy) @@ -313,24 +313,24 @@ 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 = MultiInvertedFields.getFields(r1); if (f1 == null) { // make sure r2 is empty - assertNull(MultiFields.getFields(r2)); + assertNull(MultiInvertedFields.getFields(r2)); return; } final Terms terms1 = f1.terms(idField); if (terms1 == null) { - assertTrue(MultiFields.getFields(r2) == null || - MultiFields.getFields(r2).terms(idField) == null); + assertTrue(MultiInvertedFields.getFields(r2) == null || + MultiInvertedFields.getFields(r2).terms(idField) == null); return; } final TermsEnum termsEnum = terms1.iterator(null); - final Bits liveDocs1 = MultiFields.getLiveDocs(r1); - final Bits liveDocs2 = MultiFields.getLiveDocs(r2); + final Bits liveDocs1 = MultiInvertedFields.getLiveDocs(r1); + final Bits liveDocs2 = MultiInvertedFields.getLiveDocs(r2); - Fields fields = MultiFields.getFields(r2); + InvertedFields fields = MultiInvertedFields.getFields(r2); if (fields == null) { // make sure r1 is in fact empty (eg has only all // deleted docs): @@ -393,7 +393,7 @@ 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 @@ } } - Fields tv2 = r2.getTermVectors(id2); + InvertedFields tv2 = r2.getTermVectors(id2); System.out.println(" d2=" + tv2); if (tv2 != null) { FieldsEnum fieldsEnum = tv2.iterator(); @@ -469,8 +469,8 @@ // Verify postings //System.out.println("TEST: create te1"); - final FieldsEnum fields1 = MultiFields.getFields(r1).iterator(); - final FieldsEnum fields2 = MultiFields.getFields(r2).iterator(); + final FieldsEnum fields1 = MultiInvertedFields.getFields(r1).iterator(); + final FieldsEnum fields2 = MultiInvertedFields.getFields(r2).iterator(); String field1=null, field2=null; TermsEnum termsEnum1 = null; @@ -598,7 +598,7 @@ } } - 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; Index: lucene/core/src/test/org/apache/lucene/index/TestSumDocFreq.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestSumDocFreq.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/index/TestSumDocFreq.java (working copy) @@ -75,7 +75,7 @@ private void assertSumDocFreq(IndexReader ir) throws Exception { // compute sumDocFreq across all fields - Fields fields = MultiFields.getFields(ir); + InvertedFields fields = MultiInvertedFields.getFields(ir); FieldsEnum fieldEnum = fields.iterator(); String f = null; while ((f = fieldEnum.next()) != null) { Index: lucene/core/src/test/org/apache/lucene/index/TestTermdocPerf.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestTermdocPerf.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/index/TestTermdocPerf.java (working copy) @@ -115,7 +115,7 @@ IndexReader reader = IndexReader.open(dir); - TermsEnum tenum = MultiFields.getTerms(reader, "foo").iterator(null); + TermsEnum tenum = MultiInvertedFields.getTerms(reader, "foo").iterator(null); start = System.currentTimeMillis(); @@ -123,7 +123,7 @@ DocsEnum tdocs = null; for (int i=0; i terms = new ArrayList(); - final TermsEnum termsEnum = MultiFields.getTerms(r, "body").iterator(null); + final TermsEnum termsEnum = MultiInvertedFields.getTerms(r, "body").iterator(null); BytesRef term; while((term = termsEnum.next()) != null) { terms.add(BytesRef.deepCopyOf(term)); @@ -305,7 +305,7 @@ } } - final TermsEnum te = MultiFields.getTerms(r, "f").intersect(c, startTerm); + final TermsEnum te = MultiInvertedFields.getTerms(r, "f").intersect(c, startTerm); int loc; if (startTerm == null) { @@ -485,7 +485,7 @@ assertEquals(1, docFreq(r, "xx")); assertEquals(1, docFreq(r, "aa4")); - final TermsEnum te = MultiFields.getTerms(r, FIELD).iterator(null); + final TermsEnum te = MultiInvertedFields.getTerms(r, FIELD).iterator(null); while(te.next() != null) { //System.out.println("TEST: next term=" + te.term().utf8ToString()); } @@ -515,7 +515,7 @@ w.close(); assertEquals(1, r.numDocs()); assertEquals(1, r.maxDoc()); - Terms terms = MultiFields.getTerms(r, "field"); + Terms terms = MultiInvertedFields.getTerms(r, "field"); if (terms != null) { assertNull(terms.iterator(null).next()); } @@ -617,7 +617,7 @@ System.out.println(" " + t.utf8ToString() + " " + t); } } - final TermsEnum te = MultiFields.getTerms(r, FIELD).iterator(null); + final TermsEnum te = MultiInvertedFields.getTerms(r, FIELD).iterator(null); final int END_LOC = -validTerms.length-1; Index: lucene/core/src/test/org/apache/lucene/index/TestTermsEnum2.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestTermsEnum2.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/index/TestTermsEnum2.java (working copy) @@ -116,7 +116,7 @@ for (int i = 0; i < numIterations; i++) { String reg = AutomatonTestUtil.randomRegexp(random); Automaton automaton = new RegExp(reg, RegExp.NONE).toAutomaton(); - TermsEnum te = MultiFields.getTerms(reader, "field").iterator(null); + TermsEnum te = MultiInvertedFields.getTerms(reader, "field").iterator(null); ArrayList unsortedTerms = new ArrayList(terms); Collections.shuffle(unsortedTerms, random); @@ -139,7 +139,7 @@ /** mixes up seek and next for all terms */ public void testSeekingAndNexting() throws Exception { for (int i = 0; i < numIterations; i++) { - TermsEnum te = MultiFields.getTerms(reader, "field").iterator(null); + TermsEnum te = MultiInvertedFields.getTerms(reader, "field").iterator(null); for (BytesRef term : terms) { int c = random.nextInt(3); @@ -161,7 +161,7 @@ String reg = AutomatonTestUtil.randomRegexp(random); Automaton automaton = new RegExp(reg, RegExp.NONE).toAutomaton(); CompiledAutomaton ca = new CompiledAutomaton(automaton, SpecialOperations.isFinite(automaton), false); - TermsEnum te = MultiFields.getTerms(reader, "field").intersect(ca, null); + TermsEnum te = MultiInvertedFields.getTerms(reader, "field").intersect(ca, null); Automaton expected = BasicOperations.intersection(termsAutomaton, automaton); TreeSet found = new TreeSet(); while (te.next() != null) { Index: lucene/core/src/test/org/apache/lucene/index/TestTransactionRollback.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestTransactionRollback.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/index/TestTransactionRollback.java (working copy) @@ -90,7 +90,7 @@ //Perhaps not the most efficient approach but meets our //needs here. - final Bits liveDocs = MultiFields.getLiveDocs(r); + final Bits liveDocs = MultiInvertedFields.getLiveDocs(r); for (int i = 0; i < r.maxDoc(); i++) { if (liveDocs == null || liveDocs.get(i)) { String sval=r.document(i).get(FIELD_RECORD_ID); Index: lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java =================================================================== --- lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java (working copy) @@ -23,7 +23,7 @@ import org.apache.lucene.document.Field; import org.apache.lucene.document.TextField; import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.SingleTermsEnum; import org.apache.lucene.index.Term; @@ -173,7 +173,7 @@ public void testRewriteSingleTerm() throws IOException { AutomatonQuery aq = new AutomatonQuery(newTerm("bogus"), BasicAutomata .makeString("piece")); - Terms terms = MultiFields.getTerms(searcher.getIndexReader(), FN); + Terms terms = MultiInvertedFields.getTerms(searcher.getIndexReader(), FN); assertTrue(aq.getTermsEnum(terms) instanceof SingleTermsEnum); assertEquals(1, automatonQueryNrHits(aq)); } @@ -188,7 +188,7 @@ Automaton prefixAutomaton = BasicOperations.concatenate(pfx, BasicAutomata .makeAnyString()); AutomatonQuery aq = new AutomatonQuery(newTerm("bogus"), prefixAutomaton); - Terms terms = MultiFields.getTerms(searcher.getIndexReader(), FN); + Terms terms = MultiInvertedFields.getTerms(searcher.getIndexReader(), FN); assertTrue(aq.getTermsEnum(terms) instanceof PrefixTermsEnum); assertEquals(3, automatonQueryNrHits(aq)); } @@ -201,7 +201,7 @@ .makeEmpty()); // not yet available: assertTrue(aq.getEnum(searcher.getIndexReader()) // instanceof EmptyTermEnum); - Terms terms = MultiFields.getTerms(searcher.getIndexReader(), FN); + Terms terms = MultiInvertedFields.getTerms(searcher.getIndexReader(), FN); assertSame(TermsEnum.EMPTY, aq.getTermsEnum(terms)); assertEquals(0, automatonQueryNrHits(aq)); } Index: lucene/core/src/test/org/apache/lucene/search/TestMultiPhraseQuery.java =================================================================== --- lucene/core/src/test/org/apache/lucene/search/TestMultiPhraseQuery.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/search/TestMultiPhraseQuery.java (working copy) @@ -25,14 +25,8 @@ import org.apache.lucene.document.Document; import org.apache.lucene.document.StringField; import org.apache.lucene.document.TextField; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.MultiFields; -import org.apache.lucene.index.RandomIndexWriter; -import org.apache.lucene.index.Term; -import org.apache.lucene.index.TermsEnum; -import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.index.*; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.search.similarities.DefaultSimilarity; import org.apache.lucene.store.Directory; import org.apache.lucene.store.RAMDirectory; @@ -72,7 +66,7 @@ // this TermEnum gives "piccadilly", "pie" and "pizza". String prefix = "pi"; - TermsEnum te = MultiFields.getFields(reader).terms("body").iterator(null); + TermsEnum te = MultiInvertedFields.getFields(reader).terms("body").iterator(null); te.seekCeil(new BytesRef(prefix)); do { String s = te.term().utf8ToString(); Index: lucene/core/src/test/org/apache/lucene/search/TestMultiThreadTermVectors.java =================================================================== --- lucene/core/src/test/org/apache/lucene/search/TestMultiThreadTermVectors.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/search/TestMultiThreadTermVectors.java (working copy) @@ -21,7 +21,7 @@ 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; @@ -163,7 +163,7 @@ 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 +177,7 @@ } } - 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(); Index: lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java =================================================================== --- lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java (working copy) @@ -22,13 +22,8 @@ import org.apache.lucene.document.FieldType; import org.apache.lucene.document.FloatField; import org.apache.lucene.document.IntField; -import org.apache.lucene.index.AtomicReaderContext; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; -import org.apache.lucene.index.RandomIndexWriter; -import org.apache.lucene.index.SlowCompositeReaderWrapper; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.*; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.LuceneTestCase; @@ -443,7 +438,7 @@ } private int countTerms(MultiTermQuery q) throws Exception { - final Terms terms = MultiFields.getTerms(reader, q.getField()); + final Terms terms = MultiInvertedFields.getTerms(reader, q.getField()); if (terms == null) return 0; final TermsEnum termEnum = q.getTermsEnum(terms); Index: lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java =================================================================== --- lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java (working copy) @@ -24,7 +24,7 @@ import org.apache.lucene.document.LongField; import org.apache.lucene.index.AtomicReaderContext; import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.SlowCompositeReaderWrapper; import org.apache.lucene.index.Terms; @@ -468,7 +468,7 @@ } private int countTerms(MultiTermQuery q) throws Exception { - final Terms terms = MultiFields.getTerms(reader, q.getField()); + final Terms terms = MultiInvertedFields.getTerms(reader, q.getField()); if (terms == null) return 0; final TermsEnum termEnum = q.getTermsEnum(terms); Index: lucene/core/src/test/org/apache/lucene/search/TestPhrasePrefixQuery.java =================================================================== --- lucene/core/src/test/org/apache/lucene/search/TestPhrasePrefixQuery.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/search/TestPhrasePrefixQuery.java (working copy) @@ -17,14 +17,11 @@ * limitations under the License. */ +import org.apache.lucene.index.*; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.document.Document; import org.apache.lucene.document.TextField; -import org.apache.lucene.index.RandomIndexWriter; -import org.apache.lucene.index.TermsEnum; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.Term; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.util.BytesRef; import org.apache.lucene.store.Directory; @@ -73,7 +70,7 @@ // this TermEnum gives "piccadilly", "pie" and "pizza". String prefix = "pi"; - TermsEnum te = MultiFields.getFields(reader).terms("body").iterator(null); + TermsEnum te = MultiInvertedFields.getFields(reader).terms("body").iterator(null); te.seekCeil(new BytesRef(prefix)); do { String s = te.term().utf8ToString(); Index: lucene/core/src/test/org/apache/lucene/search/TestPositionIncrement.java =================================================================== --- lucene/core/src/test/org/apache/lucene/search/TestPositionIncrement.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/search/TestPositionIncrement.java (working copy) @@ -28,13 +28,8 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; import org.apache.lucene.document.Document; import org.apache.lucene.document.TextField; -import org.apache.lucene.index.AtomicReader; -import org.apache.lucene.index.MultiFields; -import org.apache.lucene.index.DocsAndPositionsEnum; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.RandomIndexWriter; -import org.apache.lucene.index.SlowCompositeReaderWrapper; -import org.apache.lucene.index.Term; +import org.apache.lucene.index.*; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.store.Directory; import org.apache.lucene.search.payloads.PayloadSpanUtil; import org.apache.lucene.search.spans.MultiSpansWrapper; @@ -98,20 +93,20 @@ IndexSearcher searcher = newSearcher(reader); - DocsAndPositionsEnum pos = MultiFields.getTermPositionsEnum(searcher.getIndexReader(), - MultiFields.getLiveDocs(searcher.getIndexReader()), - "field", - new BytesRef("1"), - false); + DocsAndPositionsEnum pos = MultiInvertedFields.getTermPositionsEnum(searcher.getIndexReader(), + MultiInvertedFields.getLiveDocs(searcher.getIndexReader()), + "field", + new BytesRef("1"), + false); pos.nextDoc(); // first token should be at position 0 assertEquals(0, pos.nextPosition()); - pos = MultiFields.getTermPositionsEnum(searcher.getIndexReader(), - MultiFields.getLiveDocs(searcher.getIndexReader()), - "field", - new BytesRef("2"), - false); + pos = MultiInvertedFields.getTermPositionsEnum(searcher.getIndexReader(), + MultiInvertedFields.getLiveDocs(searcher.getIndexReader()), + "field", + new BytesRef("2"), + false); pos.nextDoc(); // second token should be at position 2 assertEquals(2, pos.nextPosition()); Index: lucene/core/src/test/org/apache/lucene/search/TestPrefixQuery.java =================================================================== --- lucene/core/src/test/org/apache/lucene/search/TestPrefixQuery.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/search/TestPrefixQuery.java (working copy) @@ -20,7 +20,7 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.Term; import org.apache.lucene.index.Terms; @@ -56,7 +56,7 @@ assertEquals("One in /Computers/Mac", 1, hits.length); query = new PrefixQuery(new Term("category", "")); - Terms terms = MultiFields.getTerms(searcher.getIndexReader(), "category"); + Terms terms = MultiInvertedFields.getTerms(searcher.getIndexReader(), "category"); assertFalse(query.getTermsEnum(terms) instanceof PrefixTermsEnum); hits = searcher.search(query, null, 1000).scoreDocs; assertEquals("everything", 3, hits.length); Index: lucene/core/src/test/org/apache/lucene/search/TestShardSearching.java =================================================================== --- lucene/core/src/test/org/apache/lucene/search/TestShardSearching.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/search/TestShardSearching.java (working copy) @@ -24,7 +24,7 @@ import org.apache.lucene.index.CompositeReader; import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.MultiReader; import org.apache.lucene.index.Term; import org.apache.lucene.index.TermsEnum; @@ -175,7 +175,7 @@ if (terms == null && docCount > minDocsToMakeTerms) { // TODO: try to "focus" on high freq terms sometimes too // TODO: maybe also periodically reset the terms...? - final TermsEnum termsEnum = MultiFields.getTerms(mockReader, "body").iterator(null); + final TermsEnum termsEnum = MultiInvertedFields.getTerms(mockReader, "body").iterator(null); terms = new ArrayList(); while(termsEnum.next() != null) { terms.add(BytesRef.deepCopyOf(termsEnum.term())); Index: lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java =================================================================== --- lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java (working copy) @@ -28,7 +28,7 @@ import org.apache.lucene.document.TextField; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.Terms; import org.apache.lucene.index.IndexWriterConfig.OpenMode; import org.apache.lucene.store.Directory; @@ -106,7 +106,7 @@ IndexReader reader = IndexReader.open(dir); IndexSearcher searcher = new IndexSearcher(reader); TermRangeQuery query = new TermRangeQuery("content", null, null, true, true); - Terms terms = MultiFields.getTerms(searcher.getIndexReader(), "content"); + Terms terms = MultiInvertedFields.getTerms(searcher.getIndexReader(), "content"); assertFalse(query.getTermsEnum(terms) instanceof TermRangeTermsEnum); assertEquals(4, searcher.search(query, null, 1000).scoreDocs.length); query = new TermRangeQuery("content", null, null, false, false); Index: lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java =================================================================== --- lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java (working copy) @@ -96,7 +96,7 @@ 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 @@ 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 @@ 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 @@ 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()); } @@ -251,7 +251,7 @@ writer.close(); IndexSearcher knownSearcher = newSearcher(reader); knownSearcher.setSimilarity(new DefaultSimilarity()); - FieldsEnum fields = MultiFields.getFields(knownSearcher.reader).iterator(); + FieldsEnum fields = MultiInvertedFields.getFields(knownSearcher.reader).iterator(); DocsEnum docs = null; while(fields.next() != null) { @@ -261,7 +261,7 @@ while (termsEnum.next() != null) { String text = termsEnum.term().utf8ToString(); - docs = _TestUtil.docs(random, termsEnum, MultiFields.getLiveDocs(knownSearcher.reader), docs, true); + docs = _TestUtil.docs(random, termsEnum, MultiInvertedFields.getLiveDocs(knownSearcher.reader), docs, true); while (docs.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { int docId = docs.docID(); @@ -369,7 +369,7 @@ 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 @@ 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"); Index: lucene/core/src/test/org/apache/lucene/search/TestWildcard.java =================================================================== --- lucene/core/src/test/org/apache/lucene/search/TestWildcard.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/search/TestWildcard.java (working copy) @@ -17,16 +17,13 @@ * limitations under the License. */ +import org.apache.lucene.index.*; import org.apache.lucene.store.Directory; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.document.Document; import org.apache.lucene.document.TextField; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; -import org.apache.lucene.index.RandomIndexWriter; -import org.apache.lucene.index.Term; -import org.apache.lucene.index.Terms; +import org.apache.lucene.index.MultiInvertedFields; import java.io.IOException; @@ -131,7 +128,7 @@ MultiTermQuery wq = new WildcardQuery(new Term("field", "prefix*")); assertMatches(searcher, wq, 2); - Terms terms = MultiFields.getTerms(searcher.getIndexReader(), "field"); + Terms terms = MultiInvertedFields.getTerms(searcher.getIndexReader(), "field"); assertTrue(wq.getTermsEnum(terms) instanceof PrefixTermsEnum); wq = new WildcardQuery(new Term("field", "*")); Index: lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java =================================================================== --- lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (revision 1310827) +++ lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (working copy) @@ -37,7 +37,7 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.Term; import org.apache.lucene.index.Terms; @@ -1125,7 +1125,7 @@ System.out.println("FST stores docFreq"); } } - Terms terms = MultiFields.getTerms(r, "body"); + Terms terms = MultiInvertedFields.getTerms(r, "body"); if (terms != null) { final IntsRef scratchIntsRef = new IntsRef(); final TermsEnum termsEnum = terms.iterator(null); @@ -1674,7 +1674,7 @@ } // Verify w/ MultiTermsEnum - final TermsEnum termsEnum = MultiFields.getTerms(r, "id").iterator(null); + final TermsEnum termsEnum = MultiInvertedFields.getTerms(r, "id").iterator(null); for(int iter=0;iter<2*NUM_IDS;iter++) { final String id; final String nextID; Index: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWFieldsWriter.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWFieldsWriter.java (revision 1310827) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWFieldsWriter.java (working copy) @@ -20,7 +20,7 @@ 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.util.BytesRef; import org.apache.lucene.util.IOUtils; -class PreFlexRWFieldsWriter extends FieldsConsumer { +class PreFlexRWFieldsWriter extends InvertedFieldsConsumer { private final TermInfosWriter termsOut; private final IndexOutput freqOut; Index: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWPostingsFormat.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWPostingsFormat.java (revision 1310827) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWPostingsFormat.java (working copy) @@ -19,8 +19,8 @@ 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 @@ } @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: Index: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java (revision 1310827) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java (working copy) @@ -20,17 +20,8 @@ import java.io.IOException; 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.FixedGapTermsIndexReader; -import org.apache.lucene.codecs.FixedGapTermsIndexWriter; -import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.PostingsReaderBase; -import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.TermsIndexReaderBase; -import org.apache.lucene.codecs.TermsIndexWriterBase; +import org.apache.lucene.codecs.*; +import org.apache.lucene.codecs.InvertedFieldsProducer; import org.apache.lucene.codecs.lucene40.Lucene40PostingsReader; import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter; import org.apache.lucene.index.SegmentInfo; @@ -48,7 +39,7 @@ } @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 +61,7 @@ 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 +78,7 @@ 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 +99,7 @@ success = false; try { - FieldsProducer ret = new BlockTermsReader(indexReader, + InvertedFieldsProducer ret = new BlockTermsReader(indexReader, state.dir, state.fieldInfos, state.segmentInfo.name, Index: lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java (revision 1310827) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java (working copy) @@ -20,17 +20,8 @@ import java.io.IOException; 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.FixedGapTermsIndexReader; -import org.apache.lucene.codecs.FixedGapTermsIndexWriter; -import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.PostingsReaderBase; -import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.TermsIndexReaderBase; -import org.apache.lucene.codecs.TermsIndexWriterBase; +import org.apache.lucene.codecs.*; +import org.apache.lucene.codecs.InvertedFieldsConsumer; import org.apache.lucene.codecs.intblock.FixedIntBlockIndexInput; import org.apache.lucene.codecs.intblock.FixedIntBlockIndexOutput; import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat; @@ -124,7 +115,7 @@ } @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 +131,7 @@ success = false; try { - FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter); + InvertedFieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter); success = true; return ret; } finally { @@ -155,7 +146,7 @@ } @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 +170,7 @@ success = false; try { - FieldsProducer ret = new BlockTermsReader(indexReader, + InvertedFieldsProducer ret = new BlockTermsReader(indexReader, state.dir, state.fieldInfos, state.segmentInfo.name, Index: lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java (revision 1310827) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java (working copy) @@ -20,17 +20,8 @@ import java.io.IOException; 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.FixedGapTermsIndexReader; -import org.apache.lucene.codecs.FixedGapTermsIndexWriter; -import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.PostingsReaderBase; -import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.TermsIndexReaderBase; -import org.apache.lucene.codecs.TermsIndexWriterBase; +import org.apache.lucene.codecs.*; +import org.apache.lucene.codecs.InvertedFieldsConsumer; import org.apache.lucene.codecs.intblock.VariableIntBlockIndexInput; import org.apache.lucene.codecs.intblock.VariableIntBlockIndexOutput; import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat; @@ -147,7 +138,7 @@ } @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 +154,7 @@ success = false; try { - FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter); + InvertedFieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter); success = true; return ret; } finally { @@ -178,7 +169,7 @@ } @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 +193,7 @@ success = false; try { - FieldsProducer ret = new BlockTermsReader(indexReader, + InvertedFieldsProducer ret = new BlockTermsReader(indexReader, state.dir, state.fieldInfos, state.segmentInfo.name, Index: lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java (revision 1310827) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java (working copy) @@ -24,22 +24,8 @@ import java.util.Random; import java.util.Set; -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.FixedGapTermsIndexReader; -import org.apache.lucene.codecs.FixedGapTermsIndexWriter; -import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.PostingsReaderBase; -import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.TermStats; -import org.apache.lucene.codecs.TermsIndexReaderBase; -import org.apache.lucene.codecs.TermsIndexWriterBase; -import org.apache.lucene.codecs.VariableGapTermsIndexReader; -import org.apache.lucene.codecs.VariableGapTermsIndexWriter; +import org.apache.lucene.codecs.*; +import org.apache.lucene.codecs.InvertedFieldsProducer; import org.apache.lucene.codecs.lucene40.Lucene40PostingsReader; import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter; import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat; @@ -126,7 +112,7 @@ } @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 +157,7 @@ postingsWriter = new PulsingPostingsWriter(totTFCutoff, postingsWriter); } - final FieldsConsumer fields; + final InvertedFieldsConsumer fields; if (random.nextBoolean()) { // Use BlockTree terms dict @@ -270,7 +256,7 @@ } @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 +296,7 @@ postingsReader = new PulsingPostingsReader(postingsReader); } - final FieldsProducer fields; + final InvertedFieldsProducer fields; if (random.nextBoolean()) { // Use BlockTree terms dict Index: lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java (revision 1310827) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java (working copy) @@ -20,17 +20,8 @@ import java.io.IOException; 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.FixedGapTermsIndexReader; -import org.apache.lucene.codecs.FixedGapTermsIndexWriter; -import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.PostingsReaderBase; -import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.TermsIndexReaderBase; -import org.apache.lucene.codecs.TermsIndexWriterBase; +import org.apache.lucene.codecs.*; +import org.apache.lucene.codecs.InvertedFieldsConsumer; import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat; import org.apache.lucene.codecs.sep.SepPostingsReader; import org.apache.lucene.codecs.sep.SepPostingsWriter; @@ -52,7 +43,7 @@ } @Override - public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { + public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { PostingsWriterBase postingsWriter = new SepPostingsWriter(state, new MockSingleIntFactory()); @@ -69,7 +60,7 @@ success = false; try { - FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter); + InvertedFieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter); success = true; return ret; } finally { @@ -84,7 +75,7 @@ } @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 +98,7 @@ success = false; try { - FieldsProducer ret = new BlockTermsReader(indexReader, + InvertedFieldsProducer ret = new BlockTermsReader(indexReader, state.dir, state.fieldInfos, state.segmentInfo.name, Index: lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java (revision 1310827) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java (working copy) @@ -20,13 +20,8 @@ import java.io.IOException; 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.PostingsFormat; -import org.apache.lucene.codecs.PostingsReaderBase; -import org.apache.lucene.codecs.PostingsWriterBase; +import org.apache.lucene.codecs.*; +import org.apache.lucene.codecs.InvertedFieldsConsumer; import org.apache.lucene.codecs.lucene40.Lucene40PostingsReader; import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter; import org.apache.lucene.codecs.pulsing.PulsingPostingsReader; @@ -48,7 +43,7 @@ } @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 +52,7 @@ // 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 +64,13 @@ } @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, Index: lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java (revision 1310827) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java (working copy) @@ -29,12 +29,8 @@ 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.PostingsConsumer; -import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.TermStats; -import org.apache.lucene.codecs.TermsConsumer; +import org.apache.lucene.codecs.*; +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; @@ -100,7 +96,7 @@ } // Postings state: - static class RAMPostings extends FieldsProducer { + static class RAMPostings extends InvertedFieldsProducer { final Map fieldToTerms = new TreeMap(); @Override @@ -186,7 +182,7 @@ } // 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 +530,7 @@ 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 +561,7 @@ } @Override - public FieldsProducer fieldsProducer(SegmentReadState readState) + public InvertedFieldsProducer fieldsProducer(SegmentReadState readState) throws IOException { // Load our ID: Index: lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java (revision 1310827) +++ lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java (working copy) @@ -20,8 +20,6 @@ import java.io.IOException; import java.util.Set; -import org.apache.lucene.index.FilterAtomicReader; - public final class FieldFilterAtomicReader extends FilterAtomicReader { private final Set fields; @@ -50,8 +48,8 @@ } @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 @@ } @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 @@ } private class FieldFilterFields extends FilterFields { - public FieldFilterFields(Fields in) { + public FieldFilterFields(InvertedFields in) { super(in); } Index: lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java (revision 1310827) +++ lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java (working copy) @@ -340,7 +340,7 @@ try { if (s.getIndexReader().numDocs() > 0) { smokeTestSearcher(s); - Fields fields = MultiFields.getFields(s.getIndexReader()); + InvertedFields fields = MultiInvertedFields.getFields(s.getIndexReader()); if (fields == null) { continue; } Index: lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java (revision 1310827) +++ lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java (working copy) @@ -50,7 +50,7 @@ import org.apache.lucene.index.LogMergePolicy; import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.MergeScheduler; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.TieredMergePolicy; @@ -770,7 +770,7 @@ // DocsAndFreqsEnum, DocsAndPositionsEnum. Returns null // if field/term doesn't exist: public static DocsEnum docs(Random random, IndexReader r, String field, BytesRef term, Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException { - final Terms terms = MultiFields.getTerms(r, field); + final Terms terms = MultiInvertedFields.getTerms(r, field); if (terms == null) { return null; } Index: modules/analysis/common/src/java/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java =================================================================== --- modules/analysis/common/src/java/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java (revision 1310827) +++ modules/analysis/common/src/java/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java (working copy) @@ -23,11 +23,8 @@ import org.apache.lucene.analysis.AnalyzerWrapper; import org.apache.lucene.analysis.core.StopFilter; import org.apache.lucene.analysis.util.CharArraySet; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; -import org.apache.lucene.index.Term; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.*; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.CharsRef; import org.apache.lucene.util.UnicodeUtil; @@ -85,7 +82,7 @@ Analyzer delegate, IndexReader indexReader, int maxDocFreq) throws IOException { - this(matchVersion, delegate, indexReader, MultiFields.getIndexedFields(indexReader), maxDocFreq); + this(matchVersion, delegate, indexReader, MultiInvertedFields.getIndexedFields(indexReader), maxDocFreq); } /** @@ -105,7 +102,7 @@ Analyzer delegate, IndexReader indexReader, float maxPercentDocs) throws IOException { - this(matchVersion, delegate, indexReader, MultiFields.getIndexedFields(indexReader), maxPercentDocs); + this(matchVersion, delegate, indexReader, MultiInvertedFields.getIndexedFields(indexReader), maxPercentDocs); } /** @@ -153,7 +150,7 @@ for (String field : fields) { Set stopWords = new HashSet(); - Terms terms = MultiFields.getTerms(indexReader, field); + Terms terms = MultiInvertedFields.getTerms(indexReader, field); CharsRef spare = new CharsRef(); if (terms != null) { TermsEnum te = terms.iterator(null); Index: modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestClassicAnalyzer.java =================================================================== --- modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestClassicAnalyzer.java (revision 1310827) +++ modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestClassicAnalyzer.java (working copy) @@ -5,13 +5,8 @@ import org.apache.lucene.analysis.standard.ClassicAnalyzer; import org.apache.lucene.document.Document; import org.apache.lucene.document.TextField; -import org.apache.lucene.index.DocsAndPositionsEnum; -import org.apache.lucene.index.DocsEnum; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.MultiFields; -import org.apache.lucene.index.Term; +import org.apache.lucene.index.*; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.store.RAMDirectory; import org.apache.lucene.util.BytesRef; @@ -281,11 +276,11 @@ // Make sure position is still incremented when // massive term is skipped: - DocsAndPositionsEnum tps = MultiFields.getTermPositionsEnum(reader, - MultiFields.getLiveDocs(reader), - "content", - new BytesRef("another"), - false); + DocsAndPositionsEnum tps = MultiInvertedFields.getTermPositionsEnum(reader, + MultiInvertedFields.getLiveDocs(reader), + "content", + new BytesRef("another"), + false); assertTrue(tps.nextDoc() != DocIdSetIterator.NO_MORE_DOCS); assertEquals(1, tps.freq()); assertEquals(3, tps.nextPosition()); Index: modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestKeywordAnalyzer.java =================================================================== --- modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestKeywordAnalyzer.java (revision 1310827) +++ modules/analysis/common/src/test/org/apache/lucene/analysis/core/TestKeywordAnalyzer.java (working copy) @@ -30,7 +30,7 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.store.Directory; @@ -100,7 +100,7 @@ reader, "partnum", new BytesRef("Q36"), - MultiFields.getLiveDocs(reader), + MultiInvertedFields.getLiveDocs(reader), null, false); assertTrue(td.nextDoc() != DocIdSetIterator.NO_MORE_DOCS); @@ -108,7 +108,7 @@ reader, "partnum", new BytesRef("Q37"), - MultiFields.getLiveDocs(reader), + MultiInvertedFields.getLiveDocs(reader), null, false); assertTrue(td.nextDoc() != DocIdSetIterator.NO_MORE_DOCS); Index: modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java =================================================================== --- modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java (revision 1310827) +++ modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java (working copy) @@ -30,10 +30,9 @@ import org.apache.lucene.document.Document; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexableField; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.search.Collector; import org.apache.lucene.search.TopDocs; -import org.apache.lucene.search.MultiTermQuery; import org.apache.lucene.search.TopFieldCollector; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.TopScoreDocCollector; @@ -96,7 +95,7 @@ // optionally warm and add num docs traversed to count if (withWarm()) { Document doc = null; - Bits liveDocs = MultiFields.getLiveDocs(reader); + Bits liveDocs = MultiInvertedFields.getLiveDocs(reader); for (int m = 0; m < reader.maxDoc(); m++) { if (null == liveDocs || liveDocs.get(m)) { doc = reader.document(m); Index: modules/benchmark/src/java/org/apache/lucene/benchmark/quality/utils/QualityQueriesFinder.java =================================================================== --- modules/benchmark/src/java/org/apache/lucene/benchmark/quality/utils/QualityQueriesFinder.java (revision 1310827) +++ modules/benchmark/src/java/org/apache/lucene/benchmark/quality/utils/QualityQueriesFinder.java (working copy) @@ -20,9 +20,9 @@ import java.io.IOException; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.Terms; -import org.apache.lucene.index.MultiFields; import org.apache.lucene.store.Directory; import org.apache.lucene.store.FSDirectory; import org.apache.lucene.util.PriorityQueue; @@ -89,7 +89,7 @@ IndexReader ir = IndexReader.open(dir); try { int threshold = ir.maxDoc() / 10; // ignore words too common. - Terms terms = MultiFields.getTerms(ir, field); + Terms terms = MultiInvertedFields.getTerms(ir, field); if (terms != null) { TermsEnum termsEnum = terms.iterator(null); while (termsEnum.next() != null) { Index: modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java =================================================================== --- modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java (revision 1310827) +++ modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java (working copy) @@ -39,22 +39,9 @@ import org.apache.lucene.benchmark.byTask.tasks.WriteLineDocTask; 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.FieldsEnum; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.*; +import org.apache.lucene.index.InvertedFields; import org.apache.lucene.index.IndexWriterConfig.OpenMode; -import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.LogDocMergePolicy; -import org.apache.lucene.index.LogMergePolicy; -import org.apache.lucene.index.MultiFields; -import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.index.SerialMergeScheduler; -import org.apache.lucene.index.SlowCompositeReaderWrapper; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.FieldCache.DocTermsIndex; import org.apache.lucene.search.FieldCache; @@ -484,7 +471,7 @@ int totalTokenCount2 = 0; - FieldsEnum fields = MultiFields.getFields(reader).iterator(); + FieldsEnum fields = MultiInvertedFields.getFields(reader).iterator(); String fieldName = null; while((fieldName = fields.next()) != null) { if (fieldName.equals(DocMaker.ID_FIELD) || fieldName.equals(DocMaker.DATE_MSEC_FIELD) || fieldName.equals(DocMaker.TIME_SEC_FIELD)) { @@ -497,7 +484,7 @@ TermsEnum termsEnum = terms.iterator(null); DocsEnum docs = null; while(termsEnum.next() != null) { - docs = _TestUtil.docs(random, termsEnum, MultiFields.getLiveDocs(reader), docs, true); + docs = _TestUtil.docs(random, termsEnum, MultiInvertedFields.getLiveDocs(reader), docs, true); while(docs.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { totalTokenCount2 += docs.freq(); } @@ -750,7 +737,7 @@ 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(); Index: modules/facet/src/java/org/apache/lucene/facet/search/PayloadIterator.java =================================================================== --- modules/facet/src/java/org/apache/lucene/facet/search/PayloadIterator.java (revision 1310827) +++ modules/facet/src/java/org/apache/lucene/facet/search/PayloadIterator.java (working copy) @@ -4,7 +4,7 @@ import org.apache.lucene.index.DocsAndPositionsEnum; import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.Term; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.util.Bits; @@ -52,8 +52,8 @@ throws IOException { this.buffer = buffer; // TODO (Facet): avoid Multi*? - Bits liveDocs = MultiFields.getLiveDocs(indexReader); - this.tp = MultiFields.getTermPositionsEnum(indexReader, liveDocs, term.field(), term.bytes(), false); + Bits liveDocs = MultiInvertedFields.getLiveDocs(indexReader); + this.tp = MultiInvertedFields.getTermPositionsEnum(indexReader, liveDocs, term.field(), term.bytes(), false); } /** Index: modules/facet/src/java/org/apache/lucene/facet/search/sampling/TakmiSampleFixer.java =================================================================== --- modules/facet/src/java/org/apache/lucene/facet/search/sampling/TakmiSampleFixer.java (revision 1310827) +++ modules/facet/src/java/org/apache/lucene/facet/search/sampling/TakmiSampleFixer.java (working copy) @@ -3,7 +3,7 @@ import java.io.IOException; import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.Term; import org.apache.lucene.index.DocsEnum; import org.apache.lucene.search.DocIdSetIterator; @@ -105,10 +105,10 @@ Term drillDownTerm = DrillDown.term(searchParams, catPath); // TODO (Facet): avoid Multi*? - Bits liveDocs = MultiFields.getLiveDocs(indexReader); - int updatedCount = countIntersection(MultiFields.getTermDocsEnum(indexReader, liveDocs, - drillDownTerm.field(), drillDownTerm.bytes(), - false), + Bits liveDocs = MultiInvertedFields.getLiveDocs(indexReader); + int updatedCount = countIntersection(MultiInvertedFields.getTermDocsEnum(indexReader, liveDocs, + drillDownTerm.field(), drillDownTerm.bytes(), + false), docIds.iterator()); fresNode.setValue(updatedCount); Index: modules/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java =================================================================== --- modules/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java (revision 1310827) +++ modules/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java (working copy) @@ -14,11 +14,8 @@ import org.apache.lucene.facet.taxonomy.InconsistentTaxonomyException; import org.apache.lucene.facet.taxonomy.TaxonomyReader; import org.apache.lucene.facet.taxonomy.directory.Consts.LoadFullPathOnly; -import org.apache.lucene.index.CorruptIndexException; -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.DocsEnum; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.*; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; @@ -195,8 +192,8 @@ try { indexReaderLock.readLock().lock(); // TODO (Facet): avoid Multi*? - Bits liveDocs = MultiFields.getLiveDocs(indexReader); - DocsEnum docs = MultiFields.getTermDocsEnum(indexReader, liveDocs, Consts.FULL, new BytesRef(path), false); + Bits liveDocs = MultiInvertedFields.getLiveDocs(indexReader); + DocsEnum docs = MultiInvertedFields.getTermDocsEnum(indexReader, liveDocs, Consts.FULL, new BytesRef(path), false); if (docs != null && docs.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { ret = docs.docID(); } Index: modules/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java =================================================================== --- modules/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java (revision 1310827) +++ modules/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java (working copy) @@ -21,19 +21,9 @@ import org.apache.lucene.document.FieldType; import org.apache.lucene.document.StringField; import org.apache.lucene.document.TextField; -import org.apache.lucene.index.CorruptIndexException; -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.DocsEnum; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.*; import org.apache.lucene.index.IndexWriterConfig.OpenMode; -import org.apache.lucene.index.LogByteSizeMergePolicy; -import org.apache.lucene.index.MultiFields; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.TermsEnum; -import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.index.TieredMergePolicy; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; @@ -415,10 +405,10 @@ } // TODO (Facet): avoid Multi*? - Bits liveDocs = MultiFields.getLiveDocs(reader); - DocsEnum docs = MultiFields.getTermDocsEnum(reader, liveDocs, Consts.FULL, - new BytesRef(categoryPath.toString(delimiter)), - false); + Bits liveDocs = MultiInvertedFields.getLiveDocs(reader); + DocsEnum docs = MultiInvertedFields.getTermDocsEnum(reader, liveDocs, Consts.FULL, + new BytesRef(categoryPath.toString(delimiter)), + false); if (docs == null || docs.nextDoc() == DocIdSetIterator.NO_MORE_DOCS) { return -1; // category does not exist in taxonomy } @@ -452,10 +442,10 @@ if (reader == null) { reader = openReader(); } - Bits liveDocs = MultiFields.getLiveDocs(reader); - DocsEnum docs = MultiFields.getTermDocsEnum(reader, liveDocs, Consts.FULL, - new BytesRef(categoryPath.toString(delimiter, prefixLen)), - false); + Bits liveDocs = MultiInvertedFields.getLiveDocs(reader); + DocsEnum docs = MultiInvertedFields.getTermDocsEnum(reader, liveDocs, Consts.FULL, + new BytesRef(categoryPath.toString(delimiter, prefixLen)), + false); if (docs == null || docs.nextDoc() == DocIdSetIterator.NO_MORE_DOCS) { return -1; // category does not exist in taxonomy } @@ -767,7 +757,7 @@ } CategoryPath cp = new CategoryPath(); - Terms terms = MultiFields.getTerms(reader, Consts.FULL); + Terms terms = MultiInvertedFields.getTerms(reader, Consts.FULL); // The check is done here to avoid checking it on every iteration of the // below loop. A null term wlil be returned if there are no terms in the // lexicon, or after the Consts.FULL term. However while the loop is @@ -775,7 +765,7 @@ // terms. if (terms != null) { TermsEnum termsEnum = terms.iterator(null); - Bits liveDocs = MultiFields.getLiveDocs(reader); + Bits liveDocs = MultiInvertedFields.getLiveDocs(reader); DocsEnum docsEnum = null; while (termsEnum.next() != null) { BytesRef t = termsEnum.term(); @@ -848,7 +838,7 @@ // of using the existing "reader" object: IndexReader mainreader = openReader(); // TODO (Facet): can this then go segment-by-segment and avoid MultiDocsEnum etc? - Terms terms = MultiFields.getTerms(mainreader, Consts.FULL); + Terms terms = MultiInvertedFields.getTerms(mainreader, Consts.FULL); assert terms != null; // TODO (Facet): explicit check / throw exception? TermsEnum mainte = terms.iterator(null); DocsEnum mainde = null; @@ -858,7 +848,7 @@ DocsEnum[] otherdocsEnum = new DocsEnum[taxonomies.length]; // just for reuse for (int i=0; i fields = MultiFields.getIndexedFields(ir); + Collection fields = MultiInvertedFields.getIndexedFields(ir); fieldNames = fields.toArray(new String[fields.size()]); } @@ -705,7 +700,7 @@ public PriorityQueue retrieveTerms(int docNum) throws IOException { Map termFreqMap = new HashMap(); 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); Index: modules/queries/src/java/org/apache/lucene/queries/TermsFilter.java =================================================================== --- modules/queries/src/java/org/apache/lucene/queries/TermsFilter.java (revision 1310827) +++ modules/queries/src/java/org/apache/lucene/queries/TermsFilter.java (working copy) @@ -57,7 +57,7 @@ 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; Index: modules/queryparser/src/java/org/apache/lucene/queryparser/surround/query/SrndPrefixQuery.java =================================================================== --- modules/queryparser/src/java/org/apache/lucene/queryparser/surround/query/SrndPrefixQuery.java (revision 1310827) +++ modules/queryparser/src/java/org/apache/lucene/queryparser/surround/query/SrndPrefixQuery.java (working copy) @@ -22,7 +22,7 @@ import org.apache.lucene.util.StringHelper; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import java.io.IOException; @@ -59,7 +59,7 @@ MatchingTermVisitor mtv) throws IOException { /* inspired by PrefixQuery.rewrite(): */ - Terms terms = MultiFields.getTerms(reader, fieldName); + Terms terms = MultiInvertedFields.getTerms(reader, fieldName); if (terms != null) { TermsEnum termsEnum = terms.iterator(null); Index: modules/queryparser/src/java/org/apache/lucene/queryparser/surround/query/SrndTermQuery.java =================================================================== --- modules/queryparser/src/java/org/apache/lucene/queryparser/surround/query/SrndTermQuery.java (revision 1310827) +++ modules/queryparser/src/java/org/apache/lucene/queryparser/surround/query/SrndTermQuery.java (working copy) @@ -18,11 +18,8 @@ import java.io.IOException; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.Term; -import org.apache.lucene.index.TermsEnum; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.*; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.util.BytesRef; @@ -49,7 +46,7 @@ MatchingTermVisitor mtv) throws IOException { /* check term presence in index here for symmetry with other SimpleTerm's */ - Terms terms = MultiFields.getTerms(reader, fieldName); + Terms terms = MultiInvertedFields.getTerms(reader, fieldName); if (terms != null) { TermsEnum termsEnum = terms.iterator(null); Index: modules/queryparser/src/java/org/apache/lucene/queryparser/surround/query/SrndTruncQuery.java =================================================================== --- modules/queryparser/src/java/org/apache/lucene/queryparser/surround/query/SrndTruncQuery.java (revision 1310827) +++ modules/queryparser/src/java/org/apache/lucene/queryparser/surround/query/SrndTruncQuery.java (working copy) @@ -16,13 +16,10 @@ * limitations under the License. */ -import org.apache.lucene.index.Term; -import org.apache.lucene.index.TermsEnum; -import org.apache.lucene.index.Terms; +import org.apache.lucene.index.*; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.StringHelper; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import java.io.IOException; @@ -90,7 +87,7 @@ MatchingTermVisitor mtv) throws IOException { int prefixLength = prefix.length(); - Terms terms = MultiFields.getTerms(reader, fieldName); + Terms terms = MultiInvertedFields.getTerms(reader, fieldName); if (terms != null) { Matcher matcher = pattern.matcher(""); try { Index: modules/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java =================================================================== --- modules/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java (revision 1310827) +++ modules/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java (working copy) @@ -26,7 +26,7 @@ import java.util.PriorityQueue; import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.Term; import org.apache.lucene.index.Terms; import org.apache.lucene.search.FuzzyTermsEnum; @@ -388,7 +388,7 @@ AttributeSource atts = new AttributeSource(); MaxNonCompetitiveBoostAttribute maxBoostAtt = atts.addAttribute(MaxNonCompetitiveBoostAttribute.class); - Terms terms = MultiFields.getTerms(ir, term.field()); + Terms terms = MultiInvertedFields.getTerms(ir, term.field()); if (terms == null) { return Collections.emptyList(); } Index: modules/suggest/src/java/org/apache/lucene/search/spell/HighFrequencyDictionary.java =================================================================== --- modules/suggest/src/java/org/apache/lucene/search/spell/HighFrequencyDictionary.java (revision 1310827) +++ modules/suggest/src/java/org/apache/lucene/search/spell/HighFrequencyDictionary.java (working copy) @@ -21,9 +21,9 @@ import java.util.Comparator; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.Terms; -import org.apache.lucene.index.MultiFields; import org.apache.lucene.util.BytesRefIterator; import org.apache.lucene.util.BytesRef; @@ -59,7 +59,7 @@ private long freq; HighFrequencyIterator() throws IOException { - Terms terms = MultiFields.getTerms(reader, field); + Terms terms = MultiInvertedFields.getTerms(reader, field); if (terms != null) { termsEnum = terms.iterator(null); } else { Index: modules/suggest/src/java/org/apache/lucene/search/spell/LuceneDictionary.java =================================================================== --- modules/suggest/src/java/org/apache/lucene/search/spell/LuceneDictionary.java (revision 1310827) +++ modules/suggest/src/java/org/apache/lucene/search/spell/LuceneDictionary.java (working copy) @@ -18,9 +18,9 @@ */ import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.util.BytesRefIterator; import org.apache.lucene.index.Terms; -import org.apache.lucene.index.MultiFields; import java.io.*; @@ -41,7 +41,7 @@ } public final BytesRefIterator getWordsIterator() throws IOException { - final Terms terms = MultiFields.getTerms(reader, field); + final Terms terms = MultiInvertedFields.getTerms(reader, field); if (terms != null) { return terms.iterator(null); } else { Index: solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java =================================================================== --- solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java (revision 1310827) +++ solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java (working copy) @@ -575,7 +575,7 @@ final CharsRef spare = new CharsRef(); - Fields fields = MultiFields.getFields(req.getSearcher().getIndexReader()); + InvertedFields fields = MultiInvertedFields.getFields(req.getSearcher().getIndexReader()); if (fields == null) { // No indexed fields return; Index: solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java =================================================================== --- solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java (revision 1310827) +++ solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java (working copy) @@ -532,7 +532,7 @@ 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; Index: solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java =================================================================== --- solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java (revision 1310827) +++ solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java (working copy) @@ -118,7 +118,7 @@ final AtomicReader indexReader = rb.req.getSearcher().getAtomicReader(); - Fields lfields = indexReader.fields(); + InvertedFields lfields = indexReader.fields(); for (String field : fields) { NamedList fieldTerms = new NamedList(); Index: solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java =================================================================== --- solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java (revision 1310827) +++ solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java (working copy) @@ -10,7 +10,7 @@ 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 @@ } } 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) { Index: solr/core/src/java/org/apache/solr/request/SimpleFacets.java =================================================================== --- solr/core/src/java/org/apache/solr/request/SimpleFacets.java (revision 1310827) +++ solr/core/src/java/org/apache/solr/request/SimpleFacets.java (working copy) @@ -669,7 +669,7 @@ 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; Index: solr/core/src/java/org/apache/solr/search/function/FileFloatSource.java =================================================================== --- solr/core/src/java/org/apache/solr/search/function/FileFloatSource.java (revision 1310827) +++ solr/core/src/java/org/apache/solr/search/function/FileFloatSource.java (working copy) @@ -29,7 +29,7 @@ import org.apache.lucene.index.DocsEnum; import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiInvertedFields; import org.apache.lucene.index.IndexReaderContext; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.AtomicReaderContext; @@ -242,7 +242,7 @@ BytesRef internalKey = new BytesRef(); try { - TermsEnum termsEnum = MultiFields.getTerms(reader, idName).iterator(null); + TermsEnum termsEnum = MultiInvertedFields.getTerms(reader, idName).iterator(null); DocsEnum docsEnum = null; // removing deleted docs shouldn't matter Index: solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java =================================================================== --- solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java (revision 1310827) +++ solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java (working copy) @@ -262,8 +262,8 @@ 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); Index: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java =================================================================== --- solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java (revision 1310827) +++ solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java (working copy) @@ -41,7 +41,6 @@ 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 +583,7 @@ * @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 +611,7 @@ 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 +997,7 @@ 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(); Index: solr/core/src/test/org/apache/solr/search/TestDocSet.java =================================================================== --- solr/core/src/test/org/apache/solr/search/TestDocSet.java (revision 1310827) +++ solr/core/src/test/org/apache/solr/search/TestDocSet.java (working copy) @@ -24,7 +24,7 @@ import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.DocValues; import org.apache.lucene.index.StoredFieldVisitor; -import org.apache.lucene.index.Fields; +import org.apache.lucene.index.InvertedFields; import org.apache.lucene.index.AtomicReader; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.AtomicReaderContext; @@ -368,12 +368,12 @@ } @Override - public Fields fields() { + public InvertedFields fields() { return null; } @Override - public Fields getTermVectors(int doc) { + public InvertedFields getTermVectors(int doc) { return null; } Index: solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java =================================================================== --- solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java (revision 1310827) +++ solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java (working copy) @@ -28,8 +28,6 @@ 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 int getFirstMatch(IndexReader r, Term t) throws IOException { - Fields fields = MultiFields.getFields(r); + InvertedFields fields = MultiInvertedFields.getFields(r); if (fields == null) return -1; Terms terms = fields.terms(t.field()); if (terms == null) return -1; @@ -1707,7 +1705,7 @@ if (!termsEnum.seekExact(termBytes, false)) { return -1; } - DocsEnum docs = termsEnum.docs(MultiFields.getLiveDocs(r), null, false); + DocsEnum docs = termsEnum.docs(MultiInvertedFields.getLiveDocs(r), null, false); int id = docs.nextDoc(); if (id != DocIdSetIterator.NO_MORE_DOCS) { int next = docs.nextDoc();