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 2014/10/15 03:26:31 UTC

svn commit: r1631928 [2/5] - in /lucene/dev/branches/lucene5969: ./ dev-tools/ dev-tools/idea/.idea/libraries/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ lucene/analysis/common/...

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java Wed Oct 15 01:26:26 2014
@@ -18,10 +18,23 @@ package org.apache.lucene.codecs.lucene4
  */
 
 import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
 
+import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.FieldInfosFormat;
-import org.apache.lucene.codecs.FieldInfosReader;
-import org.apache.lucene.codecs.FieldInfosWriter;
+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.SegmentInfo;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
 
 /**
  * Lucene 4.6 Field Infos format.
@@ -29,21 +42,139 @@ import org.apache.lucene.codecs.FieldInf
  */
 @Deprecated
 public final class Lucene46FieldInfosFormat extends FieldInfosFormat {
-  private final FieldInfosReader reader = new Lucene46FieldInfosReader();
-  private final FieldInfosWriter writer = new Lucene46FieldInfosWriter();
   
   /** Sole constructor. */
   public Lucene46FieldInfosFormat() {
   }
 
   @Override
-  public final FieldInfosReader getFieldInfosReader() throws IOException {
-    return reader;
+  public FieldInfos read(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, IOContext context) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene46FieldInfosFormat.EXTENSION);
+    try (ChecksumIndexInput input = directory.openChecksumInput(fileName, context)) {
+      int codecVersion = CodecUtil.checkHeader(input, Lucene46FieldInfosFormat.CODEC_NAME, 
+                                                      Lucene46FieldInfosFormat.FORMAT_START, 
+                                                      Lucene46FieldInfosFormat.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 = input.readVInt();
+        if (fieldNumber < 0) {
+          throw new CorruptIndexException("invalid field number for field: " + name + ", fieldNumber=" + fieldNumber, input);
+        }
+        byte bits = input.readByte();
+        boolean isIndexed = (bits & Lucene46FieldInfosFormat.IS_INDEXED) != 0;
+        boolean storeTermVector = (bits & Lucene46FieldInfosFormat.STORE_TERMVECTOR) != 0;
+        boolean omitNorms = (bits & Lucene46FieldInfosFormat.OMIT_NORMS) != 0;
+        boolean storePayloads = (bits & Lucene46FieldInfosFormat.STORE_PAYLOADS) != 0;
+        final IndexOptions indexOptions;
+        if (!isIndexed) {
+          indexOptions = null;
+        } else if ((bits & Lucene46FieldInfosFormat.OMIT_TERM_FREQ_AND_POSITIONS) != 0) {
+          indexOptions = IndexOptions.DOCS_ONLY;
+        } else if ((bits & Lucene46FieldInfosFormat.OMIT_POSITIONS) != 0) {
+          indexOptions = IndexOptions.DOCS_AND_FREQS;
+        } else if ((bits & Lucene46FieldInfosFormat.STORE_OFFSETS_IN_POSTINGS) != 0) {
+          indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
+        } else {
+          indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
+        }
+
+        // DV Types are packed in one byte
+        byte val = input.readByte();
+        final DocValuesType docValuesType = getDocValuesType(input, (byte) (val & 0x0F));
+        final DocValuesType normsType = getDocValuesType(input, (byte) ((val >>> 4) & 0x0F));
+        final long dvGen = input.readLong();
+        final Map<String,String> attributes = input.readStringStringMap();
+        infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, 
+          omitNorms, storePayloads, indexOptions, docValuesType, normsType, dvGen, Collections.unmodifiableMap(attributes));
+      }
+      
+      if (codecVersion >= Lucene46FieldInfosFormat.FORMAT_CHECKSUM) {
+        CodecUtil.checkFooter(input);
+      } else {
+        CodecUtil.checkEOF(input);
+      }
+      return new FieldInfos(infos);
+    }
+  }
+  
+  private static DocValuesType getDocValuesType(IndexInput input, byte b) throws IOException {
+    if (b == 0) {
+      return null;
+    } else if (b == 1) {
+      return DocValuesType.NUMERIC;
+    } else if (b == 2) {
+      return DocValuesType.BINARY;
+    } else if (b == 3) {
+      return DocValuesType.SORTED;
+    } else if (b == 4) {
+      return DocValuesType.SORTED_SET;
+    } else if (b == 5) {
+      return DocValuesType.SORTED_NUMERIC;
+    } else {
+      throw new CorruptIndexException("invalid docvalues byte: " + b, input);
+    }
   }
 
   @Override
-  public FieldInfosWriter getFieldInfosWriter() throws IOException {
-    return writer;
+  public void write(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene46FieldInfosFormat.EXTENSION);
+    try (IndexOutput output = directory.createOutput(fileName, context)) {
+      CodecUtil.writeHeader(output, Lucene46FieldInfosFormat.CODEC_NAME, Lucene46FieldInfosFormat.FORMAT_CURRENT);
+      output.writeVInt(infos.size());
+      for (FieldInfo fi : infos) {
+        IndexOptions indexOptions = fi.getIndexOptions();
+        byte bits = 0x0;
+        if (fi.hasVectors()) bits |= Lucene46FieldInfosFormat.STORE_TERMVECTOR;
+        if (fi.omitsNorms()) bits |= Lucene46FieldInfosFormat.OMIT_NORMS;
+        if (fi.hasPayloads()) bits |= Lucene46FieldInfosFormat.STORE_PAYLOADS;
+        if (fi.isIndexed()) {
+          bits |= Lucene46FieldInfosFormat.IS_INDEXED;
+          assert indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.hasPayloads();
+          if (indexOptions == IndexOptions.DOCS_ONLY) {
+            bits |= Lucene46FieldInfosFormat.OMIT_TERM_FREQ_AND_POSITIONS;
+          } else if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) {
+            bits |= Lucene46FieldInfosFormat.STORE_OFFSETS_IN_POSTINGS;
+          } else if (indexOptions == IndexOptions.DOCS_AND_FREQS) {
+            bits |= Lucene46FieldInfosFormat.OMIT_POSITIONS;
+          }
+        }
+        output.writeString(fi.name);
+        output.writeVInt(fi.number);
+        output.writeByte(bits);
+
+        // pack the DV types in one byte
+        final byte dv = docValuesByte(fi.getDocValuesType());
+        final byte nrm = docValuesByte(fi.getNormType());
+        assert (dv & (~0xF)) == 0 && (nrm & (~0x0F)) == 0;
+        byte val = (byte) (0xff & ((nrm << 4) | dv));
+        output.writeByte(val);
+        output.writeLong(fi.getDocValuesGen());
+        output.writeStringStringMap(fi.attributes());
+      }
+      CodecUtil.writeFooter(output);
+    }
+  }
+  
+  private static byte docValuesByte(DocValuesType type) {
+    if (type == null) {
+      return 0;
+    } else if (type == DocValuesType.NUMERIC) {
+      return 1;
+    } else if (type == DocValuesType.BINARY) {
+      return 2;
+    } else if (type == DocValuesType.SORTED) {
+      return 3;
+    } else if (type == DocValuesType.SORTED_SET) {
+      return 4;
+    } else if (type == DocValuesType.SORTED_NUMERIC) {
+      return 5;
+    } else {
+      throw new AssertionError();
+    }
   }
   
   /** Extension of field infos */

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46SegmentInfoFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46SegmentInfoFormat.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46SegmentInfoFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46SegmentInfoFormat.java Wed Oct 15 01:26:26 2014
@@ -17,10 +17,20 @@ package org.apache.lucene.codecs.lucene4
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.text.ParseException;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.SegmentInfoFormat;
-import org.apache.lucene.codecs.SegmentInfoReader;
-import org.apache.lucene.codecs.SegmentInfoWriter;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.Version;
 
 /**
  * Lucene 4.6 Segment info format.
@@ -28,19 +38,48 @@ import org.apache.lucene.index.SegmentIn
  */
 @Deprecated
 public class Lucene46SegmentInfoFormat extends SegmentInfoFormat {
-  private final SegmentInfoReader reader = new Lucene46SegmentInfoReader();
 
   /** Sole constructor. */
   public Lucene46SegmentInfoFormat() {
   }
   
   @Override
-  public final SegmentInfoReader getSegmentInfoReader() {
-    return reader;
+  public SegmentInfo read(Directory dir, String segment, IOContext context) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segment, "", Lucene46SegmentInfoFormat.SI_EXTENSION);
+    try (ChecksumIndexInput input = dir.openChecksumInput(fileName, context)) {
+      int codecVersion = CodecUtil.checkHeader(input, Lucene46SegmentInfoFormat.CODEC_NAME,
+                                                      Lucene46SegmentInfoFormat.VERSION_START,
+                                                      Lucene46SegmentInfoFormat.VERSION_CURRENT);
+      final Version version;
+      try {
+        version = Version.parse(input.readString());
+      } catch (ParseException pe) {
+        throw new CorruptIndexException("unable to parse version string: " + pe.getMessage(), input, pe);
+      }
+
+      final int docCount = input.readInt();
+      if (docCount < 0) {
+        throw new CorruptIndexException("invalid docCount: " + docCount, input);
+      }
+      final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
+      final Map<String,String> diagnostics = input.readStringStringMap();
+      final Set<String> files = input.readStringSet();
+
+      if (codecVersion >= Lucene46SegmentInfoFormat.VERSION_CHECKSUM) {
+        CodecUtil.checkFooter(input);
+      } else {
+        CodecUtil.checkEOF(input);
+      }
+
+      final SegmentInfo si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, null);
+      si.setFiles(files);
+
+      return si;
+    }
   }
 
   @Override
