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/02 08:34:11 UTC

svn commit: r1628889 - in /lucene/dev/branches/lucene5969/lucene: backward-codecs/src/java/org/apache/lucene/codecs/lucene40/ backward-codecs/src/test/org/apache/lucene/codecs/lucene40/ core/src/java/org/apache/lucene/index/ core/src/java/org/apache/lu...

Author: rmuir
Date: Thu Oct  2 06:34:10 2014
New Revision: 1628889

URL: http://svn.apache.org/r1628889
Log:
LUCENE-5969: start improving CFSDir

Added:
    lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile2.java   (with props)
Removed:
    lucene/dev/branches/lucene5969/lucene/misc/src/java/org/apache/lucene/index/CompoundFileExtractor.java
Modified:
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java
    lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java
    lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
    lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
    lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java
    lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java
    lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java?rev=1628889&r1=1628888&r2=1628889&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java Thu Oct  2 06:34:10 2014
@@ -85,7 +85,7 @@ final class Lucene40DocValuesReader exte
   Lucene40DocValuesReader(SegmentReadState state, String filename, String legacyKey) throws IOException {
     this.state = state;
     this.legacyKey = legacyKey;
-    this.dir = new CompoundFileDirectory(state.directory, filename, state.context, false);
+    this.dir = new CompoundFileDirectory(state.segmentInfo.getId(), state.directory, filename, state.context, false);
     ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOf(getClass()));
     merging = false;
   }

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java?rev=1628889&r1=1628888&r2=1628889&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java Thu Oct  2 06:34:10 2014
@@ -51,7 +51,7 @@ final class Lucene40DocValuesWriter exte
   Lucene40DocValuesWriter(SegmentWriteState state, String filename, String legacyKey) throws IOException {
     this.state = state;
     this.legacyKey = legacyKey;
-    this.dir = new CompoundFileDirectory(state.directory, filename, state.context, true);
+    this.dir = new CompoundFileDirectory(state.segmentInfo.getId(), state.directory, filename, state.context, true);
   }
   
   @Override

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=1628889&r1=1628888&r2=1628889&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 Thu Oct  2 06:34:10 2014
@@ -4459,7 +4459,7 @@ public class IndexWriter implements Clos
     }
     // Now merge all added files
     Collection<String> files = info.files();
