You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2011/10/25 17:47:46 UTC

svn commit: r1188738 [1/2] - in /lucene/dev/branches/lucene2621: lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/ lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/ lucene/src/java/org/apache/lucene/index/ luce...

Author: rmuir
Date: Tue Oct 25 15:47:44 2011
New Revision: 1188738

URL: http://svn.apache.org/viewvc?rev=1188738&view=rev
Log:
LUCENE-3490: Codec -> PostingsFormat

Added:
    lucene/dev/branches/lucene2621/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingPostingsFormat.java
      - copied, changed from r1188713, lucene/dev/branches/lucene2621/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingCodec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsFormat.java
      - copied, changed from r1188713, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryPostingsFormat.java
      - copied, changed from r1188713, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldPostingsFormat.java
      - copied, changed from r1188714, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldCodecWrapper.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexPostingsFormat.java
      - copied, changed from r1188713, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsFormat.java
      - copied, changed from r1188713, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextPostingsFormat.java
      - copied, changed from r1188713, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsFormat.java
      - copied, changed from r1188713, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java
      - copied, changed from r1188713, lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java
      - copied, changed from r1188713, lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java
      - copied, changed from r1188713, lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java
      - copied, changed from r1188713, lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexRWPostingsFormat.java
      - copied, changed from r1188713, lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexRWCodec.java
Removed:
    lucene/dev/branches/lucene2621/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingCodec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldCodecWrapper.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexRWCodec.java
Modified:
    lucene/dev/branches/lucene2621/lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexFileNames.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocConsumer.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentCodecs.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/ValueType.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodecProvider.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/TestExternalCodecs.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestCodecs.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/codecs/intblock/TestIntBlockCodec.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/codecs/preflex/TestSurrogates.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/codecs/pulsing/Test10KPulsings.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/codecs/pulsing/TestPulsingReuse.java
    lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/core/SchemaCodecProvider.java
    lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/core/SolrConfig.java
    lucene/dev/branches/lucene2621/solr/core/src/test/org/apache/solr/core/MockCodecProviderFactory.java
    lucene/dev/branches/lucene2621/solr/core/src/test/org/apache/solr/core/TestCodecProviderSupport.java