-  public SegmentInfoWriter getSegmentInfoWriter() {
+  public void write(Directory dir, SegmentInfo info, IOContext ioContext) throws IOException {
     throw new UnsupportedOperationException("this codec can only be used for reading");
   }
 

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java Wed Oct 15 01:26:26 2014
@@ -25,7 +25,7 @@ import java.util.TreeSet;
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.MissingOrdRemapper;
-import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosReader.LegacyDocValuesType;
+import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat.LegacyDocValuesType;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWCodec.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWCodec.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWCodec.java Wed Oct 15 01:26:26 2014
@@ -4,7 +4,6 @@ import java.io.IOException;
 
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
-import org.apache.lucene.codecs.FieldInfosWriter;
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
@@ -35,13 +34,7 @@ import org.apache.lucene.codecs.TermVect
 @Deprecated
 public final class Lucene40RWCodec extends Lucene40Codec {
   
-  private final FieldInfosFormat fieldInfos = new Lucene40FieldInfosFormat() {
-    @Override
-    public FieldInfosWriter getFieldInfosWriter() throws IOException {
-      return new Lucene40FieldInfosWriter();
-    }
-  };
-  
+  private final FieldInfosFormat fieldInfos = new Lucene40RWFieldInfosFormat();
   private final DocValuesFormat docValues = new Lucene40RWDocValuesFormat();
   private final NormsFormat norms = new Lucene40RWNormsFormat();
   private final StoredFieldsFormat stored = new Lucene40RWStoredFieldsFormat();

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWDocValuesFormat.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWDocValuesFormat.java Wed Oct 15 01:26:26 2014
@@ -35,6 +35,6 @@ public final class Lucene40RWDocValuesFo
     String filename = IndexFileNames.segmentFileName(state.segmentInfo.name, 
           "dv", 
           Lucene40CompoundFormat.COMPOUND_FILE_EXTENSION);
-    return new Lucene40DocValuesWriter(state, filename, Lucene40FieldInfosReader.LEGACY_DV_TYPE_KEY);
+    return new Lucene40DocValuesWriter(state, filename, Lucene40FieldInfosFormat.LEGACY_DV_TYPE_KEY);
   }
 }

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWNormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWNormsFormat.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWNormsFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWNormsFormat.java Wed Oct 15 01:26:26 2014
@@ -36,7 +36,7 @@ public final class Lucene40RWNormsFormat
     String filename = IndexFileNames.segmentFileName(state.segmentInfo.name, 
         "nrm", 
         Lucene40CompoundFormat.COMPOUND_FILE_EXTENSION);