-    CompoundFileDirectory cfsDir = new CompoundFileDirectory(directory, fileName, context, true);
+    CompoundFileDirectory cfsDir = new CompoundFileDirectory(info.getId(), directory, fileName, context, true);
     boolean success = false;
     try {
       for (String file : files) {

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1628889&r1=1628888&r2=1628889&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java Thu Oct  2 06:34:10 2014
@@ -99,7 +99,7 @@ final class SegmentCoreReaders implement
     
     try {
       if (si.info.getUseCompoundFile()) {
-        cfsDir = cfsReader = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(si.info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
+        cfsDir = cfsReader = new CompoundFileDirectory(si.info.getId(), dir, IndexFileNames.segmentFileName(si.info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
       } else {
         cfsReader = null;
         cfsDir = dir;

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java?rev=1628889&r1=1628888&r2=1628889&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java Thu Oct  2 06:34:10 2014
@@ -202,7 +202,7 @@ public final class SegmentReader extends
     final boolean closeDir;
     if (info.getFieldInfosGen() == -1 && info.info.getUseCompoundFile()) {
       // no fieldInfos gen and segment uses a compound file
-      dir = new CompoundFileDirectory(info.info.dir,
+      dir = new CompoundFileDirectory(info.info.getId(), info.info.dir,
           IndexFileNames.segmentFileName(info.info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION),
           IOContext.READONCE,
           false);

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java?rev=1628889&r1=1628888&r2=1628889&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java Thu Oct  2 06:34:10 2014
@@ -24,7 +24,9 @@ import org.apache.lucene.index.CorruptIn
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.store.DataOutput; // javadocs
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.StringHelper;
 
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -55,7 +57,7 @@ import java.io.IOException;
  *   <li>Compound (.cfs) --&gt; Header, FileData <sup>FileCount</sup>, Footer</li>
  *   <li>Compound Entry Table (.cfe) --&gt; Header, FileCount, &lt;FileName,
  *       DataOffset, DataLength&gt; <sup>FileCount</sup></li>
- *   <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
+ *   <li>Header --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
  *   <li>FileCount --&gt; {@link DataOutput#writeVInt VInt}</li>
  *   <li>DataOffset,DataLength,Checksum --&gt; {@link DataOutput#writeLong UInt64}</li>
  *   <li>FileName --&gt; {@link DataOutput#writeString String}</li>
@@ -89,12 +91,14 @@ public final class CompoundFileDirectory
   private final CompoundFileWriter writer;
   private final IndexInput handle;
   private int version;
+  private final byte[] segmentID;
   
   /**
    * Create a new CompoundFileDirectory.
    */
-  public CompoundFileDirectory(Directory directory, String fileName, IOContext context, boolean openForWrite) throws IOException {
+  public CompoundFileDirectory(byte[] segmentID, Directory directory, String fileName, IOContext context, boolean openForWrite) throws IOException {
     this.directory = directory;
+    this.segmentID = segmentID;
     this.fileName = fileName;
     this.readBufferSize = BufferedIndexInput.bufferSize(context);
     this.isOpen = false;
@@ -105,7 +109,17 @@ public final class CompoundFileDirectory
       try {
         this.entries = readEntries(directory, fileName);
         if (version >= CompoundFileWriter.VERSION_CHECKSUM) {
-          CodecUtil.checkHeader(handle, CompoundFileWriter.DATA_CODEC, version, version);
+          if (version >= CompoundFileWriter.VERSION_SEGMENTHEADER) {
+            // nocommit: remove this null "hack", its because old rw test codecs cant properly impersonate
+            if (segmentID == null) {
+              CodecUtil.checkHeader(handle, CompoundFileWriter.DATA_CODEC, version, version);
+              handle.skipBytes(StringHelper.ID_LENGTH);
+            } else {
+              CodecUtil.checkSegmentHeader(handle, CompoundFileWriter.DATA_CODEC, version, version, segmentID, "");
+            }
+          } else {
+            CodecUtil.checkHeader(handle, CompoundFileWriter.DATA_CODEC, version, version);
+          }
           // NOTE: data file is too costly to verify checksum against all the bytes on open,
           // but for now we at least verify proper structure of the checksum footer: which looks
           // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
@@ -124,7 +138,7 @@ public final class CompoundFileDirectory
       assert !(directory instanceof CompoundFileDirectory) : "compound file inside of compound file: " + fileName;
       this.entries = SENTINEL;
       this.isOpen = true;
-      writer = new CompoundFileWriter(directory, fileName);
+      writer = new CompoundFileWriter(segmentID, directory, fileName);
       handle = null;
     }
   }
@@ -140,6 +154,19 @@ public final class CompoundFileDirectory
                                              IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
       entriesStream = dir.openChecksumInput(entriesFileName, IOContext.READONCE);
       version = CodecUtil.checkHeader(entriesStream, CompoundFileWriter.ENTRY_CODEC, CompoundFileWriter.VERSION_START, CompoundFileWriter.VERSION_CURRENT);
+      if (version >= CompoundFileWriter.VERSION_SEGMENTHEADER) {
+        byte id[] = new byte[StringHelper.ID_LENGTH];
+        entriesStream.readBytes(id, 0, id.length);
+        // nocommit: remove this null "hack", its because old rw test codecs cant properly impersonate
+        if (segmentID != null && !Arrays.equals(id, segmentID)) {
+          throw new CorruptIndexException("file mismatch, expected segment id=" + StringHelper.idToString(segmentID) 
+                                                                     + ", got=" + StringHelper.idToString(id), entriesStream);
+        }
+        byte suffixLength = entriesStream.readByte();
+        if (suffixLength != 0) {
+          throw new CorruptIndexException("unexpected segment suffix, expected zero-length, got=" + (suffixLength & 0xFF), entriesStream);
+        }
+      }
       final int numEntries = entriesStream.readVInt();
       mapping = new HashMap<>(numEntries);
       for (int i = 0; i < numEntries; i++) {

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java?rev=1628889&r1=1628888&r2=1628889&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java Thu Oct  2 06:34:10 2014
@@ -55,7 +55,8 @@ final class CompoundFileWriter implement
   static final String DATA_CODEC = "CompoundFileWriterData";
   static final int VERSION_START = 0;
   static final int VERSION_CHECKSUM = 1;
-  static final int VERSION_CURRENT = VERSION_CHECKSUM;
+  static final int VERSION_SEGMENTHEADER = 2;
+  static final int VERSION_CURRENT = VERSION_SEGMENTHEADER;
 
   // versioning for the .cfe file
   static final String ENTRY_CODEC = "CompoundFileWriterEntries";
@@ -70,6 +71,7 @@ final class CompoundFileWriter implement
   private final AtomicBoolean outputTaken = new AtomicBoolean(false);
   final String entryTableName;
   final String dataFileName;
+  final byte[] segmentID;
 
   /**
    * Create the compound stream in the specified file. The file name is the
@@ -78,11 +80,17 @@ final class CompoundFileWriter implement
    * @throws NullPointerException
    *           if <code>dir</code> or <code>name</code> is null
    */
-  CompoundFileWriter(Directory dir, String name) {
-    if (dir == null)
+  CompoundFileWriter(byte segmentID[], Directory dir, String name) {
+    if (dir == null) {
       throw new NullPointerException("directory cannot be null");
-    if (name == null)
+    }
+    if (name == null) {
       throw new NullPointerException("name cannot be null");
+    }
+    if (segmentID == null) {
+      throw new NullPointerException("segmentID cannot be null");
+    }
+    this.segmentID = segmentID;
     directory = dir;
     entryTableName = IndexFileNames.segmentFileName(
         IndexFileNames.stripExtension(name), "",
@@ -96,7 +104,7 @@ final class CompoundFileWriter implement
       boolean success = false;
       try {
         dataOut = directory.createOutput(dataFileName, context);
-        CodecUtil.writeHeader(dataOut, DATA_CODEC, VERSION_CURRENT);
+        CodecUtil.writeSegmentHeader(dataOut, DATA_CODEC, VERSION_CURRENT, segmentID, "");
         success = true;
       } finally {
         if (!success) {
@@ -207,7 +215,7 @@ final class CompoundFileWriter implement
 
   protected void writeEntryTable(Collection<FileEntry> entries,
       IndexOutput entryOut) throws IOException {
-    CodecUtil.writeHeader(entryOut, ENTRY_CODEC, VERSION_CURRENT);
+    CodecUtil.writeSegmentHeader(entryOut, ENTRY_CODEC, VERSION_CURRENT, segmentID, "");
     entryOut.writeVInt(entries.size());
     for (FileEntry fe : entries) {
       entryOut.writeString(IndexFileNames.stripSegmentName(fe.file));

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=1628889&r1=1628888&r2=1628889&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 Thu Oct  2 06:34:10 2014
@@ -28,7 +28,6 @@ import org.apache.lucene.store.CompoundF
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
@@ -65,33 +64,33 @@ public class TestAllFilesHaveChecksumFoo
       }
     }
     riw.close();
-    checkHeaders(dir);
+    checkFooters(dir);
     dir.close();
   }
   
-  private void checkHeaders(Directory dir) throws IOException {
-    for (String file : dir.listAll()) {
-      if (file.equals(IndexWriter.WRITE_LOCK_NAME)) {
-        continue; // write.lock has no footer, thats ok
-      }
-      if (file.endsWith(IndexFileNames.COMPOUND_FILE_EXTENSION)) {
-        CompoundFileDirectory cfsDir = new CompoundFileDirectory(dir, file, newIOContext(random()), false);
-        checkHeaders(cfsDir); // recurse into cfs
-        cfsDir.close();
-      }
-      IndexInput in = null;
-      boolean success = false;
-      try {
-        in = dir.openInput(file, newIOContext(random()));
-        CodecUtil.checksumEntireFile(in);
-        success = true;
-      } finally {
-        if (success) {
-          IOUtils.close(in);
-        } else {
-          IOUtils.closeWhileHandlingException(in);
+  private void checkFooters(Directory dir) throws IOException {
+    SegmentInfos sis = new SegmentInfos();
+    sis.read(dir);
+    checkFooter(dir, sis.getSegmentsFileName());
+    
+    for (SegmentCommitInfo si : sis) {
+      for (String file : si.files()) {
+        checkFooter(dir, file);
+        if (file.endsWith(IndexFileNames.COMPOUND_FILE_EXTENSION)) {
+          // recurse into CFS
+          try (CompoundFileDirectory cfsDir = new CompoundFileDirectory(si.info.getId(), dir, file, newIOContext(random()), false)) {
+            for (String cfsFile : cfsDir.listAll()) {
+              checkFooter(cfsDir, cfsFile);
+            }
+          }
         }
       }
     }
   }
+  
+  private void checkFooter(Directory dir, String file) throws IOException {
+    try (IndexInput in = dir.openInput(file, newIOContext(random()))) {
+      CodecUtil.checksumEntireFile(in);
+    }
+  }
 }

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=1628889&r1=1628888&r2=1628889&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 Thu Oct  2 06:34:10 2014
@@ -32,7 +32,6 @@ import org.apache.lucene.store.CompoundF
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
@@ -83,39 +82,39 @@ public class TestAllFilesHaveCodecHeader
   }
   
   private void checkHeaders(Directory dir, Map<String,String> namesToExtensions) throws IOException {
-    for (String file : dir.listAll()) {
-      if (file.equals(IndexWriter.WRITE_LOCK_NAME)) {
-        continue; // write.lock has no header, thats ok
+    SegmentInfos sis = new SegmentInfos();
+    sis.read(dir);
+    checkHeader(dir, sis.getSegmentsFileName(), namesToExtensions);
+    
+    for (SegmentCommitInfo si : sis) {
+      for (String file : si.files()) {
+        checkHeader(dir, file, namesToExtensions);
+        if (file.endsWith(IndexFileNames.COMPOUND_FILE_EXTENSION)) {
+          // recurse into CFS
+          try (CompoundFileDirectory cfsDir = new CompoundFileDirectory(si.info.getId(), dir, file, newIOContext(random()), false)) {
+            for (String cfsFile : cfsDir.listAll()) {
+              checkHeader(cfsDir, cfsFile, namesToExtensions);
+            }
+          }
+        }
       }
-      if (file.endsWith(IndexFileNames.COMPOUND_FILE_EXTENSION)) {
-        CompoundFileDirectory cfsDir = new CompoundFileDirectory(dir, file, newIOContext(random()), false);
-        checkHeaders(cfsDir, namesToExtensions); // recurse into cfs
-        cfsDir.close();
+    }
+  }
+  
+  private void checkHeader(Directory dir, String file, Map<String,String> namesToExtensions) throws IOException {
+    try (IndexInput in = dir.openInput(file, newIOContext(random()))) {
+      int val = in.readInt();
+      assertEquals(file + " has no codec header, instead found: " + val, CodecUtil.CODEC_MAGIC, val);
+      String codecName = in.readString();
+      assertFalse(codecName.isEmpty());
+      String extension = IndexFileNames.getExtension(file);
+      if (extension == null) {
+        assertTrue(file.startsWith(IndexFileNames.SEGMENTS));
+        extension = "<segments> (not a real extension, designates segments file)";
       }
-      IndexInput in = null;
-      boolean success = false;
-      try {
-        in = dir.openInput(file, newIOContext(random()));
-        int val = in.readInt();
-        assertEquals(file + " has no codec header, instead found: " + val, CodecUtil.CODEC_MAGIC, val);
-        String codecName = in.readString();
-        assertFalse(codecName.isEmpty());
-        String extension = IndexFileNames.getExtension(file);
-        if (extension == null) {
-          assertTrue(file.startsWith(IndexFileNames.SEGMENTS));
-          extension = "<segments> (not a real extension, designates segments file)";
-        }
-        String previous = namesToExtensions.put(codecName, extension);
-        if (previous != null && !previous.equals(extension)) {
-          fail("extensions " + previous + " and " + extension + " share same codecName " + codecName);
-        }
-        success = true;
-      } finally {
-        if (success) {
-          IOUtils.close(in);
-        } else {
-          IOUtils.closeWhileHandlingException(in);
-        }
+      String previous = namesToExtensions.put(codecName, extension);
+      if (previous != null && !previous.equals(extension)) {
+        fail("extensions " + previous + " and " + extension + " share same codecName " + codecName);
       }
     }
   }

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java?rev=1628889&r1=1628888&r2=1628889&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java Thu Oct  2 06:34:10 2014
@@ -247,10 +247,10 @@ public class TestCodecs extends LuceneTe
     final FieldData[] fields = new FieldData[] {field};
     final FieldInfos fieldInfos = builder.finish();
     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, StringHelper.randomId());
-
+    
+    this.write(si, fieldInfos, dir, fields);
     final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random())));
 
     final Iterator<String> fieldsEnum = reader.iterator();
@@ -304,9 +304,9 @@ public class TestCodecs extends LuceneTe
       System.out.println("TEST: now write postings");
     }
 
-    this.write(fieldInfos, dir, fields);
     Codec codec = Codec.getDefault();
     final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null, StringHelper.randomId());
+    this.write(si, fieldInfos, dir, fields);
 
     if (VERBOSE) {
       System.out.println("TEST: now read postings");
@@ -798,10 +798,9 @@ public class TestCodecs extends LuceneTe
     }
   }
 
-  private void write(final FieldInfos fieldInfos, final Directory dir, final FieldData[] fields) throws Throwable {
+  private void write(SegmentInfo si, 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, StringHelper.randomId());
+    final Codec codec = si.getCodec();
     final SegmentWriteState state = new SegmentWriteState(InfoStream.getDefault(), dir, si, fieldInfos, null, newIOContext(random()));
 
     Arrays.sort(fields);

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java?rev=1628889&r1=1628888&r2=1628889&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java Thu Oct  2 06:34:10 2014
@@ -25,613 +25,277 @@ import org.apache.lucene.store.IOContext
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.store.SimpleFSDirectory;
-import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.TestUtil;
 
 import java.io.IOException;
 import java.nio.file.Path;
 
-public class TestCompoundFile extends LuceneTestCase
-{
-    private Directory dir;
-
-    @Override
-    public void setUp() throws Exception {
-       super.setUp();
-       Path file = createTempDir("testIndex");
-       // use a simple FSDir here, to be sure to have SimpleFSInputs
-       dir = new SimpleFSDirectory(file,null);
-    }
-
-    @Override
-    public void tearDown() throws Exception {
-       dir.close();
-       super.tearDown();
-    }
-
-    /** Creates a file of the specified size with random data. */
-    private void createRandomFile(Directory dir, String name, int size)
-    throws IOException
-    {
-        IndexOutput os = dir.createOutput(name, newIOContext(random()));
-        for (int i=0; i<size; i++) {
-            byte b = (byte) (Math.random() * 256);
-            os.writeByte(b);
-        }
-        os.close();
-    }
-
-    /** Creates a file of the specified size with sequential data. The first
-     *  byte is written as the start byte provided. All subsequent bytes are
-     *  computed as start + offset where offset is the number of the byte.
-     */
-    private void createSequenceFile(Directory dir,
-                                    String name,
-                                    byte start,
-                                    int size)
-    throws IOException
-    {
-        IndexOutput os = dir.createOutput(name, newIOContext(random()));
-        for (int i=0; i < size; i++) {
-            os.writeByte(start);
-            start ++;
-        }
-        os.close();
-    }
-
-
-    private void assertSameStreams(String msg,
-                                   IndexInput expected,
-                                   IndexInput test)
-    throws IOException
-    {
-        assertNotNull(msg + " null expected", expected);
-        assertNotNull(msg + " null test", test);
-        assertEquals(msg + " length", expected.length(), test.length());
-        assertEquals(msg + " position", expected.getFilePointer(),
-                                        test.getFilePointer());
-
-        byte expectedBuffer[] = new byte[512];
-        byte testBuffer[] = new byte[expectedBuffer.length];
-
-        long remainder = expected.length() - expected.getFilePointer();
-        while(remainder > 0) {
-            int readLen = (int) Math.min(remainder, expectedBuffer.length);
-            expected.readBytes(expectedBuffer, 0, readLen);
-            test.readBytes(testBuffer, 0, readLen);
-            assertEqualArrays(msg + ", remainder " + remainder, expectedBuffer,
-                testBuffer, 0, readLen);
-            remainder -= readLen;
-        }
-    }
-
-
-    private void assertSameStreams(String msg,
-                                   IndexInput expected,
-                                   IndexInput actual,
-                                   long seekTo)
-    throws IOException
-    {
-        if(seekTo >= 0 && seekTo < expected.length())
-        {
-            expected.seek(seekTo);
-            actual.seek(seekTo);
-            assertSameStreams(msg + ", seek(mid)", expected, actual);
-        }
-    }
-
-
-
-    private void assertSameSeekBehavior(String msg,
-                                        IndexInput expected,
-                                        IndexInput actual)
-    throws IOException
-    {
-        // seek to 0
-        long point = 0;
-        assertSameStreams(msg + ", seek(0)", expected, actual, point);
-
-        // seek to middle
-        point = expected.length() / 2l;
-        assertSameStreams(msg + ", seek(mid)", expected, actual, point);
-
-        // seek to end - 2
-        point = expected.length() - 2;
-        assertSameStreams(msg + ", seek(end-2)", expected, actual, point);
-
-        // seek to end - 1
-        point = expected.length() - 1;
-        assertSameStreams(msg + ", seek(end-1)", expected, actual, point);
-
-        // seek to the end
-        point = expected.length();
-        assertSameStreams(msg + ", seek(end)", expected, actual, point);
-
-        // seek past end
-        point = expected.length() + 1;
-        assertSameStreams(msg + ", seek(end+1)", expected, actual, point);
-    }
-
-
-    private void assertEqualArrays(String msg,
-                                   byte[] expected,
-                                   byte[] test,
-                                   int start,
-                                   int len)
-    {
-        assertNotNull(msg + " null expected", expected);
-        assertNotNull(msg + " null test", test);
-
-        for (int i=start; i<len; i++) {
-            assertEquals(msg + " " + i, expected[i], test[i]);
-        }
-    }
-
-
-    // ===========================================================
-    //  Tests of the basic CompoundFile functionality
-    // ===========================================================
-
-
-    /** This test creates compound file based on a single file.
-     *  Files of different sizes are tested: 0, 1, 10, 100 bytes.
-     */
-    public void testSingleFile() throws IOException {
-        int data[] = new int[] { 0, 1, 10, 100 };
-        for (int i=0; i<data.length; i++) {
-            String name = "t" + data[i];
-            createSequenceFile(dir, name, (byte) 0, data[i]);
-            CompoundFileDirectory csw = new CompoundFileDirectory(dir, name + ".cfs", newIOContext(random()), true);
-            dir.copy(csw, name, name, newIOContext(random()));
-            csw.close();
-
-            CompoundFileDirectory csr = new CompoundFileDirectory(dir, name + ".cfs", newIOContext(random()), false);
-            IndexInput expected = dir.openInput(name, newIOContext(random()));
-            IndexInput actual = csr.openInput(name, newIOContext(random()));
-            assertSameStreams(name, expected, actual);
-            assertSameSeekBehavior(name, expected, actual);
-            expected.close();
-            actual.close();
-            csr.close();
-        }
-    }
-
-
-    /** This test creates compound file based on two files.
-     *
-     */
-    public void testTwoFiles() throws IOException {
-        createSequenceFile(dir, "d1", (byte) 0, 15);
-        createSequenceFile(dir, "d2", (byte) 0, 114);
-
-        CompoundFileDirectory csw = new CompoundFileDirectory(dir, "d.cfs", newIOContext(random()), true);
-        dir.copy(csw, "d1", "d1", newIOContext(random()));
-        dir.copy(csw, "d2", "d2", newIOContext(random()));
-        csw.close();
-
-        CompoundFileDirectory csr = new CompoundFileDirectory(dir, "d.cfs", newIOContext(random()), false);
-        IndexInput expected = dir.openInput("d1", newIOContext(random()));
-        IndexInput actual = csr.openInput("d1", newIOContext(random()));
-        assertSameStreams("d1", expected, actual);
-        assertSameSeekBehavior("d1", expected, actual);
-        expected.close();
-        actual.close();
-
-        expected = dir.openInput("d2", newIOContext(random()));
-        actual = csr.openInput("d2", newIOContext(random()));
-        assertSameStreams("d2", expected, actual);
-        assertSameSeekBehavior("d2", expected, actual);
-        expected.close();
-        actual.close();
-        csr.close();
-    }
-
-    /** This test creates a compound file based on a large number of files of
-     *  various length. The file content is generated randomly. The sizes range
-     *  from 0 to 1Mb. Some of the sizes are selected to test the buffering
-     *  logic in the file reading code. For this the chunk variable is set to
-     *  the length of the buffer used internally by the compound file logic.
-     */
-    public void testRandomFiles() throws IOException {
-        // Setup the test segment
-        String segment = "test";
-        int chunk = 1024; // internal buffer size used by the stream
-        createRandomFile(dir, segment + ".zero", 0);
-        createRandomFile(dir, segment + ".one", 1);
-        createRandomFile(dir, segment + ".ten", 10);
-        createRandomFile(dir, segment + ".hundred", 100);
-        createRandomFile(dir, segment + ".big1", chunk);
-        createRandomFile(dir, segment + ".big2", chunk - 1);
-        createRandomFile(dir, segment + ".big3", chunk + 1);
-        createRandomFile(dir, segment + ".big4", 3 * chunk);
-        createRandomFile(dir, segment + ".big5", 3 * chunk - 1);
-        createRandomFile(dir, segment + ".big6", 3 * chunk + 1);
-        createRandomFile(dir, segment + ".big7", 1000 * chunk);
-
-        // Setup extraneous files
-        createRandomFile(dir, "onetwothree", 100);
-        createRandomFile(dir, segment + ".notIn", 50);
-        createRandomFile(dir, segment + ".notIn2", 51);
-
-        // Now test
-        CompoundFileDirectory csw = new CompoundFileDirectory(dir, "test.cfs", newIOContext(random()), true);
-        final String data[] = new String[] {
-            ".zero", ".one", ".ten", ".hundred", ".big1", ".big2", ".big3",
-            ".big4", ".big5", ".big6", ".big7"
-        };
-        for (int i=0; i<data.length; i++) {
-            String fileName = segment + data[i];
-            dir.copy(csw, fileName, fileName, newIOContext(random()));
-        }
-        csw.close();
-
-        CompoundFileDirectory csr = new CompoundFileDirectory(dir, "test.cfs", newIOContext(random()), false);
-        for (int i=0; i<data.length; i++) {
-            IndexInput check = dir.openInput(segment + data[i], newIOContext(random()));
-            IndexInput test = csr.openInput(segment + data[i], newIOContext(random()));
-            assertSameStreams(data[i], check, test);
-            assertSameSeekBehavior(data[i], check, test);
-            test.close();
-            check.close();
-        }
-        csr.close();
+public class TestCompoundFile extends LuceneTestCase {
+  private Directory dir;
+  
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    Path file = createTempDir("testIndex");
+    dir = newFSDirectory(file);
+  }
+  
+  @Override
+  public void tearDown() throws Exception {
+    dir.close();
+    super.tearDown();
+  }
+  
+  /** Creates a file of the specified size with random data. */
+  private void createRandomFile(Directory dir, String name, int size) throws IOException {
+    IndexOutput os = dir.createOutput(name, newIOContext(random()));
+    for (int i=0; i<size; i++) {
+      byte b = (byte) (Math.random() * 256);
+      os.writeByte(b);
     }
-
-
-    /** Setup a larger compound file with a number of components, each of
-     *  which is a sequential file (so that we can easily tell that we are
-     *  reading in the right byte). The methods sets up 20 files - f0 to f19,
-     *  the size of each file is 1000 bytes.
-     */
-    private void setUp_2() throws IOException {
-        CompoundFileDirectory cw = new CompoundFileDirectory(dir, "f.comp", newIOContext(random()), true);
-        for (int i=0; i<20; i++) {
-            createSequenceFile(dir, "f" + i, (byte) 0, 2000);
-            String fileName = "f" + i;
-            dir.copy(cw, fileName, fileName, newIOContext(random()));
-        }
-        cw.close();
+    os.close();
+  }
+  
+  /** Creates a file of the specified size with sequential data. The first
+   *  byte is written as the start byte provided. All subsequent bytes are
+   *  computed as start + offset where offset is the number of the byte.
+   */
+  static void createSequenceFile(Directory dir, String name, byte start, int size) throws IOException {
+    IndexOutput os = dir.createOutput(name, newIOContext(random()));
+    for (int i=0; i < size; i++) {
+      os.writeByte(start);
+      start ++;
     }
-
-
-    public void testReadAfterClose() throws IOException {
-        demo_FSIndexInputBug(dir, "test");
+    os.close();
+  }
+  
+  static void assertSameStreams(String msg, IndexInput expected, IndexInput test) throws IOException {
+    assertNotNull(msg + " null expected", expected);
+    assertNotNull(msg + " null test", test);
+    assertEquals(msg + " length", expected.length(), test.length());
+    assertEquals(msg + " position", expected.getFilePointer(), test.getFilePointer());
+    
+    byte expectedBuffer[] = new byte[512];
+    byte testBuffer[] = new byte[expectedBuffer.length];
+    
+    long remainder = expected.length() - expected.getFilePointer();
+    while (remainder > 0) {
+      int readLen = (int) Math.min(remainder, expectedBuffer.length);
+      expected.readBytes(expectedBuffer, 0, readLen);
+      test.readBytes(testBuffer, 0, readLen);
+      assertEqualArrays(msg + ", remainder " + remainder, expectedBuffer, testBuffer, 0, readLen);
+      remainder -= readLen;
     }
-
-    private void demo_FSIndexInputBug(Directory fsdir, String file)
-    throws IOException
-    {
-        // Setup the test file - we need more than 1024 bytes
-        IndexOutput os = fsdir.createOutput(file, IOContext.DEFAULT);
-        for(int i=0; i<2000; i++) {
-            os.writeByte((byte) i);
-        }
-        os.close();
-
-        IndexInput in = fsdir.openInput(file, IOContext.DEFAULT);
-
-        // This read primes the buffer in IndexInput
-        in.readByte();
-
-        // Close the file
-        in.close();
-
-        // ERROR: this call should fail, but succeeds because the buffer
-        // is still filled
-        in.readByte();
-
-        // ERROR: this call should fail, but succeeds for some reason as well
-        in.seek(1099);
-
-        try {
-            // OK: this call correctly fails. We are now past the 1024 internal
-            // buffer, so an actual IO is attempted, which fails
-            in.readByte();
-            fail("expected readByte() to throw exception");
-        } catch (IOException e) {
-          // expected exception
-        }
+  }
+  
+  static void assertSameStreams(String msg, IndexInput expected, IndexInput actual, long seekTo) throws IOException {
+    if (seekTo >= 0 && seekTo < expected.length()) {
+      expected.seek(seekTo);
+      actual.seek(seekTo);
+      assertSameStreams(msg + ", seek(mid)", expected, actual);
     }
-
-    public void testClonedStreamsClosing() throws IOException {
-        setUp_2();
-        CompoundFileDirectory cr = new CompoundFileDirectory(dir, "f.comp", newIOContext(random()), false);
-
-        // basic clone
-        IndexInput expected = dir.openInput("f11", newIOContext(random()));
-
-        IndexInput one = cr.openInput("f11", newIOContext(random()));
-
-        IndexInput two = one.clone();
-
-        assertSameStreams("basic clone one", expected, one);
-        expected.seek(0);
-        assertSameStreams("basic clone two", expected, two);
-
-        // Now close the first stream
-        one.close();
-
-        // The following should really fail since we couldn't expect to
-        // access a file once close has been called on it (regardless of
-        // buffering and/or clone magic)
-        expected.seek(0);
-        two.seek(0);
-        assertSameStreams("basic clone two/2", expected, two);
-
-
-        // Now close the compound reader
-        cr.close();
-
-        // The following may also fail since the compound stream is closed
-        expected.seek(0);
-        two.seek(0);
-        //assertSameStreams("basic clone two/3", expected, two);
-
-
-        // Now close the second clone
-        two.close();
-        expected.seek(0);
-        two.seek(0);
-        //assertSameStreams("basic clone two/4", expected, two);
-
-        expected.close();
+  }
+  
+  static void assertSameSeekBehavior(String msg, IndexInput expected, IndexInput actual) throws IOException {
+    // seek to 0
+    long point = 0;
+    assertSameStreams(msg + ", seek(0)", expected, actual, point);
+    
+    // seek to middle
+    point = expected.length() / 2l;
+    assertSameStreams(msg + ", seek(mid)", expected, actual, point);
+    
+    // seek to end - 2
+    point = expected.length() - 2;
+    assertSameStreams(msg + ", seek(end-2)", expected, actual, point);
+    
+    // seek to end - 1
+    point = expected.length() - 1;
+    assertSameStreams(msg + ", seek(end-1)", expected, actual, point);
+    
+    // seek to the end
+    point = expected.length();
+    assertSameStreams(msg + ", seek(end)", expected, actual, point);
+    
+    // seek past end
+    point = expected.length() + 1;
+    assertSameStreams(msg + ", seek(end+1)", expected, actual, point);
+  }
+  
+  
+  static void assertEqualArrays(String msg, byte[] expected, byte[] test, int start, int len) {
+    assertNotNull(msg + " null expected", expected);
+    assertNotNull(msg + " null test", test);
+    
+    for (int i=start; i<len; i++) {
+      assertEquals(msg + " " + i, expected[i], test[i]);
     }
-
-
-    /** This test opens two files from a compound stream and verifies that
-     *  their file positions are independent of each other.
-     */
-    public void testRandomAccess() throws IOException {
-        setUp_2();
-        CompoundFileDirectory cr = new CompoundFileDirectory(dir, "f.comp", newIOContext(random()), false);
-
-        // Open two files
-        IndexInput e1 = dir.openInput("f11", newIOContext(random()));
-        IndexInput e2 = dir.openInput("f3", newIOContext(random()));
-
-        IndexInput a1 = cr.openInput("f11", newIOContext(random()));
-        IndexInput a2 = dir.openInput("f3", newIOContext(random()));
-
-        // Seek the first pair
-        e1.seek(100);
-        a1.seek(100);
-        assertEquals(100, e1.getFilePointer());
-        assertEquals(100, a1.getFilePointer());
-        byte be1 = e1.readByte();
-        byte ba1 = a1.readByte();
-        assertEquals(be1, ba1);
-
-        // Now seek the second pair
-        e2.seek(1027);
-        a2.seek(1027);
-        assertEquals(1027, e2.getFilePointer());
-        assertEquals(1027, a2.getFilePointer());
-        byte be2 = e2.readByte();
-        byte ba2 = a2.readByte();
-        assertEquals(be2, ba2);
-
-        // Now make sure the first one didn't move
-        assertEquals(101, e1.getFilePointer());
-        assertEquals(101, a1.getFilePointer());
-        be1 = e1.readByte();
-        ba1 = a1.readByte();
-        assertEquals(be1, ba1);
-
-        // Now more the first one again, past the buffer length
-        e1.seek(1910);
-        a1.seek(1910);
-        assertEquals(1910, e1.getFilePointer());
-        assertEquals(1910, a1.getFilePointer());
-        be1 = e1.readByte();
-        ba1 = a1.readByte();
-        assertEquals(be1, ba1);
-
-        // Now make sure the second set didn't move
-        assertEquals(1028, e2.getFilePointer());
-        assertEquals(1028, a2.getFilePointer());
-        be2 = e2.readByte();
-        ba2 = a2.readByte();
-        assertEquals(be2, ba2);
-
-        // Move the second set back, again cross the buffer size
-        e2.seek(17);
-        a2.seek(17);
-        assertEquals(17, e2.getFilePointer());
-        assertEquals(17, a2.getFilePointer());
-        be2 = e2.readByte();
-        ba2 = a2.readByte();
-        assertEquals(be2, ba2);
-
-        // Finally, make sure the first set didn't move
-        // Now make sure the first one didn't move
-        assertEquals(1911, e1.getFilePointer());
-        assertEquals(1911, a1.getFilePointer());
-        be1 = e1.readByte();
-        ba1 = a1.readByte();
-        assertEquals(be1, ba1);
-
-        e1.close();
-        e2.close();
-        a1.close();
-        a2.close();
-        cr.close();
+  }
+  
+  
+  // ===========================================================
+  //  Tests of the basic CompoundFile functionality
+  // ===========================================================
+  
+  
+  /** 
+   * This test creates compound file based on a single file.
+   * Files of different sizes are tested: 0, 1, 10, 100 bytes.
+   */
+  public void testSingleFile() throws IOException {
+    int data[] = new int[] { 0, 1, 10, 100 };
+    for (int i=0; i<data.length; i++) {
+      byte id[] = StringHelper.randomId();
+      String name = "t" + data[i];
+      createSequenceFile(dir, name, (byte) 0, data[i]);
+      CompoundFileDirectory csw = new CompoundFileDirectory(id, dir, name + ".cfs", newIOContext(random()), true);
+      dir.copy(csw, name, name, newIOContext(random()));
+      csw.close();
+      
+      CompoundFileDirectory csr = new CompoundFileDirectory(id, dir, name + ".cfs", newIOContext(random()), false);
+      IndexInput expected = dir.openInput(name, newIOContext(random()));
+      IndexInput actual = csr.openInput(name, newIOContext(random()));
+      assertSameStreams(name, expected, actual);
+      assertSameSeekBehavior(name, expected, actual);
+      expected.close();
+      actual.close();
+      csr.close();
     }
-
-    /** This test opens two files from a compound stream and verifies that
-     *  their file positions are independent of each other.
-     */
-    public void testRandomAccessClones() throws IOException {
-        setUp_2();
-        CompoundFileDirectory cr = new CompoundFileDirectory(dir, "f.comp", newIOContext(random()), false);
-
-        // Open two files
-        IndexInput e1 = cr.openInput("f11", newIOContext(random()));
-        IndexInput e2 = cr.openInput("f3", newIOContext(random()));
-
-        IndexInput a1 = e1.clone();
-        IndexInput a2 = e2.clone();
-
-        // Seek the first pair
-        e1.seek(100);
-        a1.seek(100);
-        assertEquals(100, e1.getFilePointer());
-        assertEquals(100, a1.getFilePointer());
-        byte be1 = e1.readByte();
-        byte ba1 = a1.readByte();
-        assertEquals(be1, ba1);
-
-        // Now seek the second pair
-        e2.seek(1027);
-        a2.seek(1027);
-        assertEquals(1027, e2.getFilePointer());
-        assertEquals(1027, a2.getFilePointer());
-        byte be2 = e2.readByte();
-        byte ba2 = a2.readByte();
-        assertEquals(be2, ba2);
-
-        // Now make sure the first one didn't move
-        assertEquals(101, e1.getFilePointer());
-        assertEquals(101, a1.getFilePointer());
-        be1 = e1.readByte();
-        ba1 = a1.readByte();
-        assertEquals(be1, ba1);
-
-        // Now more the first one again, past the buffer length
-        e1.seek(1910);
-        a1.seek(1910);
-        assertEquals(1910, e1.getFilePointer());
-        assertEquals(1910, a1.getFilePointer());
-        be1 = e1.readByte();
-        ba1 = a1.readByte();
-        assertEquals(be1, ba1);
-
-        // Now make sure the second set didn't move
-        assertEquals(1028, e2.getFilePointer());
-        assertEquals(1028, a2.getFilePointer());
-        be2 = e2.readByte();
-        ba2 = a2.readByte();
-        assertEquals(be2, ba2);
-
-        // Move the second set back, again cross the buffer size
-        e2.seek(17);
-        a2.seek(17);
-        assertEquals(17, e2.getFilePointer());
-        assertEquals(17, a2.getFilePointer());
-        be2 = e2.readByte();
-        ba2 = a2.readByte();
-        assertEquals(be2, ba2);
-
-        // Finally, make sure the first set didn't move
-        // Now make sure the first one didn't move
-        assertEquals(1911, e1.getFilePointer());
-        assertEquals(1911, a1.getFilePointer());
-        be1 = e1.readByte();
-        ba1 = a1.readByte();
-        assertEquals(be1, ba1);
-
-        e1.close();
-        e2.close();
-        a1.close();
-        a2.close();
-        cr.close();
+  }
+  
+  /** 
+   * This test creates compound file based on two files.
+   */
+  public void testTwoFiles() throws IOException {
+    createSequenceFile(dir, "d1", (byte) 0, 15);
+    createSequenceFile(dir, "d2", (byte) 0, 114);
+    
+    byte id[] = StringHelper.randomId();
+    CompoundFileDirectory csw = new CompoundFileDirectory(id, dir, "d.cfs", newIOContext(random()), true);
+    dir.copy(csw, "d1", "d1", newIOContext(random()));
+    dir.copy(csw, "d2", "d2", newIOContext(random()));
+    csw.close();
+    
+    CompoundFileDirectory csr = new CompoundFileDirectory(id, dir, "d.cfs", newIOContext(random()), false);
+    IndexInput expected = dir.openInput("d1", newIOContext(random()));
+    IndexInput actual = csr.openInput("d1", newIOContext(random()));
+    assertSameStreams("d1", expected, actual);
+    assertSameSeekBehavior("d1", expected, actual);
+    expected.close();
+    actual.close();
+    
+    expected = dir.openInput("d2", newIOContext(random()));
+    actual = csr.openInput("d2", newIOContext(random()));
+    assertSameStreams("d2", expected, actual);
+    assertSameSeekBehavior("d2", expected, actual);
+    expected.close();
+    actual.close();
+    csr.close();
+  }
+  
+  /** 
+   * This test creates a compound file based on a large number of files of
+   * various length. The file content is generated randomly. The sizes range
+   * from 0 to 1Mb. Some of the sizes are selected to test the buffering
+   * logic in the file reading code. For this the chunk variable is set to
+   * the length of the buffer used internally by the compound file logic.
+   */
+  public void testRandomFiles() throws IOException {
+    // Setup the test segment
+    String segment = "test";
+    int chunk = 1024; // internal buffer size used by the stream
+    createRandomFile(dir, segment + ".zero", 0);
+    createRandomFile(dir, segment + ".one", 1);
+    createRandomFile(dir, segment + ".ten", 10);
+    createRandomFile(dir, segment + ".hundred", 100);
+    createRandomFile(dir, segment + ".big1", chunk);
+    createRandomFile(dir, segment + ".big2", chunk - 1);
+    createRandomFile(dir, segment + ".big3", chunk + 1);
+    createRandomFile(dir, segment + ".big4", 3 * chunk);
+    createRandomFile(dir, segment + ".big5", 3 * chunk - 1);
+    createRandomFile(dir, segment + ".big6", 3 * chunk + 1);
+    createRandomFile(dir, segment + ".big7", 1000 * chunk);
+    
+    // Setup extraneous files
+    createRandomFile(dir, "onetwothree", 100);
+    createRandomFile(dir, segment + ".notIn", 50);
+    createRandomFile(dir, segment + ".notIn2", 51);
+    
+    byte id[] = StringHelper.randomId();
+    
+    // Now test
+    CompoundFileDirectory csw = new CompoundFileDirectory(id, dir, "test.cfs", newIOContext(random()), true);
+    final String data[] = new String[] {
+        ".zero", ".one", ".ten", ".hundred", ".big1", ".big2", ".big3",
+        ".big4", ".big5", ".big6", ".big7"
+    };
+    for (int i=0; i<data.length; i++) {
+      String fileName = segment + data[i];
+      dir.copy(csw, fileName, fileName, newIOContext(random()));
     }
-
-
-    public void testFileNotFound() throws IOException {
-        setUp_2();
-        CompoundFileDirectory cr = new CompoundFileDirectory(dir, "f.comp", newIOContext(random()), false);
-
-        // Open two files
-        try {
-            cr.openInput("bogus", newIOContext(random()));
-            fail("File not found");
-
-        } catch (IOException e) {
-            /* success */
-            //System.out.println("SUCCESS: File Not Found: " + e);
-        }
-
-        cr.close();
+    csw.close();
+    
+    CompoundFileDirectory csr = new CompoundFileDirectory(id, dir, "test.cfs", newIOContext(random()), false);
+    for (int i=0; i<data.length; i++) {
+      IndexInput check = dir.openInput(segment + data[i], newIOContext(random()));
+      IndexInput test = csr.openInput(segment + data[i], newIOContext(random()));
+      assertSameStreams(data[i], check, test);
+      assertSameSeekBehavior(data[i], check, test);
+      test.close();
+      check.close();
     }
-
-
-    public void testReadPastEOF() throws IOException {
-        setUp_2();
-        CompoundFileDirectory cr = new CompoundFileDirectory(dir, "f.comp", newIOContext(random()), false);
-        IndexInput is = cr.openInput("f2", newIOContext(random()));
-        is.seek(is.length() - 10);
-        byte b[] = new byte[100];
-        is.readBytes(b, 0, 10);
-
-        try {
-            is.readByte();
-            fail("Single byte read past end of file");
-        } catch (IOException e) {
-            /* success */
-            //System.out.println("SUCCESS: single byte read past end of file: " + e);
-        }
-
-        is.seek(is.length() - 10);
-        try {
-            is.readBytes(b, 0, 50);
-            fail("Block read past end of file");
-        } catch (IOException e) {
-            /* success */
-            //System.out.println("SUCCESS: block read past end of file: " + e);
-        }
-
-        is.close();
-        cr.close();
+    csr.close();
+  }
+  
+  /** 
+   * This test that writes larger than the size of the buffer output
+   * will correctly increment the file pointer.
+   */
+  public void testLargeWrites() throws IOException {
+    IndexOutput os = dir.createOutput("testBufferStart.txt", newIOContext(random()));
+    
+    byte[] largeBuf = new byte[2048];
+    for (int i=0; i<largeBuf.length; i++) {
+      largeBuf[i] = (byte) (Math.random() * 256);
     }
-
-    /** This test that writes larger than the size of the buffer output
-     * will correctly increment the file pointer.
-     */
-    public void testLargeWrites() throws IOException {
-        IndexOutput os = dir.createOutput("testBufferStart.txt", newIOContext(random()));
-
-        byte[] largeBuf = new byte[2048];
-        for (int i=0; i<largeBuf.length; i++) {
-            largeBuf[i] = (byte) (Math.random() * 256);
-        }
-
-        long currentPos = os.getFilePointer();
-        os.writeBytes(largeBuf, largeBuf.length);
-
-        try {
-            assertEquals(currentPos + largeBuf.length, os.getFilePointer());
-        } finally {
-            os.close();
-        }
-
+    
+    long currentPos = os.getFilePointer();
+    os.writeBytes(largeBuf, largeBuf.length);
+    
+    try {
+      assertEquals(currentPos + largeBuf.length, os.getFilePointer());
+    } finally {
+      os.close();
     }
+  }
+  
+  public void testAddExternalFile() throws IOException {
+    createSequenceFile(dir, "d1", (byte) 0, 15);
     
-   public void testAddExternalFile() throws IOException {
-       createSequenceFile(dir, "d1", (byte) 0, 15);
-
-       Directory newDir = newDirectory();
-       CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random()), true);
-       dir.copy(csw, "d1", "d1", newIOContext(random()));
-       csw.close();
-
-       CompoundFileDirectory csr = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random()), false);
-       IndexInput expected = dir.openInput("d1", newIOContext(random()));
-       IndexInput actual = csr.openInput("d1", newIOContext(random()));
-       assertSameStreams("d1", expected, actual);
-       assertSameSeekBehavior("d1", expected, actual);
-       expected.close();
-       actual.close();
-       csr.close();
-       
-       newDir.close();
-   }
-   
-   
+    Directory newDir = newDirectory();
+    byte id[] = StringHelper.randomId();
+    CompoundFileDirectory csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), true);
+    dir.copy(csw, "d1", "d1", newIOContext(random()));
+    csw.close();
+    
+    CompoundFileDirectory csr = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), false);
+    IndexInput expected = dir.openInput("d1", newIOContext(random()));
+    IndexInput actual = csr.openInput("d1", newIOContext(random()));
+    assertSameStreams("d1", expected, actual);
+    assertSameSeekBehavior("d1", expected, actual);
+    expected.close();
+    actual.close();
+    csr.close();
+    
+    newDir.close();
+  }
+  
   public void testAppend() throws IOException {
     Directory newDir = newDirectory();
-    CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random()), true);
+    byte id[] = StringHelper.randomId();
+    CompoundFileDirectory csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), true);
     int size = 5 + random().nextInt(128);
     for (int j = 0; j < 2; j++) {
       IndexOutput os = csw.createOutput("seg_" + j + "_foo.txt", newIOContext(random()));
@@ -649,16 +313,15 @@ public class TestCompoundFile extends Lu
     assertEquals(1, listAll.length);
     assertEquals("d.cfs", listAll[0]);
     csw.close();
-    CompoundFileDirectory csr = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random()), false);
+    CompoundFileDirectory csr = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), false);
     for (int j = 0; j < 2; j++) {
       IndexInput openInput = csr.openInput("seg_" + j + "_foo.txt", newIOContext(random()));
       assertEquals(size * 4, openInput.length());
       for (int i = 0; i < size; i++) {
         assertEquals(i*j, openInput.readInt());
       }
-
+      
       openInput.close();
-
     }
     IndexInput expected = dir.openInput("d1", newIOContext(random()));
     IndexInput actual = csr.openInput("d1", newIOContext(random()));
@@ -672,17 +335,18 @@ public class TestCompoundFile extends Lu
   
   public void testAppendTwice() throws IOException {
     Directory newDir = newDirectory();
-    CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random()), true);
+    byte id[] = StringHelper.randomId();
+    CompoundFileDirectory csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), true);
     createSequenceFile(newDir, "d1", (byte) 0, 15);
     IndexOutput out = csw.createOutput("d.xyz", newIOContext(random()));
     out.writeInt(0);
     out.close();
     assertEquals(1, csw.listAll().length);
     assertEquals("d.xyz", csw.listAll()[0]);
-   
+    
     csw.close();
-
-    CompoundFileDirectory cfr = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random()), false);
+    
+    CompoundFileDirectory cfr = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), false);
     assertEquals(1, cfr.listAll().length);
     assertEquals("d.xyz", cfr.listAll()[0]);
     cfr.close();
@@ -691,13 +355,14 @@ public class TestCompoundFile extends Lu
   
   public void testEmptyCFS() throws IOException {
     Directory newDir = newDirectory();
-    CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random()), true);
+    byte id[] = StringHelper.randomId();
+    CompoundFileDirectory csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), true);
     csw.close();
-
-    CompoundFileDirectory csr = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random()), false);
+    
+    CompoundFileDirectory csr = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), false);
     assertEquals(0, csr.listAll().length);
     csr.close();
-
+    
     newDir.close();
   }
   
@@ -707,8 +372,9 @@ public class TestCompoundFile extends Lu
     if (newDir instanceof MockDirectoryWrapper) {
       ((MockDirectoryWrapper)newDir).setEnableVirusScanner(false);
     }
-    CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random()), true);
-    CompoundFileDirectory nested = new CompoundFileDirectory(newDir, "b.cfs", newIOContext(random()), true);
+    byte id[] = StringHelper.randomId();
+    CompoundFileDirectory csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), true);
+    CompoundFileDirectory nested = new CompoundFileDirectory(id, newDir, "b.cfs", newIOContext(random()), true);
     IndexOutput out = nested.createOutput("b.xyz", newIOContext(random()));
     IndexOutput out1 = nested.createOutput("b_1.xyz", newIOContext(random()));
     out.writeInt(0);
