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/11/14 22:27:28 UTC

svn commit: r1201912 - in /lucene/dev/branches/lucene2621/lucene: contrib/misc/src/java/org/apache/lucene/index/codecs/appending/ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/index/codecs/ src/java/org/apache/lucene/index/codecs/lucene3...

Author: rmuir
Date: Mon Nov 14 21:27:27 2011
New Revision: 1201912

URL: http://svn.apache.org/viewvc?rev=1201912&view=rev
Log:
LUCENE-2621: fieldinfos -> codec

Added:
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldInfosFormat.java   (with props)
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldInfosReader.java   (with props)
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldInfosWriter.java   (with props)
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/FieldInfosFormat.java   (with props)
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/FieldInfosReader.java   (with props)
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/FieldInfosWriter.java   (with props)
Modified:
    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/DocFieldProcessor.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/FieldInfo.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/FieldInfos.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/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/Codec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xCodec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.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/TestBackwardsCompatibility.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/TestDoc.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestFieldInfos.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestTermVectorsReader.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/codecs/lucene3x/TestTermInfosReaderIndex.java

Modified: 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/AppendingCodec.java?rev=1201912&r1=1201911&r2=1201912&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/AppendingCodec.java Mon Nov 14 21:27:27 2011
@@ -19,9 +19,11 @@ package org.apache.lucene.index.codecs.a
 
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
+import org.apache.lucene.index.codecs.DefaultFieldInfosFormat;
 import org.apache.lucene.index.codecs.DefaultStoredFieldsFormat;
 import org.apache.lucene.index.codecs.DefaultTermVectorsFormat;
 import org.apache.lucene.index.codecs.DocValuesFormat;
+import org.apache.lucene.index.codecs.FieldInfosFormat;
 import org.apache.lucene.index.codecs.StoredFieldsFormat;
 import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.SegmentInfosFormat;
@@ -47,6 +49,7 @@ public class AppendingCodec extends Code
   private final PostingsFormat postings = new AppendingPostingsFormat();
   private final SegmentInfosFormat infos = new AppendingSegmentInfosFormat();
   private final StoredFieldsFormat fields = new DefaultStoredFieldsFormat();
+  private final FieldInfosFormat fieldInfos = new DefaultFieldInfosFormat();
   private final TermVectorsFormat vectors = new DefaultTermVectorsFormat();
   private final DocValuesFormat docValues = new DefaultDocValuesFormat();
   
@@ -74,4 +77,9 @@ public class AppendingCodec extends Code
   public SegmentInfosFormat segmentInfosFormat() {
     return infos;
   }
+  
+  @Override
+  public FieldInfosFormat fieldInfosFormat() {
+    return fieldInfos;
+  }
 }

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=1201912&r1=1201911&r2=1201912&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 Mon Nov 14 21:27:27 2011
@@ -25,10 +25,13 @@ 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.DocValuesFormat;
 import org.apache.lucene.index.codecs.DocValuesConsumer;
+import org.apache.lucene.index.codecs.FieldInfosWriter;
 import org.apache.lucene.index.codecs.PerDocConsumer;
 import org.apache.lucene.index.values.PerDocFieldValues;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.IOUtils;
 
@@ -45,6 +48,7 @@ final class DocFieldProcessor extends Do
 
   final DocFieldConsumer consumer;
   final StoredFieldsConsumer fieldsWriter;
+  final Codec codec;
 
   // Holds all fields seen in current doc
   DocFieldProcessorPerField[] fields = new DocFieldProcessorPerField[1];
@@ -61,6 +65,7 @@ final class DocFieldProcessor extends Do
 
   public DocFieldProcessor(DocumentsWriterPerThread docWriter, DocFieldConsumer consumer) {
     this.docState = docWriter.docState;
+    this.codec = docWriter.codec;
     this.consumer = consumer;
     fieldsWriter = new StoredFieldsConsumer(docWriter);
   }
@@ -81,8 +86,8 @@ final class DocFieldProcessor extends Do
     // consumer can alter the FieldInfo* if necessary.  EG,
     // FreqProxTermsWriter does this with
     // FieldInfo.storePayload.
-    final String fileName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.FIELD_INFOS_EXTENSION);
-    state.fieldInfos.write(state.directory, fileName);
+    FieldInfosWriter infosWriter = codec.fieldInfosFormat().getFieldInfosWriter();
+    infosWriter.write(state.directory, state.segmentName, state.fieldInfos, IOContext.DEFAULT);
     for (DocValuesConsumerAndDocID consumers : docValues.values()) {
       consumers.docValuesConsumer.finish(state.numDocs);
     }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1201912&r1=1201911&r2=1201912&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Mon Nov 14 21:27:27 2011