-    final Lucene40DocValuesWriter impl = new Lucene40DocValuesWriter(state, filename, Lucene40FieldInfosReader.LEGACY_NORM_TYPE_KEY);
+    final Lucene40DocValuesWriter impl = new Lucene40DocValuesWriter(state, filename, Lucene40FieldInfosFormat.LEGACY_NORM_TYPE_KEY);
     return new NormsConsumer() {
       @Override
       public void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWSegmentInfoFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWSegmentInfoFormat.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWSegmentInfoFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWSegmentInfoFormat.java Wed Oct 15 01:26:26 2014
@@ -17,7 +17,16 @@ package org.apache.lucene.codecs.lucene4
  * limitations under the License.
  */
 
-import org.apache.lucene.codecs.SegmentInfoWriter;
+import java.io.IOException;
+import java.util.Collections;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.IOUtils;
 
 /**
  * Read-write version of 4.0 segmentinfo format for testing
@@ -27,7 +36,33 @@ import org.apache.lucene.codecs.SegmentI
 public final class Lucene40RWSegmentInfoFormat extends Lucene40SegmentInfoFormat {
 
   @Override
-  public SegmentInfoWriter getSegmentInfoWriter() {
-    return new Lucene40SegmentInfoWriter();
+  public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene40SegmentInfoFormat.SI_EXTENSION);
+    si.addFile(fileName);
+
+    final IndexOutput output = dir.createOutput(fileName, ioContext);
+
+    boolean success = false;
+    try {
+      CodecUtil.writeHeader(output, Lucene40SegmentInfoFormat.CODEC_NAME, Lucene40SegmentInfoFormat.VERSION_CURRENT);
+      // Write the Lucene version that created this segment, since 3.1
+      output.writeString(si.getVersion().toString());
+      output.writeInt(si.getDocCount());
+
+      output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
+      output.writeStringStringMap(si.getDiagnostics());
+      output.writeStringStringMap(Collections.<String,String>emptyMap());
+      output.writeStringSet(si.files());
+
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(output);
+        // TODO: why must we do this? do we not get tracking dir wrapper?
+        IOUtils.deleteFilesIgnoringExceptions(si.dir, fileName);
+      } else {
+        output.close();
+      }
+    }
   }
 }

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWCodec.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWCodec.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWCodec.java Wed Oct 15 01:26:26 2014
@@ -1,17 +1,13 @@
 package org.apache.lucene.codecs.lucene41;
 
-import java.io.IOException;
-
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
-import org.apache.lucene.codecs.FieldInfosWriter;
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosWriter;
 import org.apache.lucene.codecs.lucene40.Lucene40RWDocValuesFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40RWFieldInfosFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40RWNormsFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40RWSegmentInfoFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40RWTermVectorsFormat;
@@ -40,13 +36,7 @@ import org.apache.lucene.codecs.lucene40
 @Deprecated
 public final class Lucene41RWCodec extends Lucene41Codec {
   private final StoredFieldsFormat fieldsFormat = new Lucene41RWStoredFieldsFormat();
-  private final FieldInfosFormat fieldInfos = new Lucene40FieldInfosFormat() {
-    @Override
-    public FieldInfosWriter getFieldInfosWriter() throws IOException {
-      return new Lucene40FieldInfosWriter();
-    }
-  };
-  
+  private final FieldInfosFormat fieldInfos = new Lucene40RWFieldInfosFormat();
   private final DocValuesFormat docValues = new Lucene40RWDocValuesFormat();
   private final NormsFormat norms = new Lucene40RWNormsFormat();
   private final TermVectorsFormat vectors = new Lucene40RWTermVectorsFormat();

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java Wed Oct 15 01:26:26 2014
@@ -17,11 +17,8 @@ package org.apache.lucene.codecs.lucene4
  * limitations under the License.
  */
 
-import java.io.IOException;
-
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
-import org.apache.lucene.codecs.FieldInfosWriter;
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
@@ -39,13 +36,7 @@ public final class Lucene42RWCodec exten
   private static final DocValuesFormat dv = new Lucene42RWDocValuesFormat();
   private static final NormsFormat norms = new Lucene42RWNormsFormat();
   private static final StoredFieldsFormat storedFields = new Lucene41RWStoredFieldsFormat();
-
-  private final FieldInfosFormat fieldInfosFormat = new Lucene42FieldInfosFormat() {
-    @Override
-    public FieldInfosWriter getFieldInfosWriter() throws IOException {
-      return new Lucene42FieldInfosWriter();
-    }
-  };
+  private static final FieldInfosFormat fieldInfosFormat = new Lucene42RWFieldInfosFormat();
 
   @Override
   public DocValuesFormat getDocValuesFormatForField(String field) {

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java Wed Oct 15 01:26:26 2014
@@ -17,19 +17,15 @@ package org.apache.lucene.codecs.lucene4
  * limitations under the License.
  */
 
-import java.io.IOException;
-
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
-import org.apache.lucene.codecs.FieldInfosWriter;
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40RWSegmentInfoFormat;
 import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
-import org.apache.lucene.codecs.lucene42.Lucene42FieldInfosFormat;
-import org.apache.lucene.codecs.lucene42.Lucene42FieldInfosWriter;
+import org.apache.lucene.codecs.lucene42.Lucene42RWFieldInfosFormat;
 import org.apache.lucene.codecs.lucene42.Lucene42RWNormsFormat;
 import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
 
@@ -39,12 +35,7 @@ import org.apache.lucene.codecs.lucene42
 @SuppressWarnings("deprecation")
 public final class Lucene45RWCodec extends Lucene45Codec {
   
-  private final FieldInfosFormat fieldInfosFormat = new Lucene42FieldInfosFormat() {
-    @Override
-    public FieldInfosWriter getFieldInfosWriter() throws IOException {
-      return new Lucene42FieldInfosWriter();
-    }
-  };
+  private static final FieldInfosFormat fieldInfosFormat = new Lucene42RWFieldInfosFormat();
 
   @Override
   public FieldInfosFormat fieldInfosFormat() {

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene46/Lucene46RWSegmentInfoFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene46/Lucene46RWSegmentInfoFormat.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene46/Lucene46RWSegmentInfoFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene46/Lucene46RWSegmentInfoFormat.java Wed Oct 15 01:26:26 2014
@@ -17,7 +17,16 @@ package org.apache.lucene.codecs.lucene4
  * limitations under the License.
  */
 
-import org.apache.lucene.codecs.SegmentInfoWriter;
+import java.io.IOException;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.Version;
 
 /**
  * Read-Write version of 4.6 segmentinfo format for testing
@@ -26,7 +35,36 @@ import org.apache.lucene.codecs.SegmentI
 @Deprecated
 public final class Lucene46RWSegmentInfoFormat extends Lucene46SegmentInfoFormat {
   @Override
-  public SegmentInfoWriter getSegmentInfoWriter() {
-    return new Lucene46SegmentInfoWriter();
+  public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene46SegmentInfoFormat.SI_EXTENSION);
+    si.addFile(fileName);
+
+    final IndexOutput output = dir.createOutput(fileName, ioContext);
+
+    boolean success = false;
+    try {
+      CodecUtil.writeHeader(output, Lucene46SegmentInfoFormat.CODEC_NAME, Lucene46SegmentInfoFormat.VERSION_CURRENT);
+      Version version = si.getVersion();
+      if (version.major < 4) {
+        throw new IllegalArgumentException("invalid major version: should be >= 4 but got: " + version.major + " segment=" + si);
+      }
+      // Write the Lucene version that created this segment, since 3.1
+      output.writeString(version.toString());
+      output.writeInt(si.getDocCount());
+
+      output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
+      output.writeStringStringMap(si.getDiagnostics());
+      output.writeStringSet(si.files());
+      CodecUtil.writeFooter(output);
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(output);
+        // TODO: are we doing this outside of the tracking wrapper? why must SIWriter cleanup like this?
+        IOUtils.deleteFilesIgnoringExceptions(si.dir, fileName);
+      } else {
+        output.close();
+      }
+    }
   }
 }

Modified: lucene/dev/branches/lucene5969/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java Wed Oct 15 01:26:26 2014
@@ -18,10 +18,26 @@ package org.apache.lucene.codecs.simplet
  */
 
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.lucene.codecs.FieldInfosFormat;
-import org.apache.lucene.codecs.FieldInfosReader;
-import org.apache.lucene.codecs.FieldInfosWriter;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.StringHelper;
 
 /**
  * plaintext field infos format
@@ -30,16 +46,220 @@ import org.apache.lucene.codecs.FieldInf
  * @lucene.experimental
  */
 public class SimpleTextFieldInfosFormat extends FieldInfosFormat {
-  private final FieldInfosReader reader = new SimpleTextFieldInfosReader();
-  private final FieldInfosWriter writer = new SimpleTextFieldInfosWriter();
-
+  
+  /** Extension of field infos */
+  static final String FIELD_INFOS_EXTENSION = "inf";
+  
+  static final BytesRef NUMFIELDS       =  new BytesRef("number of fields ");
+  static final BytesRef NAME            =  new BytesRef("  name ");
+  static final BytesRef NUMBER          =  new BytesRef("  number ");
+  static final BytesRef ISINDEXED       =  new BytesRef("  indexed ");
+  static final BytesRef STORETV         =  new BytesRef("  term vectors ");
+  static final BytesRef STORETVPOS      =  new BytesRef("  term vector positions ");
+  static final BytesRef STORETVOFF      =  new BytesRef("  term vector offsets ");
+  static final BytesRef PAYLOADS        =  new BytesRef("  payloads ");
+  static final BytesRef NORMS           =  new BytesRef("  norms ");
+  static final BytesRef NORMS_TYPE      =  new BytesRef("  norms type ");
+  static final BytesRef DOCVALUES       =  new BytesRef("  doc values ");
+  static final BytesRef DOCVALUES_GEN   =  new BytesRef("  doc values gen ");
+  static final BytesRef INDEXOPTIONS    =  new BytesRef("  index options ");
+  static final BytesRef NUM_ATTS        =  new BytesRef("  attributes ");
+  static final BytesRef ATT_KEY         =  new BytesRef("    key ");
+  static final BytesRef ATT_VALUE       =  new BytesRef("    value ");
+  
   @Override
-  public FieldInfosReader getFieldInfosReader() throws IOException {
-    return reader;
+  public FieldInfos read(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, IOContext iocontext) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, FIELD_INFOS_EXTENSION);
+    ChecksumIndexInput input = directory.openChecksumInput(fileName, iocontext);
+    BytesRefBuilder scratch = new BytesRefBuilder();
+    
+    boolean success = false;
+    try {
+      
+      SimpleTextUtil.readLine(input, scratch);
+      assert StringHelper.startsWith(scratch.get(), NUMFIELDS);
+      final int size = Integer.parseInt(readString(NUMFIELDS.length, scratch));
+      FieldInfo infos[] = new FieldInfo[size];
+
+      for (int i = 0; i < size; i++) {
+        SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch.get(), NAME);
+        String name = readString(NAME.length, scratch);
+        
+        SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch.get(), NUMBER);
+        int fieldNumber = Integer.parseInt(readString(NUMBER.length, scratch));
+
+        SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch.get(), ISINDEXED);
+        boolean isIndexed = Boolean.parseBoolean(readString(ISINDEXED.length, scratch));
+        
+        final IndexOptions indexOptions;
+        if (isIndexed) {
+          SimpleTextUtil.readLine(input, scratch);
+          assert StringHelper.startsWith(scratch.get(), INDEXOPTIONS);
+          indexOptions = IndexOptions.valueOf(readString(INDEXOPTIONS.length, scratch));          
+        } else {
+          indexOptions = null;
+        }
+        
+        SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch.get(), STORETV);
+        boolean storeTermVector = Boolean.parseBoolean(readString(STORETV.length, scratch));
+        
+        SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch.get(), PAYLOADS);
+        boolean storePayloads = Boolean.parseBoolean(readString(PAYLOADS.length, scratch));
+        
+        SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch.get(), NORMS);
+        boolean omitNorms = !Boolean.parseBoolean(readString(NORMS.length, scratch));
+        
+        SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch.get(), NORMS_TYPE);
+        String nrmType = readString(NORMS_TYPE.length, scratch);
+        final DocValuesType normsType = docValuesType(nrmType);
+        
+        SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch.get(), DOCVALUES);
+        String dvType = readString(DOCVALUES.length, scratch);
+        final DocValuesType docValuesType = docValuesType(dvType);
+        
+        SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch.get(), DOCVALUES_GEN);
+        final long dvGen = Long.parseLong(readString(DOCVALUES_GEN.length, scratch));
+        
+        SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch.get(), NUM_ATTS);
+        int numAtts = Integer.parseInt(readString(NUM_ATTS.length, scratch));
+        Map<String,String> atts = new HashMap<>();
+
+        for (int j = 0; j < numAtts; j++) {
+          SimpleTextUtil.readLine(input, scratch);
+          assert StringHelper.startsWith(scratch.get(), ATT_KEY);
+          String key = readString(ATT_KEY.length, scratch);
+        
+          SimpleTextUtil.readLine(input, scratch);
+          assert StringHelper.startsWith(scratch.get(), ATT_VALUE);
+          String value = readString(ATT_VALUE.length, scratch);
+          atts.put(key, value);
+        }
+
+        infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, 
+          omitNorms, storePayloads, indexOptions, docValuesType, normsType, dvGen, Collections.unmodifiableMap(atts));
+      }
+
+      SimpleTextUtil.checkFooter(input);
+      
+      FieldInfos fieldInfos = new FieldInfos(infos);
+      success = true;
+      return fieldInfos;
+    } finally {
+      if (success) {
+        input.close();
+      } else {
+        IOUtils.closeWhileHandlingException(input);
+      }
+    }
+  }
+
+  public DocValuesType docValuesType(String dvType) {
+    if ("false".equals(dvType)) {
+      return null;
+    } else {
+      return DocValuesType.valueOf(dvType);
+    }
+  }
+  
+  private String readString(int offset, BytesRefBuilder scratch) {
+    return new String(scratch.bytes(), offset, scratch.length()-offset, StandardCharsets.UTF_8);
   }
 
   @Override
