You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/09/27 11:57:36 UTC

svn commit: r1627941 [2/2] - in /lucene/dev/trunk: ./ lucene/ lucene/backward-codecs/ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/ lucene/backward-codecs/src/java/...

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java?rev=1627941&r1=1627940&r2=1627941&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java Sat Sep 27 09:57:35 2014
@@ -451,7 +451,7 @@ class ReadersAndUpdates {
     final IOContext infosContext = new IOContext(new FlushInfo(info.info.getDocCount(), estInfosSize));
     // separately also track which files were created for this gen
     final TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
-    infosFormat.getFieldInfosWriter().write(trackingDir, info.info.name, segmentSuffix, fieldInfos, infosContext);
+    infosFormat.getFieldInfosWriter().write(trackingDir, info.info, segmentSuffix, fieldInfos, infosContext);
     info.advanceFieldInfosGen();
     return trackingDir.getCreatedFiles();
   }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java?rev=1627941&r1=1627940&r2=1627941&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java Sat Sep 27 09:57:35 2014
@@ -109,7 +109,14 @@ public class SegmentCommitInfo {
   @Deprecated
   public void setGenUpdatesFiles(Map<Long,Set<String>> genUpdatesFiles) {
     this.genUpdatesFiles.clear();
-    this.genUpdatesFiles.putAll(genUpdatesFiles);
+    for (Map.Entry<Long,Set<String>> kv : genUpdatesFiles.entrySet()) {
+      // rename the set
+      Set<String> set = new HashSet<>();
+      for (String file : kv.getValue()) {
+        set.add(info.namedForThisSegment(file));
+      }
+      this.genUpdatesFiles.put(kv.getKey(), set);
+    }
   }
   
   /** Returns the per-field DocValues updates files. */
@@ -120,7 +127,14 @@ public class SegmentCommitInfo {
   /** Sets the DocValues updates file names, per field number. Does not deep clone the map. */
   public void setDocValuesUpdatesFiles(Map<Integer,Set<String>> dvUpdatesFiles) {
     this.dvUpdatesFiles.clear();
-    this.dvUpdatesFiles.putAll(dvUpdatesFiles);
+    for (Map.Entry<Integer,Set<String>> kv : dvUpdatesFiles.entrySet()) {
+      // rename the set
+      Set<String> set = new HashSet<>();
+      for (String file : kv.getValue()) {
+        set.add(info.namedForThisSegment(file));
+      }
+      this.dvUpdatesFiles.put(kv.getKey(), set);
+    }
   }
   
   /** Returns the FieldInfos file names. */
@@ -131,7 +145,9 @@ public class SegmentCommitInfo {
   /** Sets the FieldInfos file names. */
   public void setFieldInfosFiles(Set<String> fieldInfosFiles) {
     this.fieldInfosFiles.clear();
-    this.fieldInfosFiles.addAll(fieldInfosFiles);
+    for (String file : fieldInfosFiles) {
+      this.fieldInfosFiles.add(info.namedForThisSegment(file));
+    }
   }
 
   /** Called when we succeed in writing deletes */

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1627941&r1=1627940&r2=1627941&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java Sat Sep 27 09:57:35 2014
@@ -18,8 +18,10 @@ package org.apache.lucene.index;
  */
 
 
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.regex.Matcher;
@@ -58,7 +60,7 @@ public final class SegmentInfo {
   private boolean isCompoundFile;
 
   /** Id that uniquely identifies this segment. */
-  private final String id;
+  private final byte[] id;
 
   private Codec codec;
 
@@ -82,22 +84,13 @@ public final class SegmentInfo {
   }
 
   /**
-   * Construct a new complete SegmentInfo instance from
-   * input, with a newly generated random id.
-   */
-  public SegmentInfo(Directory dir, Version version, String name, int docCount,
-                     boolean isCompoundFile, Codec codec, Map<String,String> diagnostics) {
-    this(dir, version, name, docCount, isCompoundFile, codec, diagnostics, null);
-  }
-
-  /**
    * Construct a new complete SegmentInfo instance from input.
    * <p>Note: this is public only to allow access from
    * the codecs package.</p>
    */
   public SegmentInfo(Directory dir, Version version, String name, int docCount,
                      boolean isCompoundFile, Codec codec, Map<String,String> diagnostics,
-                     String id) {
+                     byte[] id) {
     assert !(dir instanceof TrackingDirectoryWrapper);
     this.dir = dir;
     this.version = version;
@@ -107,6 +100,9 @@ public final class SegmentInfo {
     this.codec = codec;
     this.diagnostics = diagnostics;
     this.id = id;
+    if (id != null && id.length != StringHelper.ID_LENGTH) {
+      throw new IllegalArgumentException("invalid id: " + Arrays.toString(id));
+    }
   }
 
   /**
@@ -226,30 +222,32 @@ public final class SegmentInfo {
   }
 
   /** Return the id that uniquely identifies this segment. */
-  public String getId() {
-    return id;
+  public byte[] getId() {
+    return id == null ? null : id.clone();
   }
 
   private Set<String> setFiles;
 
   /** Sets the files written for this segment. */
-  public void setFiles(Set<String> files) {
-    checkFileNames(files);
-    setFiles = files;
+  public void setFiles(Collection<String> files) {
+    setFiles = new HashSet<>();
+    addFiles(files);
   }
 
   /** Add these files to the set of files written for this
    *  segment. */
   public void addFiles(Collection<String> files) {
     checkFileNames(files);
-    setFiles.addAll(files);
+    for (String f : files) {
+      setFiles.add(namedForThisSegment(f));
+    }
   }
 
   /** Add this file to the set of files written for this
    *  segment. */
   public void addFile(String file) {
     checkFileNames(Collections.singleton(file));
-    setFiles.add(file);
+    setFiles.add(namedForThisSegment(file));
   }
   
   private void checkFileNames(Collection<String> files) {
@@ -261,5 +259,12 @@ public final class SegmentInfo {
       }
     }
   }
-    
+  
+  /** 
+   * strips any segment name from the file, naming it with this segment
+   * this is because "segment names" can change, e.g. by addIndexes(Dir)
+   */
+  String namedForThisSegment(String file) {
+    return name + IndexFileNames.stripSegmentName(file);
+  }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1627941&r1=1627940&r2=1627941&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java Sat Sep 27 09:57:35 2014
@@ -125,8 +125,8 @@ public final class SegmentInfos implemen
   /** The file format version for the segments_N codec header, since 4.9+ */
   public static final int VERSION_49 = 3;
 
-  /** The file format version for the segments_N codec header, since 4.11+ */
-  public static final int VERSION_411 = 4;
+  /** The file format version for the segments_N codec header, since 5.0+ */
+  public static final int VERSION_50 = 4;
 
   /** Used to name new segments. */
   // TODO: should this be a long ...?
@@ -151,8 +151,8 @@ public final class SegmentInfos implemen
    */
   private static PrintStream infoStream = null;
 
-  /** Id for this commit; only written starting with Lucene 4.11 */
-  private String id;
+  /** Id for this commit; only written starting with Lucene 5.0 */
+  private byte[] id;
 
   /** Sole constructor. Typically you call this and then
    *  use {@link #read(Directory) or
@@ -262,10 +262,10 @@ public final class SegmentInfos implemen
                                                  nextGeneration);
   }
 
-  /** Since Lucene 4.11, every commit (segments_N) writes a unique id.  This will
-   *  return that id, or null if this commit was pre-4.11. */
-  public String getId() {
-    return id;
+  /** Since Lucene 5.0, every commit (segments_N) writes a unique id.  This will
+   *  return that id, or null if this commit was 5.0. */
+  public byte[] getId() {
+    return id == null ? null : id.clone();
   }
 
   /**
@@ -296,7 +296,7 @@ public final class SegmentInfos implemen
         throw new IndexFormatTooOldException(input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
       }
       // 4.0+
-      int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_40, VERSION_411);
+      int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_40, VERSION_50);
       version = input.readLong();
       counter = input.readInt();
       int numSegments = input.readInt();
@@ -361,8 +361,9 @@ public final class SegmentInfos implemen
         add(siPerCommit);
       }
       userData = input.readStringStringMap();
-      if (format >= VERSION_411) {
-        id = input.readString();
+      if (format >= VERSION_50) {
+        id = new byte[StringHelper.ID_LENGTH];
+        input.readBytes(id, 0, id.length);
       }
 
       if (format >= VERSION_48) {
@@ -425,7 +426,7 @@ public final class SegmentInfos implemen
 
     try {
       segnOutput = directory.createOutput(segmentFileName, IOContext.DEFAULT);
-      CodecUtil.writeHeader(segnOutput, "segments", VERSION_411);
+      CodecUtil.writeHeader(segnOutput, "segments", VERSION_50);
       segnOutput.writeLong(version); 
       segnOutput.writeInt(counter); // write counter
       segnOutput.writeInt(size()); // write infos
@@ -451,7 +452,8 @@ public final class SegmentInfos implemen
         assert si.dir == directory;
       }
       segnOutput.writeStringStringMap(userData);
-      segnOutput.writeString(StringHelper.randomId());
+      byte[] id = StringHelper.randomId();
+      segnOutput.writeBytes(id, 0, id.length);
       CodecUtil.writeFooter(segnOutput);
       segnOutput.close();
       directory.sync(Collections.singleton(segmentFileName));

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1627941&r1=1627940&r2=1627941&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java Sat Sep 27 09:57:35 2014
@@ -146,7 +146,7 @@ final class SegmentMerger {
     
     // write the merged infos
     FieldInfosWriter fieldInfosWriter = codec.fieldInfosFormat().getFieldInfosWriter();
-    fieldInfosWriter.write(directory, mergeState.segmentInfo.name, "", mergeState.fieldInfos, context);
+    fieldInfosWriter.write(directory, mergeState.segmentInfo, "", mergeState.fieldInfos, context);
 
     return mergeState;
   }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java?rev=1627941&r1=1627940&r2=1627941&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java Sat Sep 27 09:57:35 2014
@@ -217,7 +217,7 @@ public final class SegmentReader extends
       final String segmentSuffix = info.getFieldInfosGen() == -1 ? "" : Long.toString(info.getFieldInfosGen(), Character.MAX_RADIX);
       Codec codec = info.info.getCodec();
       FieldInfosFormat fisFormat = codec.fieldInfosFormat();
-      return fisFormat.getFieldInfosReader().read(dir, info.info.name, segmentSuffix, IOContext.READONCE);
+      return fisFormat.getFieldInfosReader().read(dir, info.info, segmentSuffix, IOContext.READONCE);
     } finally {
       if (closeDir) {
         dir.close();

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java?rev=1627941&r1=1627940&r2=1627941&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java Sat Sep 27 09:57:35 2014
@@ -31,7 +31,7 @@ import org.apache.lucene.search.DocIdSet
  * 
  * @lucene.internal
  */
-public final class FixedBitSet extends DocIdSet implements Bits {
+public final class FixedBitSet extends DocIdSet implements MutableBits {
 
   private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(FixedBitSet.class);
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/StringHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/StringHelper.java?rev=1627941&r1=1627940&r2=1627941&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/StringHelper.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/StringHelper.java Sat Sep 27 09:57:35 2014
@@ -232,21 +232,21 @@ public abstract class StringHelper {
 
   // Holds 128 bit unsigned value:
   private static BigInteger nextId;
-  private static final BigInteger idMask;
+  private static final BigInteger mask128;
   private static final Object idLock = new Object();
-  private static final String idPad = "00000000000000000000000000000000";
 
   static {
-    byte[] maskBytes = new byte[16];
-    Arrays.fill(maskBytes, (byte) 0xff);
-    idMask = new BigInteger(maskBytes);
+    // 128 bit unsigned mask
+    byte[] maskBytes128 = new byte[16];
+    Arrays.fill(maskBytes128, (byte) 0xff);
+    mask128 = new BigInteger(1, maskBytes128);
+    
     String prop = System.getProperty("tests.seed");
 
     // State for xorshift128:
     long x0;
     long x1;
 
-    long seed;
     if (prop != null) {
       // So if there is a test failure that somehow relied on this id,
       // we remain reproducible based on the test seed:
@@ -280,17 +280,25 @@ public abstract class StringHelper {
       s1 ^= s1 << 23; // a
       x1 = s1 ^ s0 ^ (s1 >>> 17) ^ (s0 >>> 26); // b, c
     }
+    
+    // 64-bit unsigned mask
+    byte[] maskBytes64 = new byte[8];
+    Arrays.fill(maskBytes64, (byte) 0xff);
+    BigInteger mask64 = new BigInteger(1, maskBytes64);
 
     // First make unsigned versions of x0, x1:
-    BigInteger unsignedX0 = new BigInteger(1, BigInteger.valueOf(x0).toByteArray());
-    BigInteger unsignedX1 = new BigInteger(1, BigInteger.valueOf(x1).toByteArray());
+    BigInteger unsignedX0 = BigInteger.valueOf(x0).and(mask64);
+    BigInteger unsignedX1 = BigInteger.valueOf(x1).and(mask64);
 
     // Concatentate bits of x0 and x1, as unsigned 128 bit integer:
     nextId = unsignedX0.shiftLeft(64).or(unsignedX1);
   }
+  
+  /** length in bytes of an ID */
+  public static final int ID_LENGTH = 16;
 
   /** Generates a non-cryptographic globally unique id. */
-  public static String randomId() {
+  public static byte[] randomId() {
 
     // NOTE: we don't use Java's UUID.randomUUID() implementation here because:
     //
@@ -306,15 +314,42 @@ public abstract class StringHelper {
     //     what impact that has on the period, whereas the simple ++ (mod 2^128)
     //     we use here is guaranteed to have the full period.
 
-    String id;
+    byte bits[];
     synchronized(idLock) {
-      id = nextId.toString(16);
-      nextId = nextId.add(BigInteger.ONE).and(idMask);
+      bits = nextId.toByteArray();
+      nextId = nextId.add(BigInteger.ONE).and(mask128);
+    }
+    
+    // toByteArray() always returns a sign bit, so it may require an extra byte (always zero)
+    if (bits.length > ID_LENGTH) {
+      assert bits.length == ID_LENGTH + 1;
+      assert bits[0] == 0;
+      return Arrays.copyOfRange(bits, 1, bits.length);
+    } else {
+      byte[] result = new byte[ID_LENGTH];
+      System.arraycopy(bits, 0, result, result.length - bits.length, bits.length);
+      return result;
+    }
+  }
+  
+  /** 
+   * Helper method to render an ID as a string, for debugging
+   * <p>
+   * Returns the string {@code (null)} if the id is null.
+   * Otherwise, returns a string representation for debugging.
+   * Never throws an exception. The returned string may
+   * indicate if the id is definitely invalid.
+   */
+  public static String idToString(byte id[]) {
+    if (id == null) {
+      return "(null)";
+    } else {
+      StringBuilder sb = new StringBuilder();
+      sb.append(new BigInteger(1, id).toString(Character.MAX_RADIX));
+      if (id.length != ID_LENGTH) {
+        sb.append(" (INVALID FORMAT)");
+      }
+      return sb.toString();
     }
-
-    assert id.length() <= 32: "id=" + id;
-    id = idPad.substring(id.length()) + id;
-
-    return id;
   }
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java?rev=1627941&r1=1627940&r2=1627941&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java Sat Sep 27 09:57:35 2014
@@ -42,6 +42,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.Version;
 import org.junit.BeforeClass;
@@ -248,7 +249,7 @@ public class TestCodecs extends LuceneTe
     final Directory dir = newDirectory();
     this.write(fieldInfos, dir, fields);
     Codec codec = Codec.getDefault();
-    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null);
+    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null, StringHelper.randomId());
 
     final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random())));
 
@@ -305,7 +306,7 @@ public class TestCodecs extends LuceneTe
 
     this.write(fieldInfos, dir, fields);
     Codec codec = Codec.getDefault();
-    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null);
+    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null, StringHelper.randomId());
 
     if (VERBOSE) {
       System.out.println("TEST: now read postings");
@@ -800,7 +801,7 @@ public class TestCodecs extends LuceneTe
   private void write(final FieldInfos fieldInfos, final Directory dir, final FieldData[] fields) throws Throwable {
 
     final Codec codec = Codec.getDefault();
-    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null);
+    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null, StringHelper.randomId());
     final SegmentWriteState state = new SegmentWriteState(InfoStream.getDefault(), dir, si, fieldInfos, null, newIOContext(random()));
 
     Arrays.sort(fields);

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDoc.java?rev=1627941&r1=1627940&r2=1627941&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDoc.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDoc.java Sat Sep 27 09:57:35 2014
@@ -43,6 +43,7 @@ import org.apache.lucene.store.MockDirec
 import org.apache.lucene.store.TrackingDirectoryWrapper;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.Version;
 
 
@@ -218,7 +219,7 @@ public class TestDoc extends LuceneTestC
 
       final Codec codec = Codec.getDefault();
       TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(si1.info.dir);
-      final SegmentInfo si = new SegmentInfo(si1.info.dir, Version.LATEST, merged, -1, false, codec, null);
+      final SegmentInfo si = new SegmentInfo(si1.info.dir, Version.LATEST, merged, -1, false, codec, null, StringHelper.randomId());
 
       SegmentMerger merger = new SegmentMerger(Arrays.<LeafReader>asList(r1, r2),
           si, InfoStream.getDefault(), trackingDir,
@@ -226,21 +227,18 @@ public class TestDoc extends LuceneTestC
 
       MergeState mergeState = merger.merge();
       r1.close();
-      r2.close();
-      final SegmentInfo info = new SegmentInfo(si1.info.dir, Version.LATEST, merged,
-                                               si1.info.getDocCount() + si2.info.getDocCount(),
-                                               false, codec, null);
-      info.setFiles(new HashSet<>(trackingDir.getCreatedFiles()));
+      r2.close();;
+      si.setFiles(new HashSet<>(trackingDir.getCreatedFiles()));
       
       if (useCompoundFile) {
-        Collection<String> filesToDelete = IndexWriter.createCompoundFile(InfoStream.getDefault(), dir, MergeState.CheckAbort.NONE, info, newIOContext(random()));
-        info.setUseCompoundFile(true);
+        Collection<String> filesToDelete = IndexWriter.createCompoundFile(InfoStream.getDefault(), dir, MergeState.CheckAbort.NONE, si, newIOContext(random()));
+        si.setUseCompoundFile(true);
         for (final String fileToDelete : filesToDelete) {
           si1.info.dir.deleteFile(fileToDelete);
         }
       }
 
-      return new SegmentCommitInfo(info, 0, -1L, -1L, -1L);
+      return new SegmentCommitInfo(si, 0, -1L, -1L, -1L);
    }
 
 

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java?rev=1627941&r1=1627940&r2=1627941&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java Sat Sep 27 09:57:35 2014
@@ -96,7 +96,7 @@ public class TestIndexFileDeleter extend
     */
 
     // TODO: fix this test better
-    String ext = Codec.getDefault().getName().equals("SimpleText") ? ".liv" : ".del";
+    String ext = ".liv";
     
     // Create a bogus separate del file for a
     // segment that already has a separate del file: 

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1627941&r1=1627940&r2=1627941&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java Sat Sep 27 09:57:35 2014
@@ -2767,11 +2767,13 @@ public class TestIndexWriter extends Luc
     
     SegmentInfos sis = new SegmentInfos();
     sis.read(d);
-    String id1 = sis.getId();
+    byte[] id1 = sis.getId();
     assertNotNull(id1);
+    assertEquals(StringHelper.ID_LENGTH, id1.length);
     
-    String id2 = sis.info(0).info.getId();
+    byte[] id2 = sis.info(0).info.getId();
     assertNotNull(id2);
+    assertEquals(StringHelper.ID_LENGTH, id2.length);
 
     // Make sure CheckIndex includes id output:
     ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
@@ -2784,14 +2786,14 @@ public class TestIndexWriter extends Luc
     assertTrue(s, indexStatus != null && indexStatus.clean);
 
     // Commit id is always stored:
-    assertTrue("missing id=" + id1 + " in:\n" + s, s.contains("id=" + id1));
+    assertTrue("missing id=" + StringHelper.idToString(id1) + " in:\n" + s, s.contains("id=" + StringHelper.idToString(id1)));
 
-    assertTrue("missing id=" + id2 + " in:\n" + s, s.contains("id=" + id2));
+    assertTrue("missing id=" + StringHelper.idToString(id1) + " in:\n" + s, s.contains("id=" + StringHelper.idToString(id1)));
     d.close();
 
     Set<String> ids = new HashSet<>();
     for(int i=0;i<100000;i++) {
-      String id = StringHelper.randomId();
+      String id = StringHelper.idToString(StringHelper.randomId());
       assertFalse("id=" + id + " i=" + i, ids.contains(id));
       ids.add(id);
     }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java?rev=1627941&r1=1627940&r2=1627941&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java Sat Sep 27 09:57:35 2014
@@ -28,6 +28,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.Version;
 
@@ -78,7 +79,7 @@ public class TestSegmentMerger extends L
 
   public void testMerge() throws IOException {
     final Codec codec = Codec.getDefault();
-    final SegmentInfo si = new SegmentInfo(mergedDir, Version.LATEST, mergedSegment, -1, false, codec, null);
+    final SegmentInfo si = new SegmentInfo(mergedDir, Version.LATEST, mergedSegment, -1, false, codec, null, StringHelper.randomId());
 
     SegmentMerger merger = new SegmentMerger(Arrays.<LeafReader>asList(reader1, reader2),
         si, InfoStream.getDefault(), mergedDir,
@@ -88,8 +89,7 @@ public class TestSegmentMerger extends L
     assertTrue(docsMerged == 2);
     //Should be able to open a new SegmentReader against the new directory
     SegmentReader mergedReader = new SegmentReader(new SegmentCommitInfo(
-                                                         new SegmentInfo(mergedDir, Version.LATEST, mergedSegment, docsMerged,
-                                                                         false, codec, null),
+                                                         mergeState.segmentInfo,
                                                          0, -1L, -1L, -1L),
                                                    newIOContext(random()));
     assertTrue(mergedReader != null);

Modified: lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java?rev=1627941&r1=1627940&r2=1627941&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java (original)
+++ lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java Sat Sep 27 09:57:35 2014
@@ -28,6 +28,7 @@ import java.util.List;
 import java.util.Locale;
 
 import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.util.StringHelper;
 
 /**
  * Command-line tool that enables listing segments in an
@@ -137,7 +138,7 @@ public class IndexSplitter {
       SegmentInfo info = infoPerCommit.info;
       // Same info just changing the dir:
       SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.name, info.getDocCount(), 
-                                            info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics());
+                                            info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics(), info.getId());
       destInfos.add(new SegmentCommitInfo(newInfo, infoPerCommit.getDelCount(),
           infoPerCommit.getDelGen(), infoPerCommit.getFieldInfosGen(),
           infoPerCommit.getDocValuesGen()));

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java?rev=1627941&r1=1627940&r2=1627941&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java Sat Sep 27 09:57:35 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs.asserti
 
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FilterCodec;
+import org.apache.lucene.codecs.LiveDocsFormat;
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
@@ -49,7 +50,7 @@ public class AssertingCodec extends Filt
   private final TermVectorsFormat vectors = new AssertingTermVectorsFormat();
   private final StoredFieldsFormat storedFields = new AssertingStoredFieldsFormat();
   private final NormsFormat norms = new AssertingNormsFormat();
-  
+  private final LiveDocsFormat liveDocs = new AssertingLiveDocsFormat();
   private final PostingsFormat defaultFormat = new AssertingPostingsFormat();
   private final DocValuesFormat defaultDVFormat = new AssertingDocValuesFormat();
 
@@ -83,6 +84,11 @@ public class AssertingCodec extends Filt
   }
 
   @Override
+  public LiveDocsFormat liveDocsFormat() {
+    return liveDocs;
+  }
+
+  @Override
   public String toString() {
     return "Asserting(" + delegate + ")";
   }

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyFieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyFieldInfosFormat.java?rev=1627941&r1=1627940&r2=1627941&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyFieldInfosFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyFieldInfosFormat.java Sat Sep 27 09:57:35 2014
@@ -24,6 +24,7 @@ import org.apache.lucene.codecs.FieldInf
 import org.apache.lucene.codecs.FieldInfosReader;
 import org.apache.lucene.codecs.FieldInfosWriter;
 import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 
@@ -59,11 +60,11 @@ class CrankyFieldInfosFormat extends Fie
     }
 
     @Override
-    public void write(Directory directory, String segmentName, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
+    public void write(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
       if (random.nextInt(100) == 0) {
         throw new IOException("Fake IOException from FieldInfosWriter.write()");
       }
-      delegate.write(directory, segmentName, segmentSuffix, infos, context);
+      delegate.write(directory, segmentInfo, segmentSuffix, infos, context);
     }
   }
 }

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java?rev=1627941&r1=1627940&r2=1627941&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java Sat Sep 27 09:57:35 2014
@@ -57,6 +57,7 @@ import org.apache.lucene.util.FixedBitSe
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.RamUsageTester;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.Version;
@@ -675,7 +676,7 @@ public abstract class BasePostingsFormat
   // randomly index at lower IndexOption
   private FieldsProducer buildIndex(Directory dir, IndexOptions maxAllowed, boolean allowPayloads, boolean alwaysTestMax) throws IOException {
     Codec codec = getCodec();
-    SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, "_0", maxDoc, false, codec, null);
+    SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, "_0", maxDoc, false, codec, null, StringHelper.randomId());
 
     int maxIndexOption = Arrays.asList(IndexOptions.values()).indexOf(maxAllowed);
     if (VERBOSE) {