You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by iv...@apache.org on 2020/12/15 09:13:45 UTC

[lucene-solr] branch master updated: LUCENE-9627: Remove unused Lucene50FieldInfosFormat codec and small refactor some codecs to separate reading header/footer from reading content of the file

This is an automated email from the ASF dual-hosted git repository.

ivera pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new 4b3e8d7  LUCENE-9627: Remove unused Lucene50FieldInfosFormat codec and small refactor some codecs  to separate reading header/footer from reading content of the file
4b3e8d7 is described below

commit 4b3e8d7ce8feba658a9730e65da70c04a7e9c52f
Author: Ignacio Vera <iv...@apache.org>
AuthorDate: Tue Dec 15 10:13:25 2020 +0100

    LUCENE-9627: Remove unused Lucene50FieldInfosFormat codec and small refactor some codecs  to separate reading header/footer from reading content of the file
---
 lucene/CHANGES.txt                                 |   3 +
 .../lucene70/Lucene70SegmentInfoFormat.java        | 320 +++++++++++----------
 .../lucene70/Lucene70RWSegmentInfoFormat.java      | 268 ++++++++---------
 .../codecs/lucene50/Lucene50CompoundFormat.java    |  85 +++---
 .../codecs/lucene50/Lucene50CompoundReader.java    |  39 +--
 .../codecs/lucene50/Lucene50FieldInfosFormat.java  | 296 -------------------
 .../codecs/lucene50/Lucene50LiveDocsFormat.java    |  68 +++--
 .../codecs/lucene60/Lucene60FieldInfosFormat.java  | 140 ++++-----
 .../codecs/lucene80/Lucene80DocValuesProducer.java |  18 +-
 .../codecs/lucene86/Lucene86SegmentInfoFormat.java | 200 ++++++-------
 .../java/org/apache/lucene/index/SegmentInfos.java | 191 ++++++------
 11 files changed, 696 insertions(+), 932 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index e9d0bb3..6d9dc4e 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -220,6 +220,9 @@ Other
 
 * LUCENE-9544: add regenerate gradle script for nori dictionary (Namgyu Kim)
 