-  public FieldInfosWriter getFieldInfosWriter() throws IOException {
-    return writer;
+  public void write(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, FIELD_INFOS_EXTENSION);
+    IndexOutput out = directory.createOutput(fileName, context);
+    BytesRefBuilder scratch = new BytesRefBuilder();
+    boolean success = false;
+    try {
+      SimpleTextUtil.write(out, NUMFIELDS);
+      SimpleTextUtil.write(out, Integer.toString(infos.size()), scratch);
+      SimpleTextUtil.writeNewline(out);
+      
+      for (FieldInfo fi : infos) {
+        SimpleTextUtil.write(out, NAME);
+        SimpleTextUtil.write(out, fi.name, scratch);
+        SimpleTextUtil.writeNewline(out);
+        
+        SimpleTextUtil.write(out, NUMBER);
+        SimpleTextUtil.write(out, Integer.toString(fi.number), scratch);
+        SimpleTextUtil.writeNewline(out);
+        
+        SimpleTextUtil.write(out, ISINDEXED);
+        SimpleTextUtil.write(out, Boolean.toString(fi.isIndexed()), scratch);
+        SimpleTextUtil.writeNewline(out);
+        
+        if (fi.isIndexed()) {
+          assert fi.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.hasPayloads();
+          SimpleTextUtil.write(out, INDEXOPTIONS);
+          SimpleTextUtil.write(out, fi.getIndexOptions().toString(), scratch);
+          SimpleTextUtil.writeNewline(out);
+        }
+        
+        SimpleTextUtil.write(out, STORETV);
+        SimpleTextUtil.write(out, Boolean.toString(fi.hasVectors()), scratch);
+        SimpleTextUtil.writeNewline(out);
+        
+        SimpleTextUtil.write(out, PAYLOADS);
+        SimpleTextUtil.write(out, Boolean.toString(fi.hasPayloads()), scratch);
+        SimpleTextUtil.writeNewline(out);
+               
+        SimpleTextUtil.write(out, NORMS);
+        SimpleTextUtil.write(out, Boolean.toString(!fi.omitsNorms()), scratch);
+        SimpleTextUtil.writeNewline(out);
+        
+        SimpleTextUtil.write(out, NORMS_TYPE);
+        SimpleTextUtil.write(out, getDocValuesType(fi.getNormType()), scratch);
+        SimpleTextUtil.writeNewline(out);
+        
+        SimpleTextUtil.write(out, DOCVALUES);
+        SimpleTextUtil.write(out, getDocValuesType(fi.getDocValuesType()), scratch);
+        SimpleTextUtil.writeNewline(out);
+        
+        SimpleTextUtil.write(out, DOCVALUES_GEN);
+        SimpleTextUtil.write(out, Long.toString(fi.getDocValuesGen()), scratch);
+        SimpleTextUtil.writeNewline(out);
+               
+        Map<String,String> atts = fi.attributes();
+        int numAtts = atts == null ? 0 : atts.size();
+        SimpleTextUtil.write(out, NUM_ATTS);
+        SimpleTextUtil.write(out, Integer.toString(numAtts), scratch);
+        SimpleTextUtil.writeNewline(out);
+      
+        if (numAtts > 0) {
+          for (Map.Entry<String,String> entry : atts.entrySet()) {
+            SimpleTextUtil.write(out, ATT_KEY);
+            SimpleTextUtil.write(out, entry.getKey(), scratch);
+            SimpleTextUtil.writeNewline(out);
+          
+            SimpleTextUtil.write(out, ATT_VALUE);
+            SimpleTextUtil.write(out, entry.getValue(), scratch);
+            SimpleTextUtil.writeNewline(out);
+          }
+        }
+      }
+      SimpleTextUtil.writeChecksum(out, scratch);
+      success = true;
+    } finally {
+      if (success) {
+        out.close();
+      } else {
+        IOUtils.closeWhileHandlingException(out);
+      }
+    }
+  }
+  
+  private static String getDocValuesType(DocValuesType type) {
+    return type == null ? "false" : type.toString();
   }
 }

Modified: lucene/dev/branches/lucene5969/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java Wed Oct 15 01:26:26 2014
@@ -17,9 +17,28 @@ package org.apache.lucene.codecs.simplet
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.text.ParseException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.lucene.codecs.SegmentInfoFormat;
-import org.apache.lucene.codecs.SegmentInfoReader;
-import org.apache.lucene.codecs.SegmentInfoWriter;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.Version;
 
 /**
  * plain text segments file format.
@@ -28,18 +47,163 @@ import org.apache.lucene.codecs.SegmentI
  * @lucene.experimental
  */
 public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
-  private final SegmentInfoReader reader = new SimpleTextSegmentInfoReader();
-  private final SegmentInfoWriter writer = new SimpleTextSegmentInfoWriter();
+  final static BytesRef SI_VERSION          = new BytesRef("    version ");
+  final static BytesRef SI_DOCCOUNT         = new BytesRef("    number of documents ");
+  final static BytesRef SI_USECOMPOUND      = new BytesRef("    uses compound file ");
+  final static BytesRef SI_NUM_DIAG         = new BytesRef("    diagnostics ");
+  final static BytesRef SI_DIAG_KEY         = new BytesRef("      key ");
+  final static BytesRef SI_DIAG_VALUE       = new BytesRef("      value ");
+  final static BytesRef SI_NUM_FILES        = new BytesRef("    files ");
+  final static BytesRef SI_FILE             = new BytesRef("      file ");
+  final static BytesRef SI_ID               = new BytesRef("    id ");
 
   public static final String SI_EXTENSION = "si";
   
   @Override
-  public SegmentInfoReader getSegmentInfoReader() {
-    return reader;
+  public SegmentInfo read(Directory directory, String segmentName, IOContext context) throws IOException {
+    BytesRefBuilder scratch = new BytesRefBuilder();
+    String segFileName = IndexFileNames.segmentFileName(segmentName, "", SimpleTextSegmentInfoFormat.SI_EXTENSION);
+    ChecksumIndexInput input = directory.openChecksumInput(segFileName, context);
+    boolean success = false;
+    try {
+      SimpleTextUtil.readLine(input, scratch);
+      assert StringHelper.startsWith(scratch.get(), SI_VERSION);
+      final Version version;
+      try {
+        version = Version.parse(readString(SI_VERSION.length, scratch));
+      } catch (ParseException pe) {
+        throw new CorruptIndexException("unable to parse version string: " + pe.getMessage(), input, pe);
+      }
+    
+      SimpleTextUtil.readLine(input, scratch);
+      assert StringHelper.startsWith(scratch.get(), SI_DOCCOUNT);
+      final int docCount = Integer.parseInt(readString(SI_DOCCOUNT.length, scratch));
+    
+      SimpleTextUtil.readLine(input, scratch);
+      assert StringHelper.startsWith(scratch.get(), SI_USECOMPOUND);
+      final boolean isCompoundFile = Boolean.parseBoolean(readString(SI_USECOMPOUND.length, scratch));
+    
+      SimpleTextUtil.readLine(input, scratch);
+      assert StringHelper.startsWith(scratch.get(), SI_NUM_DIAG);
+      int numDiag = Integer.parseInt(readString(SI_NUM_DIAG.length, scratch));
+      Map<String,String> diagnostics = new HashMap<>();
+
+      for (int i = 0; i < numDiag; i++) {
+        SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch.get(), SI_DIAG_KEY);
+        String key = readString(SI_DIAG_KEY.length, scratch);
+      
+        SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch.get(), SI_DIAG_VALUE);
+        String value = readString(SI_DIAG_VALUE.length, scratch);
+        diagnostics.put(key, value);
+      }
+      
+      SimpleTextUtil.readLine(input, scratch);
+      assert StringHelper.startsWith(scratch.get(), SI_NUM_FILES);
+      int numFiles = Integer.parseInt(readString(SI_NUM_FILES.length, scratch));
+      Set<String> files = new HashSet<>();
+
+      for (int i = 0; i < numFiles; i++) {
+        SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch.get(), SI_FILE);
+        String fileName = readString(SI_FILE.length, scratch);
+        files.add(fileName);
+      }
+      
+      SimpleTextUtil.readLine(input, scratch);
+      assert StringHelper.startsWith(scratch.get(), SI_ID);
+      final byte[] id = Arrays.copyOfRange(scratch.bytes(), SI_ID.length, scratch.length());
+
+      SimpleTextUtil.checkFooter(input);
+
+      SegmentInfo info = new SegmentInfo(directory, version, segmentName, docCount,
+                                         isCompoundFile, null, diagnostics, id);
+      info.setFiles(files);
+      success = true;
+      return info;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(input);
+      } else {
+        input.close();
+      }
+    }
   }
 
