You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2012/04/08 15:38:04 UTC

svn commit: r1310998 [1/2] - in /lucene/dev/trunk: lucene/ lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/ lucene/contrib/highlighter/src/java/org/apache/lucene/search/vectorhighlight/ lucene/contrib/memory/src/java/org/apache/l...

Author: mikemccand
Date: Sun Apr  8 13:38:01 2012
New Revision: 1310998

URL: http://svn.apache.org/viewvc?rev=1310998&view=rev
Log:
LUCENE-3109: revert

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/FieldsConsumer.java
      - copied unchanged from r1310968, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/FieldsConsumer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/FieldsProducer.java
      - copied unchanged from r1310968, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/FieldsProducer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Fields.java
      - copied unchanged from r1310968, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Fields.java
Removed:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/InvertedFieldsConsumer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/InvertedFieldsProducer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/InvertedFields.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java
    lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java
    lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
    lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/misc/HighFreqTerms.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermsReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermsWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/appending/AppendingPostingsFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xPostingsFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValues.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFieldsEnum.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/TermContext.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/appending/TestAppendingCodec.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/document/TestDocument.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocCount.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestFilterAtomicReader.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSumDocFreq.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestMultiThreadTermVectors.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWFieldsWriter.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWPostingsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
    lucene/dev/trunk/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java
    lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/TermsFilter.java
    lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/function/valuesource/SumTotalTermFreqValueSource.java
    lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java
    lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java
    lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/mlt/MoreLikeThis.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestDocSet.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Sun Apr  8 13:38:01 2012
@@ -259,9 +259,6 @@ Changes in backwards compatibility polic
 
 * LUCENE-2000: clone() now returns covariant types where possible. (ryan)
 
-* LUCENE-3109: Rename Fields/Producer/Consumer -> InvertedFields*
-  (Iulius Curt via Mike McCandless)
-
 Changes in Runtime Behavior
 
 * LUCENE-2846: omitNorms now behaves like omitTermFrequencyAndPositions, if you

