You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2011/08/24 18:05:26 UTC

svn commit: r1161183 - in /lucene/dev/trunk/lucene: contrib/misc/src/java/org/apache/lucene/store/ contrib/misc/src/test/org/apache/lucene/store/ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/index/codecs/ src/java/org/apache/lucene/stor...

Author: simonw
Date: Wed Aug 24 16:05:25 2011
New Revision: 1161183

URL: http://svn.apache.org/viewvc?rev=1161183&view=rev
Log:
LUCENE-3218: Detach CompoundFileDirectory from Directory to prevent traps for delegating Directory implementations

Removed:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/DefaultCompoundFileDirectory.java
    lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/store/MockCompoundFileDirectoryWrapper.java
Modified:
    lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/store/NRTCachingDirectory.java
    lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/Directory.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/MMapDirectory.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java
    lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCompoundFile.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/index.34.cfs.zip
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/index.34.nocfs.zip
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java

Modified: lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/store/NRTCachingDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/store/NRTCachingDirectory.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/store/NRTCachingDirectory.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/store/NRTCachingDirectory.java Wed Aug 24 16:05:25 2011
@@ -228,22 +228,21 @@ public class NRTCachingDirectory extends
     }
   }
 
-  // final due to LUCENE-3382: currently CFS backdoors the directory to create CFE
-  // by using the basic implementation and not delegating, we ensure that all 
-  // openInput/createOutput requests come thru NRTCachingDirectory.
-  @Override
-  public final CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
-    return super.openCompoundInput(name, context);
+  public IndexInputSlicer createSlicer(final String name, final IOContext context) throws IOException {
+    ensureOpen();
+    if (VERBOSE) {
+      System.out.println("nrtdir.openInput name=" + name);
+    }
+    if (cache.fileExists(name)) {
+      if (VERBOSE) {
+        System.out.println("  from cache");
+      }
+      return cache.createSlicer(name, context);
+    } else {
+      return delegate.createSlicer(name, context);
+    }
   }
   
-  // final due to LUCENE-3382: currently CFS backdoors the directory to create CFE
-  // by using the basic implementation and not delegating, we ensure that all 
-  // openInput/createOutput requests come thru NRTCachingDirectory.
-  @Override
-  public final CompoundFileDirectory createCompoundOutput(String name, IOContext context) throws IOException {
-    return super.createCompoundOutput(name, context);
-  }
-
   /** Close this directory, which flushes any cached files
    *  to the delegate and then closes the delegate. */
   @Override

Modified: lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java Wed Aug 24 16:05:25 2011
@@ -148,7 +148,7 @@ public class TestNRTCachingDirectory ext
   // LUCENE-3382 test that delegate compound files correctly.
   public void testCompoundFileAppendTwice() throws IOException {
     Directory newDir = new NRTCachingDirectory(newDirectory(), 2.0, 25.0);
-    CompoundFileDirectory csw = newDir.createCompoundOutput("d.cfs", newIOContext(random));
+    CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), true);
     createSequenceFile(newDir, "d1", (byte) 0, 15);
     IndexOutput out = csw.createOutput("d.xyz", newIOContext(random));
     out.writeInt(0);
@@ -164,7 +164,7 @@ public class TestNRTCachingDirectory ext
    
     csw.close();
 
-    CompoundFileDirectory cfr = newDir.openCompoundInput("d.cfs", newIOContext(random));
+    CompoundFileDirectory cfr = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), false);
     assertEquals(1, cfr.listAll().length);
     assertEquals("d.xyz", cfr.listAll()[0]);
     cfr.close();

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java Wed Aug 24 16:05:25 2011
@@ -1435,7 +1435,7 @@ public abstract class IndexReader implem
       String dirname = file.getAbsoluteFile().getParent();
       filename = file.getName();
       dir = FSDirectory.open(new File(dirname));