Copied: lucene/dev/branches/lucene2621/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingPostingsFormat.java (from r1188713, lucene/dev/branches/lucene2621/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingCodec.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingPostingsFormat.java?p2=lucene/dev/branches/lucene2621/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingPostingsFormat.java&p1=lucene/dev/branches/lucene2621/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingCodec.java&r1=1188713&r2=1188738&rev=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingPostingsFormat.java Tue Oct 25 15:47:44 2011
@@ -24,7 +24,7 @@ import org.apache.lucene.index.PerDocWri
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
@@ -32,7 +32,7 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.index.codecs.PerDocConsumer;
 import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
 import org.apache.lucene.index.codecs.PerDocValues;
-import org.apache.lucene.index.codecs.standard.StandardCodec;
+import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
 import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
 import org.apache.lucene.index.codecs.PostingsWriterBase;
@@ -43,7 +43,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.util.BytesRef;
 
 /**
- * This codec extends {@link StandardCodec} to work on append-only outputs, such
+ * This codec extends {@link StandardPostingsFormat} to work on append-only outputs, such
  * as plain output streams and append-only filesystems.
  *
  * <p>Note: compound file format feature is not compatible with
@@ -53,10 +53,10 @@ import org.apache.lucene.util.BytesRef;
  * compound file format.</p>
  * @lucene.experimental
  */
-public class AppendingCodec extends Codec {
+public class AppendingPostingsFormat extends PostingsFormat {
   public static String CODEC_NAME = "Appending";
   
-  public AppendingCodec() {
+  public AppendingPostingsFormat() {
     super(CODEC_NAME);
   }
 
@@ -116,7 +116,7 @@ public class AppendingCodec extends Code
               state.dir, state.fieldInfos, state.segmentInfo.name,
               docsReader,
               state.context,
-              StandardCodec.TERMS_CACHE_SIZE,
+              StandardPostingsFormat.TERMS_CACHE_SIZE,
               state.codecId);
       success = true;
       return ret;
@@ -142,7 +142,7 @@ public class AppendingCodec extends Code
 
   @Override
   public void getExtensions(Set<String> extensions) {
-    StandardCodec.getStandardExtensions(extensions);
+    StandardPostingsFormat.getStandardExtensions(extensions);
     DefaultDocValuesConsumer.getExtensions(extensions);
   }
   

Modified: lucene/dev/branches/lucene2621/lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java Tue Oct 25 15:47:44 2011
@@ -34,7 +34,7 @@ import org.apache.lucene.index.MultiFiel
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.TermsEnum.SeekStatus;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.SegmentInfosReader;
 import org.apache.lucene.index.codecs.SegmentInfosWriter;
@@ -50,14 +50,14 @@ import org.apache.lucene.util.Version;
 public class TestAppendingCodec extends LuceneTestCase {
   
   static class AppendingCodecProvider extends CodecProvider {
-    Codec appending = new AppendingCodec();
+    PostingsFormat appending = new AppendingPostingsFormat();
     SegmentInfosWriter infosWriter = new AppendingSegmentInfosWriter();
     SegmentInfosReader infosReader = new AppendingSegmentInfosReader();
     public AppendingCodecProvider() {
       setDefaultFieldCodec(appending.name);
     }
     @Override
-    public Codec lookup(String name) {
+    public PostingsFormat lookup(String name) {
       return appending;
     }
    

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java Tue Oct 25 15:47:44 2011
@@ -25,7 +25,7 @@ import java.util.HashSet;
 import java.util.Map;
 
 import org.apache.lucene.index.DocumentsWriterPerThread.DocState;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.DocValuesConsumer;
 import org.apache.lucene.index.codecs.PerDocConsumer;
 import org.apache.lucene.index.codecs.perfield.SegmentCodecs;
@@ -326,7 +326,7 @@ final class DocFieldProcessor extends Do
       PerDocWriteState perDocWriteState = docState.docWriter.newPerDocWriteState(fieldInfo.getCodecId());
       SegmentCodecs codecs = perDocWriteState.segmentCodecs;
       assert codecs.codecs.length > fieldInfo.getCodecId();
-      Codec codec = codecs.codecs[fieldInfo.getCodecId()];
+      PostingsFormat codec = codecs.codecs[fieldInfo.getCodecId()];
       perDocConsumer = codec.docsConsumer(perDocWriteState);
       perDocConsumers.put(Integer.valueOf(fieldInfo.getCodecId()), perDocConsumer);
     }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexFileNames.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexFileNames.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexFileNames.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexFileNames.java Tue Oct 25 15:47:44 2011
@@ -19,7 +19,7 @@ package org.apache.lucene.index;
 
 import java.util.regex.Pattern;
 
-import org.apache.lucene.index.codecs.Codec;  // for javadocs
+import org.apache.lucene.index.codecs.PostingsFormat;  // for javadocs
 
 /**
  * This class contains useful constants representing filenames and extensions
@@ -31,7 +31,7 @@ import org.apache.lucene.index.codecs.Co
  * {@link #segmentFileName(String, String, String) segmentFileName}).
  *
  * <p><b>NOTE</b>: extensions used by codecs are not
- * listed here.  You must interact with the {@link Codec}
+ * listed here.  You must interact with the {@link PostingsFormat}
  * directly.
  *
  * @lucene.internal

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java Tue Oct 25 15:47:44 2011
@@ -28,7 +28,7 @@ import java.util.concurrent.atomic.Atomi
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DocumentStoredFieldVisitor;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.index.values.IndexDocValues;
@@ -764,7 +764,7 @@ public abstract class IndexReader implem
    * the index.
    * 
    * @param directory where the index resides.
-   * @param codecs the {@link CodecProvider} holding all {@link Codec}s required to open the index
+   * @param codecs the {@link CodecProvider} holding all {@link PostingsFormat}s required to open the index
    * @return version number.
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
@@ -800,7 +800,7 @@ public abstract class IndexReader implem
    * this index.
    * 
    * @param directory where the index resides.
-   * @param codecs the {@link CodecProvider} provider holding all {@link Codec}s required to open the index
+   * @param codecs the {@link CodecProvider} provider holding all {@link PostingsFormat}s required to open the index
    * @return commit userData.
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java Tue Oct 25 15:47:44 2011
@@ -20,7 +20,7 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.codecs.FieldsReader;
 import org.apache.lucene.index.codecs.PerDocValues;
@@ -86,7 +86,7 @@ final class SegmentCoreReaders {
       fieldInfos = si.getFieldInfos();
       
       this.termsIndexDivisor = termsIndexDivisor;
-      final Codec codec = segmentCodecs.codec();
+      final PostingsFormat codec = segmentCodecs.codec();
       final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si, fieldInfos, context, termsIndexDivisor);
       // Ask codec for its Fields
       fields = codec.fieldsProducer(segmentReadState);

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentInfo.java Tue Oct 25 15:47:44 2011
@@ -27,7 +27,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
 import org.apache.lucene.index.codecs.perfield.SegmentCodecs;
@@ -228,7 +228,7 @@ public final class SegmentInfo implement
     } else {
       // codec ID on FieldInfo is 0 so it will simply use the first codec available
       // TODO what todo if preflex is not available in the provider? register it or fail?
-      segmentCodecs = new SegmentCodecs(codecs, new Codec[] { codecs.lookup("PreFlex")});
+      segmentCodecs = new SegmentCodecs(codecs, new PostingsFormat[] { codecs.lookup("PreFlex")});
     }
     diagnostics = input.readStringStringMap();
 

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Tue Oct 25 15:47:44 2011
@@ -27,7 +27,7 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.IndexReader.FieldOption;
 import org.apache.lucene.index.MergePolicy.MergeAbortedException;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsReader;
 import org.apache.lucene.index.codecs.FieldsWriter;
@@ -67,7 +67,7 @@ final class SegmentMerger {
       when merging stored fields */
   private final static int MAX_RAW_MERGE_DOCS = 4192;
 
-  private Codec codec;
+  private PostingsFormat codec;
   private SegmentWriteState segmentWriteState;
 
   private PayloadProcessorProvider payloadProcessorProvider;

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java Tue Oct 25 15:47:44 2011
@@ -43,14 +43,14 @@ public class CodecProvider {
   private final Map<String, String> perFieldMap = new HashMap<String, String>();
 
   
-  private final HashMap<String, Codec> codecs = new HashMap<String, Codec>();
+  private final HashMap<String, PostingsFormat> codecs = new HashMap<String, PostingsFormat>();
 
   private final Set<String> knownExtensions = new HashSet<String>();
 
 
   public final static String[] CORE_CODECS = new String[] {"Standard", "Pulsing", "PreFlex", "SimpleText", "Memory"};
 
-  public synchronized void register(Codec codec) {
+  public synchronized void register(PostingsFormat codec) {
     if (codec.name == null) {
       throw new IllegalArgumentException("code.name is null");
     }
@@ -63,12 +63,12 @@ public class CodecProvider {
   }
   
   /** @lucene.internal */
-  public synchronized void unregister(Codec codec) {
+  public synchronized void unregister(PostingsFormat codec) {
     if (codec.name == null) {
       throw new IllegalArgumentException("code.name is null");
     }
     if (codecs.containsKey(codec.name)) {
-      Codec c = codecs.get(codec.name);
+      PostingsFormat c = codecs.get(codec.name);
       if (codec == c) {
         codecs.remove(codec.name);
       } else {
@@ -86,8 +86,8 @@ public class CodecProvider {
     return knownExtensions;
   }
 
-  public synchronized Codec lookup(String name) {
-    final Codec codec = codecs.get(name);
+  public synchronized PostingsFormat lookup(String name) {
+    final PostingsFormat codec = codecs.get(name);
     if (codec == null) {
       throw new IllegalArgumentException("required codec '" + name + "' not found; known codecs: " + codecs.keySet());
     }
@@ -134,7 +134,7 @@ public class CodecProvider {
   }
   
   /**
-   * Sets the {@link Codec} for a given field. Not that setting a field's codec is
+   * Sets the {@link PostingsFormat} for a given field. Not that setting a field's codec is
    * write-once. If the field's codec is already set this method will throw an
    * {@link IllegalArgumentException}.
    * 
@@ -154,12 +154,12 @@ public class CodecProvider {
   }
 
   /**
-   * Returns the {@link Codec} name for the given field or the default codec if
+   * Returns the {@link PostingsFormat} name for the given field or the default codec if
    * not set.
    * 
    * @param name
    *          the fields name
-   * @return the {@link Codec} name for the given field or the default codec if
+   * @return the {@link PostingsFormat} name for the given field or the default codec if
    *         not set.
    */
   public synchronized String getFieldCodec(String name) {
@@ -179,16 +179,16 @@ public class CodecProvider {
   }
 
   /**
-   * Returns the default {@link Codec} for this {@link CodecProvider}
+   * Returns the default {@link PostingsFormat} for this {@link CodecProvider}
    * 
-   * @return the default {@link Codec} for this {@link CodecProvider}
+   * @return the default {@link PostingsFormat} for this {@link CodecProvider}
    */
   public synchronized String getDefaultFieldCodec() {
     return defaultFieldCodec;
   }
 
   /**
-   * Sets the default {@link Codec} for this {@link CodecProvider}
+   * Sets the default {@link PostingsFormat} for this {@link CodecProvider}
    * 
    * @param codec
    *          the codecs name

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java Tue Oct 25 15:47:44 2011
@@ -17,11 +17,11 @@ package org.apache.lucene.index.codecs;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.codecs.memory.MemoryCodec;
-import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
-import org.apache.lucene.index.codecs.pulsing.PulsingCodec;
-import org.apache.lucene.index.codecs.simpletext.SimpleTextCodec;
-import org.apache.lucene.index.codecs.standard.StandardCodec;
+import org.apache.lucene.index.codecs.memory.MemoryPostingsFormat;
+import org.apache.lucene.index.codecs.preflex.PreFlexPostingsFormat;
+import org.apache.lucene.index.codecs.pulsing.PulsingPostingsFormat;
+import org.apache.lucene.index.codecs.simpletext.SimpleTextPostingsFormat;
+import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
 
 /**
  * A CodecProvider that registers all core codecs that ship
@@ -42,10 +42,10 @@ import org.apache.lucene.index.codecs.st
 
 public class CoreCodecProvider extends CodecProvider {
   public CoreCodecProvider() {
-    register(new StandardCodec());
-    register(new PreFlexCodec());
-    register(new PulsingCodec());
-    register(new SimpleTextCodec());
-    register(new MemoryCodec());
+    register(new StandardPostingsFormat());
+    register(new PreFlexPostingsFormat());
+    register(new PulsingPostingsFormat());
+    register(new SimpleTextPostingsFormat());
+    register(new MemoryPostingsFormat());
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocConsumer.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocConsumer.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocConsumer.java Tue Oct 25 15:47:44 2011
@@ -29,7 +29,7 @@ import org.apache.lucene.index.values.Va
  * this convert field values into a Codec specific format during indexing.
  * <p>
  * The {@link PerDocConsumer} API is accessible through the
- * {@link Codec} - API providing per field consumers and producers for inverted
+ * {@link PostingsFormat} - API providing per field consumers and producers for inverted
  * data (terms, postings) as well as per-document data.
  * 
  * @lucene.experimental

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java Tue Oct 25 15:47:44 2011
@@ -28,7 +28,7 @@ import org.apache.lucene.index.values.In
  * {@link PerDocConsumer} counterpart.
  * <p>
  * The {@link PerDocValues} API is accessible through the
- * {@link Codec} - API providing per field consumers and producers for inverted
+ * {@link PostingsFormat} - API providing per field consumers and producers for inverted
  * data (terms, postings) as well as per-document data.
  * 
  * @lucene.experimental

Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsFormat.java (from r1188713, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsFormat.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsFormat.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java&r1=1188713&r2=1188738&rev=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsFormat.java Tue Oct 25 15:47:44 2011
@@ -27,13 +27,13 @@ import org.apache.lucene.index.SegmentRe
 import org.apache.lucene.store.Directory;
 
 /** @lucene.experimental */
-public abstract class Codec {
-  public static final Codec[] EMPTY = new Codec[0];
+public abstract class PostingsFormat {
+  public static final PostingsFormat[] EMPTY = new PostingsFormat[0];
   /** Unique name that's used to retrieve this codec when
    *  reading the index */
   public final String name;
   
-  protected Codec(String name) {
+  protected PostingsFormat(String name) {
     this.name = name;
   }
 

Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryPostingsFormat.java (from r1188713, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryPostingsFormat.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryPostingsFormat.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java&r1=1188713&r2=1188738&rev=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryPostingsFormat.java Tue Oct 25 15:47:44 2011
@@ -37,7 +37,7 @@ import org.apache.lucene.index.SegmentRe
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
 import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
 import org.apache.lucene.index.codecs.FieldsConsumer;
@@ -81,9 +81,12 @@ import org.apache.lucene.util.fst.FST;
  *
  * @lucene.experimental */
 
-public class MemoryCodec extends Codec {
+// TODO: Maybe name this 'Cached' or something to reflect
+// the reality that it is actually written to disk, but
+// loads itself in ram?
+public class MemoryPostingsFormat extends PostingsFormat {
   
-  public MemoryCodec() {
+  public MemoryPostingsFormat() {
     super("Memory");
   }
 

Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldPostingsFormat.java (from r1188714, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldCodecWrapper.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldPostingsFormat.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldPostingsFormat.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldCodecWrapper.java&r1=1188714&r2=1188738&rev=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldCodecWrapper.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldPostingsFormat.java Tue Oct 25 15:47:44 2011
@@ -36,7 +36,7 @@ import org.apache.lucene.index.SegmentRe
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.codecs.PerDocConsumer;
@@ -57,10 +57,10 @@ import org.apache.lucene.util.IOUtils;
  * 
  * @lucene.internal
  */
-final class PerFieldCodecWrapper extends Codec {
+final class PerFieldPostingsFormat extends PostingsFormat {
   private final SegmentCodecs segmentCodecs;
 
-  PerFieldCodecWrapper(SegmentCodecs segmentCodecs) {
+  PerFieldPostingsFormat(SegmentCodecs segmentCodecs) {
     super("PerField");
     this.segmentCodecs = segmentCodecs;
   }
@@ -76,7 +76,7 @@ final class PerFieldCodecWrapper extends
 
     public FieldsWriter(SegmentWriteState state) throws IOException {
       assert segmentCodecs == state.segmentCodecs;
-      final Codec[] codecs = segmentCodecs.codecs;
+      final PostingsFormat[] codecs = segmentCodecs.codecs;
       for (int i = 0; i < codecs.length; i++) {
         boolean success = false;
         try {
@@ -111,14 +111,14 @@ final class PerFieldCodecWrapper extends
     public FieldsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo si,
         IOContext context, int indexDivisor) throws IOException {
 
-      final Map<Codec, FieldsProducer> producers = new HashMap<Codec, FieldsProducer>();
+      final Map<PostingsFormat, FieldsProducer> producers = new HashMap<PostingsFormat, FieldsProducer>();
       boolean success = false;
       try {
         for (FieldInfo fi : fieldInfos) {
           if (fi.isIndexed) { 
             fields.add(fi.name);
             assert fi.getCodecId() != FieldInfo.UNASSIGNED_CODEC_ID;
-            Codec codec = segmentCodecs.codecs[fi.getCodecId()];
+            PostingsFormat codec = segmentCodecs.codecs[fi.getCodecId()];
             if (!producers.containsKey(codec)) {
               producers.put(codec, codec.fieldsProducer(new SegmentReadState(dir,
                                                                              si, fieldInfos, context, indexDivisor, fi.getCodecId())));
@@ -201,7 +201,7 @@ final class PerFieldCodecWrapper extends
 
   @Override
   public void getExtensions(Set<String> extensions) {
-    for (Codec codec : segmentCodecs.codecs) {
+    for (PostingsFormat codec : segmentCodecs.codecs) {
       codec.getExtensions(extensions);
     }
   }
@@ -222,13 +222,13 @@ final class PerFieldCodecWrapper extends
 
     public PerDocProducers(Directory dir, FieldInfos fieldInfos, SegmentInfo si,
         IOContext context, int indexDivisor) throws IOException {
-      final Map<Codec, PerDocValues> producers = new HashMap<Codec, PerDocValues>();
+      final Map<PostingsFormat, PerDocValues> producers = new HashMap<PostingsFormat, PerDocValues>();
       boolean success = false;
       try {
         for (FieldInfo fi : fieldInfos) {
           if (fi.hasDocValues()) { 
             assert fi.getCodecId() != FieldInfo.UNASSIGNED_CODEC_ID;
-            Codec codec = segmentCodecs.codecs[fi.getCodecId()];
+            PostingsFormat codec = segmentCodecs.codecs[fi.getCodecId()];
             if (!producers.containsKey(codec)) {
               producers.put(codec, codec.docsProducer(new SegmentReadState(dir,
                 si, fieldInfos, context, indexDivisor, fi.getCodecId())));
@@ -264,7 +264,7 @@ final class PerFieldCodecWrapper extends
   
   private final class PerDocConsumers extends PerDocConsumer {
     private final PerDocConsumer[] consumers;
-    private final Codec[] codecs;
+    private final PostingsFormat[] codecs;
     private final PerDocWriteState state;
 
     public PerDocConsumers(PerDocWriteState state) throws IOException {

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentCodecs.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentCodecs.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentCodecs.java Tue Oct 25 15:47:44 2011
@@ -27,9 +27,9 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.CodecProvider;
-import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
+import org.apache.lucene.index.codecs.preflex.PreFlexPostingsFormat;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
@@ -49,7 +49,7 @@ import org.apache.lucene.store.IndexOutp
  * {@link FieldInfos#buildSegmentCodecs(boolean)} The {@link FieldInfo#codecId}
  * assigned by {@link SegmentCodecsBuilder} refers to the codecs ordinal
  * maintained inside {@link SegmentCodecs}. This ord is later used to get the
- * right codec when the segment is opened in a reader.The {@link Codec} returned
+ * right codec when the segment is opened in a reader.The {@link PostingsFormat} returned
  * from {@link SegmentCodecs#codec()} in turn uses {@link SegmentCodecs}
  * internal structure to select and initialize the right codec for a fields when
  * it is written.
@@ -68,49 +68,49 @@ public final class SegmentCodecs impleme
    * internal structure to map codecs to fields - don't modify this from outside
    * of this class!
    */
-  public final Codec[] codecs;
+  public final PostingsFormat[] codecs;
   public final CodecProvider provider;
-  private final Codec codec;
+  private final PostingsFormat codec;
   
   public SegmentCodecs(CodecProvider provider, IndexInput input) throws IOException {
     this(provider, read(input, provider));
   }
   
-  public SegmentCodecs(CodecProvider provider, Codec... codecs) {
+  public SegmentCodecs(CodecProvider provider, PostingsFormat... codecs) {
     this.provider = provider;
     this.codecs = codecs;
-    if (codecs.length == 1 && codecs[0] instanceof PreFlexCodec) {
+    if (codecs.length == 1 && codecs[0] instanceof PreFlexPostingsFormat) {
       this.codec = codecs[0]; // hack for backwards break... don't wrap the codec in preflex
     } else {
-      this.codec = new PerFieldCodecWrapper(this);
+      this.codec = new PerFieldPostingsFormat(this);
     }
   }
 
-  public Codec codec() {
+  public PostingsFormat codec() {
     return codec;
   }
 
   public void write(IndexOutput out) throws IOException {
     out.writeVInt(codecs.length);
-    for (Codec codec : codecs) {
+    for (PostingsFormat codec : codecs) {
       out.writeString(codec.name);
     }
   }
 
-  private static Codec[] read(IndexInput in, CodecProvider provider) throws IOException {
+  private static PostingsFormat[] read(IndexInput in, CodecProvider provider) throws IOException {
     final int size = in.readVInt();
-    final ArrayList<Codec> list = new ArrayList<Codec>();
+    final ArrayList<PostingsFormat> list = new ArrayList<PostingsFormat>();
     for (int i = 0; i < size; i++) {
       final String codecName = in.readString();
-      final Codec lookup = provider.lookup(codecName);
+      final PostingsFormat lookup = provider.lookup(codecName);
       list.add(i, lookup);
     }
-    return list.toArray(Codec.EMPTY);
+    return list.toArray(PostingsFormat.EMPTY);
   }
 
   public void files(Directory dir, SegmentInfo info, Set<String> files)
       throws IOException {
-    final Codec[] codecArray = codecs;
+    final PostingsFormat[] codecArray = codecs;
     for (int i = 0; i < codecArray.length; i++) {
       codecArray[i].files(dir, info, i, files);
     }      
@@ -131,8 +131,8 @@ public final class SegmentCodecs impleme
    * @see FieldInfo#getCodecId()
    */
   public final static class SegmentCodecsBuilder {
-    private final Map<Codec, Integer> codecRegistry = new IdentityHashMap<Codec, Integer>();
-    private final ArrayList<Codec> codecs = new ArrayList<Codec>();
+    private final Map<PostingsFormat, Integer> codecRegistry = new IdentityHashMap<PostingsFormat, Integer>();
+    private final ArrayList<PostingsFormat> codecs = new ArrayList<PostingsFormat>();
     private final CodecProvider provider;
 
     private SegmentCodecsBuilder(CodecProvider provider) {
@@ -145,7 +145,7 @@ public final class SegmentCodecs impleme
     
     public SegmentCodecsBuilder tryAddAndSet(FieldInfo fi) {
       if (fi.getCodecId() == FieldInfo.UNASSIGNED_CODEC_ID) {
-        final Codec fieldCodec = provider.lookup(provider
+        final PostingsFormat fieldCodec = provider.lookup(provider
             .getFieldCodec(fi.name));
         Integer ord = codecRegistry.get(fieldCodec);
         if (ord == null) {
@@ -159,7 +159,7 @@ public final class SegmentCodecs impleme
     }
     
     public SegmentCodecs build() {
-      return new SegmentCodecs(provider, codecs.toArray(Codec.EMPTY));
+      return new SegmentCodecs(provider, codecs.toArray(PostingsFormat.EMPTY));
     }
     
     public SegmentCodecsBuilder clear() {

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java Tue Oct 25 15:47:44 2011
@@ -95,7 +95,7 @@ public class PreFlexFields extends Field
 
       // make sure that all index files have been read or are kept open
       // so that if an index update removes them we'll still have them
-      freqStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.FREQ_EXTENSION), context);
+      freqStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.FREQ_EXTENSION), context);
       boolean anyProx = false;
       for (FieldInfo fi : fieldInfos) {
         if (fi.isIndexed) {
@@ -108,7 +108,7 @@ public class PreFlexFields extends Field
       }
 
       if (anyProx) {
-        proxStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.PROX_EXTENSION), context);
+        proxStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.PROX_EXTENSION), context);
       } else {
         proxStream = null;
       }
@@ -136,16 +136,16 @@ public class PreFlexFields extends Field
   }
 
   static void files(Directory dir, SegmentInfo info, Collection<String> files) throws IOException {
-    files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.TERMS_EXTENSION));
-    files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.TERMS_INDEX_EXTENSION));
-    files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.FREQ_EXTENSION));
+    files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.TERMS_EXTENSION));
+    files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.TERMS_INDEX_EXTENSION));
+    files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.FREQ_EXTENSION));
     if (info.getHasProx()) {
       // LUCENE-1739: for certain versions of 2.9-dev,
       // hasProx would be incorrectly computed during
       // indexing as true, and then stored into the segments
       // file, when it should have been false.  So we do the
       // extra check, here:
-      final String prx = IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.PROX_EXTENSION);
+      final String prx = IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.PROX_EXTENSION);
       if (dir.fileExists(prx)) {
         files.add(prx);
       }

Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexPostingsFormat.java (from r1188713, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexPostingsFormat.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexPostingsFormat.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java&r1=1188713&r2=1188738&rev=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexPostingsFormat.java Tue Oct 25 15:47:44 2011
@@ -21,7 +21,7 @@ import java.util.Set;
 import java.io.IOException;
 
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.PerDocWriteState;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentWriteState;
@@ -40,7 +40,7 @@ import org.apache.lucene.index.codecs.Pe
  * @lucene.experimental
  */
 @Deprecated
-public class PreFlexCodec extends Codec {
+public class PreFlexPostingsFormat extends PostingsFormat {
 
   /** Extension of terms file */
   public static final String TERMS_EXTENSION = "tis";
@@ -54,7 +54,7 @@ public class PreFlexCodec extends Codec 
   /** Extension of prox postings file */
   public static final String PROX_EXTENSION = "prx";
 
-  public PreFlexCodec() {
+  public PreFlexPostingsFormat() {
     super("PreFlex");
   }
   

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java Tue Oct 25 15:47:44 2011
@@ -110,7 +110,7 @@ public final class TermInfosReader {
       segment = seg;
       fieldInfos = fis;
 
-      origEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", PreFlexCodec.TERMS_EXTENSION),
+      origEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", PreFlexPostingsFormat.TERMS_EXTENSION),
                                                          context), fieldInfos, false);
       size = origEnum.size;
 
@@ -118,7 +118,7 @@ public final class TermInfosReader {
       if (indexDivisor != -1) {
         // Load terms index
         totalIndexInterval = origEnum.indexInterval * indexDivisor;
-        final SegmentTermEnum indexEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", PreFlexCodec.TERMS_INDEX_EXTENSION),
+        final SegmentTermEnum indexEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", PreFlexPostingsFormat.TERMS_INDEX_EXTENSION),
                                                                                   context), fieldInfos, true);
 
         try {

Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsFormat.java (from r1188713, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsFormat.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsFormat.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java&r1=1188713&r2=1188738&rev=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsFormat.java Tue Oct 25 15:47:44 2011
@@ -28,14 +28,14 @@ import org.apache.lucene.index.codecs.Po
 import org.apache.lucene.index.codecs.PostingsWriterBase;
 import org.apache.lucene.index.codecs.BlockTreeTermsReader;
 import org.apache.lucene.index.codecs.BlockTreeTermsWriter;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
 import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.codecs.PerDocConsumer;
 import org.apache.lucene.index.codecs.PerDocValues;
-import org.apache.lucene.index.codecs.standard.StandardCodec;
+import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
 import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
 import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
 import org.apache.lucene.store.Directory;
@@ -48,23 +48,23 @@ import org.apache.lucene.store.Directory
  *  otherwise uses the normal "standard" codec. 
  *  @lucene.experimental */
 
-public class PulsingCodec extends Codec {
+public class PulsingPostingsFormat extends PostingsFormat {
 
   private final int freqCutoff;
   private final int minBlockSize;
   private final int maxBlockSize;
 
-  public PulsingCodec() {
+  public PulsingPostingsFormat() {
     this(1);
   }
   
-  public PulsingCodec(int freqCutoff) {
+  public PulsingPostingsFormat(int freqCutoff) {
     this(freqCutoff, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
   }
 
   /** Terms with freq <= freqCutoff are inlined into terms
    *  dict. */
-  public PulsingCodec(int freqCutoff, int minBlockSize, int maxBlockSize) {
+  public PulsingPostingsFormat(int freqCutoff, int minBlockSize, int maxBlockSize) {
     super("Pulsing");
     this.freqCutoff = freqCutoff;
     this.minBlockSize = minBlockSize;
@@ -139,7 +139,7 @@ public class PulsingCodec extends Codec 
 
   @Override
   public void getExtensions(Set<String> extensions) {
-    StandardCodec.getStandardExtensions(extensions);
+    StandardPostingsFormat.getStandardExtensions(extensions);
     DefaultDocValuesConsumer.getExtensions(extensions);
   }
 

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java Tue Oct 25 15:47:44 2011
@@ -60,7 +60,7 @@ class SimpleTextFieldsReader extends Fie
   final static BytesRef PAYLOAD = SimpleTextFieldsWriter.PAYLOAD;
 
   public SimpleTextFieldsReader(SegmentReadState state) throws IOException {
-    in = state.dir.openInput(SimpleTextCodec.getPostingsFileName(state.segmentInfo.name, state.codecId), state.context);
+    in = state.dir.openInput(SimpleTextPostingsFormat.getPostingsFileName(state.segmentInfo.name, state.codecId), state.context);
    
     fieldInfos = state.fieldInfos;
   }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java Tue Oct 25 15:47:44 2011
@@ -47,7 +47,7 @@ class SimpleTextFieldsWriter extends Fie
   final static BytesRef PAYLOAD = new BytesRef("        payload ");
 
   public SimpleTextFieldsWriter(SegmentWriteState state) throws IOException {
-    final String fileName = SimpleTextCodec.getPostingsFileName(state.segmentName, state.codecId);
+    final String fileName = SimpleTextPostingsFormat.getPostingsFileName(state.segmentName, state.codecId);
     out = state.directory.createOutput(fileName, state.context);
   }
 

Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextPostingsFormat.java (from r1188713, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextPostingsFormat.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextPostingsFormat.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java&r1=1188713&r2=1188738&rev=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextPostingsFormat.java Tue Oct 25 15:47:44 2011
@@ -25,7 +25,7 @@ import org.apache.lucene.index.SegmentIn
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
@@ -42,9 +42,9 @@ import org.apache.lucene.store.Directory
  *  any text editor, and even edit it to alter your index.
  *
  *  @lucene.experimental */
-public class SimpleTextCodec extends Codec {
+public class SimpleTextPostingsFormat extends PostingsFormat {
   
-  public SimpleTextCodec() {
+  public SimpleTextPostingsFormat() {
     super("SimpleText");
   }
 

Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsFormat.java (from r1188713, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsFormat.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsFormat.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java&r1=1188713&r2=1188738&rev=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsFormat.java Tue Oct 25 15:47:44 2011
@@ -28,7 +28,7 @@ import org.apache.lucene.index.codecs.Po
 import org.apache.lucene.index.codecs.PostingsWriterBase;
 import org.apache.lucene.index.codecs.BlockTreeTermsReader;
 import org.apache.lucene.index.codecs.BlockTreeTermsWriter;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
 import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
 import org.apache.lucene.index.codecs.FieldsConsumer;
@@ -39,16 +39,16 @@ import org.apache.lucene.store.Directory
 
 /** Default codec. 
  *  @lucene.experimental */
-public class StandardCodec extends Codec {
+public class StandardPostingsFormat extends PostingsFormat {
 
   private final int minBlockSize;
   private final int maxBlockSize;
 
-  public StandardCodec() {
+  public StandardPostingsFormat() {
     this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
   }
 
-  public StandardCodec(int minBlockSize, int maxBlockSize) {
+  public StandardPostingsFormat(int minBlockSize, int maxBlockSize) {
     super("Standard");
     this.minBlockSize = minBlockSize;
     assert minBlockSize > 1;

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java Tue Oct 25 15:47:44 2011
@@ -55,13 +55,13 @@ public class StandardPostingsReader exte
   // private String segment;
 
   public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext ioContext, int codecId) throws IOException {
-    freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.FREQ_EXTENSION),
+    freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardPostingsFormat.FREQ_EXTENSION),
                            ioContext);
     // this.segment = segmentInfo.name;
     if (segmentInfo.getHasProx()) {
       boolean success = false;
       try {
-        proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.PROX_EXTENSION),
+        proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardPostingsFormat.PROX_EXTENSION),
                                ioContext);
         success = true;
       } finally {
@@ -75,9 +75,9 @@ public class StandardPostingsReader exte
   }
 
   public static void files(Directory dir, SegmentInfo segmentInfo, int codecID, Collection<String> files) throws IOException {
-    files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardCodec.FREQ_EXTENSION));
+    files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardPostingsFormat.FREQ_EXTENSION));
     if (segmentInfo.getHasProx()) {
-      files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardCodec.PROX_EXTENSION));
+      files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardPostingsFormat.PROX_EXTENSION));
     }
   }
 

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java Tue Oct 25 15:47:44 2011
@@ -90,12 +90,12 @@ public final class StandardPostingsWrite
     this.skipInterval = skipInterval;
     this.skipMinimum = skipInterval; /* set to the same for now */
     // this.segment = state.segmentName;
-    String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.FREQ_EXTENSION);
+    String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardPostingsFormat.FREQ_EXTENSION);
     freqOut = state.directory.createOutput(fileName, state.context);
     if (state.fieldInfos.hasProx()) {
       // At least one field does not omit TF, so create the
       // prox file
-      fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.PROX_EXTENSION);
+      fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardPostingsFormat.PROX_EXTENSION);
       proxOut = state.directory.createOutput(fileName, state.context);
     } else {
       // Every field omits TF so we will write no prox file

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java Tue Oct 25 15:47:44 2011
@@ -24,7 +24,7 @@ import org.apache.lucene.document.IndexD
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.util.BytesRef;
 
@@ -40,13 +40,13 @@ import org.apache.lucene.util.BytesRef;
  * performance {@link IndexDocValues} should be consumed per-segment just like
  * IndexReader.
  * <p>
- * {@link IndexDocValues} are fully integrated into the {@link Codec} API.
+ * {@link IndexDocValues} are fully integrated into the {@link PostingsFormat} API.
  * Custom implementations can be exposed on a per field basis via
  * {@link CodecProvider}.
  * 
  * @see ValueType for limitations and default implementation documentation
  * @see IndexDocValuesField for adding values to the index
- * @see Codec#docsConsumer(org.apache.lucene.index.PerDocWriteState) for
+ * @see PostingsFormat#docsConsumer(org.apache.lucene.index.PerDocWriteState) for
  *      customization
  * @lucene.experimental
  */

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/ValueType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/ValueType.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/ValueType.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/ValueType.java Tue Oct 25 15:47:44 2011
@@ -17,7 +17,7 @@ package org.apache.lucene.index.values;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.values.IndexDocValues.SortedSource;
 import org.apache.lucene.index.values.IndexDocValues.Source;
 import org.apache.lucene.util.BytesRef;
@@ -27,7 +27,7 @@ import org.apache.lucene.util.packed.Pac
  * <code>ValueType</code> specifies the {@link IndexDocValues} type for a
  * certain field. A <code>ValueType</code> only defines the data type for a field
  * while the actual implementation used to encode and decode the values depends
- * on the the {@link Codec#docsConsumer} and {@link Codec#docsProducer} methods.
+ * on the the {@link PostingsFormat#docsConsumer} and {@link PostingsFormat#docsProducer} methods.
  * 
  * @lucene.experimental
  */

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodecProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodecProvider.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodecProvider.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodecProvider.java Tue Oct 25 15:47:44 2011
@@ -25,13 +25,13 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Random;
 
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.CodecProvider;
-import org.apache.lucene.index.codecs.memory.MemoryCodec;
-import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
-import org.apache.lucene.index.codecs.pulsing.PulsingCodec;
-import org.apache.lucene.index.codecs.simpletext.SimpleTextCodec;
-import org.apache.lucene.index.codecs.standard.StandardCodec;
+import org.apache.lucene.index.codecs.memory.MemoryPostingsFormat;
+import org.apache.lucene.index.codecs.preflex.PreFlexPostingsFormat;
+import org.apache.lucene.index.codecs.pulsing.PulsingPostingsFormat;
+import org.apache.lucene.index.codecs.simpletext.SimpleTextPostingsFormat;
+import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
 import org.apache.lucene.util._TestUtil;
 
 /**
@@ -44,8 +44,8 @@ import org.apache.lucene.util._TestUtil;
  * and reproducable.
  */
 public class RandomCodecProvider extends CodecProvider {
-  private List<Codec> knownCodecs = new ArrayList<Codec>();
-  private Map<String,Codec> previousMappings = new HashMap<String,Codec>();
+  private List<PostingsFormat> knownCodecs = new ArrayList<PostingsFormat>();
+  private Map<String,PostingsFormat> previousMappings = new HashMap<String,PostingsFormat>();
   private final int perFieldSeed;
   
   public RandomCodecProvider(Random random, boolean useNoMemoryExpensiveCodec) {
@@ -54,39 +54,39 @@ public class RandomCodecProvider extends
     // block via CL:
     int minItemsPerBlock = _TestUtil.nextInt(random, 2, 100);
     int maxItemsPerBlock = 2*(Math.max(2, minItemsPerBlock-1)) + random.nextInt(100);
-    register(new StandardCodec(minItemsPerBlock, maxItemsPerBlock));
-    register(new PreFlexCodec());
+    register(new StandardPostingsFormat(minItemsPerBlock, maxItemsPerBlock));
+    register(new PreFlexPostingsFormat());
     // TODO: make it possible to specify min/max iterms per
     // block via CL:
     minItemsPerBlock = _TestUtil.nextInt(random, 2, 100);
     maxItemsPerBlock = 2*(Math.max(1, minItemsPerBlock-1)) + random.nextInt(100);
-    register(new PulsingCodec( 1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock));
+    register(new PulsingPostingsFormat( 1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock));
     if (!useNoMemoryExpensiveCodec) {
-      register(new SimpleTextCodec());
-      register(new MemoryCodec());
+      register(new SimpleTextPostingsFormat());
+      register(new MemoryPostingsFormat());
     }
     Collections.shuffle(knownCodecs, random);
   }
   
   @Override
-  public synchronized void register(Codec codec) {
+  public synchronized void register(PostingsFormat codec) {
     if (!codec.name.equals("PreFlex"))
       knownCodecs.add(codec);
     super.register(codec);
   }
   
   @Override
-  public synchronized void unregister(Codec codec) {
+  public synchronized void unregister(PostingsFormat codec) {
     knownCodecs.remove(codec);
     super.unregister(codec);
   }
   
   @Override
   public synchronized String getFieldCodec(String name) {
-    Codec codec = previousMappings.get(name);
+    PostingsFormat codec = previousMappings.get(name);
     if (codec == null) {
       codec = knownCodecs.get(Math.abs(perFieldSeed ^ name.hashCode()) % knownCodecs.size());
-      if (codec instanceof SimpleTextCodec && perFieldSeed % 5 != 0) {
+      if (codec instanceof SimpleTextPostingsFormat && perFieldSeed % 5 != 0) {
         // make simpletext rarer, choose again
         codec = knownCodecs.get(Math.abs(perFieldSeed ^ name.toUpperCase(Locale.ENGLISH).hashCode()) % knownCodecs.size());
       }

Copied: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java (from r1188713, lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java?p2=lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java&p1=lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java&r1=1188713&r2=1188738&rev=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java Tue Oct 25 15:47:44 2011
@@ -24,7 +24,7 @@ import org.apache.lucene.index.PerDocWri
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.codecs.sep.IntStreamFactory;
@@ -34,7 +34,7 @@ import org.apache.lucene.index.codecs.se
 import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
 import org.apache.lucene.index.codecs.sep.SepPostingsReader;
 import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
-import org.apache.lucene.index.codecs.standard.StandardCodec;
+import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
 import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput;
 import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexOutput;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
@@ -57,11 +57,11 @@ import org.apache.lucene.util.IOUtils;
  * used here just writes each block as a series of vInt.
  */
 
-public class MockFixedIntBlockCodec extends Codec {
+public class MockFixedIntBlockPostingsFormat extends PostingsFormat {
 
   private final int blockSize;
 
-  public MockFixedIntBlockCodec(int blockSize) {
+  public MockFixedIntBlockPostingsFormat(int blockSize) {
     super("MockFixedIntBlock");
     this.blockSize = blockSize;
   }
@@ -187,7 +187,7 @@ public class MockFixedIntBlockCodec exte
                                                 state.segmentInfo.name,
                                                 postingsReader,
                                                 state.context,
-                                                StandardCodec.TERMS_CACHE_SIZE,
+                                                StandardPostingsFormat.TERMS_CACHE_SIZE,
                                                 state.codecId);
       success = true;
       return ret;

Copied: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java (from r1188713, lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java?p2=lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java&p1=lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java&r1=1188713&r2=1188738&rev=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java Tue Oct 25 15:47:44 2011
@@ -24,7 +24,7 @@ import org.apache.lucene.index.PerDocWri
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.codecs.sep.IntStreamFactory;
@@ -34,7 +34,7 @@ import org.apache.lucene.index.codecs.se
 import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
 import org.apache.lucene.index.codecs.sep.SepPostingsReader;
 import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
-import org.apache.lucene.index.codecs.standard.StandardCodec;
+import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
 import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexInput;
 import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexOutput;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
@@ -61,10 +61,10 @@ import org.apache.lucene.util.IOUtils;
  * int is <= 3, else 2*baseBlockSize.
  */
 
-public class MockVariableIntBlockCodec extends Codec {
+public class MockVariableIntBlockPostingsFormat extends PostingsFormat {
   private final int baseBlockSize;
   
-  public MockVariableIntBlockCodec(int baseBlockSize) {
+  public MockVariableIntBlockPostingsFormat(int baseBlockSize) {
     super("MockVariableIntBlock");
     this.baseBlockSize = baseBlockSize;
   }
@@ -210,7 +210,7 @@ public class MockVariableIntBlockCodec e
                                                 state.segmentInfo.name,
                                                 postingsReader,
                                                 state.context,
-                                                StandardCodec.TERMS_CACHE_SIZE,
+                                                StandardPostingsFormat.TERMS_CACHE_SIZE,
                                                 state.codecId);
       success = true;
       return ret;

Copied: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java (from r1188713, lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java?p2=lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java&p1=lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java&r1=1188713&r2=1188738&rev=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java Tue Oct 25 15:47:44 2011
@@ -34,7 +34,7 @@ import org.apache.lucene.index.codecs.Bl
 import org.apache.lucene.index.codecs.BlockTreeTermsWriter;
 import org.apache.lucene.index.codecs.BlockTermsReader;
 import org.apache.lucene.index.codecs.BlockTermsWriter;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
@@ -50,8 +50,8 @@ import org.apache.lucene.index.codecs.Te
 import org.apache.lucene.index.codecs.TermsIndexWriterBase;
 import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
 import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
-import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockCodec;
-import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockCodec;
+import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
+import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
 import org.apache.lucene.index.codecs.mocksep.MockSingleIntFactory;
 import org.apache.lucene.index.codecs.pulsing.PulsingPostingsReader;
 import org.apache.lucene.index.codecs.pulsing.PulsingPostingsWriter;
@@ -76,12 +76,12 @@ import org.apache.lucene.util._TestUtil;
  * Randomly combines terms index impl w/ postings impls.
  */
 
-public class MockRandomCodec extends Codec {
+public class MockRandomPostingsFormat extends PostingsFormat {
   private final boolean useSepDocValues;
   private final Random seedRandom;
   private final String SEED_EXT = "sd";
   
-  public MockRandomCodec(Random random) {
+  public MockRandomPostingsFormat(Random random) {
     super("MockRandom");
     this.useSepDocValues = random.nextBoolean();
     this.seedRandom = new Random(random.nextLong());
@@ -96,9 +96,9 @@ public class MockRandomCodec extends Cod
       salt = random.nextInt();
       delegates.add(new MockSingleIntFactory());
       final int blockSize = _TestUtil.nextInt(random, 1, 2000);
-      delegates.add(new MockFixedIntBlockCodec.MockIntFactory(blockSize));
+      delegates.add(new MockFixedIntBlockPostingsFormat.MockIntFactory(blockSize));
       final int baseBlockSize = _TestUtil.nextInt(random, 1, 127);
-      delegates.add(new MockVariableIntBlockCodec.MockIntFactory(baseBlockSize));
+      delegates.add(new MockVariableIntBlockPostingsFormat.MockIntFactory(baseBlockSize));
       // TODO: others
     }
 

Copied: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java (from r1188713, lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java?p2=lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java&p1=lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java&r1=1188713&r2=1188738&rev=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java Tue Oct 25 15:47:44 2011
@@ -24,7 +24,7 @@ import org.apache.lucene.index.PerDocWri
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
@@ -37,7 +37,7 @@ import org.apache.lucene.index.codecs.Bl
 import org.apache.lucene.index.codecs.BlockTermsWriter;
 import org.apache.lucene.index.codecs.TermsIndexReaderBase;
 import org.apache.lucene.index.codecs.TermsIndexWriterBase;
-import org.apache.lucene.index.codecs.standard.StandardCodec;
+import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
 import org.apache.lucene.index.codecs.sep.SepDocValuesConsumer;
 import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
 import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
@@ -51,9 +51,9 @@ import org.apache.lucene.util.BytesRef;
  * This is here just to test the core sep codec
  * classes.
  */
-public class MockSepCodec extends Codec {
+public class MockSepPostingsFormat extends PostingsFormat {
 
-  public MockSepCodec() {
+  public MockSepPostingsFormat() {
     super("MockSep");
   }
 
@@ -119,7 +119,7 @@ public class MockSepCodec extends Codec 
                                                 state.segmentInfo.name,
                                                 postingsReader,
                                                 state.context,
-                                                StandardCodec.TERMS_CACHE_SIZE,
+                                                StandardPostingsFormat.TERMS_CACHE_SIZE,
                                                 state.codecId);
       success = true;
       return ret;

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java?rev=1188738&r1=1188737&r2=1188738&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java Tue Oct 25 15:47:44 2011
@@ -29,7 +29,7 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.index.codecs.PostingsConsumer;
 import org.apache.lucene.index.codecs.TermStats;
 import org.apache.lucene.index.codecs.TermsConsumer;
-import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
+import org.apache.lucene.index.codecs.preflex.PreFlexPostingsFormat;
 import org.apache.lucene.index.codecs.preflex.TermInfo;
 import org.apache.lucene.index.codecs.standard.DefaultSkipListWriter;
 import org.apache.lucene.store.IndexOutput;
@@ -50,12 +50,12 @@ class PreFlexFieldsWriter extends Fields
                                    state.fieldInfos,
                                    state.termIndexInterval);
 
-    final String freqFile = IndexFileNames.segmentFileName(state.segmentName, "", PreFlexCodec.FREQ_EXTENSION);
+    final String freqFile = IndexFileNames.segmentFileName(state.segmentName, "", PreFlexPostingsFormat.FREQ_EXTENSION);
     freqOut = state.directory.createOutput(freqFile, state.context);
     totalNumDocs = state.numDocs;
 
     if (state.fieldInfos.hasProx()) {
-      final String proxFile = IndexFileNames.segmentFileName(state.segmentName, "", PreFlexCodec.PROX_EXTENSION);
+      final String proxFile = IndexFileNames.segmentFileName(state.segmentName, "", PreFlexPostingsFormat.PROX_EXTENSION);
       proxOut = state.directory.createOutput(proxFile, state.context);
     } else {
       proxOut = null;