Modified: lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java (original)
+++ lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java Sun Apr  8 13:38:01 2012
@@ -33,7 +33,7 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.InvertedFields;
+import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -70,7 +70,7 @@ public class TokenSources {
       String field, Document doc, Analyzer analyzer) throws IOException {
     TokenStream ts = null;
 
-    InvertedFields vectors = reader.getTermVectors(docId);
+    Fields vectors = reader.getTermVectors(docId);
     if (vectors != null) {
       Terms vector = vectors.terms(field);
       if (vector != null) {
@@ -102,7 +102,7 @@ public class TokenSources {
       String field, Analyzer analyzer) throws IOException {
     TokenStream ts = null;
 
-    InvertedFields vectors = reader.getTermVectors(docId);
+    Fields vectors = reader.getTermVectors(docId);
     if (vectors != null) {
       Terms vector = vectors.terms(field);
       if (vector != null) {
@@ -275,7 +275,7 @@ public class TokenSources {
   public static TokenStream getTokenStream(IndexReader reader, int docId,
       String field) throws IOException {
 
-    InvertedFields vectors = reader.getTermVectors(docId);
+    Fields vectors = reader.getTermVectors(docId);
     if (vectors == null) {
       throw new IllegalArgumentException(field + " in doc #" + docId
           + "does not have any term position data stored");

Modified: lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java (original)
+++ lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java Sun Apr  8 13:38:01 2012
@@ -22,7 +22,7 @@ import java.util.LinkedList;
 import java.util.Set;
 
 import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.InvertedFields;
+import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -78,7 +78,7 @@ public class FieldTermStack {
     // just return to make null snippet if un-matched fieldName specified when fieldMatch == true
     if( termSet == null ) return;
 
-    final InvertedFields vectors = reader.getTermVectors(docId);
+    final Fields vectors = reader.getTermVectors(docId);
     if (vectors == null) {
       // null snippet
       return;

Modified: lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (original)
+++ lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Sun Apr  8 13:38:01 2012
@@ -35,13 +35,13 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.InvertedFields;
 import org.apache.lucene.index.Norm;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.FieldInvertState;
+import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.OrdTermState;
 import org.apache.lucene.index.StoredFieldVisitor;
@@ -58,6 +58,7 @@ import org.apache.lucene.store.RAMDirect
 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;
 
 /**
@@ -715,7 +716,7 @@ public class MemoryIndex {
       return fieldInfos;
     }
 
-    private class MemoryFields extends InvertedFields {
+    private class MemoryFields extends Fields {
       @Override
       public FieldsEnum iterator() {
         return new FieldsEnum() {
@@ -790,7 +791,7 @@ public class MemoryIndex {
     }
   
     @Override
-    public InvertedFields fields() {
+    public Fields fields() {
       sortFields();
       return new MemoryFields();
     }
@@ -1016,7 +1017,7 @@ public class MemoryIndex {
     }
     
     @Override
-    public InvertedFields getTermVectors(int docID) {
+    public Fields getTermVectors(int docID) {
       if (docID == 0) {
         return fields();
       } else {

Modified: lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/misc/HighFreqTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/misc/HighFreqTerms.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/misc/HighFreqTerms.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/misc/HighFreqTerms.java Sun Apr  8 13:38:01 2012
@@ -19,8 +19,8 @@ package org.apache.lucene.misc;
 
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.InvertedFields;
 import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.Terms;
@@ -115,7 +115,7 @@ public class HighFreqTerms {
     TermStatsQueue tiq = null;
     
     if (field != null) {
-      InvertedFields fields = MultiFields.getFields(reader);
+      Fields fields = MultiFields.getFields(reader);
       if (fields == null) {
         throw new RuntimeException("field " + field + " not found");
       }
@@ -126,7 +126,7 @@ public class HighFreqTerms {
         fillQueue(termsEnum, tiq, field);
       }
     } else {
-      InvertedFields fields = MultiFields.getFields(reader);
+      Fields fields = MultiFields.getFields(reader);
       if (fields == null) {
         throw new RuntimeException("no fields found for this index");
       }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermsReader.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermsReader.java Sun Apr  8 13:38:01 2012
@@ -56,7 +56,7 @@ import org.apache.lucene.util.DoubleBarr
  * implementation of the terms dict index. 
  * @lucene.experimental */
 
-public class BlockTermsReader extends InvertedFieldsProducer {
+public class BlockTermsReader extends FieldsProducer {
   // Open input to the main terms dict file (_X.tis)
   private final IndexInput in;
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermsWriter.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermsWriter.java Sun Apr  8 13:38:01 2012
@@ -47,7 +47,7 @@ import org.apache.lucene.util.RamUsageEs
  * @lucene.experimental
  */
 
-public class BlockTermsWriter extends InvertedFieldsConsumer {
+public class BlockTermsWriter extends FieldsConsumer {
 
   final static String CODEC_NAME = "BLOCK_TERMS_DICT";
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java Sun Apr  8 13:38:01 2012
@@ -85,7 +85,7 @@ import org.apache.lucene.util.fst.Util;
  * @lucene.experimental
  */
 
-public class BlockTreeTermsReader extends InvertedFieldsProducer {
+public class BlockTreeTermsReader extends FieldsProducer {
 
   // Open input to the main terms dict file (_X.tib)
   private final IndexInput in;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java Sun Apr  8 13:38:01 2012
@@ -83,7 +83,7 @@ import org.apache.lucene.util.fst.Util;
  * @lucene.experimental
 */
 
-public class BlockTreeTermsWriter extends InvertedFieldsConsumer {
+public class BlockTreeTermsWriter extends FieldsConsumer {
 
   public final static int DEFAULT_MIN_BLOCK_SIZE = 25;
   public final static int DEFAULT_MAX_BLOCK_SIZE = 48;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java Sun Apr  8 13:38:01 2012
@@ -49,12 +49,12 @@ public abstract class PostingsFormat imp
   }
   
   /** Writes a new segment */
-  public abstract InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException;
+  public abstract FieldsConsumer 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 InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException;
+  public abstract FieldsProducer fieldsProducer(SegmentReadState state) throws IOException;
 
   /**
    * Gathers files associated with this segment

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java Sun Apr  8 13:38:01 2012
@@ -22,7 +22,7 @@ import java.io.IOException;
 
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute; // javadocs
 import org.apache.lucene.index.DocsAndPositionsEnum; // javadocs
-import org.apache.lucene.index.InvertedFields;
+import org.apache.lucene.index.Fields;
 
 /**
  * Codec API for reading term vectors:
@@ -35,7 +35,7 @@ public abstract class TermVectorsReader 
    *  term vectors were not indexed. If offsets are
    *  available they are in an {@link OffsetAttribute}
    *  available from the {@link DocsAndPositionsEnum}. */
-  public abstract InvertedFields get(int doc) throws IOException;
+  public abstract Fields get(int doc) throws IOException;
 
   /** Create a clone that one caller at a time may use to
    *  read term vectors. */

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java Sun Apr  8 13:38:01 2012
@@ -24,7 +24,7 @@ import java.util.Comparator;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.InvertedFields;
+import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.Terms;
@@ -153,7 +153,7 @@ public abstract class TermVectorsWriter 
         }
         // NOTE: it's very important to first assign to vectors then pass it to
         // termVectorsWriter.addAllDocVectors; see LUCENE-1282
-        InvertedFields vectors = reader.reader.getTermVectors(docID);
+        Fields vectors = reader.reader.getTermVectors(docID);
         addAllDocVectors(vectors, mergeState.fieldInfos);
         docCount++;
         mergeState.checkAbort.work(300);
@@ -166,9 +166,9 @@ public abstract class TermVectorsWriter 
   /** Safe (but, slowish) default method to write every
    *  vector field in the document.  This default
    *  implementation requires that the vectors implement
-   *  both InvertedFields.getUniqueFieldCount and
+   *  both Fields.getUniqueFieldCount and
    *  Terms.getUniqueTermCount. */
-  protected final void addAllDocVectors(InvertedFields vectors, FieldInfos fieldInfos) throws IOException {
+  protected final void addAllDocVectors(Fields vectors, FieldInfos fieldInfos) throws IOException {
     if (vectors == null) {
       startDocument(0);
       return;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/appending/AppendingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/appending/AppendingPostingsFormat.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/appending/AppendingPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/appending/AppendingPostingsFormat.java Sun Apr  8 13:38:01 2012
@@ -22,8 +22,8 @@ import java.util.Set;
 
 import org.apache.lucene.codecs.BlockTreeTermsReader;
 import org.apache.lucene.codecs.BlockTreeTermsWriter;
-import org.apache.lucene.codecs.InvertedFieldsConsumer;
-import org.apache.lucene.codecs.InvertedFieldsProducer;
+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;
@@ -44,11 +44,11 @@ class AppendingPostingsFormat extends Po
   }
 
   @Override
-  public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     PostingsWriterBase docsWriter = new Lucene40PostingsWriter(state);
     boolean success = false;
     try {
-      InvertedFieldsConsumer ret = new AppendingTermsWriter(state, docsWriter, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
+      FieldsConsumer ret = new AppendingTermsWriter(state, docsWriter, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
       success = true;
       return ret;
     } finally {
@@ -59,12 +59,12 @@ class AppendingPostingsFormat extends Po
   }
 
   @Override
-  public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
     PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix);
     
     boolean success = false;
     try {
-      InvertedFieldsProducer ret = new AppendingTermsReader(
+      FieldsProducer ret = new AppendingTermsReader(
                                                     state.dir,
                                                     state.fieldInfos,
                                                     state.segmentInfo.name,

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java Sun Apr  8 13:38:01 2012
@@ -25,7 +25,7 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.TreeMap;
 
-import org.apache.lucene.codecs.InvertedFieldsProducer;
+import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
@@ -49,7 +49,7 @@ import org.apache.lucene.util.UnicodeUti
  * @deprecated (4.0)
  */
 @Deprecated
-class Lucene3xFields extends InvertedFieldsProducer {
+class Lucene3xFields extends FieldsProducer {
   
   private static final boolean DEBUG_SURROGATES = false;
 
@@ -59,14 +59,18 @@ class Lucene3xFields extends InvertedFie
   public final IndexInput freqStream;
   public final IndexInput proxStream;
   final private FieldInfos fieldInfos;
+  private final SegmentInfo si;
   final TreeMap<String,FieldInfo> fields = new TreeMap<String,FieldInfo>();
   final Map<String,Terms> preTerms = new HashMap<String,Terms>();
   private final Directory dir;
+  private final IOContext context;
   private Directory cfsReader;
 
   public Lucene3xFields(Directory dir, FieldInfos fieldInfos, SegmentInfo info, IOContext context, int indexDivisor)
     throws IOException {
 
+    si = info;
+
     // NOTE: we must always load terms index, even for
     // "sequential" scan during merging, because what is
     // sequential to merger may not be to TermInfosReader
@@ -84,6 +88,7 @@ class Lucene3xFields extends InvertedFie
         tisNoIndex = null;
         tis = r;
       }
+      this.context = context;
       this.fieldInfos = fieldInfos;
 
       // make sure that all index files have been read or are kept open

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xPostingsFormat.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xPostingsFormat.java Sun Apr  8 13:38:01 2012
@@ -20,8 +20,8 @@ package org.apache.lucene.codecs.lucene3
 import java.util.Set;
 import java.io.IOException;
 
-import org.apache.lucene.codecs.InvertedFieldsConsumer;
-import org.apache.lucene.codecs.InvertedFieldsProducer;
+import org.apache.lucene.codecs.FieldsConsumer;
+import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentWriteState;
@@ -55,12 +55,12 @@ class Lucene3xPostingsFormat extends Pos
   }
   
   @Override
-  public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     throw new UnsupportedOperationException("this codec can only be used for reading");
   }
 
   @Override
-  public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
     return new Lucene3xFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor);
   }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java Sun Apr  8 13:38:01 2012
@@ -30,7 +30,7 @@ import org.apache.lucene.index.DocsAndPo
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.InvertedFields;
+import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexFormatTooNewException;
@@ -195,7 +195,7 @@ class Lucene3xTermVectorsReader extends 
     return size;
   }
 
-  private class TVFields extends InvertedFields {
+  private class TVFields extends Fields {
     private final int[] fieldNumbers;
     private final long[] fieldFPs;
     private final Map<Integer,Integer> fieldNumberToIndex = new HashMap<Integer,Integer>();
@@ -654,12 +654,12 @@ class Lucene3xTermVectorsReader extends 
   }
 
   @Override
-  public InvertedFields get(int docID) throws IOException {
+  public Fields 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) {
-      InvertedFields fields = new TVFields(docID);
+      Fields 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

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java Sun Apr  8 13:38:01 2012
@@ -22,8 +22,8 @@ import java.util.Set;
 
 import org.apache.lucene.codecs.BlockTreeTermsReader;
 import org.apache.lucene.codecs.BlockTreeTermsWriter;
-import org.apache.lucene.codecs.InvertedFieldsConsumer;
-import org.apache.lucene.codecs.InvertedFieldsProducer;
+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;
@@ -54,7 +54,7 @@ public class Lucene40PostingsFormat exte
   }
 
   @Override
-  public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     PostingsWriterBase docs = new Lucene40PostingsWriter(state);
 
     // TODO: should we make the terms index more easily
@@ -63,7 +63,7 @@ public class Lucene40PostingsFormat exte
     // Or... you must make a new Codec for this?
     boolean success = false;
     try {
-      InvertedFieldsConsumer ret = new BlockTreeTermsWriter(state, docs, minBlockSize, maxBlockSize);
+      FieldsConsumer ret = new BlockTreeTermsWriter(state, docs, minBlockSize, maxBlockSize);
       success = true;
       return ret;
     } finally {
@@ -76,12 +76,12 @@ public class Lucene40PostingsFormat exte
   public final static int TERMS_CACHE_SIZE = 1024;
 
   @Override
-  public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
     PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix);
 
     boolean success = false;
     try {
-      InvertedFieldsProducer ret = new BlockTreeTermsReader(
+      FieldsProducer ret = new BlockTreeTermsReader(
                                                     state.dir,
                                                     state.fieldInfos,
                                                     state.segmentInfo.name,

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java Sun Apr  8 13:38:01 2012
@@ -30,7 +30,7 @@ import org.apache.lucene.index.DocsAndPo
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.InvertedFields;
+import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexFormatTooNewException;
@@ -225,7 +225,7 @@ public class Lucene40TermVectorsReader e
     return size;
   }
 
-  private class TVFields extends InvertedFields {
+  private class TVFields extends Fields {
     private final int[] fieldNumbers;
     private final long[] fieldFPs;
     private final Map<Integer,Integer> fieldNumberToIndex = new HashMap<Integer,Integer>();
@@ -668,12 +668,12 @@ public class Lucene40TermVectorsReader e
   }
 
   @Override
-  public InvertedFields get(int docID) throws IOException {
+  public Fields 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) {
-      InvertedFields fields = new TVFields(docID);
+      Fields 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

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java Sun Apr  8 13:38:01 2012
@@ -23,7 +23,7 @@ import java.util.Comparator;
 import org.apache.lucene.codecs.TermVectorsReader;
 import org.apache.lucene.codecs.TermVectorsWriter;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.InvertedFields;
+import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.MergePolicy.MergeAbortedException;
 import org.apache.lucene.index.MergeState;
@@ -231,7 +231,7 @@ public final class Lucene40TermVectorsWr
   }
 
   @Override
-  public int merge(MergeState mergeState) throws IOException {
+  public final int merge(MergeState mergeState) throws IOException {
     // Used for bulk-reading raw bytes for term vectors
     int rawDocLengths[] = new int[MAX_RAW_MERGE_DOCS];
     int rawDocLengths2[] = new int[MAX_RAW_MERGE_DOCS];
@@ -309,7 +309,7 @@ public final class Lucene40TermVectorsWr
         
         // NOTE: it's very important to first assign to vectors then pass it to
         // termVectorsWriter.addAllDocVectors; see LUCENE-1282
-        InvertedFields vectors = reader.reader.getTermVectors(docNum);
+        Fields vectors = reader.reader.getTermVectors(docNum);
         addAllDocVectors(vectors, mergeState.fieldInfos);
         totalNumDocs++;
         mergeState.checkAbort.work(300);
@@ -339,7 +339,7 @@ public final class Lucene40TermVectorsWr
       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
-        InvertedFields vectors = reader.reader.getTermVectors(docNum);
+        Fields vectors = reader.reader.getTermVectors(docNum);
         addAllDocVectors(vectors, mergeState.fieldInfos);
         mergeState.checkAbort.work(300);
       }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java Sun Apr  8 13:38:01 2012
@@ -24,8 +24,8 @@ import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
-import org.apache.lucene.codecs.InvertedFieldsConsumer;
-import org.apache.lucene.codecs.InvertedFieldsProducer;
+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;
@@ -283,12 +283,12 @@ public class MemoryPostingsFormat extend
   private static String EXTENSION = "ram";
 
   @Override
-  public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+  public FieldsConsumer 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 InvertedFieldsConsumer() {
+    return new FieldsConsumer() {
       @Override
       public TermsConsumer addField(FieldInfo field) {
         //System.out.println("\naddField field=" + field.name);
@@ -840,7 +840,7 @@ public class MemoryPostingsFormat extend
   }
 
   @Override
-  public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
     final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, EXTENSION);
     final IndexInput in = state.dir.openInput(fileName, IOContext.READONCE);
 
@@ -860,7 +860,7 @@ public class MemoryPostingsFormat extend
       in.close();
     }
 
-    return new InvertedFieldsProducer() {
+    return new FieldsProducer() {
       @Override
       public FieldsEnum iterator() {
         final Iterator<TermsReader> iter = fields.values().iterator();

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java Sun Apr  8 13:38:01 2012
@@ -28,8 +28,8 @@ import java.util.ServiceLoader; // javad
 import java.util.Set;
 import java.util.TreeMap;
 
-import org.apache.lucene.codecs.InvertedFieldsConsumer;
-import org.apache.lucene.codecs.InvertedFieldsProducer;
+import org.apache.lucene.codecs.FieldsConsumer;
+import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.TermsConsumer;
 import org.apache.lucene.index.FieldInfo;
@@ -72,17 +72,17 @@ public abstract class PerFieldPostingsFo
   }
 
   @Override
-  public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state)
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state)
       throws IOException {
     return new FieldsWriter(state);
   }
 
   // NOTE: not private to avoid $accessN at runtime!!
   static class FieldsConsumerAndID implements Closeable {
-    final InvertedFieldsConsumer fieldsConsumer;
+    final FieldsConsumer fieldsConsumer;
     final String segmentSuffix;
 
-    public FieldsConsumerAndID(InvertedFieldsConsumer fieldsConsumer, String segmentSuffix) {
+    public FieldsConsumerAndID(FieldsConsumer fieldsConsumer, String segmentSuffix) {
       this.fieldsConsumer = fieldsConsumer;
       this.segmentSuffix = segmentSuffix;
     }
@@ -93,7 +93,7 @@ public abstract class PerFieldPostingsFo
     }
   };
     
-  private class FieldsWriter extends InvertedFieldsConsumer {
+  private class FieldsWriter extends FieldsConsumer {
 
     private final Map<PostingsFormat,FieldsConsumerAndID> formats = new IdentityHashMap<PostingsFormat,FieldsConsumerAndID>();
 
@@ -181,10 +181,10 @@ public abstract class PerFieldPostingsFo
     }
   }
 
-  private class FieldsReader extends InvertedFieldsProducer {
+  private class FieldsReader extends FieldsProducer {
 
-    private final Map<String,InvertedFieldsProducer> fields = new TreeMap<String,InvertedFieldsProducer>();
-    private final Map<PostingsFormat,InvertedFieldsProducer> formats = new IdentityHashMap<PostingsFormat,InvertedFieldsProducer>();
+    private final Map<String,FieldsProducer> fields = new TreeMap<String,FieldsProducer>();
+    private final Map<PostingsFormat,FieldsProducer> formats = new IdentityHashMap<PostingsFormat,FieldsProducer>();
 
     public FieldsReader(final SegmentReadState readState) throws IOException {
 
@@ -243,7 +243,7 @@ public abstract class PerFieldPostingsFo
 
     @Override
     public Terms terms(String field) throws IOException {
-      InvertedFieldsProducer fieldsProducer = fields.get(field);
+      FieldsProducer fieldsProducer = fields.get(field);
       return fieldsProducer == null ? null : fieldsProducer.terms(field);
     }
     
@@ -259,7 +259,7 @@ public abstract class PerFieldPostingsFo
   }
 
   @Override
-  public InvertedFieldsProducer fieldsProducer(SegmentReadState state)
+  public FieldsProducer fieldsProducer(SegmentReadState state)
       throws IOException {
     return new FieldsReader(state);
   }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java Sun Apr  8 13:38:01 2012
@@ -22,8 +22,8 @@ import java.util.Set;
 
 import org.apache.lucene.codecs.BlockTreeTermsReader;
 import org.apache.lucene.codecs.BlockTreeTermsWriter;
-import org.apache.lucene.codecs.InvertedFieldsConsumer;
-import org.apache.lucene.codecs.InvertedFieldsProducer;
+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;
@@ -66,7 +66,7 @@ public abstract class PulsingPostingsFor
   }
 
   @Override
-  public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     PostingsWriterBase docsWriter = wrappedPostingsBaseFormat.postingsWriterBase(state);
 
     // Terms that have <= freqCutoff number of docs are
@@ -76,7 +76,7 @@ public abstract class PulsingPostingsFor
     // Terms dict
     boolean success = false;
     try {
-      InvertedFieldsConsumer ret = new BlockTreeTermsWriter(state, pulsingWriter, minBlockSize, maxBlockSize);
+      FieldsConsumer ret = new BlockTreeTermsWriter(state, pulsingWriter, minBlockSize, maxBlockSize);
       success = true;
       return ret;
     } finally {
@@ -87,14 +87,14 @@ public abstract class PulsingPostingsFor
   }
 
   @Override
-  public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
 
     PostingsReaderBase docsReader = wrappedPostingsBaseFormat.postingsReaderBase(state);
     PostingsReaderBase pulsingReader = new PulsingPostingsReader(docsReader);
 
     boolean success = false;
     try {
-      InvertedFieldsProducer ret = new BlockTreeTermsReader(
+      FieldsProducer ret = new BlockTreeTermsReader(
                                                     state.dir, state.fieldInfos, state.segmentInfo.name,
                                                     pulsingReader,
                                                     state.context,

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java Sun Apr  8 13:38:01 2012
@@ -22,7 +22,7 @@ import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.lucene.codecs.InvertedFieldsProducer;
+import org.apache.lucene.codecs.FieldsProducer;
 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.PairOu
 import org.apache.lucene.util.fst.PositiveIntOutputs;
 import org.apache.lucene.util.fst.Util;
 
-class SimpleTextFieldsReader extends InvertedFieldsProducer {
+class SimpleTextFieldsReader extends FieldsProducer {
 
   private final IndexInput in;
   private final FieldInfos fieldInfos;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java Sun Apr  8 13:38:01 2012
@@ -18,7 +18,7 @@ package org.apache.lucene.codecs.simplet
  */
 
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.codecs.InvertedFieldsConsumer;
+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 org.apache.lucene.store.IndexOutp
 import java.io.IOException;
 import java.util.Comparator;
 
-class SimpleTextFieldsWriter extends InvertedFieldsConsumer {
+class SimpleTextFieldsWriter extends FieldsConsumer {
   
   private final IndexOutput out;
   private final BytesRef scratch = new BytesRef(10);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java Sun Apr  8 13:38:01 2012
@@ -20,8 +20,8 @@ package org.apache.lucene.codecs.simplet
 import java.io.IOException;
 import java.util.Set;
 
-import org.apache.lucene.codecs.InvertedFieldsConsumer;
-import org.apache.lucene.codecs.InvertedFieldsProducer;
+import org.apache.lucene.codecs.FieldsConsumer;
+import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentWriteState;
@@ -43,12 +43,12 @@ public class SimpleTextPostingsFormat ex
   }
 
   @Override
-  public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     return new SimpleTextFieldsWriter(state);
   }
 
   @Override
-  public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
     return new SimpleTextFieldsReader(state);
   }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java Sun Apr  8 13:38:01 2012
@@ -29,7 +29,7 @@ import java.util.TreeMap;
 import org.apache.lucene.codecs.TermVectorsReader;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.InvertedFields;
+import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
@@ -94,7 +94,7 @@ public class SimpleTextTermVectorsReader
   }
   
   @Override
-  public InvertedFields get(int doc) throws IOException {
+  public Fields get(int doc) throws IOException {
     // TestTV tests for this in testBadParams... but is this
     // really guaranteed by the API?
     if (doc < 0 || doc >= offsets.size()) {
@@ -221,7 +221,7 @@ public class SimpleTextTermVectorsReader
     return scratchUTF16.toString();
   }
   
-  private class SimpleTVFields extends InvertedFields {
+  private class SimpleTVFields extends Fields {
     private final SortedMap<String,SimpleTVTerms> fields;
     
     SimpleTVFields(SortedMap<String,SimpleTVTerms> fields) throws IOException {

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

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java Sun Apr  8 13:38:01 2012
@@ -83,7 +83,7 @@ public abstract class BaseCompositeReade
   }
 
   @Override
-  public final InvertedFields getTermVectors(int docID) throws IOException {
+  public final Fields getTermVectors(int docID) throws IOException {
     ensureOpen();
     final int i = readerIndex(docID);        // find subreader num
     return subReaders[i].getTermVectors(docID - starts[i]); // dispatch to subreader

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java Sun Apr  8 13:38:01 2012
@@ -357,7 +357,7 @@ class BufferedDeletesStream {
   // Delete by Term
   private synchronized long applyTermDeletes(Iterable<Term> termsIter, ReadersAndLiveDocs rld, SegmentReader reader) throws IOException {
     long delCount = 0;
-    InvertedFields fields = reader.fields();
+    Fields fields = reader.fields();
     if (fields == null) {
       // This reader has no postings
       return 0;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Sun Apr  8 13:38:01 2012
@@ -701,11 +701,11 @@ public class CheckIndex {
   }
 
   /**
-   * checks InvertedFields api is consistent with itself.
+   * checks Fields 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(InvertedFields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, IndexSearcher searcher) throws IOException {
+  private Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, IndexSearcher searcher) throws IOException {
     // TODO: we should probably return our own stats thing...?!
     
     final Status.TermIndexStatus status = new Status.TermIndexStatus();
@@ -1003,7 +1003,7 @@ public class CheckIndex {
         // make sure TermsEnum is empty:
         final Terms fieldTerms2 = fieldsEnum.terms();
         if (fieldTerms2 != null && fieldTerms2.iterator(null).next() != null) {
-          throw new RuntimeException("InvertedFields.terms(field=" + field + ") returned null yet the field appears to have terms");
+          throw new RuntimeException("Fields.terms(field=" + field + ") returned null yet the field appears to have terms");
         }
       } else {
         if (fieldTerms instanceof BlockTreeTermsReader.FieldReader) {
@@ -1160,7 +1160,7 @@ public class CheckIndex {
         infoStream.print("    test: terms, freq, prox...");
       }
 
-      final InvertedFields fields = reader.fields();
+      final Fields fields = reader.fields();
       status = checkFields(fields, liveDocs, maxDoc, fieldInfos, is);
       if (liveDocs != null) {
         if (infoStream != null) {
@@ -1328,7 +1328,7 @@ public class CheckIndex {
         }
       }
 
-      msg("OK [" + status.docCount + " total doc Count; Num DocValues InvertedFields "
+      msg("OK [" + status.docCount + " total doc Count; Num DocValues Fields "
           + status.totalValueFields);
     } catch (Throwable e) {
       msg("ERROR [" + String.valueOf(e.getMessage()) + "]");
@@ -1362,7 +1362,7 @@ public class CheckIndex {
 
       final Bits liveDocs = reader.getLiveDocs();
 
-      final InvertedFields postingsFields;
+      final Fields postingsFields;
       // TODO: testTermsIndex
       if (crossCheckTermVectors) {
         postingsFields = reader.fields();
@@ -1377,7 +1377,7 @@ public class CheckIndex {
         // Intentionally pull/visit (but don't count in
         // stats) deleted documents to make sure they too
         // are not corrupt:
-        InvertedFields tfv = reader.getTermVectors(j);
+        Fields tfv = reader.getTermVectors(j);
 
         // TODO: can we make a IS(FIR) that searches just
         // this term vector... to pass for searcher?

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java Sun Apr  8 13:38:01 2012
@@ -200,7 +200,7 @@ public class DocTermOrds {
     }
     if (indexedTermsArray == null) {
       //System.out.println("GET normal enum");
-      final InvertedFields fields = reader.fields();
+      final Fields fields = reader.fields();
       if (fields == null) {
         return null;
       }
@@ -241,7 +241,7 @@ public class DocTermOrds {
     final int[] lastTerm = new int[maxDoc];    // last term we saw for this document
     final byte[][] bytes = new byte[maxDoc][]; // list of term numbers for the doc (delta encoded vInts)
 
-    final InvertedFields fields = reader.fields();
+    final Fields fields = reader.fields();
     if (fields == null) {
       // No terms
       return;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValues.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValues.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValues.java Sun Apr  8 13:38:01 2012
@@ -74,7 +74,7 @@ public abstract class DocValues implemen
    * <p>
    * {@link Source} instances obtained from this method are closed / released
    * from the cache once this {@link DocValues} instance is closed by the
-   * {@link IndexReader}, {@link InvertedFields} or {@link FieldsEnum} the
+   * {@link IndexReader}, {@link Fields} or {@link FieldsEnum} the
    * {@link DocValues} was created from.
    */
   public Source getSource() throws IOException {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java Sun Apr  8 13:38:01 2012
@@ -36,12 +36,12 @@ import java.util.Comparator;
  */
 public class FilterAtomicReader extends AtomicReader {
 
-  /** Base class for filtering {@link InvertedFields}
+  /** Base class for filtering {@link Fields}
    *  implementations. */
-  public static class FilterFields extends InvertedFields {
-    protected final InvertedFields in;
+  public static class FilterFields extends Fields {
+    protected final Fields in;
 
-    public FilterFields(InvertedFields in) {
+    public FilterFields(Fields in) {
       this.in = in;
     }
 
@@ -329,7 +329,7 @@ public class FilterAtomicReader extends 
   }
 
   @Override
-  public InvertedFields getTermVectors(int docID)
+  public Fields getTermVectors(int docID)
           throws IOException {
     ensureOpen();
     return in.getTermVectors(docID);
@@ -365,7 +365,7 @@ public class FilterAtomicReader extends 
   }
   
   @Override
-  public InvertedFields fields() throws IOException {
+  public Fields fields() throws IOException {
     ensureOpen();
     return in.fields();
   }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Sun Apr  8 13:38:01 2012
@@ -22,7 +22,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.lucene.codecs.InvertedFieldsConsumer;
+import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CollectionUtil;
@@ -34,7 +34,7 @@ final class FreqProxTermsWriter extends 
   void abort() {}
 
   // TODO: would be nice to factor out more of this, eg the
-  // FreqProxFieldMergeState, and code to visit all InvertedFields
+  // FreqProxFieldMergeState, and code to visit all Fields
   // under the same FieldInfo together, up into TermsHash*.
   // Other writers would presumably share alot of this...
 
@@ -57,7 +57,7 @@ final class FreqProxTermsWriter extends 
     // Sort by field name
     CollectionUtil.quickSort(allFields);
 
-    final InvertedFieldsConsumer consumer = state.codec.postingsFormat().fieldsConsumer(state);
+    final FieldsConsumer consumer = state.codec.postingsFormat().fieldsConsumer(state);
 
     boolean success = false;
 
@@ -66,7 +66,7 @@ final class FreqProxTermsWriter extends 
       
       /*
     Current writer chain:
-      InvertedFieldsConsumer
+      FieldsConsumer
         -> IMPL: FormatPostingsTermsDictWriter
           -> TermsConsumer
             -> IMPL: FormatPostingsTermsDictWriter.TermsWriter

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java Sun Apr  8 13:38:01 2012
@@ -23,7 +23,7 @@ import java.util.Map;
 
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
-import org.apache.lucene.codecs.InvertedFieldsConsumer;
+import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.PostingsConsumer;
 import org.apache.lucene.codecs.TermStats;
 import org.apache.lucene.codecs.TermsConsumer;
@@ -316,7 +316,7 @@ final class FreqProxTermsWriterPerField 
   /* Walk through all unique text tokens (Posting
    * instances) found in this field and serialize them
    * into a single RAM segment. */
-  void flush(String fieldName, InvertedFieldsConsumer consumer,  final SegmentWriteState state)
+  void flush(String fieldName, FieldsConsumer consumer,  final SegmentWriteState state)
     throws CorruptIndexException, IOException {
 
     final TermsConsumer termsConsumer = consumer.addField(fieldInfo);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexReader.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexReader.java Sun Apr  8 13:38:01 2012
@@ -27,9 +27,11 @@ import java.util.concurrent.atomic.Atomi
 
 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;  // javadocs
+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,
@@ -371,19 +373,19 @@ public abstract class IndexReader implem
   }
 
   /** Retrieve term vectors for this document, or null if
-   *  term vectors were not indexed.  The returned InvertedFields
+   *  term vectors were not indexed.  The returned Fields
    *  instance acts like a single-document inverted index
    *  (the docID will be 0). */
-  public abstract InvertedFields getTermVectors(int docID)
+  public abstract Fields getTermVectors(int docID)
           throws IOException;
 
   /** Retrieve term vector for this document and field, or
    *  null if term vectors were not indexed.  The returned
-   *  InvertedFields instance acts like a single-document inverted
+   *  Fields instance acts like a single-document inverted
    *  index (the docID will be 0). */
   public final Terms getTermVector(int docID, String field)
     throws IOException {
-    InvertedFields vectors = getTermVectors(docID);
+    Fields vectors = getTermVectors(docID);
     if (vectors == null) {
       return null;
     }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java Sun Apr  8 13:38:01 2012
@@ -46,12 +46,12 @@ import org.apache.lucene.util.ReaderUtil
  * @lucene.experimental
  */
 
-public final class MultiFields extends InvertedFields {
-  private final InvertedFields[] subs;
+public final class MultiFields extends Fields {
+  private final Fields[] subs;
   private final ReaderUtil.Slice[] subSlices;
   private final Map<String,Terms> terms = new ConcurrentHashMap<String,Terms>();
 
-  /** Returns a single {@link InvertedFields} instance for this
+  /** Returns a single {@link Fields} instance for this
    *  reader, merging fields/terms/docs/positions on the
    *  fly.  This method will return null if the reader 
    *  has no postings.
@@ -60,7 +60,7 @@ public final class MultiFields extends I
    *  It's better to get the sub-readers (using {@link
    *  Gather}) and iterate through them
    *  yourself. */
-  public static InvertedFields getFields(IndexReader r) throws IOException {
+  public static Fields getFields(IndexReader r) throws IOException {
     if (r instanceof AtomicReader) {
       // already an atomic reader
       return ((AtomicReader) r).fields();
@@ -71,13 +71,13 @@ public final class MultiFields extends I
       // no fields
       return null;
     } else {
-      final List<InvertedFields> fields = new ArrayList<InvertedFields>();
+      final List<Fields> fields = new ArrayList<Fields>();
       final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
 
       new ReaderUtil.Gather(r) {
         @Override
         protected void add(int base, AtomicReader r) throws IOException {
-          final InvertedFields f = r.fields();
+          final Fields f = r.fields();
           if (f != null) {
             fields.add(f);
             slices.add(new ReaderUtil.Slice(base, r.maxDoc(), fields.size()-1));
@@ -90,7 +90,7 @@ public final class MultiFields extends I
       } else if (fields.size() == 1) {
         return fields.get(0);
       } else {
-        return new MultiFields(fields.toArray(InvertedFields.EMPTY_ARRAY),
+        return new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
                                        slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY));
       }
     }
@@ -130,7 +130,7 @@ public final class MultiFields extends I
 
   /**  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);
+    final Fields fields = getFields(r);
     if (fields == null) {
       return null;
     } else {
@@ -170,7 +170,7 @@ public final class MultiFields extends I
     return null;
   }
 
-  public MultiFields(InvertedFields[] subs, ReaderUtil.Slice[] subSlices) {
+  public MultiFields(Fields[] subs, ReaderUtil.Slice[] subSlices) {
     this.subs = subs;
     this.subSlices = subSlices;
   }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFieldsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFieldsEnum.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFieldsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFieldsEnum.java Sun Apr  8 13:38:01 2012
@@ -41,7 +41,7 @@ public final  class MultiFieldsEnum exte
 
   private int numTop;
 
-  private final InvertedFields fields;
+  private final Fields fields;
 
   private String currentField;
 
@@ -125,7 +125,7 @@ public final  class MultiFieldsEnum exte
     }
 
     @Override
-    protected boolean lessThan(FieldsEnumWithSlice fieldsA, FieldsEnumWithSlice fieldsB) {
+    protected final boolean lessThan(FieldsEnumWithSlice fieldsA, FieldsEnumWithSlice fieldsB) {
       // No need to break ties by field name: TermsEnum handles that
       return fieldsA.current.compareTo(fieldsB.current) < 0;
     }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java Sun Apr  8 13:38:01 2012
@@ -114,9 +114,9 @@ public final class ParallelAtomicReader 
       }
     }
     
-    // build InvertedFields instance
+    // build Fields instance
     for (final AtomicReader reader : this.parallelReaders) {
-      final InvertedFields readerFields = reader.fields();
+      final Fields readerFields = reader.fields();
       if (readerFields != null) {
         final FieldsEnum it = readerFields.iterator();
         String name;
@@ -176,7 +176,7 @@ public final class ParallelAtomicReader 
   }
   
   // Single instance of this, per ParallelReader instance
-  private final class ParallelFields extends InvertedFields {
+  private final class ParallelFields extends Fields {
     final Map<String,Terms> fields = new TreeMap<String,Terms>();
     
     ParallelFields() {
@@ -214,7 +214,7 @@ public final class ParallelAtomicReader 
   }
   
   @Override
-  public InvertedFields fields() {
+  public Fields fields() {
     ensureOpen();
     return fields;
   }
@@ -246,7 +246,7 @@ public final class ParallelAtomicReader 
   }
   
   @Override
-  public InvertedFields getTermVectors(int docID) throws IOException {
+  public Fields getTermVectors(int docID) throws IOException {
     ensureOpen();
     ParallelFields fields = null;
     for (Map.Entry<String,AtomicReader> ent : tvFieldToReader.entrySet()) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java Sun Apr  8 13:38:01 2012
@@ -24,7 +24,7 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.InvertedFieldsProducer;
+import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.PerDocProducer;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.StoredFieldsReader;
@@ -50,7 +50,7 @@ final class SegmentCoreReaders {
   
   final FieldInfos fieldInfos;
   
-  final InvertedFieldsProducer fields;
+  final FieldsProducer fields;
   final PerDocProducer perDocProducer;
   final PerDocProducer norms;
 
@@ -148,7 +148,7 @@ final class SegmentCoreReaders {
     }
   }
   
-  private void notifyCoreClosedListeners() {
+  private final void notifyCoreClosedListeners() {
     synchronized(coreClosedListeners) {
       for (CoreClosedListener listener : coreClosedListeners) {
         listener.onClose(owner);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java Sun Apr  8 13:38:01 2012
@@ -25,7 +25,7 @@ import java.util.Map;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.FieldInfosWriter;
-import org.apache.lucene.codecs.InvertedFieldsConsumer;
+import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.PerDocConsumer;
 import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.codecs.TermVectorsWriter;
@@ -334,14 +334,14 @@ final class SegmentMerger {
 
   private final void mergeTerms(SegmentWriteState segmentWriteState) throws CorruptIndexException, IOException {
     
-    final List<InvertedFields> fields = new ArrayList<InvertedFields>();
+    final List<Fields> fields = new ArrayList<Fields>();
     final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
 
     int docBase = 0;
 
     for(int readerIndex=0;readerIndex<mergeState.readers.size();readerIndex++) {
       final MergeState.IndexReaderAndLiveDocs r = mergeState.readers.get(readerIndex);
-      final InvertedFields f = r.reader.fields();
+      final Fields f = r.reader.fields();
       final int maxDoc = r.reader.maxDoc();
       if (f != null) {
         slices.add(new ReaderUtil.Slice(docBase, maxDoc, readerIndex));
@@ -350,11 +350,11 @@ final class SegmentMerger {
       docBase += maxDoc;
     }
 
-    final InvertedFieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState);
+    final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState);
     boolean success = false;
     try {
       consumer.merge(mergeState,
-                     new MultiFields(fields.toArray(InvertedFields.EMPTY_ARRAY),
+                     new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
                                      slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY)));
       success = true;
     } finally {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java Sun Apr  8 13:38:01 2012
@@ -137,7 +137,7 @@ public final class SegmentReader extends
   }
 
   @Override
-  public InvertedFields fields() throws IOException {
+  public Fields fields() throws IOException {
     ensureOpen();
     return core.fields;
   }
@@ -167,7 +167,7 @@ public final class SegmentReader extends
    * @throws IOException
    */
   @Override
-  public InvertedFields getTermVectors(int docID) throws IOException {
+  public Fields getTermVectors(int docID) throws IOException {
     TermVectorsReader termVectorsReader = getTermVectorsReader();
     if (termVectorsReader == null) {
       return null;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java Sun Apr  8 13:38:01 2012
@@ -24,6 +24,8 @@ import java.util.Map;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ReaderUtil; // javadoc
 
+import org.apache.lucene.index.DirectoryReader; // javadoc
+import org.apache.lucene.index.MultiReader; // javadoc
 
 /**
  * This class forces a composite reader (eg a {@link
@@ -45,7 +47,7 @@ public final class SlowCompositeReaderWr
 
   private final CompositeReader in;
   private final Map<String, DocValues> normsCache = new HashMap<String, DocValues>();
-  private final InvertedFields fields;
+  private final Fields fields;
   private final Bits liveDocs;
   
   /** This method is sugar for getting an {@link AtomicReader} from
@@ -75,7 +77,7 @@ public final class SlowCompositeReaderWr
   }
 
   @Override
-  public InvertedFields fields() throws IOException {
+  public Fields fields() throws IOException {
     ensureOpen();
     return fields;
   }
@@ -98,7 +100,7 @@ public final class SlowCompositeReaderWr
   }
   
   @Override
-  public InvertedFields getTermVectors(int docID)
+  public Fields getTermVectors(int docID)
           throws IOException {
     ensureOpen();
     return in.getTermVectors(docID);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java?rev=1310998&r1=1310997&r2=1310998&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java Sun Apr  8 13:38:01 2012
@@ -24,7 +24,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
 
 /**
- * Access to the terms in a specific field.  See {@link InvertedFields}.
+ * Access to the terms in a specific field.  See {@link Fields}.
  * @lucene.experimental
  */