You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2014/04/04 12:27:14 UTC

svn commit: r1584603 [4/12] - in /lucene/dev/branches/solr5914: ./ dev-tools/ dev-tools/idea/solr/core/src/test/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/java/org/apache/lucene/analysis/br/ lucene/analysis/common/src/...

Modified: lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java (original)
+++ lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java Fri Apr  4 10:27:05 2014
@@ -37,6 +37,7 @@ import org.apache.lucene.index.SortedDoc
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -77,6 +78,7 @@ class MemoryDocValuesProducer extends Do
   
   private final int maxDoc;
   private final AtomicLong ramBytesUsed;
+  private final int version;
   
   static final byte NUMBER = 0;
   static final byte BYTES = 1;
@@ -91,15 +93,15 @@ class MemoryDocValuesProducer extends Do
   
   static final int VERSION_START = 0;
   static final int VERSION_GCD_COMPRESSION = 1;
-  static final int VERSION_CURRENT = VERSION_GCD_COMPRESSION;
+  static final int VERSION_CHECKSUM = 2;
+  static final int VERSION_CURRENT = VERSION_CHECKSUM;
     
   MemoryDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
     maxDoc = state.segmentInfo.getDocCount();
     String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
     // read in the entries from the metadata file.
-    IndexInput in = state.directory.openInput(metaName, state.context);
+    ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
     boolean success = false;
-    final int version;
     try {
       version = CodecUtil.checkHeader(in, metaCodec, 
                                       VERSION_START,
@@ -108,6 +110,11 @@ class MemoryDocValuesProducer extends Do
       binaries = new HashMap<>();
       fsts = new HashMap<>();
       readFields(in, state.fieldInfos);
+      if (version >= VERSION_CHECKSUM) {
+        CodecUtil.checkFooter(in);
+      } else {
+        CodecUtil.checkEOF(in);
+      }
       ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
       success = true;
     } finally {
@@ -208,6 +215,13 @@ class MemoryDocValuesProducer extends Do
     return ramBytesUsed.get();
   }
   
+  @Override
+  public void checkIntegrity() throws IOException {
+    if (version >= VERSION_CHECKSUM) {
+      CodecUtil.checksumEntireFile(data);
+    }
+  }
+  
   private NumericDocValues loadNumeric(FieldInfo field) throws IOException {
     NumericEntry entry = numerics.get(field.number);
     data.seek(entry.offset + entry.missingBytes);

Modified: lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java (original)
+++ lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java Fri Apr  4 10:27:05 2014
@@ -25,6 +25,7 @@ import java.util.Map;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
+import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.PostingsFormat;
@@ -41,6 +42,7 @@ import org.apache.lucene.index.SegmentWr
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
@@ -271,6 +273,9 @@ public final class MemoryPostingsFormat 
   }
 
   private static String EXTENSION = "ram";
+  private static final String CODEC_NAME = "MemoryPostings";
+  private static final int VERSION_START = 0;
+  private static final int VERSION_CURRENT = VERSION_START;
 
   private class MemoryFieldsConsumer extends FieldsConsumer implements Closeable {
     private final SegmentWriteState state;
@@ -279,6 +284,15 @@ public final class MemoryPostingsFormat 
     private MemoryFieldsConsumer(SegmentWriteState state) throws IOException {
       final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, EXTENSION);
       out = state.directory.createOutput(fileName, state.context);
+      boolean success = false;
+      try {
+        CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
+        success = true;
+      } finally {
+        if (!success) {
+          IOUtils.closeWhileHandlingException(out);
+        }
+      }
       this.state = state;
     }
 
@@ -403,6 +417,7 @@ public final class MemoryPostingsFormat 
       // EOF marker:
       try {
         out.writeVInt(0);
+        CodecUtil.writeFooter(out);
       } finally {
         out.close();
       }
@@ -951,7 +966,8 @@ public final class MemoryPostingsFormat 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
     final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, EXTENSION);
-    final IndexInput in = state.directory.openInput(fileName, IOContext.READONCE);
+    final ChecksumIndexInput in = state.directory.openChecksumInput(fileName, IOContext.READONCE);
+    CodecUtil.checkHeader(in, CODEC_NAME, VERSION_START, VERSION_CURRENT);
 
     final SortedMap<String,TermsReader> fields = new TreeMap<>();
 
@@ -965,6 +981,7 @@ public final class MemoryPostingsFormat 
         // System.out.println("load field=" + termsReader.field.name);
         fields.put(termsReader.field.name, termsReader);
       }
+      CodecUtil.checkFooter(in);
     } finally {
       in.close();
     }
@@ -1002,6 +1019,9 @@ public final class MemoryPostingsFormat 
         }
         return sizeInBytes;
       }
+
+      @Override
+      public void checkIntegrity() throws IOException {}
     };
   }
 }

Modified: lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsReader.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsReader.java (original)
+++ lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsReader.java Fri Apr  4 10:27:05 2014
@@ -653,4 +653,9 @@ public class PulsingPostingsReader exten
   public long ramBytesUsed() {
     return ((wrappedPostingsReader!=null) ? wrappedPostingsReader.ramBytesUsed(): 0);
   }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    wrappedPostingsReader.checkIntegrity();
+  }
 }

Modified: lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java (original)
+++ lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java Fri Apr  4 10:27:05 2014
@@ -17,6 +17,7 @@ package org.apache.lucene.codecs.simplet
  * limitations under the License.
  */
 
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.CHECKSUM;
 import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.END;
 import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.FIELD;
 import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.LENGTH;
@@ -30,6 +31,7 @@ import static org.apache.lucene.codecs.s
 import java.io.IOException;
 import java.math.BigDecimal;
 import java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
 import java.text.DecimalFormat;
 import java.text.DecimalFormatSymbols;
 import java.text.ParseException;
@@ -47,6 +49,8 @@ import org.apache.lucene.index.NumericDo
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.store.BufferedChecksumIndexInput;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -225,7 +229,7 @@ class SimpleTextDocValuesReader extends 
           assert StringHelper.startsWith(scratch, LENGTH);
           int len;
           try {
-            len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, "UTF-8")).intValue();
+            len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, StandardCharsets.UTF_8)).intValue();
           } catch (ParseException pe) {
             CorruptIndexException e = new CorruptIndexException("failed to parse int length (resource=" + in + ")");
             e.initCause(pe);
@@ -257,7 +261,7 @@ class SimpleTextDocValuesReader extends 
           assert StringHelper.startsWith(scratch, LENGTH);
           int len;
           try {
-            len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, "UTF-8")).intValue();
+            len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, StandardCharsets.UTF_8)).intValue();
           } catch (ParseException pe) {
             CorruptIndexException e = new CorruptIndexException("failed to parse int length (resource=" + in + ")");
             e.initCause(pe);
@@ -326,7 +330,7 @@ class SimpleTextDocValuesReader extends 
           assert StringHelper.startsWith(scratch, LENGTH): "got " + scratch.utf8ToString() + " in=" + in;
           int len;
           try {
-            len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, "UTF-8")).intValue();
+            len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, StandardCharsets.UTF_8)).intValue();
           } catch (ParseException pe) {
             CorruptIndexException e = new CorruptIndexException("failed to parse int length (resource=" + in + ")");
             e.initCause(pe);
@@ -404,7 +408,7 @@ class SimpleTextDocValuesReader extends 
           assert StringHelper.startsWith(scratch, LENGTH): "got " + scratch.utf8ToString() + " in=" + in;
           int len;
           try {
-            len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, "UTF-8")).intValue();
+            len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, StandardCharsets.UTF_8)).intValue();
           } catch (ParseException pe) {
             CorruptIndexException e = new CorruptIndexException("failed to parse int length (resource=" + in + ")");
             e.initCause(pe);
@@ -460,11 +464,26 @@ class SimpleTextDocValuesReader extends 
 
   /** Used only in ctor: */
   private String stripPrefix(BytesRef prefix) throws IOException {
-    return new String(scratch.bytes, scratch.offset + prefix.length, scratch.length - prefix.length, "UTF-8");
+    return new String(scratch.bytes, scratch.offset + prefix.length, scratch.length - prefix.length, StandardCharsets.UTF_8);
   }
 
   @Override
   public long ramBytesUsed() {
     return 0;
   }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    BytesRef scratch = new BytesRef();
+    IndexInput clone = data.clone();
+    clone.seek(0);
+    ChecksumIndexInput input = new BufferedChecksumIndexInput(clone);
+    while(true) {
+      SimpleTextUtil.readLine(input, scratch);
+      if (scratch.equals(END)) {
+        SimpleTextUtil.checkFooter(input, CHECKSUM);
+        break;
+      }
+    }
+  }
 }

