Index: lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java
===================================================================
--- lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java	(revision 1032543)
+++ lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java	(working copy)
@@ -21,7 +21,6 @@
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Field.Index;
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.Field.TermVector;
@@ -32,7 +31,6 @@
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.LogMergePolicy;
 import org.apache.lucene.index.MultiFields;
-import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.TermsEnum.SeekStatus;
@@ -54,16 +52,15 @@
     Codec appending = new AppendingCodec();
     SegmentInfosWriter infosWriter = new AppendingSegmentInfosWriter();
     SegmentInfosReader infosReader = new AppendingSegmentInfosReader();
-    
+    public AppendingCodecProvider() {
+      setDefaultFieldCodec(appending.name);
+    }
     @Override
     public Codec lookup(String name) {
       return appending;
     }
+   
     @Override
-    public Codec getWriter(SegmentWriteState state) {
-      return appending;
-    }
-    @Override
     public SegmentInfosReader getSegmentInfosReader() {
       return infosReader;
     }
Index: lucene/src/java/org/apache/lucene/document/AbstractField.java
===================================================================
--- lucene/src/java/org/apache/lucene/document/AbstractField.java	(revision 1032543)
+++ lucene/src/java/org/apache/lucene/document/AbstractField.java	(working copy)
@@ -19,6 +19,7 @@
 import org.apache.lucene.search.spans.SpanQuery; // for javadocs
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.index.FieldInvertState;
+import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.util.StringHelper; // for javadocs
 
 
@@ -292,4 +293,20 @@
     result.append('>');
     return result.toString();
   }
+  
+  public String getCodec() {
+    return getCodec(CodecProvider.getDefault());
+  }
+  
+  public String getCodec(CodecProvider provider) {
+    return provider.getFieldCodec(name);
+  }
+
+  public void setCodec(String codec) {
+    setCodec(CodecProvider.getDefault(), codec);
+  }
+  
+  public void setCodec(CodecProvider provider, String codec) {
+    provider.setFieldCodec(name, codec);
+  }
 }
Index: lucene/src/java/org/apache/lucene/document/Fieldable.java
===================================================================
--- lucene/src/java/org/apache/lucene/document/Fieldable.java	(revision 1032543)
+++ lucene/src/java/org/apache/lucene/document/Fieldable.java	(working copy)
@@ -18,6 +18,7 @@
 
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.index.FieldInvertState; // for javadocs
+import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.search.PhraseQuery; // for javadocs
 import org.apache.lucene.search.spans.SpanQuery; // for javadocs
 
@@ -209,4 +210,13 @@
   * silently fail to find results.
   */
   void setOmitTermFreqAndPositions(boolean omitTermFreqAndPositions);
+  
+  //nocommit add javadoc
+  String getCodec();
+  
+  void setCodec(String codec);
+  
+  String getCodec(CodecProvider provider);
+  
+  void setCodec(CodecProvider provider, String codec);
 }
Index: lucene/src/java/org/apache/lucene/index/CheckIndex.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/CheckIndex.java	(revision 1032543)
+++ lucene/src/java/org/apache/lucene/index/CheckIndex.java	(working copy)
@@ -129,8 +129,8 @@
       /** Name of the segment. */
       public String name;
 
-      /** Name of codec used to read this segment. */
-      public String codec;
+      /** CodecInfo used to read this segment. */
+      public CodecInfo codec;
 
       /** Document count (does not take deletions into account). */
       public int docCount;
@@ -408,7 +408,7 @@
       SegmentReader reader = null;
 
       try {
-        final String codec = info.getCodec().name;
+        final CodecInfo codec = info.getCodecInfo();
         msg("    codec=" + codec);
         segInfoStat.codec = codec;
         msg("    compound=" + info.getUseCompoundFile());
@@ -602,7 +602,7 @@
         }
         
         final TermsEnum terms = fieldsEnum.terms();
-
+        assert terms != null;
         boolean hasOrd = true;
         final long termCountStart = status.termCount;
 