@@ -414,7 +414,7 @@ public class DocumentsWriterPerThread {
   private void doAfterFlush() throws IOException {
     segment = null;
     consumer.doAfterFlush();
-    fieldInfos = new FieldInfos(fieldInfos);
+    fieldInfos = FieldInfos.from(fieldInfos);
     parent.subtractFlushedNumDocs(numDocsInRAM);
     numDocsInRAM = 0;
   }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/FieldInfo.java?rev=1201912&r1=1201911&r2=1201912&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/FieldInfo.java Mon Nov 14 21:27:27 2011
@@ -51,12 +51,15 @@ public final class FieldInfo {
     DOCS_AND_FREQS_AND_POSITIONS 
   };
 
-  FieldInfo(String na, boolean tk, int nu, boolean storeTermVector, 
+  /**
+   * @lucene.experimental
+   */
+  public FieldInfo(String name, boolean isIndexed, int number, boolean storeTermVector, 
             boolean storePositionWithTermVector,  boolean storeOffsetWithTermVector, 
             boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, ValueType docValues) {
-    name = na;
-    isIndexed = tk;
-    number = nu;
+    this.name = name;
+    this.isIndexed = isIndexed;
+    this.number = number;
     this.docValues = docValues;
     if (isIndexed) {
       this.storeTermVector = storeTermVector;

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/FieldInfos.java?rev=1201912&r1=1201911&r2=1201912&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/FieldInfos.java Mon Nov 14 21:27:27 2011
@@ -31,7 +31,6 @@ import java.util.Map.Entry;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.values.ValueType;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.CodecUtil;
@@ -185,57 +184,27 @@ public final class FieldInfos implements
   private final HashMap<String,FieldInfo> byName = new HashMap<String,FieldInfo>();
   private final FieldNumberBiMap globalFieldNumbers;
   
-  // First used in 2.9; prior to 2.9 there was no format header
-  public static final int FORMAT_START = -2;
-  // First used in 3.4: omit only positional information
-  public static final int FORMAT_OMIT_POSITIONS = -3;
-  // per-field codec support, records index values for fields
-  public static final int FORMAT_FLEX = -4;
-
-  // whenever you add a new format, make it 1 smaller (negative version logic)!
-  static final int FORMAT_CURRENT = FORMAT_FLEX;
-  
-  static final int FORMAT_MINIMUM = FORMAT_START;
-  
-  static final byte IS_INDEXED = 0x1;
-  static final byte STORE_TERMVECTOR = 0x2;
-  static final byte STORE_POSITIONS_WITH_TERMVECTOR = 0x4;
-  static final byte STORE_OFFSET_WITH_TERMVECTOR = 0x8;
-  static final byte OMIT_NORMS = 0x10;
-  static final byte STORE_PAYLOADS = 0x20;
-  static final byte OMIT_TERM_FREQ_AND_POSITIONS = 0x40;
-  static final byte OMIT_POSITIONS = -128;
-
-  private int format;
   private boolean hasFreq; // only set if readonly
   private boolean hasProx; // only set if readonly
   private boolean hasVectors; // only set if readonly
   private long version; // internal use to track changes
-  
 
   /**
-   * Creates a new {@link FieldInfos} instance with a private
-   * {@link org.apache.lucene.index.FieldInfos.FieldNumberBiMap} 
-   * <p>
-   * Note: this ctor should not be used during indexing use
-   * {@link FieldInfos#FieldInfos(FieldInfos)} or
-   * {@link FieldInfos#FieldInfos(FieldNumberBiMap)}
-   * instead.
-   */
-  public FieldInfos() {
-    this(new FieldNumberBiMap());
-  }
-  
-  /**
-   * Creates a new {@link FieldInfo} instance from the given instance. If the given instance is
-   * read-only this instance will be read-only too.
+   * Creates a new read-only FieldInfos: only public to be accessible
+   * from the codecs package
    * 
-   * @see #isReadOnly()
+   * @lucene.internal
    */
-  FieldInfos(FieldInfos other) {
-    this(other.globalFieldNumbers);
+  public FieldInfos(FieldInfo[] infos, boolean hasFreq, boolean hasProx, boolean hasVectors) {
+    this(null);
+    this.hasFreq = hasFreq;
+    this.hasProx = hasProx;
+    this.hasVectors = hasVectors;
+    for (FieldInfo info : infos) {
+      putInternal(info);
+    }
   }
-  
+
   /**
    * Creates a new FieldInfos instance with the given {@link FieldNumberBiMap}. 
    * If the {@link FieldNumberBiMap} is <code>null</code> this instance will be read-only.
@@ -244,26 +213,6 @@ public final class FieldInfos implements
   FieldInfos(FieldNumberBiMap globalFieldNumbers) {
     this.globalFieldNumbers = globalFieldNumbers;
   }
-
-  /**
-   * Construct a FieldInfos object using the directory and the name of the file
-   * IndexInput. 
-   * <p>
-   * Note: The created instance will be read-only
-   * 
-   * @param d The directory to open the IndexInput from
-   * @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 {
-    this((FieldNumberBiMap)null); // use null here to make this FIs Read-Only
-    final IndexInput input = d.openInput(name, IOContext.READONCE);
-    try {
-      read(input, name);
-    } finally {
-      input.close();
-    }
-  }
   
   /**
    * adds the given field to this FieldInfos name / number mapping. The given FI
@@ -293,7 +242,6 @@ public final class FieldInfos implements
   @Override
   synchronized public Object clone() {
     FieldInfos fis = new FieldInfos(globalFieldNumbers);
-    fis.format = format;
     fis.hasFreq = hasFreq;
     fis.hasProx = hasProx;
     fis.hasVectors = hasVectors;
@@ -549,15 +497,6 @@ public final class FieldInfos implements
     return false;
   }
 
-  public void write(Directory d, String name) throws IOException {
-    IndexOutput output = d.createOutput(name, IOContext.READONCE);
-    try {
-      write(output);
-    } finally {
-      output.close();
-    }
-  }
-  
   /**
    * Returns <code>true</code> iff this instance is not backed by a
    * {@link org.apache.lucene.index.FieldInfos.FieldNumberBiMap}. Instances read from a directory via
@@ -572,182 +511,6 @@ public final class FieldInfos implements
   synchronized final long getVersion() {
     return version;
   }
-
-  public void write(IndexOutput output) throws IOException {
-    output.writeVInt(FORMAT_CURRENT);
-    output.writeVInt(size());
-    for (FieldInfo fi : this) {
-      assert fi.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS || !fi.storePayloads;
-      byte bits = 0x0;
-      if (fi.isIndexed) bits |= IS_INDEXED;
-      if (fi.storeTermVector) bits |= STORE_TERMVECTOR;
-      if (fi.storePositionWithTermVector) bits |= STORE_POSITIONS_WITH_TERMVECTOR;
-      if (fi.storeOffsetWithTermVector) bits |= STORE_OFFSET_WITH_TERMVECTOR;
-      if (fi.omitNorms) bits |= OMIT_NORMS;
-      if (fi.storePayloads) bits |= STORE_PAYLOADS;
-      if (fi.indexOptions == IndexOptions.DOCS_ONLY)
-        bits |= OMIT_TERM_FREQ_AND_POSITIONS;
-      else if (fi.indexOptions == IndexOptions.DOCS_AND_FREQS)
-        bits |= OMIT_POSITIONS;
-      output.writeString(fi.name);
-      output.writeInt(fi.number);
-      output.writeByte(bits);
-
-      final byte b;
-
-      if (fi.docValues == null) {
-        b = 0;
-      } else {
-        switch(fi.docValues) {
-        case VAR_INTS:
-          b = 1;
-          break;
-        case FLOAT_32:
-          b = 2;
-          break;
-        case FLOAT_64:
-          b = 3;
-          break;
-        case BYTES_FIXED_STRAIGHT:
-          b = 4;
-          break;
-        case BYTES_FIXED_DEREF:
-          b = 5;
-          break;
-        case BYTES_VAR_STRAIGHT:
-          b = 6;
-          break;
-        case BYTES_VAR_DEREF:
-          b = 7;
-          break;
-        case FIXED_INTS_16:
-          b = 8;
-          break;
-        case FIXED_INTS_32:
-          b = 9;
-          break;
-        case FIXED_INTS_64:
-          b = 10;
-          break;
-        case FIXED_INTS_8:
-          b = 11;
-          break;
-        case BYTES_FIXED_SORTED:
-          b = 12;
-          break;
-        case BYTES_VAR_SORTED:
-          b = 13;
-          break;
-        default:
-          throw new IllegalStateException("unhandled indexValues type " + fi.docValues);
-        }
-      }
-      output.writeByte(b);
-    }
-  }
-
-  private void read(IndexInput input, String fileName) throws IOException {
-    format = input.readVInt();
-
-    if (format > FORMAT_MINIMUM) {
-      throw new IndexFormatTooOldException(input, format, FORMAT_MINIMUM, FORMAT_CURRENT);
-    }
-    if (format < FORMAT_CURRENT) {
-      throw new IndexFormatTooNewException(input, format, FORMAT_MINIMUM, FORMAT_CURRENT);
-    }
-
-    final int size = input.readVInt(); //read in the size
-
-    for (int i = 0; i < size; i++) {
-      String name = input.readString();
-      final int fieldNumber = format <= FORMAT_FLEX? input.readInt():i;
-      byte bits = input.readByte();
-      boolean isIndexed = (bits & IS_INDEXED) != 0;
-      boolean storeTermVector = (bits & STORE_TERMVECTOR) != 0;
-      boolean storePositionsWithTermVector = (bits & STORE_POSITIONS_WITH_TERMVECTOR) != 0;
-      boolean storeOffsetWithTermVector = (bits & STORE_OFFSET_WITH_TERMVECTOR) != 0;
-      boolean omitNorms = (bits & OMIT_NORMS) != 0;
-      boolean storePayloads = (bits & STORE_PAYLOADS) != 0;
-      final IndexOptions indexOptions;
-      if ((bits & OMIT_TERM_FREQ_AND_POSITIONS) != 0) {
-        indexOptions = IndexOptions.DOCS_ONLY;
-      } else if ((bits & OMIT_POSITIONS) != 0) {
-        if (format <= FORMAT_OMIT_POSITIONS) {
-          indexOptions = IndexOptions.DOCS_AND_FREQS;
-        } else {
-          throw new CorruptIndexException("Corrupt fieldinfos, OMIT_POSITIONS set but format=" + format + " (resource: " + input + ")");
-        }
-      } else {
-        indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
-      }
-
-      // LUCENE-3027: past indices were able to write
-      // storePayloads=true when omitTFAP is also true,
-      // which is invalid.  We correct that, here:
-      if (indexOptions != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
-        storePayloads = false;
-      }
-      hasVectors |= storeTermVector;
-      hasProx |= isIndexed && indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
-      hasFreq |= isIndexed && indexOptions != IndexOptions.DOCS_ONLY;
-      ValueType docValuesType = null;
-      if (format <= FORMAT_FLEX) {
-        final byte b = input.readByte();
-        switch(b) {
-        case 0:
-          docValuesType = null;
-          break;
-        case 1:
-          docValuesType = ValueType.VAR_INTS;
-          break;
-        case 2:
-          docValuesType = ValueType.FLOAT_32;
-          break;
-        case 3:
-          docValuesType = ValueType.FLOAT_64;
-          break;
-        case 4:
-          docValuesType = ValueType.BYTES_FIXED_STRAIGHT;
-          break;
-        case 5:
-          docValuesType = ValueType.BYTES_FIXED_DEREF;
-          break;
-        case 6:
-          docValuesType = ValueType.BYTES_VAR_STRAIGHT;
-          break;
-        case 7:
-          docValuesType = ValueType.BYTES_VAR_DEREF;
-          break;
-        case 8:
-          docValuesType = ValueType.FIXED_INTS_16;
-          break;
-        case 9:
-          docValuesType = ValueType.FIXED_INTS_32;
-          break;
-        case 10:
-          docValuesType = ValueType.FIXED_INTS_64;
-          break;
-        case 11:
-          docValuesType = ValueType.FIXED_INTS_8;
-          break;
-        case 12:
-          docValuesType = ValueType.BYTES_FIXED_SORTED;
-          break;
-        case 13:
-          docValuesType = ValueType.BYTES_VAR_SORTED;
-          break;
-        
-        default:
-          throw new IllegalStateException("unhandled indexValues type " + b);
-        }
-      }
-      addInternal(name, fieldNumber, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions, docValuesType);
-    }
-
-    if (input.getFilePointer() != input.length()) {
-      throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")");
-    }    
-  }
   
   /**
    * Reverts all uncommitted changes 
@@ -783,4 +546,14 @@ public final class FieldInfos implements
 
     return false;
   }
+  
+  /**
+   * Creates a new {@link FieldInfo} instance from the given instance. If the given instance is
+   * read-only this instance will be read-only too.
+   * 
+   * @see #isReadOnly()
+   */
+  static FieldInfos from(FieldInfos other) {
+    return new FieldInfos(other.globalFieldNumbers);
+  }
 }

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=1201912&r1=1201911&r2=1201912&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 Mon Nov 14 21:27:27 2011
@@ -65,9 +65,6 @@ public final class IndexFileNames {
   /** Extension of deletes */
   public static final String DELETES_EXTENSION = "del";
 
-  /** Extension of field infos */
-  public static final String FIELD_INFOS_EXTENSION = "fnm";
-
   /** Extension of separate norms */
   public static final String SEPARATE_NORMS_EXTENSION = "s";
 
@@ -84,7 +81,6 @@ public final class IndexFileNames {
   public static final String INDEX_EXTENSIONS[] = new String[] {
     COMPOUND_FILE_EXTENSION,
     COMPOUND_FILE_ENTRIES_EXTENSION,
-    FIELD_INFOS_EXTENSION,
     DELETES_EXTENSION,
     GEN_EXTENSION,
     NORMS_EXTENSION,
@@ -93,13 +89,8 @@ public final class IndexFileNames {
   };
 
   public static final String[] NON_STORE_INDEX_EXTENSIONS = new String[] {
-    FIELD_INFOS_EXTENSION,
     NORMS_EXTENSION
   };
-  
-  static final String COMPOUND_EXTENSIONS_NOT_CODEC[] = new String[] {
-    FIELD_INFOS_EXTENSION,
-  };
 
   /**
    * Computes the full file name from base, extension and generation. If the

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=1201912&r1=1201911&r2=1201912&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 Mon Nov 14 21:27:27 2011
@@ -28,6 +28,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.FieldInfosReader;
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -198,8 +199,8 @@ public final class SegmentInfo implement
             "", IndexFileNames.COMPOUND_FILE_EXTENSION), IOContext.READONCE, false);
       }
       try {
-        fieldInfos = new FieldInfos(dir0, IndexFileNames.segmentFileName(name,
-            "", IndexFileNames.FIELD_INFOS_EXTENSION));
+        FieldInfosReader reader = codec.fieldInfosFormat().getFieldInfosReader();
+        fieldInfos = reader.read(dir0, name, IOContext.READONCE);
       } finally {
         if (dir != dir0) {
           dir0.close();

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=1201912&r1=1201911&r2=1201912&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 Mon Nov 14 21:27:27 2011
@@ -26,6 +26,7 @@ import java.util.List;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.IndexReader.FieldOption;
 import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.FieldInfosWriter;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.StoredFieldsWriter;
 import org.apache.lucene.index.codecs.PerDocConsumer;
@@ -127,7 +128,8 @@ final class SegmentMerger {
       assert numMerged == mergeState.mergedDocCount;
     }
     // write FIS once merge is done. IDV might change types or drops fields
-    mergeState.fieldInfos.write(directory, segment + "." + IndexFileNames.FIELD_INFOS_EXTENSION);
+    FieldInfosWriter fieldInfosWriter = codec.fieldInfosFormat().getFieldInfosWriter();
+    fieldInfosWriter.write(directory, segment, mergeState.fieldInfos, context);
     return mergeState;
   }
 

Modified: 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/Codec.java?rev=1201912&r1=1201911&r2=1201912&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/Codec.java Mon Nov 14 21:27:27 2011
@@ -47,6 +47,7 @@ public abstract class Codec implements N
     postingsFormat().files(dir, info, "", files);
     storedFieldsFormat().files(dir, info, files);
     termVectorsFormat().files(dir, info, files);
+    fieldInfosFormat().files(dir, info, files);
     // TODO: segmentInfosFormat should be allowed to declare additional files
     // if it wants, in addition to segments_N
     docValuesFormat().files(dir, info, files);
@@ -64,6 +65,9 @@ public abstract class Codec implements N
   /** Encodes/decodes term vectors */
   public abstract TermVectorsFormat termVectorsFormat();
   
+  /** Encodes/decodes field infos file */
+  public abstract FieldInfosFormat fieldInfosFormat();
+  
   /** Encodes/decodes segments file */
   public abstract SegmentInfosFormat segmentInfosFormat();
   

Added: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldInfosFormat.java?rev=1201912&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldInfosFormat.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldInfosFormat.java Mon Nov 14 21:27:27 2011
@@ -0,0 +1,47 @@
+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.io.IOException;
+import java.util.Set;
+
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+
+/**
+ * @lucene.experimental
+ */
+public class DefaultFieldInfosFormat extends FieldInfosFormat {
+  private final FieldInfosReader reader = new DefaultFieldInfosReader();
+  private final FieldInfosWriter writer = new DefaultFieldInfosWriter();
+  
+  @Override
+  public FieldInfosReader getFieldInfosReader() throws IOException {
+    return reader;
+  }
+
+  @Override
+  public FieldInfosWriter getFieldInfosWriter() throws IOException {
+    return writer;
+  }
+
+  @Override
+  public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
+    DefaultFieldInfosReader.files(dir, info, files);
+  }
+}

Added: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldInfosReader.java?rev=1201912&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldInfosReader.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldInfosReader.java Mon Nov 14 21:27:27 2011
@@ -0,0 +1,166 @@
+package org.apache.lucene.index.codecs;
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexFormatTooNewException;
+import org.apache.lucene.index.IndexFormatTooOldException;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.values.ValueType;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+
+/**
+ * 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.
+ */
+
+/**
+ * @lucene.experimental
+ */
+public class DefaultFieldInfosReader extends FieldInfosReader {
+
+  static final int FORMAT_MINIMUM = DefaultFieldInfosWriter.FORMAT_START;
+
+  @Override
+  public FieldInfos read(Directory directory, String segmentName, IOContext iocontext) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segmentName, "", DefaultFieldInfosWriter.FIELD_INFOS_EXTENSION);
+    IndexInput input = directory.openInput(fileName, iocontext);
+
+    boolean hasVectors = false;
+    boolean hasFreq = false;
+    boolean hasProx = false;
+    
+    try {
+      final int format = input.readVInt();
+
+      if (format > FORMAT_MINIMUM) {
+        throw new IndexFormatTooOldException(input, format, FORMAT_MINIMUM, DefaultFieldInfosWriter.FORMAT_CURRENT);
+      }
+      if (format < DefaultFieldInfosWriter.FORMAT_CURRENT) {
+        throw new IndexFormatTooNewException(input, format, FORMAT_MINIMUM, DefaultFieldInfosWriter.FORMAT_CURRENT);
+      }
+
+      final int size = input.readVInt(); //read in the size
+      FieldInfo infos[] = new FieldInfo[size];
+
+      for (int i = 0; i < size; i++) {
+        String name = input.readString();
+        final int fieldNumber = format <= DefaultFieldInfosWriter.FORMAT_FLEX? input.readInt():i;
+        byte bits = input.readByte();
+        boolean isIndexed = (bits & DefaultFieldInfosWriter.IS_INDEXED) != 0;
+        boolean storeTermVector = (bits & DefaultFieldInfosWriter.STORE_TERMVECTOR) != 0;
+        boolean storePositionsWithTermVector = (bits & DefaultFieldInfosWriter.STORE_POSITIONS_WITH_TERMVECTOR) != 0;
+        boolean storeOffsetWithTermVector = (bits & DefaultFieldInfosWriter.STORE_OFFSET_WITH_TERMVECTOR) != 0;
+        boolean omitNorms = (bits & DefaultFieldInfosWriter.OMIT_NORMS) != 0;
+        boolean storePayloads = (bits & DefaultFieldInfosWriter.STORE_PAYLOADS) != 0;
+        final IndexOptions indexOptions;
+        if ((bits & DefaultFieldInfosWriter.OMIT_TERM_FREQ_AND_POSITIONS) != 0) {
+          indexOptions = IndexOptions.DOCS_ONLY;
+        } else if ((bits & DefaultFieldInfosWriter.OMIT_POSITIONS) != 0) {
+          if (format <= DefaultFieldInfosWriter.FORMAT_OMIT_POSITIONS) {
+            indexOptions = IndexOptions.DOCS_AND_FREQS;
+          } else {
+            throw new CorruptIndexException("Corrupt fieldinfos, OMIT_POSITIONS set but format=" + format + " (resource: " + input + ")");
+          }
+        } else {
+          indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
+        }
+
+        // LUCENE-3027: past indices were able to write
+        // storePayloads=true when omitTFAP is also true,
+        // which is invalid.  We correct that, here:
+        if (indexOptions != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
+          storePayloads = false;
+        }
+        hasVectors |= storeTermVector;
+        hasProx |= isIndexed && indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
+        hasFreq |= isIndexed && indexOptions != IndexOptions.DOCS_ONLY;
+        ValueType docValuesType = null;
+        if (format <= DefaultFieldInfosWriter.FORMAT_FLEX) {
+          final byte b = input.readByte();
+          switch(b) {
+            case 0:
+              docValuesType = null;
+              break;
+            case 1:
+              docValuesType = ValueType.VAR_INTS;
+              break;
+            case 2:
+              docValuesType = ValueType.FLOAT_32;
+              break;
+            case 3:
+              docValuesType = ValueType.FLOAT_64;
+              break;
+            case 4:
+              docValuesType = ValueType.BYTES_FIXED_STRAIGHT;
+              break;
+            case 5:
+              docValuesType = ValueType.BYTES_FIXED_DEREF;
+              break;
+            case 6:
+              docValuesType = ValueType.BYTES_VAR_STRAIGHT;
+              break;
+            case 7:
+              docValuesType = ValueType.BYTES_VAR_DEREF;
+              break;
+            case 8:
+              docValuesType = ValueType.FIXED_INTS_16;
+              break;
+            case 9:
+              docValuesType = ValueType.FIXED_INTS_32;
+              break;
+            case 10:
+              docValuesType = ValueType.FIXED_INTS_64;
+              break;
+            case 11:
+              docValuesType = ValueType.FIXED_INTS_8;
+              break;
+            case 12:
+              docValuesType = ValueType.BYTES_FIXED_SORTED;
+              break;
+            case 13:
+              docValuesType = ValueType.BYTES_VAR_SORTED;
+              break;
+        
+            default:
+              throw new IllegalStateException("unhandled indexValues type " + b);
+          }
+        }
+        infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, 
+          storePositionsWithTermVector, storeOffsetWithTermVector, 
+          omitNorms, storePayloads, indexOptions, docValuesType);
+      }
+
+      if (input.getFilePointer() != input.length()) {
+        throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")");
+      }
+      
+      return new FieldInfos(infos, hasFreq, hasProx, hasVectors);
+    } finally {
+      input.close();
+    }
+  }
+  
+  public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
+    files.add(IndexFileNames.segmentFileName(info.name, "", DefaultFieldInfosWriter.FIELD_INFOS_EXTENSION));
+  }
+}

Added: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldInfosWriter.java?rev=1201912&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldInfosWriter.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldInfosWriter.java Mon Nov 14 21:27:27 2011
@@ -0,0 +1,136 @@
+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.io.IOException;
+
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+
+/**
+ * @lucene.experimental
+ */
+public class DefaultFieldInfosWriter extends FieldInfosWriter {
+  
+  /** Extension of field infos */
+  static final String FIELD_INFOS_EXTENSION = "fnm";
+  
+  // First used in 2.9; prior to 2.9 there was no format header
+  static final int FORMAT_START = -2;
+  // First used in 3.4: omit only positional information
+  static final int FORMAT_OMIT_POSITIONS = -3;
+  // per-field codec support, records index values for fields
+  static final int FORMAT_FLEX = -4;
+
+  // whenever you add a new format, make it 1 smaller (negative version logic)!
+  static final int FORMAT_CURRENT = FORMAT_FLEX;
+  
+  static final byte IS_INDEXED = 0x1;
+  static final byte STORE_TERMVECTOR = 0x2;
+  static final byte STORE_POSITIONS_WITH_TERMVECTOR = 0x4;
+  static final byte STORE_OFFSET_WITH_TERMVECTOR = 0x8;
+  static final byte OMIT_NORMS = 0x10;
+  static final byte STORE_PAYLOADS = 0x20;
+  static final byte OMIT_TERM_FREQ_AND_POSITIONS = 0x40;
+  static final byte OMIT_POSITIONS = -128;
+  
+  @Override
+  public void write(Directory directory, String segmentName, FieldInfos infos, IOContext context) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segmentName, "", FIELD_INFOS_EXTENSION);
+    IndexOutput output = directory.createOutput(fileName, context);
+    try {
+      output.writeVInt(FORMAT_CURRENT);
+      output.writeVInt(infos.size());
+      for (FieldInfo fi : infos) {
+        assert fi.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS || !fi.storePayloads;
+        byte bits = 0x0;
+        if (fi.isIndexed) bits |= IS_INDEXED;
+        if (fi.storeTermVector) bits |= STORE_TERMVECTOR;
+        if (fi.storePositionWithTermVector) bits |= STORE_POSITIONS_WITH_TERMVECTOR;
+        if (fi.storeOffsetWithTermVector) bits |= STORE_OFFSET_WITH_TERMVECTOR;
+        if (fi.omitNorms) bits |= OMIT_NORMS;
+        if (fi.storePayloads) bits |= STORE_PAYLOADS;
+        if (fi.indexOptions == IndexOptions.DOCS_ONLY)
+          bits |= OMIT_TERM_FREQ_AND_POSITIONS;
+        else if (fi.indexOptions == IndexOptions.DOCS_AND_FREQS)
+          bits |= OMIT_POSITIONS;
+        output.writeString(fi.name);
+        output.writeInt(fi.number);
+        output.writeByte(bits);
+
+        final byte b;
+
+        if (!fi.hasDocValues()) {
+          b = 0;
+        } else {
+          switch(fi.getDocValues()) {
+          case VAR_INTS:
+            b = 1;
+            break;
+          case FLOAT_32:
+            b = 2;
+            break;
+          case FLOAT_64:
+            b = 3;
+            break;
+          case BYTES_FIXED_STRAIGHT:
+            b = 4;
+            break;
+          case BYTES_FIXED_DEREF:
+            b = 5;
+            break;
+          case BYTES_VAR_STRAIGHT:
+            b = 6;
+            break;
+          case BYTES_VAR_DEREF:
+            b = 7;
+            break;
+          case FIXED_INTS_16:
+            b = 8;
+            break;
+          case FIXED_INTS_32:
+            b = 9;
+            break;
+          case FIXED_INTS_64:
+            b = 10;
+            break;
+          case FIXED_INTS_8:
+            b = 11;
+            break;
+          case BYTES_FIXED_SORTED:
+            b = 12;
+            break;
+          case BYTES_VAR_SORTED:
+            b = 13;
+            break;
+          default:
+            throw new IllegalStateException("unhandled indexValues type " + fi.getDocValues());
+          }
+        }
+        output.writeByte(b);
+      }
+    } finally {
+      output.close();
+    }
+  }
+  
+}