Modified: lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java (original)
+++ lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java Fri Apr  4 10:27:05 2014
@@ -36,6 +36,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
 class SimpleTextDocValuesWriter extends DocValuesConsumer {
+  final static BytesRef CHECKSUM = new BytesRef("checksum ");
   final static BytesRef END     = new BytesRef("END");
   final static BytesRef FIELD   = new BytesRef("field ");
   final static BytesRef TYPE    = new BytesRef("  type ");
@@ -49,7 +50,7 @@ class SimpleTextDocValuesWriter extends 
   final static BytesRef NUMVALUES = new BytesRef("  numvalues ");
   final static BytesRef ORDPATTERN = new BytesRef("  ordpattern ");
   
-  final IndexOutput data;
+  IndexOutput data;
   final BytesRef scratch = new BytesRef();
   final int numDocs;
   private final Set<String> fieldsSeen = new HashSet<>(); // for asserting
@@ -389,18 +390,25 @@ class SimpleTextDocValuesWriter extends 
   
   @Override
   public void close() throws IOException {
-    boolean success = false;
-    try {
-      assert !fieldsSeen.isEmpty();
-      // TODO: sheisty to do this here?
-      SimpleTextUtil.write(data, END);
-      SimpleTextUtil.writeNewline(data);
-      success = true;
-    } finally {
-      if (success) {
-        IOUtils.close(data);
-      } else {
-        IOUtils.closeWhileHandlingException(data);
+    if (data != null) {
+      boolean success = false;
+      try {
+        assert !fieldsSeen.isEmpty();
+        // TODO: sheisty to do this here?
+        SimpleTextUtil.write(data, END);
+        SimpleTextUtil.writeNewline(data);
+        String checksum = Long.toString(data.getChecksum());
+        SimpleTextUtil.write(data, CHECKSUM);
+        SimpleTextUtil.write(data, checksum, scratch);
+        SimpleTextUtil.writeNewline(data);
+        success = true;
+      } finally {
+        if (success) {
+          IOUtils.close(data);
+        } else {
+          IOUtils.closeWhileHandlingException(data);
+        }
+        data = null;
       }
     }
   }

Modified: lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java (original)
+++ lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java Fri Apr  4 10:27:05 2014
@@ -18,20 +18,20 @@ package org.apache.lucene.codecs.simplet
  */
 
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.lucene.codecs.FieldInfosReader;
-import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.StringHelper;
@@ -49,7 +49,7 @@ public class SimpleTextFieldInfosReader 
   @Override
   public FieldInfos read(Directory directory, String segmentName, String segmentSuffix, IOContext iocontext) throws IOException {
     final String fileName = IndexFileNames.segmentFileName(segmentName, segmentSuffix, FIELD_INFOS_EXTENSION);
-    IndexInput input = directory.openInput(fileName, iocontext);
+    ChecksumIndexInput input = directory.openChecksumInput(fileName, iocontext);
     BytesRef scratch = new BytesRef();
     
     boolean success = false;
@@ -129,9 +129,7 @@ public class SimpleTextFieldInfosReader 
         infos[i].setDocValuesGen(dvGen);
       }
 
-      if (input.getFilePointer() != input.length()) {
-        throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")");
-      }
+      SimpleTextUtil.checkFooter(input, CHECKSUM);
       
       FieldInfos fieldInfos = new FieldInfos(infos);
       success = true;
@@ -154,6 +152,6 @@ public class SimpleTextFieldInfosReader 
   }
   
   private String readString(int offset, BytesRef scratch) {
-    return new String(scratch.bytes, scratch.offset+offset, scratch.length-offset, IOUtils.CHARSET_UTF_8);
+    return new String(scratch.bytes, scratch.offset+offset, scratch.length-offset, StandardCharsets.UTF_8);
   }
 }

Modified: lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java (original)
+++ lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java Fri Apr  4 10:27:05 2014
@@ -58,6 +58,7 @@ public class SimpleTextFieldInfosWriter 
   static final BytesRef NUM_ATTS        =  new BytesRef("  attributes ");
   final static BytesRef ATT_KEY         =  new BytesRef("    key ");
   final static BytesRef ATT_VALUE       =  new BytesRef("    value ");
+  final static BytesRef CHECKSUM        =  new BytesRef("checksum ");
   
   @Override
   public void write(Directory directory, String segmentName, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
@@ -132,6 +133,10 @@ public class SimpleTextFieldInfosWriter 
           }
         }
       }
+      String checksum = Long.toString(out.getChecksum());
+      SimpleTextUtil.write(out, CHECKSUM);
+      SimpleTextUtil.write(out, checksum, scratch);
+      SimpleTextUtil.writeNewline(out);
       success = true;
     } finally {
       if (success) {

Modified: lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java Fri Apr  4 10:27:05 2014
@@ -18,6 +18,7 @@ package org.apache.lucene.codecs.simplet
  */
 
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -33,6 +34,8 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.BufferedChecksumIndexInput;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
@@ -50,22 +53,23 @@ import org.apache.lucene.util.fst.PairOu
 import org.apache.lucene.util.fst.PositiveIntOutputs;
 import org.apache.lucene.util.fst.Util;
 
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.CHECKSUM;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FIELD;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.TERM;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.DOC;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FREQ;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.POS;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.START_OFFSET;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END_OFFSET;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.PAYLOAD;
+
 class SimpleTextFieldsReader extends FieldsProducer {
   private final TreeMap<String,Long> fields;
   private final IndexInput in;
   private final FieldInfos fieldInfos;
   private final int maxDoc;
 
-  final static BytesRef END          = SimpleTextFieldsWriter.END;
-  final static BytesRef FIELD        = SimpleTextFieldsWriter.FIELD;
-  final static BytesRef TERM         = SimpleTextFieldsWriter.TERM;
-  final static BytesRef DOC          = SimpleTextFieldsWriter.DOC;
-  final static BytesRef FREQ         = SimpleTextFieldsWriter.FREQ;
-  final static BytesRef POS          = SimpleTextFieldsWriter.POS;
-  final static BytesRef START_OFFSET = SimpleTextFieldsWriter.START_OFFSET;
-  final static BytesRef END_OFFSET   = SimpleTextFieldsWriter.END_OFFSET;
-  final static BytesRef PAYLOAD      = SimpleTextFieldsWriter.PAYLOAD;
-
   public SimpleTextFieldsReader(SegmentReadState state) throws IOException {
     this.maxDoc = state.segmentInfo.getDocCount();
     fieldInfos = state.fieldInfos;
@@ -82,16 +86,18 @@ class SimpleTextFieldsReader extends Fie
   }
   
   private TreeMap<String,Long> readFields(IndexInput in) throws IOException {
+    ChecksumIndexInput input = new BufferedChecksumIndexInput(in);
     BytesRef scratch = new BytesRef(10);
     TreeMap<String,Long> fields = new TreeMap<>();
     
     while (true) {
-      SimpleTextUtil.readLine(in, scratch);
+      SimpleTextUtil.readLine(input, scratch);
       if (scratch.equals(END)) {
+        SimpleTextUtil.checkFooter(input, CHECKSUM);
         return fields;
       } else if (StringHelper.startsWith(scratch, FIELD)) {
-        String fieldName = new String(scratch.bytes, scratch.offset + FIELD.length, scratch.length - FIELD.length, "UTF-8");
-        fields.put(fieldName, in.getFilePointer());
+        String fieldName = new String(scratch.bytes, scratch.offset + FIELD.length, scratch.length - FIELD.length, StandardCharsets.UTF_8);
+        fields.put(fieldName, input.getFilePointer());
       }
     }
   }
@@ -668,4 +674,7 @@ class SimpleTextFieldsReader extends Fie
     }
     return sizeInBytes;
   }
+
+  @Override
+  public void checkIntegrity() throws IOException {}
 }