+  private String readString(int offset, BytesRefBuilder scratch) {
+    return new String(scratch.bytes(), offset, scratch.length()-offset, StandardCharsets.UTF_8);
+  }
+  
   @Override
-  public SegmentInfoWriter getSegmentInfoWriter() {
-    return writer;
+  public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
+
+    String segFileName = IndexFileNames.segmentFileName(si.name, "", SimpleTextSegmentInfoFormat.SI_EXTENSION);
+    si.addFile(segFileName);
+
+    boolean success = false;
+    IndexOutput output = dir.createOutput(segFileName, ioContext);
+
+    try {
+      BytesRefBuilder scratch = new BytesRefBuilder();
+    
+      SimpleTextUtil.write(output, SI_VERSION);
+      SimpleTextUtil.write(output, si.getVersion().toString(), scratch);
+      SimpleTextUtil.writeNewline(output);
+    
+      SimpleTextUtil.write(output, SI_DOCCOUNT);
+      SimpleTextUtil.write(output, Integer.toString(si.getDocCount()), scratch);
+      SimpleTextUtil.writeNewline(output);
+    
+      SimpleTextUtil.write(output, SI_USECOMPOUND);
+      SimpleTextUtil.write(output, Boolean.toString(si.getUseCompoundFile()), scratch);
+      SimpleTextUtil.writeNewline(output);
+    
+      Map<String,String> diagnostics = si.getDiagnostics();
+      int numDiagnostics = diagnostics == null ? 0 : diagnostics.size();
+      SimpleTextUtil.write(output, SI_NUM_DIAG);
+      SimpleTextUtil.write(output, Integer.toString(numDiagnostics), scratch);
+      SimpleTextUtil.writeNewline(output);
+    
+      if (numDiagnostics > 0) {
+        for (Map.Entry<String,String> diagEntry : diagnostics.entrySet()) {
+          SimpleTextUtil.write(output, SI_DIAG_KEY);
+          SimpleTextUtil.write(output, diagEntry.getKey(), scratch);
+          SimpleTextUtil.writeNewline(output);
+        
+          SimpleTextUtil.write(output, SI_DIAG_VALUE);
+          SimpleTextUtil.write(output, diagEntry.getValue(), scratch);
+          SimpleTextUtil.writeNewline(output);
+        }
+      }
+      
+      Set<String> files = si.files();
+      int numFiles = files == null ? 0 : files.size();
+      SimpleTextUtil.write(output, SI_NUM_FILES);
+      SimpleTextUtil.write(output, Integer.toString(numFiles), scratch);
+      SimpleTextUtil.writeNewline(output);
+
+      if (numFiles > 0) {
+        for(String fileName : files) {
+          SimpleTextUtil.write(output, SI_FILE);
+          SimpleTextUtil.write(output, fileName, scratch);
+          SimpleTextUtil.writeNewline(output);
+        }
+      }
+
+      SimpleTextUtil.write(output, SI_ID);
+      SimpleTextUtil.write(output, new BytesRef(si.getId()));
+      SimpleTextUtil.writeNewline(output);
+      
+      SimpleTextUtil.writeChecksum(output, scratch);
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(output);
+        IOUtils.deleteFilesIgnoringExceptions(dir, segFileName);
+      } else {
+        output.close();
+      }
+    }
   }
 }

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/FieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/FieldInfosFormat.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/FieldInfosFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/FieldInfosFormat.java Wed Oct 15 01:26:26 2014
@@ -20,6 +20,9 @@ package org.apache.lucene.codecs;
 import java.io.IOException;
 
 import org.apache.lucene.index.FieldInfos; // javadocs
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 
 /**
  * Encodes/decodes {@link FieldInfos}
@@ -30,12 +33,11 @@ public abstract class FieldInfosFormat {
    *  constructors, typically implicit.) */
   protected FieldInfosFormat() {
   }
+  
+  /** Read the {@link FieldInfos} previously written with {@link #write}. */
+  public abstract FieldInfos read(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, IOContext iocontext) throws IOException;
 
-  /** Returns a {@link FieldInfosReader} to read field infos
-   *  from the index */
-  public abstract FieldInfosReader getFieldInfosReader() throws IOException;
-
-  /** Returns a {@link FieldInfosWriter} to write field infos
-   *  to the index */
-  public abstract FieldInfosWriter getFieldInfosWriter() throws IOException;
+  /** Writes the provided {@link FieldInfos} to the
+   *  directory. */
+  public abstract void write(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException;
 }

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/SegmentInfoFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/SegmentInfoFormat.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/SegmentInfoFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/SegmentInfoFormat.java Wed Oct 15 01:26:26 2014
@@ -17,7 +17,11 @@ package org.apache.lucene.codecs;
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 
 /**
  * Expert: Controls the format of the 
@@ -33,11 +37,18 @@ public abstract class SegmentInfoFormat 
   protected SegmentInfoFormat() {
   }
 
-  /** Returns the {@link SegmentInfoReader} for reading
-   *  {@link SegmentInfo} instances. */
-  public abstract SegmentInfoReader getSegmentInfoReader();
+  /**
+   * Read {@link SegmentInfo} data from a directory.
+   * @param directory directory to read from
+   * @param segmentName name of the segment to read
+   * @return infos instance to be populated with data
+   * @throws IOException If an I/O error occurs
+   */
+  public abstract SegmentInfo read(Directory directory, String segmentName, IOContext context) throws IOException;
 
-  /** Returns the {@link SegmentInfoWriter} for writing
-   *  {@link SegmentInfo} instances. */
-  public abstract SegmentInfoWriter getSegmentInfoWriter();
+  /**
+   * Write {@link SegmentInfo} data. 
+   * @throws IOException If an I/O error occurs
+   */
+  public abstract void write(Directory dir, SegmentInfo info, IOContext ioContext) throws IOException;
 }

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java Wed Oct 15 01:26:26 2014
@@ -18,14 +18,25 @@ package org.apache.lucene.codecs.lucene5
  */
 
 import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