@@ -723,10 +389,10 @@ public class TestCompoundFile extends Lu
     csw.close();
     
     assertEquals(2, newDir.listAll().length);
-    csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random()), false);
+    csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), false);
     
     assertEquals(2, csw.listAll().length);
-    nested = new CompoundFileDirectory(csw, "b.cfs", newIOContext(random()), false);
+    nested = new CompoundFileDirectory(id, csw, "b.cfs", newIOContext(random()), false);
     
     assertEquals(2, nested.listAll().length);
     IndexInput openInput = nested.openInput("b.xyz", newIOContext(random()));
@@ -742,7 +408,8 @@ public class TestCompoundFile extends Lu
   
   public void testDoubleClose() throws IOException {
     Directory newDir = newDirectory();
-    CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random()), true);
+    byte id[] = StringHelper.randomId();
+    CompoundFileDirectory csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), true);
     IndexOutput out = csw.createOutput("d.xyz", newIOContext(random()));
     out.writeInt(0);
     out.close();
@@ -751,7 +418,7 @@ public class TestCompoundFile extends Lu
     // close a second time - must have no effect according to Closeable
     csw.close();
     
-    csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random()), false);
+    csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), false);
     IndexInput openInput = csw.openInput("d.xyz", newIOContext(random()));
     assertEquals(0, openInput.readInt());
     openInput.close();