Modified: lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java (original)
+++ lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java Fri Apr  4 10:27:05 2014
@@ -35,10 +35,11 @@ import org.apache.lucene.util.IOUtils;
 
 class SimpleTextFieldsWriter extends FieldsConsumer implements Closeable {
   
-  private final IndexOutput out;
+  private IndexOutput out;
   private final BytesRef scratch = new BytesRef(10);
   private final SegmentWriteState writeState;
 
+  final static BytesRef CHECKSUM     = new BytesRef("checksum ");
   final static BytesRef END          = new BytesRef("END");
   final static BytesRef FIELD        = new BytesRef("field ");
   final static BytesRef TERM         = new BytesRef("  term ");
@@ -215,11 +216,18 @@ class SimpleTextFieldsWriter extends Fie
 
   @Override
   public void close() throws IOException {
-    try {
-      write(END);
-      newline();
-    } finally {
-      out.close();
+    if (out != null) {
+      try {
+        write(END);
+        newline();
+        String checksum = Long.toString(out.getChecksum());
+        write(CHECKSUM);
+        write(checksum);
+        newline();
+      } finally {
+        out.close();
+        out = null;
+      }
     }
   }
 }

Modified: lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java (original)
+++ lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java Fri Apr  4 10:27:05 2014
@@ -24,9 +24,9 @@ import java.util.Collection;
 import org.apache.lucene.codecs.LiveDocsFormat;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentCommitInfo;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
@@ -50,6 +50,7 @@ public class SimpleTextLiveDocsFormat ex
   final static BytesRef SIZE             = new BytesRef("size ");
   final static BytesRef DOC              = new BytesRef("  doc ");
   final static BytesRef END              = new BytesRef("END");
+  final static BytesRef CHECKSUM         = new BytesRef("checksum ");
   
   @Override
   public MutableBits newLiveDocs(int size) throws IOException {
@@ -69,10 +70,10 @@ public class SimpleTextLiveDocsFormat ex
     CharsRef scratchUTF16 = new CharsRef();
     
     String fileName = IndexFileNames.fileNameFromGeneration(info.info.name, LIVEDOCS_EXTENSION, info.getDelGen());
-    IndexInput in = null;
+    ChecksumIndexInput in = null;
     boolean success = false;
     try {
-      in = dir.openInput(fileName, context);
+      in = dir.openChecksumInput(fileName, context);
       
       SimpleTextUtil.readLine(in, scratch);
       assert StringHelper.startsWith(scratch, SIZE);
@@ -88,6 +89,8 @@ public class SimpleTextLiveDocsFormat ex
         SimpleTextUtil.readLine(in, scratch);
       }
       
+      SimpleTextUtil.checkFooter(in, CHECKSUM);
+      
       success = true;
       return new SimpleTextBits(bits, size);
     } finally {
@@ -127,6 +130,10 @@ public class SimpleTextLiveDocsFormat ex
       
       SimpleTextUtil.write(out, END);
       SimpleTextUtil.writeNewline(out);
+      String checksum = Long.toString(out.getChecksum());
+      SimpleTextUtil.write(out, CHECKSUM);
+      SimpleTextUtil.write(out, checksum, scratch);
+      SimpleTextUtil.writeNewline(out);
       success = true;
     } finally {
       if (success) {

Modified: lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoReader.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoReader.java (original)
+++ lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoReader.java Fri Apr  4 10:27:05 2014
@@ -17,6 +17,7 @@ package org.apache.lucene.codecs.simplet
  * limitations under the License.
  */
 
+import static org.apache.lucene.codecs.simpletext.SimpleTextSegmentInfoWriter.SI_CHECKSUM;
 import static org.apache.lucene.codecs.simpletext.SimpleTextSegmentInfoWriter.SI_DIAG_KEY;
 import static org.apache.lucene.codecs.simpletext.SimpleTextSegmentInfoWriter.SI_DIAG_VALUE;
 import static org.apache.lucene.codecs.simpletext.SimpleTextSegmentInfoWriter.SI_DOCCOUNT;
@@ -27,6 +28,7 @@ import static org.apache.lucene.codecs.s
 import static org.apache.lucene.codecs.simpletext.SimpleTextSegmentInfoWriter.SI_VERSION;
 
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -35,9 +37,9 @@ import java.util.Set;
 import org.apache.lucene.codecs.SegmentInfoReader;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.StringHelper;
@@ -54,7 +56,7 @@ public class SimpleTextSegmentInfoReader
   public SegmentInfo read(Directory directory, String segmentName, IOContext context) throws IOException {
     BytesRef scratch = new BytesRef();
     String segFileName = IndexFileNames.segmentFileName(segmentName, "", SimpleTextSegmentInfoFormat.SI_EXTENSION);
-    IndexInput input = directory.openInput(segFileName, context);
+    ChecksumIndexInput input = directory.openChecksumInput(segFileName, context);
     boolean success = false;
     try {
       SimpleTextUtil.readLine(input, scratch);
@@ -96,6 +98,8 @@ public class SimpleTextSegmentInfoReader
         String fileName = readString(SI_FILE.length, scratch);
         files.add(fileName);
       }
+      
+      SimpleTextUtil.checkFooter(input, SI_CHECKSUM);
 
       SegmentInfo info = new SegmentInfo(directory, version, segmentName, docCount, 
                                          isCompoundFile, null, diagnostics);
@@ -112,6 +116,6 @@ public class SimpleTextSegmentInfoReader
   }
 
   private String readString(int offset, BytesRef scratch) {
-    return new String(scratch.bytes, scratch.offset+offset, scratch.length-offset, IOUtils.CHARSET_UTF_8);
+    return new String(scratch.bytes, scratch.offset+offset, scratch.length-offset, StandardCharsets.UTF_8);
   }
 }

Modified: lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoWriter.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoWriter.java (original)
+++ lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoWriter.java Fri Apr  4 10:27:05 2014
@@ -47,6 +47,7 @@ public class SimpleTextSegmentInfoWriter
   final static BytesRef SI_DIAG_VALUE       = new BytesRef("      value ");
   final static BytesRef SI_NUM_FILES        = new BytesRef("    files ");
   final static BytesRef SI_FILE             = new BytesRef("      file ");
+  final static BytesRef SI_CHECKSUM         = new BytesRef("    checksum ");
   
   @Override
   public void write(Directory dir, SegmentInfo si, FieldInfos fis, IOContext ioContext) throws IOException {
@@ -55,7 +56,7 @@ public class SimpleTextSegmentInfoWriter
     si.addFile(segFileName);
 
     boolean success = false;
-    IndexOutput output = dir.createOutput(segFileName,  ioContext);
+    IndexOutput output = dir.createOutput(segFileName, ioContext);
 
     try {
       BytesRef scratch = new BytesRef();
@@ -103,6 +104,11 @@ public class SimpleTextSegmentInfoWriter
           SimpleTextUtil.writeNewline(output);
         }
       }
+      
+      String checksum = Long.toString(output.getChecksum());
+      SimpleTextUtil.write(output, SI_CHECKSUM);
+      SimpleTextUtil.write(output, checksum, scratch);
+      SimpleTextUtil.writeNewline(output);
       success = true;
     } finally {
       if (!success) {

Modified: lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java (original)
+++ lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java Fri Apr  4 10:27:05 2014
@@ -18,6 +18,7 @@ package org.apache.lucene.codecs.simplet
  */
 
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.index.FieldInfo;
@@ -26,6 +27,8 @@ import org.apache.lucene.index.IndexFile
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.BufferedChecksumIndexInput;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -78,15 +81,17 @@ public class SimpleTextStoredFieldsReade
   // stored fields file in entirety up-front and save the offsets 
   // so we can seek to the documents later.
   private void readIndex(int size) throws IOException {
+    ChecksumIndexInput input = new BufferedChecksumIndexInput(in);
     offsets = new long[size];
     int upto = 0;
     while (!scratch.equals(END)) {
-      readLine();
+      SimpleTextUtil.readLine(input, scratch);
       if (StringHelper.startsWith(scratch, DOC)) {
-        offsets[upto] = in.getFilePointer();
+        offsets[upto] = input.getFilePointer();
         upto++;
       }
     }
+    SimpleTextUtil.checkFooter(input, CHECKSUM);
     assert upto == offsets.length;
   }
   
@@ -141,7 +146,7 @@ public class SimpleTextStoredFieldsReade
     readLine();
     assert StringHelper.startsWith(scratch, VALUE);
     if (type == TYPE_STRING) {
-      visitor.stringField(fieldInfo, new String(scratch.bytes, scratch.offset+VALUE.length, scratch.length-VALUE.length, "UTF-8"));
+      visitor.stringField(fieldInfo, new String(scratch.bytes, scratch.offset+VALUE.length, scratch.length-VALUE.length, StandardCharsets.UTF_8));
     } else if (type == TYPE_BINARY) {
       byte[] copy = new byte[scratch.length-VALUE.length];
       System.arraycopy(scratch.bytes, scratch.offset+VALUE.length, copy, 0, copy.length);
@@ -188,6 +193,11 @@ public class SimpleTextStoredFieldsReade
     return ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
   }
   
+  private String readString(int offset, BytesRef scratch) {
+    UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+offset, scratch.length-offset, scratchUTF16);
+    return scratchUTF16.toString();
+  }
+  
   private boolean equalsAt(BytesRef a, BytesRef b, int bOffset) {
     return a.length == b.length - bOffset && 
         ArrayUtil.equals(a.bytes, a.offset, b.bytes, b.offset + bOffset, b.length - bOffset);
@@ -197,4 +207,7 @@ public class SimpleTextStoredFieldsReade
   public long ramBytesUsed() {
     return 0;
   }
+
+  @Override
+  public void checkIntegrity() throws IOException {}
 }

Modified: lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java (original)
+++ lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java Fri Apr  4 10:27:05 2014
@@ -51,13 +51,14 @@ public class SimpleTextStoredFieldsWrite
   final static BytesRef TYPE_FLOAT  = new BytesRef("float");
   final static BytesRef TYPE_DOUBLE = new BytesRef("double");
 
-  final static BytesRef END     = new BytesRef("END");
-  final static BytesRef DOC     = new BytesRef("doc ");
-  final static BytesRef NUM     = new BytesRef("  numfields ");
-  final static BytesRef FIELD   = new BytesRef("  field ");
-  final static BytesRef NAME    = new BytesRef("    name ");
-  final static BytesRef TYPE    = new BytesRef("    type ");
-  final static BytesRef VALUE   = new BytesRef("    value ");
+  final static BytesRef CHECKSUM = new BytesRef("checksum ");
+  final static BytesRef END      = new BytesRef("END");
+  final static BytesRef DOC      = new BytesRef("doc ");
+  final static BytesRef NUM      = new BytesRef("  numfields ");
+  final static BytesRef FIELD    = new BytesRef("  field ");
+  final static BytesRef NAME     = new BytesRef("    name ");
+  final static BytesRef TYPE     = new BytesRef("    type ");
+  final static BytesRef VALUE    = new BytesRef("    value ");
   
   private final BytesRef scratch = new BytesRef();
   
@@ -171,6 +172,10 @@ public class SimpleTextStoredFieldsWrite
     }
     write(END);
     newLine();
+    String checksum = Long.toString(out.getChecksum());
+    write(CHECKSUM);
+    write(checksum);
+    newLine();
   }
 
   @Override

