You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by mi...@apache.org on 2007/03/13 10:06:23 UTC

svn commit: r517599 [1/2] - in /lucene/java/trunk: ./ src/java/org/apache/lucene/index/ src/site/src/documentation/content/xdocs/ src/test/org/apache/lucene/index/ src/test/org/apache/lucene/store/

Author: mikemccand
Date: Tue Mar 13 02:06:22 2007
New Revision: 517599

URL: http://svn.apache.org/viewvc?view=rev&rev=517599
Log:
LUCENE-710: add optional autoCommit=true/false mode and customizable IndexDeletionPolicy to IndexWriter

Added:
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexCommitPoint.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexDeletionPolicy.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java   (with props)
    lucene/java/trunk/src/test/org/apache/lucene/index/TestDeletionPolicy.java   (with props)
Modified:
    lucene/java/trunk/CHANGES.txt
    lucene/java/trunk/src/java/org/apache/lucene/index/DocumentWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileNameFilter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileNames.java
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/MultiReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfo.java
    lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java
    lucene/java/trunk/src/java/org/apache/lucene/index/SegmentReader.java
    lucene/java/trunk/src/site/src/documentation/content/xdocs/fileformats.xml
    lucene/java/trunk/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReader.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
    lucene/java/trunk/src/test/org/apache/lucene/store/MockRAMOutputStream.java

Modified: lucene/java/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/java/trunk/CHANGES.txt?view=diff&rev=517599&r1=517598&r2=517599
==============================================================================
--- lucene/java/trunk/CHANGES.txt (original)
+++ lucene/java/trunk/CHANGES.txt Tue Mar 13 02:06:22 2007
@@ -20,6 +20,15 @@
     classes, package-private again (they were unnecessarily made public
     as part of LUCENE-701).  (Mike McCandless)
 
+ 3. LUCENE-710: added optional autoCommit boolean to IndexWriter
+    constructors.  When this is false, index changes are not committed
+    until the writer is closed.  This gives explicit control over when
+    a reader will see the changes.  Also added optional custom
+    deletion policy to explicitly control when prior commits are
+    removed from the index.  This is intended to allow applications to
+    share an index over NFS by customizing when prior commits are
+    deleted. (Mike McCandless)
+
 Bug fixes
 
  1. LUCENE-804: Fixed build.xml to pack a fully compilable src dist.  (Doron Cohen)

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/DocumentWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocumentWriter.java?view=diff&rev=517599&r1=517598&r2=517599
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocumentWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocumentWriter.java Tue Mar 13 02:06:22 2007
@@ -388,6 +388,9 @@
     this.infoStream = infoStream;
   }
 