Added: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/FieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/FieldInfosFormat.java?rev=1201912&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/FieldInfosFormat.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/FieldInfosFormat.java Mon Nov 14 21:27:27 2011
@@ -0,0 +1,33 @@
+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.io.IOException;
+import java.util.Set;
+
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+
+/**
+ * @lucene.experimental
+ */
+public abstract class FieldInfosFormat {
+  public abstract FieldInfosReader getFieldInfosReader() throws IOException;
+  public abstract FieldInfosWriter getFieldInfosWriter() throws IOException;
+  public abstract void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException;
+}

Added: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/FieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/FieldInfosReader.java?rev=1201912&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/FieldInfosReader.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/FieldInfosReader.java Mon Nov 14 21:27:27 2011
@@ -0,0 +1,31 @@
+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.io.IOException;
+
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+/**
+ * @lucene.experimental
+ */
+public abstract class FieldInfosReader {
+  public abstract FieldInfos read(Directory directory, String segmentName, IOContext iocontext) throws IOException;
+}

Added: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/FieldInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/FieldInfosWriter.java?rev=1201912&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/FieldInfosWriter.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/FieldInfosWriter.java Mon Nov 14 21:27:27 2011
@@ -0,0 +1,31 @@
+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.io.IOException;
+
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+/**
+ * @lucene.experimental
+ */
+public abstract class FieldInfosWriter {
+  public abstract void write(Directory directory, String segmentName, FieldInfos infos, IOContext context) throws IOException;
+}

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xCodec.java?rev=1201912&r1=1201911&r2=1201912&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xCodec.java Mon Nov 14 21:27:27 2011
@@ -24,10 +24,12 @@ import org.apache.lucene.index.PerDocWri
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.DefaultFieldInfosFormat;
 import org.apache.lucene.index.codecs.DefaultStoredFieldsFormat;
 import org.apache.lucene.index.codecs.DefaultSegmentInfosFormat;
 import org.apache.lucene.index.codecs.DefaultTermVectorsFormat;
 import org.apache.lucene.index.codecs.DocValuesFormat;