Modified: lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (original)
+++ lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java Fri Apr  4 10:27:05 2014
@@ -33,6 +33,8 @@ import org.apache.lucene.index.SegmentIn
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.BufferedChecksumIndexInput;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -82,15 +84,17 @@ public class SimpleTextTermVectorsReader
   // vectors file in entirety up-front and save the offsets 
   // so we can seek to the data later.
   private void readIndex(int maxDoc) throws IOException {
+    ChecksumIndexInput input = new BufferedChecksumIndexInput(in);
     offsets = new long[maxDoc];
     int upto = 0;
     while (!scratch.equals(END)) {
-      readLine();
+      SimpleTextUtil.readLine(input, scratch);
       if (StringHelper.startsWith(scratch, DOC)) {
-        offsets[upto] = in.getFilePointer();
+        offsets[upto] = input.getFilePointer();
         upto++;
       }
     }
+    SimpleTextUtil.checkFooter(input, CHECKSUM);
     assert upto == offsets.length;
   }
   
@@ -537,4 +541,7 @@ public class SimpleTextTermVectorsReader
   public long ramBytesUsed() {
     return 0;
   }
+
+  @Override
+  public void checkIntegrity() throws IOException {}
 }

Modified: lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java (original)
+++ lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java Fri Apr  4 10:27:05 2014
@@ -37,6 +37,7 @@ import org.apache.lucene.util.IOUtils;
  */
 public class SimpleTextTermVectorsWriter extends TermVectorsWriter {
   
+  static final BytesRef CHECKSUM           = new BytesRef("checksum ");
   static final BytesRef END                = new BytesRef("END");
   static final BytesRef DOC                = new BytesRef("doc ");
   static final BytesRef NUMFIELDS          = new BytesRef("  numfields ");
@@ -177,6 +178,10 @@ public class SimpleTextTermVectorsWriter
     }
     write(END);
     newLine();
+    String checksum = Long.toString(out.getChecksum());
+    write(CHECKSUM);
+    write(checksum);
+    newLine();
   }
   
   @Override

Modified: lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextUtil.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextUtil.java (original)
+++ lucene/dev/branches/solr5914/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextUtil.java Fri Apr  4 10:27:05 2014
@@ -17,11 +17,16 @@ package org.apache.lucene.codecs.simplet
  * limitations under the License.
  */
 
+import static org.apache.lucene.codecs.simpletext.SimpleTextStoredFieldsWriter.CHECKSUM;
+
 import java.io.IOException;
 
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
 
 class SimpleTextUtil {
@@ -67,4 +72,18 @@ class SimpleTextUtil {
     scratch.offset = 0;
     scratch.length = upto;
   }
+  
+  public static void checkFooter(ChecksumIndexInput input, BytesRef prefix) throws IOException {
+    BytesRef scratch = new BytesRef();
+    String expectedChecksum = Long.toString(input.getChecksum());
+    SimpleTextUtil.readLine(input, scratch);
+    assert StringHelper.startsWith(scratch, prefix);
+    String actualChecksum = new BytesRef(scratch.bytes, prefix.length, scratch.length - prefix.length).utf8ToString();
+    if (!expectedChecksum.equals(actualChecksum)) {
+      throw new CorruptIndexException("SimpleText checksum failure: " + actualChecksum + " != " + expectedChecksum + " (resource=" + input + ")");
+    }
+    if (input.length() != input.getFilePointer()) {
+      throw new CorruptIndexException("Unexpected stuff at the end of file, please be careful with your text editor! (resource=" + input + ")");
+    }
+  }
 }

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java Fri Apr  4 10:27:05 2014
@@ -131,6 +131,11 @@ public class BlockTreeTermsReader extend
       if (indexVersion != version) {
         throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion);
       }