-import org.apache.lucene.codecs.FieldInfosReader;
-import org.apache.lucene.codecs.FieldInfosWriter;
+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.SegmentInfo;
 import org.apache.lucene.index.FieldInfo.DocValuesType;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
 
 /**
  * Lucene 5.0 Field Infos format.
@@ -91,21 +102,148 @@ import org.apache.lucene.store.DataOutpu
  * @lucene.experimental
  */
 public final class Lucene50FieldInfosFormat extends FieldInfosFormat {
-  private final FieldInfosReader reader = new Lucene50FieldInfosReader();
-  private final FieldInfosWriter writer = new Lucene50FieldInfosWriter();
   
   /** Sole constructor. */
   public Lucene50FieldInfosFormat() {
   }
-
+  
   @Override
-  public FieldInfosReader getFieldInfosReader() throws IOException {
-    return reader;
+  public FieldInfos read(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, IOContext context) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene50FieldInfosFormat.EXTENSION);
+    try (ChecksumIndexInput input = directory.openChecksumInput(fileName, context)) {
+      Throwable priorE = null;
+      FieldInfo infos[] = null;
+      try {
+        CodecUtil.checkSegmentHeader(input, Lucene50FieldInfosFormat.CODEC_NAME, 
+                                     Lucene50FieldInfosFormat.FORMAT_START, 
+                                     Lucene50FieldInfosFormat.FORMAT_CURRENT,
+                                     segmentInfo.getId(), segmentSuffix);
+        
+        final int size = input.readVInt(); //read in the size
+        infos = new FieldInfo[size];
+        
+        for (int i = 0; i < size; i++) {
+          String name = input.readString();
+          final int fieldNumber = input.readVInt();
+          if (fieldNumber < 0) {
+            throw new CorruptIndexException("invalid field number for field: " + name + ", fieldNumber=" + fieldNumber, input);
+          }
+          byte bits = input.readByte();
+          boolean isIndexed = (bits & Lucene50FieldInfosFormat.IS_INDEXED) != 0;
+          boolean storeTermVector = (bits & Lucene50FieldInfosFormat.STORE_TERMVECTOR) != 0;
+          boolean omitNorms = (bits & Lucene50FieldInfosFormat.OMIT_NORMS) != 0;
+          boolean storePayloads = (bits & Lucene50FieldInfosFormat.STORE_PAYLOADS) != 0;
+          final IndexOptions indexOptions;
+          if (!isIndexed) {
+            indexOptions = null;
+          } else if ((bits & Lucene50FieldInfosFormat.OMIT_TERM_FREQ_AND_POSITIONS) != 0) {
+            indexOptions = IndexOptions.DOCS_ONLY;
+          } else if ((bits & Lucene50FieldInfosFormat.OMIT_POSITIONS) != 0) {
+            indexOptions = IndexOptions.DOCS_AND_FREQS;
+          } else if ((bits & Lucene50FieldInfosFormat.STORE_OFFSETS_IN_POSTINGS) != 0) {
+            indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
+          } else {
+            indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
+          }
+          
+          // DV Types are packed in one byte
+          byte val = input.readByte();
+          final DocValuesType docValuesType = getDocValuesType(input, (byte) (val & 0x0F));
+          final DocValuesType normsType = getDocValuesType(input, (byte) ((val >>> 4) & 0x0F));
+          final long dvGen = input.readLong();
+          final Map<String,String> attributes = input.readStringStringMap();
+          try {
+            infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, omitNorms, storePayloads, 
+                                     indexOptions, docValuesType, normsType, dvGen, Collections.unmodifiableMap(attributes));
+            infos[i].checkConsistency();
+          } catch (IllegalStateException e) {
+            throw new CorruptIndexException("invalid fieldinfo for field: " + name + ", fieldNumber=" + fieldNumber, input, e);
+          }
+        }
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(input, priorE);
+      }
+      return new FieldInfos(infos);
+    }
+  }
+  
+  private static DocValuesType getDocValuesType(IndexInput input, byte b) throws IOException {
+    if (b == 0) {
+      return null;
+    } else if (b == 1) {
+      return DocValuesType.NUMERIC;
+    } else if (b == 2) {
+      return DocValuesType.BINARY;
+    } else if (b == 3) {
+      return DocValuesType.SORTED;
+    } else if (b == 4) {
+      return DocValuesType.SORTED_SET;
+    } else if (b == 5) {
+      return DocValuesType.SORTED_NUMERIC;
+    } else {
+      throw new CorruptIndexException("invalid docvalues byte: " + b, input);
+    }
   }
 
   @Override
-  public FieldInfosWriter getFieldInfosWriter() throws IOException {
-    return writer;
+  public void write(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene50FieldInfosFormat.EXTENSION);
+    try (IndexOutput output = directory.createOutput(fileName, context)) {
+      CodecUtil.writeSegmentHeader(output, Lucene50FieldInfosFormat.CODEC_NAME, Lucene50FieldInfosFormat.FORMAT_CURRENT, segmentInfo.getId(), segmentSuffix);
+      output.writeVInt(infos.size());
+      for (FieldInfo fi : infos) {
+        fi.checkConsistency();
+        IndexOptions indexOptions = fi.getIndexOptions();
+        byte bits = 0x0;
+        if (fi.hasVectors()) bits |= Lucene50FieldInfosFormat.STORE_TERMVECTOR;
+        if (fi.omitsNorms()) bits |= Lucene50FieldInfosFormat.OMIT_NORMS;
+        if (fi.hasPayloads()) bits |= Lucene50FieldInfosFormat.STORE_PAYLOADS;
+        if (fi.isIndexed()) {
+          bits |= Lucene50FieldInfosFormat.IS_INDEXED;
+          assert indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.hasPayloads();
+          if (indexOptions == IndexOptions.DOCS_ONLY) {
+            bits |= Lucene50FieldInfosFormat.OMIT_TERM_FREQ_AND_POSITIONS;
+          } else if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) {
+            bits |= Lucene50FieldInfosFormat.STORE_OFFSETS_IN_POSTINGS;
+          } else if (indexOptions == IndexOptions.DOCS_AND_FREQS) {
+            bits |= Lucene50FieldInfosFormat.OMIT_POSITIONS;
+          }
+        }
+        output.writeString(fi.name);
+        output.writeVInt(fi.number);
+        output.writeByte(bits);
+
+        // pack the DV types in one byte
+        final byte dv = docValuesByte(fi.getDocValuesType());
+        final byte nrm = docValuesByte(fi.getNormType());
+        assert (dv & (~0xF)) == 0 && (nrm & (~0x0F)) == 0;
+        byte val = (byte) (0xff & ((nrm << 4) | dv));
+        output.writeByte(val);
+        output.writeLong(fi.getDocValuesGen());
+        output.writeStringStringMap(fi.attributes());
+      }
+      CodecUtil.writeFooter(output);
+    }
+  }
+  
+  private static byte docValuesByte(DocValuesType type) {
+    if (type == null) {
+      return 0;
+    } else if (type == DocValuesType.NUMERIC) {
+      return 1;
+    } else if (type == DocValuesType.BINARY) {
+      return 2;
+    } else if (type == DocValuesType.SORTED) {
+      return 3;
+    } else if (type == DocValuesType.SORTED_SET) {
+      return 4;
+    } else if (type == DocValuesType.SORTED_NUMERIC) {
+      return 5;
+    } else {
+      throw new AssertionError();
+    }
   }
   
   /** Extension of field infos */

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java Wed Oct 15 01:26:26 2014
@@ -17,14 +17,25 @@ package org.apache.lucene.codecs.lucene5
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.SegmentInfoFormat;
-import org.apache.lucene.codecs.SegmentInfoReader;
-import org.apache.lucene.codecs.SegmentInfoWriter;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexWriter; // javadocs
 import org.apache.lucene.index.SegmentInfo; // javadocs
 import org.apache.lucene.index.SegmentInfos; // javadocs
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.DataOutput; // javadocs
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.Version;
 
 /**
  * Lucene 5.0 Segment info format.
@@ -67,21 +78,86 @@ import org.apache.lucene.store.DataOutpu
  * @lucene.experimental
  */
 public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
-  private final SegmentInfoReader reader = new Lucene50SegmentInfoReader();
-  private final SegmentInfoWriter writer = new Lucene50SegmentInfoWriter();
 
   /** Sole constructor. */
   public Lucene50SegmentInfoFormat() {
   }
   
   @Override
-  public SegmentInfoReader getSegmentInfoReader() {
-    return reader;
+  public SegmentInfo read(Directory dir, String segment, IOContext context) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segment, "", Lucene50SegmentInfoFormat.SI_EXTENSION);
+    try (ChecksumIndexInput input = dir.openChecksumInput(fileName, context)) {
+      Throwable priorE = null;
+      SegmentInfo si = null;
+      try {
+        CodecUtil.checkHeader(input, Lucene50SegmentInfoFormat.CODEC_NAME,
+                                     Lucene50SegmentInfoFormat.VERSION_START,
+                                     Lucene50SegmentInfoFormat.VERSION_CURRENT);
+        final Version version = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
+        
+        final int docCount = input.readInt();
+        if (docCount < 0) {
+          throw new CorruptIndexException("invalid docCount: " + docCount, input);
+        }
+        final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
+        final Map<String,String> diagnostics = input.readStringStringMap();
+        final Set<String> files = input.readStringSet();
+        
+        byte[] id = new byte[StringHelper.ID_LENGTH];
+        input.readBytes(id, 0, id.length);
+        
+        si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, id);
+        si.setFiles(files);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(input, priorE);
+      }
+      return si;
+    }
   }
 
   @Override