+import org.apache.lucene.index.codecs.FieldInfosFormat;
 import org.apache.lucene.index.codecs.StoredFieldsFormat;
 import org.apache.lucene.index.codecs.PerDocConsumer;
 import org.apache.lucene.index.codecs.PerDocValues;
@@ -53,6 +55,9 @@ public class Lucene3xCodec extends Codec
   private final TermVectorsFormat vectorsFormat = new DefaultTermVectorsFormat();
   
   // TODO: this should really be a different impl
+  private final FieldInfosFormat fieldInfosFormat = new DefaultFieldInfosFormat();
+
+  // TODO: this should really be a different impl
   // also if we want preflex to *really* be read-only it should throw exception for the writer?
   // this way IR.commit fails on delete/undelete/setNorm/etc ?
   private final SegmentInfosFormat infosFormat = new DefaultSegmentInfosFormat();
@@ -92,6 +97,11 @@ public class Lucene3xCodec extends Codec
   public TermVectorsFormat termVectorsFormat() {
     return vectorsFormat;
   }
+  
+  @Override
+  public FieldInfosFormat fieldInfosFormat() {
+    return fieldInfosFormat;
+  }
 
   @Override
   public SegmentInfosFormat segmentInfosFormat() {

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java?rev=1201912&r1=1201911&r2=1201912&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java Mon Nov 14 21:27:27 2011
@@ -19,10 +19,12 @@ package org.apache.lucene.index.codecs.l
 
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
+import org.apache.lucene.index.codecs.DefaultFieldInfosFormat;
 import org.apache.lucene.index.codecs.DefaultStoredFieldsFormat;
 import org.apache.lucene.index.codecs.DefaultSegmentInfosFormat;
 import org.apache.lucene.index.codecs.DefaultTermVectorsFormat;
 import org.apache.lucene.index.codecs.DocValuesFormat;
+import org.apache.lucene.index.codecs.FieldInfosFormat;
 import org.apache.lucene.index.codecs.StoredFieldsFormat;
 import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.SegmentInfosFormat;
@@ -43,6 +45,7 @@ public class Lucene40Codec extends Codec
   private final StoredFieldsFormat fieldsFormat = new DefaultStoredFieldsFormat();
   private final TermVectorsFormat vectorsFormat = new DefaultTermVectorsFormat();
   private final DocValuesFormat docValuesFormat = new DefaultDocValuesFormat();
+  private final FieldInfosFormat fieldInfosFormat = new DefaultFieldInfosFormat();
   private final SegmentInfosFormat infosFormat = new DefaultSegmentInfosFormat();
   private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
     @Override
@@ -76,6 +79,11 @@ public class Lucene40Codec extends Codec
   }
   
   @Override