@@ -762,37 +429,39 @@ public class TestCompoundFile extends Lu
     newDir.close();
     
   }
-
+  
   // Make sure we don't somehow use more than 1 descriptor
   // when reading a CFS with many subs:
   public void testManySubFiles() throws IOException {
-
+    
     final Directory d = newFSDirectory(createTempDir("CFSManySubFiles"));
+    byte id[] = StringHelper.randomId();
+    
     final int FILE_COUNT = atLeast(500);
-
+    
     for(int fileIdx=0;fileIdx<FILE_COUNT;fileIdx++) {
       IndexOutput out = d.createOutput("file." + fileIdx, newIOContext(random()));
       out.writeByte((byte) fileIdx);
       out.close();
     }
     
-    final CompoundFileDirectory cfd = new CompoundFileDirectory(d, "c.cfs", newIOContext(random()), true);
+    final CompoundFileDirectory cfd = new CompoundFileDirectory(id, d, "c.cfs", newIOContext(random()), true);
     for(int fileIdx=0;fileIdx<FILE_COUNT;fileIdx++) {
       final String fileName = "file." + fileIdx;
       d.copy(cfd, fileName, fileName, newIOContext(random()));
     }
     cfd.close();
-
+    
     final IndexInput[] ins = new IndexInput[FILE_COUNT];
-    final CompoundFileDirectory cfr = new CompoundFileDirectory(d, "c.cfs", newIOContext(random()), false);
+    final CompoundFileDirectory cfr = new CompoundFileDirectory(id, d, "c.cfs", newIOContext(random()), false);
     for(int fileIdx=0;fileIdx<FILE_COUNT;fileIdx++) {
       ins[fileIdx] = cfr.openInput("file." + fileIdx, newIOContext(random()));
     }
-
+    
     for(int fileIdx=0;fileIdx<FILE_COUNT;fileIdx++) {
       assertEquals((byte) fileIdx, ins[fileIdx].readByte());
     }
-
+    
     for(int fileIdx=0;fileIdx<FILE_COUNT;fileIdx++) {
       ins[fileIdx].close();
     }
@@ -824,30 +493,27 @@ public class TestCompoundFile extends Lu
       }
     }
     riw.close();