+      
+      // verify
+      if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) {
+        CodecUtil.checksumEntireFile(indexIn);
+      }
 
       // Have PostingsReader init itself
       postingsReader.init(in);
@@ -157,7 +162,7 @@ public class BlockTreeTermsReader extend
         final long sumTotalTermFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY ? -1 : in.readVLong();
         final long sumDocFreq = in.readVLong();
         final int docCount = in.readVInt();
-        final int longsSize = version >= BlockTreeTermsWriter.TERMS_VERSION_META_ARRAY ? in.readVInt() : 0;
+        final int longsSize = version >= BlockTreeTermsWriter.VERSION_META_ARRAY ? in.readVInt() : 0;
         if (docCount < 0 || docCount > info.getDocCount()) { // #docs with field must be <= #docs
           throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + info.getDocCount() + " (resource=" + in + ")");
         }
@@ -187,9 +192,9 @@ public class BlockTreeTermsReader extend
   /** Reads terms file header. */
   private int readHeader(IndexInput input) throws IOException {
     int version = CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_CODEC_NAME,
-                          BlockTreeTermsWriter.TERMS_VERSION_START,
-                          BlockTreeTermsWriter.TERMS_VERSION_CURRENT);
-    if (version < BlockTreeTermsWriter.TERMS_VERSION_APPEND_ONLY) {
+                          BlockTreeTermsWriter.VERSION_START,
+                          BlockTreeTermsWriter.VERSION_CURRENT);
+    if (version < BlockTreeTermsWriter.VERSION_APPEND_ONLY) {
       dirOffset = input.readLong();
     }
     return version;
@@ -198,9 +203,9 @@ public class BlockTreeTermsReader extend
   /** Reads index file header. */
   private int readIndexHeader(IndexInput input) throws IOException {
     int version = CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_INDEX_CODEC_NAME,
-                          BlockTreeTermsWriter.TERMS_INDEX_VERSION_START,
-                          BlockTreeTermsWriter.TERMS_INDEX_VERSION_CURRENT);
-    if (version < BlockTreeTermsWriter.TERMS_INDEX_VERSION_APPEND_ONLY) {
+                          BlockTreeTermsWriter.VERSION_START,
+                          BlockTreeTermsWriter.VERSION_CURRENT);
+    if (version < BlockTreeTermsWriter.VERSION_APPEND_ONLY) {
       indexDirOffset = input.readLong(); 
     }
     return version;
@@ -209,7 +214,10 @@ public class BlockTreeTermsReader extend
   /** Seek {@code input} to the directory offset. */
   private void seekDir(IndexInput input, long dirOffset)
       throws IOException {
-    if (version >= BlockTreeTermsWriter.TERMS_INDEX_VERSION_APPEND_ONLY) {
+    if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) {
+      input.seek(input.length() - CodecUtil.footerLength() - 8);
+      dirOffset = input.readLong();
+    } else if (version >= BlockTreeTermsWriter.VERSION_APPEND_ONLY) {
       input.seek(input.length() - 8);
       dirOffset = input.readLong();
     }
@@ -391,7 +399,7 @@ public class BlockTreeTermsReader extend
       final ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
       PrintStream out;
       try {
-        out = new PrintStream(bos, false, "UTF-8");
+        out = new PrintStream(bos, false, IOUtils.UTF_8);
       } catch (UnsupportedEncodingException bogus) {
         throw new RuntimeException(bogus);
       }
@@ -428,7 +436,7 @@ public class BlockTreeTermsReader extend
       }
 
       try {
-        return bos.toString("UTF-8");
+        return bos.toString(IOUtils.UTF_8);
       } catch (UnsupportedEncodingException bogus) {
         throw new RuntimeException(bogus);
       }
@@ -2977,4 +2985,15 @@ public class BlockTreeTermsReader extend
     }
     return sizeInByes;
   }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) {      
+      // term dictionary
+      CodecUtil.checksumEntireFile(in);
+      
+      // postings
+      postingsReader.checkIntegrity();
+    }
+  }
 }

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java Fri Apr  4 10:27:05 2014
@@ -109,7 +109,7 @@ import org.apache.lucene.util.packed.Pac
  *
  * <ul>
  *    <li>TermsDict (.tim) --&gt; Header, <i>PostingsHeader</i>, NodeBlock<sup>NumBlocks</sup>,
- *                               FieldSummary, DirOffset</li>
+ *                               FieldSummary, DirOffset, Footer</li>
  *    <li>NodeBlock --&gt; (OuterNode | InnerNode)</li>
  *    <li>OuterNode --&gt; EntryCount, SuffixLength, Byte<sup>SuffixLength</sup>, StatsLength, &lt; TermStats &gt;<sup>EntryCount</sup>, MetaLength, &lt;<i>TermMetadata</i>&gt;<sup>EntryCount</sup></li>
  *    <li>InnerNode --&gt; EntryCount, SuffixLength[,Sub?], Byte<sup>SuffixLength</sup>, StatsLength, &lt; TermStats ? &gt;<sup>EntryCount</sup>, MetaLength, &lt;<i>TermMetadata ? </i>&gt;<sup>EntryCount</sup></li>
