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/09/24 02:59:51 UTC

svn commit: r1627187 - in /lucene/dev/trunk: lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ lucene/core/src/java/org/apache/lucene/codecs/ lucene/core/src/java/org/apache/lucene/codecs/lucene410/ lucene/core/src/java/org/apache/luc...

Author: rmuir
Date: Wed Sep 24 00:59:50 2014
New Revision: 1627187

URL: http://svn.apache.org/r1627187
Log:
LUCENE-5969: Add Lucene50Codec (infos, dv, norms)

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosReader.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosWriter.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoReader.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoWriter.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCodecUtil.java   (with props)
Modified:
    lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsProducer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
    lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java

Modified: lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java?rev=1627187&r1=1627186&r2=1627187&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java (original)
+++ lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java Wed Sep 24 00:59:50 2014
@@ -30,7 +30,7 @@ import org.apache.lucene.benchmark.byTas
 import org.apache.lucene.benchmark.byTask.utils.Config;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene410.Lucene410Codec;
+import org.apache.lucene.codecs.lucene50.Lucene50Codec;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.index.IndexDeletionPolicy;
@@ -139,7 +139,7 @@ public class CreateIndexTask extends Per
     if (defaultCodec == null && postingsFormat != null) {
       try {
         final PostingsFormat postingsFormatChosen = PostingsFormat.forName(postingsFormat);
-        iwConf.setCodec(new Lucene410Codec(){
+        iwConf.setCodec(new Lucene50Codec(){
           @Override
           public PostingsFormat getPostingsFormatForField(String field) {
             return postingsFormatChosen;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/Codec.java?rev=1627187&r1=1627186&r2=1627187&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/Codec.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/Codec.java Wed Sep 24 00:59:50 2014
@@ -119,7 +119,7 @@ public abstract class Codec implements N
     loader.reload(classloader);
   }
   
-  private static Codec defaultCodec = Codec.forName("Lucene410");
+  private static Codec defaultCodec = Codec.forName("Lucene50");
   
   /** expert: returns the default codec used for newly created
    *  {@link IndexWriterConfig}s.

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java?rev=1627187&r1=1627186&r2=1627187&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java Wed Sep 24 00:59:50 2014
@@ -30,6 +30,7 @@ import org.apache.lucene.store.DataOutpu
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
 
 /**
  * Utility class for reading and writing versioned headers.
@@ -77,6 +78,7 @@ public final class CodecUtil {
    *              less than 128 characters in length.
    * @param version Version number
    * @throws IOException If there is an I/O error writing to the underlying medium.
+   * @throws IllegalArgumentException If the codec name is not simple ASCII, or is more than 127 characters in length
    */
   public static void writeHeader(DataOutput out, String codec, int version)
     throws IOException {
@@ -211,13 +213,63 @@ public final class CodecUtil {
       throw new CorruptIndexException("checksum failed (hardware problem?) : expected=" + Long.toHexString(expectedChecksum) +  
                                                        " actual=" + Long.toHexString(actualChecksum), in);
     }
-    if (in.getFilePointer() != in.length()) {
-      throw new CorruptIndexException("did not read all bytes from file: read " + in.getFilePointer() + " vs size " + in.length(), in);
-    }
     return actualChecksum;
   }
   
   /** 
+   * Validates the codec footer previously written by {@link #writeFooter}, optionally
+   * passing an unexpected exception that has already occurred.
+   * <p>
+   * When a {@code priorException} is provided, this method will add a suppressed exception 
+   * indicating whether the checksum for the stream passes, fails, or cannot be computed, and 
+   * rethrow it. Otherwise it behaves the same as {@link #checkFooter(ChecksumIndexInput)}.
+   * <p>
+   * Example usage:
+   * <pre class="prettyprint">
+   * try (ChecksumIndexInput input = ...) {
+   *   Throwable priorE = null;
+   *   try {
+   *     // ... read a bunch of stuff ... 
+   *   } catch (Throwable exception) {
+   *     priorE = exception;
+   *   } finally {
+   *     CodecUtil.checkFooter(input, priorE);
+   *   }
+   * }
+   * </pre>
+   */
+  public static void checkFooter(ChecksumIndexInput in, Throwable priorException) throws IOException {
+    if (priorException == null) {
+      checkFooter(in);
+    } else {
+      try {
+        long remaining = in.length() - in.getFilePointer();
+        if (remaining < footerLength()) {
+          // corruption caused us to read into the checksum footer already: we can't proceed
+          priorException.addSuppressed(new CorruptIndexException("checksum status indeterminate: remaining=" + remaining +
+                                                                 ", please run checkindex for more details", in));
+        } else {
+          // otherwise, skip any unread bytes.
+          in.skipBytes(remaining - footerLength());
+          
+          // now check the footer
+          try {
+            long checksum = checkFooter(in);
+            priorException.addSuppressed(new CorruptIndexException("checksum passed (" + Long.toHexString(checksum) + 
+                                                                   "). possibly transient resource issue, or a Lucene or JVM bug", in));
+          } catch (CorruptIndexException t) {
+            priorException.addSuppressed(t);
+          }
+        }
+      } catch (Throwable t) {
+        // catch-all for things that shouldn't go wrong (e.g. OOM during readInt) but could...
+        priorException.addSuppressed(new CorruptIndexException("checksum status indeterminate: unexpected exception", in, t));
+      }
+      IOUtils.reThrow(priorException);
+    }
+  }
+  
+  /** 
    * Returns (but does not validate) the checksum previously written by {@link #checkFooter}.
    * @return actual checksum value
    * @throws IOException if the footer is invalid
@@ -229,6 +281,14 @@ public final class CodecUtil {
   }
   
   private static void validateFooter(IndexInput in) throws IOException {
+    long remaining = in.length() - in.getFilePointer();
+    long expected = footerLength();
+    if (remaining < expected) {
+      throw new CorruptIndexException("misplaced codec footer (file truncated?): remaining=" + remaining + ", expected=" + expected, in);
+    } else if (remaining > expected) {
+      throw new CorruptIndexException("misplaced codec footer (file extended?): remaining=" + remaining + ", expected=" + expected, in);
+    }
+    
     final int magic = in.readInt();
     if (magic != FOOTER_MAGIC) {
       throw new CorruptIndexException("codec footer mismatch: actual footer=" + magic + " vs expected footer=" + FOOTER_MAGIC, in);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java?rev=1627187&r1=1627186&r2=1627187&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java Wed Sep 24 00:59:50 2014
@@ -76,17 +76,16 @@ import org.apache.lucene.util.packed.Mon
 
 /** reader for {@link Lucene410DocValuesFormat} */
 class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable {
-  private final Map<String,NumericEntry> numerics;
-  private final Map<String,BinaryEntry> binaries;
-  private final Map<String,SortedSetEntry> sortedSets;
-  private final Map<String,SortedSetEntry> sortedNumerics;
-  private final Map<String,NumericEntry> ords;
-  private final Map<String,NumericEntry> ordIndexes;
+  private final Map<String,NumericEntry> numerics = new HashMap<>();
+  private final Map<String,BinaryEntry> binaries = new HashMap<>();
+  private final Map<String,SortedSetEntry> sortedSets = new HashMap<>();
+  private final Map<String,SortedSetEntry> sortedNumerics = new HashMap<>();
+  private final Map<String,NumericEntry> ords = new HashMap<>();
+  private final Map<String,NumericEntry> ordIndexes = new HashMap<>();
   private final int numFields;
   private final AtomicLong ramBytesUsed;
   private final IndexInput data;
   private final int maxDoc;
-  private final int version;
 
   // memory-resident structures
   private final Map<String,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
@@ -98,22 +97,25 @@ class Lucene410DocValuesProducer extends
     String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
     this.maxDoc = state.segmentInfo.getDocCount();
     
+    int version = -1;
+    int numFields = -1;
+    
     // read in the entries from the metadata file.
     try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
-      version = CodecUtil.checkHeader(in, metaCodec, 
-                                      Lucene410DocValuesFormat.VERSION_START,
-                                      Lucene410DocValuesFormat.VERSION_CURRENT);
-      numerics = new HashMap<>();
-      ords = new HashMap<>();
-      ordIndexes = new HashMap<>();
-      binaries = new HashMap<>();
-      sortedSets = new HashMap<>();
-      sortedNumerics = new HashMap<>();
-      numFields = readFields(in, state.fieldInfos);
-
-      CodecUtil.checkFooter(in);
+      Throwable priorE = null;
+      try {
+        version = CodecUtil.checkHeader(in, metaCodec, 
+                                        Lucene410DocValuesFormat.VERSION_START,
+                                        Lucene410DocValuesFormat.VERSION_CURRENT);
+        numFields = readFields(in, state.fieldInfos);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(in, priorE);
+      }
     }
-
+    
+    this.numFields = numFields;
     String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
     this.data = state.directory.openInput(dataName, state.context);
     boolean success = false;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsProducer.java?rev=1627187&r1=1627186&r2=1627187&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsProducer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsProducer.java Wed Sep 24 00:59:50 2014
@@ -54,7 +54,6 @@ class Lucene49NormsProducer extends Norm
   // metadata maps (just file pointers and minimal stuff)
   private final Map<String,NormsEntry> norms = new HashMap<>();
   private final IndexInput data;
-  private final int version;
   
   // ram instances we have already loaded
   final Map<String,NumericDocValues> instances = new HashMap<>();
@@ -68,12 +67,19 @@ class Lucene49NormsProducer extends Norm
     maxDoc = state.segmentInfo.getDocCount();
     String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
     ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
+    int version = -1;
     
     // read in the entries from the metadata file.
     try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
-      version = CodecUtil.checkHeader(in, metaCodec, VERSION_START, VERSION_CURRENT);
-      readFields(in, state.fieldInfos);
-      CodecUtil.checkFooter(in);
+      Throwable priorE = null;
+      try {
+        version = CodecUtil.checkHeader(in, metaCodec, VERSION_START, VERSION_CURRENT);
+        readFields(in, state.fieldInfos);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(in, priorE);
+      }
     }
 
     String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java?rev=1627187&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java Wed Sep 24 00:59:50 2014
@@ -0,0 +1,135 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.FilterCodec;
+import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
+import org.apache.lucene.codecs.lucene49.Lucene49NormsFormat;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+
+/**
+ * Implements the Lucene 5.0 index format, with configurable per-field postings
+ * and docvalues formats.
+ * <p>
+ * If you want to reuse functionality of this codec in another codec, extend
+ * {@link FilterCodec}.
+ *
+ * @see org.apache.lucene.codecs.lucene50 package documentation for file format details.
+ * @lucene.experimental
+ */
+public class Lucene50Codec extends Codec {
+  private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
+  private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
+  private final FieldInfosFormat fieldInfosFormat = new Lucene50FieldInfosFormat();
+  private final SegmentInfoFormat segmentInfosFormat = new Lucene50SegmentInfoFormat();
+  private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
+  
+  private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
+    @Override
+    public PostingsFormat getPostingsFormatForField(String field) {
+      return Lucene50Codec.this.getPostingsFormatForField(field);
+    }
+  };
+  
+  private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
+    @Override
+    public DocValuesFormat getDocValuesFormatForField(String field) {
+      return Lucene50Codec.this.getDocValuesFormatForField(field);
+    }
+  };
+
+  /** Sole constructor. */
+  public Lucene50Codec() {
+    super("Lucene50");
+  }
+  
+  @Override
+  public final StoredFieldsFormat storedFieldsFormat() {
+    return fieldsFormat;
+  }
+  
+  @Override
+  public final TermVectorsFormat termVectorsFormat() {
+    return vectorsFormat;
+  }
+
+  @Override
+  public final PostingsFormat postingsFormat() {
+    return postingsFormat;
+  }
+  
+  @Override
+  public final FieldInfosFormat fieldInfosFormat() {
+    return fieldInfosFormat;
+  }
+  
+  @Override
+  public final SegmentInfoFormat segmentInfoFormat() {
+    return segmentInfosFormat;
+  }
+  
+  @Override
+  public final LiveDocsFormat liveDocsFormat() {
+    return liveDocsFormat;
+  }
+
+  /** Returns the postings format that should be used for writing 
+   *  new segments of <code>field</code>.
+   *  
+   *  The default implementation always returns "Lucene41"
+   */
+  public PostingsFormat getPostingsFormatForField(String field) {
+    return defaultFormat;
+  }
+  
+  /** Returns the docvalues format that should be used for writing 
+   *  new segments of <code>field</code>.
+   *  
+   *  The default implementation always returns "Lucene410"
+   */
+  public DocValuesFormat getDocValuesFormatForField(String field) {
+    return defaultDVFormat;
+  }
+  
+  @Override
+  public final DocValuesFormat docValuesFormat() {
+    return docValuesFormat;
+  }
+
+  private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
+  private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene410");
+
+  private final NormsFormat normsFormat = new Lucene49NormsFormat();
+
+  @Override
+  public final NormsFormat normsFormat() {
+    return normsFormat;
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java?rev=1627187&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java Wed Sep 24 00:59:50 2014
@@ -0,0 +1,127 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.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.FieldInfo.DocValuesType;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * Lucene 5.0 Field Infos format.
+ * <p>
+ * <p>Field names are stored in the field info file, with suffix <tt>.fnm</tt>.</p>
+ * <p>FieldInfos (.fnm) --&gt; Header,FieldsCount, &lt;FieldName,FieldNumber,
+ * FieldBits,DocValuesBits,DocValuesGen,Attributes&gt; <sup>FieldsCount</sup>,Footer</p>
+ * <p>Data types:
+ * <ul>
+ *   <li>Header --&gt; {@link CodecUtil#checkHeader CodecHeader}</li>
+ *   <li>FieldsCount --&gt; {@link DataOutput#writeVInt VInt}</li>
+ *   <li>FieldName --&gt; {@link DataOutput#writeString String}</li>
+ *   <li>FieldBits, DocValuesBits --&gt; {@link DataOutput#writeByte Byte}</li>
+ *   <li>FieldNumber --&gt; {@link DataOutput#writeInt VInt}</li>
+ *   <li>Attributes --&gt; {@link DataOutput#writeStringStringMap Map&lt;String,String&gt;}</li>
+ *   <li>DocValuesGen --&gt; {@link DataOutput#writeLong(long) Int64}</li>
+ *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
+ * </ul>
+ * </p>
+ * 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 is one for indexed fields, and zero for non-indexed
+ *             fields.</li>
+ *         <li>The second lowest-order bit is one for fields that have term vectors
+ *             stored, and zero for fields without term vectors.</li>
+ *         <li>If the third lowest order-bit is set (0x4), offsets are stored into
+ *             the postings list in addition to positions.</li>
+ *         <li>Fourth bit is unused.</li>
+ *         <li>If the fifth lowest-order bit is set (0x10), norms are omitted for the
+ *             indexed field.</li>
+ *         <li>If the sixth lowest-order bit is set (0x20), payloads are stored for the
+ *             indexed field.</li>
+ *         <li>If the seventh lowest-order bit is set (0x40), term frequencies and
+ *             positions omitted for the indexed field.</li>
+ *         <li>If the eighth lowest-order bit is set (0x80), positions are omitted for the
+ *             indexed field.</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 {
+  private final FieldInfosReader reader = new Lucene50FieldInfosReader();
+  private final FieldInfosWriter writer = new Lucene50FieldInfosWriter();
+  
+  /** Sole constructor. */
+  public Lucene50FieldInfosFormat() {
+  }
+
+  @Override
+  public FieldInfosReader getFieldInfosReader() throws IOException {
+    return reader;
+  }
+
+  @Override
+  public FieldInfosWriter getFieldInfosWriter() throws IOException {
+    return writer;
+  }
+  
+  /** Extension of field infos */
+  static final String EXTENSION = "fnm";
+  
+  // Codec header
+  static final String CODEC_NAME = "Lucene50FieldInfos";
+  static final int FORMAT_START = 0;
+  static final int FORMAT_CURRENT = FORMAT_START;
+  
+  // Field flags
+  static final byte IS_INDEXED = 0x1;
+  static final byte STORE_TERMVECTOR = 0x2;
+  static final byte STORE_OFFSETS_IN_POSTINGS = 0x4;
+  static final byte OMIT_NORMS = 0x10;
+  static final byte STORE_PAYLOADS = 0x20;
+  static final byte OMIT_TERM_FREQ_AND_POSITIONS = 0x40;
+  static final byte OMIT_POSITIONS = -128;
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosReader.java?rev=1627187&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosReader.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosReader.java Wed Sep 24 00:59:50 2014
@@ -0,0 +1,127 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.FieldInfosReader;
+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.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;
+
+/**
+ * Lucene 5.0 FieldInfos reader.
+ * 
+ * @lucene.experimental
+ * @see Lucene50FieldInfosFormat
+ */
+final class Lucene50FieldInfosReader extends FieldInfosReader {
+
+  /** Sole constructor. */
+  public Lucene50FieldInfosReader() {
+  }
+
+  @Override
+  public FieldInfos read(Directory directory, String segmentName, String segmentSuffix, IOContext context) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segmentName, segmentSuffix, Lucene50FieldInfosFormat.EXTENSION);
+    try (ChecksumIndexInput input = directory.openChecksumInput(fileName, context)) {
+      Throwable priorE = null;
+      FieldInfo infos[] = null;
+      try {
+        CodecUtil.checkHeader(input, Lucene50FieldInfosFormat.CODEC_NAME, 
+                                     Lucene50FieldInfosFormat.FORMAT_START, 
+                                     Lucene50FieldInfosFormat.FORMAT_CURRENT);
+        
+        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);
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosWriter.java?rev=1627187&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosWriter.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosWriter.java Wed Sep 24 00:59:50 2014
@@ -0,0 +1,103 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.FieldInfosWriter;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+/**
+ * Lucene 5.0 FieldInfos writer.
+ * 
+ * @see Lucene50FieldInfosFormat
+ * @lucene.experimental
+ */
+final class Lucene50FieldInfosWriter extends FieldInfosWriter {
+  
+  /** Sole constructor. */
+  public Lucene50FieldInfosWriter() {
+  }
+  
+  @Override
+  public void write(Directory directory, String segmentName, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segmentName, segmentSuffix, Lucene50FieldInfosFormat.EXTENSION);
+    try (IndexOutput output = directory.createOutput(fileName, context)) {
+      CodecUtil.writeHeader(output, Lucene50FieldInfosFormat.CODEC_NAME, Lucene50FieldInfosFormat.FORMAT_CURRENT);
+      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();
+    }
+  }  
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java?rev=1627187&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java Wed Sep 24 00:59:50 2014
@@ -0,0 +1,92 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import 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.IndexWriter; // javadocs
+import org.apache.lucene.index.SegmentInfo; // javadocs
+import org.apache.lucene.index.SegmentInfos; // javadocs
+import org.apache.lucene.store.DataOutput; // javadocs
+
+/**
+ * Lucene 5.0 Segment info format.
+ * <p>
+ * Files:
+ * <ul>
+ *   <li><tt>.si</tt>: Header, SegVersion, SegSize, IsCompoundFile, Diagnostics, Files, Id, Footer
+ * </ul>
+ * </p>
+ * Data types:
+ * <p>
+ * <ul>
+ *   <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
+ *   <li>SegSize --&gt; {@link DataOutput#writeInt Int32}</li>
+ *   <li>SegVersion --&gt; {@link DataOutput#writeString String}</li>
+ *   <li>Files --&gt; {@link DataOutput#writeStringSet Set&lt;String&gt;}</li>
+ *   <li>Diagnostics --&gt; {@link DataOutput#writeStringStringMap Map&lt;String,String&gt;}</li>
+ *   <li>IsCompoundFile --&gt; {@link DataOutput#writeByte Int8}</li>
+ *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
+ *   <li>Id --&gt; {@link DataOutput#writeString String}</li>
+ * </ul>
+ * </p>
+ * Field Descriptions:
+ * <p>
+ * <ul>
+ *   <li>SegVersion is the code version that created the segment.</li>
+ *   <li>SegSize is the number of documents contained in the segment index.</li>
+ *   <li>IsCompoundFile records whether the segment is written as a compound file or
+ *       not. If this is -1, the segment is not a compound file. If it is 1, the segment
+ *       is a compound file.</li>
+ *   <li>The Diagnostics Map is privately written by {@link IndexWriter}, as a debugging aid,
+ *       for each segment it creates. It includes metadata like the current Lucene
+ *       version, OS, Java version, why the segment was created (merge, flush,
+ *       addIndexes), etc.</li>
+ *   <li>Files is a list of files referred to by this segment.</li>
+ * </ul>
+ * </p>
+ * 
+ * @see SegmentInfos
+ * @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;
+  }
+
+  @Override
+  public SegmentInfoWriter getSegmentInfoWriter() {
+    return writer;
+  }
+
+  /** File extension used to store {@link SegmentInfo}. */
+  public final static String SI_EXTENSION = "si";
+  static final String CODEC_NAME = "Lucene50SegmentInfo";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoReader.java?rev=1627187&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoReader.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoReader.java Wed Sep 24 00:59:50 2014
@@ -0,0 +1,84 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.text.ParseException;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.SegmentInfoReader;
+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 5.0 implementation of {@link SegmentInfoReader}.
+ * 
+ * @see Lucene50SegmentInfoFormat
+ * @lucene.experimental
+ */
+public class Lucene50SegmentInfoReader extends SegmentInfoReader {
+
+  /** Sole constructor. */
+  public Lucene50SegmentInfoReader() {
+  }
+
+  @Override
+  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;
+        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();
+        
+        String id = input.readString();
+        
+        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;
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoWriter.java?rev=1627187&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoWriter.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoWriter.java Wed Sep 24 00:59:50 2014
@@ -0,0 +1,80 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.SegmentInfoWriter;
+import org.apache.lucene.index.FieldInfos;
+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;
+
+/**
+ * Lucene 5.0 implementation of {@link SegmentInfoWriter}.
+ * 
+ * @see Lucene50SegmentInfoFormat
+ * @lucene.experimental
+ */
+public class Lucene50SegmentInfoWriter extends SegmentInfoWriter {
+
+  /** Sole constructor. */
+  public Lucene50SegmentInfoWriter() {
+  }
+
+  /** Save a single segment's info. */
+  @Override
+  public void write(Directory dir, SegmentInfo si, FieldInfos fis, IOContext ioContext) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene50SegmentInfoFormat.SI_EXTENSION);
+    si.addFile(fileName);
+
+    final IndexOutput output = dir.createOutput(fileName, ioContext);
+
+    boolean success = false;
+    try {
+      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.writeString(version.toString());
+      output.writeInt(si.getDocCount());
+
+      output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
+      output.writeStringStringMap(si.getDiagnostics());
+      output.writeStringSet(si.files());
+      output.writeString(si.getId());
+      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();
+      }
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html?rev=1627187&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html Wed Sep 24 00:59:50 2014
@@ -0,0 +1,25 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<html>
+<head>
+   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+Lucene 5.0 file format.
+</body>
+</html>

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1627187&r1=1627186&r2=1627187&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Wed Sep 24 00:59:50 2014
@@ -169,21 +169,21 @@ public class CheckIndex {
 
       /** Current deletions generation. */
       public long deletionsGen;
-    
-      /** Number of deleted documents. */
-      public int numDeleted;
 
       /** True if we were able to open an AtomicReader on this
        *  segment. */
       public boolean openReaderPassed;
 
-      /** Number of fields in this segment. */
-      int numFields;
-
       /** Map that includes certain
        *  debugging details that IndexWriter records into
        *  each segment it creates */
       public Map<String,String> diagnostics;
+      
+      /** Status for testing of livedocs */
+      public LiveDocStatus liveDocStatus;
+      
+      /** Status for testing of field infos */
+      public FieldInfoStatus fieldInfoStatus;
 
       /** Status for testing of field norms (null if field norms could not be tested). */
       public FieldNormStatus fieldNormStatus;
@@ -200,6 +200,34 @@ public class CheckIndex {
       /** Status for testing of DocValues (null if DocValues could not be tested). */
       public DocValuesStatus docValuesStatus;
     }
+    
+    /**
+     * Status from testing livedocs
+     */
+    public static final class LiveDocStatus {
+      private LiveDocStatus() {
+      }
+      
+      /** Number of deleted documents. */
+      public int numDeleted;
+      
+      /** Exception thrown during term index test (null on success) */
+      public Throwable error = null;
+    }
+    
+    /**
+     * Status from testing field infos.
+     */
+    public static final class FieldInfoStatus {
+      private FieldInfoStatus() {
+      }
+
+      /** Number of fields successfully tested */
+      public long totFields = 0L;
+
+      /** Exception thrown during term index test (null on success) */
+      public Throwable error = null;
+    }
 
     /**
      * Status from testing field norms.
@@ -574,63 +602,34 @@ public class CheckIndex {
         reader.checkIntegrity();
         msg(infoStream, "OK");
 
-        if (infoStream != null)
-          infoStream.print("    test: check live docs.....");
+        if (reader.maxDoc() != info.info.getDocCount()) {
+          throw new RuntimeException("SegmentReader.maxDoc() " + reader.maxDoc() + " != SegmentInfos.docCount " + info.info.getDocCount());
+        }
+        
         final int numDocs = reader.numDocs();
         toLoseDocCount = numDocs;
+        
         if (reader.hasDeletions()) {
           if (reader.numDocs() != info.info.getDocCount() - info.getDelCount()) {
             throw new RuntimeException("delete count mismatch: info=" + (info.info.getDocCount() - info.getDelCount()) + " vs reader=" + reader.numDocs());
           }
-          if ((info.info.getDocCount()-reader.numDocs()) > reader.maxDoc()) {
-            throw new RuntimeException("too many deleted docs: maxDoc()=" + reader.maxDoc() + " vs del count=" + (info.info.getDocCount()-reader.numDocs()));
-          }
-          if (info.info.getDocCount() - numDocs != info.getDelCount()) {
-            throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.info.getDocCount() - numDocs));
+          if ((info.info.getDocCount() - reader.numDocs()) > reader.maxDoc()) {
+            throw new RuntimeException("too many deleted docs: maxDoc()=" + reader.maxDoc() + " vs del count=" + (info.info.getDocCount() - reader.numDocs()));
           }
-          Bits liveDocs = reader.getLiveDocs();
-          if (liveDocs == null) {
-            throw new RuntimeException("segment should have deletions, but liveDocs is null");
-          } else {
-            int numLive = 0;
-            for (int j = 0; j < liveDocs.length(); j++) {
-              if (liveDocs.get(j)) {
-                numLive++;
-              }
-            }
-            if (numLive != numDocs) {
-              throw new RuntimeException("liveDocs count mismatch: info=" + numDocs + ", vs bits=" + numLive);
-            }
+          if (info.info.getDocCount() - reader.numDocs() != info.getDelCount()) {
+            throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.info.getDocCount() - reader.numDocs()));
           }
-          
-          segInfoStat.numDeleted = info.info.getDocCount() - numDocs;
-          msg(infoStream, "OK [" + (segInfoStat.numDeleted) + " deleted docs]");
         } else {
           if (info.getDelCount() != 0) {
-            throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.info.getDocCount() - numDocs));
-          }
-          Bits liveDocs = reader.getLiveDocs();
-          if (liveDocs != null) {
-            // its ok for it to be non-null here, as long as none are set right?
-            for (int j = 0; j < liveDocs.length(); j++) {
-              if (!liveDocs.get(j)) {
-                throw new RuntimeException("liveDocs mismatch: info says no deletions but doc " + j + " is deleted.");
-              }
-            }
+            throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.info.getDocCount() - reader.numDocs()));
           }
-          msg(infoStream, "OK");
-        }
-        if (reader.maxDoc() != info.info.getDocCount()) {
-          throw new RuntimeException("SegmentReader.maxDoc() " + reader.maxDoc() + " != SegmentInfos.docCount " + info.info.getDocCount());
         }
+        
+        // Test Livedocs
+        segInfoStat.liveDocStatus = testLiveDocs(reader, infoStream, failFast);
 
-        // Test getFieldInfos()
-        if (infoStream != null) {
-          infoStream.print("    test: fields..............");
-        }         
-        FieldInfos fieldInfos = reader.getFieldInfos();
-        msg(infoStream, "OK [" + fieldInfos.size() + " fields]");
-        segInfoStat.numFields = fieldInfos.size();
+        // Test Fieldinfos
+        segInfoStat.fieldInfoStatus = testFieldInfos(reader, infoStream, failFast);
         
         // Test Field Norms
         segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
@@ -648,7 +647,11 @@ public class CheckIndex {
 
         // Rethrow the first exception we encountered
         //  This will cause stats for failed segments to be incremented properly
-        if (segInfoStat.fieldNormStatus.error != null) {
+        if (segInfoStat.liveDocStatus.error != null) {
+          throw new RuntimeException("Live docs test failed");
+        } else if (segInfoStat.fieldInfoStatus.error != null) {
+          throw new RuntimeException("Field Info test failed");
+        } else if (segInfoStat.fieldNormStatus.error != null) {
           throw new RuntimeException("Field Norm test failed");
         } else if (segInfoStat.termIndexStatus.error != null) {
           throw new RuntimeException("Term Index test failed");
@@ -707,6 +710,94 @@ public class CheckIndex {
 
     return result;
   }
+  
+  /**
+   * Test live docs.
+   * @lucene.experimental
+   */
+  public static Status.LiveDocStatus testLiveDocs(LeafReader reader, PrintStream infoStream, boolean failFast) throws IOException {
+    final Status.LiveDocStatus status = new Status.LiveDocStatus();
+    
+    try {
+      if (infoStream != null)
+        infoStream.print("    test: check live docs.....");
+      final int numDocs = reader.numDocs();
+      if (reader.hasDeletions()) {
+        Bits liveDocs = reader.getLiveDocs();
+        if (liveDocs == null) {
+          throw new RuntimeException("segment should have deletions, but liveDocs is null");
+        } else {
+          int numLive = 0;
+          for (int j = 0; j < liveDocs.length(); j++) {
+            if (liveDocs.get(j)) {
+              numLive++;
+            }
+          }
+          if (numLive != numDocs) {
+            throw new RuntimeException("liveDocs count mismatch: info=" + numDocs + ", vs bits=" + numLive);
+          }
+        }
+        
+        status.numDeleted = reader.numDeletedDocs();
+        msg(infoStream, "OK [" + (status.numDeleted) + " deleted docs]");
+      } else {
+        Bits liveDocs = reader.getLiveDocs();
+        if (liveDocs != null) {
+          // its ok for it to be non-null here, as long as none are set right?
+          for (int j = 0; j < liveDocs.length(); j++) {
+            if (!liveDocs.get(j)) {
+              throw new RuntimeException("liveDocs mismatch: info says no deletions but doc " + j + " is deleted.");
+            }
+          }
+        }
+        msg(infoStream, "OK");
+      }
+      
+    } catch (Throwable e) {
+      if (failFast) {
+        IOUtils.reThrow(e);
+      }
+      msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
+      status.error = e;
+      if (infoStream != null) {
+        e.printStackTrace(infoStream);
+      }
+    }
+    
+    return status;
+  }
+  
+  /**
+   * Test field infos.
+   * @lucene.experimental
+   */
+  public static Status.FieldInfoStatus testFieldInfos(LeafReader reader, PrintStream infoStream, boolean failFast) throws IOException {
+    final Status.FieldInfoStatus status = new Status.FieldInfoStatus();
+    
+    try {
+      // Test Field Infos
+      if (infoStream != null) {
+        infoStream.print("    test: field infos.........");
+      }
+      FieldInfos fieldInfos = reader.getFieldInfos();
+      for (FieldInfo f : fieldInfos) {
+        f.checkConsistency();
+      }
+      msg(infoStream, "OK [" + fieldInfos.size() + " fields]");
+      status.totFields = fieldInfos.size();
+    } catch (Throwable e) {
+      if (failFast) {
+        IOUtils.reThrow(e);
+      }
+      msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
+      status.error = e;
+      if (infoStream != null) {
+        e.printStackTrace(infoStream);
+      }
+    }
+    
+    return status;
+  }
 
   /**
    * Test field norms.

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java?rev=1627187&r1=1627186&r2=1627187&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java Wed Sep 24 00:59:50 2014
@@ -149,24 +149,45 @@ public final class FieldInfo {
     assert checkConsistency();
   }
 
-  private boolean checkConsistency() {
-    if (!indexed) {
-      assert !storeTermVector;
-      assert !storePayloads;
-      assert !omitNorms;
-      assert normType == null;
-      assert indexOptions == null;
-    } else {
-      assert indexOptions != null;
+  /** 
+   * Performs internal consistency checks.
+   * Always returns true (or throws IllegalStateException) 
+   */
+  public boolean checkConsistency() {
+    if (indexed) {
+      if (indexOptions == null) {
+        throw new IllegalStateException("indexed field '" + name + "' must have index options");
+      }
       if (omitNorms) {
-        assert normType == null;
+        if (normType != null) {
+          throw new IllegalStateException("indexed field '" + name + "' cannot both omit norms and have norms");
+        }
       }
       // Cannot store payloads unless positions are indexed:
-      assert indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !this.storePayloads;
+      if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0 && storePayloads) {
+        throw new IllegalStateException("indexed field '" + name + "' cannot have payloads without positions");
+      }
+    } else {
+      if (storeTermVector) {
+        throw new IllegalStateException("non-indexed field '" + name + "' cannot store term vectors");
+      }
+      if (storePayloads) {
+        throw new IllegalStateException("non-indexed field '" + name + "' cannot store payloads");
+      }
+      if (omitNorms) {
+        throw new IllegalStateException("non-indexed field '" + name + "' cannot omit norms");
+      }
+      if (normType != null) {
+        throw new IllegalStateException("non-indexed field '" + name + "' cannot have norms");
+      }
+      if (indexOptions != null) {
+        throw new IllegalStateException("non-indexed field '" + name + "' cannot have index options");
+
+      }
     }
     
-    if (dvGen != -1) {
-      assert docValueType != null;
+    if (dvGen != -1 && docValueType == null) {
+      throw new IllegalStateException("field '" + name + "' cannot have a docvalues update generation without having docvalues");
     }
 
     return true;

Modified: lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec?rev=1627187&r1=1627186&r2=1627187&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (original)
+++ lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec Wed Sep 24 00:59:50 2014
@@ -14,3 +14,4 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.lucene410.Lucene410Codec
+org.apache.lucene.codecs.lucene50.Lucene50Codec

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCodecUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCodecUtil.java?rev=1627187&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCodecUtil.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCodecUtil.java Wed Sep 24 00:59:50 2014
@@ -0,0 +1,197 @@
+package org.apache.lucene.index;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.store.BufferedChecksumIndexInput;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMFile;
+import org.apache.lucene.store.RAMInputStream;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.LuceneTestCase;
+
+/** tests for codecutil methods */
+public class TestCodecUtil extends LuceneTestCase {
+  
+  public void testHeaderLength() throws Exception {
+    RAMFile file = new RAMFile();
+    IndexOutput output = new RAMOutputStream(file, true);
+    CodecUtil.writeHeader(output, "FooBar", 5);
+    output.writeString("this is the data");
+    output.close();
+    
+    IndexInput input = new RAMInputStream("file", file);
+    input.seek(CodecUtil.headerLength("FooBar"));
+    assertEquals("this is the data", input.readString());
+    input.close();
+  }
+  
+  public void testWriteTooLongHeader() throws Exception {
+    StringBuilder tooLong = new StringBuilder();
+    for (int i = 0; i < 128; i++) {
+      tooLong.append('a');
+    }
+    RAMFile file = new RAMFile();
+    IndexOutput output = new RAMOutputStream(file, true);
+    try {
+      CodecUtil.writeHeader(output, tooLong.toString(), 5);
+      fail("didn't get expected exception");
+    } catch (IllegalArgumentException expected) {
+      // expected
+    }
+  }
+  
+  public void testWriteNonAsciiHeader() throws Exception {
+    RAMFile file = new RAMFile();
+    IndexOutput output = new RAMOutputStream(file, true);
+    try {
+      CodecUtil.writeHeader(output, "\u1234", 5);
+      fail("didn't get expected exception");
+    } catch (IllegalArgumentException expected) {
+      // expected
+    }
+  }
+  
+  public void testReadHeaderWrongMagic() throws Exception {
+    RAMFile file = new RAMFile();
+    IndexOutput output = new RAMOutputStream(file, true);
+    output.writeInt(1234);
+    output.close();
+    
+    IndexInput input = new RAMInputStream("file", file);
+    try {
+      CodecUtil.checkHeader(input, "bogus", 1, 1);
+      fail("didn't get expected exception");
+    } catch (CorruptIndexException expected) {
+      // expected
+    }
+  }
+  
+  public void testChecksumEntireFile() throws Exception {
+    RAMFile file = new RAMFile();
+    IndexOutput output = new RAMOutputStream(file, true);
+    CodecUtil.writeHeader(output, "FooBar", 5);
+    output.writeString("this is the data");
+    CodecUtil.writeFooter(output);
+    output.close();
+    
+    IndexInput input = new RAMInputStream("file", file);
+    CodecUtil.checksumEntireFile(input);
+    input.close();
+  }
+  
+  public void testCheckFooterValid() throws Exception {
+    RAMFile file = new RAMFile();
+    IndexOutput output = new RAMOutputStream(file, true);
+    CodecUtil.writeHeader(output, "FooBar", 5);
+    output.writeString("this is the data");
+    CodecUtil.writeFooter(output);
+    output.close();
+    
+    ChecksumIndexInput input = new BufferedChecksumIndexInput(new RAMInputStream("file", file));
+    Exception mine = new RuntimeException("fake exception");
+    try {
+      CodecUtil.checkFooter(input, mine);
+      fail("didn't get expected exception");
+    } catch (RuntimeException expected) {
+      assertEquals("fake exception", expected.getMessage());
+      Throwable suppressed[] = expected.getSuppressed();
+      assertEquals(1, suppressed.length);
+      assertTrue(suppressed[0].getMessage().contains("checksum passed"));
+    }
+    input.close();
+  }
+  
+  public void testCheckFooterValidAtFooter() throws Exception {
+    RAMFile file = new RAMFile();
+    IndexOutput output = new RAMOutputStream(file, true);
+    CodecUtil.writeHeader(output, "FooBar", 5);
+    output.writeString("this is the data");
+    CodecUtil.writeFooter(output);
+    output.close();
+    
+    ChecksumIndexInput input = new BufferedChecksumIndexInput(new RAMInputStream("file", file));
+    CodecUtil.checkHeader(input, "FooBar", 5, 5);
+    assertEquals("this is the data", input.readString());
+    Exception mine = new RuntimeException("fake exception");
+    try {
+      CodecUtil.checkFooter(input, mine);
+      fail("didn't get expected exception");
+    } catch (RuntimeException expected) {
+      assertEquals("fake exception", expected.getMessage());
+      Throwable suppressed[] = expected.getSuppressed();
+      assertEquals(1, suppressed.length);
+      assertTrue(suppressed[0].getMessage().contains("checksum passed"));
+    }
+    input.close();
+  }
+  
+  public void testCheckFooterValidPastFooter() throws Exception {
+    RAMFile file = new RAMFile();
+    IndexOutput output = new RAMOutputStream(file, true);
+    CodecUtil.writeHeader(output, "FooBar", 5);
+    output.writeString("this is the data");
+    CodecUtil.writeFooter(output);
+    output.close();
+    
+    ChecksumIndexInput input = new BufferedChecksumIndexInput(new RAMInputStream("file", file));
+    CodecUtil.checkHeader(input, "FooBar", 5, 5);
+    assertEquals("this is the data", input.readString());
+    // bogusly read a byte too far (can happen)
+    input.readByte();
+    Exception mine = new RuntimeException("fake exception");
+    try {
+      CodecUtil.checkFooter(input, mine);
+      fail("didn't get expected exception");
+    } catch (RuntimeException expected) {
+      assertEquals("fake exception", expected.getMessage());
+      Throwable suppressed[] = expected.getSuppressed();
+      assertEquals(1, suppressed.length);
+      assertTrue(suppressed[0].getMessage().contains("checksum status indeterminate"));
+    }
+    input.close();
+  }
+  
+  public void testCheckFooterInvalid() throws Exception {
+    RAMFile file = new RAMFile();
+    IndexOutput output = new RAMOutputStream(file, true);
+    CodecUtil.writeHeader(output, "FooBar", 5);
+    output.writeString("this is the data");
+    output.writeInt(CodecUtil.FOOTER_MAGIC);
+    output.writeInt(0);
+    output.writeLong(1234567); // write a bogus checksum
+    output.close();
+    
+    ChecksumIndexInput input = new BufferedChecksumIndexInput(new RAMInputStream("file", file));
+    CodecUtil.checkHeader(input, "FooBar", 5, 5);
+    assertEquals("this is the data", input.readString());
+    Exception mine = new RuntimeException("fake exception");
+    try {
+      CodecUtil.checkFooter(input, mine);
+      fail("didn't get expected exception");
+    } catch (RuntimeException expected) {
+      assertEquals("fake exception", expected.getMessage());
+      Throwable suppressed[] = expected.getSuppressed();
+      assertEquals(1, suppressed.length);
+      assertTrue(suppressed[0].getMessage().contains("checksum failed"));
+    }
+    input.close();
+  }
+}

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java?rev=1627187&r1=1627186&r2=1627187&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java Wed Sep 24 00:59:50 2014
@@ -46,8 +46,8 @@ import org.apache.lucene.codecs.DocValue
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.asserting.AssertingCodec;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
-import org.apache.lucene.codecs.lucene410.Lucene410Codec;
 import org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50Codec;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 import org.apache.lucene.document.BinaryDocValuesField;
@@ -64,11 +64,6 @@ import org.apache.lucene.index.FilterLea
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.CheckIndex;
-import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
-import org.apache.lucene.index.CheckIndex.Status.FieldNormStatus;
-import org.apache.lucene.index.CheckIndex.Status.StoredFieldStatus;
-import org.apache.lucene.index.CheckIndex.Status.TermIndexStatus;
-import org.apache.lucene.index.CheckIndex.Status.TermVectorStatus;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
@@ -248,11 +243,13 @@ public final class TestUtil {
     PrintStream infoStream = new PrintStream(bos, false, IOUtils.UTF_8);
 
     reader.checkIntegrity();
-    FieldNormStatus fieldNormStatus = CheckIndex.testFieldNorms(reader, infoStream, true);
-    TermIndexStatus termIndexStatus = CheckIndex.testPostings(reader, infoStream, false, true);
-    StoredFieldStatus storedFieldStatus = CheckIndex.testStoredFields(reader, infoStream, true);
-    TermVectorStatus termVectorStatus = CheckIndex.testTermVectors(reader, infoStream, false, crossCheckTermVectors, true);
-    DocValuesStatus docValuesStatus = CheckIndex.testDocValues(reader, infoStream, true);
+    CheckIndex.testLiveDocs(reader, infoStream, true);
+    CheckIndex.testFieldInfos(reader, infoStream, true);
+    CheckIndex.testFieldNorms(reader, infoStream, true);
+    CheckIndex.testPostings(reader, infoStream, false, true);
+    CheckIndex.testStoredFields(reader, infoStream, true);
+    CheckIndex.testTermVectors(reader, infoStream, false, crossCheckTermVectors, true);
+    CheckIndex.testDocValues(reader, infoStream, true);
     
     if (LuceneTestCase.INFOSTREAM) {
       System.out.println(bos.toString(IOUtils.UTF_8));
@@ -737,7 +734,7 @@ public final class TestUtil {
    * This may be different than {@link Codec#getDefault()} because that is randomized. 
    */
   public static Codec getDefaultCodec() {
-    return new Lucene410Codec();
+    return new Lucene50Codec();
   }
   
   /** 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java?rev=1627187&r1=1627186&r2=1627187&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java Wed Sep 24 00:59:50 2014
@@ -3,7 +3,7 @@ package org.apache.solr.core;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene410.Lucene410Codec;
+import org.apache.lucene.codecs.lucene50.Lucene50Codec;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.util.plugin.SolrCoreAware;
@@ -51,7 +51,7 @@ public class SchemaCodecFactory extends 
   @Override
   public void init(NamedList args) {
     super.init(args);
-    codec = new Lucene410Codec() {
+    codec = new Lucene50Codec() {
       @Override
       public PostingsFormat getPostingsFormatForField(String field) {
         final SchemaField schemaField = core.getLatestSchema().getFieldOrNull(field);