-    checkFiles(dir);
-    dir.close();
-  }
-  
-  // checks that we can open all files returned by listAll!
-  private void checkFiles(Directory dir) throws IOException {
-    for (String file : dir.listAll()) {
+    SegmentInfos infos = new SegmentInfos();
+    infos.read(dir);
+    for (String file : infos.files(dir, true)) {
+      try (IndexInput in = dir.openInput(file, IOContext.DEFAULT)) {}
       if (file.endsWith(IndexFileNames.COMPOUND_FILE_EXTENSION)) {
-        CompoundFileDirectory cfsDir = new CompoundFileDirectory(dir, file, newIOContext(random()), false);
-        checkFiles(cfsDir); // recurse into cfs
-        cfsDir.close();
-      }
-      IndexInput in = null;
-      boolean success = false;
-      try {
-        in = dir.openInput(file, newIOContext(random()));
-        success = true;
-      } finally {
-        if (success) {
-          IOUtils.close(in);
-        } else {
-          IOUtils.closeWhileHandlingException(in);
+        String segment = IndexFileNames.parseSegmentName(file);
+        // warning: N^2
+        boolean found = false;
+        for (SegmentCommitInfo si : infos) {
+          if (si.info.name.equals(segment)) {
+            found = true;
+            try (CompoundFileDirectory cfs = new CompoundFileDirectory(si.info.getId(), dir, file, IOContext.DEFAULT, false)) {
+              for (String cfsFile : cfs.listAll()) {
+                try (IndexInput cfsIn = cfs.openInput(cfsFile, IOContext.DEFAULT)) {}
+              }
+            }
+          }
         }
+        assertTrue(found);
       }
     }
+    dir.close();
   }
 }

Added: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile2.java?rev=1628889&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile2.java (added)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile2.java Thu Oct  2 06:34:10 2014
@@ -0,0 +1,296 @@
+package org.apache.lucene.index;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.store.CompoundFileDirectory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.StringHelper;
+
+/** 
+ * Setup a large compound file with a number of components, each of
+ * which is a sequential file (so that we can easily tell that we are
+ * reading in the right byte). The methods sets up 20 files - f0 to f19,
+ * the size of each file is 1000 bytes.
+ */
+public class TestCompoundFile2 extends LuceneTestCase {
+  private Directory dir;
+  byte id[];
+  
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    id = StringHelper.randomId();
+    dir = newDirectory();
+    CompoundFileDirectory cw = new CompoundFileDirectory(id, dir, "f.comp", newIOContext(random()), true);
+    for (int i=0; i<20; i++) {
+      TestCompoundFile.createSequenceFile(dir, "f" + i, (byte) 0, 2000);
+      String fileName = "f" + i;
+      dir.copy(cw, fileName, fileName, newIOContext(random()));
+    }
+    cw.close();
+  }
+  
+  @Override
+  public void tearDown() throws Exception {
+    dir.close();
+    super.tearDown();
+  }
+  
+  public void testClonedStreamsClosing() throws IOException {
+    CompoundFileDirectory cr = new CompoundFileDirectory(id, dir, "f.comp", newIOContext(random()), false);
+    
+    // basic clone
+    IndexInput expected = dir.openInput("f11", newIOContext(random()));
+    
+    IndexInput one = cr.openInput("f11", newIOContext(random()));
+    
+    IndexInput two = one.clone();
+    
+    TestCompoundFile.assertSameStreams("basic clone one", expected, one);
+    expected.seek(0);
+    TestCompoundFile.assertSameStreams("basic clone two", expected, two);
+    
+    // Now close the first stream
+    one.close();
+    
+    // The following should really fail since we couldn't expect to
+    // access a file once close has been called on it (regardless of
+    // buffering and/or clone magic)
+    expected.seek(0);
+    two.seek(0);
+    TestCompoundFile.assertSameStreams("basic clone two/2", expected, two);
+    
+    // Now close the compound reader
+    cr.close();
+    
+    // The following may also fail since the compound stream is closed
+    expected.seek(0);
+    two.seek(0);
+    //assertSameStreams("basic clone two/3", expected, two);
+    
+    // Now close the second clone
+    two.close();
+    expected.seek(0);
+    //assertSameStreams("basic clone two/4", expected, two);
+    
+    expected.close();
+  }
+  
+  /** This test opens two files from a compound stream and verifies that
+   *  their file positions are independent of each other.
+   */
+  public void testRandomAccess() throws IOException {
+    CompoundFileDirectory cr = new CompoundFileDirectory(id, dir, "f.comp", newIOContext(random()), false);
+    
+    // Open two files
+    IndexInput e1 = dir.openInput("f11", newIOContext(random()));
+    IndexInput e2 = dir.openInput("f3", newIOContext(random()));
+    
+    IndexInput a1 = cr.openInput("f11", newIOContext(random()));
+    IndexInput a2 = dir.openInput("f3", newIOContext(random()));
+    
+    // Seek the first pair
+    e1.seek(100);
+    a1.seek(100);
+    assertEquals(100, e1.getFilePointer());
+    assertEquals(100, a1.getFilePointer());
+    byte be1 = e1.readByte();
+    byte ba1 = a1.readByte();
+    assertEquals(be1, ba1);
+    
+    // Now seek the second pair
+    e2.seek(1027);
+    a2.seek(1027);
+    assertEquals(1027, e2.getFilePointer());
+    assertEquals(1027, a2.getFilePointer());
+    byte be2 = e2.readByte();
+    byte ba2 = a2.readByte();
+    assertEquals(be2, ba2);
+    
+    // Now make sure the first one didn't move
+    assertEquals(101, e1.getFilePointer());
+    assertEquals(101, a1.getFilePointer());
+    be1 = e1.readByte();
+    ba1 = a1.readByte();
+    assertEquals(be1, ba1);
+    
+    // Now more the first one again, past the buffer length
+    e1.seek(1910);
+    a1.seek(1910);
+    assertEquals(1910, e1.getFilePointer());
+    assertEquals(1910, a1.getFilePointer());
+    be1 = e1.readByte();
+    ba1 = a1.readByte();
+    assertEquals(be1, ba1);
+    
+    // Now make sure the second set didn't move
+    assertEquals(1028, e2.getFilePointer());
+    assertEquals(1028, a2.getFilePointer());
+    be2 = e2.readByte();
+    ba2 = a2.readByte();
+    assertEquals(be2, ba2);
+    
+    // Move the second set back, again cross the buffer size
+    e2.seek(17);
+    a2.seek(17);
+    assertEquals(17, e2.getFilePointer());
+    assertEquals(17, a2.getFilePointer());
+    be2 = e2.readByte();
+    ba2 = a2.readByte();
+    assertEquals(be2, ba2);
+    
+    // Finally, make sure the first set didn't move
+    // Now make sure the first one didn't move
+    assertEquals(1911, e1.getFilePointer());
+    assertEquals(1911, a1.getFilePointer());
+    be1 = e1.readByte();
+    ba1 = a1.readByte();
+    assertEquals(be1, ba1);
+    
+    e1.close();
+    e2.close();
+    a1.close();
+    a2.close();
+    cr.close();
+  }
+  
+  /** This test opens two files from a compound stream and verifies that
+   *  their file positions are independent of each other.
+   */
+  public void testRandomAccessClones() throws IOException {
+    CompoundFileDirectory cr = new CompoundFileDirectory(id, dir, "f.comp", newIOContext(random()), false);
+    
+    // Open two files
+    IndexInput e1 = cr.openInput("f11", newIOContext(random()));
+    IndexInput e2 = cr.openInput("f3", newIOContext(random()));
+    
+    IndexInput a1 = e1.clone();
+    IndexInput a2 = e2.clone();
+    
+    // Seek the first pair
+    e1.seek(100);
+    a1.seek(100);
+    assertEquals(100, e1.getFilePointer());
+    assertEquals(100, a1.getFilePointer());
+    byte be1 = e1.readByte();
+    byte ba1 = a1.readByte();
+    assertEquals(be1, ba1);
+    
+    // Now seek the second pair
+    e2.seek(1027);
+    a2.seek(1027);
+    assertEquals(1027, e2.getFilePointer());
+    assertEquals(1027, a2.getFilePointer());
+    byte be2 = e2.readByte();
+    byte ba2 = a2.readByte();
+    assertEquals(be2, ba2);
+    
+    // Now make sure the first one didn't move
+    assertEquals(101, e1.getFilePointer());
+    assertEquals(101, a1.getFilePointer());
+    be1 = e1.readByte();
+    ba1 = a1.readByte();
+    assertEquals(be1, ba1);
+    
+    // Now more the first one again, past the buffer length
+    e1.seek(1910);
+    a1.seek(1910);
+    assertEquals(1910, e1.getFilePointer());
+    assertEquals(1910, a1.getFilePointer());
+    be1 = e1.readByte();
+    ba1 = a1.readByte();
+    assertEquals(be1, ba1);
+    
+    // Now make sure the second set didn't move
+    assertEquals(1028, e2.getFilePointer());
+    assertEquals(1028, a2.getFilePointer());
+    be2 = e2.readByte();
+    ba2 = a2.readByte();
+    assertEquals(be2, ba2);
+    
+    // Move the second set back, again cross the buffer size
+    e2.seek(17);
+    a2.seek(17);
+    assertEquals(17, e2.getFilePointer());
+    assertEquals(17, a2.getFilePointer());
+    be2 = e2.readByte();
+    ba2 = a2.readByte();
+    assertEquals(be2, ba2);
+    
+    // Finally, make sure the first set didn't move
+    // Now make sure the first one didn't move
+    assertEquals(1911, e1.getFilePointer());
+    assertEquals(1911, a1.getFilePointer());
+    be1 = e1.readByte();
+    ba1 = a1.readByte();
+    assertEquals(be1, ba1);
+    
+    e1.close();
+    e2.close();
+    a1.close();
+    a2.close();
+    cr.close();
+  }
+  
+  public void testFileNotFound() throws IOException {
+    CompoundFileDirectory cr = new CompoundFileDirectory(id, dir, "f.comp", newIOContext(random()), false);
+    
+    // Open two files
+    try {
+      cr.openInput("bogus", newIOContext(random()));
+      fail("File not found");
+    } catch (IOException e) {
+      /* success */
+      //System.out.println("SUCCESS: File Not Found: " + e);
+    }
+    
+    cr.close();
+  }
+  
+  public void testReadPastEOF() throws IOException {
+    CompoundFileDirectory cr = new CompoundFileDirectory(id, dir, "f.comp", newIOContext(random()), false);
+    IndexInput is = cr.openInput("f2", newIOContext(random()));
+    is.seek(is.length() - 10);
+    byte b[] = new byte[100];
+    is.readBytes(b, 0, 10);
+    
+    try {
+      is.readByte();
+      fail("Single byte read past end of file");
+    } catch (IOException e) {
+      /* success */
+      //System.out.println("SUCCESS: single byte read past end of file: " + e);
+    }
+    
+    is.seek(is.length() - 10);
+    try {
+      is.readBytes(b, 0, 50);
+      fail("Block read past end of file");
+    } catch (IOException e) {
+      /* success */
+      //System.out.println("SUCCESS: block read past end of file: " + e);
+    }
+    
+    is.close();
+    cr.close();
+  }
+}

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java?rev=1628889&r1=1628888&r2=1628889&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java Thu Oct  2 06:34:10 2014
@@ -35,6 +35,7 @@ import org.apache.lucene.search.TermQuer
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.TestUtil;
 
 public class TestNRTCachingDirectory extends BaseDirectoryTestCase {
@@ -134,7 +135,7 @@ public class TestNRTCachingDirectory ext
     }
     out.close();
 