Index: lucene/src/java/org/apache/lucene/index/CodecInfo.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/CodecInfo.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/index/CodecInfo.java	(revision 0)
@@ -0,0 +1,118 @@
+package org.apache.lucene.index;
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+
+/**
+ * 
+ * TODO add JAVADOC
+ */
+public final class CodecInfo implements Cloneable {
+  final ArrayList<Codec> codecs = new ArrayList<Codec>();
+  final Map<Codec, Integer> codecRegistry = new IdentityHashMap<Codec, Integer>();
+  final CodecProvider provider;
+  private final Codec codec = new PerFieldCodecWrapper(this);
+  
+  public CodecInfo(CodecProvider provider) {
+    this.provider = provider;
+  }
+  
+  public Codec byField(String field) {
+    return provider.lookup(provider.getFieldCodec(field));
+  }
+  
+  public int assignId(Codec codec) {
+    final Integer codecId;
+    if((codecId = codecRegistry.get(codec)) != null){
+      return codecId.intValue();
+    }
+    final Integer newId = Integer.valueOf(codecs.size());
+    codecRegistry.put(codec, newId);
+    codecs.add(codec);
+    return newId.intValue();
+  }
+  
+  public Codec codec() {
+    return codec;
+  }
+
+  public void write(IndexOutput out) throws IOException {
+    out.writeVInt(codecs.size());
+    for (Codec codec : codecs) {
+      out.writeString(codec.name);
+    }
+  }
+  
+  public void read(IndexInput in) throws IOException {
+    int size = in.readVInt();
+    for (int i = 0; i < size; i++) {
+      String codecName = in.readString();
+      final Codec lookup = provider.lookup(codecName);
+      codecs.add(lookup);
+    }
+  }
+  
+  public void close() {
+    this.codecs.clear();
+  }
+
+  public void files(Directory dir, SegmentInfo info, Set<String> files)
+      throws IOException {
+    Iterator<Codec> it = codecs.iterator();
+    Set<Codec> seen = new HashSet<Codec>();
+    while (it.hasNext()) {
+      final Codec codec = it.next();
+      if (!seen.contains(codec)) {
+        seen.add(codec);
+        codec.files(dir, info, files);
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "CodecInfo [codecs=" + codecs + "]";
+  }
+  
+  public void initFrom(CodecInfo info) {
+    assert this.codecs.isEmpty();
+    this.codecs.addAll(info.codecs);
+  }
+  
+  static class CodecID {
+    final int id;
+    final Codec codec;
+    public CodecID(int id, Codec codec) {
+      super();
+      this.id = id;
+      this.codec = codec;
+    }
+    
+  }
+}
\ No newline at end of file

Property changes on: lucene/src/java/org/apache/lucene/index/CodecInfo.java
___________________________________________________________________
Added: svn:keywords
   + Date Author Id Revision HeadURL
Added: svn:eol-style
   + native

Index: lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java	(revision 1032543)
+++ lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java	(working copy)
@@ -34,11 +34,12 @@
 final class DocFieldProcessor extends DocConsumer {
 
   final DocumentsWriter docWriter;
-  final FieldInfos fieldInfos = new FieldInfos();
+  final FieldInfos fieldInfos;
   final DocFieldConsumer consumer;
   final StoredFieldsWriter fieldsWriter;
 
   public DocFieldProcessor(DocumentsWriter docWriter, DocFieldConsumer consumer) {
+    this.fieldInfos = new FieldInfos(new CodecInfo(docWriter.writer.codecs));
     this.docWriter = docWriter;
     this.consumer = consumer;
     consumer.setFieldInfos(fieldInfos);
Index: lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java	(revision 1032543)
+++ lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java	(working copy)
@@ -196,7 +196,6 @@
         FieldInfo fi = fieldInfos.add(fieldName, field.isIndexed(), field.isTermVectorStored(),
                                       field.isStorePositionWithTermVector(), field.isStoreOffsetWithTermVector(),
                                       field.getOmitNorms(), false, field.getOmitTermFreqAndPositions());
-
         fp = new DocFieldProcessorPerField(this, fi);
         fp.next = fieldHash[hashPos];
         fieldHash[hashPos] = fp;
Index: lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/DocumentsWriter.java	(revision 1032543)
+++ lucene/src/java/org/apache/lucene/index/DocumentsWriter.java	(working copy)
@@ -605,13 +605,12 @@
   synchronized private void initFlushState(boolean onlyDocStore) {
     initSegmentName(onlyDocStore);
     flushState = new SegmentWriteState(infoStream, directory, segment, docFieldProcessor.fieldInfos,
-                                       docStoreSegment, numDocsInRAM, numDocsInStore, writer.getConfig().getTermIndexInterval(),
-                                       writer.codecs);
+                                       docStoreSegment, numDocsInRAM, numDocsInStore, writer.getConfig().getTermIndexInterval());
   }
 
   /** Returns the codec used to flush the last segment */
-  Codec getCodec() {
-    return flushState.codec;
+  CodecInfo getCodecInfo() {
+    return flushState.codecInfo;
   }
   
   /** Flush all pending docs to a new segment */
@@ -653,7 +652,7 @@
       if (infoStream != null) {
         SegmentInfo si = new SegmentInfo(flushState.segmentName,
             flushState.numDocs, directory, false, -1, flushState.segmentName,
-            false, hasProx(), flushState.codec);
+            false, hasProx(), flushState.codecInfo);
         final long newSegmentSize = si.sizeInBytes();
         String message = "  ramUsed=" + nf.format(startNumBytesUsed/1024./1024.) + " MB" +
           " newFlushedSize=" + newSegmentSize +
Index: lucene/src/java/org/apache/lucene/index/FieldInfo.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/FieldInfo.java	(revision 1032543)
+++ lucene/src/java/org/apache/lucene/index/FieldInfo.java	(working copy)
@@ -32,6 +32,7 @@
   public boolean omitTermFreqAndPositions;
 
   public boolean storePayloads; // whether this field stores payloads together with term positions
+  public int codecId = 0;
 
   FieldInfo(String na, boolean tk, int nu, boolean storeTermVector, 
             boolean storePositionWithTermVector,  boolean storeOffsetWithTermVector, 
Index: lucene/src/java/org/apache/lucene/index/FieldInfos.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/FieldInfos.java	(revision 1032543)
+++ lucene/src/java/org/apache/lucene/index/FieldInfos.java	(working copy)
@@ -19,6 +19,7 @@
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
@@ -38,9 +39,10 @@
 
   // First used in 2.9; prior to 2.9 there was no format header
   public static final int FORMAT_START = -2;
+  public static final int FORMAT_PER_FIELD_CODEC = -3;
 
   // whenever you add a new format, make it 1 smaller (negative version logic)!
-  static final int FORMAT_CURRENT = FORMAT_START;
+  static final int FORMAT_CURRENT = FORMAT_PER_FIELD_CODEC;
   
   static final int FORMAT_MINIMUM = FORMAT_START;
   
@@ -56,8 +58,12 @@
   private final HashMap<String,FieldInfo> byName = new HashMap<String,FieldInfo>();
   private int format;
 
-  public FieldInfos() { }
+  final CodecInfo codecInfo;
 
+  public FieldInfos(CodecInfo codecInfo) {
+    this.codecInfo = codecInfo;
+  }
+
   /**
    * Construct a FieldInfos object using the directory and the name of the file
    * IndexInput
@@ -65,7 +71,8 @@
    * @param name The name of the file to open the IndexInput from in the Directory
    * @throws IOException
    */
-  public FieldInfos(Directory d, String name) throws IOException {
+  public FieldInfos(Directory d, String name, CodecInfo codecInfo) throws IOException {
+    this.codecInfo = codecInfo;
     IndexInput input = d.openInput(name);
     try {
       read(input, name);
@@ -79,7 +86,7 @@
    */
   @Override
   synchronized public Object clone() {
-    FieldInfos fis = new FieldInfos();
+    FieldInfos fis = new FieldInfos(this.codecInfo);
     final int numField = byNumber.size();
     for(int i=0;i<numField;i++) {
       FieldInfo fi = (FieldInfo) ( byNumber.get(i)).clone();
@@ -230,6 +237,10 @@
                                  storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
     byNumber.add(fi);
     byName.put(name, fi);
+    if(isIndexed) { // TODO once per doc codec support is available fix this!
+      final Codec byField = codecInfo.byField(name);
+      fi.codecId = codecInfo.assignId(byField);
+    }
     return fi;
   }
 
@@ -301,8 +312,8 @@
       if (fi.omitNorms) bits |= OMIT_NORMS;
       if (fi.storePayloads) bits |= STORE_PAYLOADS;
       if (fi.omitTermFreqAndPositions) bits |= OMIT_TERM_FREQ_AND_POSITIONS;
-      
       output.writeString(fi.name);
+      output.writeInt(fi.codecId);
       output.writeByte(bits);
     }
   }
@@ -321,6 +332,8 @@
 
     for (int i = 0; i < size; i++) {
       String name = StringHelper.intern(input.readString());
+      // if this is a previous format codec 0 will be preflex!
+      final int codecId = format <= FORMAT_PER_FIELD_CODEC? input.readInt():0;
       byte bits = input.readByte();
       boolean isIndexed = (bits & IS_INDEXED) != 0;
       boolean storeTermVector = (bits & STORE_TERMVECTOR) != 0;
@@ -329,8 +342,8 @@
       boolean omitNorms = (bits & OMIT_NORMS) != 0;
       boolean storePayloads = (bits & STORE_PAYLOADS) != 0;
       boolean omitTermFreqAndPositions = (bits & OMIT_TERM_FREQ_AND_POSITIONS) != 0;
-      
-      addInternal(name, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
+      final FieldInfo addInternal = addInternal(name, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
+      addInternal.codecId = codecId;
     }
 
     if (input.getFilePointer() != input.length()) {
Index: lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java	(revision 1032543)
+++ lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java	(working copy)
@@ -77,8 +77,7 @@
     // Sort by field name
     CollectionUtil.quickSort(allFields);
 
-    // TODO: allow Lucene user to customize this codec:
-    final FieldsConsumer consumer = state.codec.fieldsConsumer(state);
+    final FieldsConsumer consumer = state.codecInfo.codec().fieldsConsumer(state);
 
     /*
     Current writer chain:
Index: lucene/src/java/org/apache/lucene/index/IndexWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/IndexWriter.java	(revision 1032543)
+++ lucene/src/java/org/apache/lucene/index/IndexWriter.java	(working copy)
@@ -623,7 +623,7 @@
         // TODO: we may want to avoid doing this while
         // synchronized
         // Returns a ref, which we xfer to readerMap:
-        sr = SegmentReader.get(false, info.dir, info, readBufferSize, doOpenStores, termsIndexDivisor, codecs);
+        sr = SegmentReader.get(false, info.dir, info, readBufferSize, doOpenStores, termsIndexDivisor);
 
         if (info.dir == directory) {
           // Only pool if reader is not external
@@ -2997,7 +2997,7 @@
       SegmentInfo info = null;
       synchronized(this) {
         info = new SegmentInfo(mergedName, docCount, directory, false, -1,
-            null, false, merger.hasProx(), merger.getCodec());
+            null, false, merger.hasProx(),merger.getCodecInfo());
         setDiagnostics(info, "addIndexes(IndexReader...)");
         segmentInfos.add(info);
         checkpoint();
@@ -3375,10 +3375,10 @@
                                      directory, false, docStoreOffset,
                                      docStoreSegment, docStoreIsCompoundFile,
                                      docWriter.hasProx(),    
-                                     docWriter.getCodec());
+                                     docWriter.getCodecInfo());
 
         if (infoStream != null) {
-          message("flush codec=" + docWriter.getCodec().name);
+          message("flush codec=" + docWriter.getCodecInfo());
         }
         setDiagnostics(newSegment, "flush");
       }
@@ -4068,10 +4068,10 @@
       mergedDocCount = merge.info.docCount = merger.merge(merge.mergeDocStores);
 
       // Record which codec was used to write the segment
-      merge.info.setCodec(merger.getCodec());
+      merge.info.setCodecInfo(merger.getCodecInfo());
 
       if (infoStream != null) {
-        message("merge codec=" + merger.getCodec().name);
+        message("merge codec=" + merger.getCodecInfo());
       }
       
       assert mergedDocCount == totDocCount;
Index: lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java	(revision 0)
@@ -0,0 +1,225 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.IdentityHashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.FieldsConsumer;
+import org.apache.lucene.index.codecs.FieldsProducer;
+import org.apache.lucene.index.codecs.TermsConsumer;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+
+
+/** 
+ * Enables native per field codec support
+ * @lucene.internal */
+public class PerFieldCodecWrapper extends Codec {
+  private final CodecInfo codecInfo;
+
+  PerFieldCodecWrapper(CodecInfo codecInfo) {
+    name = "PerField"; // TODO this is bogus
+    this.codecInfo = codecInfo;
+  }
+
+      
+  @Override
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    return new FieldsWriter(state);
+  }
+
+  private class FieldsWriter extends FieldsConsumer {
+    private final SegmentWriteState state;
+    private final Map<Codec,FieldsConsumer> codecs = new IdentityHashMap<Codec,FieldsConsumer>();
+
+    public FieldsWriter(SegmentWriteState state) throws IOException {
+      this.state = state;
+      initFieldsConsumer();
+      
+    }
+    
+    private void initFieldsConsumer() throws IOException {
+      final FieldInfos infos = state.fieldInfos;
+      final int size = infos.size(); 
+      for (int i = 0; i < size; i++) { // init all indexed fields to make sure their files are initialized even if 0 docs are added
+        final FieldInfo fieldInfo = infos.fieldInfo(i);
+        if(fieldInfo.isIndexed) {
+          getFieldsConsumer(fieldInfo);
+        }
+      }
+    }
+    
+    private FieldsConsumer getFieldsConsumer(FieldInfo fieldInfo) throws IOException {
+      final Codec codec = codecInfo.byField(fieldInfo.name);
+      FieldsConsumer fields = codecs.get(codec);
+      if (fields == null) {
+        state.currentCodecId = fieldInfo.codecId;
+        fields  = codec.fieldsConsumer(state);
+        codecs.put(codec, fields);
+      }
+      return fields;
+
+    }
+
+    @Override
+    public TermsConsumer addField(FieldInfo field) throws IOException {
+      final FieldsConsumer fields = getFieldsConsumer(field);
+      return fields.addField(field);
+    }
+
+    @Override
+    public void close() throws IOException {
+      Iterator<FieldsConsumer> it = codecs.values().iterator();
+      IOException err = null;
+      while(it.hasNext()) {
+        try {
+          it.next().close();
+        } catch (IOException ioe) {
+          // keep first IOException we hit but keep
+          // closing the rest
+          if (err == null) {
+            err = ioe;
+          }
+        }
+      }
+      if (err != null) {
+        throw err;
+      }
+    }
+  }
+
+  private class FieldsReader extends FieldsProducer {
+
+    private final Set<String> fields = new TreeSet<String>();
+    private final Map<String, FieldsProducer> codecs = new HashMap<String, FieldsProducer>();
+    public FieldsReader(Directory dir, FieldInfos fieldInfos,
+                        SegmentInfo si, int readBufferSize,
+                        int indexDivisor) throws IOException {
+
+      final int fieldCount = fieldInfos.size();
+      final Map<Codec, FieldsProducer> producers = new HashMap<Codec, FieldsProducer>();
+      for(int i=0;i<fieldCount;i++) {
+        FieldInfo fi = fieldInfos.fieldInfo(i);
+        if (fi.isIndexed) { // TODO this does not work for  non-indexed fields
+          fields.add(fi.name);
+          Codec codec = codecInfo.codecs.get(fi.codecId);
+          if (!producers.containsKey(codec)) {
+            producers.put(codec, codec.fieldsProducer(new SegmentReadState(dir, si, fieldInfos, readBufferSize, indexDivisor)));
+          }
+          codecs.put(fi.name, producers.get(codec));
+        }
+      }
+    }
+
+    private final class FieldsIterator extends FieldsEnum {
+      private final Iterator<String> it;
+      private String current;
+
+      public FieldsIterator() {
+        it = fields.iterator();
+      }
+
+      @Override
+      public String next() {
+        if (it.hasNext()) {
+          current = it.next();
+        } else {
+          current = null;
+        }
+
+        return current;
+      }
+
+      @Override
+      public TermsEnum terms() throws IOException {
+        Terms terms = codecs.get(current).terms(current);
+        if (terms != null) {
+          return terms.iterator();
+        } else {
+          return TermsEnum.EMPTY;
+        }
+      }
+    }
+      
+    @Override
+    public FieldsEnum iterator() throws IOException {
+      return new FieldsIterator();
+    }
+
+    @Override
+    public Terms terms(String field) throws IOException {
+      FieldsProducer fields = codecs.get(field);
+      return fields == null? null:fields.terms(field);
+    }
+
+    @Override
+    public void close() throws IOException {
+      Iterator<FieldsProducer> it = codecs.values().iterator();
+      IOException err = null;
+      while(it.hasNext()) {
+        try {
+          it.next().close();
+        } catch (IOException ioe) {
+          // keep first IOException we hit but keep
+          // closing the rest
+          if (err == null) {
+            err = ioe;
+          }
+        }
+      }
+      if (err != null) {
+        throw err;
+      }
+    }
+
+    @Override
+    public void loadTermsIndex(int indexDivisor) throws IOException {
+      Iterator<FieldsProducer> it = codecs.values().iterator();
+      while(it.hasNext()) {
+        it.next().loadTermsIndex(indexDivisor);
+      }
+    }
+  }
+
+  public FieldsProducer fieldsProducer(SegmentReadState state)
+    throws IOException {
+    return new FieldsReader(state.dir, state.fieldInfos, state.segmentInfo, state.readBufferSize, state.termsIndexDivisor);
+  }
+
+  @Override
+  public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
+      codecInfo.files(dir, info, files);
+  }
+
+  @Override
+  public void getExtensions(Set<String> extensions) {
+    Iterator<Codec> it = codecInfo.codecs.iterator();
+    while(it.hasNext()) {
+      final Codec codec = it.next();
+      codec.getExtensions(extensions);
+    }
+  }
+}

Property changes on: lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java
___________________________________________________________________
Added: svn:keywords
   + Date Author Id Revision HeadURL
Added: svn:eol-style
   + native

Index: lucene/src/java/org/apache/lucene/index/SegmentInfo.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/SegmentInfo.java	(revision 1032543)
+++ lucene/src/java/org/apache/lucene/index/SegmentInfo.java	(working copy)
@@ -80,13 +80,12 @@
 
   private boolean hasProx;                        // True if this segment has any fields with omitTermFreqAndPositions==false
   
-  private Codec codec;
+  private CodecInfo codecInfo;
 
-
   private Map<String,String> diagnostics;
 
   public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile, int docStoreOffset, 
-                     String docStoreSegment, boolean docStoreIsCompoundFile, boolean hasProx, Codec codec) { 
+                     String docStoreSegment, boolean docStoreIsCompoundFile, boolean hasProx, CodecInfo codecInfo) { 
     this.name = name;
     this.docCount = docCount;
     this.dir = dir;
@@ -96,7 +95,7 @@
     this.docStoreSegment = docStoreSegment;
     this.docStoreIsCompoundFile = docStoreIsCompoundFile;
     this.hasProx = hasProx;
-    this.codec = codec;
+    this.codecInfo = codecInfo;
     delCount = 0;
     assert docStoreOffset == -1 || docStoreSegment != null: "dso=" + docStoreOffset + " dss=" + docStoreSegment + " docCount=" + docCount;
   }
@@ -120,7 +119,7 @@
     }
     isCompoundFile = src.isCompoundFile;
     delCount = src.delCount;
-    codec = src.codec;
+    codecInfo = src.codecInfo;
   }
 
   void setDiagnostics(Map<String, String> diagnostics) {
@@ -145,7 +144,6 @@
     this.dir = dir;
     name = input.readString();
     docCount = input.readInt();
-    final String codecName;
     delGen = input.readLong();
     docStoreOffset = input.readInt();
     if (docStoreOffset != -1) {
@@ -177,14 +175,15 @@
     hasProx = input.readByte() == YES;
     
     // System.out.println(Thread.currentThread().getName() + ": si.read hasProx=" + hasProx + " seg=" + name);
-    
-    if (format <= DefaultSegmentInfosWriter.FORMAT_4_0)
-      codecName = input.readString();
-    else
-      codecName = "PreFlex";
-    
+    codecInfo = new CodecInfo(codecs);
+    if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
+      codecInfo.read(input);
+    } 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?
+      codecInfo.assignId(codecs.lookup("PreFlex"));
+    }
     diagnostics = input.readStringStringMap();
-    codec = codecs.lookup(codecName);
   }
   
   /** Returns total size in bytes of all of files used by
@@ -230,7 +229,7 @@
 
   @Override
   public Object clone() {
-    SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, docStoreOffset, docStoreSegment, docStoreIsCompoundFile, hasProx, codec);
+    SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, docStoreOffset, docStoreSegment, docStoreIsCompoundFile, hasProx, codecInfo);
     si.isCompoundFile = isCompoundFile;
     si.delGen = delGen;
     si.delCount = delCount;
@@ -242,7 +241,6 @@
     si.docStoreOffset = docStoreOffset;
     si.docStoreSegment = docStoreSegment;
     si.docStoreIsCompoundFile = docStoreIsCompoundFile;
-    si.codec = codec;
     return si;
   }
 
@@ -400,7 +398,7 @@
     output.writeByte((byte) (isCompoundFile ? YES : NO));
     output.writeInt(delCount);
     output.writeByte((byte) (hasProx ? 1:0));
-    output.writeString(codec.name);
+    codecInfo.write(output);
     output.writeStringStringMap(diagnostics);
   }
 
@@ -414,16 +412,16 @@
   }
 
   /** Can only be called once. */
-  public void setCodec(Codec codec) {
-    assert this.codec == null;
-    if (codec == null) {
-      throw new IllegalArgumentException("codec must be non-null");
+  public void setCodecInfo(CodecInfo codecInfo) {
+    assert this.codecInfo == null;
+    if (codecInfo == null) {
+      throw new IllegalArgumentException("codecInfo must be non-null");
     }
-    this.codec = codec;
+    this.codecInfo = codecInfo;
   }
 
-  Codec getCodec() {
-    return codec;
+  CodecInfo getCodecInfo() {
+    return codecInfo;
   }
 
   private void addIfExists(Set<String> files, String fileName) throws IOException {
@@ -454,7 +452,7 @@
       for(String ext : IndexFileNames.NON_STORE_INDEX_EXTENSIONS) {
         addIfExists(fileSet, IndexFileNames.segmentFileName(name, "", ext));
       }
-      codec.files(dir, this, fileSet);
+      codecInfo.files(dir, this, fileSet);
     }
 
     if (docStoreOffset != -1) {
Index: lucene/src/java/org/apache/lucene/index/SegmentInfos.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/SegmentInfos.java	(revision 1032543)
+++ lucene/src/java/org/apache/lucene/index/SegmentInfos.java	(working copy)
@@ -308,7 +308,7 @@
     SegmentInfos sis = (SegmentInfos) super.clone();
     for(int i=0;i<sis.size();i++) {
       final SegmentInfo info = sis.info(i);
-      assert info.getCodec() != null;
+      assert info.getCodecInfo() != null;
       sis.set(i, (SegmentInfo) info.clone());
     }
     sis.userData = new HashMap<String,String>(userData);
Index: lucene/src/java/org/apache/lucene/index/SegmentMerger.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/SegmentMerger.java	(revision 1032543)
+++ lucene/src/java/org/apache/lucene/index/SegmentMerger.java	(working copy)
@@ -184,7 +184,7 @@
         fileSet.add(IndexFileNames.segmentFileName(segment, "", ext));
     }
 
-    codec.files(directory, info, fileSet);
+    segmentWriteState.codecInfo.files(directory, info, fileSet);
     
     // Fieldable norm files
     int numFIs = fieldInfos.size();
@@ -269,6 +269,7 @@
    * @throws IOException if there is a low-level IO error
    */
   private final int mergeFields() throws CorruptIndexException, IOException {
+    CodecInfo info = new CodecInfo(codecs);
 
     if (!mergeDocStores) {
       // When we are not merging by doc stores, their field
@@ -278,7 +279,7 @@
       final SegmentReader sr = (SegmentReader) readers.get(readers.size()-1);
       fieldInfos = (FieldInfos) sr.core.fieldInfos.clone();
     } else {
-      fieldInfos = new FieldInfos();		  // merge field names
+      fieldInfos = new FieldInfos(info);// merge field names
     }
 
     for (IndexReader reader : readers) {
@@ -357,8 +358,8 @@
       }
     }
 
-    segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, null, docCount, 0, termIndexInterval, codecs);
-
+    segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, null, docCount, 0, termIndexInterval);
+    
     return docCount;
   }
 
@@ -554,15 +555,15 @@
     }
   }
 
-  Codec getCodec() {
-    return codec;
+  CodecInfo getCodecInfo() {
+    assert segmentWriteState != null;
+    return segmentWriteState.codecInfo;
   }
 
   private final void mergeTerms() throws CorruptIndexException, IOException {
 
     // Let CodecProvider decide which codec will be used to write
     // the new segment:
-    codec = codecs.getWriter(segmentWriteState);
     
     int docBase = 0;
 
@@ -644,7 +645,7 @@
       }
     }
     starts[mergeState.readerCount] = inputDocBase;
-
+    codec = segmentWriteState.codecInfo.codec();
     final FieldsConsumer consumer = codec.fieldsConsumer(segmentWriteState);
 
     // NOTE: this is silly, yet, necessary -- we create a
Index: lucene/src/java/org/apache/lucene/index/SegmentReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/SegmentReader.java	(revision 1032543)
+++ lucene/src/java/org/apache/lucene/index/SegmentReader.java	(working copy)
@@ -90,7 +90,6 @@
     final FieldInfos fieldInfos;
 
     final FieldsProducer fields;
-    final CodecProvider codecs;
     
     final Directory dir;
     final Directory cfsDir;
@@ -104,17 +103,14 @@
     CompoundFileReader cfsReader;
     CompoundFileReader storeCFSReader;
 
-    CoreReaders(SegmentReader origInstance, Directory dir, SegmentInfo si, int readBufferSize, int termsIndexDivisor, CodecProvider codecs) throws IOException {
+    CoreReaders(SegmentReader origInstance, Directory dir, SegmentInfo si, int readBufferSize, int termsIndexDivisor) throws IOException {
 
       if (termsIndexDivisor == 0) {
         throw new IllegalArgumentException("indexDivisor must be < 0 (don't load terms index) or greater than 0 (got 0)");
       }
 
       segment = si.name;
-      if (codecs == null) {
-        codecs = CodecProvider.getDefault();
-      }
-      this.codecs = codecs;      
+      final CodecInfo codecInfo = si.getCodecInfo();
       this.readBufferSize = readBufferSize;
       this.dir = dir;
 
@@ -128,12 +124,16 @@
         }
         cfsDir = dir0;
 
-        fieldInfos = new FieldInfos(cfsDir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELD_INFOS_EXTENSION));
-
+        fieldInfos = new FieldInfos(cfsDir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELD_INFOS_EXTENSION), codecInfo);
+        
         this.termsIndexDivisor = termsIndexDivisor;
+        
+        if(si.getCodecInfo().codecs.isEmpty()) {
+          fields = codecInfo.codec().fieldsProducer(new SegmentReadState(cfsDir, si, fieldInfos, readBufferSize, termsIndexDivisor));
 
+        } else
         // Ask codec for its Fields
-        fields = si.getCodec().fieldsProducer(new SegmentReadState(cfsDir, si, fieldInfos, readBufferSize, termsIndexDivisor));
+        fields = codecInfo.codec().fieldsProducer(new SegmentReadState(cfsDir, si, fieldInfos, readBufferSize, termsIndexDivisor));
         assert fields != null;
 
         success = true;
@@ -506,7 +506,7 @@
    * @throws IOException if there is a low-level IO error
    */
   public static SegmentReader get(boolean readOnly, SegmentInfo si, int termInfosIndexDivisor) throws CorruptIndexException, IOException {
-    return get(readOnly, si.dir, si, BufferedIndexInput.BUFFER_SIZE, true, termInfosIndexDivisor, null);
+    return get(readOnly, si.dir, si, BufferedIndexInput.BUFFER_SIZE, true, termInfosIndexDivisor);
   }
 
   /**
@@ -518,12 +518,8 @@
                                   SegmentInfo si,
                                   int readBufferSize,
                                   boolean doOpenStores,
-                                  int termInfosIndexDivisor,
-                                  CodecProvider codecs)
+                                  int termInfosIndexDivisor)
     throws CorruptIndexException, IOException {
-    if (codecs == null)  {
-      codecs = CodecProvider.getDefault();
-    }
     
     SegmentReader instance = new SegmentReader();
     instance.readOnly = readOnly;
@@ -533,7 +529,7 @@
     boolean success = false;
 
     try {
-      instance.core = new CoreReaders(instance, dir, si, readBufferSize, termInfosIndexDivisor, codecs);
+      instance.core = new CoreReaders(instance, dir, si, readBufferSize, termInfosIndexDivisor);
       if (doOpenStores) {
         instance.core.openDocStores(si);
       }
Index: lucene/src/java/org/apache/lucene/index/SegmentWriteState.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/SegmentWriteState.java	(revision 1032543)
+++ lucene/src/java/org/apache/lucene/index/SegmentWriteState.java	(working copy)
@@ -17,13 +17,11 @@
  * limitations under the License.
  */
 
-import java.util.HashSet;
-import java.util.Collection;
 import java.io.PrintStream;
+import java.util.Collection;
+import java.util.HashSet;
 
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.CodecProvider;
 
 /**
  * @lucene.experimental
@@ -38,8 +36,8 @@
   public int numDocsInStore;
   public final Collection<String> flushedFiles;
 
-  // Actual codec used
-  final Codec codec;
+  final CodecInfo codecInfo;
+  public int currentCodecId;
 
   /** Expert: The fraction of terms in the "dictionary" which should be stored
    * in RAM.  Smaller values use more memory, but make searching slightly
@@ -59,11 +57,12 @@
    * slightly smaller indexes, but slower skipping in big posting lists.
    */
   public final int maxSkipLevels = 10;
+  
 
+
   public SegmentWriteState(PrintStream infoStream, Directory directory, String segmentName, FieldInfos fieldInfos,
                            String docStoreSegmentName, int numDocs,
-                           int numDocsInStore, int termIndexInterval,
-                           CodecProvider codecs) {
+                           int numDocsInStore, int termIndexInterval) {
     this.infoStream = infoStream;
     this.directory = directory;
     this.segmentName = segmentName;
@@ -72,7 +71,7 @@
     this.numDocs = numDocs;
     this.numDocsInStore = numDocsInStore;
     this.termIndexInterval = termIndexInterval;
-    this.codec = codecs.getWriter(this);
+    this.codecInfo = fieldInfos.codecInfo;
     flushedFiles = new HashSet<String>();
   }
 }
