Index: lucene/CHANGES.txt
===================================================================
--- lucene/CHANGES.txt	(revision 1137870)
+++ lucene/CHANGES.txt	(working copy)
@@ -454,6 +454,14 @@
   MultiTermQuery now stores TermState per leaf reader during rewrite to re-
   seek the term dictionary in TermQuery / TermWeight.
   (Simon Willnauer, Mike McCandless, Robert Muir)
+  
+* LUCENE-3201, LUCENE-3218: CompoundFileSystem code has been consolidated 
+  into a Directory implementation. Reading is optimized for MMapDirectory,
+  NIOFSDirectory and SimpleFSDirectory to only map requested parts of the
+  CFS into an IndexInput. Writing to a CFS now tries to append to the CF
+  directly if possible and merges separately written files on the fly instead
+  of during close. (Simon Willnauer, Robert Muir)
+  
 
 Bug fixes
 
Index: lucene/contrib/misc/src/java/org/apache/lucene/store/NRTCachingDirectory.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/store/NRTCachingDirectory.java	(revision 1137870)
+++ lucene/contrib/misc/src/java/org/apache/lucene/store/NRTCachingDirectory.java	(working copy)
@@ -234,6 +234,25 @@
   }
 
   @Override
+  public synchronized CompoundFileDirectory openCompoundInput(String name, int bufferSize) throws IOException {
+    if (cache.fileExists(name)) {
+      return cache.openCompoundInput(name, bufferSize);
+    } else {
+      return delegate.openCompoundInput(name, bufferSize);
+    }
+  }
+  
+  @Override
+  public synchronized CompoundFileDirectory openCompoundOutput(String name)
+      throws IOException {
+    if (cache.fileExists(name)) {
+      throw new IOException("File " + name + "already exists");
+    } else {
+      return delegate.openCompoundOutput(name);
+    }
+  }
+
+  @Override
   public synchronized IndexInput openInput(String name, int bufferSize) throws IOException {
     if (cache.fileExists(name)) {
       return cache.openInput(name, bufferSize);
Index: lucene/contrib/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java
===================================================================
--- lucene/contrib/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java	(revision 1137870)
+++ lucene/contrib/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java	(working copy)
@@ -78,7 +78,7 @@
     _TestUtil.rmDir(destDir2);
     destDir2.mkdirs();
     IndexSplitter.main(new String[] {dir.getAbsolutePath(), destDir2.getAbsolutePath(), splitSegName});
-    assertEquals(3, destDir2.listFiles().length);
+    assertEquals(4, destDir2.listFiles().length);
     Directory fsDirDest2 = newFSDirectory(destDir2);
     r = IndexReader.open(fsDirDest2, true);
     assertEquals(50, r.maxDoc());
Index: lucene/src/java/org/apache/lucene/index/CompoundFileReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/CompoundFileReader.java	(revision 1137870)
+++ lucene/src/java/org/apache/lucene/index/CompoundFileReader.java	(working copy)
@@ -1,307 +0,0 @@
-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 org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.BufferedIndexInput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.Lock;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-
-/**
- * Class for accessing a compound stream.
- * This class implements a directory, but is limited to only read operations.
- * Directory methods that would normally modify data throw an exception.
- * @lucene.experimental
- */
-public class CompoundFileReader extends Directory {
-  
-  private int readBufferSize;
-  
-  private static final class FileEntry {
-    long offset;
-    long length;
-  }
-  
-  // Base info
-  private Directory directory;
-  private String fileName;
-  
-  private IndexInput stream;
-  private HashMap<String,FileEntry> entries = new HashMap<String,FileEntry>();
-  
-  public CompoundFileReader(Directory dir, String name) throws IOException {
-    this(dir, name, BufferedIndexInput.BUFFER_SIZE);
-  }
-  
-  public CompoundFileReader(Directory dir, String name, int readBufferSize) throws IOException {
-    assert !(dir instanceof CompoundFileReader) : "compound file inside of compound file: " + name;
-    directory = dir;
-    fileName = name;
-    this.readBufferSize = readBufferSize;
-    
-    boolean success = false;
-    
-    try {
-      stream = dir.openInput(name, readBufferSize);
-      
-      // read the first VInt. If it is negative, it's the version number
-      // otherwise it's the count (pre-3.1 indexes)
-      int firstInt = stream.readVInt();
-      
-      final int count;
-      final boolean stripSegmentName;
-      if (firstInt < CompoundFileWriter.FORMAT_PRE_VERSION) {
-        if (firstInt < CompoundFileWriter.FORMAT_CURRENT) {
-          throw new CorruptIndexException("Incompatible format version: "
-              + firstInt + " expected " + CompoundFileWriter.FORMAT_CURRENT);
-        }
-        // It's a post-3.1 index, read the count.
-        count = stream.readVInt();
-        stripSegmentName = false;
-      } else {
-        count = firstInt;
-        stripSegmentName = true;
-      }
-      
-      // read the directory and init files
-      FileEntry entry = null;
-      for (int i=0; i<count; i++) {
-        long offset = stream.readLong();
-        String id = stream.readString();
-        
-        if (stripSegmentName) {
-          // Fix the id to not include the segment names. This is relevant for
-          // pre-3.1 indexes.
-          id = IndexFileNames.stripSegmentName(id);
-        }
-        
-        if (entry != null) {
-          // set length of the previous entry
-          entry.length = offset - entry.offset;
-        }
-        
-        entry = new FileEntry();
-        entry.offset = offset;
-        entries.put(id, entry);
-      }
-      
-      // set the length of the final entry
-      if (entry != null) {
-        entry.length = stream.length() - entry.offset;
-      }
-      
-      success = true;
-      
-    } finally {
-      if (!success && (stream != null)) {
-        try {
-          stream.close();
-        } catch (IOException e) { }
-      }
-    }
-  }
-  
-  public Directory getDirectory() {
-    return directory;
-  }
-  
-  public String getName() {
-    return fileName;
-  }
-  
-  @Override
-  public synchronized void close() throws IOException {
-    if (stream == null)
-      throw new IOException("Already closed");
-    
-    entries.clear();
-    stream.close();
-    stream = null;
-  }
-  
-  @Override
-  public synchronized IndexInput openInput(String id) throws IOException {
-    // Default to readBufferSize passed in when we were opened
-    return openInput(id, readBufferSize);
-  }
-  
-  @Override
-  public synchronized IndexInput openInput(String id, int readBufferSize) throws IOException {
-    if (stream == null)
-      throw new IOException("Stream closed");
-    
-    id = IndexFileNames.stripSegmentName(id);
-    final FileEntry entry = entries.get(id);
-    if (entry == null)
-      throw new IOException("No sub-file with id " + id + " found (files: " + entries.keySet() + ")");
-    
-    return new CSIndexInput(stream, entry.offset, entry.length, readBufferSize);
-  }
-  
-  /** Returns an array of strings, one for each file in the directory. */
-  @Override
-  public String[] listAll() {
-    String[] res = entries.keySet().toArray(new String[entries.size()]);
-    // Add the segment name
-    String seg = fileName.substring(0, fileName.indexOf('.'));
-    for (int i = 0; i < res.length; i++) {
-      res[i] = seg + res[i];
-    }
-    return res;
-  }
-  
-  /** Returns true iff a file with the given name exists. */
-  @Override
-  public boolean fileExists(String name) {
-    return entries.containsKey(IndexFileNames.stripSegmentName(name));
-  }
-  
-  /** Returns the time the compound file was last modified. */
-  @Override
-  public long fileModified(String name) throws IOException {
-    return directory.fileModified(fileName);
-  }
-  
-  /** Not implemented
-   * @throws UnsupportedOperationException */
-  @Override
-  public void deleteFile(String name) {
-    throw new UnsupportedOperationException();
-  }
-  
-  /** Not implemented
-   * @throws UnsupportedOperationException */
-  public void renameFile(String from, String to) {
-    throw new UnsupportedOperationException();
-  }
-  
-  /** Returns the length of a file in the directory.
-   * @throws IOException if the file does not exist */
-  @Override
-  public long fileLength(String name) throws IOException {
-    FileEntry e = entries.get(IndexFileNames.stripSegmentName(name));
-    if (e == null)
-      throw new FileNotFoundException(name);
-    return e.length;
-  }
-  
-  /** Not implemented
-   * @throws UnsupportedOperationException */
-  @Override
-  public IndexOutput createOutput(String name) {
-    throw new UnsupportedOperationException();
-  }
-  
-  @Override
-  public void sync(Collection<String> names) throws IOException {
-  }
-  
-  /** Not implemented
-   * @throws UnsupportedOperationException */
-  @Override
-  public Lock makeLock(String name) {
-    throw new UnsupportedOperationException();
-  }
-  
-  /** Implementation of an IndexInput that reads from a portion of the
-   *  compound file. The visibility is left as "package" *only* because
-   *  this helps with testing since JUnit test cases in a different class
-   *  can then access package fields of this class.
-   */
-  static final class CSIndexInput extends BufferedIndexInput {
-    IndexInput base;
-    long fileOffset;
-    long length;
-    
-    CSIndexInput(final IndexInput base, final long fileOffset, final long length) {
-      this(base, fileOffset, length, BufferedIndexInput.BUFFER_SIZE);
-    }
-    
-    CSIndexInput(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() {
-      CSIndexInput clone = (CSIndexInput)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);
-      }
-    }
-  }
-}
Index: lucene/src/java/org/apache/lucene/index/CompoundFileWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/CompoundFileWriter.java	(revision 1137870)
+++ lucene/src/java/org/apache/lucene/index/CompoundFileWriter.java	(working copy)
@@ -1,252 +0,0 @@
-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 java.util.HashSet;
-import java.util.LinkedList;
-
-import org.apache.lucene.index.codecs.MergeState;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.IOUtils;
-
-/**
- * Combines multiple files into a single compound file.
- * The file format:<br>
- * <ul>
- *     <li>VInt fileCount</li>
- *     <li>{Directory}
- *         fileCount entries with the following structure:</li>
- *         <ul>
- *             <li>long dataOffset</li>
- *             <li>String fileName</li>
- *         </ul>
- *     <li>{File Data}
- *         fileCount entries with the raw data of the corresponding file</li>
- * </ul>
- *
- * The fileCount integer indicates how many files are contained in this compound
- * file. The {directory} that follows has that many entries. Each directory entry
- * contains a long pointer to the start of this file's data section, and a String
- * with that file's name.
- * 
- * @lucene.internal
- */
-public final class CompoundFileWriter {
-
-    private static final class FileEntry {
-        /** source file */
-        String file;
-
-        /** temporary holder for the start of directory entry for this file */
-        long directoryOffset;
-
-        /** temporary holder for the start of this file's data section */
-        long dataOffset;
-        
-        /** the directory which contains the file. */
-        Directory dir;
-    }
-
-    // Before versioning started.
-    static final int FORMAT_PRE_VERSION = 0;
-    
-    // Segment name is not written in the file names.
-    static final int FORMAT_NO_SEGMENT_PREFIX = -1;
-
-    // NOTE: if you introduce a new format, make it 1 lower
-    // than the current one, and always change this if you
-    // switch to a new format!
-    static final int FORMAT_CURRENT = FORMAT_NO_SEGMENT_PREFIX;
-
-    private Directory directory;
-    private String fileName;
-    private HashSet<String> ids;
-    private LinkedList<FileEntry> entries;
-    private boolean merged = false;
-    private MergeState.CheckAbort checkAbort;
-
-    /** Create the compound stream in the specified file. The file name is the
-     *  entire name (no extensions are added).
-     *  @throws NullPointerException if <code>dir</code> or <code>name</code> is null
-     */
-    public CompoundFileWriter(Directory dir, String name) {
-      this(dir, name, null);
-    }
-
-    CompoundFileWriter(Directory dir, String name, MergeState.CheckAbort checkAbort) {
-        if (dir == null)
-            throw new NullPointerException("directory cannot be null");
-        if (name == null)
-            throw new NullPointerException("name cannot be null");
-        this.checkAbort = checkAbort;
-        directory = dir;
-        fileName = name;
-        ids = new HashSet<String>();
-        entries = new LinkedList<FileEntry>();
-    }
-
-    /** Returns the directory of the compound file. */
-    public Directory getDirectory() {
-        return directory;
-    }
-
-    /** Returns the name of the compound file. */
-    public String getName() {
-        return fileName;
-    }
-
-    /** Add a source stream. <code>file</code> is the string by which the 
-     *  sub-stream will be known in the compound stream.
-     * 
-     *  @throws IllegalStateException if this writer is closed
-     *  @throws NullPointerException if <code>file</code> is null
-     *  @throws IllegalArgumentException if a file with the same name
-     *   has been added already
-     */
-    public void addFile(String file) {
-      addFile(file, directory);
-    }
-
-    /**
-     * Same as {@link #addFile(String)}, only for files that are found in an
-     * external {@link Directory}.
-     */
-    public void addFile(String file, Directory dir) {
-        if (merged)
-            throw new IllegalStateException(
-                "Can't add extensions after merge has been called");
-
-        if (file == null)
-            throw new NullPointerException(
-                "file cannot be null");
-
-        if (! ids.add(file))
-            throw new IllegalArgumentException(
-                "File " + file + " already added");
-
-        FileEntry entry = new FileEntry();
-        entry.file = file;
-        entry.dir = dir;
-        entries.add(entry);
-    }
-
-    /** Merge files with the extensions added up to now.
-     *  All files with these extensions are combined sequentially into the
-     *  compound stream.
-     *  @throws IllegalStateException if close() had been called before or
-     *   if no file has been added to this object
-     */
-    public void close() throws IOException {
-        if (merged)
-            throw new IllegalStateException("Merge already performed");
-
-        if (entries.isEmpty())
-            throw new IllegalStateException("No entries to merge have been defined");
-
-        merged = true;
-
-        // open the compound stream
-        IndexOutput os = directory.createOutput(fileName);
-        IOException priorException = null;
-        try {
-            // Write the Version info - must be a VInt because CFR reads a VInt
-            // in older versions!
-            os.writeVInt(FORMAT_CURRENT);
-            
-            // Write the number of entries
-            os.writeVInt(entries.size());
-
-            // Write the directory with all offsets at 0.
-            // Remember the positions of directory entries so that we can
-            // adjust the offsets later
-            long totalSize = 0;
-            for (FileEntry fe : entries) {
-                fe.directoryOffset = os.getFilePointer();
-                os.writeLong(0);    // for now
-                os.writeString(IndexFileNames.stripSegmentName(fe.file));
-                totalSize += fe.dir.fileLength(fe.file);
-            }
-
-            // Pre-allocate size of file as optimization --
-            // this can potentially help IO performance as
-            // we write the file and also later during
-            // searching.  It also uncovers a disk-full
-            // situation earlier and hopefully without
-            // actually filling disk to 100%:
-            final long finalLength = totalSize+os.getFilePointer();
-            os.setLength(finalLength);
-
-            // Open the files and copy their data into the stream.
-            // Remember the locations of each file's data section.
-            for (FileEntry fe : entries) {
-                fe.dataOffset = os.getFilePointer();
-                copyFile(fe, os);
-            }
-
-            // Write the data offsets into the directory of the compound stream
-            for (FileEntry fe : entries) {
-                os.seek(fe.directoryOffset);
-                os.writeLong(fe.dataOffset);
-            }
-
-            assert finalLength == os.length();
-
-            // Close the output stream. Set the os to null before trying to
-            // close so that if an exception occurs during the close, the
-            // finally clause below will not attempt to close the stream
-            // the second time.
-            IndexOutput tmp = os;
-            os = null;
-            tmp.close();
-        } catch (IOException e) {
-          priorException = e;
-        } finally {
-          IOUtils.closeSafely(priorException, os);
-        }
-    }
-
-  /**
-   * Copy the contents of the file with specified extension into the provided
-   * output stream.
-   */
-  private void copyFile(FileEntry source, IndexOutput os) throws IOException {
-    IndexInput is = source.dir.openInput(source.file);
-    try {
-      long startPtr = os.getFilePointer();
-      long length = is.length();
-      os.copyBytes(is, length);
-
-      if (checkAbort != null) {
-        checkAbort.work(length);
-      }
-
-      // Verify that the output length diff is equal to original file
-      long endPtr = os.getFilePointer();
-      long diff = endPtr - startPtr;
-      if (diff != length)
-        throw new IOException("Difference in the output file offsets " + diff
-            + " does not match the original file length " + length);
-
-    } finally {
-      is.close();
-    }
-  }
-}
Index: lucene/src/java/org/apache/lucene/index/IndexFileNames.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/IndexFileNames.java	(revision 1137870)
+++ lucene/src/java/org/apache/lucene/index/IndexFileNames.java	(working copy)
@@ -68,6 +68,9 @@
 
   /** Extension of compound file */
   public static final String COMPOUND_FILE_EXTENSION = "cfs";
+  
+  /** Extension of compound file entries */
+  public static final String COMPOUND_FILE_ENTRIES_EXTENSION = "cfe";
 
   /** Extension of compound file for doc store files*/
   public static final String COMPOUND_FILE_STORE_EXTENSION = "cfx";
@@ -93,6 +96,7 @@
    */
   public static final String INDEX_EXTENSIONS[] = new String[] {
     COMPOUND_FILE_EXTENSION,
+    COMPOUND_FILE_ENTRIES_EXTENSION,
     FIELD_INFOS_EXTENSION,
     FIELDS_INDEX_EXTENSION,
     FIELDS_EXTENSION,
@@ -244,7 +248,15 @@
     }
     return filename;
   }