+  int getNumFields() {
+    return fieldInfos.size();
+  }
 }
 
 final class Posting {				  // info about a Term in a doc

Added: lucene/java/trunk/src/java/org/apache/lucene/index/IndexCommitPoint.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexCommitPoint.java?view=auto&rev=517599
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexCommitPoint.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexCommitPoint.java Tue Mar 13 02:06:22 2007
@@ -0,0 +1,41 @@
+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.
+ */
+
+/**
+ * Represents a single commit into an index as seen by the
+ * {@link IndexDeletionPolicy}.
+ */
+
+public interface IndexCommitPoint {
+
+  /**
+   * Get the segments file (ie, <code>segments_N</code>) of
+   * this commit point.
+   */
+  public String getSegmentsFileName();
+  
+  /**
+   * Notify the writer that this commit point should be
+   * deleted.  This should only be called by the {@link
+   * IndexDeletionPolicy} during its {@link
+   * IndexDeletionPolicy#onInit} or {@link
+  * IndexDeletionPolicy#onCommit} method.
+  */
+  public void delete();
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/index/IndexCommitPoint.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/src/java/org/apache/lucene/index/IndexDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexDeletionPolicy.java?view=auto&rev=517599
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexDeletionPolicy.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexDeletionPolicy.java Tue Mar 13 02:06:22 2007
@@ -0,0 +1,83 @@
+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.util.List;
+import java.io.IOException;
+
+/**
+ * <p>Expert: implement this interface, and pass it to one
+ * of the {@link IndexWriter} or {@link IndexReader}
+ * constructors, to customize when "point in time" commits
+ * are deleted from an index.  The default deletion policy
+ * is {@link KeepOnlyLastCommitDeletionPolicy}, which always
+ * removes old commits as soon as a new commit is done (this
+ * matches the behavior before 2.2).</p>
+ *
+ * <p>One expected use case for this (and the reason why it
+ * was first created) is to work around problems with an
+ * index directory accessed via filesystems like NFS because
+ * NFS does not provide the "delete on last close" semantics
+ * that Lucene's "point in time" search normally relies on.
+ * By implementing a custom deletion policy, such as "a
+ * commit is only removed once it has been stale for more
+ * than X minutes", you can give your readers time to
+ * refresh to the new commit before {@link IndexWriter}
+ * removes the old commits.  Note that doing so will
+ * increase the storage requirements of the index.  See <a
+ * target="top"
+ * href="http://issues.apache.org/jira/browse/LUCENE-710">LUCENE-710</a>
+ * for details.</p>
+ */
+
+public interface IndexDeletionPolicy {
+
+  /**
+   * <p>This is called once when a writer is first
+   * instantiated to give the policy a chance to remove old
+   * commit points.</p>
+   * 
+   * <p>The writer locates all commits present in the index
+   * and calls this method.  The policy may choose to delete
+   * commit points.  To delete a commit point, call the
+   * {@link IndexCommitPoint#delete} method.</p>
+   *
+   * @param commits List of {@link IndexCommitPoint},
+   *  sorted by age (the 0th one is the oldest commit).
+   */
+  public void onInit(List commits) throws IOException;
+
+  /**
+   * <p>This is called each time the writer commits.  This
+   * gives the policy a chance to remove old commit points
+   * with each commit.</p>
+   *
+   * <p>If writer has <code>autoCommit = true</code> then
+   * this method will in general be called many times during
+   * one instance of {@link IndexWriter}.  If
+   * <code>autoCommit = false</code> then this method is
+   * only called once when {@link IndexWriter#close} is
+   * called, or not at all if the {@link IndexWriter#abort}
+   * is called.  The policy may now choose to delete old
+   * commit points by calling {@link IndexCommitPoint#delete}.
+   *
+   * @param commits List of {@link IndexCommitPoint}>,
+   *  sorted by age (the 0th one is the oldest commit).
+   */
+  public void onCommit(List commits) throws IOException;
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/index/IndexDeletionPolicy.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java?view=diff&rev=517599&r1=517598&r2=517599
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java Tue Mar 13 02:06:22 2007
@@ -18,286 +18,488 @@
  */
 
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.IndexFileNameFilter;
 import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.store.Directory;
 
 import java.io.IOException;
 import java.io.PrintStream;
-import java.util.Vector;
+import java.util.Map;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
-
-/**
- * A utility class (used by both IndexReader and
- * IndexWriter) to keep track of files that need to be
- * deleted because they are no longer referenced by the
- * index.
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Collections;
+
+/*
+ * This class keeps track of each SegmentInfos instance that
+ * is still "live", either because it corresponds to a
+ * segments_N in the Directory (a real commit) or because
+ * it's the in-memory SegmentInfos that a writer is actively
+ * updating but has not yet committed (currently this only
+ * applies when autoCommit=false in IndexWriter).  This
+ * class uses simple reference counting to map the live
+ * SegmentInfos instances to individual files in the
+ * Directory.
+ *
+ * A separate deletion policy interface
+ * (IndexDeletionPolicy) is consulted on creation (onInit)
+ * and once per commit (onCommit), to decide when a commit
+ * should be removed.
+ * 
+ * The current default deletion policy is {@link
+ * KeepOnlyLastCommitDeletionPolicy}, which removes all
+ * prior commits when a new commit has completed.  This
+ * matches the behavior before 2.2.
+ *
+ * Note that you must hold the write.lock before
+ * instantiating this class.  It opens segments_N file(s)
+ * directly with no retry logic.
  */
+
 final class IndexFileDeleter {
-  private Vector deletable;
-  private HashSet pending;
-  private Directory directory;
-  private SegmentInfos segmentInfos;
-  private PrintStream infoStream;
 
-  IndexFileDeleter(SegmentInfos segmentInfos, Directory directory)
-    throws IOException {
-    this.segmentInfos = segmentInfos;
-    this.directory = directory;
-  }
-  void setSegmentInfos(SegmentInfos segmentInfos) {
-    this.segmentInfos = segmentInfos;
-  }
-  SegmentInfos getSegmentInfos() {
-    return segmentInfos;
-  }
+  /* Files that we tried to delete but failed (likely
+   * because they are open and we are running on Windows),
+   * so we will retry them again later: */
+  private List deletable;
+
+  /* Reference count for all files in the index.  Maps
+   * String to RefCount (class below) instances: */
+  private Map refCounts = new HashMap();
+
+  /* Holds all commits (segments_N) currently in the index.
+   * This will have just 1 commit if you are using the
+   * default delete policy (KeepOnlyLastCommitDeletionPolicy).
+   * Other policies may leave commit points live for longer
+   * in which case this list would be longer than 1: */
+  private List commits = new ArrayList();
+
+  /* Holds files we had incref'd from the previous
+   * non-commit checkpoint: */
+  private List lastFiles = new ArrayList();
+
+  private PrintStream infoStream;
+  private List toDelete = new ArrayList();
+  private Directory directory;
+  private IndexDeletionPolicy policy;
 
   void setInfoStream(PrintStream infoStream) {
     this.infoStream = infoStream;
   }
+  
+  private void message(String message) {
+    infoStream.println(this + " " + Thread.currentThread().getName() + ": " + message);
+  }
 
-  /** Determine index files that are no longer referenced
-   * and therefore should be deleted.  This is called once
-   * (by the writer), and then subsequently we add onto
-   * deletable any files that are no longer needed at the
-   * point that we create the unused file (eg when merging
-   * segments), and we only remove from deletable when a
-   * file is successfully deleted.
+  /**
+   * Initialize the deleter: find all previous commits in
+   * the Directory, incref the files they reference, call
+   * the policy to let it delete commits.  The incoming
+   * segmentInfos must have been loaded from a commit point
+   * and not yet modified.  This will remove any files not
+   * referenced by any of the commits.
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
    */
+  public IndexFileDeleter(Directory directory, IndexDeletionPolicy policy, SegmentInfos segmentInfos, PrintStream infoStream)
+    throws CorruptIndexException, IOException {
 
-  void findDeletableFiles() throws IOException {
-
-    // Gather all "current" segments:
-    HashMap current = new HashMap();
-    for(int j=0;j<segmentInfos.size();j++) {
-      SegmentInfo segmentInfo = (SegmentInfo) segmentInfos.elementAt(j);
-      current.put(segmentInfo.name, segmentInfo);
-    }
-
-    // Then go through all files in the Directory that are
-    // Lucene index files, and add to deletable if they are
-    // not referenced by the current segments info:
+    this.infoStream = infoStream;
+    this.policy = policy;
+    this.directory = directory;
 
-    String segmentsInfosFileName = segmentInfos.getCurrentSegmentFileName();
+    // First pass: walk the files and initialize our ref
+    // counts:
+    long currentGen = segmentInfos.getGeneration();
     IndexFileNameFilter filter = IndexFileNameFilter.getFilter();
 
     String[] files = directory.list();
     if (files == null)
       throw new IOException("cannot read directory " + directory + ": list() returned null");
 
-    for (int i = 0; i < files.length; i++) {
+    CommitPoint currentCommitPoint = null;
 
-      if (filter.accept(null, files[i]) && !files[i].equals(segmentsInfosFileName) && !files[i].equals(IndexFileNames.SEGMENTS_GEN)) {
+    for(int i=0;i<files.length;i++) {
 
-        String segmentName;
-        String extension;
+      String fileName = files[i];
 
-        // First remove any extension:
-        int loc = files[i].indexOf('.');
-        if (loc != -1) {
-          extension = files[i].substring(1+loc);
-          segmentName = files[i].substring(0, loc);
-        } else {
-          extension = null;
-          segmentName = files[i];
-        }
+      if (filter.accept(null, fileName) && !fileName.equals(IndexFileNames.SEGMENTS_GEN)) {
 
-        // Then, remove any generation count:
-        loc = segmentName.indexOf('_', 1);
-        if (loc != -1) {
-          segmentName = segmentName.substring(0, loc);
-        }
+        // Add this file to refCounts with initial count 0:
+        getRefCount(fileName);
+
+        if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
 
-        // Delete this file if it's not a "current" segment,
-        // or, it is a single index file but there is now a
-        // corresponding compound file:
-        boolean doDelete = false;
-
-        if (!current.containsKey(segmentName)) {
-          // Delete if segment is not referenced:
-          doDelete = true;
-        } else {
-          // OK, segment is referenced, but file may still
-          // be orphan'd:
-          SegmentInfo info = (SegmentInfo) current.get(segmentName);
-
-          if (filter.isCFSFile(files[i]) && info.getUseCompoundFile()) {
-            // This file is in fact stored in a CFS file for
-            // this segment:
-            doDelete = true;
-          } else {
-            
-            if ("del".equals(extension)) {
-              // This is a _segmentName_N.del file:
-              if (!files[i].equals(info.getDelFileName())) {
-                // If this is a seperate .del file, but it
-                // doesn't match the current del filename for
-                // this segment, then delete it:
-                doDelete = true;
-              }
-            } else if (extension != null && extension.startsWith("s") && extension.matches("s\\d+")) {
-              int field = Integer.parseInt(extension.substring(1));
-              // This is a _segmentName_N.sX file:
-              if (!files[i].equals(info.getNormFileName(field))) {
-                // This is an orphan'd separate norms file:
-                doDelete = true;
-              }
-            } else if ("cfs".equals(extension) && !info.getUseCompoundFile()) {
-              // This is a partially written
-              // _segmentName.cfs:
-              doDelete = true;
+          // This is a commit (segments or segments_N), and
+          // it's valid (<= the max gen).  Load it, then
+          // incref all files it refers to:
+          if (SegmentInfos.generationFromSegmentsFileName(fileName) <= currentGen) {
+            if (infoStream != null) {
+              message("init: load commit \"" + fileName + "\"");
             }
+            SegmentInfos sis = new SegmentInfos();
+            sis.read(directory, fileName);
+            CommitPoint commitPoint = new CommitPoint(sis);
+            if (sis.getGeneration() == segmentInfos.getGeneration()) {
+              currentCommitPoint = commitPoint;
+            }
+            commits.add(commitPoint);
+            incRef(sis, true);
           }
         }
+      }
+    }
 
-        if (doDelete) {
-          addDeletableFile(files[i]);
-          if (infoStream != null) {
-            infoStream.println("IndexFileDeleter: file \"" + files[i] + "\" is unreferenced in index and will be deleted on next commit");
-          }
+    if (currentCommitPoint == null) {
+      throw new CorruptIndexException("failed to locate current segments_N file");
+    }
+
+    // We keep commits list in sorted order (oldest to newest):
+    Collections.sort(commits);
+
+    // Now delete anything with ref count at 0.  These are
+    // presumably abandoned files eg due to crash of
+    // IndexWriter.
+    Iterator it = refCounts.keySet().iterator();
+    while(it.hasNext()) {
+      String fileName = (String) it.next();
+      RefCount rc = (RefCount) refCounts.get(fileName);
+      if (0 == rc.count) {
+        if (infoStream != null) {
+          message("init: removing unreferenced file \"" + fileName + "\"");
         }
+        deleteFile(fileName);
       }
     }
+
+    // Finally, give policy a chance to remove things on
+    // startup:
+    policy.onInit(commits);
+
+    // It's OK for the onInit to remove the current commit
+    // point; we just have to checkpoint our in-memory
+    // SegmentInfos to protect those files that it uses:
+    if (currentCommitPoint.deleted) {
+      checkpoint(segmentInfos, false);
+    }
+    
+    deleteCommits();
   }
 
-  /*
-   * Some operating systems (e.g. Windows) don't permit a file to be deleted
-   * while it is opened for read (e.g. by another process or thread). So we
-   * assume that when a delete fails it is because the file is open in another
-   * process, and queue the file for subsequent deletion.
+  /**
+   * Remove the CommitPoints in the toDelete List by
+   * DecRef'ing all files from each SegmentInfos.
    */
+  private void deleteCommits() throws IOException {
+
+    int size = toDelete.size();
 
-  void deleteSegments(Vector segments) throws IOException {
+    if (size > 0) {
 
-    deleteFiles();                                // try to delete files that we couldn't before
+      // First decref all files that had been referred to by
+      // the now-deleted commits:
+      for(int i=0;i<size;i++) {
+        CommitPoint commit = (CommitPoint) toDelete.get(i);
+        if (infoStream != null) {
+          message("deleteCommits: now remove commit \"" + commit.getSegmentsFileName() + "\"");
+        }
+        int size2 = commit.files.size();
+        for(int j=0;j<size2;j++) {
+          decRef((List) commit.files.get(j));
+        }
+        decRef(commit.getSegmentsFileName());
+      }
+      toDelete.clear();
 
-    for (int i = 0; i < segments.size(); i++) {
-      SegmentReader reader = (SegmentReader)segments.elementAt(i);
-      if (reader.directory() == this.directory)
-        deleteFiles(reader.files()); // try to delete our files
-      else
-        deleteFiles(reader.files(), reader.directory()); // delete other files
+      // Now compact commits to remove deleted ones:
+      size = commits.size();
+      int readFrom = 0;
+      int writeTo = 0;
+      while(readFrom < size) {
+        CommitPoint commit = (CommitPoint) commits.get(readFrom);
+        if (!commit.deleted) {
+          if (writeTo != readFrom) {
+            commits.set(writeTo, commits.get(readFrom));
+          }
+          writeTo++;
+        }
+        readFrom++;
+      }
+
+      while(size > writeTo) {
+        commits.remove(size-1);
+        size--;
+      }
     }
   }
 
   /**
-   * Delete these segments, as long as they are not listed
-   * in protectedSegments.  If they are, then, instead, add
-   * them to the pending set.
-  */
-     
-  void deleteSegments(Vector segments, HashSet protectedSegments) throws IOException {
-
-    deleteFiles();                                // try to delete files that we couldn't before
-
-    for (int i = 0; i < segments.size(); i++) {
-      SegmentReader reader = (SegmentReader)segments.elementAt(i);
-      if (reader.directory() == this.directory) {
-        if (protectedSegments.contains(reader.getSegmentName())) {
-          addPendingFiles(reader.files()); // record these for deletion on commit
-        } else {
-          deleteFiles(reader.files()); // try to delete our files
+   * Writer calls this when it has hit an error and had to
+   * roll back, to tell us that there may now be
+   * unreferenced files in the filesystem.  So we re-list
+   * the filesystem and delete such files:
+   */
+  public void refresh() throws IOException {
+    String[] files = directory.list();
+    if (files == null)
+      throw new IOException("cannot read directory " + directory + ": list() returned null");
+    IndexFileNameFilter filter = IndexFileNameFilter.getFilter();
+    for(int i=0;i<files.length;i++) {
+      String fileName = files[i];
+      if (filter.accept(null, fileName) && !refCounts.containsKey(fileName) && !fileName.equals(IndexFileNames.SEGMENTS_GEN)) {
+        // Unreferenced file, so remove it
+        if (infoStream != null) {
+          message("refresh: removing newly created unreferenced file \"" + fileName + "\"");
         }
-      }  else {
-        deleteFiles(reader.files(), reader.directory()); // delete other files
+        deleteFile(fileName);
       }
     }
   }
-  
-  void deleteFiles(Vector files, Directory directory)
-       throws IOException {
-    for (int i = 0; i < files.size(); i++)
-      directory.deleteFile((String)files.elementAt(i));
-  }
 
-  void deleteFiles(Vector files)
-       throws IOException {
-    deleteFiles();                                // try to delete files that we couldn't before
-    for (int i = 0; i < files.size(); i++) {
-      deleteFile((String) files.elementAt(i));
+  /**
+   * Writer calls this when it has made a "consistent
+   * change" to the index, meaning new files are written to
+   * the index and the in-memory SegmentInfos have been
+   * modified to point to those files.
+   *
+   * This may or may not be a commit (segments_N may or may
+   * not have been written).
+   *
+   * We simply incref the files referenced by the new
+   * SegmentInfos and decref the files we had previously
+   * seen (if any).
+   *
+   * If this is a commit, we also call the policy to give it
+   * a chance to remove other commits.  If any commits are
+   * removed, we decref their files as well.
+   */
+  public void checkpoint(SegmentInfos segmentInfos, boolean isCommit) throws IOException {
+
+    if (infoStream != null) {
+      message("now checkpoint \"" + segmentInfos.getCurrentSegmentFileName() + "\" [isCommit = " + isCommit + "]");
     }
-  }
 
-  void deleteFile(String file)
-       throws IOException {
-    try {
-      directory.deleteFile(file);		  // try to delete each file
-    } catch (IOException e) {			  // if delete fails
-      if (directory.fileExists(file)) {
-        if (infoStream != null)
-          infoStream.println("IndexFileDeleter: unable to remove file \"" + file + "\": " + e.toString() + "; Will re-try later.");
-        addDeletableFile(file);                  // add to deletable
+    // Try again now to delete any previously un-deletable
+    // files (because they were in use, on Windows):
+    if (deletable != null) {
+      List oldDeletable = deletable;
+      deletable = null;
+      int size = oldDeletable.size();
+      for(int i=0;i<size;i++) {
+        deleteFile((String) oldDeletable.get(i));
+      }
+    }
+
+    // Incref the files:
+    incRef(segmentInfos, isCommit);
+
+    if (isCommit) {
+      // Append to our commits list:
+      commits.add(new CommitPoint(segmentInfos));
+
+      // Tell policy so it can remove commits:
+      policy.onCommit(commits);
+
+      // Decref files for commits that were deleted by the policy:
+      deleteCommits();
+    }
+
+    // DecRef old files from the last checkpoint, if any:
+    int size = lastFiles.size();
+    if (size > 0) {
+      for(int i=0;i<size;i++) {
+        decRef((List) lastFiles.get(i));
+      }
+      lastFiles.clear();
+    }
+
+    if (!isCommit) {
+      // Save files so we can decr on next checkpoint/commit:
+      size = segmentInfos.size();
+      for(int i=0;i<size;i++) {
+        SegmentInfo segmentInfo = segmentInfos.info(i);
+        if (segmentInfo.dir == directory) {
+          lastFiles.add(segmentInfo.files());
+        }
       }
     }
   }
 
-  void clearPendingFiles() {
-    pending = null;
+  private void incRef(SegmentInfos segmentInfos, boolean isCommit) throws IOException {
+    int size = segmentInfos.size();
+    for(int i=0;i<size;i++) {
+      SegmentInfo segmentInfo = segmentInfos.info(i);
+      if (segmentInfo.dir == directory) {
+        incRef(segmentInfo.files());
+      }
+    }
+
+    if (isCommit) {
+      // Since this is a commit point, also incref its
+      // segments_N file:
+      getRefCount(segmentInfos.getCurrentSegmentFileName()).IncRef();
+    }
   }
 
-  /*
-    Record that the files for these segments should be
-    deleted, once the pending deletes are committed.
-   */
-  void addPendingSegments(Vector segments) throws IOException {
-    for (int i = 0; i < segments.size(); i++) {
-      SegmentReader reader = (SegmentReader)segments.elementAt(i);
-      if (reader.directory() == this.directory) {
-        addPendingFiles(reader.files());
+  private void incRef(List files) throws IOException {
+    int size = files.size();
+    for(int i=0;i<size;i++) {
+      String fileName = (String) files.get(i);
+      RefCount rc = getRefCount(fileName);
+      if (infoStream != null) {
+        message("  IncRef \"" + fileName + "\": pre-incr count is " + rc.count);
       }
+      rc.IncRef();
     }
   }
 
-  /*
-    Record list of files for deletion, but do not delete
-    them until commitPendingFiles is called.
-  */
-  void addPendingFiles(Vector files) {
-    for(int i=0;i<files.size();i++) {
-      addPendingFile((String) files.elementAt(i));
+  private void decRef(List files) throws IOException {
+    int size = files.size();
+    for(int i=0;i<size;i++) {
+      decRef((String) files.get(i));
+    }
+  }
+
+  private void decRef(String fileName) throws IOException {
+    RefCount rc = getRefCount(fileName);
+    if (infoStream != null) {
+      message("  DecRef \"" + fileName + "\": pre-decr count is " + rc.count);
+    }
+    if (0 == rc.DecRef()) {
+      // This file is no longer referenced by any past
+      // commit points nor by the in-memory SegmentInfos:
+      deleteFile(fileName);
+      refCounts.remove(fileName);
     }
   }
 
-  /*
-    Record a file for deletion, but do not delete it until
-    commitPendingFiles is called.
-  */
-  void addPendingFile(String fileName) {
-    if (pending == null) {
-      pending = new HashSet();
+  private RefCount getRefCount(String fileName) {
+    RefCount rc;
+    if (!refCounts.containsKey(fileName)) {
+      rc = new RefCount();
+      refCounts.put(fileName, rc);
+    } else {
+      rc = (RefCount) refCounts.get(fileName);
     }
-    pending.add(fileName);
+    return rc;
   }
 
-  void commitPendingFiles() throws IOException {
-    if (pending != null) {
-      if (deletable == null) {
-        deletable = new Vector();
+  private void deleteFile(String fileName)
+       throws IOException {
+    try {
+      if (infoStream != null) {
+        message("delete \"" + fileName + "\"");
       }
-      Iterator it = pending.iterator();
-      while(it.hasNext()) {
-        deletable.addElement(it.next());
+      directory.deleteFile(fileName);
+    } catch (IOException e) {			  // if delete fails
+      if (directory.fileExists(fileName)) {
+
+        // Some operating systems (e.g. Windows) don't
+        // permit a file to be deleted while it is opened
+        // for read (e.g. by another process or thread). So
+        // we assume that when a delete fails it is because
+        // the file is open in another process, and queue
+        // the file for subsequent deletion.
+
+        if (infoStream != null) {
+          message("IndexFileDeleter: unable to remove file \"" + fileName + "\": " + e.toString() + "; Will re-try later.");
+        }
+        if (deletable == null) {
+          deletable = new ArrayList();
+        }
+        deletable.add(fileName);                  // add to deletable
       }
-      pending = null;
-      deleteFiles();
     }
   }
 
-  void addDeletableFile(String fileName) {
-    if (deletable == null) {
-      deletable = new Vector();
+  /**
+   * Blindly delete the files used by the specific segments,
+   * with no reference counting and no retry.  This is only
+   * currently used by writer to delete its RAM segments
+   * from a RAMDirectory.
+   */
+  public void deleteDirect(Directory otherDir, List segments) throws IOException {
+    int size = segments.size();
+    for(int i=0;i<size;i++) {
+      List toDelete = ((SegmentInfo) segments.get(i)).files();
+      int size2 = toDelete.size();
+      for(int j=0;j<size2;j++) {
+        otherDir.deleteFile((String) toDelete.get(j));
+      }
     }
-    deletable.addElement(fileName);
   }
 
-  void deleteFiles()
-    throws IOException {
-    if (deletable != null) {
-      Vector oldDeletable = deletable;
-      deletable = null;
-      deleteFiles(oldDeletable); // try to delete deletable
+  /**
+   * Tracks the reference count for a single index file:
+   */
+  final private static class RefCount {
+
+    int count;
+
+    final private int IncRef() {
+      return ++count;
+    }
+
+    final private int DecRef() {
+      return --count;
+    }
+  }
+
+  /**
+   * Holds details for each commit point.  This class is
+   * also passed to the deletion policy.  Note: this class
+   * has a natural ordering that is inconsistent with
+   * equals.
+   */
+
+  final private class CommitPoint implements Comparable, IndexCommitPoint {
+
+    long gen;
+    List files;
+    String segmentsFileName;
+    boolean deleted;
+
+    public CommitPoint(SegmentInfos segmentInfos) throws IOException {
+      segmentsFileName = segmentInfos.getCurrentSegmentFileName();
+      int size = segmentInfos.size();
+      files = new ArrayList(size);
+      gen = segmentInfos.getGeneration();
+      for(int i=0;i<size;i++) {
+        SegmentInfo segmentInfo = segmentInfos.info(i);
+        if (segmentInfo.dir == directory) {
+          files.add(segmentInfo.files());
+        }
+      }
+    }
+
+    /**
+     * Get the segments_N file for this commit point.
+     */
+    public String getSegmentsFileName() {
+      return segmentsFileName;
+    }
+
+    /**
+     * Called only be the deletion policy, to remove this
+     * commit point from the index.
+     */
+    public void delete() {
+      if (!deleted) {
+        deleted = true;
+        toDelete.add(this);
+      }
+    }
+
+    public int compareTo(Object obj) {
+      CommitPoint commit = (CommitPoint) obj;
+      if (gen < commit.gen) {
+        return -1;
+      } else if (gen > commit.gen) {
+        return 1;
+      } else {
+        return 0;
+      }
     }
   }
 }

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileNameFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileNameFilter.java?view=diff&rev=517599&r1=517598&r2=517599
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileNameFilter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileNameFilter.java Tue Mar 13 02:06:22 2007
@@ -31,12 +31,17 @@
 
   static IndexFileNameFilter singleton = new IndexFileNameFilter();
   private HashSet extensions;
+  private HashSet extensionsInCFS;
 
   public IndexFileNameFilter() {
     extensions = new HashSet();
     for (int i = 0; i < IndexFileNames.INDEX_EXTENSIONS.length; i++) {
       extensions.add(IndexFileNames.INDEX_EXTENSIONS[i]);
     }
+    extensionsInCFS = new HashSet();
+    for (int i = 0; i < IndexFileNames.INDEX_EXTENSIONS_IN_COMPOUND_FILE.length; i++) {
+      extensionsInCFS.add(IndexFileNames.INDEX_EXTENSIONS_IN_COMPOUND_FILE[i]);
+    }
   }
 
   /* (non-Javadoc)
@@ -72,10 +77,7 @@
     int i = name.lastIndexOf('.');
     if (i != -1) {
       String extension = name.substring(1+i);
-      if (extensions.contains(extension) &&
-           !extension.equals("del") &&
-           !extension.equals("gen") &&
-          !extension.equals("cfs")) {
+      if (extensionsInCFS.contains(extension)) {
         return true;
       }
       if (extension.startsWith("f") &&

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileNames.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileNames.java?view=diff&rev=517599&r1=517598&r2=517599
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileNames.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileNames.java Tue Mar 13 02:06:22 2007
@@ -37,7 +37,19 @@
    
   /** Extension of norms file */
   static final String NORMS_EXTENSION = "nrm";
-  
+
+  /** Extension of compound file */
+  static final String COMPOUND_FILE_EXTENSION = "cfs";
+
+  /** Extension of deletes */
+  static final String DELETES_EXTENSION = "del";
+
+  /** Extension of single norms */
+  static final String SINGLE_NORMS_EXTENSION = "f";
+
+  /** Extension of separate norms */
+  static final String SEPARATE_NORMS_EXTENSION = "s";
+
   /**
    * This array contains all filename extensions used by
    * Lucene's index files, with two exceptions, namely the
@@ -49,6 +61,13 @@
   static final String INDEX_EXTENSIONS[] = new String[] {
       "cfs", "fnm", "fdx", "fdt", "tii", "tis", "frq", "prx", "del",
       "tvx", "tvd", "tvf", "gen", "nrm" 
+  };
+
+  /** File extensions that are added to a compound file
+   * (same as above, minus "del", "gen", "cfs"). */
+  static final String[] INDEX_EXTENSIONS_IN_COMPOUND_FILE = new String[] {
+      "fnm", "fdx", "fdt", "tii", "tis", "frq", "prx",
+      "tvx", "tvd", "tvf", "nrm" 
   };
   
   /** File extensions of old-style index files */

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexReader.java?view=diff&rev=517599&r1=517598&r2=517599
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexReader.java Tue Mar 13 02:06:22 2007
@@ -114,7 +114,7 @@
   private Directory directory;
   private boolean directoryOwner;
   private boolean closeDirectory;
-  protected IndexFileDeleter deleter;
+  private IndexDeletionPolicy deletionPolicy;
   private boolean isClosed;
 
   private SegmentInfos segmentInfos;
@@ -131,29 +131,43 @@
    path.
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
-   */
+   * @param path the path to the index directory */
   public static IndexReader open(String path) throws CorruptIndexException, IOException {
-    return open(FSDirectory.getDirectory(path), true);
+    return open(FSDirectory.getDirectory(path), true, null);
   }
 
   /** Returns an IndexReader reading the index in an FSDirectory in the named
-   path.
+   * path.
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
-  */
+   * @param path the path to the index directory */
   public static IndexReader open(File path) throws CorruptIndexException, IOException {
-    return open(FSDirectory.getDirectory(path), true);
+    return open(FSDirectory.getDirectory(path), true, null);
   }
 
   /** Returns an IndexReader reading the index in the given Directory.
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
+   * @param directory the index directory
    */
   public static IndexReader open(final Directory directory) throws CorruptIndexException, IOException {
-    return open(directory, false);
+    return open(directory, false, null);
+  }
+
+  /** Expert: returns an IndexReader reading the index in the given
+   * Directory, with a custom {@link IndexDeletionPolicy}.
+   * @param directory the index directory
+   * @param deletionPolicy a custom deletion policy (only used
+   *  if you use this reader to perform deletes or to set
+   *  norms); see {@link IndexWriter} for details.
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
+   */
+  public static IndexReader open(final Directory directory, IndexDeletionPolicy deletionPolicy) throws CorruptIndexException, IOException {
+    return open(directory, false, deletionPolicy);
   }
 
-  private static IndexReader open(final Directory directory, final boolean closeDirectory) throws CorruptIndexException, IOException {
+  private static IndexReader open(final Directory directory, final boolean closeDirectory, final IndexDeletionPolicy deletionPolicy) throws CorruptIndexException, IOException {
 
     return (IndexReader) new SegmentInfos.FindSegmentsFile(directory) {
 
@@ -162,8 +176,10 @@
         SegmentInfos infos = new SegmentInfos();
         infos.read(directory, segmentFileName);
 
+        IndexReader reader;
+
         if (infos.size() == 1) {		  // index is optimized
-          return SegmentReader.get(infos, infos.info(0), closeDirectory);
+          reader = SegmentReader.get(infos, infos.info(0), closeDirectory);
         } else {
 
           // To reduce the chance of hitting FileNotFound
@@ -184,8 +200,10 @@
             }
           }
 
-          return new MultiReader(directory, infos, closeDirectory, readers);
+          reader = new MultiReader(directory, infos, closeDirectory, readers);
         }
+        reader.deletionPolicy = deletionPolicy;
+        return reader;
       }
     }.run();
   }
@@ -715,19 +733,13 @@
    */
   protected final synchronized void commit() throws IOException {
     if(hasChanges){
-      if (deleter == null) {
-        // In the MultiReader case, we share this deleter
-        // across all SegmentReaders:
-        setDeleter(new IndexFileDeleter(segmentInfos, directory));
-      }
       if(directoryOwner){
 
-        // Should not be necessary: no prior commit should
-        // have left pending files, so just defensive:
-        deleter.clearPendingFiles();
-
-        String oldInfoFileName = segmentInfos.getCurrentSegmentFileName();
-        String nextSegmentsFileName = segmentInfos.getNextSegmentFileName();
+        // Default deleter (for backwards compatibility) is
+        // KeepOnlyLastCommitDeleter:
+        IndexFileDeleter deleter =  new IndexFileDeleter(directory,
+                                                         deletionPolicy == null ? new KeepOnlyLastCommitDeletionPolicy() : deletionPolicy,
+                                                         segmentInfos, null);
 
         // Checkpoint the state we are about to change, in
         // case we have to roll back:
@@ -749,24 +761,16 @@
             // actually in the index):
             rollbackCommit();
 
-            // Erase any pending files that we were going to delete:
-            deleter.clearPendingFiles();
-
-            // Remove possibly partially written next
-            // segments file:
-            deleter.deleteFile(nextSegmentsFileName);
-
             // Recompute deletable files & remove them (so
             // partially written .del files, etc, are
             // removed):
-            deleter.findDeletableFiles();
-            deleter.deleteFiles();
+            deleter.refresh();
           }
         }
 
-        // Attempt to delete all files we just obsoleted:
-        deleter.deleteFile(oldInfoFileName);
-        deleter.commitPendingFiles();
+        // Have the deleter remove any now unreferenced
+        // files due to this commit:
+        deleter.checkpoint(segmentInfos, true);
 
         if (writeLock != null) {
           writeLock.release();  // release write lock
@@ -777,13 +781,6 @@
         doCommit();
     }
     hasChanges = false;
-  }
-
-  protected void setDeleter(IndexFileDeleter deleter) {
-    this.deleter = deleter;
-  }
-  protected IndexFileDeleter getDeleter() {
-    return deleter;
   }
 
   /** Implements commit. */

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java?view=diff&rev=517599&r1=517598&r2=517599
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java Tue Mar 13 02:06:22 2007
@@ -29,48 +29,100 @@
 import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
-import java.util.Vector;
-import java.util.HashSet;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map.Entry;
 
 /**
-  An IndexWriter creates and maintains an index.
+  An <code>IndexWriter</code> creates and maintains an index.
 
-  <p>The third argument (<code>create</code>) to the 
+  <p>The <code>create</code> argument to the 
   <a href="#IndexWriter(org.apache.lucene.store.Directory, org.apache.lucene.analysis.Analyzer, boolean)"><b>constructor</b></a>
   determines whether a new index is created, or whether an existing index is
-  opened for the addition of new documents.  Note that you
-  can open an index with create=true even while readers are
+  opened.  Note that you
+  can open an index with <code>create=true</code> even while readers are
   using the index.  The old readers will continue to search
   the "point in time" snapshot they had opened, and won't
-  see the newly created index until they re-open.</p>
-
-  <p>In either case, documents are added with the <a
-  href="#addDocument(org.apache.lucene.document.Document)"><b>addDocument</b></a> method.  
-  When finished adding documents, <a href="#close()"><b>close</b></a> should be called.</p>
-
+  see the newly created index until they re-open.  There are
+  also <a href="#IndexWriter(org.apache.lucene.store.Directory, org.apache.lucene.analysis.Analyzer)"><b>constructors</b></a>
+  with no <code>create</code> argument which
+  will create a new index if there is not already an index at the
+  provided path and otherwise open the existing index.</p>
+
+  <p>In either case, documents are added with <a
+  href="#addDocument(org.apache.lucene.document.Document)"><b>addDocument</b></a>
+  and removed with <a
+  href="#deleteDocuments(org.apache.lucene.index.Term)"><b>deleteDocuments</b></a>.
+  A document can be updated with <a href="#updateDocument(org.apache.lucene.index.Term, org.apache.lucene.document.Document)"><b>updateDocument</b></a> 
+  (which just deletes and then adds). When finished adding, deleting and updating documents, <a href="#close()"><b>close</b></a> should be called.</p>
+
+  <p>These changes are buffered in memory and periodically
+  flushed to the {@link Directory} (during the above method calls).  A flush is triggered when there are
+  enough buffered deletes (see {@link
+  #setMaxBufferedDeleteTerms}) or enough added documents
+  (see {@link #setMaxBufferedDocs}) since the last flush,
+  whichever is sooner.  When a flush occurs, both pending
+  deletes and added documents are flushed to the index.  A
+  flush may also trigger one or more segment merges.</p>
+
+  <a name="autoCommit"></a>
+  <p>The optional <code>autoCommit</code> argument to the
+  <a href="#IndexWriter(org.apache.lucene.store.Directory, boolean, org.apache.lucene.analysis.Analyzer)"><b>constructors</b></a>
+  controls visibility of the changes to {@link IndexReader} instances reading the same index.
+  When this is <code>false</code>, changes are not
+  visible until {@link #close()} is called.
+  Note that changes will still be flushed to the
+  {@link org.apache.lucene.store.Directory} as new files,
+  but are not committed (no new <code>segments_N</code> file
+  is written referencing the new files) until {@link #close} is
+  called.  If something goes terribly wrong (for example the
+  JVM crashes) before {@link #close()}, then
+  the index will reflect none of the changes made (it will
+  remain in its starting state).
+  You can also call {@link #abort()}, which closes the writer without committing any
+  changes, and removes any index
+  files that had been flushed but are now unreferenced.
+  This mode is useful for preventing readers from refreshing
+  at a bad time (for example after you've done all your
+  deletes but before you've done your adds).
+  It can also be used to implement simple single-writer
+  transactional semantics ("all or none").</p>
+
+  <p>When <code>autoCommit</code> is <code>true</code> then
+  every flush is also a commit ({@link IndexReader}
+  instances will see each flush as changes to the index).
+  This is the default, to match the behavior before 2.2.
+  When running in this mode, be careful not to refresh your
+  readers while optimize or segment merges are taking place
+  as this can tie up substantial disk space.</p>
+  
   <p>If an index will not have more documents added for a while and optimal search
   performance is desired, then the <a href="#optimize()"><b>optimize</b></a>
   method should be called before the index is closed.</p>
-  
-  <p>Opening an IndexWriter creates a lock file for the directory in use. Trying to open
-  another IndexWriter on the same directory will lead to a
+
+  <p>Opening an <code>IndexWriter</code> creates a lock file for the directory in use. Trying to open
+  another <code>IndexWriter</code> on the same directory will lead to a
   {@link LockObtainFailedException}. The {@link LockObtainFailedException}
   is also thrown if an IndexReader on the same directory is used to delete documents
   from the index.</p>
   
-  <p>As of <b>2.1</b>, IndexWriter can now delete documents
-  by {@link Term} (see {@link #deleteDocuments} ) and update
-  (delete then add) documents (see {@link #updateDocument}).
-  Deletes are buffered until {@link
-  #setMaxBufferedDeleteTerms} <code>Terms</code> at which
-  point they are flushed to the index.  Note that a flush
-  occurs when there are enough buffered deletes or enough
-  added documents, whichever is sooner.  When a flush
-  occurs, both pending deletes and added documents are
-  flushed to the index.</p>
+  <a name="deletionPolicy"></a>
+  <p>Expert: <code>IndexWriter</code> allows an optional
+  {@link IndexDeletionPolicy} implementation to be
+  specified.  You can use this to control when prior commits
+  are deleted from the index.  The default policy is {@link
+  KeepOnlyLastCommitDeletionPolicy} which removes all prior
+  commits as soon as a new commit is done (this matches
+  behavior before 2.2).  Creating your own policy can allow
+  you to explicitly keep previous "point in time" commits
+  alive in the index for some time, to allow readers to
+  refresh to the new commit without having the old commit
+  deleted out from under them.  This is necessary on
+  filesystems like NFS that do not support "delete on last
+  close" semantics, which Lucene's "point in time" search
+  normally relies on. </p>
   */
 
 public class IndexWriter {
@@ -83,6 +135,9 @@
 
   private long writeLockTimeout = WRITE_LOCK_TIMEOUT;
 
+  /**
+   * Name of the write lock in the index.
+   */
   public static final String WRITE_LOCK_NAME = "write.lock";
 
   /**
@@ -120,11 +175,13 @@
 
   private Similarity similarity = Similarity.getDefault(); // how to normalize
 
-  private boolean inTransaction = false; // true iff we are in a transaction
   private boolean commitPending; // true if segmentInfos has changes not yet committed
-  private HashSet protectedSegments; // segment names that should not be deleted until commit
   private SegmentInfos rollbackSegmentInfos;      // segmentInfos we will fallback to if the commit fails
 
+  private SegmentInfos localRollbackSegmentInfos;      // segmentInfos we will fallback to if the commit fails
+  private boolean localAutoCommit;                // saved autoCommit during local transaction
+  private boolean autoCommit = true;              // false if we should commit only on close
+
   SegmentInfos segmentInfos = new SegmentInfos();       // the segments
   SegmentInfos ramSegmentInfos = new SegmentInfos();    // the segments in ramDirectory
   private final RAMDirectory ramDirectory = new RAMDirectory(); // for temp segs
@@ -238,7 +295,7 @@
    */
   public IndexWriter(String path, Analyzer a, boolean create)
        throws CorruptIndexException, LockObtainFailedException, IOException {
-    init(path, a, create);
+    init(FSDirectory.getDirectory(path), a, create, true, null, true);
   }
 
   /**
@@ -263,7 +320,7 @@
    */
   public IndexWriter(File path, Analyzer a, boolean create)
        throws CorruptIndexException, LockObtainFailedException, IOException {
-    init(path, a, create);
+    init(FSDirectory.getDirectory(path), a, create, true, null, true);
   }
 
   /**
@@ -288,14 +345,14 @@
    */
   public IndexWriter(Directory d, Analyzer a, boolean create)
        throws CorruptIndexException, LockObtainFailedException, IOException {
-    init(d, a, create, false);
+    init(d, a, create, false, null, true);
   }
 
   /**
    * Constructs an IndexWriter for the index in
-   * <code>path</code>, creating it first if it does not
-   * already exist, otherwise appending to the existing
-   * index.  Text will be analyzed with <code>a</code>.
+   * <code>path</code>, first creating it if it does not
+   * already exist.  Text will be analyzed with
+   * <code>a</code>.
    *
    * @param path the path to the index directory
    * @param a the analyzer to use
@@ -309,18 +366,13 @@
    */
   public IndexWriter(String path, Analyzer a) 
     throws CorruptIndexException, LockObtainFailedException, IOException {
-    if (IndexReader.indexExists(path)) {
-      init(path, a, false);
-    } else {
-      init(path, a, true);
-    }
+    init(FSDirectory.getDirectory(path), a, true, null, true);
   }
 
   /**
    * Constructs an IndexWriter for the index in
-   * <code>path</code>, creating it first if it does not
-   * already exist, otherwise appending to the existing
-   * index.  Text will be analyzed with
+   * <code>path</code>, first creating it if it does not
+   * already exist.  Text will be analyzed with
    * <code>a</code>.
    *
    * @param path the path to the index directory
@@ -335,18 +387,14 @@
    */
   public IndexWriter(File path, Analyzer a) 
     throws CorruptIndexException, LockObtainFailedException, IOException {
-    if (IndexReader.indexExists(path)) {
-      init(path, a, false);
-    } else {
-      init(path, a, true);
-    }
+    init(FSDirectory.getDirectory(path), a, true, null, true);
   }
 
   /**
    * Constructs an IndexWriter for the index in
-   * <code>d</code>, creating it first if it does not
-   * already exist, otherwise appending to the existing
-   * index.  Text will be analyzed with <code>a</code>.
+   * <code>d</code>, first creating it if it does not
+   * already exist.  Text will be analyzed with
+   * <code>a</code>.
    *
    * @param d the index directory
    * @param a the analyzer to use
@@ -360,28 +408,124 @@
    */
   public IndexWriter(Directory d, Analyzer a) 
     throws CorruptIndexException, LockObtainFailedException, IOException {
-    if (IndexReader.indexExists(d)) {
-      init(d, a, false, false);
-    } else {
-      init(d, a, true, false);
-    }
+    init(d, a, false, null, true);
+  }
+
+  /**
+   * Constructs an IndexWriter for the index in
+   * <code>d</code>, first creating it if it does not
+   * already exist.  Text will be analyzed with
+   * <code>a</code>.
+   *
+   * @param d the index directory
+   * @param autoCommit see <a href="#autoCommit">above</a>
+   * @param a the analyzer to use
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws LockObtainFailedException if another writer
+   *  has this index open (<code>write.lock</code> could not
+   *  be obtained)
+   * @throws IOException if the directory cannot be
+   *  read/written to or if there is any other low-level
+   *  IO error
+   */
+  public IndexWriter(Directory d, boolean autoCommit, Analyzer a) 
+    throws CorruptIndexException, LockObtainFailedException, IOException {
+    init(d, a, false, null, autoCommit);
+  }
+
+  /**
+   * Constructs an IndexWriter for the index in <code>d</code>.
+   * Text will be analyzed with <code>a</code>.  If <code>create</code>
+   * is true, then a new, empty index will be created in
+   * <code>d</code>, replacing the index already there, if any.
+   *
+   * @param d the index directory
+   * @param autoCommit see <a href="#autoCommit">above</a>
+   * @param a the analyzer to use
+   * @param create <code>true</code> to create the index or overwrite
+   *  the existing one; <code>false</code> to append to the existing
+   *  index
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws LockObtainFailedException if another writer
+   *  has this index open (<code>write.lock</code> could not
+   *  be obtained)
+   * @throws IOException if the directory cannot be read/written to, or
+   *  if it does not exist and <code>create</code> is
+   *  <code>false</code> or if there is any other low-level
+   *  IO error
+   */
+  public IndexWriter(Directory d, boolean autoCommit, Analyzer a, boolean create)
+       throws CorruptIndexException, LockObtainFailedException, IOException {
+    init(d, a, create, false, null, autoCommit);
   }
 
-  private void init(String path, Analyzer a, final boolean create)
+  /**
+   * Expert: constructs an IndexWriter with a custom {@link
+   * IndexDeletionPolicy}, for the index in <code>d</code>,
+   * first creating it if it does not already exist.  Text
+   * will be analyzed with <code>a</code>.
+   *
+   * @param d the index directory
+   * @param autoCommit see <a href="#autoCommit">above</a>
+   * @param a the analyzer to use
+   * @param deletionPolicy see <a href="#deletionPolicy">above</a>
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws LockObtainFailedException if another writer
+   *  has this index open (<code>write.lock</code> could not
+   *  be obtained)
+   * @throws IOException if the directory cannot be
+   *  read/written to or if there is any other low-level
+   *  IO error
+   */
+  public IndexWriter(Directory d, boolean autoCommit, Analyzer a, IndexDeletionPolicy deletionPolicy) 
     throws CorruptIndexException, LockObtainFailedException, IOException {
-    init(FSDirectory.getDirectory(path), a, create, true);
+    init(d, a, false, deletionPolicy, autoCommit);
+  }
+
+  /**
+   * Expert: constructs an IndexWriter with a custom {@link
+   * IndexDeletionPolicy}, for the index in <code>d</code>.
+   * Text will be analyzed with <code>a</code>.  If
+   * <code>create</code> is true, then a new, empty index
+   * will be created in <code>d</code>, replacing the index
+   * already there, if any.
+   *
+   * @param d the index directory
+   * @param autoCommit see <a href="#autoCommit">above</a>
+   * @param a the analyzer to use
+   * @param create <code>true</code> to create the index or overwrite
+   *  the existing one; <code>false</code> to append to the existing
+   *  index
+   * @param deletionPolicy see <a href="#deletionPolicy">above</a>
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws LockObtainFailedException if another writer
+   *  has this index open (<code>write.lock</code> could not
+   *  be obtained)
+   * @throws IOException if the directory cannot be read/written to, or
+   *  if it does not exist and <code>create</code> is
+   *  <code>false</code> or if there is any other low-level
+   *  IO error
+   */
+  public IndexWriter(Directory d, boolean autoCommit, Analyzer a, boolean create, IndexDeletionPolicy deletionPolicy)
+       throws CorruptIndexException, LockObtainFailedException, IOException {
+    init(d, a, create, false, deletionPolicy, autoCommit);
   }
 
-  private void init(File path, Analyzer a, final boolean create)
+  private void init(Directory d, Analyzer a, boolean closeDir, IndexDeletionPolicy deletionPolicy, boolean autoCommit)
     throws CorruptIndexException, LockObtainFailedException, IOException {
-    init(FSDirectory.getDirectory(path), a, create, true);
+    if (IndexReader.indexExists(d)) {
+      init(d, a, false, closeDir, deletionPolicy, autoCommit);
+    } else {
+      init(d, a, true, closeDir, deletionPolicy, autoCommit);
+    }
   }
 
-  private void init(Directory d, Analyzer a, final boolean create, boolean closeDir)
+  private void init(Directory d, Analyzer a, final boolean create, boolean closeDir, IndexDeletionPolicy deletionPolicy, boolean autoCommit)
     throws CorruptIndexException, LockObtainFailedException, IOException {
     this.closeDir = closeDir;
     directory = d;
     analyzer = a;
+    this.infoStream = defaultInfoStream;
 
     if (create) {
       // Clear the write lock in case it's leftover:
@@ -410,12 +554,16 @@
         segmentInfos.read(directory);
       }
 
-      // Create a deleter to keep track of which files can
-      // be deleted:
-      deleter = new IndexFileDeleter(segmentInfos, directory);
-      deleter.setInfoStream(infoStream);
-      deleter.findDeletableFiles();
-      deleter.deleteFiles();
+      this.autoCommit = autoCommit;
+      if (!autoCommit) {
+        rollbackSegmentInfos = (SegmentInfos) segmentInfos.clone();
+      }
+
+      // Default deleter (for backwards compatibility) is
+      // KeepOnlyLastCommitDeleter:
+      deleter = new IndexFileDeleter(directory,
+                                     deletionPolicy == null ? new KeepOnlyLastCommitDeletionPolicy() : deletionPolicy,
+                                     segmentInfos, infoStream);
 
     } catch (IOException e) {
       this.writeLock.release();
@@ -533,11 +681,28 @@
     return mergeFactor;
   }
 
-  /** If non-null, information about merges and a message when
-   * maxFieldLength is reached will be printed to this.
+  /** If non-null, this will be the default infoStream used
+   * by a newly instantiated IndexWriter.
+   * @see #setInfoStream
+   */
+  public static void setDefaultInfoStream(PrintStream infoStream) {
+    IndexWriter.defaultInfoStream = infoStream;
+  }
+
+  /**
+   * @see #setDefaultInfoStream
+   */
+  public static PrintStream getDefaultInfoStream() {
+    return IndexWriter.defaultInfoStream;
+  }
+
+  /** If non-null, information about merges, deletes and a
+   * message when maxFieldLength is reached will be printed
+   * to this.
    */
   public void setInfoStream(PrintStream infoStream) {
     this.infoStream = infoStream;
+    deleter.setInfoStream(infoStream);
   }
 
   /**
@@ -613,6 +778,14 @@
    */
   public synchronized void close() throws CorruptIndexException, IOException {
     flushRamSegments();
+
+    if (commitPending) {
+      segmentInfos.write(directory);         // now commit changes
+      deleter.checkpoint(segmentInfos, true);
+      commitPending = false;
+      rollbackSegmentInfos = null;
+    }
+
     ramDirectory.close();
     if (writeLock != null) {
       writeLock.release();                          // release write lock
@@ -737,7 +910,9 @@
     dw.setInfoStream(infoStream);
     String segmentName = newRamSegmentName();
     dw.addDocument(segmentName, doc);
-    return new SegmentInfo(segmentName, 1, ramDirectory, false, false);
+    SegmentInfo si = new SegmentInfo(segmentName, 1, ramDirectory, false, false);
+    si.setNumFields(dw.getNumFields());
+    return si;
   }
 
   /**
@@ -871,6 +1046,7 @@
 
    */
   private PrintStream infoStream = null;
+  private static PrintStream defaultInfoStream = null;
 
   /** Merges all segments together into a single segment,
    * optimizing an index for search.
@@ -949,21 +1125,18 @@
    * merges that happen (or ram segments flushed) will not
    * write a new segments file and will not remove any files
    * that were present at the start of the transaction.  You
-   * must make a matched (try/finall) call to
+   * must make a matched (try/finally) call to
    * commitTransaction() or rollbackTransaction() to finish
    * the transaction.
    */
   private void startTransaction() throws IOException {
-    if (inTransaction) {
-      throw new IOException("transaction is already in process");
-    }
-    rollbackSegmentInfos = (SegmentInfos) segmentInfos.clone();
-    protectedSegments = new HashSet();
-    for(int i=0;i<segmentInfos.size();i++) {
-      SegmentInfo si = (SegmentInfo) segmentInfos.elementAt(i);
-      protectedSegments.add(si.name);
+    localRollbackSegmentInfos = (SegmentInfos) segmentInfos.clone();
+    localAutoCommit = autoCommit;
+    if (localAutoCommit) {
+      flushRamSegments();
+      // Turn off auto-commit during our local transaction:
+      autoCommit = false;
     }
-    inTransaction = true;
   }
 
   /*
@@ -972,20 +1145,21 @@
    */
   private void rollbackTransaction() throws IOException {
 
+    // First restore autoCommit in case we hit an exception below:
+    autoCommit = localAutoCommit;
+
     // Keep the same segmentInfos instance but replace all
     // of its SegmentInfo instances.  This is so the next
     // attempt to commit using this instance of IndexWriter
     // will always write to a new generation ("write once").
     segmentInfos.clear();
-    segmentInfos.addAll(rollbackSegmentInfos);
-
-    // Ask deleter to locate unreferenced files & remove
-    // them:
-    deleter.clearPendingFiles();
-    deleter.findDeletableFiles();
-    deleter.deleteFiles();
+    segmentInfos.addAll(localRollbackSegmentInfos);
+    localRollbackSegmentInfos = null;
 
-    clearTransaction();
+    // Ask deleter to locate unreferenced files we had
+    // created & remove them:
+    deleter.checkpoint(segmentInfos, false);
+    deleter.refresh();
   }
 
   /*
@@ -994,34 +1168,78 @@
    * accumulated during the transaction
    */
   private void commitTransaction() throws IOException {
-    if (commitPending) {
-      boolean success = false;
-      try {
-        // If we hit eg disk full during this write we have
-        // to rollback.:
-        segmentInfos.write(directory);         // commit changes
-        success = true;
-      } finally {
-        if (!success) {
-          rollbackTransaction();
-        }
+
+    // First restore autoCommit in case we hit an exception below:
+    autoCommit = localAutoCommit;
+
+    boolean success = false;
+    try {
+      checkpoint();
+      success = true;
+    } finally {
+      if (!success) {
+        rollbackTransaction();
       }
-      deleter.commitPendingFiles();
-      commitPending = false;
     }
+    localRollbackSegmentInfos = null;
 
-    clearTransaction();
+    // Give deleter a chance to remove files now:
+    deleter.checkpoint(segmentInfos, autoCommit);
   }
 
-  /* Should only be called by rollbackTransaction &
-   * commitTransaction */
-  private void clearTransaction() {
-    protectedSegments = null;
-    rollbackSegmentInfos = null;
-    inTransaction = false;
-  }
+  /**
+   * Close the <code>IndexWriter</code> without committing
+   * any of the changes that have occurred since it was
+   * opened. This removes any temporary files that had been
+   * created, after which the state of the index will be the
+   * same as it was when this writer was first opened.  This
+   * can only be called when this IndexWriter was opened
+   * with <code>autoCommit=false</code>.
+   * @throws IllegalStateException if this is called when
+   *  the writer was opened with <code>autoCommit=true</code>.
+   * @throws IOException if there is a low-level IO error
+   */
+  public void abort() throws IOException {
+    if (!autoCommit) {
+
+      // Keep the same segmentInfos instance but replace all
+      // of its SegmentInfo instances.  This is so the next
+      // attempt to commit using this instance of IndexWriter
+      // will always write to a new generation ("write once").
+      segmentInfos.clear();
+      segmentInfos.addAll(rollbackSegmentInfos);
+
+      // Ask deleter to locate unreferenced files & remove
+      // them:
+      deleter.checkpoint(segmentInfos, false);
+      deleter.refresh();
+
+      ramSegmentInfos = new SegmentInfos();
+      bufferedDeleteTerms.clear();
+      numBufferedDeleteTerms = 0;
 
+      commitPending = false;
+      close();
 
+    } else {
+      throw new IllegalStateException("abort() can only be called when IndexWriter was opened with autoCommit=false");
+    }
+  }
+ 
+  /*
+   * Called whenever the SegmentInfos has been updated and
+   * the index files referenced exist (correctly) in the
+   * index directory.  If we are in autoCommit mode, we
+   * commit the change immediately.  Else, we mark
+   * commitPending.
+   */
+  private void checkpoint() throws IOException {
+    if (autoCommit) {
+      segmentInfos.write(directory);
+    } else {
+      commitPending = true;
+    }
+  }
 
   /** Merges all segments from an array of indexes into this index.
    *
@@ -1266,16 +1484,13 @@
     final String mergedName = newSegmentName();
     SegmentMerger merger = new SegmentMerger(this, mergedName);
 
-    final Vector segmentsToDelete = new Vector();
     SegmentInfo info;
-    String segmentsInfosFileName = segmentInfos.getCurrentSegmentFileName();
 
     IndexReader sReader = null;
     try {
       if (segmentInfos.size() == 1){ // add existing index, if any
         sReader = SegmentReader.get(segmentInfos.info(0));
         merger.add(sReader);
-        segmentsToDelete.addElement(sReader);   // queue segment for deletion
       }
 
       for (int i = 0; i < readers.length; i++)      // add new indexes
@@ -1288,16 +1503,15 @@
       try {
         int docCount = merger.merge();                // merge 'em
 
-        segmentInfos.setSize(0);                      // pop old infos & add new
-        info = new SegmentInfo(mergedName, docCount, directory, false, true);
-        segmentInfos.addElement(info);
-        commitPending = true;
-
         if(sReader != null) {
           sReader.close();
           sReader = null;
         }
 
+        segmentInfos.setSize(0);                      // pop old infos & add new
+        info = new SegmentInfo(mergedName, docCount, directory, false, true);
+        segmentInfos.addElement(info);
+
         success = true;
 
       } finally {
@@ -1312,26 +1526,16 @@
         sReader.close();
       }
     }
-
-    deleter.deleteFile(segmentsInfosFileName);    // delete old segments_N file
-    deleter.deleteSegments(segmentsToDelete);     // delete now-unused segments
-
+    
     if (useCompoundFile) {
-      boolean success = false;
 
-      segmentsInfosFileName = segmentInfos.getCurrentSegmentFileName();
-      Vector filesToDelete;
+      boolean success = false;
 
       startTransaction();
 
       try {
-
-        filesToDelete = merger.createCompoundFile(mergedName + ".cfs");
-
+        merger.createCompoundFile(mergedName + ".cfs");
         info.setUseCompoundFile(true);
-        commitPending = true;
-        success = true;
-
       } finally {
         if (!success) {
           rollbackTransaction();
@@ -1339,9 +1543,6 @@
           commitTransaction();
         }
       }
-
-      deleter.deleteFile(segmentsInfosFileName);  // delete old segments_N file
-      deleter.deleteFiles(filesToDelete); // delete now unused files of segment 
     }
   }
 
@@ -1500,14 +1701,12 @@
     final String mergedName = newSegmentName();
     SegmentMerger merger = null;
 
-    final Vector segmentsToDelete = new Vector();
-
-    String segmentsInfosFileName = segmentInfos.getCurrentSegmentFileName();
-    String nextSegmentsFileName = segmentInfos.getNextSegmentFileName();
+    final List ramSegmentsToDelete = new ArrayList();
 
     SegmentInfo newSegment = null;
 
     int mergedDocCount = 0;
+    boolean anyDeletes = (bufferedDeleteTerms.size() != 0);
 
     // This is try/finally to make sure merger's readers are closed:
     try {
@@ -1522,9 +1721,9 @@
             infoStream.print(" " + si.name + " (" + si.docCount + " docs)");
           IndexReader reader = SegmentReader.get(si); // no need to set deleter (yet)
           merger.add(reader);
-          if ((reader.directory() == this.directory) || // if we own the directory
-              (reader.directory() == this.ramDirectory))
-            segmentsToDelete.addElement(reader);   // queue segment for deletion
+          if (reader.directory() == this.ramDirectory) {
+            ramSegmentsToDelete.add(si);
+          }
         }
       }
 
@@ -1545,9 +1744,8 @@
           newSegment = new SegmentInfo(mergedName, mergedDocCount,
                                        directory, false, true);
         }
-
-        if (!inTransaction
-            && (sourceSegments != ramSegmentInfos || bufferedDeleteTerms.size() > 0)) {
+        
+        if (sourceSegments != ramSegmentInfos || anyDeletes) {
           // Now save the SegmentInfo instances that
           // we are replacing:
           rollback = (SegmentInfos) segmentInfos.clone();
@@ -1565,18 +1763,11 @@
         }
 
         if (sourceSegments == ramSegmentInfos) {
-          // Should not be necessary: no prior commit should
-          // have left pending files, so just defensive:
-          deleter.clearPendingFiles();
           maybeApplyDeletes(doMerge);
           doAfterFlush();
         }
-
-        if (!inTransaction) {
-          segmentInfos.write(directory);     // commit before deleting
-        } else {
-          commitPending = true;
-        }
+        
+        checkpoint();
 
         success = true;
 
@@ -1589,11 +1780,10 @@
           if (sourceSegments == ramSegmentInfos) {
             ramSegmentInfos.removeAllElements();
           }
-        } else if (!inTransaction) {  
+        } else {
 
           // Must rollback so our state matches index:
-
-          if (sourceSegments == ramSegmentInfos && 0 == bufferedDeleteTerms.size()) {
+          if (sourceSegments == ramSegmentInfos && !anyDeletes) {
             // Simple case: newSegment may or may not have
             // been added to the end of our segment infos,
             // so just check & remove if so:
@@ -1611,14 +1801,8 @@
             segmentInfos.addAll(rollback);
           }
 
-          // Erase any pending files that we were going to delete:
-          // i.e. old del files added by SegmentReader.doCommit() 
-          deleter.clearPendingFiles();
-
-          // Delete any partially created files:
-          deleter.deleteFile(nextSegmentsFileName);
-          deleter.findDeletableFiles();
-          deleter.deleteFiles();
+          // Delete any partially created and now unreferenced files:
+          deleter.refresh();
         }
       }
     } finally {
@@ -1626,53 +1810,33 @@
       if (doMerge) merger.closeReaders();
     }
 
-    if (!inTransaction) {
-      // Attempt to delete all files we just obsoleted:
-      deleter.deleteFile(segmentsInfosFileName);    // delete old segments_N file
-      deleter.deleteSegments(segmentsToDelete);     // delete now-unused segments
-      // Includes the old del files
-      deleter.commitPendingFiles();
-    } else {
-      deleter.addPendingFile(segmentsInfosFileName);    // delete old segments_N file
-      deleter.deleteSegments(segmentsToDelete, protectedSegments);     // delete now-unused segments
-    }
-
-    if (useCompoundFile && doMerge) {
+    // Delete the RAM segments
+    deleter.deleteDirect(ramDirectory, ramSegmentsToDelete);
 
-      segmentsInfosFileName = nextSegmentsFileName;
-      nextSegmentsFileName = segmentInfos.getNextSegmentFileName();
+    // Give deleter a chance to remove files now.
+    deleter.checkpoint(segmentInfos, autoCommit);
 
-      Vector filesToDelete;
+    if (useCompoundFile && doMerge) {
 
       boolean success = false;
 
       try {
 
-        filesToDelete = merger.createCompoundFile(mergedName + ".cfs");
+        merger.createCompoundFile(mergedName + ".cfs");
         newSegment.setUseCompoundFile(true);
-        if (!inTransaction) {
-          segmentInfos.write(directory);     // commit again so readers know we've switched this segment to a compound file
-        }
+        checkpoint();
         success = true;
 
       } finally {
-        if (!success && !inTransaction) {  
+        if (!success) {  
           // Must rollback:
           newSegment.setUseCompoundFile(false);
-          deleter.deleteFile(mergedName + ".cfs");
-          deleter.deleteFile(nextSegmentsFileName);
+          deleter.refresh();
         }
       }
-
-      if (!inTransaction) {
-        deleter.deleteFile(segmentsInfosFileName);  // delete old segments_N file
-      }
-
-      // We can delete these segments whether or not we are
-      // in a transaction because we had just written them
-      // above so they can't need protection by the
-      // transaction:
-      deleter.deleteFiles(filesToDelete);  // delete now-unused segments
+      
+      // Give deleter a chance to remove files now.
+      deleter.checkpoint(segmentInfos, autoCommit);
     }
 
     return mergedDocCount;
@@ -1692,7 +1856,6 @@
         IndexReader reader = null;
         try {
           reader = SegmentReader.get(segmentInfos.info(segmentInfos.size() - 1));
-          reader.setDeleter(deleter);
 
           // Apply delete terms to the segment just flushed from ram
           // apply appropriately so that a delete term is only applied to
@@ -1718,7 +1881,6 @@
         IndexReader reader = null;
         try {
           reader = SegmentReader.get(segmentInfos.info(i));
-          reader.setDeleter(deleter);
 
           // Apply delete terms to disk segments
           // except the one just flushed from ram.
@@ -1769,7 +1931,7 @@
   }
 
   // Number of ram segments a delete term applies to.
-  private class Num {
+  private static class Num {
     private int num;
 
     Num(int num) {

Added: lucene/java/trunk/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java?view=auto&rev=517599
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java Tue Mar 13 02:06:22 2007
@@ -0,0 +1,50 @@
+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.util.List;
+
+/**
+ * This {@link IndexDeletionPolicy} implementation that
+ * keeps only the most recent commit and immediately removes
+ * all prior commits after a new commit is done.  This is
+ * the default deletion policy.
+ */
+
+public final class KeepOnlyLastCommitDeletionPolicy implements IndexDeletionPolicy {
+
+  /**
+   * Deletes all commits except the most recent one.
+   */
+  public void onInit(List commits) {
+    // Note that commits.size() should normally be 1:
+    onCommit(commits);
+  }
+
+  /**
+   * Deletes all commits except the most recent one.
+   */
+  public void onCommit(List commits) {
+    // Note that commits.size() should normally be 2 (if not
+    // called by onInit above):
+    int size = commits.size();
+    for(int i=0;i<size-1;i++) {
+      ((IndexCommitPoint) commits.get(i)).delete();
+    }
+  }
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/MultiReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/MultiReader.java?view=diff&rev=517599&r1=517598&r2=517599
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/MultiReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/MultiReader.java Tue Mar 13 02:06:22 2007
@@ -220,13 +220,6 @@
     return new MultiTermPositions(subReaders, starts);
   }
 
-  protected void setDeleter(IndexFileDeleter deleter) {
-    // Share deleter to our SegmentReaders:
-    this.deleter = deleter;
-    for (int i = 0; i < subReaders.length; i++)
-      subReaders[i].setDeleter(deleter);
-  }
-
   protected void doCommit() throws IOException {
     for (int i = 0; i < subReaders.length; i++)
       subReaders[i].commit();

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfo.java?view=diff&rev=517599&r1=517598&r2=517599
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfo.java Tue Mar 13 02:06:22 2007
@@ -21,6 +21,8 @@
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.IndexInput;
 import java.io.IOException;
+import java.util.List;
+import java.util.ArrayList;
 
 final class SegmentInfo {
   public String name;				  // unique name in dir
@@ -50,6 +52,9 @@
                                                   // and true for newly created merged segments (both
                                                   // compound and non compound).
   
+  private List files;                             // cached list of files that this segment uses
+                                                  // in the Directory
+
   public SegmentInfo(String name, int docCount, Directory dir) {
     this.name = name;
     this.docCount = docCount;
@@ -71,6 +76,7 @@
    * Copy everything from src SegmentInfo into our instance.
    */
   void reset(SegmentInfo src) {
+    files = null;
     name = src.name;
     docCount = src.docCount;
     dir = src.dir;
@@ -134,7 +140,7 @@
 
       if (!preLockless) {
         // This is a FORMAT_LOCKLESS segment, which means
-        // there are no norms:
+        // there are no separate norms:
         for(int i=0;i<numFields;i++) {
           normGen[i] = -1;
         }
@@ -174,10 +180,12 @@
     } else {
       delGen++;
     }
+    files = null;
   }
 
   void clearDelGen() {
     delGen = -1;
+    files = null;
   }
 
   public Object clone () {
@@ -199,7 +207,7 @@
       return null;
     } else {
       // If delGen is 0, it's the pre-lockless-commit file format
-      return IndexFileNames.fileNameFromGeneration(name, ".del", delGen);
+      return IndexFileNames.fileNameFromGeneration(name, "." + IndexFileNames.DELETES_EXTENSION, delGen);
     }
   }
 
@@ -283,6 +291,7 @@
     } else {
       normGen[fieldIndex]++;
     }
+    files = null;
   }
 
   /**
@@ -329,6 +338,7 @@
     } else {
       this.isCompoundFile = -1;
     }
+    files = null;
   }
 
   /**
@@ -341,7 +351,7 @@
     } else if (isCompoundFile == 1) {
       return true;
     } else {
-      return dir.fileExists(name + ".cfs");
+      return dir.fileExists(name + "." + IndexFileNames.COMPOUND_FILE_EXTENSION);
     }
   }
   
@@ -363,5 +373,90 @@
       }
     }
     output.writeByte(isCompoundFile);
+  }
+
+  /*
+   * Return all files referenced by this SegmentInfo.  The
+   * returns List is a locally cached List so you should not
+   * modify it.
+   */
+
+  public List files() throws IOException {
+
+    if (files != null) {
+      // Already cached:
+      return files;
+    }
+    
+    files = new ArrayList();
+    
+    boolean useCompoundFile = getUseCompoundFile();
+
+    if (useCompoundFile) {
+      files.add(name + "." + IndexFileNames.COMPOUND_FILE_EXTENSION);
+    } else {
+      for (int i = 0; i < IndexFileNames.INDEX_EXTENSIONS_IN_COMPOUND_FILE.length; i++) {
+        String ext = IndexFileNames.INDEX_EXTENSIONS_IN_COMPOUND_FILE[i];
+        String fileName = name + "." + ext;
+        if (dir.fileExists(fileName)) {
+          files.add(fileName);
+        }
+      }
+    }
+
+    String delFileName = IndexFileNames.fileNameFromGeneration(name, "." + IndexFileNames.DELETES_EXTENSION, delGen);
+    if (delFileName != null && (delGen > 0 || dir.fileExists(delFileName))) {
+      files.add(delFileName);
+    }
+
+    // Careful logic for norms files:
+    if (normGen != null) {
+      for(int i=0;i<normGen.length;i++) {
+        long gen = normGen[i];
+        if (gen > 0) {
+          // Definitely a separate norm file, with generation:
+          files.add(IndexFileNames.fileNameFromGeneration(name, "." + IndexFileNames.SEPARATE_NORMS_EXTENSION + i, gen));
+        } else if (-1 == gen) {
+          // No separate norms but maybe non-separate norms
+          // in the non compound file case:
+          if (!hasSingleNormFile && !useCompoundFile) {
+            String fileName = name + "." + IndexFileNames.SINGLE_NORMS_EXTENSION + i;
+            if (dir.fileExists(fileName)) {
+              files.add(fileName);
+            }
+          }
+        } else if (0 == gen) {
+          // Pre-2.1: we have to check file existence
+          String fileName = null;
+          if (useCompoundFile) {
+            fileName = name + "." + IndexFileNames.SEPARATE_NORMS_EXTENSION + i;
+          } else if (!hasSingleNormFile) {
+            fileName = name + "." + IndexFileNames.SINGLE_NORMS_EXTENSION + i;
+          }
+          if (fileName != null && dir.fileExists(fileName)) {
+            files.add(fileName);
+          }
+        }
+      }
+    } else if (preLockless || (!hasSingleNormFile && !useCompoundFile)) {
+      // Pre-2.1: we have to scan the dir to find all
+      // matching _X.sN/_X.fN files for our segment:
+      String prefix;
+      if (useCompoundFile)
+        prefix = name + "." + IndexFileNames.SEPARATE_NORMS_EXTENSION;
+      else
+        prefix = name + "." + IndexFileNames.SINGLE_NORMS_EXTENSION;
+      int prefixLength = prefix.length();
+      String[] allFiles = dir.list();
+      if (allFiles == null)
+        throw new IOException("cannot read directory " + dir + ": list() returned null");
+      for(int i=0;i<allFiles.length;i++) {
+        String fileName = allFiles[i];
+        if (fileName.length() > prefixLength && Character.isDigit(fileName.charAt(prefixLength)) && fileName.startsWith(prefix)) {
+          files.add(fileName);
+        }
+      }
+    }
+    return files;
   }
 }