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 12:09:23 UTC

svn commit: r1310969 [2/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...

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

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=1310969&r1=1310968&r2=1310969&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 10:09:19 2012
@@ -137,7 +137,7 @@ public final class SegmentReader extends
   }
 
   @Override
-  public Fields fields() throws IOException {
+  public InvertedFields fields() throws IOException {
     ensureOpen();
     return core.fields;
   }
@@ -167,7 +167,7 @@ public final class SegmentReader extends
    * @throws IOException
    */
   @Override
-  public Fields getTermVectors(int docID) throws IOException {
+  public InvertedFields getTermVectors(int docID) throws IOException {
     TermVectorsReader termVectorsReader = getTermVectorsReader();
     if (termVectorsReader == null) {
       return null;

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=1310969&r1=1310968&r2=1310969&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 10:09:19 2012
@@ -24,8 +24,6 @@ import java.util.Map;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ReaderUtil; // javadoc
 
-import org.apache.lucene.index.DirectoryReader; // javadoc
-import org.apache.lucene.index.MultiReader; // javadoc
 
 /**
  * This class forces a composite reader (eg a {@link
@@ -47,7 +45,7 @@ public final class SlowCompositeReaderWr
 
   private final CompositeReader in;
   private final Map<String, DocValues> normsCache = new HashMap<String, DocValues>();
-  private final Fields fields;
+  private final InvertedFields fields;
   private final Bits liveDocs;
   
   /** This method is sugar for getting an {@link AtomicReader} from
@@ -77,7 +75,7 @@ public final class SlowCompositeReaderWr
   }
 
   @Override
-  public Fields fields() throws IOException {
+  public InvertedFields fields() throws IOException {
     ensureOpen();
     return fields;
   }
@@ -100,7 +98,7 @@ public final class SlowCompositeReaderWr
   }
   
   @Override
-  public Fields getTermVectors(int docID)
+  public InvertedFields 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=1310969&r1=1310968&r2=1310969&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 10:09:19 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 Fields}.
+ * Access to the terms in a specific field.  See {@link InvertedFields}.
  * @lucene.experimental
  */
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java Sun Apr  8 10:09:19 2012
@@ -21,7 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.InvertedFields;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -85,7 +85,7 @@ public class MultiTermQueryWrapperFilter
   @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;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java Sun Apr  8 10:09:19 2012
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.util.Comparator;
 
 import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.InvertedFields;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.Term;
@@ -50,7 +50,7 @@ abstract class TermCollectingRewrite<Q e
     Comparator<BytesRef> lastTermComp = null;
     final AtomicReaderContext[] leaves = topReaderContext.leaves();
     for (AtomicReaderContext context : leaves) {
-      final Fields fields = context.reader().fields();
+      final InvertedFields fields = context.reader().fields();
       if (fields == null) {
         // reader has no fields
         continue;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java Sun Apr  8 10:09:19 2012
@@ -18,7 +18,7 @@ package org.apache.lucene.search.spans;
  */
 
 import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.InvertedFields;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.TermState;
@@ -93,7 +93,7 @@ public class SpanTermQuery extends SpanQ
     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) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/TermContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/TermContext.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/TermContext.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/TermContext.java Sun Apr  8 10:09:19 2012
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.InvertedFields;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.Term;
@@ -89,7 +89,7 @@ public final class TermContext {
     //if (DEBUG) System.out.println("prts.build term=" + term);
     for (int i = 0; i < leaves.length; i++) {
       //if (DEBUG) System.out.println("  r=" + leaves[i].reader);
-      final Fields fields = leaves[i].reader().fields();
+      final InvertedFields fields = leaves[i].reader().fields();
       if (fields != null) {
         final Terms terms = fields.terms(field);
         if (terms != null) {

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/appending/TestAppendingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/appending/TestAppendingCodec.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/appending/TestAppendingCodec.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/appending/TestAppendingCodec.java Sun Apr  8 10:09:19 2012
@@ -21,12 +21,11 @@ import java.io.IOException;
 import java.util.Random;
 
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.codecs.appending.AppendingCodec;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.InvertedFields;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
@@ -129,7 +128,7 @@ public class TestAppendingCodec extends 
     assertEquals(2, reader.numDocs());
     Document doc2 = reader.document(0);
     assertEquals(text, doc2.get("f"));
-    Fields fields = MultiFields.getFields(reader);
+    InvertedFields fields = MultiFields.getFields(reader);
     Terms terms = fields.terms("f");
     assertNotNull(terms);
     TermsEnum te = terms.iterator(null);

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/document/TestDocument.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/document/TestDocument.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/document/TestDocument.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/document/TestDocument.java Sun Apr  8 10:09:19 2012
@@ -21,10 +21,9 @@ import java.io.StringReader;
 
 import org.apache.lucene.analysis.EmptyTokenizer;
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.InvertedFields;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
@@ -342,7 +341,7 @@ public class TestDocument extends Lucene
     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());

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

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocCount.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocCount.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocCount.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocCount.java Sun Apr  8 10:09:19 2012
@@ -60,7 +60,7 @@ public class TestDocCount extends Lucene
   }
   
   private void verifyCount(IndexReader ir) throws Exception {
-    Fields fields = MultiFields.getFields(ir);
+    InvertedFields fields = MultiFields.getFields(ir);
     if (fields == null) {
       return;
     }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java Sun Apr  8 10:09:19 2012
@@ -157,10 +157,10 @@ public class TestDuelingCodecs extends L
   }
   
   /** 
-   * Fields api equivalency 
+   * InvertedFields api equivalency
    */
-  public void assertFields(Fields leftFields, Fields rightFields, boolean deep) throws Exception {
-    // Fields could be null if there are no postings,
+  public void assertFields(InvertedFields leftFields, InvertedFields rightFields, boolean deep) throws Exception {
+    // InvertedFields could be null if there are no postings,
     // but then it must be null for both
     if (leftFields == null || rightFields == null) {
       assertNull(info, leftFields);
@@ -181,9 +181,9 @@ public class TestDuelingCodecs extends L
   }
   
   /** 
-   * checks that top-level statistics on Fields are the same 
+   * checks that top-level statistics on InvertedFields are the same
    */
-  public void assertFieldStatistics(Fields leftFields, Fields rightFields) throws Exception {
+  public void assertFieldStatistics(InvertedFields leftFields, InvertedFields rightFields) throws Exception {
     if (leftFields.getUniqueFieldCount() != -1 && rightFields.getUniqueFieldCount() != -1) {
       assertEquals(info, leftFields.getUniqueFieldCount(), rightFields.getUniqueFieldCount());
     }
@@ -448,9 +448,9 @@ public class TestDuelingCodecs extends L
    * checks that norms are the same across all fields 
    */
   public void assertNorms(IndexReader leftReader, IndexReader rightReader) throws Exception {
-    Fields leftFields = MultiFields.getFields(leftReader);
-    Fields rightFields = MultiFields.getFields(rightReader);
-    // Fields could be null if there are no postings,
+    InvertedFields leftFields = MultiFields.getFields(leftReader);
+    InvertedFields rightFields = MultiFields.getFields(rightReader);
+    // InvertedFields could be null if there are no postings,
     // but then it must be null for both
     if (leftFields == null || rightFields == null) {
       assertNull(info, leftFields);
@@ -522,8 +522,8 @@ public class TestDuelingCodecs extends L
   public void assertTermVectors(IndexReader leftReader, IndexReader rightReader) throws Exception {
     assert leftReader.maxDoc() == rightReader.maxDoc();
     for (int i = 0; i < leftReader.maxDoc(); i++) {
-      Fields leftFields = leftReader.getTermVectors(i);
-      Fields rightFields = rightReader.getTermVectors(i);
+      InvertedFields leftFields = leftReader.getTermVectors(i);
+      InvertedFields rightFields = rightReader.getTermVectors(i);
       assertFields(leftFields, rightFields, rarely());
     }
   }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestFilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestFilterAtomicReader.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestFilterAtomicReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestFilterAtomicReader.java Sun Apr  8 10:09:19 2012
@@ -37,7 +37,7 @@ public class TestFilterAtomicReader exte
 
     /** Filter that only permits terms containing 'e'.*/
     private static class TestFields extends FilterFields {
-      TestFields(Fields in) {
+      TestFields(InvertedFields in) {
         super(in);
       }
       @Override
@@ -117,7 +117,7 @@ public class TestFilterAtomicReader exte
     }
 
     @Override
-    public Fields fields() throws IOException {
+    public InvertedFields fields() throws IOException {
       return new TestFields(super.fields());
     }
   }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java Sun Apr  8 10:09:19 2012
@@ -281,7 +281,7 @@ public class TestIndexableField extends 
             // TODO: offsets
             
           } else {
-            Fields vectors = r.getTermVectors(docID);
+            InvertedFields vectors = r.getTermVectors(docID);
             assertTrue(vectors == null || vectors.terms(name) == null);
           }
 

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java Sun Apr  8 10:09:19 2012
@@ -222,7 +222,7 @@ public class TestPerSegmentDeletes exten
 
   public static int[] toDocsArray(Term term, Bits bits, IndexReader reader)
       throws IOException {
-    Fields fields = MultiFields.getFields(reader);
+    InvertedFields fields = MultiFields.getFields(reader);
     Terms cterms = fields.terms(term.field);
     TermsEnum ctermsEnum = cterms.iterator(null);
     if (ctermsEnum.seekExact(new BytesRef(term.text()), false)) {

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java Sun Apr  8 10:09:19 2012
@@ -202,7 +202,7 @@ public class TestSegmentReader extends L
       assertTrue(freq > 0);
     }
 
-    Fields results = reader.getTermVectors(0);
+    InvertedFields results = reader.getTermVectors(0);
     assertTrue(results != null);
     assertEquals("We do not have 3 term freq vectors", 3, results.getUniqueFieldCount());      
   }    

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java Sun Apr  8 10:09:19 2012
@@ -313,7 +313,7 @@ public class TestStressIndexing2 extends
     int[] r2r1 = new int[r2.maxDoc()];   // r2 id to r1 id mapping
 
     // create mapping from id2 space to id2 based on idField
-    final Fields f1 = MultiFields.getFields(r1);
+    final InvertedFields f1 = MultiFields.getFields(r1);
     if (f1 == null) {
       // make sure r2 is empty
       assertNull(MultiFields.getFields(r2));
@@ -330,7 +330,7 @@ public class TestStressIndexing2 extends
     final Bits liveDocs1 = MultiFields.getLiveDocs(r1);
     final Bits liveDocs2 = MultiFields.getLiveDocs(r2);
     
-    Fields fields = MultiFields.getFields(r2);
+    InvertedFields fields = MultiFields.getFields(r2);
     if (fields == null) {
       // make sure r1 is in fact empty (eg has only all
       // deleted docs):
@@ -393,7 +393,7 @@ public class TestStressIndexing2 extends
         verifyEquals(r1.getTermVectors(id1), r2.getTermVectors(id2));
       } catch (Throwable e) {
         System.out.println("FAILED id=" + term + " id1=" + id1 + " id2=" + id2);
-        Fields tv1 = r1.getTermVectors(id1);
+        InvertedFields tv1 = r1.getTermVectors(id1);
         System.out.println("  d1=" + tv1);
         if (tv1 != null) {
           FieldsEnum fieldsEnum = tv1.iterator();
@@ -427,7 +427,7 @@ public class TestStressIndexing2 extends
           }
         }
         
-        Fields tv2 = r2.getTermVectors(id2);
+        InvertedFields tv2 = r2.getTermVectors(id2);
         System.out.println("  d2=" + tv2);
         if (tv2 != null) {
           FieldsEnum fieldsEnum = tv2.iterator();
@@ -598,7 +598,7 @@ public class TestStressIndexing2 extends
       }
     }
 
-  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;

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSumDocFreq.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSumDocFreq.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSumDocFreq.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSumDocFreq.java Sun Apr  8 10:09:19 2012
@@ -75,7 +75,7 @@ public class TestSumDocFreq extends Luce
   
   private void assertSumDocFreq(IndexReader ir) throws Exception {
     // compute sumDocFreq across all fields
-    Fields fields = MultiFields.getFields(ir);
+    InvertedFields fields = MultiFields.getFields(ir);
     FieldsEnum fieldEnum = fields.iterator();
     String f = null;
     while ((f = fieldEnum.next()) != null) {

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestMultiThreadTermVectors.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestMultiThreadTermVectors.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestMultiThreadTermVectors.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestMultiThreadTermVectors.java Sun Apr  8 10:09:19 2012
@@ -21,7 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.*;
-import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.InvertedFields;
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
@@ -154,7 +154,6 @@ class MultiThreadTermVectorsReader imple
       catch (Exception e) {
         e.printStackTrace();
       }
-      return;
   }
   
   private void testTermVectors() throws Exception {
@@ -163,7 +162,7 @@ class MultiThreadTermVectorsReader imple
     long start = 0L;
     for (int docId = 0; docId < numDocs; docId++) {
       start = System.currentTimeMillis();
-      Fields vectors = reader.getTermVectors(docId);
+      InvertedFields vectors = reader.getTermVectors(docId);
       timeElapsed += System.currentTimeMillis()-start;
       
       // verify vectors result
@@ -177,7 +176,7 @@ class MultiThreadTermVectorsReader imple
     }
   }
   
-  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();

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java Sun Apr  8 10:09:19 2012
@@ -96,7 +96,7 @@ public class TestTermVectors extends Luc
     assertEquals(100, hits.length);
       
     for (int i = 0; i < hits.length; i++) {
-      Fields vectors = searcher.reader.getTermVectors(hits[i].doc);
+      InvertedFields vectors = searcher.reader.getTermVectors(hits[i].doc);
       assertNotNull(vectors);
       assertEquals("doc=" + hits[i].doc + " tv=" + vectors, 1, vectors.getUniqueFieldCount());
     }
@@ -120,7 +120,7 @@ public class TestTermVectors extends Luc
     writer.addDocument(doc);
     IndexReader reader = writer.getReader();
     writer.close();
-    Fields v = reader.getTermVectors(0);
+    InvertedFields v = reader.getTermVectors(0);
     assertEquals(4, v.getUniqueFieldCount());
     String[] expectedFields = new String[]{"a", "b", "c", "x"};
     int[] expectedPositions = new int[]{1, 2, 0};
@@ -164,7 +164,7 @@ public class TestTermVectors extends Luc
 
     DocsAndPositionsEnum dpEnum = null;
     for (int i = 0; i < hits.length; i++) {
-      Fields vectors = searcher.reader.getTermVectors(hits[i].doc);
+      InvertedFields vectors = searcher.reader.getTermVectors(hits[i].doc);
       assertNotNull(vectors);
       assertEquals(1, vectors.getUniqueFieldCount());
       
@@ -203,7 +203,7 @@ public class TestTermVectors extends Luc
     assertEquals(100, hits.length);
       
     for (int i = 0; i < hits.length; i++) {
-      Fields vectors = searcher.reader.getTermVectors(hits[i].doc);
+      InvertedFields vectors = searcher.reader.getTermVectors(hits[i].doc);
       assertNotNull(vectors);
       assertEquals(1, vectors.getUniqueFieldCount());
     }
@@ -369,7 +369,7 @@ public class TestTermVectors extends Luc
     assertEquals(10, hits.length);
     for (int i = 0; i < hits.length; i++) {
 
-      Fields vectors = searcher.reader.getTermVectors(hits[i].doc);
+      InvertedFields vectors = searcher.reader.getTermVectors(hits[i].doc);
       assertNotNull(vectors);
       assertEquals(1, vectors.getUniqueFieldCount());
     }
@@ -416,7 +416,7 @@ public class TestTermVectors extends Luc
     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");

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWFieldsWriter.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWFieldsWriter.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWFieldsWriter.java Sun Apr  8 10:09:19 2012
@@ -20,7 +20,7 @@ package org.apache.lucene.codecs.lucene3
 import java.io.IOException;
 import java.util.Comparator;
 
-import org.apache.lucene.codecs.FieldsConsumer;
+import org.apache.lucene.codecs.InvertedFieldsConsumer;
 import org.apache.lucene.codecs.PostingsConsumer;
 import org.apache.lucene.codecs.TermStats;
 import org.apache.lucene.codecs.TermsConsumer;
@@ -34,7 +34,7 @@ import org.apache.lucene.store.IndexOutp
 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;

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWPostingsFormat.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWPostingsFormat.java Sun Apr  8 10:09:19 2012
@@ -19,8 +19,8 @@ package org.apache.lucene.codecs.lucene3
 
 import java.io.IOException;
 
-import org.apache.lucene.codecs.FieldsConsumer;
-import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.InvertedFieldsConsumer;
+import org.apache.lucene.codecs.InvertedFieldsProducer;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.util.LuceneTestCase;
@@ -38,12 +38,12 @@ class PreFlexRWPostingsFormat extends Lu
   }
   
   @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:

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java Sun Apr  8 10:09:19 2012
@@ -22,8 +22,8 @@ import java.util.Set;
 
 import org.apache.lucene.codecs.BlockTermsReader;
 import org.apache.lucene.codecs.BlockTermsWriter;
-import org.apache.lucene.codecs.FieldsConsumer;
-import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.InvertedFieldsConsumer;
+import org.apache.lucene.codecs.InvertedFieldsProducer;
 import org.apache.lucene.codecs.FixedGapTermsIndexReader;
 import org.apache.lucene.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.codecs.PostingsFormat;
@@ -48,7 +48,7 @@ public class Lucene40WithOrds extends Po
   }
 
   @Override
-  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+  public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     PostingsWriterBase docs = new Lucene40PostingsWriter(state);
 
     // TODO: should we make the terms index more easily
@@ -70,7 +70,7 @@ public class Lucene40WithOrds extends Po
     try {
       // Must use BlockTermsWriter (not BlockTree) because
       // BlockTree doens't support ords (yet)...
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, docs);
+      InvertedFieldsConsumer ret = new BlockTermsWriter(indexWriter, state, docs);
       success = true;
       return ret;
     } finally {
@@ -87,7 +87,7 @@ public class Lucene40WithOrds extends Po
   public final static int TERMS_CACHE_SIZE = 1024;
 
   @Override
-  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+  public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
     PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix);
     TermsIndexReaderBase indexReader;
 
@@ -108,7 +108,7 @@ public class Lucene40WithOrds extends Po
 
     success = false;
     try {
-      FieldsProducer ret = new BlockTermsReader(indexReader,
+      InvertedFieldsProducer ret = new BlockTermsReader(indexReader,
                                                 state.dir,
                                                 state.fieldInfos,
                                                 state.segmentInfo.name,

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java Sun Apr  8 10:09:19 2012
@@ -22,8 +22,8 @@ import java.util.Set;
 
 import org.apache.lucene.codecs.BlockTermsReader;
 import org.apache.lucene.codecs.BlockTermsWriter;
-import org.apache.lucene.codecs.FieldsConsumer;
-import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.InvertedFieldsConsumer;
+import org.apache.lucene.codecs.InvertedFieldsProducer;
 import org.apache.lucene.codecs.FixedGapTermsIndexReader;
 import org.apache.lucene.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.codecs.PostingsFormat;
@@ -124,7 +124,7 @@ public class MockFixedIntBlockPostingsFo
   }
 
   @Override
-  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+  public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     PostingsWriterBase postingsWriter = new SepPostingsWriter(state, new MockIntFactory(blockSize));
 
     boolean success = false;
@@ -140,7 +140,7 @@ public class MockFixedIntBlockPostingsFo
 
     success = false;
     try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
+      InvertedFieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
       success = true;
       return ret;
     } finally {
@@ -155,7 +155,7 @@ public class MockFixedIntBlockPostingsFo
   }
 
   @Override
-  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+  public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
     PostingsReaderBase postingsReader = new SepPostingsReader(state.dir,
                                                               state.segmentInfo,
                                                               state.context,
@@ -179,7 +179,7 @@ public class MockFixedIntBlockPostingsFo
 
     success = false;
     try {
-      FieldsProducer ret = new BlockTermsReader(indexReader,
+      InvertedFieldsProducer ret = new BlockTermsReader(indexReader,
                                                 state.dir,
                                                 state.fieldInfos,
                                                 state.segmentInfo.name,

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java Sun Apr  8 10:09:19 2012
@@ -22,8 +22,8 @@ import java.util.Set;
 
 import org.apache.lucene.codecs.BlockTermsReader;
 import org.apache.lucene.codecs.BlockTermsWriter;
-import org.apache.lucene.codecs.FieldsConsumer;
-import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.InvertedFieldsConsumer;
+import org.apache.lucene.codecs.InvertedFieldsProducer;
 import org.apache.lucene.codecs.FixedGapTermsIndexReader;
 import org.apache.lucene.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.codecs.PostingsFormat;
@@ -147,7 +147,7 @@ public class MockVariableIntBlockPosting
   }
 
   @Override
-  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+  public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     PostingsWriterBase postingsWriter = new SepPostingsWriter(state, new MockIntFactory(baseBlockSize));
 
     boolean success = false;
@@ -163,7 +163,7 @@ public class MockVariableIntBlockPosting
 
     success = false;
     try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
+      InvertedFieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
       success = true;
       return ret;
     } finally {
@@ -178,7 +178,7 @@ public class MockVariableIntBlockPosting
   }
 
   @Override
-  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+  public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
     PostingsReaderBase postingsReader = new SepPostingsReader(state.dir,
                                                               state.segmentInfo,
                                                               state.context,
@@ -202,7 +202,7 @@ public class MockVariableIntBlockPosting
 
     success = false;
     try {
-      FieldsProducer ret = new BlockTermsReader(indexReader,
+      InvertedFieldsProducer ret = new BlockTermsReader(indexReader,
                                                 state.dir,
                                                 state.fieldInfos,
                                                 state.segmentInfo.name,

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java Sun Apr  8 10:09:19 2012
@@ -28,8 +28,8 @@ import org.apache.lucene.codecs.BlockTer
 import org.apache.lucene.codecs.BlockTermsWriter;
 import org.apache.lucene.codecs.BlockTreeTermsReader;
 import org.apache.lucene.codecs.BlockTreeTermsWriter;
-import org.apache.lucene.codecs.FieldsConsumer;
-import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.InvertedFieldsConsumer;
+import org.apache.lucene.codecs.InvertedFieldsProducer;
 import org.apache.lucene.codecs.FixedGapTermsIndexReader;
 import org.apache.lucene.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.codecs.PostingsFormat;
@@ -126,7 +126,7 @@ public class MockRandomPostingsFormat ex
   }
 
   @Override
-  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+  public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     // we pull this before the seed intentionally: because its not consumed at runtime
     // (the skipInterval is written into postings header)
     int skipInterval = _TestUtil.nextInt(seedRandom, 2, 10);
@@ -171,7 +171,7 @@ public class MockRandomPostingsFormat ex
       postingsWriter = new PulsingPostingsWriter(totTFCutoff, postingsWriter);
     }
 
-    final FieldsConsumer fields;
+    final InvertedFieldsConsumer fields;
 
     if (random.nextBoolean()) {
       // Use BlockTree terms dict
@@ -270,7 +270,7 @@ public class MockRandomPostingsFormat ex
   }
 
   @Override
-  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+  public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
 
     final String seedFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, SEED_EXT);
     final IndexInput in = state.dir.openInput(seedFileName, state.context);
@@ -310,7 +310,7 @@ public class MockRandomPostingsFormat ex
       postingsReader = new PulsingPostingsReader(postingsReader);
     }
 
-    final FieldsProducer fields;
+    final InvertedFieldsProducer fields;
 
     if (random.nextBoolean()) {
       // Use BlockTree terms dict

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java Sun Apr  8 10:09:19 2012
@@ -22,8 +22,8 @@ import java.util.Set;
 
 import org.apache.lucene.codecs.BlockTermsReader;
 import org.apache.lucene.codecs.BlockTermsWriter;
-import org.apache.lucene.codecs.FieldsConsumer;
-import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.InvertedFieldsConsumer;
+import org.apache.lucene.codecs.InvertedFieldsProducer;
 import org.apache.lucene.codecs.FixedGapTermsIndexReader;
 import org.apache.lucene.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.codecs.PostingsFormat;
@@ -52,7 +52,7 @@ public class MockSepPostingsFormat exten
   }
 
   @Override
-  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+  public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
 
     PostingsWriterBase postingsWriter = new SepPostingsWriter(state, new MockSingleIntFactory());
 
@@ -69,7 +69,7 @@ public class MockSepPostingsFormat exten
 
     success = false;
     try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
+      InvertedFieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
       success = true;
       return ret;
     } finally {
@@ -84,7 +84,7 @@ public class MockSepPostingsFormat exten
   }
 
   @Override
-  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+  public InvertedFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
 
     PostingsReaderBase postingsReader = new SepPostingsReader(state.dir, state.segmentInfo,
         state.context, new MockSingleIntFactory(), state.segmentSuffix);
@@ -107,7 +107,7 @@ public class MockSepPostingsFormat exten
 
     success = false;
     try {
-      FieldsProducer ret = new BlockTermsReader(indexReader,
+      InvertedFieldsProducer ret = new BlockTermsReader(indexReader,
                                                 state.dir,
                                                 state.fieldInfos,
                                                 state.segmentInfo.name,

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java Sun Apr  8 10:09:19 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.FieldsConsumer;
-import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.InvertedFieldsConsumer;
+import org.apache.lucene.codecs.InvertedFieldsProducer;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.codecs.PostingsWriterBase;
@@ -48,7 +48,7 @@ public class NestedPulsingPostingsFormat
   }
   
   @Override
-  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+  public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     PostingsWriterBase docsWriter = new Lucene40PostingsWriter(state);
 
     PostingsWriterBase pulsingWriterInner = new PulsingPostingsWriter(2, docsWriter);
@@ -57,7 +57,7 @@ public class NestedPulsingPostingsFormat
     // Terms dict
     boolean success = false;
     try {
-      FieldsConsumer ret = new BlockTreeTermsWriter(state, pulsingWriter, 
+      InvertedFieldsConsumer ret = new BlockTreeTermsWriter(state, pulsingWriter,
           BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
       success = true;
       return ret;
@@ -69,13 +69,13 @@ public class NestedPulsingPostingsFormat
   }
 
   @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,

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java Sun Apr  8 10:09:19 2012
@@ -29,8 +29,8 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.lucene.codecs.FieldsConsumer;
-import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.InvertedFieldsConsumer;
+import org.apache.lucene.codecs.InvertedFieldsProducer;
 import org.apache.lucene.codecs.PostingsConsumer;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.TermStats;
@@ -100,7 +100,7 @@ public class RAMOnlyPostingsFormat exten
   }
     
   // Postings state:
-  static class RAMPostings extends FieldsProducer {
+  static class RAMPostings extends InvertedFieldsProducer {
     final Map<String,RAMField> fieldToTerms = new TreeMap<String,RAMField>();
 
     @Override
@@ -186,7 +186,7 @@ public class RAMOnlyPostingsFormat exten
   }
 
   // Classes for writing to the postings state
-  private static class RAMFieldsConsumer extends FieldsConsumer {
+  private static class RAMFieldsConsumer extends InvertedFieldsConsumer {
 
     private final RAMPostings postings;
     private final RAMTermsConsumer termsConsumer = new RAMTermsConsumer();
@@ -534,7 +534,7 @@ public class RAMOnlyPostingsFormat exten
   private static final String ID_EXTENSION = "id";
 
   @Override
-  public FieldsConsumer fieldsConsumer(SegmentWriteState writeState) throws IOException {
+  public InvertedFieldsConsumer fieldsConsumer(SegmentWriteState writeState) throws IOException {
     final int id = nextID.getAndIncrement();
 
     // TODO -- ok to do this up front instead of
@@ -565,7 +565,7 @@ public class RAMOnlyPostingsFormat exten
   }
 
   @Override
-  public FieldsProducer fieldsProducer(SegmentReadState readState)
+  public InvertedFieldsProducer fieldsProducer(SegmentReadState readState)
     throws IOException {
 
     // Load our ID:

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java Sun Apr  8 10:09:19 2012
@@ -20,8 +20,6 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.util.Set;
 
-import org.apache.lucene.index.FilterAtomicReader;
-
 public final class FieldFilterAtomicReader extends FilterAtomicReader {
   
   private final Set<String> fields;
@@ -50,8 +48,8 @@ public final class FieldFilterAtomicRead
   }
 
   @Override
-  public Fields getTermVectors(int docID) throws IOException {
-    Fields f = super.getTermVectors(docID);
+  public InvertedFields getTermVectors(int docID) throws IOException {
+    InvertedFields f = super.getTermVectors(docID);
     if (f == null) {
       return null;
     }
@@ -101,8 +99,8 @@ public final class FieldFilterAtomicRead
   }
 
   @Override
-  public Fields fields() throws IOException {
-    final Fields f = super.fields();
+  public InvertedFields fields() throws IOException {
+    final InvertedFields f = super.fields();
     return (f == null) ? null : new FieldFilterFields(f);
   }
 
@@ -125,7 +123,7 @@ public final class FieldFilterAtomicRead
   }
   
   private class FieldFilterFields extends FilterFields {
-    public FieldFilterFields(Fields in) {
+    public FieldFilterFields(InvertedFields in) {
       super(in);
     }
 

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java Sun Apr  8 10:09:19 2012
@@ -340,7 +340,7 @@ public abstract class ThreadedIndexingAn
                 try {
                   if (s.getIndexReader().numDocs() > 0) {
                     smokeTestSearcher(s);
-                    Fields fields = MultiFields.getFields(s.getIndexReader());
+                    InvertedFields fields = MultiFields.getFields(s.getIndexReader());
                     if (fields == null) {
                       continue;
                     }

Modified: lucene/dev/trunk/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java (original)
+++ lucene/dev/trunk/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java Sun Apr  8 10:09:19 2012
@@ -41,7 +41,7 @@ import org.apache.lucene.collation.Colla
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.InvertedFields;
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
@@ -750,7 +750,7 @@ public class TestPerfTasksLogic extends 
     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();

Modified: lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/TermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/TermsFilter.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/TermsFilter.java (original)
+++ lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/TermsFilter.java Sun Apr  8 10:09:19 2012
@@ -57,7 +57,7 @@ public class TermsFilter extends Filter 
   public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
     AtomicReader reader = context.reader();
     FixedBitSet result = new FixedBitSet(reader.maxDoc());
-    Fields fields = reader.fields();
+    InvertedFields fields = reader.fields();
 
     if (fields == null) {
       return result;

Modified: lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/function/valuesource/SumTotalTermFreqValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/function/valuesource/SumTotalTermFreqValueSource.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/function/valuesource/SumTotalTermFreqValueSource.java (original)
+++ lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/function/valuesource/SumTotalTermFreqValueSource.java Sun Apr  8 10:09:19 2012
@@ -18,13 +18,12 @@
 package org.apache.lucene.queries.function.valuesource;
 
 import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.InvertedFields;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.docvalues.LongDocValues;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.util.BytesRef;
 
 import java.io.IOException;
 import java.util.Map;
@@ -58,7 +57,7 @@ public class SumTotalTermFreqValueSource
   public void createWeight(Map context, IndexSearcher searcher) throws IOException {
     long sumTotalTermFreq = 0;
     for (AtomicReaderContext readerContext : searcher.getTopReaderContext().leaves()) {
-      Fields fields = readerContext.reader().fields();
+      InvertedFields fields = readerContext.reader().fields();
       if (fields == null) continue;
       Terms terms = fields.terms(indexedField);
       if (terms == null) continue;

Modified: lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java (original)
+++ lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java Sun Apr  8 10:09:19 2012
@@ -40,7 +40,7 @@ public class TFValueSource extends TermF
 
   @Override
   public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
-    Fields fields = readerContext.reader().fields();
+    InvertedFields fields = readerContext.reader().fields();
     final Terms terms = fields.terms(field);
     IndexSearcher searcher = (IndexSearcher)context.get("searcher");
     final TFIDFSimilarity similarity = IDFValueSource.asTFIDF(searcher.getSimilarity(), field);

Modified: lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java (original)
+++ lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java Sun Apr  8 10:09:19 2012
@@ -38,7 +38,7 @@ public class TermFreqValueSource extends
 
   @Override
   public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
-    Fields fields = readerContext.reader().fields();
+    InvertedFields fields = readerContext.reader().fields();
     final Terms terms = fields.terms(field);
 
     return new IntDocValues(this) {

Modified: lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/mlt/MoreLikeThis.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/mlt/MoreLikeThis.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/mlt/MoreLikeThis.java (original)
+++ lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/mlt/MoreLikeThis.java Sun Apr  8 10:09:19 2012
@@ -22,7 +22,7 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.InvertedFields;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.MultiFields;
@@ -705,7 +705,7 @@ public final class MoreLikeThis {
   public PriorityQueue<Object[]> retrieveTerms(int docNum) throws IOException {
     Map<String, Int> termFreqMap = new HashMap<String, Int>();
     for (String fieldName : fieldNames) {
-      final Fields vectors = ir.getTermVectors(docNum);
+      final InvertedFields vectors = ir.getTermVectors(docNum);
       final Terms vector;
       if (vectors != null) {
         vector = vectors.terms(fieldName);

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java Sun Apr  8 10:09:19 2012
@@ -575,7 +575,7 @@ public class LukeRequestHandler extends 
 
     final CharsRef spare = new CharsRef();
 
-    Fields fields = MultiFields.getFields(req.getSearcher().getIndexReader());
+    InvertedFields fields = MultiFields.getFields(req.getSearcher().getIndexReader());
 
     if (fields == null) { // No indexed fields
       return;

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java Sun Apr  8 10:09:19 2012
@@ -532,7 +532,7 @@ public class QueryElevationComponent ext
       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;

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java Sun Apr  8 10:09:19 2012
@@ -10,7 +10,7 @@ import java.util.Map;
 
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.InvertedFields;
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.StoredFieldVisitor;
@@ -251,7 +251,7 @@ public class TermVectorComponent extends
         }
       } 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) {

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java Sun Apr  8 10:09:19 2012
@@ -118,7 +118,7 @@ public class TermsComponent extends Sear
 
 
     final AtomicReader indexReader = rb.req.getSearcher().getAtomicReader();
-    Fields lfields = indexReader.fields();
+    InvertedFields lfields = indexReader.fields();
 
     for (String field : fields) {
       NamedList<Integer> fieldTerms = new NamedList<Integer>();

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/SimpleFacets.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/SimpleFacets.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/SimpleFacets.java Sun Apr  8 10:09:19 2012
@@ -669,7 +669,7 @@ public class SimpleFacets {
       startTermBytes = new BytesRef(indexedPrefix);
     }
 
-    Fields fields = r.fields();
+    InvertedFields fields = r.fields();
     Terms terms = fields==null ? null : fields.terms(field);
     TermsEnum termsEnum = null;
     SolrIndexSearcher.DocsEnumState deState = null;

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java Sun Apr  8 10:09:19 2012
@@ -262,8 +262,8 @@ class JoinQuery extends Query {
         fastForRandomSet = new HashDocSet(sset.getDocs(), 0, sset.size());
       }
 
-      Fields fromFields = fromSearcher.getAtomicReader().fields();
-      Fields toFields = fromSearcher==toSearcher ? fromFields : toSearcher.getAtomicReader().fields();
+      InvertedFields fromFields = fromSearcher.getAtomicReader().fields();
+      InvertedFields toFields = fromSearcher==toSearcher ? fromFields : toSearcher.getAtomicReader().fields();
       if (fromFields == null) return DocSet.EMPTY;
       Terms terms = fromFields.terms(fromField);
       Terms toTerms = toFields.terms(toField);

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java Sun Apr  8 10:09:19 2012
@@ -34,14 +34,12 @@ import org.apache.lucene.document.LongFi
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.*;
-import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.search.*;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.OpenBitSet;
-import org.apache.lucene.util.ReaderUtil;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
@@ -584,7 +582,7 @@ public class SolrIndexSearcher extends I
    * @return the first document number containing the term
    */
   public int getFirstMatch(Term t) throws IOException {
-    Fields fields = atomicReader.fields();
+    InvertedFields fields = atomicReader.fields();
     if (fields == null) return -1;
     Terms terms = fields.terms(t.field());
     if (terms == null) return -1;
@@ -612,7 +610,7 @@ public class SolrIndexSearcher extends I
       final AtomicReaderContext leaf = leaves[i];
       final AtomicReader reader = leaf.reader();
 
-      final Fields fields = reader.fields();
+      final InvertedFields fields = reader.fields();
       if (fields == null) continue;
 
       final Bits liveDocs = reader.getLiveDocs();
@@ -998,7 +996,7 @@ public class SolrIndexSearcher extends I
           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();
           

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestDocSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestDocSet.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestDocSet.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestDocSet.java Sun Apr  8 10:09:19 2012
@@ -23,8 +23,8 @@ import java.util.Random;
 
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.InvertedFields;
 import org.apache.lucene.index.StoredFieldVisitor;
-import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.AtomicReaderContext;
@@ -368,12 +368,12 @@ public class TestDocSet extends LuceneTe
       }
 
       @Override
-      public Fields fields() {
+      public InvertedFields fields() {
         return null;
       }
 
       @Override
-      public Fields getTermVectors(int doc) {
+      public InvertedFields getTermVectors(int doc) {
         return null;
       }
 

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java?rev=1310969&r1=1310968&r2=1310969&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java Sun Apr  8 10:09:19 2012
@@ -28,8 +28,6 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.noggit.ObjectBuilder;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.update.UpdateHandler;
 import org.apache.solr.update.UpdateLog;
@@ -1698,7 +1696,7 @@ public class TestRealTimeGet extends Sol
 
 
   public int getFirstMatch(IndexReader r, Term t) throws IOException {
-    Fields fields = MultiFields.getFields(r);
+    InvertedFields fields = MultiFields.getFields(r);
     if (fields == null) return -1;
     Terms terms = fields.terms(t.field());
     if (terms == null) return -1;