@@ -122,6 +122,7 @@ import org.apache.lucene.util.packed.Pac
  *        FieldNumber,RootCodeLength,DocCount --&gt; {@link DataOutput#writeVInt VInt}</li>
  *    <li>TotalTermFreq,NumTerms,SumTotalTermFreq,SumDocFreq --&gt; 
  *        {@link DataOutput#writeVLong VLong}</li>
+ *    <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
  * </ul>
  * <p>Notes:</p>
  * <ul>
@@ -150,12 +151,13 @@ import org.apache.lucene.util.packed.Pac
  * when a given term cannot exist on disk (in the .tim file), saving a disk seek.</p>
  * <ul>
  *   <li>TermsIndex (.tip) --&gt; Header, FSTIndex<sup>NumFields</sup>
- *                                &lt;IndexStartFP&gt;<sup>NumFields</sup>, DirOffset</li>
+ *                                &lt;IndexStartFP&gt;<sup>NumFields</sup>, DirOffset, Footer</li>
  *   <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
  *   <li>DirOffset --&gt; {@link DataOutput#writeLong Uint64}</li>
  *   <li>IndexStartFP --&gt; {@link DataOutput#writeVLong VLong}</li>
  *   <!-- TODO: better describe FST output here -->
  *   <li>FSTIndex --&gt; {@link FST FST&lt;byte[]&gt;}</li>
+ *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
  * </ul>
  * <p>Notes:</p>
  * <ul>
@@ -178,7 +180,6 @@ import org.apache.lucene.util.packed.Pac
  * @see BlockTreeTermsReader
  * @lucene.experimental
  */
-
 public class BlockTreeTermsWriter extends FieldsConsumer implements Closeable {
 
   /** Suggested default value for the {@code
@@ -204,33 +205,24 @@ public class BlockTreeTermsWriter extend
   final static String TERMS_CODEC_NAME = "BLOCK_TREE_TERMS_DICT";
 
   /** Initial terms format. */
-  public static final int TERMS_VERSION_START = 0;
+  public static final int VERSION_START = 0;
   
   /** Append-only */
-  public static final int TERMS_VERSION_APPEND_ONLY = 1;
+  public static final int VERSION_APPEND_ONLY = 1;
 
   /** Meta data as array */
-  public static final int TERMS_VERSION_META_ARRAY = 2;
+  public static final int VERSION_META_ARRAY = 2;
+  
+  /** checksums */
+  public static final int VERSION_CHECKSUM = 3;
 
   /** Current terms format. */
-  public static final int TERMS_VERSION_CURRENT = TERMS_VERSION_META_ARRAY;
+  public static final int VERSION_CURRENT = VERSION_CHECKSUM;
 
   /** Extension of terms index file */
   static final String TERMS_INDEX_EXTENSION = "tip";
   final static String TERMS_INDEX_CODEC_NAME = "BLOCK_TREE_TERMS_INDEX";
 
-  /** Initial index format. */
-  public static final int TERMS_INDEX_VERSION_START = 0;
-  
-  /** Append-only */
-  public static final int TERMS_INDEX_VERSION_APPEND_ONLY = 1;
-
-  /** Meta data as array */
-  public static final int TERMS_INDEX_VERSION_META_ARRAY = 2;
-
-  /** Current index format. */
-  public static final int TERMS_INDEX_VERSION_CURRENT = TERMS_INDEX_VERSION_META_ARRAY;
-
   private final IndexOutput out;
   private final IndexOutput indexOut;
   final int maxDoc;
@@ -326,12 +318,12 @@ public class BlockTreeTermsWriter extend
 
   /** Writes the terms file header. */
   private void writeHeader(IndexOutput out) throws IOException {
-    CodecUtil.writeHeader(out, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT);   
+    CodecUtil.writeHeader(out, TERMS_CODEC_NAME, VERSION_CURRENT);   
   }
 
   /** Writes the index file header. */
   private void writeIndexHeader(IndexOutput out) throws IOException {
-    CodecUtil.writeHeader(out, TERMS_INDEX_CODEC_NAME, TERMS_INDEX_VERSION_CURRENT); 
+    CodecUtil.writeHeader(out, TERMS_INDEX_CODEC_NAME, VERSION_CURRENT); 
   }
 
   /** Writes the terms file trailer. */
@@ -1139,13 +1131,13 @@ public class BlockTreeTermsWriter extend
         }
         out.writeVLong(field.sumDocFreq);
         out.writeVInt(field.docCount);
-        if (TERMS_VERSION_CURRENT >= TERMS_VERSION_META_ARRAY) {
-          out.writeVInt(field.longsSize);
-        }
+        out.writeVInt(field.longsSize);
         indexOut.writeVLong(field.indexStartFP);
       }
       writeTrailer(out, dirStart);
+      CodecUtil.writeFooter(out);
       writeIndexTrailer(indexOut, indexDirStart);
+      CodecUtil.writeFooter(indexOut);
     } catch (IOException ioe2) {
       ioe = ioe2;
     } finally {

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java Fri Apr  4 10:27:05 2014
@@ -23,8 +23,12 @@ import java.io.IOException;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexFormatTooNewException;
 import org.apache.lucene.index.IndexFormatTooOldException;
+import org.apache.lucene.store.BufferedChecksumIndexInput;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -43,6 +47,10 @@ public final class CodecUtil {
    * Constant to identify the start of a codec header.
    */
   public final static int CODEC_MAGIC = 0x3fd76c17;
+  /**
+   * Constant to identify the start of a codec footer.
+   */
+  public final static int FOOTER_MAGIC = ~CODEC_MAGIC;
 
   /**
    * Writes a codec header, which records both a string to
@@ -150,4 +158,119 @@ public final class CodecUtil {
 
     return actualVersion;
   }
+  
+  /**
+   * Writes a codec footer, which records both a checksum
+   * algorithm ID and a checksum. This footer can
+   * be parsed and validated with 
+   * {@link #checkFooter(ChecksumIndexInput) checkFooter()}.
+   * <p>
+   * CodecFooter --&gt; Magic,AlgorithmID,Checksum
+   * <ul>
+   *    <li>Magic --&gt; {@link DataOutput#writeInt Uint32}. This
+   *        identifies the start of the footer. It is always {@value #FOOTER_MAGIC}.
+   *    <li>AlgorithmID --&gt; {@link DataOutput#writeInt Uint32}. This
+   *        indicates the checksum algorithm used. Currently this is always 0,
+   *        for zlib-crc32.
+   *    <li>Checksum --&gt; {@link DataOutput#writeLong Uint32}. The
+   *        actual checksum value for all previous bytes in the stream, including
+   *        the bytes from Magic and AlgorithmID.
+   * </ul>
+   * 
+   * @param out Output stream
+   * @throws IOException If there is an I/O error writing to the underlying medium.
+   */
+  public static void writeFooter(IndexOutput out) throws IOException {
+    out.writeInt(FOOTER_MAGIC);
+    out.writeInt(0);
+    out.writeLong(out.getChecksum());
+  }
+  
+  /**
+   * Computes the length of a codec footer.
+   * 
+   * @return length of the entire codec footer.
+   * @see #writeFooter(IndexOutput)
+   */
+  public static int footerLength() {
+    return 16;
+  }
+  
+  /** 
+   * Validates the codec footer previously written by {@link #writeFooter}. 
+   * @return actual checksum value
+   * @throws IOException if the footer is invalid, if the checksum does not match, 
+   *                     or if {@code in} is not properly positioned before the footer
+   *                     at the end of the stream.
+   */
+  public static long checkFooter(ChecksumIndexInput in) throws IOException {
+    validateFooter(in);
+    long actualChecksum = in.getChecksum();
+    long expectedChecksum = in.readLong();
+    if (expectedChecksum != actualChecksum) {
+      throw new CorruptIndexException("checksum failed (hardware problem?) : expected=" + Long.toHexString(expectedChecksum) +  
+                                                       " actual=" + Long.toHexString(actualChecksum) +
+                                                       " (resource=" + in + ")");
+    }
+    if (in.getFilePointer() != in.length()) {
+      throw new CorruptIndexException("did not read all bytes from file: read " + in.getFilePointer() + " vs size " + in.length() + " (resource: " + in + ")");
+    }
+    return actualChecksum;
+  }
+  
+  /** 
+   * Returns (but does not validate) the checksum previously written by {@link #checkFooter}.
+   * @return actual checksum value
+   * @throws IOException if the footer is invalid
+   */
+  public static long retrieveChecksum(IndexInput in) throws IOException {
+    in.seek(in.length() - footerLength());
+    validateFooter(in);
+    return in.readLong();
+  }
+  
+  private static void validateFooter(IndexInput in) throws IOException {
+    final int magic = in.readInt();
+    if (magic != FOOTER_MAGIC) {
+      throw new CorruptIndexException("codec footer mismatch: actual footer=" + magic + " vs expected footer=" + FOOTER_MAGIC + " (resource: " + in + ")");
+    }
+    
+    final int algorithmID = in.readInt();
+    if (algorithmID != 0) {
+      throw new CorruptIndexException("codec footer mismatch: unknown algorithmID: " + algorithmID);
+    }
+  }
+  
+  /**
+   * Checks that the stream is positioned at the end, and throws exception
+   * if it is not. 
+   * @deprecated Use {@link #checkFooter} instead, this should only used for files without checksums 
+   */
+  @Deprecated
+  public static void checkEOF(IndexInput in) throws IOException {
+    if (in.getFilePointer() != in.length()) {
+      throw new CorruptIndexException("did not read all bytes from file: read " + in.getFilePointer() + " vs size " + in.length() + " (resource: " + in + ")");
+    }
+  }
+  
+  /** 
+   * Clones the provided input, reads all bytes from the file, and calls {@link #checkFooter} 
+   * <p>
+   * Note that this method may be slow, as it must process the entire file.
+   * If you just need to extract the checksum value, call {@link #retrieveChecksum}.
+   */
+  public static long checksumEntireFile(IndexInput input) throws IOException {
+    IndexInput clone = input.clone();
+    clone.seek(0);
+    ChecksumIndexInput in = new BufferedChecksumIndexInput(clone);
+    assert in.getFilePointer() == 0;
+    final byte[] buffer = new byte[1024];
+    long bytesToRead = in.length() - footerLength();
+    for (long skipped = 0; skipped < bytesToRead; ) {
+      final int toRead = (int) Math.min(bytesToRead - skipped, buffer.length);
+      in.readBytes(buffer, 0, toRead);
+      skipped += toRead;
+    }
+    return checkFooter(in);
+  }
 }

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java Fri Apr  4 10:27:05 2014
@@ -68,6 +68,15 @@ public abstract class DocValuesProducer 
   public abstract long ramBytesUsed();
   
   /** 
+   * Checks consistency of this producer
+   * <p>
+   * Note that this may be costly in terms of I/O, e.g. 
+   * may involve computing a checksum value against large data files.
+   * @lucene.internal
+   */
+  public abstract void checkIntegrity() throws IOException;
+  
+  /** 
    * A simple implementation of {@link DocValuesProducer#getDocsWithField} that 
    * returns {@code true} if a document has an ordinal &gt;= 0
    * <p>

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/FieldsProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/FieldsProducer.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/FieldsProducer.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/FieldsProducer.java Fri Apr  4 10:27:05 2014
@@ -39,4 +39,13 @@ public abstract class FieldsProducer ext
   
   /** Returns approximate RAM bytes used */
   public abstract long ramBytesUsed();
+  
+  /** 
+   * Checks consistency of this reader.
+   * <p>
+   * Note that this may be costly in terms of I/O, e.g. 
+   * may involve computing a checksum value against large data files.
+   * @lucene.internal
+   */
+  public abstract void checkIntegrity() throws IOException;
 }

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java Fri Apr  4 10:27:05 2014
@@ -72,6 +72,15 @@ public abstract class PostingsReaderBase
   /** Returns approximate RAM bytes used */
   public abstract long ramBytesUsed();
   
+  /** 
+   * Checks consistency of this reader.
+   * <p>
+   * Note that this may be costly in terms of I/O, e.g. 
+   * may involve computing a checksum value against large data files.
+   * @lucene.internal
+   */
+  public abstract void checkIntegrity() throws IOException;
+  
   @Override
   public abstract void close() throws IOException;
 }

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsReader.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsReader.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsReader.java Fri Apr  4 10:27:05 2014
@@ -43,4 +43,13 @@ public abstract class StoredFieldsReader
   
   /** Returns approximate RAM bytes used */
   public abstract long ramBytesUsed();
+  
+  /** 
+   * Checks consistency of this reader.
+   * <p>
+   * Note that this may be costly in terms of I/O, e.g. 
+   * may involve computing a checksum value against large data files.
+   * @lucene.internal
+   */
+  public abstract void checkIntegrity() throws IOException;
 }

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java Fri Apr  4 10:27:05 2014
@@ -45,6 +45,15 @@ public abstract class TermVectorsReader 
   /** Returns approximate RAM bytes used */
   public abstract long ramBytesUsed();
   