Index: lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java	(revision 1032543)
+++ lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java	(working copy)
@@ -20,13 +20,15 @@
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.lucene.index.SegmentWriteState;
 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.simpletext.SimpleTextCodec;
 
 /** Holds a set of codecs, keyed by name.  You subclass
  *  this, instantiate it, and register your codecs, then
@@ -39,7 +41,7 @@
 public abstract class CodecProvider {
   private SegmentInfosWriter infosWriter = new DefaultSegmentInfosWriter();
   private SegmentInfosReader infosReader = new DefaultSegmentInfosReader();
-
+  
   private final HashMap<String, Codec> codecs = new HashMap<String, Codec>();
 
   private final Set<String> knownExtensions = new HashSet<String>();
@@ -86,8 +88,6 @@
     return codec;
   }
 
-  public abstract Codec getWriter(SegmentWriteState state);
-  
   public SegmentInfosWriter getSegmentInfosWriter() {
     return infosWriter;
   }
@@ -110,6 +110,28 @@
   public synchronized static String getDefaultCodec() {
     return defaultCodec;
   }
+  
+  private String defaultFieldCodec = defaultCodec;
+  private final Map<String, String> perFieldMap = new HashMap<String, String>();
+
+  public void setFieldCodec(String name, String codec) {
+    perFieldMap.put(name, codec);
+  }
+
+  public String getFieldCodec(String name) {
+    final String codec;
+    if((codec = perFieldMap.get(name)) == null)
+      return defaultFieldCodec;
+    return codec;
+  }
+  
+  public String getDefaultFieldCodec() {
+    return defaultFieldCodec;
+  }
+  
+  public void setDefaultFieldCodec(String codec) {
+    defaultFieldCodec = codec;
+  }
 }
 
 class DefaultCodecProvider extends CodecProvider {
@@ -118,10 +140,6 @@
     register(new PreFlexCodec());
     register(new PulsingCodec(1));
     register(new SimpleTextCodec());
+    this.setDefaultFieldCodec(CodecProvider.getDefaultCodec());
   }
-
-  @Override
-  public Codec getWriter(SegmentWriteState state) {
-    return lookup(CodecProvider.getDefaultCodec());
-  }
 }
Index: lucene/src/java/org/apache/lucene/index/codecs/PerFieldCodecWrapper.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/PerFieldCodecWrapper.java	(revision 1032543)
+++ lucene/src/java/org/apache/lucene/index/codecs/PerFieldCodecWrapper.java	(working copy)
@@ -1,238 +0,0 @@
-package org.apache.lucene.index.codecs;
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.util.Map;
-import java.util.HashMap;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.IdentityHashMap;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.io.IOException;
-
-import org.apache.lucene.index.FieldsEnum;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.Terms;
-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.SegmentReadState;
-import org.apache.lucene.store.Directory;
-
-
-/** Simple Codec that dispatches field-specific codecs.
- *  You must ensure every field you index has a Codec, or
- *  the defaultCodec is non null.  Also, the separate
- *  codecs cannot conflict on file names.
- *
- * @lucene.experimental */
-public class PerFieldCodecWrapper extends Codec {
-  private final Map<String,Codec> fields = new IdentityHashMap<String,Codec>();
-  private final Codec defaultCodec;
-
-  public PerFieldCodecWrapper(Codec defaultCodec) {
-    name = "PerField";
-    this.defaultCodec = defaultCodec;
-  }
-
-  public void add(String field, Codec codec) {
-    fields.put(field, codec);
-  }
-
-  public Codec getCodec(String field) {
-    Codec codec = fields.get(field);
-    if (codec != null) {
-      return codec;
-    } else {
-      return defaultCodec;
-    }
-  }
-      
-  @Override
-  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    return new FieldsWriter(state);
-  }
-
-  private class FieldsWriter extends FieldsConsumer {
-    private final SegmentWriteState state;
-    private final Map<Codec,FieldsConsumer> codecs = new HashMap<Codec,FieldsConsumer>();
-    private final Set<String> fieldsSeen = new TreeSet<String>();
-
-    public FieldsWriter(SegmentWriteState state) {
-      this.state = state;
-    }
-
-    @Override
-    public TermsConsumer addField(FieldInfo field) throws IOException {
-      fieldsSeen.add(field.name);
-      Codec codec = getCodec(field.name);
-
-      FieldsConsumer fields = codecs.get(codec);
-      if (fields == null) {
-        fields = codec.fieldsConsumer(state);
-        codecs.put(codec, fields);
-      }
-      return fields.addField(field);
-    }
-
-    @Override
-    public void close() throws IOException {
-      Iterator<FieldsConsumer> it = codecs.values().iterator();
-      IOException err = null;
-      while(it.hasNext()) {
-        try {
-          it.next().close();
-        } catch (IOException ioe) {
-          // keep first IOException we hit but keep
-          // closing the rest
-          if (err == null) {
-            err = ioe;
-          }
-        }
-      }
-      if (err != null) {
-        throw err;
-      }
-    }
-  }
-
-  private class FieldsReader extends FieldsProducer {
-
-    private final Set<String> fields = new TreeSet<String>();
-    private final Map<Codec,FieldsProducer> codecs = new HashMap<Codec,FieldsProducer>();
-
-    public FieldsReader(Directory dir, FieldInfos fieldInfos,
-                        SegmentInfo si, int readBufferSize,
-                        int indexDivisor) throws IOException {
-
-      final int fieldCount = fieldInfos.size();
-      for(int i=0;i<fieldCount;i++) {
-        FieldInfo fi = fieldInfos.fieldInfo(i);
-        if (fi.isIndexed) {
-          fields.add(fi.name);
-          Codec codec = getCodec(fi.name);
-          if (!codecs.containsKey(codec)) {
-            codecs.put(codec, codec.fieldsProducer(new SegmentReadState(dir, si, fieldInfos, readBufferSize, indexDivisor)));
-          }
-        }
-      }
-    }
-
-    private final class FieldsIterator extends FieldsEnum {
-      private final Iterator<String> it;
-      private String current;
-
-      public FieldsIterator() {
-        it = fields.iterator();
-      }
-
-      @Override
-      public String next() {
-        if (it.hasNext()) {
-          current = it.next();
-        } else {
-          current = null;
-        }
-
-        return current;
-      }
-
-      @Override
-      public TermsEnum terms() throws IOException {
-        Terms terms = codecs.get(getCodec(current)).terms(current);
-        if (terms != null) {
-          return terms.iterator();
-        } else {
-          return null;
-        }
-      }
-    }
-      
-    @Override
-    public FieldsEnum iterator() throws IOException {
-      return new FieldsIterator();
-    }
-
-    @Override
-    public Terms terms(String field) throws IOException {
-      Codec codec = getCodec(field);
-
-      FieldsProducer fields = codecs.get(codec);
-      assert fields != null;
-      return fields.terms(field);
-    }
-
-    @Override
-    public void close() throws IOException {
-      Iterator<FieldsProducer> it = codecs.values().iterator();
-      IOException err = null;
-      while(it.hasNext()) {
-        try {
-          it.next().close();
-        } catch (IOException ioe) {
-          // keep first IOException we hit but keep
-          // closing the rest
-          if (err == null) {
-            err = ioe;
-          }
-        }
-      }
-      if (err != null) {
-        throw err;
-      }
-    }
-
-    @Override
-    public void loadTermsIndex(int indexDivisor) throws IOException {
-      Iterator<FieldsProducer> it = codecs.values().iterator();
-      while(it.hasNext()) {
-        it.next().loadTermsIndex(indexDivisor);
-      }
-    }
-  }
-
-  public FieldsProducer fieldsProducer(SegmentReadState state)
-    throws IOException {
-    return new FieldsReader(state.dir, state.fieldInfos, state.segmentInfo, state.readBufferSize, state.termsIndexDivisor);
-  }
-
-  @Override
-  public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
-    Iterator<Codec> it = fields.values().iterator();
-    Set<Codec> seen = new HashSet<Codec>();
-    while(it.hasNext()) {
-      final Codec codec = it.next();
-      if (!seen.contains(codec)) {
-        seen.add(codec);
-        codec.files(dir, info, files);
-      }
-    }
-  }
-
-  @Override
-  public void getExtensions(Set<String> extensions) {
-    Iterator<Codec> it = fields.values().iterator();
-    while(it.hasNext()) {
-      final Codec codec = it.next();
-      codec.getExtensions(extensions);
-    }
-  }
-}
Index: lucene/src/test/org/apache/lucene/TestExternalCodecs.java
===================================================================
--- lucene/src/test/org/apache/lucene/TestExternalCodecs.java	(revision 1032543)
+++ lucene/src/test/org/apache/lucene/TestExternalCodecs.java	(working copy)
@@ -71,7 +71,10 @@
   //   - good improvement would be to write through to disk,
   //     and then load into ram from disk
   public static class RAMOnlyCodec extends Codec {
-
+    
+    public RAMOnlyCodec() {
+      name = "RamOnly";
+    }
     // Postings state:
     static class RAMPostings extends FieldsProducer {
       final Map<String,RAMField> fieldToTerms = new TreeMap<String,RAMField>();
@@ -468,6 +471,7 @@
     @Override
     public FieldsProducer fieldsProducer(SegmentReadState readState)
       throws IOException {
+    
       return state.get(readState.segmentInfo.name);
     }
 
@@ -481,21 +485,11 @@
   }
 
   public static class MyCodecs extends CodecProvider {
-    PerFieldCodecWrapper perField;
-
     MyCodecs() {
       Codec ram = new RAMOnlyCodec();
-      Codec pulsing = new PulsingReverseTermsCodec();
-      perField = new PerFieldCodecWrapper(ram);
-      perField.add("field2", pulsing);
-      perField.add("id", pulsing);
-      register(perField);
+      register(ram);
+      setDefaultFieldCodec(ram.name);
     }
-    
-    @Override
-    public Codec getWriter(SegmentWriteState state) {
-      return perField;
-    }
   }
 
   // copied from PulsingCodec, just changing the terms
@@ -617,20 +611,27 @@
   // whose term sort is backwards unicode code point, and
   // storing "field1" as a custom entirely-in-RAM codec
   public void testPerFieldCodec() throws Exception {
+    CodecProvider provider = new MyCodecs();
+    Codec pulsing = new PulsingReverseTermsCodec();
+    provider.register(pulsing);
     
+    
     final int NUM_DOCS = 173;
     Directory dir = newDirectory();
     IndexWriter w = new IndexWriter(dir,
-                                    newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(MockTokenizer.WHITESPACE, true, true)).setCodecProvider(new MyCodecs()));
+                                    newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(MockTokenizer.WHITESPACE, true, true)).setCodecProvider(provider));
 
     w.setMergeFactor(3);
     Document doc = new Document();
     // uses default codec:
     doc.add(newField("field1", "this field uses the standard codec as the test", Field.Store.NO, Field.Index.ANALYZED));
     // uses pulsing codec:
