You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/10/22 06:21:05 UTC

svn commit: r1633514 [2/3] - in /lucene/dev/branches/lucene5969/lucene: backward-codecs/src/java/org/apache/lucene/codecs/lucene40/ backward-codecs/src/java/org/apache/lucene/codecs/lucene46/ backward-codecs/src/test/org/apache/lucene/index/ benchmark/...

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java Wed Oct 22 04:21:03 2014
@@ -93,7 +93,7 @@ class Lucene50NormsProducer extends Norm
     try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
       Throwable priorE = null;
       try {
-        version = CodecUtil.checkSegmentHeader(in, metaCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+        version = CodecUtil.checkIndexHeader(in, metaCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
         readFields(in, state.fieldInfos);
       } catch (Throwable exception) {
         priorE = exception;
@@ -106,7 +106,7 @@ class Lucene50NormsProducer extends Norm
     this.data = state.directory.openInput(dataName, state.context);
     boolean success = false;
     try {
-      final int version2 = CodecUtil.checkSegmentHeader(data, dataCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+      final int version2 = CodecUtil.checkIndexHeader(data, dataCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
       if (version != version2) {
         throw new CorruptIndexException("Format versions mismatch: meta=" + version + ",data=" + version2, data);
       }

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java Wed Oct 22 04:21:03 2014
@@ -128,14 +128,14 @@ import org.apache.lucene.util.packed.Pac
  *   <li>PostingsHeader --&gt; Header, PackedBlockSize</li>
  *   <li>TermMetadata --&gt; (DocFPDelta|SingletonDocID), PosFPDelta?, PosVIntBlockFPDelta?, PayFPDelta?, 
  *                            SkipFPDelta?</li>
- *   <li>Header, --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
+ *   <li>Header, --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
  *   <li>PackedBlockSize, SingletonDocID --&gt; {@link DataOutput#writeVInt VInt}</li>
  *   <li>DocFPDelta, PosFPDelta, PayFPDelta, PosVIntBlockFPDelta, SkipFPDelta --&gt; {@link DataOutput#writeVLong VLong}</li>
  *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
  * </ul>
  * <p>Notes:</p>
  * <ul>
- *    <li>Header is a {@link CodecUtil#writeSegmentHeader SegmentHeader} storing the version information
+ *    <li>Header is a {@link CodecUtil#writeIndexHeader IndexHeader} storing the version information
  *        for the postings.</li>
  *    <li>PackedBlockSize is the fixed block size for packed blocks. In packed block, bit width is 
  *        determined by the largest integer. Smaller block size result in smaller variance among width 
@@ -191,7 +191,7 @@ import org.apache.lucene.util.packed.Pac
  *
  * <ul>
  *   <li>docFile(.doc) --&gt; Header, &lt;TermFreqs, SkipData?&gt;<sup>TermCount</sup>, Footer</li>
- *   <li>Header --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
+ *   <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
  *   <li>TermFreqs --&gt; &lt;PackedBlock&gt; <sup>PackedDocBlockNum</sup>,  
  *                        VIntBlock? </li>
  *   <li>PackedBlock --&gt; PackedDocDeltaBlock, PackedFreqBlock?
@@ -275,7 +275,7 @@ import org.apache.lucene.util.packed.Pac
  *    sometimes stores part of payloads and offsets for speedup.</p>
  * <ul>
  *   <li>PosFile(.pos) --&gt; Header, &lt;TermPositions&gt; <sup>TermCount</sup>, Footer</li>
- *   <li>Header --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
+ *   <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
  *   <li>TermPositions --&gt; &lt;PackedPosDeltaBlock&gt; <sup>PackedPosBlockNum</sup>,  
  *                            VIntBlock? </li>
  *   <li>VIntBlock --&gt; &lt;PositionDelta[, PayloadLength?], PayloadData?, 
@@ -328,7 +328,7 @@ import org.apache.lucene.util.packed.Pac
  *    Some payloads and offsets will be separated out into .pos file, for performance reasons.</p>
  * <ul>
  *   <li>PayFile(.pay): --&gt; Header, &lt;TermPayloads, TermOffsets?&gt; <sup>TermCount</sup>, Footer</li>
- *   <li>Header --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
+ *   <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
  *   <li>TermPayloads --&gt; &lt;PackedPayLengthBlock, SumPayLength, PayData&gt; <sup>PackedPayBlockNum</sup>
  *   <li>TermOffsets --&gt; &lt;PackedOffsetStartDeltaBlock, PackedOffsetLengthBlock&gt; <sup>PackedPayBlockNum</sup>
  *   <li>PackedPayLengthBlock, PackedOffsetStartDeltaBlock, PackedOffsetLengthBlock --&gt; {@link PackedInts PackedInts}</li>

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java Wed Oct 22 04:21:03 2014
@@ -83,20 +83,20 @@ public final class Lucene50PostingsReade
     String docName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene50PostingsFormat.DOC_EXTENSION);
     try {
       docIn = state.directory.openInput(docName, state.context);
-      version = CodecUtil.checkSegmentHeader(docIn, DOC_CODEC, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+      version = CodecUtil.checkIndexHeader(docIn, DOC_CODEC, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
       forUtil = new ForUtil(docIn);
       CodecUtil.retrieveChecksum(docIn);
 
       if (state.fieldInfos.hasProx()) {
         String proxName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene50PostingsFormat.POS_EXTENSION);
         posIn = state.directory.openInput(proxName, state.context);
-        CodecUtil.checkSegmentHeader(posIn, POS_CODEC, version, version, state.segmentInfo.getId(), state.segmentSuffix);
+        CodecUtil.checkIndexHeader(posIn, POS_CODEC, version, version, state.segmentInfo.getId(), state.segmentSuffix);
         CodecUtil.retrieveChecksum(posIn);
 
         if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) {
           String payName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene50PostingsFormat.PAY_EXTENSION);
           payIn = state.directory.openInput(payName, state.context);
-          CodecUtil.checkSegmentHeader(payIn, PAY_CODEC, version, version, state.segmentInfo.getId(), state.segmentSuffix);
+          CodecUtil.checkIndexHeader(payIn, PAY_CODEC, version, version, state.segmentInfo.getId(), state.segmentSuffix);
           CodecUtil.retrieveChecksum(payIn);
         }
       }
@@ -115,7 +115,7 @@ public final class Lucene50PostingsReade
   @Override
   public void init(IndexInput termsIn, SegmentReadState state) throws IOException {
     // Make sure we are talking to the matching postings writer
-    CodecUtil.checkSegmentHeader(termsIn, TERMS_CODEC, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+    CodecUtil.checkIndexHeader(termsIn, TERMS_CODEC, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
     final int indexBlockSize = termsIn.readVInt();
     if (indexBlockSize != BLOCK_SIZE) {
       throw new IllegalStateException("index-time BLOCK_SIZE (" + indexBlockSize + ") != read-time BLOCK_SIZE (" + BLOCK_SIZE + ")");

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsWriter.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsWriter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsWriter.java Wed Oct 22 04:21:03 2014
@@ -107,14 +107,14 @@ public final class Lucene50PostingsWrite
     IndexOutput payOut = null;
     boolean success = false;
     try {
-      CodecUtil.writeSegmentHeader(docOut, DOC_CODEC, VERSION_CURRENT, 
+      CodecUtil.writeIndexHeader(docOut, DOC_CODEC, VERSION_CURRENT, 
                                    state.segmentInfo.getId(), state.segmentSuffix);
       forUtil = new ForUtil(acceptableOverheadRatio, docOut);
       if (state.fieldInfos.hasProx()) {
         posDeltaBuffer = new int[MAX_DATA_SIZE];
         String posFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene50PostingsFormat.POS_EXTENSION);
         posOut = state.directory.createOutput(posFileName, state.context);
-        CodecUtil.writeSegmentHeader(posOut, POS_CODEC, VERSION_CURRENT,
+        CodecUtil.writeIndexHeader(posOut, POS_CODEC, VERSION_CURRENT,
                                      state.segmentInfo.getId(), state.segmentSuffix);
 
         if (state.fieldInfos.hasPayloads()) {
@@ -136,7 +136,7 @@ public final class Lucene50PostingsWrite
         if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) {
           String payFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene50PostingsFormat.PAY_EXTENSION);
           payOut = state.directory.createOutput(payFileName, state.context);
-          CodecUtil.writeSegmentHeader(payOut, PAY_CODEC, VERSION_CURRENT,
+          CodecUtil.writeIndexHeader(payOut, PAY_CODEC, VERSION_CURRENT,
                                        state.segmentInfo.getId(), state.segmentSuffix);
         }
       } else {
@@ -176,7 +176,7 @@ public final class Lucene50PostingsWrite
 
   @Override
   public void init(IndexOutput termsOut, SegmentWriteState state) throws IOException {
-    CodecUtil.writeSegmentHeader(termsOut, TERMS_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+    CodecUtil.writeIndexHeader(termsOut, TERMS_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
     termsOut.writeVInt(BLOCK_SIZE);
   }
 

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java Wed Oct 22 04:21:03 2014
@@ -48,7 +48,7 @@ import org.apache.lucene.util.Version;
  * Data types:
  * <p>
  * <ul>
- *   <li>Header --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
+ *   <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</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>
@@ -83,21 +83,16 @@ public class Lucene50SegmentInfoFormat e
   }
   
   @Override
-  public SegmentInfo read(Directory dir, String segment, IOContext context) throws IOException {
+  public SegmentInfo read(Directory dir, String segment, byte[] segmentID, 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);
-        byte id[] = new byte[StringHelper.ID_LENGTH];
-        input.readBytes(id, 0, id.length);
-        String suffix = input.readString();
-        if (!suffix.isEmpty()) {
-          throw new CorruptIndexException("invalid codec header: got unexpected suffix: " + suffix, input);
-        }
+        CodecUtil.checkIndexHeader(input, Lucene50SegmentInfoFormat.CODEC_NAME,
+                                          Lucene50SegmentInfoFormat.VERSION_START,
+                                          Lucene50SegmentInfoFormat.VERSION_CURRENT,
+                                          segmentID, "");
         final Version version = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
         
         final int docCount = input.readInt();
@@ -108,7 +103,7 @@ public class Lucene50SegmentInfoFormat e
         final Map<String,String> diagnostics = input.readStringStringMap();
         final Set<String> files = input.readStringSet();
         
-        si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, id);
+        si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID);
         si.setFiles(files);
       } catch (Throwable exception) {
         priorE = exception;
@@ -126,8 +121,7 @@ public class Lucene50SegmentInfoFormat e
 
     boolean success = false;
     try (IndexOutput output = dir.createOutput(fileName, ioContext)) {
-      // NOTE: we encode ID in the segment header, for format consistency with all other per-segment files
-      CodecUtil.writeSegmentHeader(output, 
+      CodecUtil.writeIndexHeader(output, 
                                    Lucene50SegmentInfoFormat.CODEC_NAME, 
                                    Lucene50SegmentInfoFormat.VERSION_CURRENT,
                                    si.getId(),

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java Wed Oct 22 04:21:03 2014
@@ -52,7 +52,7 @@ import org.apache.lucene.util.packed.Pac
  * <p>Here is a more detailed description of the field data file format:</p>
  * <ul>
  * <li>FieldData (.fdt) --&gt; &lt;Header&gt;, PackedIntsVersion, &lt;Chunk&gt;<sup>ChunkCount</sup></li>
- * <li>Header --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
+ * <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
  * <li>PackedIntsVersion --&gt; {@link PackedInts#VERSION_CURRENT} as a {@link DataOutput#writeVInt VInt}</li>
  * <li>ChunkCount is not known in advance and is the number of chunks necessary to store all document of the segment</li>
  * <li>Chunk --&gt; DocBase, ChunkDocs, DocFieldCounts, DocLengths, &lt;CompressedDocs&gt;</li>
@@ -104,7 +104,7 @@ import org.apache.lucene.util.packed.Pac
  * <p>A fields index file (extension <tt>.fdx</tt>).</p>
  * <ul>
  * <li>FieldsIndex (.fdx) --&gt; &lt;Header&gt;, &lt;ChunkIndex&gt;</li>
- * <li>Header --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
+ * <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
  * <li>ChunkIndex: See {@link CompressingStoredFieldsIndexWriter}</li>
  * </ul>
  * </li>

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50TermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50TermVectorsFormat.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50TermVectorsFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50TermVectorsFormat.java Wed Oct 22 04:21:03 2014
@@ -59,7 +59,7 @@ import org.apache.lucene.util.packed.Pac
  * <p>Here is a more detailed description of the field data file format:</p>
  * <ul>
  * <li>VectorData (.tvd) --&gt; &lt;Header&gt;, PackedIntsVersion, ChunkSize, &lt;Chunk&gt;<sup>ChunkCount</sup>, Footer</li>
- * <li>Header --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
+ * <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
  * <li>PackedIntsVersion --&gt; {@link PackedInts#VERSION_CURRENT} as a {@link DataOutput#writeVInt VInt}</li>
  * <li>ChunkSize is the number of bytes of terms to accumulate before flushing, as a {@link DataOutput#writeVInt VInt}</li>
  * <li>ChunkCount is not known in advance and is the number of chunks necessary to store all document of the segment</li>
@@ -113,7 +113,7 @@ import org.apache.lucene.util.packed.Pac
  * <p>An index file (extension <tt>.tvx</tt>).</p>
  * <ul>
  * <li>VectorIndex (.tvx) --&gt; &lt;Header&gt;, &lt;ChunkIndex&gt;, Footer</li>
- * <li>Header --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
+ * <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
  * <li>ChunkIndex: See {@link CompressingStoredFieldsIndexWriter}</li>
  * <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
  * </ul>

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Wed Oct 22 04:21:03 2014
@@ -451,7 +451,7 @@ public class CheckIndex implements Close
   public Status checkIndex(List<String> onlySegments) throws IOException {
     ensureOpen();
     NumberFormat nf = NumberFormat.getInstance(Locale.ROOT);
-    SegmentInfos sis = new SegmentInfos();
+    SegmentInfos sis = null;
     Status result = new Status();
     result.dir = dir;
     String[] files = dir.listAll();
@@ -462,7 +462,7 @@ public class CheckIndex implements Close
     try {
       // Do not use SegmentInfos.read(Directory) since the spooky
       // retrying it does is not necessary here (we hold the write lock):
-      sis.read(dir, lastSegmentsFile);
+      sis = SegmentInfos.readCommit(dir, lastSegmentsFile);
     } catch (Throwable t) {
       if (failFast) {
         IOUtils.reThrow(t);

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java Wed Oct 22 04:21:03 2014
@@ -225,8 +225,7 @@ public abstract class DirectoryReader ex
 
     List<IndexCommit> commits = new ArrayList<>();
 
-    SegmentInfos latest = new SegmentInfos();
-    latest.read(dir);
+    SegmentInfos latest = SegmentInfos.readLatestCommit(dir);
     final long currentGen = latest.getGeneration();
 
     commits.add(new StandardDirectoryReader.ReaderCommit(latest, dir));
@@ -239,11 +238,11 @@ public abstract class DirectoryReader ex
           !fileName.equals(IndexFileNames.OLD_SEGMENTS_GEN) &&
           SegmentInfos.generationFromSegmentsFileName(fileName) < currentGen) {
 
-        SegmentInfos sis = new SegmentInfos();
+        SegmentInfos sis = null;
         try {
           // IOException allowed to throw there, in case
           // segments_N is corrupt
-          sis.read(dir, fileName);
+          sis = SegmentInfos.readCommit(dir, fileName);
         } catch (FileNotFoundException | NoSuchFileException fnfe) {
           // LUCENE-948: on NFS (and maybe others), if
           // you have writers switching back and forth
@@ -252,7 +251,6 @@ public abstract class DirectoryReader ex
           // file segments_X exists when in fact it
           // doesn't.  So, we catch this and handle it
           // as if the file does not exist
-          sis = null;
         }
 
         if (sis != null)

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java Wed Oct 22 04:21:03 2014
@@ -165,9 +165,9 @@ final class IndexFileDeleter implements 
             if (infoStream.isEnabled("IFD")) {
               infoStream.message("IFD", "init: load commit \"" + fileName + "\"");
             }
-            SegmentInfos sis = new SegmentInfos();
+            SegmentInfos sis = null;
             try {
-              sis.read(directory, fileName);
+              sis = SegmentInfos.readCommit(directory, fileName);
             } catch (FileNotFoundException | NoSuchFileException e) {
               // LUCENE-948: on NFS (and maybe others), if
               // you have writers switching back and forth
@@ -179,7 +179,6 @@ final class IndexFileDeleter implements 
               if (infoStream.isEnabled("IFD")) {
                 infoStream.message("IFD", "init: hit FileNotFoundException when loading commit \"" + fileName + "\"; skipping this commit point");
               }
-              sis = null;
             } catch (IOException e) {
               if (SegmentInfos.generationFromSegmentsFileName(fileName) <= currentGen && directory.fileLength(fileName) > 0) {
                 throw e;
@@ -187,7 +186,6 @@ final class IndexFileDeleter implements 
                 // Most likely we are opening an index that
                 // has an aborted "future" commit, so suppress
                 // exc in this case
-                sis = null;
               }
             }
             if (sis != null) {
@@ -215,9 +213,9 @@ final class IndexFileDeleter implements 
       // listing was stale (eg when index accessed via NFS
       // client with stale directory listing cache).  So we
       // try now to explicitly open this commit point:
-      SegmentInfos sis = new SegmentInfos();
+      SegmentInfos sis = null;
       try {
-        sis.read(directory, currentSegmentsFile);
+        sis = SegmentInfos.readCommit(directory, currentSegmentsFile);
       } catch (IOException e) {
         throw new CorruptIndexException("unable to read current segments_N file", currentSegmentsFile, e);
       }

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Wed Oct 22 04:21:03 2014
@@ -773,7 +773,6 @@ public class IndexWriter implements Clos
 
       // If index is too old, reading the segments will throw
       // IndexFormatTooOldException.
-      segmentInfos = new SegmentInfos();
 
       boolean initialIndexExists = true;
 
@@ -782,13 +781,17 @@ public class IndexWriter implements Clos
         // against an index that's currently open for
         // searching.  In this case we write the next
         // segments_N file with no segments:
+        SegmentInfos sis = null;
         try {
-          segmentInfos.read(directory);
-          segmentInfos.clear();
+          sis = SegmentInfos.readLatestCommit(directory);
+          sis.clear();
         } catch (IOException e) {
           // Likely this means it's a fresh directory
           initialIndexExists = false;
+          sis = new SegmentInfos();
         }
+        
+        segmentInfos = sis;
 
         // Record that we have a change (zero out all
         // segments) pending:
@@ -802,7 +805,7 @@ public class IndexWriter implements Clos
 
         // Do not use SegmentInfos.read(Directory) since the spooky
         // retrying it does is not necessary here (we hold the write lock):
-        segmentInfos.read(directory, lastSegmentsFile);
+        segmentInfos = SegmentInfos.readCommit(directory, lastSegmentsFile);
 
         IndexCommit commit = config.getIndexCommit();
         if (commit != null) {
@@ -813,8 +816,7 @@ public class IndexWriter implements Clos
           // points.
           if (commit.getDirectory() != directory)
             throw new IllegalArgumentException("IndexCommit's directory doesn't match my directory");
-          SegmentInfos oldInfos = new SegmentInfos();
-          oldInfos.read(directory, commit.getSegmentsFileName());
+          SegmentInfos oldInfos = SegmentInfos.readCommit(directory, commit.getSegmentsFileName());
           segmentInfos.replace(oldInfos);
           changed();
           if (infoStream.isEnabled("IW")) {
@@ -2401,8 +2403,7 @@ public class IndexWriter implements Clos
           if (infoStream.isEnabled("IW")) {
             infoStream.message("IW", "addIndexes: process directory " + dir);
           }
-          SegmentInfos sis = new SegmentInfos(); // read infos from dir
-          sis.read(dir);
+          SegmentInfos sis = SegmentInfos.readLatestCommit(dir); // read infos from dir
           totalDocCount += sis.totalDocCount();
 
           for (SegmentCommitInfo info : sis) {

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java Wed Oct 22 04:21:03 2014
@@ -67,8 +67,8 @@ import org.apache.lucene.util.StringHelp
  * Data types:
  * <p>
  * <ul>
- * <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
- * <li>GenHeader, NameCounter, SegCount, DeletionCount --&gt;
+ * <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
+ * <li>NameCounter, SegCount, DeletionCount --&gt;
  * {@link DataOutput#writeInt Int32}</li>
  * <li>Generation, Version, DelGen, Checksum, FieldInfosGen, DocValuesGen --&gt;
  * {@link DataOutput#writeLong Int64}</li>
@@ -155,8 +155,8 @@ public final class SegmentInfos implemen
   private byte[] id;
 
   /** Sole constructor. Typically you call this and then
-   *  use {@link #read(Directory) or
-   *  #read(Directory,String)} to populate each {@link
+   *  use {@link #readLatestCommit(Directory) or
+   *  #readCommit(Directory,String)} to populate each {@link
    *  SegmentCommitInfo}.  Alternatively, you can add/remove your
    *  own {@link SegmentCommitInfo}s. */
   public SegmentInfos() {
@@ -245,21 +245,14 @@ public final class SegmentInfos implemen
       throw new IllegalArgumentException("fileName \"" + fileName + "\" is not a segments file");
     }
   }
-
-  /**
-   * Get the next pending_segments_N filename that will be written.
-   */
-  public String getNextPendingSegmentFileName() {
-    long nextGeneration;
-
+  
+  /** return generation of the next pending_segments_N that will be written */
+  private long getNextPendingGeneration() {
     if (generation == -1) {
-      nextGeneration = 1;
+      return 1;
     } else {
-      nextGeneration = generation+1;
+      return generation+1;
     }
-    return IndexFileNames.fileNameFromGeneration(IndexFileNames.PENDING_SEGMENTS,
-                                                 "",
-                                                 nextGeneration);
   }
 
   /** Since Lucene 5.0, every commit (segments_N) writes a unique id.  This will
@@ -277,18 +270,10 @@ public final class SegmentInfos implemen
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public final void read(Directory directory, String segmentFileName) throws IOException {
-    boolean success = false;
-
-    // Clear any previous segments:
-    this.clear();
-
-    generation = generationFromSegmentsFileName(segmentFileName);
+  public static final SegmentInfos readCommit(Directory directory, String segmentFileName) throws IOException {
 
-    lastGeneration = generation;
-
-    ChecksumIndexInput input = directory.openChecksumInput(segmentFileName, IOContext.READ);
-    try {
+    long generation = generationFromSegmentsFileName(segmentFileName);
+    try (ChecksumIndexInput input = directory.openChecksumInput(segmentFileName, IOContext.READ)) {
       // NOTE: as long as we want to throw indexformattooold (vs corruptindexexception), we need
       // to read the magic ourselves.
       int magic = input.readInt();
@@ -297,17 +282,42 @@ public final class SegmentInfos implemen
       }
       // 4.0+
       int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_40, VERSION_50);
-      version = input.readLong();
-      counter = input.readInt();
+      // 5.0+
+      byte id[] = null;
+      if (format >= VERSION_50) {
+        id = new byte[StringHelper.ID_LENGTH];
+        input.readBytes(id, 0, id.length);
+        CodecUtil.checkIndexHeaderSuffix(input, Long.toString(generation, Character.MAX_RADIX));
+      }
+      
+      SegmentInfos infos = new SegmentInfos();
+      infos.id = id;
+      infos.generation = generation;
+      infos.lastGeneration = generation;
+      infos.version = input.readLong();
+      infos.counter = input.readInt();
       int numSegments = input.readInt();
       if (numSegments < 0) {
         throw new CorruptIndexException("invalid segment count: " + numSegments, input);
       }
       for (int seg = 0; seg < numSegments; seg++) {
         String segName = input.readString();
+        final byte segmentID[];
+        if (format >= VERSION_50) {
+          byte hasID = input.readByte();
+          if (hasID == 1) {
+            segmentID = new byte[StringHelper.ID_LENGTH];
+            input.readBytes(segmentID, 0, segmentID.length);
+          } else if (hasID == 0) {
+            segmentID = null; // 4.x segment, doesn't have an ID
+          } else {
+            throw new CorruptIndexException("invalid hasID byte, got: " + hasID, input);
+          }
+        } else {
+          segmentID = null;
+        }
         Codec codec = Codec.forName(input.readString());
-        //System.out.println("SIS.read seg=" + seg + " codec=" + codec);
-        SegmentInfo info = codec.segmentInfoFormat().read(directory, segName, IOContext.READ);
+        SegmentInfo info = codec.segmentInfoFormat().read(directory, segName, segmentID, IOContext.READ);
         info.setCodec(codec);
         long delGen = input.readLong();
         int delCount = input.readInt();
@@ -358,13 +368,9 @@ public final class SegmentInfos implemen
             siPerCommit.setDocValuesUpdatesFiles(dvUpdateFiles);
           }
         }
-        add(siPerCommit);
-      }
-      userData = input.readStringStringMap();
-      if (format >= VERSION_50) {
-        id = new byte[StringHelper.ID_LENGTH];
-        input.readBytes(id, 0, id.length);
+        infos.add(siPerCommit);
       }
+      infos.userData = input.readStringStringMap();
 
       if (format >= VERSION_48) {
         CodecUtil.checkFooter(input);
@@ -378,30 +384,17 @@ public final class SegmentInfos implemen
         CodecUtil.checkEOF(input);
       }
 
-      success = true;
-    } finally {
-      if (!success) {
-        // Clear any segment infos we had loaded so we
-        // have a clean slate on retry:
-        this.clear();
-        IOUtils.closeWhileHandlingException(input);
-      } else {
-        input.close();
-      }
+      return infos;
     }
   }
 
   /** Find the latest commit ({@code segments_N file}) and
    *  load all {@link SegmentCommitInfo}s. */
-  public final void read(Directory directory) throws IOException {
-    generation = lastGeneration = -1;
-
-    new FindSegmentsFile(directory) {
-
+  public static final SegmentInfos readLatestCommit(Directory directory) throws IOException {
+    return new FindSegmentsFile<SegmentInfos>(directory) {
       @Override
-      protected Object doBody(String segmentFileName) throws IOException {
-        read(directory, segmentFileName);
-        return null;
+      protected SegmentInfos doBody(String segmentFileName) throws IOException {
+        return readCommit(directory, segmentFileName);
       }
     }.run();
   }
@@ -412,27 +405,38 @@ public final class SegmentInfos implemen
 
   private void write(Directory directory) throws IOException {
 
-    String segmentFileName = getNextPendingSegmentFileName();
+    long nextGeneration = getNextPendingGeneration();
+    String segmentFileName = IndexFileNames.fileNameFromGeneration(IndexFileNames.PENDING_SEGMENTS,
+                                                                   "",
+                                                                   nextGeneration);
     
     // Always advance the generation on write:
-    if (generation == -1) {
-      generation = 1;
-    } else {
-      generation++;
-    }
+    generation = nextGeneration;
     
     IndexOutput segnOutput = null;
     boolean success = false;
 
     try {
       segnOutput = directory.createOutput(segmentFileName, IOContext.DEFAULT);
-      CodecUtil.writeHeader(segnOutput, "segments", VERSION_50);
+      CodecUtil.writeIndexHeader(segnOutput, "segments", VERSION_50, 
+                                   StringHelper.randomId(), Long.toString(nextGeneration, Character.MAX_RADIX));
       segnOutput.writeLong(version); 
       segnOutput.writeInt(counter); // write counter
       segnOutput.writeInt(size()); // write infos
       for (SegmentCommitInfo siPerCommit : this) {
         SegmentInfo si = siPerCommit.info;
         segnOutput.writeString(si.name);
+        byte segmentID[] = si.getId();
+        // TODO: remove this in lucene 6, we don't need to include 4.x segments in commits anymore
+        if (segmentID == null) {
+          segnOutput.writeByte((byte)0);
+        } else {
+          if (segmentID.length != StringHelper.ID_LENGTH) {
+            throw new IllegalStateException("cannot write segment: invalid id segment=" + si.name + "id=" + StringHelper.idToString(segmentID));
+          }
+          segnOutput.writeByte((byte)1);
+          segnOutput.writeBytes(segmentID, segmentID.length);
+        }
         segnOutput.writeString(si.getCodec().getName());
         segnOutput.writeLong(siPerCommit.getDelGen());
         int delCount = siPerCommit.getDelCount();
@@ -452,8 +456,6 @@ public final class SegmentInfos implemen
         assert si.dir == directory;
       }
       segnOutput.writeStringStringMap(userData);
-      byte[] id = StringHelper.randomId();
-      segnOutput.writeBytes(id, 0, id.length);
       CodecUtil.writeFooter(segnOutput);
       segnOutput.close();
       directory.sync(Collections.singleton(segmentFileName));
@@ -547,7 +549,7 @@ public final class SegmentInfos implemen
    * time, etc., it could have been deleted due to a writer
    * commit finishing.
    */
-  public abstract static class FindSegmentsFile {
+  public abstract static class FindSegmentsFile<T> {
 
     final Directory directory;
 
@@ -558,12 +560,12 @@ public final class SegmentInfos implemen
 
     /** Locate the most recent {@code segments} file and
      *  run {@link #doBody} on it. */
-    public Object run() throws IOException {
+    public T run() throws IOException {
       return run(null);
     }
     
     /** Run {@link #doBody} on the provided commit. */
-    public Object run(IndexCommit commit) throws IOException {
+    public T run(IndexCommit commit) throws IOException {
       if (commit != null) {
         if (directory != commit.getDirectory())
           throw new IOException("the specified commit does not match the specified Directory");
@@ -607,11 +609,11 @@ public final class SegmentInfos implemen
           String segmentFileName = IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", gen);
         
           try {
-            Object v = doBody(segmentFileName);
+            T t = doBody(segmentFileName);
             if (infoStream != null) {
               message("success on " + segmentFileName);
             }
-            return v;
+            return t;
           } catch (IOException err) {
             // Save the original root cause:
             if (exc == null) {
@@ -634,7 +636,7 @@ public final class SegmentInfos implemen
      * during the processing that could have been caused by
      * a writer committing.
      */
-    protected abstract Object doBody(String segmentFileName) throws IOException;
+    protected abstract T doBody(String segmentFileName) throws IOException;
   }
 
   // Carry over generation numbers from another SegmentInfos

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java Wed Oct 22 04:21:03 2014
@@ -47,11 +47,10 @@ final class StandardDirectoryReader exte
 
   /** called from DirectoryReader.open(...) methods */
   static DirectoryReader open(final Directory directory, final IndexCommit commit) throws IOException {
-    return (DirectoryReader) new SegmentInfos.FindSegmentsFile(directory) {
+    return new SegmentInfos.FindSegmentsFile<DirectoryReader>(directory) {
       @Override
-      protected Object doBody(String segmentFileName) throws IOException {
-        SegmentInfos sis = new SegmentInfos();
-        sis.read(directory, segmentFileName);
+      protected DirectoryReader doBody(String segmentFileName) throws IOException {
+        SegmentInfos sis = SegmentInfos.readCommit(directory, segmentFileName);
         final SegmentReader[] readers = new SegmentReader[sis.size()];
         for (int i = sis.size()-1; i >= 0; i--) {
           boolean success = false;
@@ -309,11 +308,10 @@ final class StandardDirectoryReader exte
   }
 
   private DirectoryReader doOpenFromCommit(IndexCommit commit) throws IOException {
-    return (DirectoryReader) new SegmentInfos.FindSegmentsFile(directory) {
+    return new SegmentInfos.FindSegmentsFile<DirectoryReader>(directory) {
       @Override
-      protected Object doBody(String segmentFileName) throws IOException {
-        final SegmentInfos infos = new SegmentInfos();
-        infos.read(directory, segmentFileName);
+      protected DirectoryReader doBody(String segmentFileName) throws IOException {
+        final SegmentInfos infos = SegmentInfos.readCommit(directory, segmentFileName);
         return doOpenIfChanged(infos);
       }
     }.run(commit);
@@ -338,8 +336,7 @@ final class StandardDirectoryReader exte
       // IndexWriter.prepareCommit has been called (but not
       // yet commit), then the reader will still see itself as
       // current:
-      SegmentInfos sis = new SegmentInfos();
-      sis.read(directory);
+      SegmentInfos sis = SegmentInfos.readLatestCommit(directory);
 
       // we loaded SegmentInfos from the directory
       return sis.getVersion() == segmentInfos.getVersion();

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java Wed Oct 22 04:21:03 2014
@@ -1116,8 +1116,7 @@ public class TestAddIndexes extends Luce
     w3.close();
     // we should now see segments_X,
     // _Y.cfs,_Y.cfe, _Z.si
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     assertEquals("Only one compound segment should exist", 1, sis.size());
     assertTrue(sis.info(0).info.getUseCompoundFile());
     dir.close();

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java Wed Oct 22 04:21:03 2014
@@ -26,7 +26,6 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
@@ -36,10 +35,6 @@ import org.apache.lucene.util.TestUtil;
 public class TestAllFilesHaveChecksumFooter extends LuceneTestCase {
   public void test() throws Exception {
     Directory dir = newDirectory();
-    if (dir instanceof MockDirectoryWrapper) {
-      // Else we might remove .cfe but not the corresponding .cfs, causing false exc when trying to verify headers:
-      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
-    }
     IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
     conf.setCodec(TestUtil.getDefaultCodec());
     RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
@@ -68,8 +63,7 @@ public class TestAllFilesHaveChecksumFoo
   }
   
   private void checkFooters(Directory dir) throws IOException {
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     checkFooter(dir, sis.getSegmentsFileName());
     
     for (SegmentCommitInfo si : sis) {

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java Wed Oct 22 04:21:03 2014
@@ -30,9 +30,7 @@ import org.apache.lucene.document.Numeri
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.TestUtil;
 
 /**
@@ -42,11 +40,6 @@ public class TestAllFilesHaveCodecHeader
   public void test() throws Exception {
     Directory dir = newDirectory();
 
-    if (dir instanceof MockDirectoryWrapper) {
-      // Else we might remove .cfe but not the corresponding .cfs, causing false exc when trying to verify headers:
-      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
-    }
-
     IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
     conf.setCodec(TestUtil.getDefaultCodec());
     RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
@@ -71,10 +64,12 @@ public class TestAllFilesHaveCodecHeader
       if (random().nextInt(7) == 0) {
         riw.commit();
       }
-      // TODO: we should make a new format with a clean header...
-      // if (random().nextInt(20) == 0) {
-      //  riw.deleteDocuments(new Term("id", Integer.toString(i)));
-      // }
+      if (random().nextInt(20) == 0) {
+        riw.deleteDocuments(new Term("id", Integer.toString(i)));
+      }
+      if (random().nextInt(15) == 0) {
+        riw.updateNumericDocValue(new Term("id"), "dv", Long.valueOf(i));
+      }
     }
     riw.close();
     checkHeaders(dir, new HashMap<String,String>());
@@ -82,9 +77,8 @@ public class TestAllFilesHaveCodecHeader
   }
   
   private void checkHeaders(Directory dir, Map<String,String> namesToExtensions) throws IOException {
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
-    checkHeader(dir, sis.getSegmentsFileName(), namesToExtensions, null);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
+    checkHeader(dir, sis.getSegmentsFileName(), namesToExtensions, sis.getId());
     
     for (SegmentCommitInfo si : sis) {
       assertNotNull(si.info.getId());
@@ -118,13 +112,8 @@ public class TestAllFilesHaveCodecHeader
       }
       // read version
       in.readInt();
-      // read segment id (except for segments_N)
-      if (id != null) {
-        byte actualID[] = new byte[StringHelper.ID_LENGTH];
-        in.readBytes(actualID, 0, actualID.length);
-        assertArrayEquals("expected " + StringHelper.idToString(id) + ", got " + StringHelper.idToString(actualID), id, actualID);
-      }
-      
+      // read object id
+      CodecUtil.checkIndexHeaderID(in, id);      
     }
   }
 }

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCodecUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCodecUtil.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCodecUtil.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCodecUtil.java Wed Oct 22 04:21:03 2014
@@ -199,12 +199,12 @@ public class TestCodecUtil extends Lucen
   public void testSegmentHeaderLength() throws Exception {
     RAMFile file = new RAMFile();
     IndexOutput output = new RAMOutputStream(file, true);
-    CodecUtil.writeSegmentHeader(output, "FooBar", 5, StringHelper.randomId(), "xyz");
+    CodecUtil.writeIndexHeader(output, "FooBar", 5, StringHelper.randomId(), "xyz");
     output.writeString("this is the data");
     output.close();
     
     IndexInput input = new RAMInputStream("file", file);
-    input.seek(CodecUtil.segmentHeaderLength("FooBar", "xyz"));
+    input.seek(CodecUtil.indexHeaderLength("FooBar", "xyz"));
     assertEquals("this is the data", input.readString());
     input.close();
   }
@@ -217,7 +217,7 @@ public class TestCodecUtil extends Lucen
     RAMFile file = new RAMFile();
     IndexOutput output = new RAMOutputStream(file, true);
     try {
-      CodecUtil.writeSegmentHeader(output, "foobar", 5, StringHelper.randomId(), tooLong.toString());
+      CodecUtil.writeIndexHeader(output, "foobar", 5, StringHelper.randomId(), tooLong.toString());
       fail("didn't get expected exception");
     } catch (IllegalArgumentException expected) {
       // expected
@@ -232,13 +232,13 @@ public class TestCodecUtil extends Lucen
     RAMFile file = new RAMFile();
     IndexOutput output = new RAMOutputStream(file, true);
     byte[] id = StringHelper.randomId();
-    CodecUtil.writeSegmentHeader(output, "foobar", 5, id, justLongEnough.toString());
+    CodecUtil.writeIndexHeader(output, "foobar", 5, id, justLongEnough.toString());
     output.close();
     
     IndexInput input = new RAMInputStream("file", file);
-    CodecUtil.checkSegmentHeader(input, "foobar", 5, 5, id, justLongEnough.toString());
+    CodecUtil.checkIndexHeader(input, "foobar", 5, 5, id, justLongEnough.toString());
     assertEquals(input.getFilePointer(), input.length());
-    assertEquals(input.getFilePointer(), CodecUtil.segmentHeaderLength("foobar", justLongEnough.toString()));
+    assertEquals(input.getFilePointer(), CodecUtil.indexHeaderLength("foobar", justLongEnough.toString()));
     input.close();
   }
   
@@ -246,7 +246,7 @@ public class TestCodecUtil extends Lucen
     RAMFile file = new RAMFile();
     IndexOutput output = new RAMOutputStream(file, true);
     try {
-      CodecUtil.writeSegmentHeader(output, "foobar", 5, StringHelper.randomId(), "\u1234");
+      CodecUtil.writeIndexHeader(output, "foobar", 5, StringHelper.randomId(), "\u1234");
       fail("didn't get expected exception");
     } catch (IllegalArgumentException expected) {
       // expected

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java Wed Oct 22 04:21:03 2014
@@ -64,8 +64,7 @@ public class TestConsistentFieldNumbers 
 
       writer.close();
 
-      SegmentInfos sis = new SegmentInfos();
-      sis.read(dir);
+      SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
       assertEquals(2, sis.size());
 
       FieldInfos fis1 = IndexWriter.readFieldInfos(sis.info(0));
@@ -82,8 +81,7 @@ public class TestConsistentFieldNumbers 
       writer.forceMerge(1);
       writer.close();
 
-      sis = new SegmentInfos();
-      sis.read(dir);
+      sis = SegmentInfos.readLatestCommit(dir);
       assertEquals(1, sis.size());
 
       FieldInfos fis3 = IndexWriter.readFieldInfos(sis.info(0));
@@ -130,8 +128,7 @@ public class TestConsistentFieldNumbers 
     writer.addIndexes(dir2);
     writer.close();
 
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir1);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir1);
     assertEquals(2, sis.size());
 
     FieldInfos fis1 = IndexWriter.readFieldInfos(sis.info(0));
@@ -161,8 +158,7 @@ public class TestConsistentFieldNumbers 
         d.add(new TextField("f2", "d1 second field", Field.Store.YES));
         writer.addDocument(d);
         writer.close();
-        SegmentInfos sis = new SegmentInfos();
-        sis.read(dir);
+        SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
         assertEquals(1, sis.size());
         FieldInfos fis1 = IndexWriter.readFieldInfos(sis.info(0));
         assertEquals("f1", fis1.fieldInfo(0).name);
@@ -178,8 +174,7 @@ public class TestConsistentFieldNumbers 
         d.add(new StoredField("f3", new byte[] { 1, 2, 3 }));
         writer.addDocument(d);
         writer.close();
-        SegmentInfos sis = new SegmentInfos();
-        sis.read(dir);
+        SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
         assertEquals(2, sis.size());
         FieldInfos fis1 = IndexWriter.readFieldInfos(sis.info(0));
         FieldInfos fis2 = IndexWriter.readFieldInfos(sis.info(1));
@@ -199,8 +194,7 @@ public class TestConsistentFieldNumbers 
         d.add(new StoredField("f3", new byte[] { 1, 2, 3, 4, 5 }));
         writer.addDocument(d);
         writer.close();
-        SegmentInfos sis = new SegmentInfos();
-        sis.read(dir);
+        SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
         assertEquals(3, sis.size());
         FieldInfos fis1 = IndexWriter.readFieldInfos(sis.info(0));
         FieldInfos fis2 = IndexWriter.readFieldInfos(sis.info(1));
@@ -231,8 +225,7 @@ public class TestConsistentFieldNumbers 
       writer.forceMerge(1);
       writer.close();
 
-      SegmentInfos sis = new SegmentInfos();
-      sis.read(dir);
+      SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
       assertEquals(1, sis.size());
       FieldInfos fis1 = IndexWriter.readFieldInfos(sis.info(0));
       assertEquals("f1", fis1.fieldInfo(0).name);
@@ -269,8 +262,7 @@ public class TestConsistentFieldNumbers 
     writer.forceMerge(1);
     writer.close();
 
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     for (SegmentCommitInfo si : sis) {
       FieldInfos fis = IndexWriter.readFieldInfos(si);
 

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java Wed Oct 22 04:21:03 2014
@@ -285,8 +285,7 @@ public class TestDeletionPolicy extends 
         // if we are on a filesystem that seems to have only
         // 1 second resolution, allow +1 second in commit
         // age tolerance:
-        SegmentInfos sis = new SegmentInfos();
-        sis.read(dir, fileName);
+        SegmentInfos sis = SegmentInfos.readCommit(dir, fileName);
         long modTime = Long.parseLong(sis.getUserData().get("commitTime"));
         oneSecondResolution &= (modTime % 1000) == 0;
         final long leeway = (long) ((SECONDS + (oneSecondResolution ? 1.0:0.0))*1000);

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java Wed Oct 22 04:21:03 2014
@@ -664,8 +664,7 @@ public void testFilesOpenClose() throws 
       addDocumentWithFields(writer);
     writer.close();
 
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(d);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(d);
     DirectoryReader r = DirectoryReader.open(d);
     IndexCommit c = r.getIndexCommit();
 

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java Wed Oct 22 04:21:03 2014
@@ -85,8 +85,7 @@ public class TestIndexFileDeleter extend
     writer.close();
     
     // read in index to try to not depend on codec-specific filenames so much
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     SegmentInfo si0 = sis.info(0).info;
     SegmentInfo si1 = sis.info(1).info;
     SegmentInfo si3 = sis.info(3).info;
@@ -123,10 +122,6 @@ public class TestIndexFileDeleter extend
 
     // Create a bogus fnm file when the CFS already exists:
     copyFile(dir, cfsFiles0[0], "_0.fnm");
-    
-    // Create some old segments file:
-    copyFile(dir, "segments_2", "segments");
-    copyFile(dir, "segments_2", "segments_1");
 
     // Create a bogus cfs file shadowing a non-cfs segment:
     
@@ -143,8 +138,7 @@ public class TestIndexFileDeleter extend
     
     String[] filesPre = dir.listAll();
 
-    // Open & close a writer: it should delete the above 4
-    // files and nothing more:
+    // Open & close a writer: it should delete the above files and nothing more:
     writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
                                     .setOpenMode(OpenMode.APPEND));
     writer.close();
@@ -265,8 +259,7 @@ public class TestIndexFileDeleter extend
     // empty commit
     new IndexWriter(dir, new IndexWriterConfig(null)).close();   
     
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     assertEquals(1, sis.getGeneration());
     
     // no inflation
@@ -283,8 +276,7 @@ public class TestIndexFileDeleter extend
     // empty commit
     new IndexWriter(dir, new IndexWriterConfig(null)).close();   
     
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     assertEquals(1, sis.getGeneration());
     
     // add trash commit
@@ -308,8 +300,7 @@ public class TestIndexFileDeleter extend
     // empty commit
     new IndexWriter(dir, new IndexWriterConfig(null)).close();   
     
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     assertEquals(0, sis.counter);
     
     // no inflation
@@ -333,8 +324,7 @@ public class TestIndexFileDeleter extend
     iw.addDocument(new Document());
     iw.commit();
     iw.close();
-    sis = new SegmentInfos();
-    sis.read(dir);
+    sis = SegmentInfos.readLatestCommit(dir);
     assertEquals("_4", sis.info(0).info.name);
     assertEquals(5, sis.counter);
     
@@ -351,8 +341,7 @@ public class TestIndexFileDeleter extend
     iw.close();   
     
     // no deletes: start at 1
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     assertEquals(1, sis.info(0).getNextDelGen());
     
     // no inflation
@@ -376,8 +365,7 @@ public class TestIndexFileDeleter extend
     // empty commit
     new IndexWriter(dir, new IndexWriterConfig(null)).close();   
     
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     assertEquals(1, sis.getGeneration());
     
     // add trash file
@@ -400,8 +388,7 @@ public class TestIndexFileDeleter extend
     iw.close();   
     
     // no deletes: start at 1
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     assertEquals(1, sis.info(0).getNextDelGen());
     
     // add trash file

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java Wed Oct 22 04:21:03 2014
@@ -575,8 +575,7 @@ public class TestIndexWriter extends Luc
         writer.addDocument(doc);
       writer.flush(false, true);
       writer.close();
-      SegmentInfos sis = new SegmentInfos();
-      sis.read(dir);
+      SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
       // Since we flushed w/o allowing merging we should now
       // have 10 segments
       assertEquals(10, sis.size());
@@ -2767,8 +2766,7 @@ public class TestIndexWriter extends Luc
     w.addDocument(new Document());
     w.close();
     
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(d);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(d);
     byte[] id1 = sis.getId();
     assertNotNull(id1);
     assertEquals(StringHelper.ID_LENGTH, id1.length);

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java Wed Oct 22 04:21:03 2014
@@ -1236,8 +1236,7 @@ public class TestIndexWriterExceptions e
     assertTrue("segment generation should be > 0 but got " + gen, gen > 0);
     
     boolean corrupted = false;
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     for (SegmentCommitInfo si : sis) {
       assertTrue(si.info.getUseCompoundFile());
       String cfsFiles[] = si.info.getCodec().compoundFormat().files(si.info);
@@ -1314,8 +1313,7 @@ public class TestIndexWriterExceptions e
         w.close();
         IndexReader reader = DirectoryReader.open(dir);
         assertTrue(reader.numDocs() > 0);
-        SegmentInfos sis = new SegmentInfos();
-        sis.read(dir);
+        SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
         for(LeafReaderContext context : reader.leaves()) {
           assertFalse(context.reader().getFieldInfos().hasVectors());
         }
@@ -1682,7 +1680,7 @@ public class TestIndexWriterExceptions e
       if (doFail && name.startsWith("segments_")) {
         StackTraceElement[] trace = new Exception().getStackTrace();
         for (int i = 0; i < trace.length; i++) {
-          if ("read".equals(trace[i].getMethodName())) {
+          if ("readCommit".equals(trace[i].getMethodName()) || "readLatestCommit".equals(trace[i].getMethodName())) {
             throw new UnsupportedOperationException("expected UOE");
           }
         }

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterForceMerge.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterForceMerge.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterForceMerge.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterForceMerge.java Wed Oct 22 04:21:03 2014
@@ -48,8 +48,7 @@ public class TestIndexWriterForceMerge e
         writer.addDocument(doc);
       writer.close();
 
-      SegmentInfos sis = new SegmentInfos();
-      sis.read(dir);
+      SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
       final int segCount = sis.size();
 
       ldmp = new LogDocMergePolicy();
@@ -59,8 +58,7 @@ public class TestIndexWriterForceMerge e
       writer.forceMerge(3);
       writer.close();
 
-      sis = new SegmentInfos();
-      sis.read(dir);
+      sis = SegmentInfos.readLatestCommit(dir);
       final int optSegCount = sis.size();
 
       if (segCount < 3)
@@ -93,16 +91,14 @@ public class TestIndexWriterForceMerge e
       writer.waitForMerges();
       writer.commit();
 
-      SegmentInfos sis = new SegmentInfos();
-      sis.read(dir);
+      SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
 
       final int segCount = sis.size();
       writer.forceMerge(7);
       writer.commit();
       writer.waitForMerges();
 
-      sis = new SegmentInfos();
-      sis.read(dir);
+      sis = SegmentInfos.readLatestCommit(dir);
       final int optSegCount = sis.size();
 
       if (segCount < 7)
@@ -226,8 +222,7 @@ public class TestIndexWriterForceMerge e
         assertTrue(reader.leaves().size() > 1);
         reader.close();
 
-        SegmentInfos infos = new SegmentInfos();
-        infos.read(dir);
+        SegmentInfos infos = SegmentInfos.readLatestCommit(dir);
         assertEquals(2, infos.size());
       }
     }

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java Wed Oct 22 04:21:03 2014
@@ -34,8 +34,10 @@ import org.apache.lucene.document.TextFi
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.TestUtil;
 
 public class TestIndexWriterThreadsToSegments extends LuceneTestCase {
@@ -331,7 +333,8 @@ public class TestIndexWriterThreadsToSeg
           String segName = IndexFileNames.parseSegmentName(fileName);
           if (segSeen.contains(segName) == false) {
             segSeen.add(segName);
-            SegmentInfo si = TestUtil.getDefaultCodec().segmentInfoFormat().read(dir, segName, IOContext.DEFAULT);
+            byte id[] = readSegmentInfoID(dir, fileName);
+            SegmentInfo si = TestUtil.getDefaultCodec().segmentInfoFormat().read(dir, segName, id, IOContext.DEFAULT);
             si.setCodec(codec);
             SegmentCommitInfo sci = new SegmentCommitInfo(si, 0, -1, -1, -1);
             SegmentReader sr = new SegmentReader(sci, IOContext.DEFAULT);
@@ -349,4 +352,17 @@ public class TestIndexWriterThreadsToSeg
     w.close();
     dir.close();
   }
+  
+  // TODO: remove this hack and fix this test to be better?
+  // the whole thing relies on default codec too...
+  byte[] readSegmentInfoID(Directory dir, String file) throws IOException {
+    try (IndexInput in = dir.openInput(file, IOContext.DEFAULT)) {
+      in.readInt(); // magic
+      in.readString(); // codec name
+      in.readInt(); // version
+      byte id[] = new byte[StringHelper.ID_LENGTH];
+      in.readBytes(id, 0, id.length);
+      return id;
+    }
+  }
 }

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java Wed Oct 22 04:21:03 2014
@@ -143,8 +143,7 @@ public class TestRollingUpdates extends 
     docs.close();
     
     // LUCENE-4455:
-    SegmentInfos infos = new SegmentInfos();
-    infos.read(dir);
+    SegmentInfos infos = SegmentInfos.readLatestCommit(dir);
     long totalBytes = 0;
     for(SegmentCommitInfo sipc : infos) {
       totalBytes += sipc.sizeInBytes();

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestSizeBoundedForceMerge.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestSizeBoundedForceMerge.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestSizeBoundedForceMerge.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestSizeBoundedForceMerge.java Wed Oct 22 04:21:03 2014
@@ -66,8 +66,7 @@ public class TestSizeBoundedForceMerge e
     }
     writer.close();
 
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     double min = sis.info(0).sizeInBytes();
 
     conf = newWriterConfig();
@@ -80,8 +79,7 @@ public class TestSizeBoundedForceMerge e
     writer.close();
 
     // Should only be 3 segments in the index, because one of them exceeds the size limit
-    sis = new SegmentInfos();
-    sis.read(dir);
+    sis = SegmentInfos.readLatestCommit(dir);
     assertEquals(3, sis.size());
   }
 
@@ -113,8 +111,7 @@ public class TestSizeBoundedForceMerge e
     writer.close();
 
     // Should only be 3 segments in the index, because one of them exceeds the size limit
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     assertEquals(3, sis.size());
   }
 
@@ -140,8 +137,7 @@ public class TestSizeBoundedForceMerge e
     writer.forceMerge(1);
     writer.close();
 
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     assertEquals(2, sis.size());
   }
   
@@ -167,8 +163,7 @@ public class TestSizeBoundedForceMerge e
     writer.forceMerge(1);
     writer.close();
     
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     assertEquals(2, sis.size());
   }
   
@@ -194,8 +189,7 @@ public class TestSizeBoundedForceMerge e
     writer.forceMerge(1);
     writer.close();
     
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     assertEquals(1, sis.size());
   }
   
@@ -220,8 +214,7 @@ public class TestSizeBoundedForceMerge e
     writer.forceMerge(1);
     writer.close();
     
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     assertEquals(3, sis.size());
   }
   
@@ -247,8 +240,7 @@ public class TestSizeBoundedForceMerge e
     writer.forceMerge(1);
     writer.close();
     
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     assertEquals(4, sis.size());
   }
   
@@ -280,8 +272,7 @@ public class TestSizeBoundedForceMerge e
     
     // Should only be 4 segments in the index, because of the merge factor and
     // max merge docs settings.
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     assertEquals(4, sis.size());
   }
   
@@ -309,8 +300,7 @@ public class TestSizeBoundedForceMerge e
     writer.close();
     
     // Verify that the last segment does not have deletions.
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     assertEquals(3, sis.size());
     assertFalse(sis.info(2).hasDeletions());
   }
@@ -335,8 +325,7 @@ public class TestSizeBoundedForceMerge e
     writer.close();
     
     // Verify that the last segment does not have deletions.
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     assertEquals(1, sis.size());
   }
 
@@ -363,8 +352,7 @@ public class TestSizeBoundedForceMerge e
     writer.close();
     
     // Verify that the last segment does not have deletions.
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(dir);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(dir);
     assertEquals(1, sis.size());
     assertTrue(sis.info(0).hasDeletions());
   }

Modified: lucene/dev/branches/lucene5969/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java Wed Oct 22 04:21:03 2014
@@ -131,8 +131,7 @@ public class DirectoryTaxonomyWriter imp
 
   /** Reads the commit data from a Directory. */
   private static Map<String, String> readCommitData(Directory dir) throws IOException {
-    SegmentInfos infos = new SegmentInfos();
-    infos.read(dir);
+    SegmentInfos infos = SegmentInfos.readLatestCommit(dir);
     return infos.getUserData();
   }
   

Modified: lucene/dev/branches/lucene5969/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyWriter.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyWriter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyWriter.java Wed Oct 22 04:21:03 2014
@@ -324,8 +324,7 @@ public class TestDirectoryTaxonomyWriter
   }
 
   private long getEpoch(Directory taxoDir) throws IOException {
-    SegmentInfos infos = new SegmentInfos();
-    infos.read(taxoDir);
+    SegmentInfos infos = SegmentInfos.readLatestCommit(taxoDir);
     return Long.parseLong(infos.getUserData().get(DirectoryTaxonomyWriter.INDEX_EPOCH));
   }
   

Modified: lucene/dev/branches/lucene5969/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java Wed Oct 22 04:21:03 2014
@@ -90,8 +90,7 @@ public class IndexSplitter {
   public IndexSplitter(Path dir) throws IOException {
     this.dir = dir;
     fsDir = FSDirectory.open(dir);
-    infos = new SegmentInfos();
-    infos.read(fsDir);
+    infos = SegmentInfos.readLatestCommit(fsDir);
   }
 
   public void listSegments() throws IOException {

Modified: lucene/dev/branches/lucene5969/lucene/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java Wed Oct 22 04:21:03 2014
@@ -78,8 +78,7 @@ public class TestIndexSplitter extends L
     Path destDir2 = createTempDir(LuceneTestCase.getTestClass().getSimpleName());
     IndexSplitter.main(new String[] {dir.toAbsolutePath().toString(), destDir2.toAbsolutePath().toString(), splitSegName});
     Directory fsDirDest2 = newFSDirectory(destDir2);
-    SegmentInfos sis = new SegmentInfos();
-    sis.read(fsDirDest2);
+    SegmentInfos sis = SegmentInfos.readLatestCommit(fsDirDest2);
     assertEquals(1, sis.size());
     r = DirectoryReader.open(fsDirDest2);
     assertEquals(50, r.maxDoc());

Modified: lucene/dev/branches/lucene5969/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java (original)
+++ lucene/dev/branches/lucene5969/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java Wed Oct 22 04:21:03 2014
@@ -37,7 +37,7 @@ final class IDVersionPostingsReader exte
   @Override
   public void init(IndexInput termsIn, SegmentReadState state) throws IOException {
     // Make sure we are talking to the matching postings writer
-    CodecUtil.checkSegmentHeader(termsIn,
+    CodecUtil.checkIndexHeader(termsIn,
                                  IDVersionPostingsWriter.TERMS_CODEC,
                                  IDVersionPostingsWriter.VERSION_START,
                                  IDVersionPostingsWriter.VERSION_CURRENT,

Modified: lucene/dev/branches/lucene5969/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java Wed Oct 22 04:21:03 2014
@@ -57,7 +57,7 @@ final class IDVersionPostingsWriter exte
 
   @Override
   public void init(IndexOutput termsOut, SegmentWriteState state) throws IOException {
-    CodecUtil.writeSegmentHeader(termsOut, TERMS_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+    CodecUtil.writeIndexHeader(termsOut, TERMS_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
   }
 
   @Override

Modified: lucene/dev/branches/lucene5969/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsReader.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsReader.java (original)
+++ lucene/dev/branches/lucene5969/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsReader.java Wed Oct 22 04:21:03 2014
@@ -72,7 +72,7 @@ public final class VersionBlockTreeTerms
     IndexInput indexIn = null;
 
     try {
-      int termsVersion = CodecUtil.checkSegmentHeader(in, VersionBlockTreeTermsWriter.TERMS_CODEC_NAME,
+      int termsVersion = CodecUtil.checkIndexHeader(in, VersionBlockTreeTermsWriter.TERMS_CODEC_NAME,
                                                           VersionBlockTreeTermsWriter.VERSION_START,
                                                           VersionBlockTreeTermsWriter.VERSION_CURRENT,
                                                           state.segmentInfo.getId(), state.segmentSuffix);
@@ -81,7 +81,7 @@ public final class VersionBlockTreeTerms
                                                         state.segmentSuffix, 
                                                         VersionBlockTreeTermsWriter.TERMS_INDEX_EXTENSION);
       indexIn = state.directory.openInput(indexFile, state.context);
-      int indexVersion = CodecUtil.checkSegmentHeader(indexIn, VersionBlockTreeTermsWriter.TERMS_INDEX_CODEC_NAME,
+      int indexVersion = CodecUtil.checkIndexHeader(indexIn, VersionBlockTreeTermsWriter.TERMS_INDEX_CODEC_NAME,
                                                                VersionBlockTreeTermsWriter.VERSION_START,
                                                                VersionBlockTreeTermsWriter.VERSION_CURRENT,
                                                                state.segmentInfo.getId(), state.segmentSuffix);

Modified: lucene/dev/branches/lucene5969/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java Wed Oct 22 04:21:03 2014
@@ -199,13 +199,13 @@ public final class VersionBlockTreeTerms
       fieldInfos = state.fieldInfos;
       this.minItemsInBlock = minItemsInBlock;
       this.maxItemsInBlock = maxItemsInBlock;
-      CodecUtil.writeSegmentHeader(out, TERMS_CODEC_NAME, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);   
+      CodecUtil.writeIndexHeader(out, TERMS_CODEC_NAME, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);   
 
       //DEBUG = state.segmentName.equals("_4a");
 
       final String termsIndexFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_INDEX_EXTENSION);
       indexOut = state.directory.createOutput(termsIndexFileName, state.context);
-      CodecUtil.writeSegmentHeader(indexOut, TERMS_INDEX_CODEC_NAME, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); 
+      CodecUtil.writeIndexHeader(indexOut, TERMS_INDEX_CODEC_NAME, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); 
 
       this.postingsWriter = postingsWriter;
       // segment = state.segmentInfo.name;

Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankySegmentInfoFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankySegmentInfoFormat.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankySegmentInfoFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankySegmentInfoFormat.java Wed Oct 22 04:21:03 2014
@@ -35,8 +35,8 @@ class CrankySegmentInfoFormat extends Se
   }
   
   @Override
-  public SegmentInfo read(Directory directory, String segmentName, IOContext context) throws IOException {
-    return delegate.read(directory, segmentName, context);
+  public SegmentInfo read(Directory directory, String segmentName, byte[] segmentID, IOContext context) throws IOException {
+    return delegate.read(directory, segmentName, segmentID, context);
   }
 
   @Override

Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java?rev=1633514&r1=1633513&r2=1633514&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java (original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java Wed Oct 22 04:21:03 2014
@@ -197,8 +197,7 @@ public abstract class BaseCompoundFormat
       }
     }
     riw.close();
-    SegmentInfos infos = new SegmentInfos();
-    infos.read(dir);
+    SegmentInfos infos = SegmentInfos.readLatestCommit(dir);
     for (SegmentCommitInfo si : infos) {
       if (si.info.getUseCompoundFile()) {
         try (Directory cfsDir = si.info.getCodec().compoundFormat().getCompoundReader(dir, si.info, newIOContext(random()))) {