-    Directory cfsDir = new CompoundFileDirectory(dir, "big.cfs", context, true);
+    Directory cfsDir = new CompoundFileDirectory(StringHelper.randomId(), dir, "big.cfs", context, true);
     dir.copy(cfsDir, "big.bin", "big.bin", context);
     cfsDir.close();
     dir.close();

Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java?rev=1628889&r1=1628888&r2=1628889&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java (original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java Thu Oct  2 06:34:10 2014
@@ -34,6 +34,7 @@ import org.apache.lucene.index.Directory
 import org.apache.lucene.index.IndexNotFoundException;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.TestUtil;
 
 /** Base class for per-Directory tests. */
@@ -587,7 +588,8 @@ public abstract class BaseDirectoryTestC
   // LUCENE-3382 test that delegate compound files correctly.
   public void testCompoundFileAppendTwice() throws IOException {
     Directory newDir = getDirectory(createTempDir("testCompoundFileAppendTwice"));
-    CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random()), true);
+    byte id[] = StringHelper.randomId();
+    CompoundFileDirectory csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), true);
     createSequenceFile(newDir, "d1", (byte) 0, 15);
     IndexOutput out = csw.createOutput("d.xyz", newIOContext(random()));
     out.writeInt(0);
@@ -597,7 +599,7 @@ public abstract class BaseDirectoryTestC
    
     csw.close();
 
-    CompoundFileDirectory cfr = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random()), false);
+    CompoundFileDirectory cfr = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), false);
     assertEquals(1, cfr.listAll().length);
     assertEquals("d.xyz", cfr.listAll()[0]);
     cfr.close();