+  public FieldInfosFormat fieldInfosFormat() {
+    return fieldInfosFormat;
+  }
+  
+  @Override
   public SegmentInfosFormat segmentInfosFormat() {
     return infosFormat;
   }

Modified: 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/SimpleTextCodec.java?rev=1201912&r1=1201911&r2=1201912&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/SimpleTextCodec.java Mon Nov 14 21:27:27 2011
@@ -19,8 +19,10 @@ package org.apache.lucene.index.codecs.s
 
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
+import org.apache.lucene.index.codecs.DefaultFieldInfosFormat;
 import org.apache.lucene.index.codecs.DefaultTermVectorsFormat;
 import org.apache.lucene.index.codecs.DocValuesFormat;
+import org.apache.lucene.index.codecs.FieldInfosFormat;
 import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.SegmentInfosFormat;
 import org.apache.lucene.index.codecs.StoredFieldsFormat;
@@ -39,6 +41,8 @@ public final class SimpleTextCodec exten
   
   // nocommit: need a plain-text impl
   private final TermVectorsFormat vectorsFormat = new DefaultTermVectorsFormat();
+  // nocommit: need a plain-text impl
+  private final FieldInfosFormat fieldInfosFormat = new DefaultFieldInfosFormat();
   // TODO: need a plain-text impl
   private final DocValuesFormat docValues = new DefaultDocValuesFormat();
   