+* LUCENE-9627: Remove unused Lucene50FieldInfosFormat codec and small refactor some codecs
+  to separate reading header/footer from reading content of the file. (Ignacio Vera)
+
 ======================= Lucene 8.8.0 =======================
 
 New Features
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70SegmentInfoFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70SegmentInfoFormat.java
index 88ef783..c02a18a 100644
--- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70SegmentInfoFormat.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70SegmentInfoFormat.java
@@ -34,6 +34,7 @@ import org.apache.lucene.search.SortedNumericSortField;
 import org.apache.lucene.search.SortedSetSelector;
 import org.apache.lucene.search.SortedSetSortField;
 import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -80,6 +81,12 @@ import org.apache.lucene.util.Version;
  */
 public class Lucene70SegmentInfoFormat extends SegmentInfoFormat {
 
+  /** File extension used to store {@link SegmentInfo}. */
+  public final static String SI_EXTENSION = "si";
+  static final String CODEC_NAME = "Lucene70SegmentInfo";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+
   /** Sole constructor. */
   public Lucene70SegmentInfoFormat() {
   }
@@ -95,187 +102,188 @@ public class Lucene70SegmentInfoFormat extends SegmentInfoFormat {
                                                 Lucene70SegmentInfoFormat.VERSION_START,
                                                 Lucene70SegmentInfoFormat.VERSION_CURRENT,
                                                 segmentID, "");
-        final Version version = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
-        byte hasMinVersion = input.readByte();
-        final Version minVersion;
-        switch (hasMinVersion) {
+
+        si = parseSegmentInfo(dir, input, segment, segmentID);
+       
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(input, priorE);
+      }
+      return si;
+    }
+  }
+
+  private SegmentInfo parseSegmentInfo(Directory dir, DataInput input, String segment, byte[] segmentID) throws IOException {
+    final Version version = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
+    byte hasMinVersion = input.readByte();
+    final Version minVersion;
+    switch (hasMinVersion) {
+      case 0:
+        minVersion = null;
+        break;
+      case 1:
+        minVersion = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
+        break;
+      default:
+        throw new CorruptIndexException("Illegal boolean value " + hasMinVersion, input);
+    }
+
+    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.readMapOfStrings();
+    final Set<String> files = input.readSetOfStrings();
+    final Map<String,String> attributes = input.readMapOfStrings();
+
+    int numSortFields = input.readVInt();
+    Sort indexSort;
+    if (numSortFields > 0) {
+      SortField[] sortFields = new SortField[numSortFields];
+      for(int i=0;i<numSortFields;i++) {
+        String fieldName = input.readString();
+        int sortTypeID = input.readVInt();
+        SortField.Type sortType;
+        SortedSetSelector.Type sortedSetSelector = null;
+        SortedNumericSelector.Type sortedNumericSelector = null;
+        switch(sortTypeID) {
           case 0:
-            minVersion = null;
+            sortType = SortField.Type.STRING;
             break;
           case 1:
-            minVersion = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
+            sortType = SortField.Type.LONG;
+            break;
+          case 2:
+            sortType = SortField.Type.INT;
+            break;
+          case 3:
+            sortType = SortField.Type.DOUBLE;
+            break;
+          case 4:
+            sortType = SortField.Type.FLOAT;
+            break;
+          case 5:
+            sortType = SortField.Type.STRING;
+            byte selector = input.readByte();
+            if (selector == 0) {
+              sortedSetSelector = SortedSetSelector.Type.MIN;
+            } else if (selector == 1) {
+              sortedSetSelector = SortedSetSelector.Type.MAX;
+            } else if (selector == 2) {
+              sortedSetSelector = SortedSetSelector.Type.MIDDLE_MIN;
+            } else if (selector == 3) {
+              sortedSetSelector = SortedSetSelector.Type.MIDDLE_MAX;
+            } else {
+              throw new CorruptIndexException("invalid index SortedSetSelector ID: " + selector, input);
+            }
+            break;
+          case 6:
+            byte type = input.readByte();
+            if (type == 0) {
+              sortType = SortField.Type.LONG;
+            } else if (type == 1) {
+              sortType = SortField.Type.INT;
+            } else if (type == 2) {
+              sortType = SortField.Type.DOUBLE;
+            } else if (type == 3) {
+              sortType = SortField.Type.FLOAT;
+            } else {
+              throw new CorruptIndexException("invalid index SortedNumericSortField type ID: " + type, input);
+            }
+            byte numericSelector = input.readByte();
+            if (numericSelector == 0) {
+              sortedNumericSelector = SortedNumericSelector.Type.MIN;
+            } else if (numericSelector == 1) {
+              sortedNumericSelector = SortedNumericSelector.Type.MAX;
+            } else {
+              throw new CorruptIndexException("invalid index SortedNumericSelector ID: " + numericSelector, input);
+            }
             break;
           default:
-            throw new CorruptIndexException("Illegal boolean value " + hasMinVersion, input);
+            throw new CorruptIndexException("invalid index sort field type ID: " + sortTypeID, input);
         }
-
-        final int docCount = input.readInt();
-        if (docCount < 0) {
-          throw new CorruptIndexException("invalid docCount: " + docCount, input);
+        byte b = input.readByte();
+        boolean reverse;
+        if (b == 0) {
+          reverse = true;
+        } else if (b == 1) {
+          reverse = false;
+        } else {
+          throw new CorruptIndexException("invalid index sort reverse: " + b, input);
         }
-        final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
 
-        final Map<String,String> diagnostics = input.readMapOfStrings();
-        final Set<String> files = input.readSetOfStrings();
-        final Map<String,String> attributes = input.readMapOfStrings();
+        if (sortedSetSelector != null) {
+          sortFields[i] = new SortedSetSortField(fieldName, reverse, sortedSetSelector);
+        } else if (sortedNumericSelector != null) {
+          sortFields[i] = new SortedNumericSortField(fieldName, sortType, reverse, sortedNumericSelector);
+        } else {
+          sortFields[i] = new SortField(fieldName, sortType, reverse);
+        }
 
-        int numSortFields = input.readVInt();
-        Sort indexSort;
-        if (numSortFields > 0) {
-          SortField[] sortFields = new SortField[numSortFields];
-          for(int i=0;i<numSortFields;i++) {
-            String fieldName = input.readString();
-            int sortTypeID = input.readVInt();
-            SortField.Type sortType;
-            SortedSetSelector.Type sortedSetSelector = null;
-            SortedNumericSelector.Type sortedNumericSelector = null;
-            switch(sortTypeID) {
-            case 0:
-              sortType = SortField.Type.STRING;
-              break;
-            case 1:
-              sortType = SortField.Type.LONG;
-              break;
-            case 2:
-              sortType = SortField.Type.INT;
-              break;
-            case 3:
-              sortType = SortField.Type.DOUBLE;
+        Object missingValue;
+        b = input.readByte();
+        if (b == 0) {
+          missingValue = null;
+        } else {
+          switch(sortType) {
+            case STRING:
+              if (b == 1) {
+                missingValue = SortField.STRING_LAST;
+              } else if (b == 2) {
+                missingValue = SortField.STRING_FIRST;
+              } else {
+                throw new CorruptIndexException("invalid missing value flag: " + b, input);
+              }
               break;
-            case 4:
-              sortType = SortField.Type.FLOAT;
+            case LONG:
+              if (b != 1) {
+                throw new CorruptIndexException("invalid missing value flag: " + b, input);
+              }
+              missingValue = input.readLong();
               break;
-            case 5:
-              sortType = SortField.Type.STRING;
-              byte selector = input.readByte();
-              if (selector == 0) {
-                sortedSetSelector = SortedSetSelector.Type.MIN;
-              } else if (selector == 1) {
-                sortedSetSelector = SortedSetSelector.Type.MAX;
-              } else if (selector == 2) {
-                sortedSetSelector = SortedSetSelector.Type.MIDDLE_MIN;
-              } else if (selector == 3) {
-                sortedSetSelector = SortedSetSelector.Type.MIDDLE_MAX;
-              } else {
-                throw new CorruptIndexException("invalid index SortedSetSelector ID: " + selector, input);
+            case INT:
+              if (b != 1) {
+                throw new CorruptIndexException("invalid missing value flag: " + b, input);
               }
+              missingValue = input.readInt();
               break;
-            case 6:
-              byte type = input.readByte();
-              if (type == 0) {
-                sortType = SortField.Type.LONG;
-              } else if (type == 1) {
-                sortType = SortField.Type.INT;
-              } else if (type == 2) {
-                sortType = SortField.Type.DOUBLE;
-              } else if (type == 3) {
-                sortType = SortField.Type.FLOAT;
-              } else {
-                throw new CorruptIndexException("invalid index SortedNumericSortField type ID: " + type, input);
+            case DOUBLE:
+              if (b != 1) {
+                throw new CorruptIndexException("invalid missing value flag: " + b, input);
               }
-              byte numericSelector = input.readByte();
-              if (numericSelector == 0) {
-                sortedNumericSelector = SortedNumericSelector.Type.MIN;
-              } else if (numericSelector == 1) {
-                sortedNumericSelector = SortedNumericSelector.Type.MAX;
-              } else {
-                throw new CorruptIndexException("invalid index SortedNumericSelector ID: " + numericSelector, input);
+              missingValue = Double.longBitsToDouble(input.readLong());
+              break;
+            case FLOAT:
+              if (b != 1) {
+                throw new CorruptIndexException("invalid missing value flag: " + b, input);
               }
+              missingValue = Float.intBitsToFloat(input.readInt());
               break;
             default:
-              throw new CorruptIndexException("invalid index sort field type ID: " + sortTypeID, input);
-            }
-            byte b = input.readByte();
-            boolean reverse;
-            if (b == 0) {
-              reverse = true;
-            } else if (b == 1) {
-              reverse = false;
-            } else {
-              throw new CorruptIndexException("invalid index sort reverse: " + b, input);
-            }
-
-            if (sortedSetSelector != null) {
-              sortFields[i] = new SortedSetSortField(fieldName, reverse, sortedSetSelector);
-            } else if (sortedNumericSelector != null) {
-              sortFields[i] = new SortedNumericSortField(fieldName, sortType, reverse, sortedNumericSelector);
-            } else {
-              sortFields[i] = new SortField(fieldName, sortType, reverse);
-            }
-
-            Object missingValue;
-            b = input.readByte();
-            if (b == 0) {
-              missingValue = null;
-            } else {
-              switch(sortType) {
-              case STRING:
-                if (b == 1) {
-                  missingValue = SortField.STRING_LAST;
-                } else if (b == 2) {
-                  missingValue = SortField.STRING_FIRST;
-                } else {
-                  throw new CorruptIndexException("invalid missing value flag: " + b, input);
-                }
-                break;
-              case LONG:
-                if (b != 1) {
-                  throw new CorruptIndexException("invalid missing value flag: " + b, input);
-                }
-                missingValue = input.readLong();
-                break;
-              case INT:
-                if (b != 1) {
-                  throw new CorruptIndexException("invalid missing value flag: " + b, input);
-                }
-                missingValue = input.readInt();
-                break;
-              case DOUBLE:
-                if (b != 1) {
-                  throw new CorruptIndexException("invalid missing value flag: " + b, input);
-                }
-                missingValue = Double.longBitsToDouble(input.readLong());
-                break;
-              case FLOAT:
-                if (b != 1) {
-                  throw new CorruptIndexException("invalid missing value flag: " + b, input);
-                }
-                missingValue = Float.intBitsToFloat(input.readInt());
-                break;
-              default:
-                throw new AssertionError("unhandled sortType=" + sortType);
-              }
-            }
-            if (missingValue != null) {
-              sortFields[i].setMissingValue(missingValue);
-            }
+              throw new AssertionError("unhandled sortType=" + sortType);
           }
-          indexSort = new Sort(sortFields);
-        } else if (numSortFields < 0) {
-          throw new CorruptIndexException("invalid index sort field count: " + numSortFields, input);
-        } else {
-          indexSort = null;
         }
-
-        si = new SegmentInfo(dir, version, minVersion, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, indexSort);
-        si.setFiles(files);
-      } catch (Throwable exception) {
-        priorE = exception;
-      } finally {
-        CodecUtil.checkFooter(input, priorE);
+        if (missingValue != null) {
+          sortFields[i].setMissingValue(missingValue);
+        }
       }
-      return si;
+      indexSort = new Sort(sortFields);
+    } else if (numSortFields < 0) {
+      throw new CorruptIndexException("invalid index sort field count: " + numSortFields, input);
+    } else {
+      indexSort = null;
     }
+
+    SegmentInfo si = new SegmentInfo(dir, version, minVersion, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, indexSort);
+    si.setFiles(files);
+    return si;
   }
 
   @Override
   public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
     throw new UnsupportedOperationException("Old formats can't be used for writing");
   }
-
-  /** File extension used to store {@link SegmentInfo}. */
-  public final static String SI_EXTENSION = "si";
-  static final String CODEC_NAME = "Lucene70SegmentInfo";
-  static final int VERSION_START = 0;
-  static final int VERSION_CURRENT = VERSION_START;
 }
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/Lucene70RWSegmentInfoFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/Lucene70RWSegmentInfoFormat.java
index 90ccf4a..3665bfd 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/Lucene70RWSegmentInfoFormat.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/Lucene70RWSegmentInfoFormat.java
@@ -47,158 +47,162 @@ public class Lucene70RWSegmentInfoFormat extends Lucene70SegmentInfoFormat {
       // Only add the file once we've successfully created it, else IFD assert can trip:
       si.addFile(fileName);
       CodecUtil.writeIndexHeader(output,
-          Lucene70SegmentInfoFormat.CODEC_NAME,
-          Lucene70SegmentInfoFormat.VERSION_CURRENT,
-          si.getId(),
-          "");
-      Version version = si.getVersion();
-      if (version.major < 7) {
-        throw new IllegalArgumentException("invalid major version: should be >= 7 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);
+              Lucene70SegmentInfoFormat.CODEC_NAME,
+              Lucene70SegmentInfoFormat.VERSION_CURRENT,
+              si.getId(),
+              "");
+     
+      writeSegmentInfo(output, si);
 
-      // Write the min Lucene version that contributed docs to the segment, since 7.0
-      if (si.getMinVersion() != null) {
-        output.writeByte((byte) 1);
-        Version minVersion = si.getMinVersion();
-        output.writeInt(minVersion.major);
-        output.writeInt(minVersion.minor);
-        output.writeInt(minVersion.bugfix);
-      } else {
-        output.writeByte((byte) 0);
-      }
+      CodecUtil.writeFooter(output);
+    }
+  }
+  
+  private void writeSegmentInfo(IndexOutput output, SegmentInfo si) throws IOException {
+    Version version = si.getVersion();
+    if (version.major < 7) {
+      throw new IllegalArgumentException("invalid major version: should be >= 7 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.maxDoc());
+    // Write the min Lucene version that contributed docs to the segment, since 7.0
+    if (si.getMinVersion() != null) {
+      output.writeByte((byte) 1);
+      Version minVersion = si.getMinVersion();
+      output.writeInt(minVersion.major);
+      output.writeInt(minVersion.minor);
+      output.writeInt(minVersion.bugfix);
+    } else {
+      output.writeByte((byte) 0);
+    }
+
+    assert version.prerelease == 0;
+    output.writeInt(si.maxDoc());
 
-      output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
-      output.writeMapOfStrings(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.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
+    output.writeMapOfStrings(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.writeSetOfStrings(files);
+    output.writeMapOfStrings(si.getAttributes());
+
+    Sort indexSort = si.getIndexSort();
+    int numSortFields = indexSort == null ? 0 : indexSort.getSort().length;
+    output.writeVInt(numSortFields);
+    for (int i = 0; i < numSortFields; ++i) {
+      SortField sortField = indexSort.getSort()[i];
+      SortField.Type sortType = sortField.getType();
+      output.writeString(sortField.getField());
+      int sortTypeID;
+      switch (sortField.getType()) {
+        case STRING:
+          sortTypeID = 0;
+          break;
+        case LONG:
+          sortTypeID = 1;
+          break;
+        case INT:
+          sortTypeID = 2;
+          break;
+        case DOUBLE:
+          sortTypeID = 3;
+          break;
+        case FLOAT:
+          sortTypeID = 4;
+          break;
+        case CUSTOM:
+          if (sortField instanceof SortedSetSortField) {
+            sortTypeID = 5;
+            sortType = SortField.Type.STRING;
+          } else if (sortField instanceof SortedNumericSortField) {
+            sortTypeID = 6;
+            sortType = ((SortedNumericSortField) sortField).getNumericType();
+          } else {
+            throw new IllegalStateException("Unexpected SortedNumericSortField " + sortField);
+          }
+          break;
+        default:
+          throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
+      }
+      output.writeVInt(sortTypeID);
+      if (sortTypeID == 5) {
+        SortedSetSortField ssf = (SortedSetSortField) sortField;
+        if (ssf.getSelector() == SortedSetSelector.Type.MIN) {
+          output.writeByte((byte) 0);
+        } else if (ssf.getSelector() == SortedSetSelector.Type.MAX) {
+          output.writeByte((byte) 1);
+        } else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MIN) {
+          output.writeByte((byte) 2);
+        } else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MAX) {
+          output.writeByte((byte) 3);
+        } else {
+          throw new IllegalStateException("Unexpected SortedSetSelector type: " + ssf.getSelector());
+        }
+      } else if (sortTypeID == 6) {
+        SortedNumericSortField snsf = (SortedNumericSortField) sortField;
+        if (snsf.getNumericType() == SortField.Type.LONG) {
+          output.writeByte((byte) 0);
+        } else if (snsf.getNumericType() == SortField.Type.INT) {
+          output.writeByte((byte) 1);
+        } else if (snsf.getNumericType() == SortField.Type.DOUBLE) {
+          output.writeByte((byte) 2);
+        } else if (snsf.getNumericType() == SortField.Type.FLOAT) {
+          output.writeByte((byte) 3);
+        } else {
+          throw new IllegalStateException("Unexpected SortedNumericSelector type: " + snsf.getNumericType());
+        }
+        if (snsf.getSelector() == SortedNumericSelector.Type.MIN) {
+          output.writeByte((byte) 0);
+        } else if (snsf.getSelector() == SortedNumericSelector.Type.MAX) {
+          output.writeByte((byte) 1);
+        } else {
+          throw new IllegalStateException("Unexpected sorted numeric selector type: " + snsf.getSelector());
         }
       }
-      output.writeSetOfStrings(files);
-      output.writeMapOfStrings(si.getAttributes());
+      output.writeByte((byte) (sortField.getReverse() ? 0 : 1));
 
-      Sort indexSort = si.getIndexSort();
-      int numSortFields = indexSort == null ? 0 : indexSort.getSort().length;
-      output.writeVInt(numSortFields);
-      for (int i = 0; i < numSortFields; ++i) {
-        SortField sortField = indexSort.getSort()[i];
-        SortField.Type sortType = sortField.getType();
-        output.writeString(sortField.getField());
-        int sortTypeID;
-        switch (sortField.getType()) {
+      // write missing value
+      Object missingValue = sortField.getMissingValue();
+      if (missingValue == null) {
+        output.writeByte((byte) 0);
+      } else {
+        switch(sortType) {
           case STRING:
-            sortTypeID = 0;
+            if (missingValue == SortField.STRING_LAST) {
+              output.writeByte((byte) 1);
+            } else if (missingValue == SortField.STRING_FIRST) {
+              output.writeByte((byte) 2);
+            } else {
+              throw new AssertionError("unrecognized missing value for STRING field \"" + sortField.getField() + "\": " + missingValue);
+            }
             break;
           case LONG:
-            sortTypeID = 1;
+            output.writeByte((byte) 1);
+            output.writeLong(((Long) missingValue).longValue());
             break;
           case INT:
-            sortTypeID = 2;
+            output.writeByte((byte) 1);
+            output.writeInt(((Integer) missingValue).intValue());
             break;
           case DOUBLE:
-            sortTypeID = 3;
+            output.writeByte((byte) 1);
+            output.writeLong(Double.doubleToLongBits(((Double) missingValue).doubleValue()));
             break;
           case FLOAT:
-            sortTypeID = 4;
-            break;
-          case CUSTOM:
-            if (sortField instanceof SortedSetSortField) {
-              sortTypeID = 5;
-              sortType = SortField.Type.STRING;
-            } else if (sortField instanceof SortedNumericSortField) {
-              sortTypeID = 6;
-              sortType = ((SortedNumericSortField) sortField).getNumericType();
-            } else {
-              throw new IllegalStateException("Unexpected SortedNumericSortField " + sortField);
-            }
+            output.writeByte((byte) 1);
+            output.writeInt(Float.floatToIntBits(((Float) missingValue).floatValue()));
             break;
           default:
             throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
         }
-        output.writeVInt(sortTypeID);
-        if (sortTypeID == 5) {
-          SortedSetSortField ssf = (SortedSetSortField) sortField;
-          if (ssf.getSelector() == SortedSetSelector.Type.MIN) {
-            output.writeByte((byte) 0);
-          } else if (ssf.getSelector() == SortedSetSelector.Type.MAX) {
-            output.writeByte((byte) 1);
-          } else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MIN) {
-            output.writeByte((byte) 2);
-          } else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MAX) {
-            output.writeByte((byte) 3);
-          } else {
-            throw new IllegalStateException("Unexpected SortedSetSelector type: " + ssf.getSelector());
-          }
-        } else if (sortTypeID == 6) {
-          SortedNumericSortField snsf = (SortedNumericSortField) sortField;
-          if (snsf.getNumericType() == SortField.Type.LONG) {
-            output.writeByte((byte) 0);
-          } else if (snsf.getNumericType() == SortField.Type.INT) {
-            output.writeByte((byte) 1);
-          } else if (snsf.getNumericType() == SortField.Type.DOUBLE) {
-            output.writeByte((byte) 2);
-          } else if (snsf.getNumericType() == SortField.Type.FLOAT) {
-            output.writeByte((byte) 3);
-          } else {
-            throw new IllegalStateException("Unexpected SortedNumericSelector type: " + snsf.getNumericType());
-          }
-          if (snsf.getSelector() == SortedNumericSelector.Type.MIN) {
-            output.writeByte((byte) 0);
-          } else if (snsf.getSelector() == SortedNumericSelector.Type.MAX) {
-            output.writeByte((byte) 1);
-          } else {
-            throw new IllegalStateException("Unexpected sorted numeric selector type: " + snsf.getSelector());
-          }
-        }
-        output.writeByte((byte) (sortField.getReverse() ? 0 : 1));
-
-        // write missing value
-        Object missingValue = sortField.getMissingValue();
-        if (missingValue == null) {
-          output.writeByte((byte) 0);
-        } else {
-          switch(sortType) {
-            case STRING:
-              if (missingValue == SortField.STRING_LAST) {
-                output.writeByte((byte) 1);
-              } else if (missingValue == SortField.STRING_FIRST) {
-                output.writeByte((byte) 2);
-              } else {
-                throw new AssertionError("unrecognized missing value for STRING field \"" + sortField.getField() + "\": " + missingValue);
-              }
-              break;
-            case LONG:
-              output.writeByte((byte) 1);
-              output.writeLong(((Long) missingValue).longValue());
-              break;
-            case INT:
-              output.writeByte((byte) 1);
-              output.writeInt(((Integer) missingValue).intValue());
-              break;
-            case DOUBLE:
-              output.writeByte((byte) 1);
-              output.writeLong(Double.doubleToLongBits(((Double) missingValue).doubleValue()));
-              break;
-            case FLOAT:
-              output.writeByte((byte) 1);
-              output.writeInt(Float.floatToIntBits(((Float) missingValue).floatValue()));
-              break;
-            default:
-              throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
-          }
-        }
       }
-
-      CodecUtil.writeFooter(output);
     }
   }
-
 }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java
index 01bc917..2180dad 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java
@@ -62,6 +62,15 @@ import org.apache.lucene.store.IndexOutput;
  */
 public final class Lucene50CompoundFormat extends CompoundFormat {
 
+  /** Extension of compound file */
+  static final String DATA_EXTENSION = "cfs";
+  /** Extension of compound file entries */
+  static final String ENTRIES_EXTENSION = "cfe";
+  static final String DATA_CODEC = "Lucene50CompoundData";
+  static final String ENTRY_CODEC = "Lucene50CompoundEntries";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+  
   /** Sole constructor. */
   public Lucene50CompoundFormat() {
   }
@@ -80,52 +89,46 @@ public final class Lucene50CompoundFormat extends CompoundFormat {
          IndexOutput entries = dir.createOutput(entriesFile, context)) {
       CodecUtil.writeIndexHeader(data,    DATA_CODEC, VERSION_CURRENT, si.getId(), "");
       CodecUtil.writeIndexHeader(entries, ENTRY_CODEC, VERSION_CURRENT, si.getId(), "");
-      
-      // write number of files
-      entries.writeVInt(si.files().size());
-      for (String file : si.files()) {
-        
-        // write bytes for file
-        long startOffset = data.getFilePointer();
-        try (ChecksumIndexInput in = dir.openChecksumInput(file, IOContext.READONCE)) {
-
-          // just copies the index header, verifying that its id matches what we expect
-          CodecUtil.verifyAndCopyIndexHeader(in, data, si.getId());
-          
-          // copy all bytes except the footer
-          long numBytesToCopy = in.length() - CodecUtil.footerLength() - in.getFilePointer();
-          data.copyBytes(in, numBytesToCopy);
-
-          // verify footer (checksum) matches for the incoming file we are copying
-          long checksum = CodecUtil.checkFooter(in);
 
-          // this is poached from CodecUtil.writeFooter, but we need to use our own checksum, not data.getChecksum(), but I think
-          // adding a public method to CodecUtil to do that is somewhat dangerous:
-          data.writeInt(CodecUtil.FOOTER_MAGIC);
-          data.writeInt(0);
-          data.writeLong(checksum);
-        }
-        long endOffset = data.getFilePointer();
-        
-        long length = endOffset - startOffset;
-        
-        // write entry for file
-        entries.writeString(IndexFileNames.stripSegmentName(file));
-        entries.writeLong(startOffset);
-        entries.writeLong(length);
-      }
+      writeCompoundFile(entries, data, dir, si);
       
       CodecUtil.writeFooter(data);
       CodecUtil.writeFooter(entries);
     }
   }
+  
+  private void writeCompoundFile(IndexOutput entries, IndexOutput data, Directory dir, SegmentInfo si) throws IOException {
+    // write number of files
+    entries.writeVInt(si.files().size());
+    for (String file : si.files()) {
+      // write bytes for file
+      long startOffset = data.getFilePointer();
+      try (ChecksumIndexInput in = dir.openChecksumInput(file, IOContext.READONCE)) {
 
-  /** Extension of compound file */
-  static final String DATA_EXTENSION = "cfs";
-  /** Extension of compound file entries */
-  static final String ENTRIES_EXTENSION = "cfe";
-  static final String DATA_CODEC = "Lucene50CompoundData";
-  static final String ENTRY_CODEC = "Lucene50CompoundEntries";
-  static final int VERSION_START = 0;
-  static final int VERSION_CURRENT = VERSION_START;
+        // just copies the index header, verifying that its id matches what we expect
+        CodecUtil.verifyAndCopyIndexHeader(in, data, si.getId());
+
+        // copy all bytes except the footer
+        long numBytesToCopy = in.length() - CodecUtil.footerLength() - in.getFilePointer();
+        data.copyBytes(in, numBytesToCopy);
+
+        // verify footer (checksum) matches for the incoming file we are copying
+        long checksum = CodecUtil.checkFooter(in);
+
+        // this is poached from CodecUtil.writeFooter, but we need to use our own checksum, not data.getChecksum(), but I think
+        // adding a public method to CodecUtil to do that is somewhat dangerous:
+        data.writeInt(CodecUtil.FOOTER_MAGIC);
+        data.writeInt(0);
+        data.writeLong(checksum);
+      }
+      long endOffset = data.getFilePointer();
+
+      long length = endOffset - startOffset;
+
+      // write entry for file
+      entries.writeString(IndexFileNames.stripSegmentName(file));
+      entries.writeLong(startOffset);
+      entries.writeLong(length);
+    }
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java
index f4e7b9f..99cb3d2 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java
@@ -100,25 +100,16 @@ final class Lucene50CompoundReader extends CompoundDirectory {
 
   /** Helper method that reads CFS entries from an input stream */
   private Map<String, FileEntry> readEntries(byte[] segmentID, Directory dir, String entriesFileName) throws IOException {
-    Map<String,FileEntry> mapping = null;
+    Map<String, FileEntry> mapping = null;
     try (ChecksumIndexInput entriesStream = dir.openChecksumInput(entriesFileName, IOContext.READONCE)) {
       Throwable priorE = null;
       try {
-        version = CodecUtil.checkIndexHeader(entriesStream, Lucene50CompoundFormat.ENTRY_CODEC, 
-                                                              Lucene50CompoundFormat.VERSION_START, 
-                                                              Lucene50CompoundFormat.VERSION_CURRENT, segmentID, "");
-        final int numEntries = entriesStream.readVInt();
-        mapping = new HashMap<>(numEntries);
-        for (int i = 0; i < numEntries; i++) {
-          final FileEntry fileEntry = new FileEntry();
-          final String id = entriesStream.readString();
-          FileEntry previous = mapping.put(id, fileEntry);
-          if (previous != null) {
-            throw new CorruptIndexException("Duplicate cfs entry id=" + id + " in CFS ", entriesStream);
-          }
-          fileEntry.offset = entriesStream.readLong();
-          fileEntry.length = entriesStream.readLong();
-        }
+        version = CodecUtil.checkIndexHeader(entriesStream, Lucene50CompoundFormat.ENTRY_CODEC,
+                Lucene50CompoundFormat.VERSION_START,
+                Lucene50CompoundFormat.VERSION_CURRENT, segmentID, "");
+        
+        mapping = readMapping(entriesStream);
+        
       } catch (Throwable exception) {
         priorE = exception;
       } finally {
@@ -128,6 +119,22 @@ final class Lucene50CompoundReader extends CompoundDirectory {
     return Collections.unmodifiableMap(mapping);
   }
   
+  private Map<String,FileEntry> readMapping(IndexInput entriesStream) throws IOException {
+    final int numEntries = entriesStream.readVInt();
+    Map<String,FileEntry> mapping = new HashMap<>(numEntries);
+    for (int i = 0; i < numEntries; i++) {
+      final FileEntry fileEntry = new FileEntry();
+      final String id = entriesStream.readString();
+      FileEntry previous = mapping.put(id, fileEntry);
+      if (previous != null) {
+        throw new CorruptIndexException("Duplicate cfs entry id=" + id + " in CFS ", entriesStream);
+      }
+      fileEntry.offset = entriesStream.readLong();
+      fileEntry.length = entriesStream.readLong();
+    }
+    return mapping;
+  }
+  
   @Override
   public void close() throws IOException {
     IOUtils.close(handle);
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
deleted file mode 100644
index b632dad..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
+++ /dev/null
@@ -1,296 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.codecs.lucene50;
-
-
-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.index.CorruptIndexException;
-import org.apache.lucene.index.DocValuesType;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.IndexOptions;
-import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.VectorValues;
-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.
- * <p>Field names are stored in the field info file, with suffix <code>.fnm</code>.
- * <p>FieldInfos (.fnm) --&gt; Header,FieldsCount, &lt;FieldName,FieldNumber,
- * FieldBits,DocValuesBits,DocValuesGen,Attributes&gt; <sup>FieldsCount</sup>,Footer
- * <p>Data types:
- * <ul>
- *   <li>Header --&gt; {@link CodecUtil#checkIndexHeader IndexHeader}</li>
- *   <li>FieldsCount --&gt; {@link DataOutput#writeVInt VInt}</li>
- *   <li>FieldName --&gt; {@link DataOutput#writeString String}</li>
- *   <li>FieldBits, IndexOptions, DocValuesBits --&gt; {@link DataOutput#writeByte Byte}</li>
- *   <li>FieldNumber --&gt; {@link DataOutput#writeInt VInt}</li>
- *   <li>Attributes --&gt; {@link DataOutput#writeMapOfStrings Map&lt;String,String&gt;}</li>
- *   <li>DocValuesGen --&gt; {@link DataOutput#writeLong(long) Int64}</li>
- *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- * </ul>
- * Field Descriptions:
- * <ul>
- *   <li>FieldsCount: the number of fields in this file.</li>
- *   <li>FieldName: name of the field as a UTF-8 String.</li>
- *   <li>FieldNumber: the field's number. Note that unlike previous versions of
- *       Lucene, the fields are not numbered implicitly by their order in the
- *       file, instead explicitly.</li>
- *   <li>FieldBits: a byte containing field options.
- *     <ul>
- *       <li>The low order bit (0x1) is one for fields that have term vectors
- *           stored, and zero for fields without term vectors.</li>
- *       <li>If the second lowest order-bit is set (0x2), norms are omitted for the
- *           indexed field.</li>
- *       <li>If the third lowest-order bit is set (0x4), payloads are stored for the
- *           indexed field.</li>
- *     </ul>
- *   </li>
- *   <li>IndexOptions: a byte containing index options.
- *     <ul>
- *       <li>0: not indexed</li>
- *       <li>1: indexed as DOCS_ONLY</li>
- *       <li>2: indexed as DOCS_AND_FREQS</li>
- *       <li>3: indexed as DOCS_AND_FREQS_AND_POSITIONS</li>
- *       <li>4: indexed as DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS</li>
- *     </ul>
- *   </li>
- *   <li>DocValuesBits: a byte containing per-document value types. The type
- *       recorded as two four-bit integers, with the high-order bits representing
- *       <code>norms</code> options, and the low-order bits representing 
- *       {@code DocValues} options. Each four-bit integer can be decoded as such:
- *     <ul>
- *       <li>0: no DocValues for this field.</li>
- *       <li>1: NumericDocValues. ({@link DocValuesType#NUMERIC})</li>
- *       <li>2: BinaryDocValues. ({@code DocValuesType#BINARY})</li>
- *       <li>3: SortedDocValues. ({@code DocValuesType#SORTED})</li>
- *      </ul>
- *   </li>
- *   <li>DocValuesGen is the generation count of the field's DocValues. If this is -1,
- *       there are no DocValues updates to that field. Anything above zero means there 
- *       are updates stored by {@link DocValuesFormat}.</li>
- *   <li>Attributes: a key-value map of codec-private attributes.</li>
- * </ul>
- *
- * @lucene.experimental
- */
-public final class Lucene50FieldInfosFormat extends FieldInfosFormat {
-
-  /** Sole constructor. */
-  public Lucene50FieldInfosFormat() {
-  }
-  
-  @Override
-  public FieldInfos read(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, IOContext context) throws IOException {
-    final String fileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, EXTENSION);
-    try (ChecksumIndexInput input = directory.openChecksumInput(fileName, context)) {
-      Throwable priorE = null;
-      FieldInfo infos[] = null;
-      try {
-        CodecUtil.checkIndexHeader(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];
-        
-        // previous field's attribute map, we share when possible:
-        Map<String,String> lastAttributes = Collections.emptyMap();
-        
-        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 storeTermVector = (bits & STORE_TERMVECTOR) != 0;
-          boolean omitNorms = (bits & OMIT_NORMS) != 0;
-          boolean storePayloads = (bits & STORE_PAYLOADS) != 0;
-
-          final IndexOptions indexOptions = getIndexOptions(input, input.readByte());
-          
-          // DV Types are packed in one byte
-          final DocValuesType docValuesType = getDocValuesType(input, input.readByte());
-          final long dvGen = input.readLong();
-          Map<String,String> attributes = input.readMapOfStrings();
-
-          // just use the last field's map if its the same
-          if (attributes.equals(lastAttributes)) {
-            attributes = lastAttributes;
-          }
-          lastAttributes = attributes;
-          try {
-            infos[i] = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads, 
-                                     indexOptions, docValuesType, dvGen, attributes, 0, 0, 0,
-                                     0, VectorValues.SearchStrategy.NONE, false);
-          } 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);
-    }
-  }
-  
-  static {
-    // We "mirror" DocValues enum values with the constants below; let's try to ensure if we add a new DocValuesType while this format is
-    // still used for writing, we remember to fix this encoding:
-    assert DocValuesType.values().length == 6;
-  }
-
-  private static byte docValuesByte(DocValuesType type) {
-    switch(type) {
-    case NONE:
-      return 0;
-    case NUMERIC:
-      return 1;
-    case BINARY:
-      return 2;
-    case SORTED:
-      return 3;
-    case SORTED_SET:
-      return 4;
-    case SORTED_NUMERIC:
-      return 5;
-    default:
-      // BUG
-      throw new AssertionError("unhandled DocValuesType: " + type);
-    }
-  }
-
-  private static DocValuesType getDocValuesType(IndexInput input, byte b) throws IOException {
-    switch(b) {
-    case 0:
-      return DocValuesType.NONE;
-    case 1:
-      return DocValuesType.NUMERIC;
-    case 2:
-      return DocValuesType.BINARY;
-    case 3:
-      return DocValuesType.SORTED;
-    case 4:
-      return DocValuesType.SORTED_SET;
-    case 5:
-      return DocValuesType.SORTED_NUMERIC;
-    default:
-      throw new CorruptIndexException("invalid docvalues byte: " + b, input);
-    }
-  }
-
-  static {
-    // We "mirror" IndexOptions enum values with the constants below; let's try to ensure if we add a new IndexOption while this format is
-    // still used for writing, we remember to fix this encoding:
-    assert IndexOptions.values().length == 5;
-  }
-
-  private static byte indexOptionsByte(IndexOptions indexOptions) {
-    switch (indexOptions) {
-    case NONE:
-      return 0;
-    case DOCS:
-      return 1;
-    case DOCS_AND_FREQS:
-      return 2;
-    case DOCS_AND_FREQS_AND_POSITIONS:
-      return 3;
-    case DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS:
-      return 4;
-    default:
-      // BUG:
-      throw new AssertionError("unhandled IndexOptions: " + indexOptions);
-    }
-  }
-  
-  private static IndexOptions getIndexOptions(IndexInput input, byte b) throws IOException {
-    switch (b) {
-    case 0:
-      return IndexOptions.NONE;
-    case 1:
-      return IndexOptions.DOCS;
-    case 2:
-      return IndexOptions.DOCS_AND_FREQS;
-    case 3:
-      return IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
-    case 4:
-      return IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
-    default:
-      // BUG
-      throw new CorruptIndexException("invalid IndexOptions byte: " + b, input);
-    }
-  }
-
-  @Override
-  public void write(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
-    final String fileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, EXTENSION);
-    try (IndexOutput output = directory.createOutput(fileName, context)) {
-      CodecUtil.writeIndexHeader(output, Lucene50FieldInfosFormat.CODEC_NAME, Lucene50FieldInfosFormat.FORMAT_CURRENT, segmentInfo.getId(), segmentSuffix);
-      output.writeVInt(infos.size());
-      for (FieldInfo fi : infos) {
-        fi.checkConsistency();
-
-        output.writeString(fi.name);
-        output.writeVInt(fi.number);
-
-        byte bits = 0x0;
-        if (fi.hasVectors()) bits |= STORE_TERMVECTOR;
-        if (fi.omitsNorms()) bits |= OMIT_NORMS;
-        if (fi.hasPayloads()) bits |= STORE_PAYLOADS;
-        output.writeByte(bits);
-
-        output.writeByte(indexOptionsByte(fi.getIndexOptions()));
-
-        // pack the DV type and hasNorms in one byte
-        output.writeByte(docValuesByte(fi.getDocValuesType()));
-        output.writeLong(fi.getDocValuesGen());
-        output.writeMapOfStrings(fi.attributes());
-      }
-      CodecUtil.writeFooter(output);
-    }
-  }
-  
-  /** Extension of field infos */
-  static final String EXTENSION = "fnm";
-  
-  // Codec header
-  static final String CODEC_NAME = "Lucene50FieldInfos";
-  static final int FORMAT_SAFE_MAPS = 1;
-  static final int FORMAT_START = FORMAT_SAFE_MAPS;
-  static final int FORMAT_CURRENT = FORMAT_SAFE_MAPS;
-  
-  // Field flags
-  static final byte STORE_TERMVECTOR = 0x1;
-  static final byte OMIT_NORMS = 0x2;
-  static final byte STORE_PAYLOADS = 0x4;
-}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java
index 2343de2..4141932 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java
@@ -29,6 +29,7 @@ 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;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.FixedBitSet;
@@ -46,21 +47,21 @@ import org.apache.lucene.util.FixedBitSet;
  * </ul>
  */
 public final class Lucene50LiveDocsFormat extends LiveDocsFormat {
-  
-  /** Sole constructor. */
-  public Lucene50LiveDocsFormat() {
-  }
-  
+
   /** extension of live docs */
   private static final String EXTENSION = "liv";
-  
+
   /** codec of live docs */
   private static final String CODEC_NAME = "Lucene50LiveDocs";
-  
+
   /** supported version range */
   private static final int VERSION_START = 0;
   private static final int VERSION_CURRENT = VERSION_START;
-
+  
+  /** Sole constructor. */
+  public Lucene50LiveDocsFormat() {
+  }
+  
   @Override
   public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException {
     long gen = info.getDelGen();
@@ -71,11 +72,9 @@ public final class Lucene50LiveDocsFormat extends LiveDocsFormat {
       try {
         CodecUtil.checkIndexHeader(input, CODEC_NAME, VERSION_START, VERSION_CURRENT, 
                                      info.info.getId(), Long.toString(gen, Character.MAX_RADIX));
-        long data[] = new long[FixedBitSet.bits2words(length)];
-        for (int i = 0; i < data.length; i++) {
-          data[i] = input.readLong();
-        }
-        FixedBitSet fbs = new FixedBitSet(data, length);
+        
+        FixedBitSet fbs = readFixedBitSet(input, length);
+        
         if (fbs.length() - fbs.cardinality() != info.getDelCount()) {
           throw new CorruptIndexException("bits.deleted=" + (fbs.length() - fbs.cardinality()) + 
                                           " info.delcount=" + info.getDelCount(), input);
@@ -89,26 +88,26 @@ public final class Lucene50LiveDocsFormat extends LiveDocsFormat {
     }
     throw new AssertionError();
   }
+  
+  private FixedBitSet readFixedBitSet(IndexInput input, int length) throws IOException {
+    long data[] = new long[FixedBitSet.bits2words(length)];
+    for (int i = 0; i < data.length; i++) {
+      data[i] = input.readLong();
+    }
+    return new FixedBitSet(data, length);
+  }
 
   @Override
   public void writeLiveDocs(Bits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
     long gen = info.getNextDelGen();
     String name = IndexFileNames.fileNameFromGeneration(info.info.name, EXTENSION, gen);
-    int delCount = 0;
+    int delCount;
     try (IndexOutput output = dir.createOutput(name, context)) {
+      
       CodecUtil.writeIndexHeader(output, CODEC_NAME, VERSION_CURRENT, info.info.getId(), Long.toString(gen, Character.MAX_RADIX));
-      final int longCount = FixedBitSet.bits2words(bits.length());
-      for (int i = 0; i < longCount; ++i) {
-        long currentBits = 0;
-        for (int j = i << 6, end = Math.min(j + 63, bits.length() - 1); j <= end; ++j) {
-          if (bits.get(j)) {
-            currentBits |= 1L << j; // mod 64
-          } else {
-            delCount += 1;
-          }
-        }
-        output.writeLong(currentBits);
-      }
+      
+      delCount = writeBits(output, bits);
+      
       CodecUtil.writeFooter(output);
     }
     if (delCount != info.getDelCount() + newDelCount) {
@@ -117,6 +116,23 @@ public final class Lucene50LiveDocsFormat extends LiveDocsFormat {
     }
   }
 
+  private int writeBits(IndexOutput output, Bits bits) throws IOException {
+    int delCount = 0;
+    final int longCount = FixedBitSet.bits2words(bits.length());
+    for (int i = 0; i < longCount; ++i) {
+      long currentBits = 0;
+      for (int j = i << 6, end = Math.min(j + 63, bits.length() - 1); j <= end; ++j) {
+        if (bits.get(j)) {
+          currentBits |= 1L << j; // mod 64
+        } else {
+          delCount += 1;
+        }
+      }
+      output.writeLong(currentBits);
+    }
+    return delCount;
+  }
+
   @Override
   public void files(SegmentCommitInfo info, Collection<String> files) throws IOException {
     if (info.hasDeletions()) {
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java
index 972e3c6..c3f450d 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java
@@ -104,6 +104,22 @@ import org.apache.lucene.store.IndexOutput;
  */
 public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
 
+  /** Extension of field infos */
+  static final String EXTENSION = "fnm";
+
+  // Codec header
+  static final String CODEC_NAME = "Lucene60FieldInfos";
+  static final int FORMAT_START = 0;
+  static final int FORMAT_SOFT_DELETES = 1;
+  static final int FORMAT_SELECTIVE_INDEXING = 2;
+  static final int FORMAT_CURRENT = FORMAT_SELECTIVE_INDEXING;
+
+  // Field flags
+  static final byte STORE_TERMVECTOR = 0x1;
+  static final byte OMIT_NORMS = 0x2;
+  static final byte STORE_PAYLOADS = 0x4;
+  static final byte SOFT_DELETES_FIELD = 0x8;
+
   /** Sole constructor. */
   public Lucene60FieldInfosFormat() {
   }
@@ -113,7 +129,7 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
     final String fileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, EXTENSION);
     try (ChecksumIndexInput input = directory.openChecksumInput(fileName, context)) {
       Throwable priorE = null;
-      FieldInfo infos[] = null;
+      FieldInfo[] infos = null;
       try {
         int version = CodecUtil.checkIndexHeader(input,
                                    Lucene60FieldInfosFormat.CODEC_NAME, 
@@ -121,56 +137,8 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
                                    Lucene60FieldInfosFormat.FORMAT_CURRENT,
                                    segmentInfo.getId(), segmentSuffix);
         
-        final int size = input.readVInt(); //read in the size
-        infos = new FieldInfo[size];
-        
-        // previous field's attribute map, we share when possible:
-        Map<String,String> lastAttributes = Collections.emptyMap();
-        
-        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 storeTermVector = (bits & STORE_TERMVECTOR) != 0;
-          boolean omitNorms = (bits & OMIT_NORMS) != 0;
-          boolean storePayloads = (bits & STORE_PAYLOADS) != 0;
-          boolean isSoftDeletesField = (bits & SOFT_DELETES_FIELD) != 0;
-
-          final IndexOptions indexOptions = getIndexOptions(input, input.readByte());
-          
-          // DV Types are packed in one byte
-          final DocValuesType docValuesType = getDocValuesType(input, input.readByte());
-          final long dvGen = input.readLong();
-          Map<String,String> attributes = input.readMapOfStrings();
-          // just use the last field's map if its the same
-          if (attributes.equals(lastAttributes)) {
-            attributes = lastAttributes;
-          }
-          lastAttributes = attributes;
-          int pointDataDimensionCount = input.readVInt();
-          int pointNumBytes;
-          int pointIndexDimensionCount = pointDataDimensionCount;
-          if (pointDataDimensionCount != 0) {
-            if (version >= Lucene60FieldInfosFormat.FORMAT_SELECTIVE_INDEXING) {
-              pointIndexDimensionCount = input.readVInt();
-            }
-            pointNumBytes = input.readVInt();
-          } else {
-            pointNumBytes = 0;
-          }
-
-          try {
-            infos[i] = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads, 
-                                     indexOptions, docValuesType, dvGen, attributes,
-                                     pointDataDimensionCount, pointIndexDimensionCount, pointNumBytes,
-                                     0, VectorValues.SearchStrategy.NONE, isSoftDeletesField);
-          } catch (IllegalStateException e) {
-            throw new CorruptIndexException("invalid fieldinfo for field: " + name + ", fieldNumber=" + fieldNumber, input, e);
-          }
-        }
+        infos = readFieldInfos(input, version);
+       
       } catch (Throwable exception) {
         priorE = exception;
       } finally {
@@ -179,6 +147,60 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
       return new FieldInfos(infos);
     }
   }
+
+  private FieldInfo[] readFieldInfos(IndexInput input, int version) throws IOException {
+    final int size = input.readVInt(); //read in the size
+    FieldInfo[] infos = new FieldInfo[size];
+
+    // previous field's attribute map, we share when possible:
+    Map<String,String> lastAttributes = Collections.emptyMap();
+
+    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 storeTermVector = (bits & STORE_TERMVECTOR) != 0;
+      boolean omitNorms = (bits & OMIT_NORMS) != 0;
+      boolean storePayloads = (bits & STORE_PAYLOADS) != 0;
+      boolean isSoftDeletesField = (bits & SOFT_DELETES_FIELD) != 0;
+
+      final IndexOptions indexOptions = getIndexOptions(input, input.readByte());
+
+      // DV Types are packed in one byte
+      final DocValuesType docValuesType = getDocValuesType(input, input.readByte());
+      final long dvGen = input.readLong();
+      Map<String,String> attributes = input.readMapOfStrings();
+      // just use the last field's map if its the same
+      if (attributes.equals(lastAttributes)) {
+        attributes = lastAttributes;
+      }
+      lastAttributes = attributes;
+      int pointDataDimensionCount = input.readVInt();
+      int pointNumBytes;
+      int pointIndexDimensionCount = pointDataDimensionCount;
+      if (pointDataDimensionCount != 0) {
+        if (version >= Lucene60FieldInfosFormat.FORMAT_SELECTIVE_INDEXING) {
+          pointIndexDimensionCount = input.readVInt();
+        }
+        pointNumBytes = input.readVInt();
+      } else {
+        pointNumBytes = 0;
+      }
+
+      try {
+        infos[i] = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads,
+                indexOptions, docValuesType, dvGen, attributes,
+                pointDataDimensionCount, pointIndexDimensionCount, pointNumBytes,
+                0, VectorValues.SearchStrategy.NONE, isSoftDeletesField);
+      } catch (IllegalStateException e) {
+        throw new CorruptIndexException("invalid fieldinfo for field: " + name + ", fieldNumber=" + fieldNumber, input, e);
+      }
+    }
+    return infos;
+  }
   
   static {
     // We "mirror" DocValues enum values with the constants below; let's try to ensure if we add a new DocValuesType while this format is
@@ -301,20 +323,4 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
       CodecUtil.writeFooter(output);
     }
   }
-  
-  /** Extension of field infos */
-  static final String EXTENSION = "fnm";
-  
-  // Codec header
-  static final String CODEC_NAME = "Lucene60FieldInfos";
-  static final int FORMAT_START = 0;
-  static final int FORMAT_SOFT_DELETES = 1;
-  static final int FORMAT_SELECTIVE_INDEXING = 2;
-  static final int FORMAT_CURRENT = FORMAT_SELECTIVE_INDEXING;
-  
-  // Field flags
-  static final byte STORE_TERMVECTOR = 0x1;
-  static final byte OMIT_NORMS = 0x2;
-  static final byte STORE_PAYLOADS = 0x4;
-  static final byte SOFT_DELETES_FIELD = 0x8;
 }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
index bc71d10..a2994b7 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
@@ -78,7 +78,9 @@ final class Lucene80DocValuesProducer extends DocValuesProducer implements Close
                                         Lucene80DocValuesFormat.VERSION_CURRENT,
                                         state.segmentInfo.getId(),
                                         state.segmentSuffix);
+        
         readFields(in, state.fieldInfos);
+        
       } catch (Throwable exception) {
         priorE = exception;
       } finally {
@@ -113,7 +115,7 @@ final class Lucene80DocValuesProducer extends DocValuesProducer implements Close
     }
   }
 
-  private void readFields(ChecksumIndexInput meta, FieldInfos infos) throws IOException {
+  private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
     for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
       FieldInfo info = infos.fieldInfo(fieldNumber);
       if (info == null) {
@@ -136,13 +138,13 @@ final class Lucene80DocValuesProducer extends DocValuesProducer implements Close
     }
   }
 
-  private NumericEntry readNumeric(ChecksumIndexInput meta) throws IOException {
+  private NumericEntry readNumeric(IndexInput meta) throws IOException {
     NumericEntry entry = new NumericEntry();
     readNumeric(meta, entry);
     return entry;
   }
 
-  private void readNumeric(ChecksumIndexInput meta, NumericEntry entry) throws IOException {
+  private void readNumeric(IndexInput meta, NumericEntry entry) throws IOException {
     entry.docsWithFieldOffset = meta.readLong();
     entry.docsWithFieldLength = meta.readLong();
     entry.jumpTableEntryCount = meta.readShort();
@@ -172,7 +174,7 @@ final class Lucene80DocValuesProducer extends DocValuesProducer implements Close
     entry.valueJumpTableOffset = meta.readLong();
   }
 
-  private BinaryEntry readBinary(ChecksumIndexInput meta) throws IOException {
+  private BinaryEntry readBinary(IndexInput meta) throws IOException {
     BinaryEntry entry = new BinaryEntry();
     if (version >= Lucene80DocValuesFormat.VERSION_CONFIGURABLE_COMPRESSION) {
       int b = meta.readByte();
@@ -218,7 +220,7 @@ final class Lucene80DocValuesProducer extends DocValuesProducer implements Close
     return entry;
   }
 
-  private SortedEntry readSorted(ChecksumIndexInput meta) throws IOException {
+  private SortedEntry readSorted(IndexInput meta) throws IOException {
     SortedEntry entry = new SortedEntry();
     entry.docsWithFieldOffset = meta.readLong();
     entry.docsWithFieldLength = meta.readLong();
@@ -232,7 +234,7 @@ final class Lucene80DocValuesProducer extends DocValuesProducer implements Close
     return entry;
   }
 
-  private SortedSetEntry readSortedSet(ChecksumIndexInput meta) throws IOException {
+  private SortedSetEntry readSortedSet(IndexInput meta) throws IOException {
     SortedSetEntry entry = new SortedSetEntry();
     byte multiValued = meta.readByte();
     switch (multiValued) {
@@ -261,7 +263,7 @@ final class Lucene80DocValuesProducer extends DocValuesProducer implements Close
     return entry;
   }
 
-  private static void readTermDict(ChecksumIndexInput meta, TermsDictEntry entry) throws IOException {
+  private static void readTermDict(IndexInput meta, TermsDictEntry entry) throws IOException {
     entry.termsDictSize = meta.readVLong();
     entry.termsDictBlockShift = meta.readInt();
     final int blockShift = meta.readInt();
@@ -281,7 +283,7 @@ final class Lucene80DocValuesProducer extends DocValuesProducer implements Close
     entry.termsIndexAddressesLength = meta.readLong();
   }
 
-  private SortedNumericEntry readSortedNumeric(ChecksumIndexInput meta) throws IOException {
+  private SortedNumericEntry readSortedNumeric(IndexInput meta) throws IOException {
     SortedNumericEntry entry = new SortedNumericEntry();
     readNumeric(meta, entry);
     entry.numDocsWithField = meta.readInt();
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene86/Lucene86SegmentInfoFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene86/Lucene86SegmentInfoFormat.java
index b2bcdc2..0691c60 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene86/Lucene86SegmentInfoFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene86/Lucene86SegmentInfoFormat.java
@@ -81,6 +81,12 @@ import org.apache.lucene.util.Version;
  */
 public class Lucene86SegmentInfoFormat extends SegmentInfoFormat {
 
+  /** File extension used to store {@link SegmentInfo}. */
+  public final static String SI_EXTENSION = "si";
+  static final String CODEC_NAME = "Lucene86SegmentInfo";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+
   /** Sole constructor. */
   public Lucene86SegmentInfoFormat() {
   }
@@ -96,47 +102,9 @@ public class Lucene86SegmentInfoFormat extends SegmentInfoFormat {
             VERSION_START,
             VERSION_CURRENT,
             segmentID, "");
-        final Version version = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
-        byte hasMinVersion = input.readByte();
-        final Version minVersion;
-        switch (hasMinVersion) {
-          case 0:
-            minVersion = null;
-            break;
-          case 1:
-            minVersion = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
-            break;
-          default:
-            throw new CorruptIndexException("Illegal boolean value " + hasMinVersion, input);
-        }
-
-        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.readMapOfStrings();
-        final Set<String> files = input.readSetOfStrings();
-        final Map<String,String> attributes = input.readMapOfStrings();
-
-        int numSortFields = input.readVInt();
-        Sort indexSort;
-        if (numSortFields > 0) {
-          SortField[] sortFields = new SortField[numSortFields];
-          for(int i=0;i<numSortFields;i++) {
-            String name = input.readString();
-            sortFields[i] = SortFieldProvider.forName(name).readSortField(input);
-          }
-          indexSort = new Sort(sortFields);
-        } else if (numSortFields < 0) {
-          throw new CorruptIndexException("invalid index sort field count: " + numSortFields, input);
-        } else {
-          indexSort = null;
-        }
-
-        si = new SegmentInfo(dir, version, minVersion, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, indexSort);
-        si.setFiles(files);
+        
+        si = parseSegmentInfo(dir, input, segment, segmentID);
+        
       } catch (Throwable exception) {
         priorE = exception;
       } finally {
@@ -146,6 +114,51 @@ public class Lucene86SegmentInfoFormat extends SegmentInfoFormat {
     }
   }
 
+  private SegmentInfo parseSegmentInfo(Directory dir, DataInput input, String segment, byte[] segmentID) throws IOException {
+    final Version version = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
+    byte hasMinVersion = input.readByte();
+    final Version minVersion;
+    switch (hasMinVersion) {
+      case 0:
+        minVersion = null;
+        break;
+      case 1:
+        minVersion = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
+        break;
+      default:
+        throw new CorruptIndexException("Illegal boolean value " + hasMinVersion, input);
+    }
+
+    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.readMapOfStrings();
+    final Set<String> files = input.readSetOfStrings();
+    final Map<String,String> attributes = input.readMapOfStrings();
+
+    int numSortFields = input.readVInt();
+    Sort indexSort;
+    if (numSortFields > 0) {
+      SortField[] sortFields = new SortField[numSortFields];
+      for(int i=0;i<numSortFields;i++) {
+        String name = input.readString();
+        sortFields[i] = SortFieldProvider.forName(name).readSortField(input);
+      }
+      indexSort = new Sort(sortFields);
+    } else if (numSortFields < 0) {
+      throw new CorruptIndexException("invalid index sort field count: " + numSortFields, input);
+    } else {
+      indexSort = null;
+    }
+
+    SegmentInfo si = new SegmentInfo(dir, version, minVersion, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, indexSort);
+    si.setFiles(files);
+    return si;
+  }
+  
   @Override
   public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
     final String fileName = IndexFileNames.segmentFileName(si.name, "", SI_EXTENSION);
@@ -153,65 +166,60 @@ public class Lucene86SegmentInfoFormat extends SegmentInfoFormat {
     try (IndexOutput output = dir.createOutput(fileName, ioContext)) {
       // Only add the file once we've successfully created it, else IFD assert can trip:
       si.addFile(fileName);
-      CodecUtil.writeIndexHeader(output,
-          CODEC_NAME,
-          VERSION_CURRENT,
-          si.getId(),
-          "");
-      Version version = si.getVersion();
-      if (version.major < 7) {
-        throw new IllegalArgumentException("invalid major version: should be >= 7 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);
-
-      // Write the min Lucene version that contributed docs to the segment, since 7.0
-      if (si.getMinVersion() != null) {
-        output.writeByte((byte) 1);
-        Version minVersion = si.getMinVersion();
-        output.writeInt(minVersion.major);
-        output.writeInt(minVersion.minor);
-        output.writeInt(minVersion.bugfix);
-      } else {
-        output.writeByte((byte) 0);
-      }
-
-      assert version.prerelease == 0;
-      output.writeInt(si.maxDoc());
+      CodecUtil.writeIndexHeader(output, CODEC_NAME, VERSION_CURRENT, si.getId(), "");
 
-      output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
-      output.writeMapOfStrings(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.writeSetOfStrings(files);
-      output.writeMapOfStrings(si.getAttributes());
-
-      Sort indexSort = si.getIndexSort();
-      int numSortFields = indexSort == null ? 0 : indexSort.getSort().length;
-      output.writeVInt(numSortFields);
-      for (int i = 0; i < numSortFields; ++i) {
-        SortField sortField = indexSort.getSort()[i];
-        IndexSorter sorter = sortField.getIndexSorter();
-        if (sorter == null) {
-          throw new IllegalArgumentException("cannot serialize SortField " + sortField);
-        }
-        output.writeString(sorter.getProviderName());
-        SortFieldProvider.write(sortField, output);
-      }
+      writeSegmentInfo(output, si);
 
       CodecUtil.writeFooter(output);
     }
   }
 
-  /** File extension used to store {@link SegmentInfo}. */
-  public final static String SI_EXTENSION = "si";
-  static final String CODEC_NAME = "Lucene86SegmentInfo";
-  static final int VERSION_START = 0;
-  static final int VERSION_CURRENT = VERSION_START;
+  private void writeSegmentInfo(DataOutput output, SegmentInfo si) throws IOException {
+    Version version = si.getVersion();
+    if (version.major < 7) {
+      throw new IllegalArgumentException("invalid major version: should be >= 7 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);
+
+    // Write the min Lucene version that contributed docs to the segment, since 7.0
+    if (si.getMinVersion() != null) {
+      output.writeByte((byte) 1);
+      Version minVersion = si.getMinVersion();
+      output.writeInt(minVersion.major);
+      output.writeInt(minVersion.minor);
+      output.writeInt(minVersion.bugfix);
+    } else {
+      output.writeByte((byte) 0);
+    }
+
+    assert version.prerelease == 0;
+    output.writeInt(si.maxDoc());
+
+    output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
+    output.writeMapOfStrings(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.writeSetOfStrings(files);
+    output.writeMapOfStrings(si.getAttributes());
+
+    Sort indexSort = si.getIndexSort();
+    int numSortFields = indexSort == null ? 0 : indexSort.getSort().length;
+    output.writeVInt(numSortFields);
+    for (int i = 0; i < numSortFields; ++i) {
+      SortField sortField = indexSort.getSort()[i];
+      IndexSorter sorter = sortField.getIndexSorter();
+      if (sorter == null) {
+        throw new IllegalArgumentException("cannot serialize SortField " + sortField);
+      }
+      output.writeString(sorter.getProviderName());
+      SortFieldProvider.write(sortField, output);
+    }
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
index b561884..e26afba 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
@@ -336,114 +336,117 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
       infos.generation = generation;
       infos.lastGeneration = generation;
       infos.luceneVersion = luceneVersion;
-
-      infos.version = input.readLong();
-      //System.out.println("READ sis version=" + infos.version);
-      if (format > VERSION_70) {
-        infos.counter = input.readVLong();
-      } else {
-        infos.counter = input.readInt();
-      }
-      int numSegments = input.readInt();
-      if (numSegments < 0) {
-        throw new CorruptIndexException("invalid segment count: " + numSegments, input);
-      }
-
-      if (numSegments > 0) {
-        infos.minSegmentLuceneVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt());
+      parseSegmentInfos(directory, input, infos, format);
+      return infos;
+      
+    } catch (Throwable t) {
+      priorE = t;
+    } finally {
+      if (format >= VERSION_70) { // oldest supported version
+        CodecUtil.checkFooter(input, priorE);
       } else {
-        // else leave as null: no segments
+        throw IOUtils.rethrowAlways(priorE);
       }
+    }
+    throw new Error("Unreachable code");
+  }
 
-      long totalDocs = 0;
-      for (int seg = 0; seg < numSegments; seg++) {
-        String segName = input.readString();
-        byte[] segmentID = new byte[StringHelper.ID_LENGTH];
-        input.readBytes(segmentID, 0, segmentID.length);
-        Codec codec = readCodec(input);
-        SegmentInfo info = codec.segmentInfoFormat().read(directory, segName, segmentID, IOContext.READ);
-        info.setCodec(codec);
-        totalDocs += info.maxDoc();
-        long delGen = input.readLong();
-        int delCount = input.readInt();
-        if (delCount < 0 || delCount > info.maxDoc()) {
-          throw new CorruptIndexException("invalid deletion count: " + delCount + " vs maxDoc=" + info.maxDoc(), input);
-        }
-        long fieldInfosGen = input.readLong();
-        long dvGen = input.readLong();
-        int softDelCount = format > VERSION_72 ? input.readInt() : 0;
-        if (softDelCount < 0 || softDelCount > info.maxDoc()) {
-          throw new CorruptIndexException("invalid deletion count: " + softDelCount + " vs maxDoc=" + info.maxDoc(), input);
-        }
-        if (softDelCount + delCount > info.maxDoc()) {
-          throw new CorruptIndexException("invalid deletion count: " + (softDelCount + delCount) + " vs maxDoc=" + info.maxDoc(), input);
-        }
-        final byte[] sciId;
-        if (format > VERSION_74) {
-          byte marker = input.readByte();
-          switch (marker) {
-            case 1:
-              sciId = new byte[StringHelper.ID_LENGTH];
-              input.readBytes(sciId, 0, sciId.length);
-              break;
-            case 0:
-              sciId = null;
-              break;
-            default:
-              throw new CorruptIndexException("invalid SegmentCommitInfo ID marker: " + marker, input);
-          }
-        } else {
-          sciId = null;
-        }
-        SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, softDelCount, delGen, fieldInfosGen, dvGen, sciId);
-        siPerCommit.setFieldInfosFiles(input.readSetOfStrings());
-        final Map<Integer,Set<String>> dvUpdateFiles;
-        final int numDVFields = input.readInt();
-        if (numDVFields == 0) {
-          dvUpdateFiles = Collections.emptyMap();
-        } else {
-          Map<Integer,Set<String>> map = new HashMap<>(numDVFields);
-          for (int i = 0; i < numDVFields; i++) {
-            map.put(input.readInt(), input.readSetOfStrings());
-          }
-          dvUpdateFiles = Collections.unmodifiableMap(map);
-        }
-        siPerCommit.setDocValuesUpdatesFiles(dvUpdateFiles);
-        infos.add(siPerCommit);
-
-        Version segmentVersion = info.getVersion();
+  private static void parseSegmentInfos(Directory directory, DataInput input, SegmentInfos infos, int format) throws IOException {
+    infos.version = input.readLong();
+    //System.out.println("READ sis version=" + infos.version);
+    if (format > VERSION_70) {
+      infos.counter = input.readVLong();
+    } else {
+      infos.counter = input.readInt();
+    }
+    int numSegments = input.readInt();
+    if (numSegments < 0) {
+      throw new CorruptIndexException("invalid segment count: " + numSegments, input);
+    }
 
-        if (segmentVersion.onOrAfter(infos.minSegmentLuceneVersion) == false) {
-          throw new CorruptIndexException("segments file recorded minSegmentLuceneVersion=" + infos.minSegmentLuceneVersion + " but segment=" + info + " has older version=" + segmentVersion, input);
-        }
+    if (numSegments > 0) {
+      infos.minSegmentLuceneVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt());
+    } else {
+      // else leave as null: no segments
+    }
 
-        if (infos.indexCreatedVersionMajor >= 7 && segmentVersion.major < infos.indexCreatedVersionMajor) {
-          throw new CorruptIndexException("segments file recorded indexCreatedVersionMajor=" + infos.indexCreatedVersionMajor + " but segment=" + info + " has older version=" + segmentVersion, input);
+    long totalDocs = 0;
+    for (int seg = 0; seg < numSegments; seg++) {
+      String segName = input.readString();
+      byte[] segmentID = new byte[StringHelper.ID_LENGTH];
+      input.readBytes(segmentID, 0, segmentID.length);
+      Codec codec = readCodec(input);
+      SegmentInfo info = codec.segmentInfoFormat().read(directory, segName, segmentID, IOContext.READ);
+      info.setCodec(codec);
+      totalDocs += info.maxDoc();
+      long delGen = input.readLong();
+      int delCount = input.readInt();
+      if (delCount < 0 || delCount > info.maxDoc()) {
+        throw new CorruptIndexException("invalid deletion count: " + delCount + " vs maxDoc=" + info.maxDoc(), input);
+      }
+      long fieldInfosGen = input.readLong();
+      long dvGen = input.readLong();
+      int softDelCount = format > VERSION_72 ? input.readInt() : 0;
+      if (softDelCount < 0 || softDelCount > info.maxDoc()) {
+        throw new CorruptIndexException("invalid deletion count: " + softDelCount + " vs maxDoc=" + info.maxDoc(), input);
+      }
+      if (softDelCount + delCount > info.maxDoc()) {
+        throw new CorruptIndexException("invalid deletion count: " + (softDelCount + delCount) + " vs maxDoc=" + info.maxDoc(), input);
+      }
+      final byte[] sciId;
+      if (format > VERSION_74) {
+        byte marker = input.readByte();
+        switch (marker) {
+          case 1:
+            sciId = new byte[StringHelper.ID_LENGTH];
+            input.readBytes(sciId, 0, sciId.length);
+            break;
+          case 0:
+            sciId = null;
+            break;
+          default:
+            throw new CorruptIndexException("invalid SegmentCommitInfo ID marker: " + marker, input);
         }
-
-        if (infos.indexCreatedVersionMajor >= 7 && info.getMinVersion() == null) {
-          throw new CorruptIndexException("segments infos must record minVersion with indexCreatedVersionMajor=" + infos.indexCreatedVersionMajor, input);
+      } else {
+        sciId = null;
+      }
+      SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, softDelCount, delGen, fieldInfosGen, dvGen, sciId);
+      siPerCommit.setFieldInfosFiles(input.readSetOfStrings());
+      final Map<Integer,Set<String>> dvUpdateFiles;
+      final int numDVFields = input.readInt();
+      if (numDVFields == 0) {
+        dvUpdateFiles = Collections.emptyMap();
+      } else {
+        Map<Integer,Set<String>> map = new HashMap<>(numDVFields);
+        for (int i = 0; i < numDVFields; i++) {
+          map.put(input.readInt(), input.readSetOfStrings());
         }
+        dvUpdateFiles = Collections.unmodifiableMap(map);
       }
+      siPerCommit.setDocValuesUpdatesFiles(dvUpdateFiles);
+      infos.add(siPerCommit);
 
-      infos.userData = input.readMapOfStrings();
+      Version segmentVersion = info.getVersion();
 
-      // LUCENE-6299: check we are in bounds
-      if (totalDocs > IndexWriter.getActualMaxDocs()) {
-        throw new CorruptIndexException("Too many documents: an index cannot exceed " + IndexWriter.getActualMaxDocs() + " but readers have total maxDoc=" + totalDocs, input);
+      if (segmentVersion.onOrAfter(infos.minSegmentLuceneVersion) == false) {
+        throw new CorruptIndexException("segments file recorded minSegmentLuceneVersion=" + infos.minSegmentLuceneVersion + " but segment=" + info + " has older version=" + segmentVersion, input);
       }
 
-      return infos;
-    } catch (Throwable t) {
-      priorE = t;
-    } finally {
-      if (format >= VERSION_70) { // oldest supported version
-        CodecUtil.checkFooter(input, priorE);
-      } else {
-        throw IOUtils.rethrowAlways(priorE);
+      if (infos.indexCreatedVersionMajor >= 7 && segmentVersion.major < infos.indexCreatedVersionMajor) {
+        throw new CorruptIndexException("segments file recorded indexCreatedVersionMajor=" + infos.indexCreatedVersionMajor + " but segment=" + info + " has older version=" + segmentVersion, input);
+      }
+
+      if (infos.indexCreatedVersionMajor >= 7 && info.getMinVersion() == null) {
+        throw new CorruptIndexException("segments infos must record minVersion with indexCreatedVersionMajor=" + infos.indexCreatedVersionMajor, input);
       }
     }
-    throw new Error("Unreachable code");
+
+    infos.userData = input.readMapOfStrings();
+
+    // LUCENE-6299: check we are in bounds
+    if (totalDocs > IndexWriter.getActualMaxDocs()) {
+      throw new CorruptIndexException("Too many documents: an index cannot exceed " + IndexWriter.getActualMaxDocs() + " but readers have total maxDoc=" + totalDocs, input);
+    }
   }
 
   private static Codec readCodec(DataInput input) throws IOException {