+  /** 
+   * Checks consistency of this reader.
+   * <p>
+   * Note that this may be costly in terms of I/O, e.g. 
+   * may involve computing a checksum value against large data files.
+   * @lucene.internal
+   */
+  public abstract void checkIntegrity() throws IOException;
+  
   /** Create a clone that one caller at a time may use to
    *  read term vectors. */
   @Override

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexWriter.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexWriter.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexWriter.java Fri Apr  4 10:27:05 2014
@@ -21,6 +21,7 @@ import java.io.Closeable;
 import java.io.IOException;
 
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.packed.PackedInts;
@@ -52,6 +53,7 @@ import org.apache.lucene.util.packed.Pac
  * <li>AvgChunkSize --&gt; the average size of a chunk of compressed documents, as a {@link DataOutput#writeVLong VLong}</li>
  * <li>BitsPerStartPointerDelta --&gt; number of bits required to represent a delta from the average using <a href="https://developers.google.com/protocol-buffers/docs/encoding#types">ZigZag encoding</a></li>
  * <li>StartPointerDeltas --&gt; {@link PackedInts packed} array of BlockChunks elements of BitsPerStartPointerDelta bits each, representing the deltas from the average start pointer using <a href="https://developers.google.com/protocol-buffers/docs/encoding#types">ZigZag encoding</a></li>
+ * <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
  * </ul>
  * <p>Notes</p>
  * <ul>
@@ -198,6 +200,7 @@ public final class CompressingStoredFiel
       writeBlock();
     }
     fieldsIndexOut.writeVInt(0); // end marker
+    CodecUtil.writeFooter(fieldsIndexOut);
   }
 
   @Override

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java Fri Apr  4 10:27:05 2014
@@ -28,6 +28,7 @@ import static org.apache.lucene.codecs.c
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.TYPE_BITS;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.TYPE_MASK;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.VERSION_BIG_CHUNKS;
+import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.VERSION_CHECKSUM;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.VERSION_CURRENT;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.VERSION_START;
 import static org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsWriter.FIELDS_EXTENSION;
@@ -35,6 +36,7 @@ import static org.apache.lucene.codecs.l
 
 import java.io.EOFException;
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 
 import org.apache.lucene.codecs.CodecUtil;
@@ -47,6 +49,7 @@ import org.apache.lucene.index.SegmentIn
 import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
@@ -113,17 +116,20 @@ public final class CompressingStoredFiel
     boolean success = false;
     fieldInfos = fn;
     numDocs = si.getDocCount();
-    IndexInput indexStream = null;
+    ChecksumIndexInput indexStream = null;
     try {
       // Load the index into memory
       final String indexStreamFN = IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_INDEX_EXTENSION);
-      indexStream = d.openInput(indexStreamFN, context);
+      indexStream = d.openChecksumInput(indexStreamFN, context);
       final String codecNameIdx = formatName + CODEC_SFX_IDX;
       version = CodecUtil.checkHeader(indexStream, codecNameIdx, VERSION_START, VERSION_CURRENT);
       assert CodecUtil.headerLength(codecNameIdx) == indexStream.getFilePointer();
       indexReader = new CompressingStoredFieldsIndexReader(indexStream, si);
-      if (indexStream.getFilePointer() != indexStream.length()) {
-        throw new CorruptIndexException("did not read all bytes from file \"" + indexStreamFN + "\": read " + indexStream.getFilePointer() + " vs size " + indexStream.length() + " (resource: " + indexStream + ")");
+      
+      if (version >= VERSION_CHECKSUM) {
+        CodecUtil.checkFooter(indexStream);
+      } else {
+        CodecUtil.checkEOF(indexStream);
       }
       indexStream.close();
       indexStream = null;
@@ -187,7 +193,7 @@ public final class CompressingStoredFiel
         length = in.readVInt();
         data = new byte[length];
         in.readBytes(data, 0, length);
-        visitor.stringField(info, new String(data, IOUtils.CHARSET_UTF_8));
+        visitor.stringField(info, new String(data, StandardCharsets.UTF_8));
         break;
       case NUMERIC_INT:
         visitor.intField(info, in.readInt());
@@ -509,4 +515,11 @@ public final class CompressingStoredFiel
     return indexReader.ramBytesUsed();
   }
 
+  @Override
+  public void checkIntegrity() throws IOException {
+    if (version >= VERSION_CHECKSUM) {
+      CodecUtil.checksumEntireFile(fieldsStream);
+    }
+  }
+
 }

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java Fri Apr  4 10:27:05 2014
@@ -71,7 +71,8 @@ public final class CompressingStoredFiel
   static final String CODEC_SFX_DAT = "Data";
   static final int VERSION_START = 0;
   static final int VERSION_BIG_CHUNKS = 1;