@@ -65,6 +69,11 @@ public final class SimpleTextCodec exten
   public TermVectorsFormat termVectorsFormat() {
     return vectorsFormat;
   }
+  
+  @Override
+  public FieldInfosFormat fieldInfosFormat() {
+    return fieldInfosFormat;
+  }
 
   @Override
   public SegmentInfosFormat segmentInfosFormat() {

Modified: lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1201912&r1=1201911&r2=1201912&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java Mon Nov 14 21:27:27 2011
@@ -32,10 +32,12 @@ import org.apache.lucene.document.TextFi
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
+import org.apache.lucene.index.codecs.DefaultFieldInfosFormat;
 import org.apache.lucene.index.codecs.DefaultStoredFieldsFormat;
 import org.apache.lucene.index.codecs.DefaultSegmentInfosFormat;
 import org.apache.lucene.index.codecs.DefaultTermVectorsFormat;
 import org.apache.lucene.index.codecs.DocValuesFormat;
+import org.apache.lucene.index.codecs.FieldInfosFormat;
 import org.apache.lucene.index.codecs.StoredFieldsFormat;
 import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.SegmentInfosFormat;
@@ -1167,6 +1169,11 @@ public class TestAddIndexes extends Luce
     public TermVectorsFormat termVectorsFormat() {
       return new DefaultTermVectorsFormat();
     }
+    
+    @Override
+    public FieldInfosFormat fieldInfosFormat() {
+      return new DefaultFieldInfosFormat();
+    }
 
     @Override
     public SegmentInfosFormat segmentInfosFormat() {

Modified: lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1201912&r1=1201911&r2=1201912&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Mon Nov 14 21:27:27 2011
@@ -35,6 +35,8 @@ import org.apache.lucene.document.String
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.FieldInfosReader;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.IndexSearcher;
@@ -44,6 +46,7 @@ import org.apache.lucene.search.TermQuer
 import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -550,7 +553,8 @@ public class TestBackwardsCompatibility 
       // "content", and then set our expected file names below
       // accordingly:
       CompoundFileDirectory cfsReader = new CompoundFileDirectory(dir, "_0.cfs", newIOContext(random), false);
-      FieldInfos fieldInfos = new FieldInfos(cfsReader, "_0.fnm");
+      FieldInfosReader infosReader = Codec.getDefault().fieldInfosFormat().getFieldInfosReader();
+      FieldInfos fieldInfos = infosReader.read(cfsReader, "_0", IOContext.READONCE);
       int contentFieldIndex = -1;
       for (FieldInfo fi : fieldInfos) {
         if (fi.name.equals("content")) {

Modified: lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestCodecs.java?rev=1201912&r1=1201911&r2=1201912&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestCodecs.java Mon Nov 14 21:27:27 2011
@@ -249,7 +249,7 @@ public class TestCodecs extends LuceneTe
       terms[i] = new TermData(text, docs, null);
     }
 
-    final FieldInfos fieldInfos = new FieldInfos();
+    final FieldInfos fieldInfos = new FieldInfos(new FieldInfos.FieldNumberBiMap());
 
     final FieldData field = new FieldData("field", fieldInfos, terms, true, false);
     final FieldData[] fields = new FieldData[] {field};
@@ -296,7 +296,7 @@ public class TestCodecs extends LuceneTe
   }
 
   public void testRandomPostings() throws Throwable {
-    final FieldInfos fieldInfos = new FieldInfos();
+    final FieldInfos fieldInfos = new FieldInfos(new FieldInfos.FieldNumberBiMap());
 
     final FieldData[] fields = new FieldData[NUM_FIELDS];
     for(int i=0;i<NUM_FIELDS;i++) {

Modified: lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestDoc.java?rev=1201912&r1=1201911&r2=1201912&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestDoc.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestDoc.java Mon Nov 14 21:27:27 2011
@@ -200,7 +200,7 @@ public class TestDoc extends LuceneTestC
       SegmentReader r2 = SegmentReader.get(true, si2, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, context);
 
       final Codec codec = Codec.getDefault();
-      SegmentMerger merger = new SegmentMerger(InfoStream.getDefault(), si1.dir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL, merged, MergeState.CheckAbort.NONE, null, new FieldInfos(), codec, context);
+      SegmentMerger merger = new SegmentMerger(InfoStream.getDefault(), si1.dir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL, merged, MergeState.CheckAbort.NONE, null, new FieldInfos(new FieldInfos.FieldNumberBiMap()), codec, context);
 
       merger.add(r1);
       merger.add(r2);

Modified: lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestFieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestFieldInfos.java?rev=1201912&r1=1201911&r2=1201912&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestFieldInfos.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestFieldInfos.java Mon Nov 14 21:27:27 2011
@@ -21,8 +21,13 @@ import org.apache.lucene.util.LuceneTest
 import org.apache.lucene.util._TestUtil;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.FieldInfosReader;
+import org.apache.lucene.index.codecs.FieldInfosWriter;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
+import org.junit.Ignore;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -42,7 +47,7 @@ public class TestFieldInfos extends Luce
   public FieldInfos createAndWriteFieldInfos(Directory dir, String filename) throws IOException{
   //Positive test of FieldInfos
     assertTrue(testDoc != null);
-    FieldInfos fieldInfos = new FieldInfos();
+    FieldInfos fieldInfos = new FieldInfos(new FieldInfos.FieldNumberBiMap());
     _TestUtil.add(testDoc, fieldInfos);
     //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,16 +57,23 @@ public class TestFieldInfos extends Luce
     assertTrue(output != null);
     //Use a RAMOutputStream
   
-    fieldInfos.write(output);
+    FieldInfosWriter writer = Codec.getDefault().fieldInfosFormat().getFieldInfosWriter();
+    writer.write(dir, filename, fieldInfos, IOContext.DEFAULT);
     output.close();
     return fieldInfos;
   }
+  
+  public FieldInfos readFieldInfos(Directory dir, String filename) throws IOException {
+    FieldInfosReader reader = Codec.getDefault().fieldInfosFormat().getFieldInfosReader();
+    return reader.read(dir, filename, IOContext.DEFAULT);
+  }
+  
   public void test() throws IOException {
     String name = "testFile";
     Directory dir = newDirectory();
     FieldInfos fieldInfos = createAndWriteFieldInfos(dir, name);
-    assertTrue(dir.fileLength(name) > 0);
-    FieldInfos readIn = new FieldInfos(dir, name);
+
+    FieldInfos readIn = readFieldInfos(dir, name);
     assertTrue(fieldInfos.size() == readIn.size());
     FieldInfo info = readIn.fieldInfo("textField1");
     assertTrue(info != null);
@@ -90,7 +102,7 @@ public class TestFieldInfos extends Luce
     String name = "testFile";
     Directory dir = newDirectory();
     FieldInfos fieldInfos = createAndWriteFieldInfos(dir, name);
-    FieldInfos readOnly = new FieldInfos(dir, name);
+    FieldInfos readOnly = readFieldInfos(dir, name);
     assertReadOnly(readOnly, fieldInfos);
     FieldInfos readOnlyClone = (FieldInfos)readOnly.clone();
     assertNotSame(readOnly, readOnlyClone);

Modified: lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java?rev=1201912&r1=1201911&r2=1201912&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java Mon Nov 14 21:27:27 2011
@@ -48,7 +48,7 @@ public class TestFieldsReader extends Lu
 
   @BeforeClass
   public static void beforeClass() throws Exception {
-    fieldInfos = new FieldInfos();
+    fieldInfos = new FieldInfos(new FieldInfos.FieldNumberBiMap());
     DocHelper.setupDoc(testDoc);
     _TestUtil.add(testDoc, fieldInfos);
     dir = newDirectory();

Modified: lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java?rev=1201912&r1=1201911&r2=1201912&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java Mon Nov 14 21:27:27 2011
@@ -25,9 +25,12 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.FieldInfosReader;
 import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.MockDirectoryWrapper;
@@ -91,7 +94,8 @@ public class TestIndexFileDeleter extend
     // "content", and then set our expected file names below
     // accordingly:
     CompoundFileDirectory cfsReader = new CompoundFileDirectory(dir, "_2.cfs", newIOContext(random), false);
-    FieldInfos fieldInfos = new FieldInfos(cfsReader, "_2.fnm");
+    FieldInfosReader infosReader = Codec.getDefault().fieldInfosFormat().getFieldInfosReader();
+    FieldInfos fieldInfos = infosReader.read(cfsReader, "2", IOContext.READONCE);
     int contentFieldIndex = -1;
     for (FieldInfo fi : fieldInfos) {
       if (fi.name.equals("content")) {

Modified: lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java?rev=1201912&r1=1201911&r2=1201912&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java Mon Nov 14 21:27:27 2011
@@ -77,7 +77,7 @@ public class TestSegmentMerger extends L
 
   public void testMerge() throws IOException {
     final Codec codec = Codec.getDefault();
-    SegmentMerger merger = new SegmentMerger(InfoStream.getDefault(), mergedDir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL, mergedSegment, MergeState.CheckAbort.NONE, null, new FieldInfos(), codec, newIOContext(random));
+    SegmentMerger merger = new SegmentMerger(InfoStream.getDefault(), mergedDir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL, mergedSegment, MergeState.CheckAbort.NONE, null, new FieldInfos(new FieldInfos.FieldNumberBiMap()), codec, newIOContext(random));
     merger.add(reader1);
     merger.add(reader2);
     MergeState mergeState = merger.merge();

Modified: lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestTermVectorsReader.java?rev=1201912&r1=1201911&r2=1201912&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestTermVectorsReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestTermVectorsReader.java Mon Nov 14 21:27:27 2011
@@ -44,7 +44,7 @@ public class TestTermVectorsReader exten
   private int[][] positions = new int[testTerms.length][];
   private Directory dir;
   private SegmentInfo seg;
-  private FieldInfos fieldInfos = new FieldInfos();
+  private FieldInfos fieldInfos = new FieldInfos(new FieldInfos.FieldNumberBiMap());
   private static int TERM_FREQ = 3;
 
   private class TestToken implements Comparable<TestToken> {
@@ -124,7 +124,7 @@ public class TestTermVectorsReader exten
     seg = writer.newestSegment();
     writer.close();
 
-    fieldInfos = new FieldInfos(dir, IndexFileNames.segmentFileName(seg.name, "", IndexFileNames.FIELD_INFOS_EXTENSION));
+    fieldInfos = seg.getFieldInfos(); //new FieldInfos(dir, IndexFileNames.segmentFileName(seg.name, "", IndexFileNames.FIELD_INFOS_EXTENSION));
   }
   
   @Override

Modified: lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/codecs/lucene3x/TestTermInfosReaderIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/codecs/lucene3x/TestTermInfosReaderIndex.java?rev=1201912&r1=1201911&r2=1201912&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/codecs/lucene3x/TestTermInfosReaderIndex.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/codecs/lucene3x/TestTermInfosReaderIndex.java Mon Nov 14 21:27:27 2011
@@ -40,11 +40,14 @@ import org.apache.lucene.index.SegmentRe
 import org.apache.lucene.index.Term;
 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.FieldInfosReader;
 import org.apache.lucene.index.codecs.preflexrw.PreFlexRWCodec;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.util.LuceneTestCase;
@@ -81,7 +84,8 @@ public class TestTermInfosReaderIndex ex
     String segment = r.getSegmentName();
     r.close();
 
-    FieldInfos fieldInfos = new FieldInfos(directory, IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELD_INFOS_EXTENSION));
+    FieldInfosReader infosReader = Codec.getDefault().fieldInfosFormat().getFieldInfosReader();
+    FieldInfos fieldInfos = infosReader.read(directory, segment, IOContext.READONCE);
     String segmentFileName = IndexFileNames.segmentFileName(segment, "", Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION);
     long tiiFileLength = directory.fileLength(segmentFileName);
     IndexInput input = directory.openInput(segmentFileName, newIOContext(random));