-    doc.add(newField("field2", "this field uses the pulsing codec as the test", Field.Store.NO, Field.Index.ANALYZED));
+    Field field2 = newField("field2", "this field uses the pulsing codec as the test", Field.Store.NO, Field.Index.ANALYZED);
+    field2.setCodec(provider, pulsing.name);
+    doc.add(field2);
     
     Field idField = newField("id", "", Field.Store.NO, Field.Index.NOT_ANALYZED);
+    idField.setCodec(provider, pulsing.name);
     doc.add(idField);
     for(int i=0;i<NUM_DOCS;i++) {
       idField.setValue(""+i);
Index: lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java	(revision 1032543)
+++ lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java	(working copy)
@@ -126,7 +126,7 @@
     // If we are writing with PreFlexRW, force a full
     // IndexReader.open so terms are sorted in codepoint
     // order during searching:
-    if (!w.codecs.getWriter(null).name.equals("PreFlex") && r.nextBoolean()) {
+    if (!w.codecs.getDefaultFieldCodec().equals("PreFlex") && r.nextBoolean()) {
       if (LuceneTestCase.VERBOSE) {
         System.out.println("RIW.getReader: use NRT reader");
       }
Index: lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java	(revision 1032543)
+++ lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java	(working copy)
@@ -37,6 +37,7 @@
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.document.NumericField;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.FieldCache;
@@ -553,7 +554,7 @@
       // "content", and then set our expected file names below
       // accordingly:
       CompoundFileReader cfsReader = new CompoundFileReader(dir, "_0.cfs");
-      FieldInfos fieldInfos = new FieldInfos(cfsReader, "_0.fnm");
+      FieldInfos fieldInfos = new FieldInfos(cfsReader, "_0.fnm", new CodecInfo(CodecProvider.getDefault()));
       int contentFieldIndex = -1;
       for(int i=0;i<fieldInfos.size();i++) {
         FieldInfo fi = fieldInfos.fieldInfo(i);
Index: lucene/src/test/org/apache/lucene/index/TestCodecs.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestCodecs.java	(revision 1032543)
+++ lucene/src/test/org/apache/lucene/index/TestCodecs.java	(working copy)
@@ -256,6 +256,7 @@
   }
 
   public void testFixedPostings() throws Throwable {
+    final CodecInfo codecInfo = new CodecInfo(CodecProvider.getDefault());
     final int NUM_TERMS = 100;
     final TermData[] terms = new TermData[NUM_TERMS];
     for(int i=0;i<NUM_TERMS;i++) {
@@ -264,17 +265,17 @@
       terms[i] = new TermData(text, docs, null);
     }
 
-    final FieldInfos fieldInfos = new FieldInfos();
+    final FieldInfos fieldInfos = new FieldInfos(codecInfo);
 
     final FieldData field = new FieldData("field", fieldInfos, terms, true, false);
     final FieldData[] fields = new FieldData[] {field};
 
     final Directory dir = newDirectory();
     this.write(fieldInfos, dir, fields);
-    final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, -1, SEGMENT, false, true, CodecProvider.getDefault().getWriter(null));
+    final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, -1, SEGMENT, false, true, codecInfo);
     si.setHasProx(false);
 
-    final FieldsProducer reader = si.getCodec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, 64, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
+    final FieldsProducer reader = si.getCodecInfo().codec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, 64, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
 
     final FieldsEnum fieldsEnum = reader.iterator();
     assertNotNull(fieldsEnum.next());
@@ -307,7 +308,8 @@
   }
 
   public void testRandomPostings() throws Throwable {
-    final FieldInfos fieldInfos = new FieldInfos();
+    final CodecInfo codecInfo = new CodecInfo(CodecProvider.getDefault());
+    final FieldInfos fieldInfos = new FieldInfos(codecInfo);
 
     final FieldData[] fields = new FieldData[NUM_FIELDS];
     for(int i=0;i<NUM_FIELDS;i++) {
@@ -319,9 +321,9 @@
     final Directory dir = newDirectory();
 
     this.write(fieldInfos, dir, fields);
-    final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, -1, SEGMENT, false, true, CodecProvider.getDefault().getWriter(null));
+    final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, -1, SEGMENT, false, true, codecInfo);
 
-    final FieldsProducer terms = si.getCodec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, 1024, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
+    final FieldsProducer terms = si.getCodecInfo().codec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, 1024, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
 
     final Verify[] threads = new Verify[NUM_TEST_THREADS-1];
     for(int i=0;i<NUM_TEST_THREADS-1;i++) {
@@ -402,13 +404,9 @@
 
     protected MockSepCodecs() {
       this.register(new MockSepCodec());
+      this.setDefaultFieldCodec("MockSep");
     }
-
-    @Override
-    public Codec getWriter(final SegmentWriteState state) {
-      return this.lookup("MockSep");
-    }
-
+    
   }
 
   private class Verify extends Thread {
@@ -612,10 +610,9 @@
 
     final int termIndexInterval = this.nextInt(13, 27);
 
-    final SegmentWriteState state = new SegmentWriteState(null, dir, SEGMENT, fieldInfos, null, 10000, 10000, termIndexInterval,
-                                                    CodecProvider.getDefault());
+    final SegmentWriteState state = new SegmentWriteState(null, dir, SEGMENT, fieldInfos, null, 10000, 10000, termIndexInterval);
 
-    final FieldsConsumer consumer = state.codec.fieldsConsumer(state);
+    final FieldsConsumer consumer = state.codecInfo.codec().fieldsConsumer(state);
     Arrays.sort(fields);
     for (final FieldData field : fields) {
       field.write(consumer);
Index: lucene/src/test/org/apache/lucene/index/TestDoc.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestDoc.java	(revision 1032543)
+++ lucene/src/test/org/apache/lucene/index/TestDoc.java	(working copy)
@@ -197,7 +197,7 @@
       merger.closeReaders();
       
       final SegmentInfo info = new SegmentInfo(merged, si1.docCount + si2.docCount, si1.dir,
-                                               useCompoundFile, -1, null, false, merger.hasProx(), merger.getCodec());
+                                               useCompoundFile, -1, null, false, merger.hasProx(), merger.getCodecInfo());
       
       if (useCompoundFile) {
         List<String> filesToDelete = merger.createCompoundFile(merged + ".cfs", info);
Index: lucene/src/test/org/apache/lucene/index/TestFieldInfos.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestFieldInfos.java	(revision 1032543)
+++ lucene/src/test/org/apache/lucene/index/TestFieldInfos.java	(working copy)
@@ -19,6 +19,7 @@
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexOutput;
 
@@ -37,9 +38,10 @@
   }
 
   public void test() throws IOException {
+    final CodecInfo codecInfo = new CodecInfo(CodecProvider.getDefault());
     //Positive test of FieldInfos
     assertTrue(testDoc != null);
-    FieldInfos fieldInfos = new FieldInfos();
+    FieldInfos fieldInfos = new FieldInfos(codecInfo);
     fieldInfos.add(testDoc);
     //Since the complement is stored as well in the fields map
     assertTrue(fieldInfos.size() == DocHelper.all.size()); //this is all b/c we are using the no-arg constructor
@@ -52,7 +54,7 @@
       fieldInfos.write(output);
       output.close();
       assertTrue(dir.fileLength(name) > 0);
-      FieldInfos readIn = new FieldInfos(dir, name);
+      FieldInfos readIn = new FieldInfos(dir, name, codecInfo);
       assertTrue(fieldInfos.size() == readIn.size());
       FieldInfo info = readIn.fieldInfo("textField1");
       assertTrue(info != null);
Index: lucene/src/test/org/apache/lucene/index/TestFieldsReader.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestFieldsReader.java	(revision 1032543)
+++ lucene/src/test/org/apache/lucene/index/TestFieldsReader.java	(working copy)
@@ -33,6 +33,7 @@
 import org.apache.lucene.document.LoadFirstFieldSelector;
 import org.apache.lucene.document.SetBasedFieldSelector;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.store.Directory;
@@ -51,7 +52,8 @@
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    fieldInfos = new FieldInfos();
+    final CodecInfo codecInfo = new CodecInfo(CodecProvider.getDefault());
+    fieldInfos = new FieldInfos(codecInfo);
     DocHelper.setupDoc(testDoc);
     fieldInfos.add(testDoc);
     dir = newDirectory();
Index: lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java	(revision 1032543)
+++ lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java	(working copy)
@@ -26,6 +26,7 @@
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.index.codecs.CodecProvider;
 
 import java.io.*;
 import java.util.*;
@@ -83,7 +84,7 @@
     // "content", and then set our expected file names below
     // accordingly:
     CompoundFileReader cfsReader = new CompoundFileReader(dir, "_2.cfs");
-    FieldInfos fieldInfos = new FieldInfos(cfsReader, "_2.fnm");
+    FieldInfos fieldInfos = new FieldInfos(cfsReader, "_2.fnm", new CodecInfo(CodecProvider.getDefault()));
     int contentFieldIndex = -1;
     for(i=0;i<fieldInfos.size();i++) {
       FieldInfo fi = fieldInfos.fieldInfo(i);
Index: lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java	(revision 1032543)
+++ lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java	(working copy)
@@ -81,7 +81,7 @@
     assertTrue(docsMerged == 2);
     //Should be able to open a new SegmentReader against the new directory
     SegmentReader mergedReader = SegmentReader.get(false, mergedDir, new SegmentInfo(mergedSegment, docsMerged, mergedDir, false, -1,
-        null, false, merger.hasProx(), merger.getCodec()), BufferedIndexInput.BUFFER_SIZE, true, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, null);
+        null, false, merger.hasProx(), merger.getCodecInfo()), BufferedIndexInput.BUFFER_SIZE, true, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR);
 
     assertTrue(mergedReader != null);
     assertTrue(mergedReader.numDocs() == 2);
Index: lucene/src/test/org/apache/lucene/index/TestTermVectorsReader.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestTermVectorsReader.java	(revision 1032543)
+++ lucene/src/test/org/apache/lucene/index/TestTermVectorsReader.java	(working copy)
@@ -31,6 +31,7 @@
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
@@ -45,7 +46,7 @@
   private TermVectorOffsetInfo[][] offsets = new TermVectorOffsetInfo[testTerms.length][];
   private Directory dir;
   private String seg;
-  private FieldInfos fieldInfos = new FieldInfos();
+  private FieldInfos fieldInfos = new FieldInfos(new CodecInfo(CodecProvider.getDefault()));
   private static int TERM_FREQ = 3;
 
   private class TestToken implements Comparable<TestToken> {
@@ -117,7 +118,7 @@
     seg = writer.newestSegment().name;
     writer.close();
 
-    fieldInfos = new FieldInfos(dir, IndexFileNames.segmentFileName(seg, "", IndexFileNames.FIELD_INFOS_EXTENSION));
+    fieldInfos = new FieldInfos(dir, IndexFileNames.segmentFileName(seg, "", IndexFileNames.FIELD_INFOS_EXTENSION), new CodecInfo(CodecProvider.getDefault()));
   }
   
   @Override
Index: lucene/src/test/org/apache/lucene/util/_TestUtil.java
===================================================================
--- lucene/src/test/org/apache/lucene/util/_TestUtil.java	(revision 1032543)
+++ lucene/src/test/org/apache/lucene/util/_TestUtil.java	(working copy)
@@ -192,13 +192,8 @@
   }
 
   public static CodecProvider alwaysCodec(final Codec c) {
-    return new CodecProvider() {
+    CodecProvider p = new CodecProvider() {
       @Override
-      public Codec getWriter(SegmentWriteState state) {
-        return c;
-      }
-
-      @Override
       public Codec lookup(String name) {
         // can't do this until we fix PreFlexRW to not
         //impersonate PreFlex:
@@ -209,6 +204,8 @@
         }
       }
     };
+    p.setDefaultFieldCodec(c.name);
+    return p;
   }
 
   /** Return a CodecProvider that can read any of the