-
+  
+  public static String stripExtension(String filename) {
+    int idx = filename.indexOf('.');
+    if (idx != -1) {
+      filename = filename.substring(0, idx);
+    }
+    return filename;
+  }
+  
   /**
    * Returns true if the given filename ends with the separate norms file
    * pattern: {@code SEPARATE_NORMS_EXTENSION + "[0-9]+"}.
Index: lucene/src/java/org/apache/lucene/index/IndexReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/IndexReader.java	(revision 1137870)
+++ lucene/src/java/org/apache/lucene/index/IndexReader.java	(working copy)
@@ -1432,14 +1432,14 @@
     }
 
     Directory dir = null;
-    CompoundFileReader cfr = null;
+    CompoundFileDirectory cfr = null;
 
     try {
       File file = new File(filename);
       String dirname = file.getAbsoluteFile().getParent();
       filename = file.getName();
       dir = FSDirectory.open(new File(dirname));
-      cfr = new CompoundFileReader(dir, filename);
+      cfr = dir.openCompoundInput(filename, BufferedIndexInput.BUFFER_SIZE);
 
       String [] files = cfr.listAll();
       ArrayUtil.mergeSort(files);   // sort the array of filename so that the output is more readable
Index: lucene/src/java/org/apache/lucene/index/IndexWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/IndexWriter.java	(revision 1137870)
+++ lucene/src/java/org/apache/lucene/index/IndexWriter.java	(working copy)
@@ -46,12 +46,14 @@
 import org.apache.lucene.search.Query;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.BufferedIndexInput;
+import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.Lock;
 import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.util.BitVector;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.ThreadInterruptedException;
 import org.apache.lucene.util.MapBackedSet;
@@ -2190,13 +2192,19 @@
         String compoundFileName = IndexFileNames.segmentFileName(newSegment.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
         message("creating compound file " + compoundFileName);
         // Now build compound file
-        CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName);
-        for(String fileName : newSegment.files()) {
-          cfsWriter.addFile(fileName);
+        final CompoundFileDirectory cfsDir = directory.openCompoundOutput(compoundFileName);
+        IOException prior = null;
+        try {
+          for(String fileName : newSegment.files()) {
+            directory.copy(cfsDir, fileName, fileName);
+          }
+        } catch(IOException ex) {
+          prior = ex;
+        } finally {
+          IOUtils.closeSafely(prior, cfsDir);
         }
-
         // Perform the merge
-        cfsWriter.close();
+        
         synchronized(this) {
           deleter.deleteNewFiles(newSegment.files());
         }
@@ -2502,21 +2510,22 @@
   private void copySegmentIntoCFS(SegmentInfo info, String segName) throws IOException {
     String segFileName = IndexFileNames.segmentFileName(segName, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
     Collection<String> files = info.files();
-    CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, segFileName);
-    for (String file : files) {
-      String newFileName = segName + IndexFileNames.stripSegmentName(file);
-      if (!IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION)
-          && !IndexFileNames.isSeparateNormsFile(file)) {
-        cfsWriter.addFile(file, info.dir);
-      } else {
-        assert !directory.fileExists(newFileName): "file \"" + newFileName + "\" already exists";
-        info.dir.copy(directory, file, newFileName);
+    final CompoundFileDirectory cfsdir = directory.openCompoundOutput(segFileName);
+    try {
+      for (String file : files) {
+        String newFileName = segName + IndexFileNames.stripSegmentName(file);
+        if (!IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION)
+            && !IndexFileNames.isSeparateNormsFile(file)) {
+          info.dir.copy(cfsdir, file, file);
+        } else {
+          assert !directory.fileExists(newFileName): "file \"" + newFileName + "\" already exists";
+          info.dir.copy(directory, file, newFileName);
+        }
       }
+    } finally {
+      IOUtils.closeSafely(true, cfsdir);
     }
     
-    // Create the .cfs
-    cfsWriter.close();
-    
     info.dir = directory;
     info.name = segName;
     info.setUseCompoundFile(true);
@@ -3515,6 +3524,8 @@
 
             synchronized(this) {
               deleter.deleteFile(compoundFileName);
+              
+              deleter.deleteFile(IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
               deleter.deleteNewFiles(merge.info.files());
             }
           }
Index: lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java	(revision 1137870)
+++ lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java	(working copy)
@@ -23,6 +23,7 @@
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.codecs.PerDocValues;
+import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 
 /** Holds core readers that are shared (unchanged) when
@@ -52,8 +53,8 @@
   
   FieldsReader fieldsReaderOrig;
   TermVectorsReader termVectorsReaderOrig;
-  CompoundFileReader cfsReader;
-  CompoundFileReader storeCFSReader;
+  CompoundFileDirectory cfsReader;
+  CompoundFileDirectory storeCFSReader;
 
   
   
@@ -73,7 +74,7 @@
     try {
       Directory dir0 = dir;
       if (si.getUseCompoundFile()) {
-        cfsReader = new CompoundFileReader(dir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), readBufferSize);
+        cfsReader = dir.openCompoundInput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), readBufferSize);
         dir0 = cfsReader;
       }
       cfsDir = dir0;
@@ -161,7 +162,7 @@
       if (si.getDocStoreOffset() != -1) {
         if (si.getDocStoreIsCompoundFile()) {
           assert storeCFSReader == null;
-          storeCFSReader = new CompoundFileReader(dir,
+          storeCFSReader = dir.openCompoundInput(
               IndexFileNames.segmentFileName(si.getDocStoreSegment(), "", IndexFileNames.COMPOUND_FILE_STORE_EXTENSION),
               readBufferSize);
           storeDir = storeCFSReader;
@@ -175,7 +176,7 @@
         // was not used, but then we are asked to open doc
         // stores after the segment has switched to CFS
         if (cfsReader == null) {
-          cfsReader = new CompoundFileReader(dir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), readBufferSize);
+          cfsReader = dir.openCompoundInput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), readBufferSize);
         }
         storeDir = cfsReader;
         assert storeDir != null;
Index: lucene/src/java/org/apache/lucene/index/SegmentInfo.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/SegmentInfo.java	(revision 1137870)
+++ lucene/src/java/org/apache/lucene/index/SegmentInfo.java	(working copy)
@@ -30,10 +30,12 @@
 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.BufferedIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.StringHelper;
 
 /**
  * Information about a segment such as it's name, directory, and files related
@@ -245,7 +247,7 @@
       }
       final Directory dirToTest;
       if (isCompoundFile) {
-        dirToTest = new CompoundFileReader(dir, IndexFileNames.segmentFileName(storesSegment, "", ext));
+        dirToTest = dir.openCompoundInput(IndexFileNames.segmentFileName(storesSegment, "", ext), BufferedIndexInput.BUFFER_SIZE);
       } else {
         dirToTest = dir;
       }
@@ -263,8 +265,8 @@
     if (fieldInfos == null) {
       Directory dir0 = dir;
       if (isCompoundFile && checkCompoundFile) {
-        dir0 = new CompoundFileReader(dir, IndexFileNames.segmentFileName(name,
-            "", IndexFileNames.COMPOUND_FILE_EXTENSION));
+        dir0 = dir.openCompoundInput(IndexFileNames.segmentFileName(name,
+            "", IndexFileNames.COMPOUND_FILE_EXTENSION), BufferedIndexInput.BUFFER_SIZE);
       }
       try {
         fieldInfos = new FieldInfos(dir0, IndexFileNames.segmentFileName(name,
@@ -454,7 +456,7 @@
     this.isCompoundFile = isCompoundFile;
     clearFilesCache();
   }
-
+  
   /**
    * Returns true if this segment is stored as a compound
    * file; else, false.
@@ -617,6 +619,10 @@
 
     if (useCompoundFile) {
       fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.COMPOUND_FILE_EXTENSION));
+      if (version != null && StringHelper.getVersionComparator().compare("4.0", version) <= 0) {
+        fileSet.add(IndexFileNames.segmentFileName(name, "",
+            IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
+      }
     } else {
       for(String ext : IndexFileNames.NON_STORE_INDEX_EXTENSIONS) {
         addIfExists(fileSet, IndexFileNames.segmentFileName(name, "", ext));
Index: lucene/src/java/org/apache/lucene/index/SegmentMerger.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/SegmentMerger.java	(revision 1137870)
+++ lucene/src/java/org/apache/lucene/index/SegmentMerger.java	(working copy)
@@ -31,6 +31,7 @@
 import org.apache.lucene.index.codecs.MergeState;
 import org.apache.lucene.index.codecs.PerDocConsumer;
 import org.apache.lucene.index.codecs.PerDocValues;
+import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
@@ -132,18 +133,20 @@
 
     // Now merge all added files
     Collection<String> files = info.files();
-    CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, fileName, checkAbort);
-    for (String file : files) {
-      assert !IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION) 
-                : ".del file is not allowed in .cfs: " + file;
-      assert !IndexFileNames.isSeparateNormsFile(file) 
-                : "separate norms file (.s[0-9]+) is not allowed in .cfs: " + file;
-      cfsWriter.addFile(file);
+    CompoundFileDirectory cfsDir = directory.openCompoundOutput(fileName);
+    try {
+      cfsDir.setAbortCheck(checkAbort);
+      for (String file : files) {
+        assert !IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION) 
+                  : ".del file is not allowed in .cfs: " + file;
+        assert !IndexFileNames.isSeparateNormsFile(file) 
+                  : "separate norms file (.s[0-9]+) is not allowed in .cfs: " + file;
+        directory.copy(cfsDir, file, file);
+      }
+    } finally {
+      cfsDir.close();
     }
 
-    // Perform the merge
-    cfsWriter.close();
-
     return files;
   }
   
Index: lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java	(revision 1137870)
+++ lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java	(working copy)
@@ -19,7 +19,6 @@
 
 import java.io.IOException;
 
-import org.apache.lucene.index.CompoundFileReader;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldsReader;
 import org.apache.lucene.index.IndexFileNames;
@@ -68,12 +67,12 @@
           Directory dir = directory;
           if (si.getDocStoreOffset() != -1) {
             if (si.getDocStoreIsCompoundFile()) {
-              dir = new CompoundFileReader(dir, IndexFileNames.segmentFileName(
+              dir = dir.openCompoundInput(IndexFileNames.segmentFileName(
                   si.getDocStoreSegment(), "",
                   IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), 1024);
             }
           } else if (si.getUseCompoundFile()) {
-            dir = new CompoundFileReader(dir, IndexFileNames.segmentFileName(
+            dir = dir.openCompoundInput(IndexFileNames.segmentFileName(
                 si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), 1024);
           }
 
Index: lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java	(revision 1137870)
+++ lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java	(working copy)
@@ -25,7 +25,6 @@
 import java.util.Map;
 import java.util.TreeMap;
 
-import org.apache.lucene.index.CompoundFileReader;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
@@ -37,6 +36,7 @@
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.codecs.FieldsProducer;
+import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Bits;
@@ -177,8 +177,8 @@
         // terms reader with index, the segment has switched
         // to CFS
 
-        if (!(dir instanceof CompoundFileReader)) {
-          dir0 = cfsReader = new CompoundFileReader(dir, IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), readBufferSize);
+        if (!(dir instanceof CompoundFileDirectory)) {
+          dir0 = cfsReader = dir.openCompoundInput(IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), readBufferSize);
         } else {
           dir0 = dir;
         }
Index: lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java	(revision 1137870)
+++ lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java	(working copy)
@@ -1,4 +1,4 @@
-package org.apache.lucene.index;
+package org.apache.lucene.store;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,236 +17,358 @@
  * limitations under the License.
  */
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.HashSet;
+import java.util.Collection;
+import java.util.HashMap;
 import java.util.LinkedList;
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.MergePolicy.MergeAbortedException;
 import org.apache.lucene.index.codecs.MergeState;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.IOUtils;
 
 /**
  * Combines multiple files into a single compound file.
- * The file format:<br>
+ * 
+ * The file format data file:<br>
  * <ul>
- *     <li>VInt fileCount</li>
- *     <li>{Directory}
- *         fileCount entries with the following structure:</li>
- *         <ul>
- *             <li>long dataOffset</li>
- *             <li>String fileName</li>
- *         </ul>
- *     <li>{File Data}
- *         fileCount entries with the raw data of the corresponding file</li>
+ * <li>VInt Version</li>
+ * <li>{File Data} fileCount entries with the raw data of the corresponding file
+ * </li>
+ * <ul>
+ * File format entry table:<br>
+ * <ul>
+ * <li>int Version</li>
+ * <li>VInt fileCount - number of entries with the following structure:</li>
+ * <ul>
+ * <li>String fileName</li>
+ * <li>long dataOffset</li>
+ * <li>long dataLength</li>
  * </ul>
- *
- * The fileCount integer indicates how many files are contained in this compound
- * file. The {directory} that follows has that many entries. Each directory entry
- * contains a long pointer to the start of this file's data section, and a String
- * with that file's name.
+ * </li> </ul> The fileCount integer indicates how many files are contained in
+ * this compound file. The entry table that follows has that many entries. Each
+ * directory entry contains a long pointer to the start of this file's data
+ * section, the files length, and a String with that file's name.
  * 
  * @lucene.internal
  */