-  static final int VERSION_CURRENT = VERSION_BIG_CHUNKS;
+  static final int VERSION_CHECKSUM = 2;
+  static final int VERSION_CURRENT = VERSION_CHECKSUM;
 
   private final Directory directory;
   private final String segment;
@@ -106,9 +107,11 @@ public final class CompressingStoredFiel
     this.numBufferedDocs = 0;
 
     boolean success = false;
-    IndexOutput indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_INDEX_EXTENSION), context);
+    IndexOutput indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_INDEX_EXTENSION), 
+                                                                     context);
     try {
-      fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_EXTENSION), context);
+      fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_EXTENSION),
+                                                    context);
 
       final String codecNameIdx = formatName + CODEC_SFX_IDX;
       final String codecNameDat = formatName + CODEC_SFX_DAT;
@@ -314,6 +317,7 @@ public final class CompressingStoredFiel
       throw new RuntimeException("Wrote " + docBase + " docs, finish called with numDocs=" + numDocs);
     }
     indexWriter.finish(numDocs);
+    CodecUtil.writeFooter(fieldsStream);
     assert bufferedDocs.length == 0;
   }
 

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java Fri Apr  4 10:27:05 2014
@@ -28,6 +28,7 @@ import static org.apache.lucene.codecs.c
 import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VECTORS_INDEX_EXTENSION;
 import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VERSION_CURRENT;
 import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VERSION_START;
+import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VERSION_CHECKSUM;
 
 import java.io.Closeable;
 import java.io.IOException;
@@ -48,6 +49,7 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -69,6 +71,7 @@ public final class CompressingTermVector
   private final FieldInfos fieldInfos;
   final CompressingStoredFieldsIndexReader indexReader;
   final IndexInput vectorsStream;
+  private final int version;
   private final int packedIntsVersion;
   private final CompressionMode compressionMode;
   private final Decompressor decompressor;
@@ -88,6 +91,7 @@ public final class CompressingTermVector
     this.chunkSize = reader.chunkSize;
     this.numDocs = reader.numDocs;
     this.reader = new BlockPackedReaderIterator(vectorsStream, packedIntsVersion, BLOCK_SIZE, 0);
+    this.version = reader.version;
     this.closed = false;
   }
 
@@ -99,17 +103,20 @@ public final class CompressingTermVector
     boolean success = false;
     fieldInfos = fn;
     numDocs = si.getDocCount();
-    IndexInput indexStream = null;
+    ChecksumIndexInput indexStream = null;
     try {
       // Load the index into memory
       final String indexStreamFN = IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_INDEX_EXTENSION);
-      indexStream = d.openInput(indexStreamFN, context);
+      indexStream = d.openChecksumInput(indexStreamFN, context);
       final String codecNameIdx = formatName + CODEC_SFX_IDX;
-      int version = CodecUtil.checkHeader(indexStream, codecNameIdx, VERSION_START, VERSION_CURRENT);
+      version = CodecUtil.checkHeader(indexStream, codecNameIdx, VERSION_START, VERSION_CURRENT);
       assert CodecUtil.headerLength(codecNameIdx) == indexStream.getFilePointer();
       indexReader = new CompressingStoredFieldsIndexReader(indexStream, si);
-      if (indexStream.getFilePointer() != indexStream.length()) {
-        throw new CorruptIndexException("did not read all bytes from file \"" + indexStreamFN + "\": read " + indexStream.getFilePointer() + " vs size " + indexStream.length() + " (resource: " + indexStream + ")");
+      
+      if (version >= VERSION_CHECKSUM) {
+        CodecUtil.checkFooter(indexStream);
+      } else {
+        CodecUtil.checkEOF(indexStream);
       }
       indexStream.close();
       indexStream = null;
@@ -1045,5 +1052,12 @@ public final class CompressingTermVector
   public long ramBytesUsed() {
     return indexReader.ramBytesUsed();
   }
+  
+  @Override
+  public void checkIntegrity() throws IOException {
+    if (version >= VERSION_CHECKSUM) {
+      CodecUtil.checksumEntireFile(vectorsStream);
+    }
+  }
 
 }

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java Fri Apr  4 10:27:05 2014
@@ -66,7 +66,8 @@ public final class CompressingTermVector
   static final String CODEC_SFX_DAT = "Data";
 
   static final int VERSION_START = 0;
-  static final int VERSION_CURRENT = VERSION_START;
+  static final int VERSION_CHECKSUM = 1;
+  static final int VERSION_CURRENT = VERSION_CHECKSUM;
 
   static final int BLOCK_SIZE = 64;
 
@@ -220,9 +221,11 @@ public final class CompressingTermVector
     lastTerm = new BytesRef(ArrayUtil.oversize(30, 1));
 
     boolean success = false;
-    IndexOutput indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_INDEX_EXTENSION), context);
+    IndexOutput indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_INDEX_EXTENSION), 
+                                                                     context);
     try {
-      vectorsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_EXTENSION), context);
+      vectorsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_EXTENSION),
+                                                     context);
 
       final String codecNameIdx = formatName + CODEC_SFX_IDX;
       final String codecNameDat = formatName + CODEC_SFX_DAT;
@@ -659,6 +662,7 @@ public final class CompressingTermVector
       throw new RuntimeException("Wrote " + this.numDocs + " docs, finish called with numDocs=" + numDocs);
     }
     indexWriter.finish(numDocs);
+    CodecUtil.writeFooter(vectorsStream);
   }
 
   @Override

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/lucene40/BitVector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/lucene40/BitVector.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/lucene40/BitVector.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/codecs/lucene40/BitVector.java Fri Apr  4 10:27:05 2014
@@ -21,6 +21,8 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.IndexFormatTooOldException;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -198,9 +200,12 @@ final class BitVector implements Cloneab
   // Changed DGaps to encode gaps between cleared bits, not
   // set:
   public final static int VERSION_DGAPS_CLEARED = 1;
+  
+  // added checksum
+  public final static int VERSION_CHECKSUM = 2;
 
   // Increment version to change it:
-  public final static int VERSION_CURRENT = VERSION_DGAPS_CLEARED;
+  public final static int VERSION_CURRENT = VERSION_CHECKSUM;
 
   public int getVersion() {
     return version;
@@ -221,6 +226,7 @@ final class BitVector implements Cloneab
       } else {
         writeBits(output);
       }
+      CodecUtil.writeFooter(output);
       assert verifyCount();
     } finally {
       IOUtils.close(output);
@@ -324,7 +330,7 @@ final class BitVector implements Cloneab
     <code>d</code>, as written by the {@link #write} method.
     */
   public BitVector(Directory d, String name, IOContext context) throws IOException {
-    IndexInput input = d.openInput(name, context);
+    ChecksumIndexInput input = d.openChecksumInput(name, context);
 
     try {
       final int firstInt = input.readInt();
@@ -334,8 +340,8 @@ final class BitVector implements Cloneab
         version = CodecUtil.checkHeader(input, CODEC, VERSION_START, VERSION_CURRENT);
         size = input.readInt();
       } else {
-        version = VERSION_PRE;
-        size = firstInt;
+        // we started writing full header well before 4.0
+        throw new IndexFormatTooOldException(input.toString(), Integer.toString(firstInt));
       }
       if (size == -1) {
         if (version >= VERSION_DGAPS_CLEARED) {
@@ -351,6 +357,11 @@ final class BitVector implements Cloneab
         invertAll();
       }
 
+      if (version >= VERSION_CHECKSUM) {
+        CodecUtil.checkFooter(input);
+      } else {
+        CodecUtil.checkEOF(input);
+      }
       assert verifyCount();
     } finally {
       input.close();