-  public SegmentInfoWriter getSegmentInfoWriter() {
-    return writer;
+  public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene50SegmentInfoFormat.SI_EXTENSION);
+    si.addFile(fileName);
+
+    boolean success = false;
+    try (IndexOutput output = dir.createOutput(fileName, ioContext)) {
+      CodecUtil.writeHeader(output, Lucene50SegmentInfoFormat.CODEC_NAME, Lucene50SegmentInfoFormat.VERSION_CURRENT);
+      Version version = si.getVersion();
+      if (version.major < 5) {
+        throw new IllegalArgumentException("invalid major version: should be >= 5 but got: " + version.major + " segment=" + si);
+      }
+      // Write the Lucene version that created this segment, since 3.1
+      output.writeInt(version.major);
+      output.writeInt(version.minor);
+      output.writeInt(version.bugfix);
+      assert version.prerelease == 0;
+      output.writeInt(si.getDocCount());
+
+      output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
+      output.writeStringStringMap(si.getDiagnostics());
+      Set<String> files = si.files();
+      for (String file : files) {
+        if (!IndexFileNames.parseSegmentName(file).equals(si.name)) {
+          throw new IllegalArgumentException("invalid files: expected segment=" + si.name + ", got=" + files);
+        }
+      }
+      output.writeStringSet(files);
+      byte[] id = si.getId();
+      if (id.length != StringHelper.ID_LENGTH) {
+        throw new IllegalArgumentException("invalid id, got=" + StringHelper.idToString(id));
+      }
+      output.writeBytes(id, 0, id.length);
+      CodecUtil.writeFooter(output);
+      success = true;
+    } finally {
+      if (!success) {
+        // TODO: are we doing this outside of the tracking wrapper? why must SIWriter cleanup like this?
+        IOUtils.deleteFilesIgnoringExceptions(si.dir, fileName);
+      }
+    }
   }
 
   /** File extension used to store {@link SegmentInfo}. */

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java Wed Oct 15 01:26:26 2014
@@ -25,7 +25,6 @@ import java.util.Map;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.FieldInfosWriter;
 import org.apache.lucene.codecs.NormsConsumer;
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.StoredFieldsWriter;
@@ -118,8 +117,7 @@ final class DefaultIndexingChain extends
     // consumer can alter the FieldInfo* if necessary.  EG,
     // FreqProxTermsWriter does this with
     // FieldInfo.storePayload.
-    FieldInfosWriter infosWriter = docWriter.codec.fieldInfosFormat().getFieldInfosWriter();
-    infosWriter.write(state.directory, state.segmentInfo, "", state.fieldInfos, IOContext.DEFAULT);
+    docWriter.codec.fieldInfosFormat().write(state.directory, state.segmentInfo, "", state.fieldInfos, IOContext.DEFAULT);
   }
 
   /** Writes all buffered doc values (called from {@link #flush}). */
@@ -622,6 +620,10 @@ final class DefaultIndexingChain extends
             invertState.lastStartOffset = startOffset;
           }
 
+          invertState.length++;
+          if (invertState.length < 0) {
+            throw new IllegalArgumentException("too many tokens in field '" + field.name() + "'");
+          }
           //System.out.println("  term=" + invertState.termAttribute);
 
           // If we hit an exception in here, we abort
@@ -633,8 +635,6 @@ final class DefaultIndexingChain extends
           aborting = true;
           termsHashPerField.add();
           aborting = false;