-public final class CompoundFileWriter {
+final class CompoundFileWriter {
 
-    private static final class FileEntry {
-        /** source file */
-        String file;
+  private static final class FileEntry {
+    /** source file */
+    String file;
+    long length;
+    /** temporary holder for the start of this file's data section */
+    long offset;
+    /** the directory which contains the file. */
+    Directory dir;
+  }
 
-        /** temporary holder for the start of directory entry for this file */
-        long directoryOffset;
+  // Before versioning started.
+  static final int FORMAT_PRE_VERSION = 0;
 
-        /** temporary holder for the start of this file's data section */
-        long dataOffset;
-        
-        /** the directory which contains the file. */
-        Directory dir;
-    }
+  // Segment name is not written in the file names.
+  static final int FORMAT_NO_SEGMENT_PREFIX = -1;
+  static final int FORMAT_APPEND_FILES = -2;
 
-    // Before versioning started.
-    static final int FORMAT_PRE_VERSION = 0;
-    
-    // Segment name is not written in the file names.
-    static final int FORMAT_NO_SEGMENT_PREFIX = -1;
+  static final int ENTRY_FORMAT_CURRENT = -1;
 
-    // NOTE: if you introduce a new format, make it 1 lower
-    // than the current one, and always change this if you
-    // switch to a new format!
-    static final int FORMAT_CURRENT = FORMAT_NO_SEGMENT_PREFIX;
+  // NOTE: if you introduce a new format, make it 1 lower
+  // than the current one, and always change this if you
+  // switch to a new format!
+  /** @lucene.internal */
+  static final int FORMAT_CURRENT = FORMAT_APPEND_FILES;
 
-    private Directory directory;
-    private String fileName;
-    private HashSet<String> ids;
-    private LinkedList<FileEntry> entries;
-    private boolean merged = false;
-    private MergeState.CheckAbort checkAbort;
+  private final Directory directory;
+  private final Map<String, FileEntry> entries = new HashMap<String, FileEntry>();
+  // all entries that are written to a sep. file but not yet moved into CFS
+  private final Queue<FileEntry> pendingEntries = new LinkedList<FileEntry>();
+  private boolean closed = false;
+  private MergeState.CheckAbort checkAbort;
+  private volatile IndexOutput dataOut;
+  private final AtomicBoolean outputTaken = new AtomicBoolean(false);
+  private final String entryTableName;
+  private final String dataFileName;
 
-    /** Create the compound stream in the specified file. The file name is the
-     *  entire name (no extensions are added).
-     *  @throws NullPointerException if <code>dir</code> or <code>name</code> is null
-     */
-    public CompoundFileWriter(Directory dir, String name) {
-      this(dir, name, null);
-    }
+  /**
+   * Create the compound stream in the specified file. The file name is the
+   * entire name (no extensions are added).
+   * 
+   * @throws NullPointerException
+   *           if <code>dir</code> or <code>name</code> is null
+   */
+  CompoundFileWriter(Directory dir, String name) {
+    this(dir, name, null);
+  }
 
-    CompoundFileWriter(Directory dir, String name, MergeState.CheckAbort checkAbort) {
-        if (dir == null)
-            throw new NullPointerException("directory cannot be null");
-        if (name == null)
-            throw new NullPointerException("name cannot be null");
-        this.checkAbort = checkAbort;
-        directory = dir;
-        fileName = name;
-        ids = new HashSet<String>();
-        entries = new LinkedList<FileEntry>();
-    }
+  CompoundFileWriter(Directory dir, String name,
+      MergeState.CheckAbort checkAbort) {
+    if (dir == null)
+      throw new NullPointerException("directory cannot be null");
+    if (name == null)
+      throw new NullPointerException("name cannot be null");
+    this.checkAbort = checkAbort;
+    directory = dir;
+    entryTableName = IndexFileNames.segmentFileName(
+        IndexFileNames.stripExtension(name), "",
+        IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
+    dataFileName = name;
+  }
 
-    /** Returns the directory of the compound file. */
-    public Directory getDirectory() {
-        return directory;
-    }
+  /** Returns the directory of the compound file. */
+  Directory getDirectory() {
+    return directory;
+  }
 
-    /** Returns the name of the compound file. */
-    public String getName() {
-        return fileName;
+  /** Returns the name of the compound file. */
+  String getName() {
+    return dataFileName;
+  }
+
+  /**
+   * Closes all resources and writes the entry table
+   * 
+   * @throws IllegalStateException
+   *           if close() had been called before or if no file has been added to
+   *           this object
+   */
+  void close() throws IOException {
+    if (closed) {
+      throw new IllegalStateException("already closed");
     }
+    closed = true;
+    // open the compound stream
+    IOException priorException = null;
+    IndexOutput entryTableOut = null;
+    try {
+      assert !entries.isEmpty() : "CFS has no entries";
+      assert pendingEntries.isEmpty();
+      assert !outputTaken.get() : "some files are still open";
+      assert dataOut != null;
+      long finalLength = dataOut.getFilePointer();
+      assert assertFileLength(finalLength, dataOut);
+      // Close the output stream. Set the os to null before trying to
+      // close so that if an exception occurs during the close, the
+      // finally clause below will not attempt to close the stream
+      // the second time.
+      entryTableOut = directory.createOutput(entryTableName);
+      writeEntryTable(entries.values(), entryTableOut);
+    } catch (IOException e) {
+      priorException = e;
+    } finally {
+      IOUtils.closeSafely(priorException, dataOut, entryTableOut);
+    }
+  }
 
-    /** Add a source stream. <code>file</code> is the string by which the 
-     *  sub-stream will be known in the compound stream.
-     * 
-     *  @throws IllegalStateException if this writer is closed
-     *  @throws NullPointerException if <code>file</code> is null
-     *  @throws IllegalArgumentException if a file with the same name
-     *   has been added already
-     */
-    public void addFile(String file) {
-      addFile(file, directory);
+  private static boolean assertFileLength(long expected, IndexOutput out)
+      throws IOException {
+    out.flush();
+    assert expected == out.length() : "expected: " + expected + " was "
+        + out.length();
+    return true;
+  }
+
+  private final void ensureOpen() {
+    if (closed) {
+      throw new IllegalStateException("CFS Directory is already closed");
     }
+  }
 
-    /**
-     * Same as {@link #addFile(String)}, only for files that are found in an
-     * external {@link Directory}.
-     */
-    public void addFile(String file, Directory dir) {
-        if (merged)
-            throw new IllegalStateException(
-                "Can't add extensions after merge has been called");
+  /**
+   * Copy the contents of the file with specified extension into the provided
+   * output stream.
+   */
+  private final long copyFileEntry(IndexOutput dataOut, FileEntry fileEntry)
+      throws IOException, MergeAbortedException {
+    final IndexInput is = fileEntry.dir.openInput(fileEntry.file);
+    try {
+      final long startPtr = dataOut.getFilePointer();
+      final long length = fileEntry.length;
+      dataOut.copyBytes(is, length);
+      if (checkAbort != null) {
+        checkAbort.work(length);
+      }
 
-        if (file == null)
-            throw new NullPointerException(
-                "file cannot be null");
+      // Verify that the output length diff is equal to original file
+      long endPtr = dataOut.getFilePointer();
+      long diff = endPtr - startPtr;
+      if (diff != length)
+        throw new IOException("Difference in the output file offsets " + diff
+            + " does not match the original file length " + length);
+      fileEntry.offset = startPtr;
+      return length;
+    } finally {
+      is.close();
+    }
 
-        if (! ids.add(file))
-            throw new IllegalArgumentException(
-                "File " + file + " already added");
+  }
 
-        FileEntry entry = new FileEntry();
-        entry.file = file;
-        entry.dir = dir;
-        entries.add(entry);
+  protected void writeEntryTable(Collection<FileEntry> entries,
+      IndexOutput entryOut) throws IOException {
+    entryOut.writeInt(ENTRY_FORMAT_CURRENT);
+    entryOut.writeVInt(entries.size());
+    for (FileEntry fe : entries) {
+      entryOut.writeString(IndexFileNames.stripSegmentName(fe.file));
+      entryOut.writeLong(fe.offset);
+      entryOut.writeLong(fe.length);
     }
+  }
 
-    /** Merge files with the extensions added up to now.
-     *  All files with these extensions are combined sequentially into the
-     *  compound stream.
-     *  @throws IllegalStateException if close() had been called before or
-     *   if no file has been added to this object
-     */
-    public void close() throws IOException {
-        if (merged)
-            throw new IllegalStateException("Merge already performed");
+  IndexOutput createOutput(String name) throws IOException {
+    ensureOpen();
+    boolean success = false;
+    try {
+      assert name != null : "name must not be null";
+      if (entries.containsKey(name)) {
+        throw new IllegalArgumentException("File " + name + " already exists");
+      }
+      final FileEntry entry = new FileEntry();
+      entry.file = name;
+      entries.put(name, entry);
+      final DirectCFSIndexOutput out;
+      if (outputTaken.compareAndSet(false, true)) {
+        initDataOut();
+        success = true;
+        out = new DirectCFSIndexOutput(dataOut, entry, false);
+      } else {
+        entry.dir = this.directory;
+        out = new DirectCFSIndexOutput(directory.createOutput(name), entry,
+            true);
+      }
+      success = true;
+      return out;
+    } finally {
+      if (!success) {
+        entries.remove(name);
+      }
+    }
+  }
 
-        if (entries.isEmpty())
-            throw new IllegalStateException("No entries to merge have been defined");
+  final void releaseOutputLock() {
+    outputTaken.compareAndSet(true, false);
+  }
 
-        merged = true;
+  private synchronized final void initDataOut() throws IOException {
+    if (dataOut == null) {
+      boolean success = false;
+      try {
+        dataOut = directory.createOutput(dataFileName);
+        dataOut.writeVInt(FORMAT_CURRENT);
+        success = true;
+      } finally {
+        if (!success) {
+          IOUtils.closeSafely(true, dataOut);
+        }
+      }
+    }
+  }
 
-        // open the compound stream
-        IndexOutput os = directory.createOutput(fileName);
-        IOException priorException = null;
-        try {
-            // Write the Version info - must be a VInt because CFR reads a VInt
-            // in older versions!
-            os.writeVInt(FORMAT_CURRENT);
-            
-            // Write the number of entries
-            os.writeVInt(entries.size());
+  private final void prunePendingEntries() throws IOException {
+    // claim the output and copy all pending files in
+    if (outputTaken.compareAndSet(false, true)) {
+      try {
+        while (!pendingEntries.isEmpty()) {
+          FileEntry entry = pendingEntries.poll();
+          copyFileEntry(dataOut, entry);
+          entries.put(entry.file, entry);
+        }
+      } finally {
+        final boolean compareAndSet = outputTaken.compareAndSet(true, false);
+        assert compareAndSet;
+      }
+    }
+  }
 
-            // Write the directory with all offsets at 0.
-            // Remember the positions of directory entries so that we can
-            // adjust the offsets later
-            long totalSize = 0;
-            for (FileEntry fe : entries) {
-                fe.directoryOffset = os.getFilePointer();
-                os.writeLong(0);    // for now
-                os.writeString(IndexFileNames.stripSegmentName(fe.file));
-                totalSize += fe.dir.fileLength(fe.file);
-            }
+  long fileLenght(String name) throws IOException {
+    FileEntry fileEntry = entries.get(name);
+    if (fileEntry == null) {
+      throw new FileNotFoundException(name + " does not exist");
+    }
+    return fileEntry.length;
+  }
 
-            // Pre-allocate size of file as optimization --
-            // this can potentially help IO performance as
-            // we write the file and also later during
-            // searching.  It also uncovers a disk-full
-            // situation earlier and hopefully without
-            // actually filling disk to 100%:
-            final long finalLength = totalSize+os.getFilePointer();
-            os.setLength(finalLength);
+  boolean fileExists(String name) {
+    return entries.containsKey(name);
+  }
 
-            // Open the files and copy their data into the stream.
-            // Remember the locations of each file's data section.
-            for (FileEntry fe : entries) {
-                fe.dataOffset = os.getFilePointer();
-                copyFile(fe, os);
-            }
+  String[] listAll() {
+    return entries.keySet().toArray(new String[0]);
+  }
 
-            // Write the data offsets into the directory of the compound stream
-            for (FileEntry fe : entries) {
-                os.seek(fe.directoryOffset);
-                os.writeLong(fe.dataOffset);
-            }
+  private final class DirectCFSIndexOutput extends IndexOutput {
+    private final IndexOutput delegate;
+    private final long offset;
+    private boolean closed;
+    private FileEntry entry;
+    private long writtenBytes;
+    private final boolean isSeparate;
 
-            assert finalLength == os.length();
+    DirectCFSIndexOutput(IndexOutput delegate, FileEntry entry,
+        boolean isSeparate) {
+      super();
+      this.delegate = delegate;
+      this.entry = entry;
+      entry.offset = offset = delegate.getFilePointer();
+      this.isSeparate = isSeparate;
 
-            // Close the output stream. Set the os to null before trying to
-            // close so that if an exception occurs during the close, the
-            // finally clause below will not attempt to close the stream
-            // the second time.
-            IndexOutput tmp = os;
-            os = null;
-            tmp.close();
-        } catch (IOException e) {
-          priorException = e;
-        } finally {
-          IOUtils.closeSafely(priorException, os);
-        }
     }
 
-  /**
-   * Copy the contents of the file with specified extension into the provided
-   * output stream.
-   */
-  private void copyFile(FileEntry source, IndexOutput os) throws IOException {
-    IndexInput is = source.dir.openInput(source.file);
-    try {
-      long startPtr = os.getFilePointer();
-      long length = is.length();
-      os.copyBytes(is, length);
+    @Override
+    public void flush() throws IOException {
+      delegate.flush();
+    }
 
-      if (checkAbort != null) {
-        checkAbort.work(length);
+    @Override
+    public void close() throws IOException {
+      if (!closed) {
+        closed = true;
+        entry.length = writtenBytes;
+        if (isSeparate) {
+          // we are a separate file - push into the pending entries
+          pendingEntries.add(entry);
+        } else {
+          // we have been written into the CFS directly - release the lock
+          releaseOutputLock();
+        }
+        // now prune all pending entries and push them into the CFS
+        prunePendingEntries();
       }
+    }
 
-      // Verify that the output length diff is equal to original file
-      long endPtr = os.getFilePointer();
-      long diff = endPtr - startPtr;
-      if (diff != length)
-        throw new IOException("Difference in the output file offsets " + diff
-            + " does not match the original file length " + length);
+    @Override
+    public long getFilePointer() {
+      return delegate.getFilePointer() - offset;
+    }
 
-    } finally {
-      is.close();
+    @Override
+    public void seek(long pos) throws IOException {
+      assert !closed;
+      delegate.seek(offset + pos);
     }
+
+    @Override
+    public long length() throws IOException {
+      assert !closed;
+      return delegate.length() - offset;
+    }
+
+    @Override
+    public void writeByte(byte b) throws IOException {
+      assert !closed;
+      writtenBytes++;
+      delegate.writeByte(b);
+    }
+
+    @Override
+    public void writeBytes(byte[] b, int offset, int length) throws IOException {
+      assert !closed;
+      writtenBytes += length;
+      delegate.writeBytes(b, offset, length);
+    }
   }
+
 }
Index: lucene/src/java/org/apache/lucene/store/Directory.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/Directory.java	(revision 1137870)
+++ lucene/src/java/org/apache/lucene/store/Directory.java	(working copy)
@@ -117,6 +117,22 @@
   public IndexInput openInput(String name, int bufferSize) throws IOException {
     return openInput(name);
   }
+  
+  /** 
+   * 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, int bufferSize) throws IOException {
+    return new DefaultCompoundFileDirectory(this, name, bufferSize, false);
+  }
+  
+  public CompoundFileDirectory openCompoundOutput(String name) throws IOException {
+    return new DefaultCompoundFileDirectory(this, name, 1024, true);
+  }
 
   /** Construct a {@link Lock}.
    * @param name the name of the lock file
Index: lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java	(revision 1137870)
+++ lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java	(working copy)
@@ -148,4 +148,14 @@
   public IndexInput openInput(String name) throws IOException {
     return getDirectory(name).openInput(name);
   }
+
+  @Override
+  public CompoundFileDirectory openCompoundInput(String name, int bufferSize) throws IOException {
+    return getDirectory(name).openCompoundInput(name, bufferSize);
+  }
+  
+  @Override
+  public CompoundFileDirectory openCompoundOutput(String name) throws IOException {
+    return getDirectory(name).openCompoundOutput(name);
+  }
 }
Index: lucene/src/java/org/apache/lucene/store/MMapDirectory.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/MMapDirectory.java	(revision 1137870)
+++ lucene/src/java/org/apache/lucene/store/MMapDirectory.java	(working copy)
@@ -32,6 +32,7 @@
 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
@@ -213,12 +214,50 @@
     File f = new File(getDirectory(), name);
     RandomAccessFile raf = new RandomAccessFile(f, "r");
     try {
-      return new MMapIndexInput(raf, chunkSizePower);
+      return new MMapIndexInput(raf, 0, raf.length(), chunkSizePower);
     } finally {
       raf.close();
     }
   }
+  
+  @Override
+  public CompoundFileDirectory openCompoundInput(String name, int bufferSize) throws IOException {
+    return new MMapCompoundFileDirectory(name, bufferSize);
+  }
+  
+  private final class MMapCompoundFileDirectory extends CompoundFileDirectory {
+    private RandomAccessFile raf = null;
 
+    public MMapCompoundFileDirectory(String fileName, int readBufferSize) throws IOException {
+      super(MMapDirectory.this, fileName, readBufferSize);
+      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);
+      }
+    }
+
+    @Override
+    public IndexInput openInputSlice(String id, long offset, long length, int readBufferSize) throws IOException {
+      return new MMapIndexInput(raf, offset, length, chunkSizePower);
+    }
+
+    @Override
+    public synchronized void close() throws IOException {
+      try {
+        raf.close();
+      } finally {
+        super.close();
+      }
+    }
+  }
+
   // Because Java's ByteBuffer uses an int to address the
   // values, it's necessary to access a file >
   // Integer.MAX_VALUE in size using multiple byte buffers.
@@ -235,8 +274,8 @@
   
     private boolean isClone = false;
     
-    MMapIndexInput(RandomAccessFile raf, int chunkSizePower) throws IOException {
-      this.length = raf.length();
+    MMapIndexInput(RandomAccessFile raf, long offset, long length, int chunkSizePower) throws IOException {
+      this.length = length;
       this.chunkSizePower = chunkSizePower;
       this.chunkSize = 1L << chunkSizePower;
       this.chunkSizeMask = chunkSize - 1L;
@@ -261,7 +300,7 @@
           ? chunkSize
           : (length - bufferStart)
         );
-        this.buffers[bufNr] = rafc.map(MapMode.READ_ONLY, bufferStart, bufSize);
+        this.buffers[bufNr] = rafc.map(MapMode.READ_ONLY, offset + bufferStart, bufSize);
         bufferStart += bufSize;
       }
       seek(0L);
Index: lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java	(revision 1137870)
+++ lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java	(working copy)
@@ -24,6 +24,9 @@
 import java.nio.channels.FileChannel;
 import java.util.concurrent.Future; // javadoc
 
+import org.apache.lucene.store.SimpleFSDirectory.SimpleFSIndexInput;
+import org.apache.lucene.util.IOUtils;
+
 /**
  * An {@link FSDirectory} implementation that uses java.nio's FileChannel's
  * positional read, which allows multiple threads to read from the same file
@@ -77,7 +80,48 @@
     ensureOpen();
     return new NIOFSIndexInput(new File(getDirectory(), name), bufferSize, getReadChunkSize());
   }
+  
+  @Override
+  public CompoundFileDirectory openCompoundInput(String name, int bufferSize) throws IOException {
+    return new NIOFSCompoundFileDirectory(name, bufferSize);
+  }
 
+  private final class NIOFSCompoundFileDirectory extends CompoundFileDirectory {
+    private SimpleFSIndexInput.Descriptor fd;
+    private FileChannel fc;
+
+    public NIOFSCompoundFileDirectory(String fileName, int readBufferSize) throws IOException {
+      super(NIOFSDirectory.this, fileName, readBufferSize);
+      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 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();
+      }
+    }
+  }
+
   protected static class NIOFSIndexInput extends SimpleFSDirectory.SimpleFSIndexInput {
 
     private ByteBuffer byteBuf; // wraps the buffer for NIO
@@ -91,6 +135,12 @@
       super(path, bufferSize, chunkSize);
       channel = file.getChannel();
     }
+    
+    public NIOFSIndexInput(Descriptor file, FileChannel fc, long off, long length, int bufferSize, int chunkSize) throws IOException {
+      super(file, off, length, bufferSize, chunkSize);
+      channel = fc;
+      isClone = true;
+    }
 
     @Override
     protected void newBuffer(byte[] newBuffer) {
@@ -145,7 +195,11 @@
       int readLength = bb.limit() - readOffset;
       assert readLength == len;
 
-      long pos = getFilePointer();
+      long pos = getFilePointer() + off;
+      
+      if (pos + len > end) {
+        throw new IOException("read past EOF");
+      }
 
       try {
         while (readLength > 0) {
@@ -159,9 +213,6 @@
           }
           bb.limit(limit);
           int i = channel.read(bb, pos);
-          if (i == -1) {
-            throw new IOException("read past EOF");
-          }
           pos += i;
           readOffset += i;
           readLength -= i;
Index: lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java	(revision 1137870)
+++ lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java	(working copy)
@@ -21,6 +21,8 @@
 import java.io.IOException;
 import java.io.RandomAccessFile;
 
+import org.apache.lucene.util.IOUtils;
+
 /** A straightforward implementation of {@link FSDirectory}
  *  using java.io.RandomAccessFile.  However, this class has
  *  poor concurrent performance (multiple threads will
@@ -55,7 +57,46 @@
     ensureOpen();
     return new SimpleFSIndexInput(new File(directory, name), bufferSize, getReadChunkSize());
   }
+  
+  @Override
+  public CompoundFileDirectory openCompoundInput(String name, int bufferSize) throws IOException {
+    return new SimpleFSCompoundFileDirectory(name, bufferSize);
+  }
 
+  private final class SimpleFSCompoundFileDirectory extends CompoundFileDirectory {
+    private SimpleFSIndexInput.Descriptor fd;
+
+    public SimpleFSCompoundFileDirectory(String fileName, int readBufferSize) throws IOException {
+      super(SimpleFSDirectory.this, fileName, readBufferSize);
+      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 IndexInput openInputSlice(String id, long offset, long length, int readBufferSize) throws IOException {
+      return new SimpleFSIndexInput(fd, offset, length, readBufferSize, getReadChunkSize());
+    }
+
+    @Override
+    public synchronized void close() throws IOException {
+      try {
+        fd.close();
+      } finally {
+        super.close();
+      }
+    }
+  }
+
   protected static class SimpleFSIndexInput extends BufferedIndexInput {
   
     protected static class Descriptor extends RandomAccessFile {
@@ -84,25 +125,42 @@
     boolean isClone;
     //  LUCENE-1566 - maximum read length on a 32bit JVM to prevent incorrect OOM 
     protected final int chunkSize;
+    protected final long off;
+    protected final long end;
     
     public SimpleFSIndexInput(File path, int bufferSize, int chunkSize) throws IOException {
       super(bufferSize);
-      file = new Descriptor(path, "r");
+      this.file = new Descriptor(path, "r"); 
       this.chunkSize = chunkSize;
+      this.off = 0L;
+      this.end = file.length;
     }
+    
+    public SimpleFSIndexInput(Descriptor file, long off, long length, int bufferSize, int chunkSize) throws IOException {
+      super(bufferSize);
+      this.file = file;
+      this.chunkSize = chunkSize;
+      this.off = off;
+      this.end = off + length;
+      this.isClone = true; // well, we are sorta?
+    }
   
     /** IndexInput methods */
     @Override
     protected void readInternal(byte[] b, int offset, int len)
          throws IOException {
       synchronized (file) {
-        long position = getFilePointer();
+        long position = off + getFilePointer();
         if (position != file.position) {
           file.seek(position);
           file.position = position;
         }
         int total = 0;
 
+        if (position + len > end) {
+          throw new IOException("read past EOF");
+        }
+
         try {
           do {
             final int readLength;
@@ -113,9 +171,6 @@
               readLength = chunkSize;
             }
             final int i = file.read(b, offset + total, readLength);
-            if (i == -1) {
-              throw new IOException("read past EOF");
-            }
             file.position += i;
             total += i;
           } while (total < len);
@@ -144,7 +199,7 @@
   
     @Override
     public long length() {
-      return file.length;
+      return end - off;
     }
   
     @Override
Index: lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java
===================================================================
--- lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java	(revision 1137870)
+++ lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java	(working copy)
@@ -388,7 +388,7 @@
     }
   }
 
-  private void addFileHandle(Closeable c, String name, boolean input) {
+  void addFileHandle(Closeable c, String name, boolean input) {
     Integer v = openFiles.get(name);
     if (v != null) {
       v = Integer.valueOf(v.intValue()+1);
@@ -416,7 +416,13 @@
     addFileHandle(ii, name, true);
     return ii;
   }
-
+  
+  @Override
+  public synchronized CompoundFileDirectory openCompoundInput(String name, int bufferSize) throws IOException {
+    maybeYield();
+    return new MockCompoundFileDirectoryWrapper(name, this, delegate.openCompoundInput(name, bufferSize));
+  }
+  
   /** Provided for testing purposes.  Use sizeInBytes() instead. */
   public synchronized final long getRecomputedSizeInBytes() throws IOException {
     if (!(delegate instanceof RAMDirectory))
@@ -481,7 +487,7 @@
     delegate.close();
   }
 
-  private synchronized void removeOpenFile(Closeable c, String name) {
+  synchronized void removeOpenFile(Closeable c, String name) {
     Integer v = openFiles.get(name);
     // Could be null when crash() was called
     if (v != null) {
Index: lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestAddIndexes.java	(revision 1137870)
+++ lucene/src/test/org/apache/lucene/index/TestAddIndexes.java	(working copy)
@@ -1075,8 +1075,8 @@
     IndexWriter w3 = new IndexWriter(dir, conf);
     w3.addIndexes(readers);
     w3.close();
-    // we should now see segments_X, segments.gen,_Y.cfs, _Z.fnx
-    assertEquals("Only one compound segment should exist", 4, dir.listAll().length);
+    // we should now see segments_X, segments.gen,_Y.cfs,_Y.cfe, _Z.fnx
+    assertEquals("Only one compound segment should exist", 5, dir.listAll().length);
   }
   
   // LUCENE-3126: tests that if a non-CFS segment is copied, it is converted to
Index: lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java	(revision 1137870)
+++ lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java	(working copy)
@@ -39,8 +39,8 @@
 import org.apache.lucene.search.NumericRangeQuery;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Similarity;
-import org.apache.lucene.search.SimilarityProvider;
 import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.Bits;
@@ -536,7 +536,7 @@
       // figure out which field number corresponds to
       // "content", and then set our expected file names below
       // accordingly:
-      CompoundFileReader cfsReader = new CompoundFileReader(dir, "_0.cfs");
+      CompoundFileDirectory cfsReader = dir.openCompoundInput("_0.cfs", 1024);
       FieldInfos fieldInfos = new FieldInfos(cfsReader, "_0.fnm");
       int contentFieldIndex = -1;
       for (FieldInfo fi : fieldInfos) {
@@ -549,7 +549,7 @@
       assertTrue("could not locate the 'content' field number in the _2.cfs segment", contentFieldIndex != -1);
 
       // Now verify file names:
-      String[] expected = new String[] {"_0.cfs",
+      String[] expected = new String[] {"_0.cfs", "_0.cfe",
                                "_0_1.del",
                                "_0_1.s" + contentFieldIndex,
                                "segments_2",
Index: lucene/src/test/org/apache/lucene/index/TestCompoundFile.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestCompoundFile.java	(revision 1137870)
+++ lucene/src/test/org/apache/lucene/index/TestCompoundFile.java	(working copy)
@@ -23,6 +23,8 @@
 import org.apache.lucene.util.LuceneTestCase;
 import junit.framework.TestSuite;
 import junit.textui.TestRunner;
+
+import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
@@ -200,11 +202,11 @@
         for (int i=0; i<data.length; i++) {
             String name = "t" + data[i];
             createSequenceFile(dir, name, (byte) 0, data[i]);
-            CompoundFileWriter csw = new CompoundFileWriter(dir, name + ".cfs");
-            csw.addFile(name);
+            CompoundFileDirectory csw = dir.openCompoundOutput(name + ".cfs");
+            dir.copy(csw, name, name);
             csw.close();
 
-            CompoundFileReader csr = new CompoundFileReader(dir, name + ".cfs");
+            CompoundFileDirectory csr = dir.openCompoundInput(name + ".cfs", 1024);
             IndexInput expected = dir.openInput(name);
             IndexInput actual = csr.openInput(name);
             assertSameStreams(name, expected, actual);
@@ -223,12 +225,12 @@
         createSequenceFile(dir, "d1", (byte) 0, 15);
         createSequenceFile(dir, "d2", (byte) 0, 114);
 
-        CompoundFileWriter csw = new CompoundFileWriter(dir, "d.csf");
-        csw.addFile("d1");
-        csw.addFile("d2");
+        CompoundFileDirectory csw = dir.openCompoundOutput("d.csf");
+        dir.copy(csw, "d1", "d1");
+        dir.copy(csw, "d2", "d2");
         csw.close();
 
-        CompoundFileReader csr = new CompoundFileReader(dir, "d.csf");
+        CompoundFileDirectory csr = dir.openCompoundInput("d.csf", 1024);
         IndexInput expected = dir.openInput("d1");
         IndexInput actual = csr.openInput("d1");
         assertSameStreams("d1", expected, actual);
@@ -273,17 +275,18 @@
         createRandomFile(dir, segment + ".notIn2", 51);
 
         // Now test
-        CompoundFileWriter csw = new CompoundFileWriter(dir, "test.cfs");
+        CompoundFileDirectory csw = dir.openCompoundOutput("test.cfs");
         final String data[] = new String[] {
             ".zero", ".one", ".ten", ".hundred", ".big1", ".big2", ".big3",
             ".big4", ".big5", ".big6", ".big7"
         };
         for (int i=0; i<data.length; i++) {
-            csw.addFile(segment + data[i]);
+            String fileName = segment + data[i];
+            dir.copy(csw, fileName, fileName);
         }
         csw.close();
 
-        CompoundFileReader csr = new CompoundFileReader(dir, "test.cfs");
+        CompoundFileDirectory csr = dir.openCompoundInput("test.cfs", 1024);
         for (int i=0; i<data.length; i++) {
             IndexInput check = dir.openInput(segment + data[i]);
             IndexInput test = csr.openInput(segment + data[i]);
@@ -302,10 +305,11 @@
      *  the size of each file is 1000 bytes.
      */
     private void setUp_2() throws IOException {
-        CompoundFileWriter cw = new CompoundFileWriter(dir, "f.comp");
+        CompoundFileDirectory cw = dir.openCompoundOutput("f.comp");
         for (int i=0; i<20; i++) {
             createSequenceFile(dir, "f" + i, (byte) 0, 2000);
-            cw.addFile("f" + i);
+            String fileName = "f" + i;
+            dir.copy(cw, fileName, fileName);
         }
         cw.close();
     }
@@ -350,26 +354,9 @@
         }
     }
 
-
-    static boolean isCSIndexInput(IndexInput is) {
-        return is instanceof CompoundFileReader.CSIndexInput;
-    }
-
-    static boolean isCSIndexInputOpen(IndexInput is) throws IOException {
-        if (isCSIndexInput(is)) {
-            CompoundFileReader.CSIndexInput cis =
-            (CompoundFileReader.CSIndexInput) is;
-
-            return _TestHelper.isSimpleFSIndexInputOpen(cis.base);
-        } else {
-            return false;
-        }
-    }
-
-
     public void testClonedStreamsClosing() throws IOException {
         setUp_2();
-        CompoundFileReader cr = new CompoundFileReader(dir, "f.comp");
+        CompoundFileDirectory cr = dir.openCompoundInput("f.comp", 1024);
 
         // basic clone
         IndexInput expected = dir.openInput("f11");
@@ -379,10 +366,8 @@
         assertTrue(_TestHelper.isSimpleFSIndexInputOpen(expected));
 
         IndexInput one = cr.openInput("f11");
-        assertTrue(isCSIndexInputOpen(one));
 
         IndexInput two = (IndexInput) one.clone();
-        assertTrue(isCSIndexInputOpen(two));
 
         assertSameStreams("basic clone one", expected, one);
         expected.seek(0);
@@ -390,7 +375,6 @@
 
         // Now close the first stream
         one.close();
-        assertTrue("Only close when cr is closed", isCSIndexInputOpen(one));
 
         // The following should really fail since we couldn't expect to
         // access a file once close has been called on it (regardless of
@@ -402,8 +386,6 @@
 
         // Now close the compound reader
         cr.close();
-        assertFalse("Now closed one", isCSIndexInputOpen(one));
-        assertFalse("Now closed two", isCSIndexInputOpen(two));
 
         // The following may also fail since the compound stream is closed
         expected.seek(0);
@@ -426,7 +408,7 @@
      */
     public void testRandomAccess() throws IOException {
         setUp_2();
-        CompoundFileReader cr = new CompoundFileReader(dir, "f.comp");
+        CompoundFileDirectory cr = dir.openCompoundInput("f.comp", 1024);
 
         // Open two files
         IndexInput e1 = dir.openInput("f11");
@@ -505,7 +487,7 @@
      */
     public void testRandomAccessClones() throws IOException {
         setUp_2();
-        CompoundFileReader cr = new CompoundFileReader(dir, "f.comp");
+        CompoundFileDirectory cr = dir.openCompoundInput("f.comp", 1024);
 
         // Open two files
         IndexInput e1 = cr.openInput("f11");
@@ -582,7 +564,7 @@
 
     public void testFileNotFound() throws IOException {
         setUp_2();
-        CompoundFileReader cr = new CompoundFileReader(dir, "f.comp");
+        CompoundFileDirectory cr = dir.openCompoundInput("f.comp", 1024);
 
         // Open two files
         try {
@@ -600,7 +582,7 @@
 
     public void testReadPastEOF() throws IOException {
         setUp_2();
-        CompoundFileReader cr = new CompoundFileReader(dir, "f.comp");
+        CompoundFileDirectory cr = dir.openCompoundInput("f.comp", 1024);
         IndexInput is = cr.openInput("f2");
         is.seek(is.length() - 10);
         byte b[] = new byte[100];
@@ -653,11 +635,11 @@
        createSequenceFile(dir, "d1", (byte) 0, 15);
 
        Directory newDir = newDirectory();
-       CompoundFileWriter csw = new CompoundFileWriter(newDir, "d.csf");
-       csw.addFile("d1", dir);
+       CompoundFileDirectory csw = newDir.openCompoundOutput("d.csf");
+       dir.copy(csw, "d1", "d1");
        csw.close();
 
-       CompoundFileReader csr = new CompoundFileReader(newDir, "d.csf");
+       CompoundFileDirectory csr = newDir.openCompoundInput("d.csf", 1024);
        IndexInput expected = dir.openInput("d1");
        IndexInput actual = csr.openInput("d1");
        assertSameStreams("d1", expected, actual);
@@ -668,5 +650,72 @@
        
        newDir.close();
    }
+   
+   
+  public void testAppend() throws IOException {
+    Directory newDir = newDirectory();
+    CompoundFileDirectory csw = newDir.openCompoundOutput("d.csf");
+    int size = 5 + random.nextInt(128);
+    for (int j = 0; j < 2; j++) {
+      IndexOutput os = csw.createOutput("seg" + j + "_foo.txt");
+      for (int i = 0; i < size; i++) {
+        os.writeInt(i);
+      }
+      os.close();
+      String[] listAll = newDir.listAll();
+      assertEquals(1, listAll.length);
+      assertEquals("d.csf", listAll[0]);
+    }
+    createSequenceFile(dir, "d1", (byte) 0, 15);
+    dir.copy(csw, "d1", "d1");
+    String[] listAll = newDir.listAll();
+    assertEquals(1, listAll.length);
+    assertEquals("d.csf", listAll[0]);
+    csw.close();
+    CompoundFileDirectory csr = newDir.openCompoundInput("d.csf", 1024);
+    for (int j = 0; j < 2; j++) {
+      IndexInput openInput = csr.openInput("seg" + j + "_foo.txt");
+      assertEquals(size * 4, openInput.length());
+      for (int i = 0; i < size; i++) {
+        assertEquals(i, openInput.readInt());
+      }
 
+      openInput.close();
+
+    }
+    IndexInput expected = dir.openInput("d1");
+    IndexInput actual = csr.openInput("d1");
+    assertSameStreams("d1", expected, actual);
+    assertSameSeekBehavior("d1", expected, actual);
+    expected.close();
+    actual.close();
+    csr.close();
+    newDir.close();
+  }
+  
+  public void testAppendTwice() throws IOException {
+    Directory newDir = newDirectory();
+    CompoundFileDirectory csw = newDir.openCompoundOutput("d.csf");
+    createSequenceFile(newDir, "d1", (byte) 0, 15);
+    IndexOutput out = csw.createOutput("d.xyz");
+    out.writeInt(0);
+    try {
+      newDir.copy(csw, "d1", "d1");
+      fail("file does already exist");
+    } catch (IOException e) {
+      //
+    }
+    out.close();
+    assertEquals(1, csw.listAll().length);
+    assertEquals("d.xyz", csw.listAll()[0]);
+   
+    csw.close();
+
+    CompoundFileDirectory cfr = newDir.openCompoundInput("d.csf", 1024);
+    assertEquals(1, cfr.listAll().length);
+    assertEquals("d.xyz", cfr.listAll()[0]);
+    cfr.close();
+    newDir.close();
+  }
+
 }
Index: lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java	(revision 1137870)
+++ lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java	(working copy)
@@ -20,6 +20,7 @@
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.search.DefaultSimilarity;
 import org.apache.lucene.search.Similarity;
+import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
@@ -91,7 +92,7 @@
     // figure out which field number corresponds to
     // "content", and then set our expected file names below
     // accordingly:
-    CompoundFileReader cfsReader = new CompoundFileReader(dir, "_2.cfs");
+    CompoundFileDirectory cfsReader = dir.openCompoundInput("_2.cfs", 1024);
     FieldInfos fieldInfos = new FieldInfos(cfsReader, "_2.fnm");
     int contentFieldIndex = -1;
     for (FieldInfo fi : fieldInfos) {