-      cfr = dir.openCompoundInput(filename, IOContext.DEFAULT);
+      cfr = new CompoundFileDirectory(dir, filename, IOContext.DEFAULT, false);
 
       String [] files = cfr.listAll();
       ArrayUtil.mergeSort(files);   // sort the array of filename so that the output is more readable

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java Wed Aug 24 16:05:25 2011
@@ -2262,7 +2262,7 @@ public class IndexWriter implements Clos
         String compoundFileName = IndexFileNames.segmentFileName(newSegment.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
         message("creating compound file " + compoundFileName);
         // Now build compound file
-        final Directory cfsDir = directory.createCompoundOutput(compoundFileName, context);
+        final Directory cfsDir = new CompoundFileDirectory(directory, compoundFileName, context, true);
         IOException prior = null;
         try {
           for(String fileName : newSegment.files()) {
@@ -2594,7 +2594,7 @@ public class IndexWriter implements Clos
   private void copySegmentIntoCFS(SegmentInfo info, String segName, IOContext context) throws IOException {
     String segFileName = IndexFileNames.segmentFileName(segName, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
     Collection<String> files = info.files();
-    final CompoundFileDirectory cfsdir = directory.createCompoundOutput(segFileName, context);
+    final CompoundFileDirectory cfsdir = new CompoundFileDirectory(directory, segFileName, context, true);
     try {
       for (String file : files) {
         String newFileName = segName + IndexFileNames.stripSegmentName(file);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java Wed Aug 24 16:05:25 2011
@@ -76,7 +76,7 @@ final class SegmentCoreReaders {
     try {
       Directory dir0 = dir;
       if (si.getUseCompoundFile()) {
-        cfsReader = dir.openCompoundInput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
+        cfsReader = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
         dir0 = cfsReader;
       }
       cfsDir = dir0;
@@ -140,9 +140,9 @@ final class SegmentCoreReaders {
       if (si.getDocStoreOffset() != -1) {
         if (si.getDocStoreIsCompoundFile()) {
           assert storeCFSReader == null;
-          storeCFSReader = dir.openCompoundInput(
+          storeCFSReader = new CompoundFileDirectory(dir,
               IndexFileNames.segmentFileName(si.getDocStoreSegment(), "", IndexFileNames.COMPOUND_FILE_STORE_EXTENSION),
-              context);
+              context, false);
           storeDir = storeCFSReader;
           assert storeDir != null;
         } else {
@@ -154,7 +154,7 @@ final class SegmentCoreReaders {
         // was not used, but then we are asked to open doc
         // stores after the segment has switched to CFS
         if (cfsReader == null) {
-          cfsReader = dir.openCompoundInput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
+          cfsReader = new CompoundFileDirectory(dir,IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
         }
         storeDir = cfsReader;
         assert storeDir != null;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java Wed Aug 24 16:05:25 2011
@@ -30,6 +30,7 @@ import java.util.Set;
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
+import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -247,7 +248,7 @@ public final class SegmentInfo implement
       }
       final Directory dirToTest;
       if (isCompoundFile) {
-        dirToTest = dir.openCompoundInput(IndexFileNames.segmentFileName(storesSegment, "", ext), IOContext.READONCE);
+        dirToTest = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(storesSegment, "", ext), IOContext.READONCE, false);
       } else {
         dirToTest = dir;
       }
@@ -265,8 +266,8 @@ public final class SegmentInfo implement
     if (fieldInfos == null) {
       Directory dir0 = dir;
       if (isCompoundFile && checkCompoundFile) {
-        dir0 = dir.openCompoundInput(IndexFileNames.segmentFileName(name,
-            "", IndexFileNames.COMPOUND_FILE_EXTENSION), IOContext.READONCE);
+        dir0 = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(name,
+            "", IndexFileNames.COMPOUND_FILE_EXTENSION), IOContext.READONCE, false);
       }
       try {
         fieldInfos = new FieldInfos(dir0, IndexFileNames.segmentFileName(name,
@@ -619,7 +620,7 @@ public final class SegmentInfo implement
 
     if (useCompoundFile) {
       fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.COMPOUND_FILE_EXTENSION));
-      if (version != null && StringHelper.getVersionComparator().compare("3.4", version) <= 0) {
+      if (version != null && StringHelper.getVersionComparator().compare("4.0", version) <= 0) {
         fileSet.add(IndexFileNames.segmentFileName(name, "",
             IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
       }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Wed Aug 24 16:05:25 2011
@@ -153,7 +153,7 @@ final class SegmentMerger {
 
     // Now merge all added files
     Collection<String> files = info.files();
-    CompoundFileDirectory cfsDir = directory.createCompoundOutput(fileName, context);
+    CompoundFileDirectory cfsDir = new CompoundFileDirectory(directory, fileName, context, true);
     try {
       for (String file : files) {
         assert !IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION) 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java Wed Aug 24 16:05:25 2011
@@ -28,6 +28,7 @@ import org.apache.lucene.index.IndexFile
 import org.apache.lucene.index.PerDocWriteState;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.values.Writer;
+import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.BytesRef;
@@ -51,9 +52,9 @@ public class DefaultDocValuesConsumer ex
     this.bytesUsed = state.bytesUsed;
     this.context = state.context;
     //TODO maybe we should enable a global CFS that all codecs can pull on demand to further reduce the number of files?
-    this.directory = useCompoundFile ? state.directory.createCompoundOutput(
+    this.directory = useCompoundFile ? new CompoundFileDirectory(state.directory,
         IndexFileNames.segmentFileName(segmentName, codecId,
-            IndexFileNames.COMPOUND_FILE_EXTENSION), context) : state.directory;
+            IndexFileNames.COMPOUND_FILE_EXTENSION), context, true) : state.directory;
     this.comparator = comparator;
     this.useCompoundFile = useCompoundFile;
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java Wed Aug 24 16:05:25 2011
@@ -32,6 +32,7 @@ import org.apache.lucene.index.values.In
 import org.apache.lucene.index.values.Floats;
 import org.apache.lucene.index.values.Ints;
 import org.apache.lucene.index.values.ValueType;
+import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.BytesRef;
@@ -78,7 +79,7 @@ public class DefaultDocValuesProducer ex
     this.sortComparator = sortComparator;
     final Directory directory;
     if (useCompoundFile) {
-      cfs = directory = dir.openCompoundInput(IndexFileNames.segmentFileName(si.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION), context);
+      cfs = directory = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(si.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
     } else {
       cfs = null;
       directory = dir;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java Wed Aug 24 16:05:25 2011
@@ -27,6 +27,7 @@ import org.apache.lucene.index.IndexForm
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentInfos;
 import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -68,13 +69,13 @@ public class DefaultSegmentInfosReader e
           Directory dir = directory;
           if (si.getDocStoreOffset() != -1) {
             if (si.getDocStoreIsCompoundFile()) {
-              dir = dir.openCompoundInput(IndexFileNames.segmentFileName(
+              dir = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(
                   si.getDocStoreSegment(), "",
-                  IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), context);
+                  IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), context, false);
             }
           } else if (si.getUseCompoundFile()) {
-            dir = dir.openCompoundInput(IndexFileNames.segmentFileName(
-                si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
+            dir = new CompoundFileDirectory(dir,IndexFileNames.segmentFileName(
+                si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
           }
 
           try {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java Wed Aug 24 16:05:25 2011
@@ -34,7 +34,7 @@ import java.io.IOException;
  * Directory methods that would normally modify data throw an exception.
  * @lucene.experimental
  */
-public abstract class CompoundFileDirectory extends Directory {
+public final class CompoundFileDirectory extends Directory {
   
   /** Offset/Length for a slice inside of a compound file */
   public static final class FileEntry {
@@ -45,68 +45,86 @@ public abstract class CompoundFileDirect
   private final Directory directory;
   private final String fileName;
   protected final int readBufferSize;  
-  private Map<String,FileEntry> entries;
-  private boolean openForWrite;
+  private final Map<String,FileEntry> entries;
+  private final boolean openForWrite;
   private static final Map<String,FileEntry> SENTINEL = Collections.emptyMap();
-  private CompoundFileWriter writer;
+  private final CompoundFileWriter writer;
+  private final IndexInputSlicer handle;
   
   /**
    * Create a new CompoundFileDirectory.
    * <p>
    * NOTE: subclasses must call {@link #initForRead(Map)} before the directory can be used.
    */
-  public CompoundFileDirectory(Directory directory, String fileName, IOContext context) throws IOException {
-
+  public CompoundFileDirectory(Directory directory, String fileName, IOContext context, boolean openForWrite) throws IOException {
     this.directory = directory;
     this.fileName = fileName;
     this.readBufferSize = BufferedIndexInput.bufferSize(context);
     this.isOpen = false;
+    this.openForWrite = openForWrite;
+    if (!openForWrite) {
+      boolean success = false;
+      handle = directory.createSlicer(fileName, context);
+      try {
+        this.entries = readEntries(handle, directory, fileName);
+        success = true;
+      } finally {
+        if (!success) {
+          IOUtils.closeSafely(true, handle);
+        }
+      }
+      this.isOpen = true;
+      writer = null;
+    } else {
+      assert !(directory instanceof CompoundFileDirectory) : "compound file inside of compound file: " + fileName;
+      this.entries = SENTINEL;
+      this.isOpen = true;
+      writer = new CompoundFileWriter(directory, fileName);
+      handle = null;
+    }
   }
-  
-  /** Initialize with a map of filename->slices */
-  protected final void initForRead(Map<String,FileEntry> entries) {
-    this.entries = entries;
-    this.isOpen = true;
-    this.openForWrite = false;
-  }
-  
-  protected final void initForWrite() throws IOException {
-    assert !(directory instanceof CompoundFileDirectory) : "compound file inside of compound file: " + fileName;
-    this.entries = SENTINEL;
-    this.openForWrite = true;
-    this.isOpen = true;
-    writer = new CompoundFileWriter(directory, fileName);
-  }
-  
+
   /** Helper method that reads CFS entries from an input stream */
-  public static final Map<String,FileEntry> readEntries(IndexInput stream, Directory dir, String name) throws IOException {
+  private static final Map<String, FileEntry> readEntries(
+      IndexInputSlicer handle, Directory dir, String name) throws IOException {
     // read the first VInt. If it is negative, it's the version number
     // otherwise it's the count (pre-3.1 indexes)
-    final int firstInt = stream.readVInt();
-    if (firstInt == CompoundFileWriter.FORMAT_CURRENT) {
-      IndexInput input = null;
-      try {
-        input = dir.openInput(IndexFileNames.segmentFileName(IndexFileNames.stripExtension(name), "",
-            IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION), IOContext.READONCE);
-        final int readInt = input.readInt(); // unused right now
-        assert readInt == CompoundFileWriter.ENTRY_FORMAT_CURRENT;
-        final int numEntries = input.readVInt();
-        final Map<String, FileEntry> mapping = new HashMap<String, CompoundFileDirectory.FileEntry>(
-            numEntries);
-        for (int i = 0; i < numEntries; i++) {
-          final FileEntry fileEntry = new FileEntry();
-          mapping.put(input.readString(), fileEntry);
-          fileEntry.offset = input.readLong();
-          fileEntry.length = input.readLong();
+    final IndexInput stream = handle.openFullSlice();
+    final Map<String, FileEntry> mapping;
+    boolean success = false;
+    try {
+      final int firstInt = stream.readVInt();
+      if (firstInt == CompoundFileWriter.FORMAT_CURRENT) {
+        IndexInput input = null;
+        try {
+          input = dir.openInput(IndexFileNames.segmentFileName(
+              IndexFileNames.stripExtension(name), "",
+              IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION),
+              IOContext.READONCE);
+          final int readInt = input.readInt(); // unused right now
+          assert readInt == CompoundFileWriter.ENTRY_FORMAT_CURRENT;
+          final int numEntries = input.readVInt();
+          mapping = new HashMap<String, CompoundFileDirectory.FileEntry>(
+              numEntries);
+          for (int i = 0; i < numEntries; i++) {
+            final FileEntry fileEntry = new FileEntry();
+            mapping.put(input.readString(), fileEntry);
+            fileEntry.offset = input.readLong();
+            fileEntry.length = input.readLong();
+          }
+          return mapping;
+        } finally {
+          IOUtils.closeSafely(true, input);
         }
-        return mapping;
-      } finally {
-        IOUtils.closeSafely(true, input);
+      } else {
+        // TODO remove once 3.x is not supported anymore
+        mapping = readLegacyEntries(stream, firstInt);
       }
+      success = true;
+      return mapping;
+    } finally {
+      IOUtils.closeSafely(!success, stream);
     }
-    
-    // TODO remove once 3.x is not supported anymore
-    return readLegacyEntries(stream, firstInt);
   }
 
   private static Map<String, FileEntry> readLegacyEntries(IndexInput stream,
@@ -173,32 +191,29 @@ public abstract class CompoundFileDirect
   public synchronized void close() throws IOException {
     if (!isOpen) {
       // allow double close - usually to be consistent with other closeables
-      assert entries == null; 
       return; // already closed
      }
-    entries = null;
     isOpen = false;
     if (writer != null) {
       assert openForWrite;
       writer.close();
+    } else {
+      IOUtils.closeSafely(false, handle);
     }
   }
   
   @Override
-  public synchronized IndexInput openInput(String fileName, IOContext context) throws IOException {
+  public synchronized IndexInput openInput(String name, IOContext context) throws IOException {
     ensureOpen();
     assert !openForWrite;
-    final String id = IndexFileNames.stripSegmentName(fileName);
+    final String id = IndexFileNames.stripSegmentName(name);
     final FileEntry entry = entries.get(id);
     if (entry == null) {
-      throw new IOException("No sub-file with id " + id + " found (fileName=" + fileName + " files: " + entries.keySet() + ")");
+      throw new IOException("No sub-file with id " + id + " found (fileName=" + name + " files: " + entries.keySet() + ")");
     }
-    return openInputSlice(id, entry.offset, entry.length, readBufferSize);
+    return handle.openSlice(entry.offset, entry.length);
   }
   
-  /** Return an IndexInput that represents a "slice" or portion of the CFS file. */
-  public abstract IndexInput openInputSlice(String id, long offset, long length, int readBufferSize) throws IOException;
-  
   /** Returns an array of strings, one for each file in the directory. */
   @Override
   public String[] listAll() {
@@ -279,51 +294,31 @@ public abstract class CompoundFileDirect
   public Lock makeLock(String name) {
     throw new UnsupportedOperationException();
   }
-  
-  @Override
-  public CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
-    FileEntry fileEntry = this.entries.get(IndexFileNames.stripSegmentName(name));
-    if (fileEntry == null) {
-      throw new FileNotFoundException("file " + name + " does not exists in this CFS");
-    }
-    return new NestedCompoundFileDirectory(name, context, fileEntry.offset, fileEntry.length);
-  }
-  
-  /** Not implemented
-  * @throws UnsupportedOperationException */
+
   @Override
-  public CompoundFileDirectory createCompoundOutput(String name, IOContext context)
+  public IndexInputSlicer createSlicer(final String name, IOContext context)
       throws IOException {
-    throw new UnsupportedOperationException("can not create nested CFS, create seperately and use Directory.copy instead");
-  }
-   
-  private class NestedCompoundFileDirectory extends CompoundFileDirectory {
-
-    private final long cfsOffset;
-    private final long cfsLength;
-
-    public NestedCompoundFileDirectory(String fileName, IOContext context, long offset, long length)
-        throws IOException {
-      super(directory, fileName, context);
-      this.cfsOffset = offset;
-      this.cfsLength = length;
-      IndexInput input = null;
-      try {
-        input = CompoundFileDirectory.this.openInput(fileName, IOContext.READONCE);
-        initForRead(CompoundFileDirectory.readEntries(input,
-            CompoundFileDirectory.this, fileName));
-      } finally {
-        IOUtils.closeSafely(false, input);
-      }
+    ensureOpen();
+    assert !openForWrite;
+    final String id = IndexFileNames.stripSegmentName(name);
+    final FileEntry entry = entries.get(id);
+    if (entry == null) {
+      throw new IOException("No sub-file with id " + id + " found (fileName=" + name + " files: " + entries.keySet() + ")");
     }
+    return new IndexInputSlicer() {
+      @Override
+      public void close() throws IOException {
+      }
+      
+      @Override
+      public IndexInput openSlice(long offset, long length) throws IOException {
+        return handle.openSlice(entry.offset + offset, length);
+      }
 
-    @Override
-    public IndexInput openInputSlice(String id, long offset, long length,
-        int readBufferSize) throws IOException {
-      assert offset + length <= cfsLength; 
-      return CompoundFileDirectory.this.openInputSlice(id, cfsOffset + offset, length, readBufferSize);
-    }
-    
+      @Override
+      public IndexInput openFullSlice() throws IOException {
+        return openSlice(0, entry.length);
+      }
+    };
   }
-  
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java Wed Aug 24 16:05:25 2011
@@ -190,6 +190,7 @@ final class CompoundFileWriter implement
   private final long copyFileEntry(IndexOutput dataOut, FileEntry fileEntry)
       throws IOException, MergeAbortedException {
     final IndexInput is = fileEntry.dir.openInput(fileEntry.file, IOContext.READONCE);
+    boolean success = false;
     try {
       final long startPtr = dataOut.getFilePointer();
       final long length = fileEntry.length;
@@ -201,11 +202,14 @@ final class CompoundFileWriter implement
         throw new IOException("Difference in the output file offsets " + diff
             + " does not match the original file length " + length);
       fileEntry.offset = startPtr;
-      // copy successful - delete file
-      fileEntry.dir.deleteFile(fileEntry.file);
+      success = true;
       return length;
     } finally {
-      is.close();
+      IOUtils.closeSafely(!success, is);
+      if (success) {
+        // copy successful - delete file
+        fileEntry.dir.deleteFile(fileEntry.file);
+      }
     }
 
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/Directory.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/Directory.java Wed Aug 24 16:05:25 2011
@@ -112,30 +112,6 @@ public abstract class Directory implemen
   */
   public abstract IndexInput openInput(String name, IOContext context) throws IOException; 
   
-  /** 
-   * Returns a {@link CompoundFileDirectory} capable of
-   * reading the Lucene compound file format.  
-   * <p>
-   * The default implementation returns 
-   * {@link DefaultCompoundFileDirectory}.
-   * @lucene.experimental
-   */
-  public CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
-    return new DefaultCompoundFileDirectory(this, name, context, false);
-  }
-  
-  /** 
-   * Returns a {@link CompoundFileDirectory} capable of
-   * writing the Lucene compound file format.  
-   * <p>
-   * The default implementation returns 
-   * {@link DefaultCompoundFileDirectory}.
-   * @lucene.experimental
-   */
-  public CompoundFileDirectory createCompoundOutput(String name, IOContext context) throws IOException {
-    return new DefaultCompoundFileDirectory(this, name, context, true);
-  }
-
   /** Construct a {@link Lock}.
    * @param name the name of the lock file
    */
@@ -233,10 +209,140 @@ public abstract class Directory implemen
   }
 
   /**
+   * Creates an {@link IndexInputSlicer} for the given file name.
+   * IndexInputSlicer allows other {@link Directory} implementations to
+   * efficiently open one or more sliced {@link IndexInput} instances from a
+   * single file handle. The underlying file handle is kept open until the
+   * {@link IndexInputSlicer} is closed.
+   *
+   * @throws IOException
+   *           if an {@link IOException} occurs
+   * @lucene.internal
+   * @lucene.experimental
+   */
+  public IndexInputSlicer createSlicer(final String name, final IOContext context) throws IOException {
+    ensureOpen();
+    return new IndexInputSlicer() {
+      private final IndexInput base = Directory.this.openInput(name, context);
+      @Override
+      public IndexInput openSlice(long offset, long length) {
+        return new SlicedIndexInput(base, offset, length);
+      }
+      @Override
+      public void close() throws IOException {
+        base.close();
+      }
+      @Override
+      public IndexInput openFullSlice() throws IOException {
+        return (IndexInput) base.clone();
+      }
+    };
+  }
+
+  /**
    * @throws AlreadyClosedException if this Directory is closed
    */
   protected final void ensureOpen() throws AlreadyClosedException {
     if (!isOpen)
       throw new AlreadyClosedException("this Directory is closed");
   }
+  
+  /**
+   * Allows to create one or more sliced {@link IndexInput} instances from a single 
+   * file handle. Some {@link Directory} implementations may be able to efficiently map slices of a file
+   * into memory when only certain parts of a file are required.   
+   * @lucene.internal
+   * @lucene.experimental
+   */
+  public abstract class IndexInputSlicer implements Closeable {
+    /**
+     * Returns an {@link IndexInput} slice starting at the given offset with the given length.
+     */
+    public abstract IndexInput openSlice(long offset, long length) throws IOException;
+
+    /**
+     * Returns an {@link IndexInput} slice starting at offset <i>0</i> with a
+     * length equal to the length of the underlying file
+     */
+    public abstract IndexInput openFullSlice() throws IOException;
+  }
+  
+  /** Implementation of an IndexInput that reads from a portion of
+   *  a file.
+   */
+  private static final class SlicedIndexInput extends BufferedIndexInput {
+    IndexInput base;
+    long fileOffset;
+    long length;
+    
+    SlicedIndexInput(final IndexInput base, final long fileOffset, final long length) {
+      this(base, fileOffset, length, BufferedIndexInput.BUFFER_SIZE);
+    }
+    
+    SlicedIndexInput(final IndexInput base, final long fileOffset, final long length, int readBufferSize) {
+      super(readBufferSize);
+      this.base = (IndexInput) base.clone();
+      this.fileOffset = fileOffset;
+      this.length = length;
+    }
+    
+    @Override
+    public Object clone() {
+      SlicedIndexInput clone = (SlicedIndexInput)super.clone();
+      clone.base = (IndexInput)base.clone();
+      clone.fileOffset = fileOffset;
+      clone.length = length;
+      return clone;
+    }
+    
+    /** Expert: implements buffer refill.  Reads bytes from the current
+     *  position in the input.
+     * @param b the array to read bytes into
+     * @param offset the offset in the array to start storing bytes
+     * @param len the number of bytes to read
+     */
+    @Override
+    protected void readInternal(byte[] b, int offset, int len) throws IOException {
+      long start = getFilePointer();
+      if(start + len > length)
+        throw new IOException("read past EOF");
+      base.seek(fileOffset + start);
+      base.readBytes(b, offset, len, false);
+    }
+    
+    /** Expert: implements seek.  Sets current position in this file, where
+     *  the next {@link #readInternal(byte[],int,int)} will occur.
+     * @see #readInternal(byte[],int,int)
+     */
+    @Override
+    protected void seekInternal(long pos) {}
+    
+    /** Closes the stream to further operations. */
+    @Override
+    public void close() throws IOException {
+      base.close();
+    }
+    
+    @Override
+    public long length() {
+      return length;
+    }
+    
+    @Override
+    public void copyBytes(IndexOutput out, long numBytes) throws IOException {
+      // Copy first whatever is in the buffer
+      numBytes -= flushBuffer(out, numBytes);
+      
+      // If there are more bytes left to copy, delegate the copy task to the
+      // base IndexInput, in case it can do an optimized copy.
+      if (numBytes > 0) {
+        long start = getFilePointer();
+        if (start + numBytes > length) {
+          throw new IOException("read past EOF");
+        }
+        base.seek(fileOffset + start);
+        base.copyBytes(out, numBytes);
+      }
+    }
+  }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java Wed Aug 24 16:05:25 2011
@@ -177,19 +177,9 @@ public class FileSwitchDirectory extends
     return getDirectory(name).openInput(name, context);
   }
 
-  // final due to LUCENE-3380: currently CFS backdoors the directory to create CFE
-  // by using the basic implementation and not delegating, we ensure that all 
-  // openInput/createOutput requests come thru NRTCachingDirectory.
   @Override
-  public final CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
-    return super.openCompoundInput(name, context);
-  }
-  
-  // final due to LUCENE-3380: currently CFS backdoors the directory to create CFE
-  // by using the basic implementation and not delegating, we ensure that all 
-  // openInput/createOutput requests come thru NRTCachingDirectory.
-  @Override
-  public final CompoundFileDirectory createCompoundOutput(String name, IOContext context) throws IOException {
-    return super.createCompoundOutput(name, context);
+  public IndexInputSlicer createSlicer(String name, IOContext context)
+      throws IOException {
+    return getDirectory(name).createSlicer(name, context);
   }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/MMapDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/MMapDirectory.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/MMapDirectory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/MMapDirectory.java Wed Aug 24 16:05:25 2011
@@ -32,7 +32,6 @@ import java.security.PrivilegedActionExc
 import java.lang.reflect.Method;
 
 import org.apache.lucene.util.Constants;
-import org.apache.lucene.util.IOUtils;
 
 /** File-based {@link Directory} implementation that uses
  *  mmap for reading, and {@link
@@ -220,42 +219,26 @@ public class MMapDirectory extends FSDir
     }
   }
   
-  @Override
-  public CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
-    return new MMapCompoundFileDirectory(name, context);
-  }
-  
-  private final class MMapCompoundFileDirectory extends CompoundFileDirectory {
-    private RandomAccessFile raf = null;
-
-    public MMapCompoundFileDirectory(String fileName, IOContext context) throws IOException {
-      super(MMapDirectory.this, fileName, context);
-      IndexInput stream = null;
-      try {
-        File f = new File(MMapDirectory.this.getDirectory(), fileName);
-        raf = new RandomAccessFile(f, "r");
-        stream = new MMapIndexInput(raf, 0, raf.length(), chunkSizePower);
-        initForRead(CompoundFileDirectory.readEntries(stream, MMapDirectory.this, fileName));
-        stream.close();
-      } catch (IOException e) {
-        // throw our original exception
-        IOUtils.closeSafely(e, raf, stream);
+  public IndexInputSlicer createSlicer(final String name, final IOContext context) throws IOException {
+    ensureOpen();
+    File f = new File(getDirectory(), name);
+    final RandomAccessFile raf = new RandomAccessFile(f, "r");
+    return new IndexInputSlicer() {
+      @Override
+      public void close() throws IOException {
+        raf.close();
       }
-    }
 
-    @Override
-    public IndexInput openInputSlice(String id, long offset, long length, int readBufferSize) throws IOException {
-      return new MMapIndexInput(raf, offset, length, chunkSizePower);
-    }
+      @Override
+      public IndexInput openSlice(long offset, long length) throws IOException {
+        return new MMapIndexInput(raf, offset, length, chunkSizePower);
+      }
 
-    @Override
-    public synchronized void close() throws IOException {
-      try {
-        raf.close();
-      } finally {
-        super.close();
+      @Override
+      public IndexInput openFullSlice() throws IOException {
+        return openSlice(0, raf.length());
       }
-    }
+    };
   }
 
   // Because Java's ByteBuffer uses an int to address the

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java Wed Aug 24 16:05:25 2011
@@ -24,8 +24,7 @@ import java.nio.channels.ClosedChannelEx
 import java.nio.channels.FileChannel;
 import java.util.concurrent.Future; // javadoc
 
-import org.apache.lucene.store.SimpleFSDirectory.SimpleFSIndexInput;
-import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.store.SimpleFSDirectory.SimpleFSIndexInput.Descriptor;
 
 /**
  * An {@link FSDirectory} implementation that uses java.nio's FileChannel's
@@ -81,45 +80,29 @@ public class NIOFSDirectory extends FSDi
     return new NIOFSIndexInput(new File(getDirectory(), name), context, getReadChunkSize());
   }
   
-  @Override
-  public CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
-    return new NIOFSCompoundFileDirectory(name, context);
-  }
+  public IndexInputSlicer createSlicer(final String name,
+      final IOContext context) throws IOException {
+    ensureOpen();
+    final File file = new File(getDirectory(), name);
+    final Descriptor descriptor = new Descriptor(file, "r");
+    return new Directory.IndexInputSlicer() {
 
-  private final class NIOFSCompoundFileDirectory extends CompoundFileDirectory {
-    private SimpleFSIndexInput.Descriptor fd;
-    private FileChannel fc;
-
-    public NIOFSCompoundFileDirectory(String fileName, IOContext context) throws IOException {
-      super(NIOFSDirectory.this, fileName, context);
-      IndexInput stream = null;
-      try {
-        File f = new File(NIOFSDirectory.this.getDirectory(), fileName);
-        fd = new SimpleFSIndexInput.Descriptor(f, "r");
-        fc = fd.getChannel();
-        stream = new NIOFSIndexInput(fd, fc, 0, fd.length, readBufferSize,
-            getReadChunkSize());
-        initForRead(CompoundFileDirectory.readEntries(stream, NIOFSDirectory.this, fileName));
-        stream.close();
-      } catch (IOException e) {
-        // throw our original exception
-        IOUtils.closeSafely(e, fc, fd, stream);
+      @Override
+      public void close() throws IOException {
+        descriptor.close();
       }
-    }
-    
-    @Override
-    public IndexInput openInputSlice(String id, long offset, long length, int readBufferSize) throws IOException {
-      return new NIOFSIndexInput(fd, fc, offset, length, readBufferSize, getReadChunkSize());
-    }
 
-    @Override
-    public synchronized void close() throws IOException {
-      try {
-        IOUtils.closeSafely(false, fc, fd);
-      } finally {
-        super.close();
+      @Override
+      public IndexInput openSlice(long offset, long length) throws IOException {
+        return new NIOFSIndexInput(descriptor, descriptor.getChannel(), offset,
+            length, BufferedIndexInput.bufferSize(context), getReadChunkSize());
       }
-    }
+
+      @Override
+      public IndexInput openFullSlice() throws IOException {
+        return openSlice(0, descriptor.length);
+      }
+    };
   }
 
   protected static class NIOFSIndexInput extends SimpleFSDirectory.SimpleFSIndexInput {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java Wed Aug 24 16:05:25 2011
@@ -21,8 +21,7 @@ import java.io.File;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 
-import org.apache.lucene.util.IOUtils;
-
+import org.apache.lucene.store.SimpleFSDirectory.SimpleFSIndexInput.Descriptor;
 
 /** A straightforward implementation of {@link FSDirectory}
  *  using java.io.RandomAccessFile.  However, this class has
@@ -59,45 +58,34 @@ public class SimpleFSDirectory extends F
     return new SimpleFSIndexInput(new File(directory, name), context, getReadChunkSize());
   }
   
-  @Override
-  public CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
-    return new SimpleFSCompoundFileDirectory(name, context);
-  }
+  
 
-  private final class SimpleFSCompoundFileDirectory extends CompoundFileDirectory {
-    private SimpleFSIndexInput.Descriptor fd;
+  public IndexInputSlicer createSlicer(final String name,
+      final IOContext context) throws IOException {
+    ensureOpen();
+    final File file = new File(getDirectory(), name);
+    final Descriptor descriptor = new Descriptor(file, "r");
+    return new IndexInputSlicer() {
 
-    public SimpleFSCompoundFileDirectory(String fileName, IOContext context) throws IOException {
-      super(SimpleFSDirectory.this, fileName, context);
-      IndexInput stream = null;
-      try {
-        final File f = new File(SimpleFSDirectory.this.getDirectory(), fileName);
-        fd = new SimpleFSIndexInput.Descriptor(f, "r");
-        stream = new SimpleFSIndexInput(fd, 0, fd.length, readBufferSize,
-            getReadChunkSize());
-        initForRead(CompoundFileDirectory.readEntries(stream, SimpleFSDirectory.this, fileName));
-        stream.close();
-      } catch (IOException e) {
-        // throw our original exception
-        IOUtils.closeSafely(e, fd, stream);
+      @Override
+      public void close() throws IOException {
+        descriptor.close();
       }
-    }
 
-    @Override
-    public IndexInput openInputSlice(String id, long offset, long length, int readBufferSize) throws IOException {
-      return new SimpleFSIndexInput(fd, offset, length, readBufferSize, getReadChunkSize());
-    }
+      @Override
+      public IndexInput openSlice(long offset, long length) throws IOException {
+        return new SimpleFSIndexInput(descriptor, offset,
+            length, BufferedIndexInput.bufferSize(context), getReadChunkSize());
+      }
 
-    @Override
-    public synchronized void close() throws IOException {
-      try {
-        fd.close();
-      } finally {
-        super.close();
+      @Override
+      public IndexInput openFullSlice() throws IOException {
+        return openSlice(0, descriptor.length);
       }
-    }
+    };
   }
 
+
   protected static class SimpleFSIndexInput extends BufferedIndexInput {
   
     protected static class Descriptor extends RandomAccessFile {

Modified: lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java Wed Aug 24 16:05:25 2011
@@ -377,7 +377,7 @@ public class MockDirectoryWrapper extend
     
     //System.out.println(Thread.currentThread().getName() + ": MDW: create " + name);
     IndexOutput io = new MockIndexOutputWrapper(this, delegate.createOutput(name, LuceneTestCase.newIOContext(randomState)), name);
-    addFileHandle(io, name, false);
+    addFileHandle(io, name, Handle.Output);
     openFilesForWrite.add(name);
     
     // throttling REALLY slows down tests, so don't do it very often for SOMETIMES.
@@ -391,8 +391,12 @@ public class MockDirectoryWrapper extend
       return io;
     }
   }
+  
+  private static enum Handle {
+    Input, Output, Slice
+  }
 
-  synchronized void addFileHandle(Closeable c, String name, boolean input) {
+  synchronized void addFileHandle(Closeable c, String name, Handle handle) {
     Integer v = openFiles.get(name);
     if (v != null) {
       v = Integer.valueOf(v.intValue()+1);
@@ -401,7 +405,7 @@ public class MockDirectoryWrapper extend
       openFiles.put(name, Integer.valueOf(1));
     }
     
-    openFileHandles.put(c, new RuntimeException("unclosed Index" + (input ? "Input" : "Output") + ": " + name));
+    openFileHandles.put(c, new RuntimeException("unclosed Index" + handle.name() + ": " + name));
   }
   
   @Override
@@ -417,22 +421,10 @@ public class MockDirectoryWrapper extend
     }
 
     IndexInput ii = new MockIndexInputWrapper(this, name, delegate.openInput(name, LuceneTestCase.newIOContext(randomState)));
-    addFileHandle(ii, name, true);
+    addFileHandle(ii, name, Handle.Input);
     return ii;
   }
   
-  @Override
-  public synchronized CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
-    maybeYield();
-    return new MockCompoundFileDirectoryWrapper(name, this, delegate.openCompoundInput(name, context), false);
-  }
-   
-  @Override
-  public CompoundFileDirectory createCompoundOutput(String name, IOContext context) throws IOException {
-    maybeYield();
-    return new MockCompoundFileDirectoryWrapper(name, this, delegate.createCompoundOutput(name, context), true);
-  }
-
   /** Provided for testing purposes.  Use sizeInBytes() instead. */
   public synchronized final long getRecomputedSizeInBytes() throws IOException {
     if (!(delegate instanceof RAMDirectory))
@@ -658,5 +650,50 @@ public class MockDirectoryWrapper extend
     // randomize the IOContext here?
     delegate.copy(to, src, dest, context);
   }
-  
+
+  @Override
+  public IndexInputSlicer createSlicer(final String name, IOContext context)
+      throws IOException {
+    maybeYield();
+    if (!delegate.fileExists(name))
+      throw new FileNotFoundException(name);
+    // cannot open a file for input if it's still open for
+    // output, except for segments.gen and segments_N
+    if (openFilesForWrite.contains(name) && !name.startsWith("segments")) {
+      throw fillOpenTrace(new IOException("MockDirectoryWrapper: file \"" + name + "\" is still open for writing"), name, false);
+    }
+    
+    final IndexInputSlicer delegateHandle = delegate.createSlicer(name, context);
+    final IndexInputSlicer handle = new IndexInputSlicer() {
+      
+      private boolean isClosed;
+      @Override
+      public void close() throws IOException {
+        if (!isClosed) {
+          delegateHandle.close();
+          MockDirectoryWrapper.this.removeOpenFile(this, name);
+          isClosed = true;
+        }
+      }
+
+      @Override
+      public IndexInput openSlice(long offset, long length) throws IOException {
+        maybeYield();
+        IndexInput ii = new MockIndexInputWrapper(MockDirectoryWrapper.this, name, delegateHandle.openSlice(offset, length));
+        addFileHandle(ii, name, Handle.Input);
+        return ii;
+      }
+
+      @Override
+      public IndexInput openFullSlice() throws IOException {
+        maybeYield();
+        IndexInput ii = new MockIndexInputWrapper(MockDirectoryWrapper.this, name, delegateHandle.openFullSlice());
+        addFileHandle(ii, name, Handle.Input);
+        return ii;
+      }
+      
+    };
+    addFileHandle(handle, name, Handle.Slice);
+    return handle;
+  }
 }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Wed Aug 24 16:05:25 2011
@@ -548,7 +548,7 @@ public class TestBackwardsCompatibility 
       // figure out which field number corresponds to
       // "content", and then set our expected file names below
       // accordingly:
-      CompoundFileDirectory cfsReader = dir.openCompoundInput("_0.cfs", newIOContext(random));
+      CompoundFileDirectory cfsReader = new CompoundFileDirectory(dir, "_0.cfs", newIOContext(random), false);
       FieldInfos fieldInfos = new FieldInfos(cfsReader, "_0.fnm");
       int contentFieldIndex = -1;
       for (FieldInfo fi : fieldInfos) {

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCompoundFile.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCompoundFile.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCompoundFile.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCompoundFile.java Wed Aug 24 16:05:25 2011
@@ -182,11 +182,11 @@ public class TestCompoundFile extends Lu
         for (int i=0; i<data.length; i++) {
             String name = "t" + data[i];
             createSequenceFile(dir, name, (byte) 0, data[i]);
-            CompoundFileDirectory csw = dir.createCompoundOutput(name + ".cfs", newIOContext(random));
+            CompoundFileDirectory csw = new CompoundFileDirectory(dir, name + ".cfs", newIOContext(random), true);
             dir.copy(csw, name, name, newIOContext(random));
             csw.close();
 
-            CompoundFileDirectory csr = dir.openCompoundInput(name + ".cfs", newIOContext(random));
+            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);
@@ -205,12 +205,12 @@ public class TestCompoundFile extends Lu
         createSequenceFile(dir, "d1", (byte) 0, 15);
         createSequenceFile(dir, "d2", (byte) 0, 114);
 
-        CompoundFileDirectory csw = dir.createCompoundOutput("d.cfs", newIOContext(random));
+        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 = dir.openCompoundInput("d.cfs", newIOContext(random));
+        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);
@@ -255,7 +255,7 @@ public class TestCompoundFile extends Lu
         createRandomFile(dir, segment + ".notIn2", 51);
 
         // Now test
-        CompoundFileDirectory csw = dir.createCompoundOutput("test.cfs", newIOContext(random));
+        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"
@@ -266,7 +266,7 @@ public class TestCompoundFile extends Lu
         }
         csw.close();
 
-        CompoundFileDirectory csr = dir.openCompoundInput("test.cfs", newIOContext(random));
+        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));
@@ -285,7 +285,7 @@ public class TestCompoundFile extends Lu
      *  the size of each file is 1000 bytes.
      */
     private void setUp_2() throws IOException {
-        CompoundFileDirectory cw = dir.createCompoundOutput("f.comp", newIOContext(random));
+        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;
@@ -336,7 +336,7 @@ public class TestCompoundFile extends Lu
 
     public void testClonedStreamsClosing() throws IOException {
         setUp_2();
-        CompoundFileDirectory cr = dir.openCompoundInput("f.comp", newIOContext(random));
+        CompoundFileDirectory cr = new CompoundFileDirectory(dir, "f.comp", newIOContext(random), false);
 
         // basic clone
         IndexInput expected = dir.openInput("f11", newIOContext(random));
@@ -388,7 +388,7 @@ public class TestCompoundFile extends Lu
      */
     public void testRandomAccess() throws IOException {
         setUp_2();
-        CompoundFileDirectory cr = dir.openCompoundInput("f.comp", newIOContext(random));
+        CompoundFileDirectory cr = new CompoundFileDirectory(dir, "f.comp", newIOContext(random), false);
 
         // Open two files
         IndexInput e1 = dir.openInput("f11", newIOContext(random));
@@ -467,7 +467,7 @@ public class TestCompoundFile extends Lu
      */
     public void testRandomAccessClones() throws IOException {
         setUp_2();
-        CompoundFileDirectory cr = dir.openCompoundInput("f.comp", newIOContext(random));
+        CompoundFileDirectory cr = new CompoundFileDirectory(dir, "f.comp", newIOContext(random), false);
 
         // Open two files
         IndexInput e1 = cr.openInput("f11", newIOContext(random));
@@ -544,7 +544,7 @@ public class TestCompoundFile extends Lu
 
     public void testFileNotFound() throws IOException {
         setUp_2();
-        CompoundFileDirectory cr = dir.openCompoundInput("f.comp", newIOContext(random));
+        CompoundFileDirectory cr = new CompoundFileDirectory(dir, "f.comp", newIOContext(random), false);
 
         // Open two files
         try {
@@ -562,7 +562,7 @@ public class TestCompoundFile extends Lu
 
     public void testReadPastEOF() throws IOException {
         setUp_2();
-        CompoundFileDirectory cr = dir.openCompoundInput("f.comp", newIOContext(random));
+        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];
@@ -615,11 +615,11 @@ public class TestCompoundFile extends Lu
        createSequenceFile(dir, "d1", (byte) 0, 15);
 
        Directory newDir = newDirectory();
-       CompoundFileDirectory csw = newDir.createCompoundOutput("d.cfs", newIOContext(random));
+       CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), true);
        dir.copy(csw, "d1", "d1", newIOContext(random));
        csw.close();
 
-       CompoundFileDirectory csr = newDir.openCompoundInput("d.cfs", newIOContext(random));
+       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);
@@ -634,7 +634,7 @@ public class TestCompoundFile extends Lu
    
   public void testAppend() throws IOException {
     Directory newDir = newDirectory();
-    CompoundFileDirectory csw = newDir.createCompoundOutput("d.cfs", newIOContext(random));
+    CompoundFileDirectory csw = new CompoundFileDirectory(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));
@@ -652,7 +652,7 @@ public class TestCompoundFile extends Lu
     assertEquals(1, listAll.length);
     assertEquals("d.cfs", listAll[0]);
     csw.close();
-    CompoundFileDirectory csr = newDir.openCompoundInput("d.cfs", newIOContext(random));
+    CompoundFileDirectory csr = new CompoundFileDirectory(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());
@@ -675,7 +675,7 @@ public class TestCompoundFile extends Lu
   
   public void testAppendTwice() throws IOException {
     Directory newDir = newDirectory();
-    CompoundFileDirectory csw = newDir.createCompoundOutput("d.cfs", newIOContext(random));
+    CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), true);
     createSequenceFile(newDir, "d1", (byte) 0, 15);
     IndexOutput out = csw.createOutput("d.xyz", newIOContext(random));
     out.writeInt(0);
@@ -691,7 +691,7 @@ public class TestCompoundFile extends Lu
    
     csw.close();
 
-    CompoundFileDirectory cfr = newDir.openCompoundInput("d.cfs", newIOContext(random));
+    CompoundFileDirectory cfr = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), false);
     assertEquals(1, cfr.listAll().length);
     assertEquals("d.xyz", cfr.listAll()[0]);
     cfr.close();
@@ -700,10 +700,10 @@ public class TestCompoundFile extends Lu
   
   public void testEmptyCFS() throws IOException {
     Directory newDir = newDirectory();
-    CompoundFileDirectory csw = newDir.createCompoundOutput("d.cfs", newIOContext(random));
+    CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), true);
     csw.close();
 
-    CompoundFileDirectory csr = newDir.openCompoundInput("d.cfs", newIOContext(random));
+    CompoundFileDirectory csr = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), false);
     assertEquals(0, csr.listAll().length);
     csr.close();
 
@@ -712,8 +712,8 @@ public class TestCompoundFile extends Lu
   
   public void testReadNestedCFP() throws IOException {
     Directory newDir = newDirectory();
-    CompoundFileDirectory csw = newDir.createCompoundOutput("d.cfs", newIOContext(random));
-    CompoundFileDirectory nested = newDir.createCompoundOutput("b.cfs", newIOContext(random));
+    CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), true);
+    CompoundFileDirectory nested = new CompoundFileDirectory(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);
@@ -728,10 +728,10 @@ public class TestCompoundFile extends Lu
     csw.close();
     
     assertEquals(2, newDir.listAll().length);
-    csw = newDir.openCompoundInput("d.cfs", newIOContext(random));
+    csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), false);
     
     assertEquals(2, csw.listAll().length);
-    nested = csw.openCompoundInput("b.cfs", newIOContext(random));
+    nested = new CompoundFileDirectory(csw, "b.cfs", newIOContext(random), false);
     
     assertEquals(2, nested.listAll().length);
     IndexInput openInput = nested.openInput("b.xyz", newIOContext(random));
@@ -747,7 +747,7 @@ public class TestCompoundFile extends Lu
   
   public void testDoubleClose() throws IOException {
     Directory newDir = newDirectory();
-    CompoundFileDirectory csw = newDir.createCompoundOutput("d.cfs", newIOContext(random));
+    CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), true);
     IndexOutput out = csw.createOutput("d.xyz", newIOContext(random));
     out.writeInt(0);
     out.close();
@@ -756,7 +756,7 @@ public class TestCompoundFile extends Lu
     // close a second time - must have no effect according to Closeable
     csw.close();
     
-    csw = newDir.openCompoundInput("d.cfs", newIOContext(random));
+    csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), false);
     IndexInput openInput = csw.openInput("d.xyz", newIOContext(random));
     assertEquals(0, openInput.readInt());
     openInput.close();
@@ -781,7 +781,7 @@ public class TestCompoundFile extends Lu
       out.close();
     }
     
-    final CompoundFileDirectory cfd = d.createCompoundOutput("c.cfs", newIOContext(random));
+    final CompoundFileDirectory cfd = new CompoundFileDirectory(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));
@@ -789,7 +789,7 @@ public class TestCompoundFile extends Lu
     cfd.close();
 
     final IndexInput[] ins = new IndexInput[FILE_COUNT];
-    final CompoundFileDirectory cfr = d.openCompoundInput("c.cfs", newIOContext(random));
+    final CompoundFileDirectory cfr = new CompoundFileDirectory(d, "c.cfs", newIOContext(random), false);
     for(int fileIdx=0;fileIdx<FILE_COUNT;fileIdx++) {
       ins[fileIdx] = cfr.openInput("file." + fileIdx, newIOContext(random));
     }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java Wed Aug 24 16:05:25 2011
@@ -91,7 +91,7 @@ public class TestIndexFileDeleter extend
     // figure out which field number corresponds to
     // "content", and then set our expected file names below
     // accordingly:
-    CompoundFileDirectory cfsReader = dir.openCompoundInput("_2.cfs", newIOContext(random));
+    CompoundFileDirectory cfsReader = new CompoundFileDirectory(dir, "_2.cfs", newIOContext(random), false);
     FieldInfos fieldInfos = new FieldInfos(cfsReader, "_2.fnm");
     int contentFieldIndex = -1;
     for (FieldInfo fi : fieldInfos) {

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/index.34.cfs.zip
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/index.34.cfs.zip?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
Binary files - no diff available.

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/index.34.nocfs.zip
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/index.34.nocfs.zip?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
Binary files - no diff available.

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java?rev=1161183&r1=1161182&r2=1161183&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java Wed Aug 24 16:05:25 2011
@@ -116,7 +116,7 @@ public class TestFileSwitchDirectory ext
   // LUCENE-3380 test that delegate compound files correctly.
   public void testCompoundFileAppendTwice() throws IOException {
     Directory newDir = newFSSwitchDirectory(Collections.singleton("cfs"));
-    CompoundFileDirectory csw = newDir.createCompoundOutput("d.cfs", newIOContext(random));
+    CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), true);
     createSequenceFile(newDir, "d1", (byte) 0, 15);
     IndexOutput out = csw.createOutput("d.xyz", newIOContext(random));
     out.writeInt(0);
@@ -132,7 +132,7 @@ public class TestFileSwitchDirectory ext
    
     csw.close();
 
-    CompoundFileDirectory cfr = newDir.openCompoundInput("d.cfs", newIOContext(random));
+    CompoundFileDirectory cfr = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), false);
     assertEquals(1, cfr.listAll().length);
     assertEquals("d.xyz", cfr.listAll()[0]);
     cfr.close();