-
-          invertState.length++;
         }
 
         // trigger streams to perform end-of-stream operations

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Wed Oct 15 01:26:26 2014
@@ -517,7 +517,7 @@ class DocumentsWriterPerThread {
       // creating CFS so that 1) .si isn't slurped into CFS,
       // and 2) .si reflects useCompoundFile=true change
       // above:
-      codec.segmentInfoFormat().getSegmentInfoWriter().write(directory, newSegment.info, flushedSegment.fieldInfos, context);
+      codec.segmentInfoFormat().write(directory, newSegment.info, context);
 
       // TODO: ideally we would freeze newSegment here!!
       // because any changes after writing the .si will be

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java Wed Oct 15 01:26:26 2014
@@ -25,9 +25,11 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Set;
 import java.util.regex.Matcher;
 
 import org.apache.lucene.store.AlreadyClosedException;
@@ -78,7 +80,7 @@ final class IndexFileDeleter implements 
   /* Files that we tried to delete but failed (likely
    * because they are open and we are running on Windows),
    * so we will retry them again later: */
-  private List<String> deletable;
+  private Set<String> deletable;
 
   /* Reference count for all files in the index.
    * Counts how many existing commits reference a file.
@@ -359,7 +361,7 @@ final class IndexFileDeleter implements 
    * Remove the CommitPoints in the commitsToDelete List by
    * DecRef'ing all files from each SegmentInfos.
    */
-  private void deleteCommits() throws IOException {
+  private void deleteCommits() {
 
     int size = commitsToDelete.size();
 
@@ -384,7 +386,7 @@ final class IndexFileDeleter implements 
       commitsToDelete.clear();
 
       // NOTE: does nothing if firstThrowable is null
-      IOUtils.reThrow(firstThrowable);
+      IOUtils.reThrowUnchecked(firstThrowable);
 
       // Now compact commits to remove deleted ones (preserving the sort):
       size = commits.size();
@@ -462,7 +464,7 @@ final class IndexFileDeleter implements 
   }
 
   @Override
-  public void close() throws IOException {
+  public void close() {
     // DecRef old files from the last checkpoint, if any:
     assert locked();
 
@@ -498,14 +500,12 @@ final class IndexFileDeleter implements 
     }
   }
 
-  public void deletePendingFiles() throws IOException {
+  public void deletePendingFiles() {
     assert locked();
     if (deletable != null) {
-      List<String> oldDeletable = deletable;
+      Set<String> oldDeletable = deletable;
       deletable = null;
-      int size = oldDeletable.size();
-      for(int i=0;i<size;i++) {
-        String fileName = oldDeletable.get(i);
+      for(String fileName : oldDeletable) {
         if (infoStream.isEnabled("IFD")) {
           infoStream.message("IFD", "delete pending file " + fileName);
         }
@@ -611,7 +611,7 @@ final class IndexFileDeleter implements 
   }
 
   /** Decrefs all provided files, even on exception; throws first exception hit, if any. */
-  void decRef(Collection<String> files) throws IOException {
+  void decRef(Collection<String> files) {
     assert locked();
     Throwable firstThrowable = null;
     for(final String file : files) {
@@ -626,12 +626,12 @@ final class IndexFileDeleter implements 
     }
 
     // NOTE: does nothing if firstThrowable is null
-    IOUtils.reThrow(firstThrowable);
+    IOUtils.reThrowUnchecked(firstThrowable);
   }
 
   /** Decrefs all provided files, ignoring any exceptions hit; call this if
    *  you are already handling an exception. */
-  void decRefWhileHandlingException(Collection<String> files) throws IOException {
+  void decRefWhileHandlingException(Collection<String> files) {
     assert locked();
     for(final String file : files) {
       try {
@@ -641,7 +641,7 @@ final class IndexFileDeleter implements 
     }
   }
 
-  void decRef(String fileName) throws IOException {
+  void decRef(String fileName) {
     assert locked();
     RefCount rc = getRefCount(fileName);
     if (infoStream.isEnabled("IFD")) {
@@ -679,6 +679,8 @@ final class IndexFileDeleter implements 
     RefCount rc;
     if (!refCounts.containsKey(fileName)) {
       rc = new RefCount(fileName);
+      // We should never incRef a file we are already wanting to delete:
+      assert deletable == null || deletable.contains(fileName) == false: "file \"" + fileName + "\" cannot be incRef'd: it's already pending delete";
       refCounts.put(fileName, rc);
     } else {
       rc = refCounts.get(fileName);
@@ -686,7 +688,7 @@ final class IndexFileDeleter implements 
     return rc;
   }
 
-  void deleteFiles(List<String> files) throws IOException {
+  void deleteFiles(List<String> files) {
     assert locked();
     for(final String file: files) {
       deleteFile(file);
@@ -695,7 +697,7 @@ final class IndexFileDeleter implements 
 
   /** Deletes the specified files, but only if they are new
    *  (have not yet been incref'd). */
-  void deleteNewFiles(Collection<String> files) throws IOException {
+  void deleteNewFiles(Collection<String> files) {
     assert locked();
     for (final String fileName: files) {
       // NOTE: it's very unusual yet possible for the
@@ -713,8 +715,7 @@ final class IndexFileDeleter implements 
     }
   }
 
-  void deleteFile(String fileName)
-       throws IOException {
+  void deleteFile(String fileName) {
     assert locked();
     ensureOpen();
     try {
@@ -734,7 +735,7 @@ final class IndexFileDeleter implements 
         infoStream.message("IFD", "unable to remove file \"" + fileName + "\": " + e.toString() + "; Will re-try later.");
       }
       if (deletable == null) {
-        deletable = new ArrayList<>();
+        deletable = new HashSet<>();
       }
       deletable.add(fileName);                  // add to deletable
     }

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Wed Oct 15 01:26:26 2014
@@ -41,7 +41,7 @@ import java.util.concurrent.atomic.Atomi
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.FieldInfosReader;
+import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
 import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
@@ -871,7 +871,7 @@ public class IndexWriter implements Clos
   // TODO: fix tests abusing this method!
   static FieldInfos readFieldInfos(SegmentCommitInfo si) throws IOException {
     Codec codec = si.info.getCodec();
-    FieldInfosReader reader = codec.fieldInfosFormat().getFieldInfosReader();
+    FieldInfosFormat reader = codec.fieldInfosFormat();
     
     if (si.hasFieldUpdates()) {
       // there are updates, we read latest (always outside of CFS)
@@ -2024,6 +2024,10 @@ public class IndexWriter implements Clos
         
         deleter.close();
 
+        // Must set closed while inside same sync block where we call deleter.refresh, else concurrent threads may try to sneak a flush in,
+        // after we leave this sync block and before we enter the sync block in the finally clause below that sets closed:
+        closed = true;
+
         IOUtils.close(writeLock);                     // release write lock
         writeLock = null;
         
@@ -2267,6 +2271,7 @@ public class IndexWriter implements Clos
     try {
       synchronized (this) {
         // Lock order IW -> BDS
+        ensureOpen(false);
         synchronized (bufferedUpdatesStream) {
           if (infoStream.isEnabled("IW")) {
             infoStream.message("IW", "publishFlushedSegment");
@@ -2542,10 +2547,9 @@ public class IndexWriter implements Clos
         return;
       }
 
-      MergeState mergeState;
       boolean success = false;
       try {
-        mergeState = merger.merge();                // merge 'em
+        merger.merge();                // merge 'em
         success = true;
       } finally {
         if (!success) { 
@@ -2594,7 +2598,7 @@ public class IndexWriter implements Clos
       // above:
       success = false;
       try {
-        codec.segmentInfoFormat().getSegmentInfoWriter().write(trackingDir, info, mergeState.mergeFieldInfos, context);
+        codec.segmentInfoFormat().write(trackingDir, info, context);
         success = true;
       } finally {
         if (!success) {
@@ -3845,8 +3849,6 @@ public class IndexWriter implements Clos
 
     merge.checkAborted(directory);
 
-    final String mergedName = merge.info.info.name;
-
     List<SegmentCommitInfo> sourceSegments = merge.segments;
     
     IOContext context = new IOContext(merge.getMergeInfo());
@@ -4060,7 +4062,7 @@ public class IndexWriter implements Clos
       // above:
       boolean success2 = false;
       try {
-        codec.segmentInfoFormat().getSegmentInfoWriter().write(directory, merge.info.info, mergeState.mergeFieldInfos, context);
+        codec.segmentInfoFormat().write(directory, merge.info.info, context);
         success2 = true;
       } finally {
         if (!success2) {
@@ -4511,7 +4513,7 @@ public class IndexWriter implements Clos
     Set<String> siFiles = new HashSet<>();
     for (String cfsFile : cfsFiles) {
       siFiles.add(cfsFile);
-    };
+    }
     info.setFiles(siFiles);
 
     return files;

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java Wed Oct 15 01:26:26 2014
@@ -117,8 +117,9 @@ public abstract class MergePolicy {
      * @param segments List of {@link SegmentCommitInfo}s
      *        to be merged. */
     public OneMerge(List<SegmentCommitInfo> segments) {
-      if (0 == segments.size())
+      if (0 == segments.size()) {
         throw new RuntimeException("segments must include at least one segment");
+      }
       // clone the list, as the in list may be based off original SegmentInfos and may be modified
       this.segments = new ArrayList<>(segments);
       int count = 0;
@@ -239,14 +240,17 @@ public abstract class MergePolicy {
       StringBuilder b = new StringBuilder();
       final int numSegments = segments.size();
       for(int i=0;i<numSegments;i++) {
-        if (i > 0) b.append(' ');
+        if (i > 0) {
+          b.append(' ');
+        }
         b.append(segments.get(i).toString(dir, 0));
       }
       if (info != null) {
         b.append(" into ").append(info.info.name);
       }
-      if (maxNumSegments != -1)
+      if (maxNumSegments != -1) {
         b.append(" [maxNumSegments=" + maxNumSegments + "]");
+      }
       if (aborted) {
         b.append(" [ABORTED]");
       }
@@ -312,8 +316,9 @@ public abstract class MergePolicy {
       StringBuilder b = new StringBuilder();
       b.append("MergeSpec:\n");
       final int count = merges.size();
-      for(int i=0;i<count;i++)
+      for(int i=0;i<count;i++) {
         b.append("  ").append(1 + i).append(": ").append(merges.get(i).segString(dir));
+      }
       return b.toString();
     }
   }
@@ -477,9 +482,9 @@ public abstract class MergePolicy {
   protected long size(SegmentCommitInfo info, IndexWriter writer) throws IOException {
     long byteSize = info.sizeInBytes();
     int delCount = writer.numDeletedDocs(info);
-    double delRatio = (info.info.getDocCount() <= 0 ? 0.0f : ((float)delCount / (float)info.info.getDocCount()));
+    double delRatio = info.info.getDocCount() <= 0 ? 0.0f : (float) delCount / (float) info.info.getDocCount();
     assert delRatio <= 1.0;
-    return (info.info.getDocCount() <= 0 ?  byteSize : (long)(byteSize * (1.0 - delRatio)));
+    return (info.info.getDocCount() <= 0 ? byteSize : (long) (byteSize * (1.0 - delRatio)));
   }
   
   /** Returns true if this single info is already fully merged (has no
@@ -527,7 +532,7 @@ public abstract class MergePolicy {
       throw new IllegalArgumentException("maxCFSSegmentSizeMB must be >=0 (got " + v + ")");
     }
     v *= 1024 * 1024;
-    this.maxCFSSegmentSize = (v > Long.MAX_VALUE) ? Long.MAX_VALUE : (long) v;
+    this.maxCFSSegmentSize = v > Long.MAX_VALUE ? Long.MAX_VALUE : (long) v;
   }
 
 }

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java Wed Oct 15 01:26:26 2014
@@ -451,7 +451,7 @@ class ReadersAndUpdates {
     final IOContext infosContext = new IOContext(new FlushInfo(info.info.getDocCount(), estInfosSize));
     // separately also track which files were created for this gen
     final TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
-    infosFormat.getFieldInfosWriter().write(trackingDir, info.info, segmentSuffix, fieldInfos, infosContext);
+    infosFormat.write(trackingDir, info.info, segmentSuffix, fieldInfos, infosContext);
     info.advanceFieldInfosGen();
     return trackingDir.getCreatedFiles();
   }

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java Wed Oct 15 01:26:26 2014
@@ -109,7 +109,7 @@ final class SegmentCoreReaders implement
         cfsDir = dir;
       }
 
-      coreFieldInfos = codec.fieldInfosFormat().getFieldInfosReader().read(cfsDir, si.info, "", context);
+      coreFieldInfos = codec.fieldInfosFormat().read(cfsDir, si.info, "", context);
       
       final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si.info, coreFieldInfos, context);
       final PostingsFormat format = codec.postingsFormat();

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1631928&r1=1631927&r2=1631928&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java Wed Oct 15 01:26:26 2014
@@ -307,7 +307,7 @@ public final class SegmentInfos implemen
         String segName = input.readString();
         Codec codec = Codec.forName(input.readString());
         //System.out.println("SIS.read seg=" + seg + " codec=" + codec);
-        SegmentInfo info = codec.segmentInfoFormat().getSegmentInfoReader().read(directory, segName, IOContext.READ);
+        SegmentInfo info = codec.segmentInfoFormat().read(directory, segName, IOContext.READ);
         info.setCodec(codec);
         long delGen = input.readLong();
         int delCount = input.readInt();