You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2013/05/02 19:16:08 UTC

svn commit: r1478452 - in /lucene/dev/trunk/lucene: ./ core/src/java/org/apache/lucene/index/ core/src/test/org/apache/lucene/index/

Author: mikemccand
Date: Thu May  2 17:16:07 2013
New Revision: 1478452

URL: http://svn.apache.org/r1478452
Log:
LUCENE-4973: Persistent/SnapshotDeletionPolicy no longer require a unique id

Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SnapshotDeletionPolicy.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestPersistentSnapshotDeletionPolicy.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1478452&r1=1478451&r2=1478452&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Thu May  2 17:16:07 2013
@@ -52,6 +52,9 @@ Changes in backwards compatibility polic
   see which indexing changes, so that it can work with any
   ReferenceManager (Mike McCandless)
 
+* LUCENE-4973: SnapshotDeletionPolicy no longer requires a unique
+  String id (Mike McCandless, Shai Erera)
+
 Bug Fixes
 
 * LUCENE-4935: CustomScoreQuery wrongly applied its query boost twice 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java?rev=1478452&r1=1478451&r2=1478452&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java Thu May  2 17:16:07 2013
@@ -19,10 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
 import java.util.Map.Entry;
-import java.util.Map;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.StoredField;
@@ -34,7 +31,7 @@ import org.apache.lucene.util.Version;
  * A {@link SnapshotDeletionPolicy} which adds a persistence layer so that
  * snapshots can be maintained across the life of an application. The snapshots
  * are persisted in a {@link Directory} and are committed as soon as
- * {@link #snapshot(String)} or {@link #release(String)} is called.
+ * {@link #snapshot()} or {@link #release(IndexCommit)} is called.
  * <p>
  * <b>NOTE:</b> this class receives a {@link Directory} to persist the data into
  * a Lucene index. It is highly recommended to use a dedicated directory (and on
@@ -50,12 +47,17 @@ import org.apache.lucene.util.Version;
  * should make sure every {@link IndexWriter} has its own
  * {@link PersistentSnapshotDeletionPolicy} and that they all write to a
  * different {@link Directory}.
+ *
+ * <p> This class adds a {@link #release(long)} method to
+ * release commits from a previous snapshot's {@link IndexCommit#getGeneration}.
+ *
+ * @lucene.experimental
  */
 public class PersistentSnapshotDeletionPolicy extends SnapshotDeletionPolicy implements Closeable {
 
   // Used to validate that the given directory includes just one document w/ the
-  // given ID field. Otherwise, it's not a valid Directory for snapshotting.
-  private static final String SNAPSHOTS_ID = "$SNAPSHOTS_DOC$";
+  // given gen field. Otherwise, it's not a valid Directory for snapshotting.
+  private static final String SNAPSHOTS_GENS = "$SNAPSHOTS_DOC$";
 
   // The index writer which maintains the snapshots metadata
   private final IndexWriter writer;
@@ -66,20 +68,20 @@ public class PersistentSnapshotDeletionP
    * cannot instantiate the deletion policy (because e.g., some other process
    * keeps a lock on the snapshots directory).
    */
-  public static Map<String, String> readSnapshotsInfo(Directory dir) throws IOException {
+  private void loadPriorSnapshots(Directory dir) throws IOException {
     IndexReader r = DirectoryReader.open(dir);
-    Map<String, String> snapshots = new HashMap<String, String>();
     try {
       int numDocs = r.numDocs();
       // index is allowed to have exactly one document or 0.
       if (numDocs == 1) {
         StoredDocument doc = r.document(r.maxDoc() - 1);
-        if (doc.getField(SNAPSHOTS_ID) == null) {
+        if (doc.getField(SNAPSHOTS_GENS) == null) {
           throw new IllegalStateException("directory is not a valid snapshots store!");
         }
         for (StorableField f : doc) {
-          if (!f.name().equals(SNAPSHOTS_ID))
-            snapshots.put(f.name(), f.stringValue());
+          if (!f.name().equals(SNAPSHOTS_GENS)) {
+            refCounts.put(Long.parseLong(f.name()), Integer.parseInt(f.stringValue()));
+          }
         }
       } else if (numDocs != 0) {
         throw new IllegalStateException(
@@ -88,7 +90,6 @@ public class PersistentSnapshotDeletionP
     } finally {
       r.close();
     }
-    return snapshots;
   }
   
   /**
@@ -98,7 +99,7 @@ public class PersistentSnapshotDeletionP
    * @param primary
    *          the {@link IndexDeletionPolicy} that is used on non-snapshotted
    *          commits. Snapshotted commits, by definition, are not deleted until
-   *          explicitly released via {@link #release(String)}.
+   *          explicitly released via {@link #release}.
    * @param dir
    *          the {@link Directory} which will be used to persist the snapshots
    *          information.
@@ -112,7 +113,7 @@ public class PersistentSnapshotDeletionP
    */
   public PersistentSnapshotDeletionPolicy(IndexDeletionPolicy primary,
       Directory dir, OpenMode mode, Version matchVersion) throws IOException {
-    super(primary, null);
+    super(primary);
 
     // Initialize the index writer over the snapshot directory.
     writer = new IndexWriter(dir, new IndexWriterConfig(matchVersion, null).setOpenMode(mode));
@@ -127,9 +128,7 @@ public class PersistentSnapshotDeletionP
       // Initializes the snapshots information. This code should basically run
       // only if mode != CREATE, but if it is, it's no harm as we only open the
       // reader once and immediately close it.
-      for (Entry<String, String> e : readSnapshotsInfo(dir).entrySet()) {
-        registerSnapshotInfo(e.getKey(), e.getValue(), null);
-      }
+      loadPriorSnapshots(dir);
     } catch (RuntimeException e) {
       writer.close(); // don't leave any open file handles
       throw e;
@@ -139,43 +138,41 @@ public class PersistentSnapshotDeletionP
     }
   }
 
+  /**
+   * Snapshots the last commit. Once this method returns, the
+   * snapshot information is persisted in the directory.
+   * 
+   * @see SnapshotDeletionPolicy#snapshot
+   */
   @Override
-  public synchronized void onInit(List<? extends IndexCommit> commits)
-  throws IOException {
-    // super.onInit() needs to be called first to ensure that initialization
-    // behaves as expected. The superclass, SnapshotDeletionPolicy, ensures
-    // that any snapshot IDs with empty IndexCommits are released. Since this 
-    // happens, this class needs to persist these changes.
-    super.onInit(commits);
-    persistSnapshotInfos(null, null);
+  public synchronized IndexCommit snapshot() throws IOException {
+    IndexCommit ic = super.snapshot();
+    persist();
+    return ic;
   }
 
   /**
-   * Snapshots the last commit using the given ID. Once this method returns, the
-   * snapshot information is persisted in the directory.
+   * Deletes a snapshotted commit. Once this method returns, the snapshot
+   * information is persisted in the directory.
    * 
-   * @see SnapshotDeletionPolicy#snapshot(String)
+   * @see SnapshotDeletionPolicy#release
    */
   @Override
-  public synchronized IndexCommit snapshot(String id) throws IOException {
-    checkSnapshotted(id);
-    if (SNAPSHOTS_ID.equals(id)) {
-      throw new IllegalArgumentException(id + " is reserved and cannot be used as a snapshot id");
-    }
-    persistSnapshotInfos(id, lastCommit.getSegmentsFileName());
-    return super.snapshot(id);
+  public synchronized void release(IndexCommit commit) throws IOException {
+    super.release(commit);
+    persist();
   }
 
   /**
-   * Deletes a snapshotted commit by ID. Once this method returns, the snapshot
-   * information is committed to the directory.
+   * Deletes a snapshotted commit by generation. Once this method returns, the snapshot
+   * information is persisted in the directory.
    * 
-   * @see SnapshotDeletionPolicy#release(String)
+   * @see IndexCommit#getGeneration
+   * @see SnapshotDeletionPolicy#release
    */
-  @Override
-  public synchronized void release(String id) throws IOException {
-    super.release(id);
-    persistSnapshotInfos(null, null);
+  public synchronized void release(long gen) throws IOException {
+    super.releaseGen(gen);
+    persist();
   }
 
   /** Closes the index which writes the snapshots to the directory. */
@@ -184,18 +181,14 @@ public class PersistentSnapshotDeletionP
   }
 
   /**
-   * Persists all snapshots information. If the given id and segment are not
-   * null, it persists their information as well.
+   * Persists all snapshots information.
    */
-  private void persistSnapshotInfos(String id, String segment) throws IOException {
+  private void persist() throws IOException {
     writer.deleteAll();
     Document d = new Document();
-    d.add(new StoredField(SNAPSHOTS_ID, ""));
-    for (Entry<String, String> e : super.getSnapshots().entrySet()) {
-      d.add(new StoredField(e.getKey(), e.getValue()));
-    }
-    if (id != null) {
-      d.add(new StoredField(id, segment));
+    d.add(new StoredField(SNAPSHOTS_GENS, ""));
+    for (Entry<Long,Integer> e : refCounts.entrySet()) {
+      d.add(new StoredField(e.getKey().toString(), e.getValue().toString()));
     }
     writer.addDocument(d);
     writer.commit();

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SnapshotDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SnapshotDeletionPolicy.java?rev=1478452&r1=1478451&r2=1478452&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SnapshotDeletionPolicy.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SnapshotDeletionPolicy.java Thu May  2 17:16:07 2013
@@ -19,18 +19,15 @@ package org.apache.lucene.index;
 
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.ArrayList;
 import java.util.Map;
-import java.util.Set;
-import java.util.Map.Entry;
 import java.io.IOException;
 
 import org.apache.lucene.store.Directory;
 
 /**
- * An {@link IndexDeletionPolicy} that wraps around any other
+ * An {@link IndexDeletionPolicy} that wraps any other
  * {@link IndexDeletionPolicy} and adds the ability to hold and later release
  * snapshots of an index. While a snapshot is held, the {@link IndexWriter} will
  * not remove any files associated with it even if the index is otherwise being
@@ -41,37 +38,158 @@ import org.apache.lucene.store.Directory
  * 
  * <p>
  * This class maintains all snapshots in-memory, and so the information is not
- * persisted and not protected against system failures. If persistency is
- * important, you can use {@link PersistentSnapshotDeletionPolicy} (or your own
- * extension) and when creating a new instance of this deletion policy, pass the
- * persistent snapshots information to
- * {@link #SnapshotDeletionPolicy(IndexDeletionPolicy, Map)}.
+ * persisted and not protected against system failures. If persistence is
+ * important, you can use {@link PersistentSnapshotDeletionPolicy}.
  * 
  * @lucene.experimental
  */
 public class SnapshotDeletionPolicy extends IndexDeletionPolicy {
 
-  /** Holds a Snapshot's information. */
-  private static class SnapshotInfo {
-    String id;
-    String segmentsFileName;
-    IndexCommit commit;
-    
-    public SnapshotInfo(String id, String segmentsFileName, IndexCommit commit) {
-      this.id = id;
-      this.segmentsFileName = segmentsFileName;
-      this.commit = commit;
+  /** Records how many snapshots are held against each
+   *  commit generation */
+  protected Map<Long,Integer> refCounts = new HashMap<Long,Integer>();
+
+  /** Used to map gen to IndexCommit. */
+  protected Map<Long,IndexCommit> indexCommits = new HashMap<Long,IndexCommit>();
+
+  /** Wrapped {@link IndexDeletionPolicy} */
+  private IndexDeletionPolicy primary;
+
+  /** Most recently committed {@link IndexCommit}. */
+  protected IndexCommit lastCommit;
+
+  /** Sole constructor, taking the incoming {@link
+   *  IndexDeletionPolicy} to wrap. */
+  public SnapshotDeletionPolicy(IndexDeletionPolicy primary) {
+    this.primary = primary;
+  }
+
+  @Override
+  public synchronized void onCommit(List<? extends IndexCommit> commits)
+      throws IOException {
+    primary.onCommit(wrapCommits(commits));
+    lastCommit = commits.get(commits.size() - 1);
+  }
+
+  @Override
+  public synchronized void onInit(List<? extends IndexCommit> commits)
+      throws IOException {
+    primary.onInit(wrapCommits(commits));
+    for(IndexCommit commit : commits) {
+      if (refCounts.containsKey(commit.getGeneration())) {
+        indexCommits.put(commit.getGeneration(), commit);
+      }
     }
-    
-    @Override
-    public String toString() {
-      return id + " : " + segmentsFileName;
+    lastCommit = commits.get(commits.size() - 1);
+  }
+
+  /**
+   * Release a snapshotted commit.
+   * 
+   * @param commit
+   *          the commit previously returned by {@link #snapshot}
+   */
+  public synchronized void release(IndexCommit commit) throws IOException {
+    long gen = commit.getGeneration();
+    releaseGen(gen);
+  }
+
+  /** Release a snapshot by generation. */
+  protected void releaseGen(long gen) throws IOException {
+    Integer refCount = refCounts.get(gen);
+    if (refCount == null) {
+      throw new IllegalArgumentException("commit gen=" + gen + " is not currently snapshotted");
+    }
+    int refCountInt = refCount.intValue();
+    assert refCountInt > 0;
+    refCountInt--;
+    if (refCountInt == 0) {
+      refCounts.remove(gen);
+      indexCommits.remove(gen);
+    } else {
+      refCounts.put(gen, refCountInt);
     }
   }
 
+  /**
+   * Snapshots the last commit and returns it. Once a commit is 'snapshotted,' it is protected
+   * from deletion (as long as this {@link IndexDeletionPolicy} is used). The
+   * snapshot can be removed by calling {@link #release(IndexCommit)} followed
+   * by a call to {@link IndexWriter#deleteUnusedFiles()}.
+   *
+   * <p>
+   * <b>NOTE:</b> while the snapshot is held, the files it references will not
+   * be deleted, which will consume additional disk space in your index. If you
+   * take a snapshot at a particularly bad time (say just before you call
+   * forceMerge) then in the worst case this could consume an extra 1X of your
+   * total index size, until you release the snapshot.
+   * 
+   * @throws IllegalStateException
+   *           if this index does not have any commits yet
+   * @return the {@link IndexCommit} that was snapshotted.
+   */
+  public synchronized IndexCommit snapshot() throws IOException {
+    if (lastCommit == null) {
+      // No commit yet, eg this is a new IndexWriter:
+      throw new IllegalStateException("No index commit to snapshot");
+    }
+
+    long gen = lastCommit.getGeneration();
+
+    Integer refCount = refCounts.get(gen);
+    int refCountInt;
+    if (refCount == null) {
+      indexCommits.put(gen, lastCommit);
+      refCountInt = 0;
+    } else {
+      refCountInt = refCount.intValue();
+    }
+
+    refCounts.put(gen, refCountInt+1);
+
+    return lastCommit;
+  }
+
+  /** Returns the total number of snapshots currently held. */
+  public synchronized int getSnapshotCount() {
+    int total = 0;
+    for(Integer refCount : refCounts.values()) {
+      total += refCount.intValue();
+    }
+
+    return total;
+  }
+
+  /** Retrieve an {@link IndexCommit} from its generation;
+   *  returns null if this IndexCommit is not currently
+   *  snapshotted  */
+  public synchronized IndexCommit getIndexCommit(long gen) {
+    return indexCommits.get(gen);
+  }
+
+  @Override
+  public synchronized IndexDeletionPolicy clone() {
+    SnapshotDeletionPolicy other = (SnapshotDeletionPolicy) super.clone();
+    other.primary = this.primary.clone();
+    other.lastCommit = null;
+    other.refCounts = new HashMap<Long,Integer>(refCounts);
+    other.indexCommits = new HashMap<Long,IndexCommit>(indexCommits);
+    return other;
+  }
+
+  /** Wraps each {@link IndexCommit} as a {@link
+   *  SnapshotCommitPoint}. */
+  private List<IndexCommit> wrapCommits(List<? extends IndexCommit> commits) {
+    List<IndexCommit> wrappedCommits = new ArrayList<IndexCommit>(commits.size());
+    for (IndexCommit ic : commits) {
+      wrappedCommits.add(new SnapshotCommitPoint(ic));
+    }
+    return wrappedCommits;
+  }
+
   /** Wraps a provided {@link IndexCommit} and prevents it
    *  from being deleted. */
-  protected class SnapshotCommitPoint extends IndexCommit {
+  private class SnapshotCommitPoint extends IndexCommit {
 
     /** The {@link IndexCommit} we are preventing from deletion. */
     protected IndexCommit cp;
@@ -87,20 +205,12 @@ public class SnapshotDeletionPolicy exte
       return "SnapshotDeletionPolicy.SnapshotCommitPoint(" + cp + ")";
     }
 
-    /**
-     * Returns true if this segment can be deleted. The default implementation
-     * returns false if this segment is currently held as snapshot.
-     */
-    protected boolean shouldDelete(String segmentsFileName) {
-      return !segmentsFileToIDs.containsKey(segmentsFileName);
-    }
-
     @Override
     public void delete() {
       synchronized (SnapshotDeletionPolicy.this) {
         // Suppress the delete request if this commit point is
         // currently snapshotted.
-        if (shouldDelete(getSegmentsFileName())) {
+        if (!refCounts.containsKey(cp.getGeneration())) {
           cp.delete();
         }
       }
@@ -141,246 +251,4 @@ public class SnapshotDeletionPolicy exte
       return cp.getSegmentCount();
     }
   }
-
-  /** Snapshots info */
-  private Map<String, SnapshotInfo> idToSnapshot = new HashMap<String, SnapshotInfo>();
-
-  // multiple IDs could point to the same commit point (segments file name)
-  private Map<String, Set<String>> segmentsFileToIDs = new HashMap<String, Set<String>>();
-
-  private IndexDeletionPolicy primary;
-
-  /** Most recently committed {@link IndexCommit}. */
-  protected IndexCommit lastCommit;
-
-  /** Sole constructor, taking the incoming {@link
-   *  IndexDeletionPolicy} to wrap. */
-  public SnapshotDeletionPolicy(IndexDeletionPolicy primary) {
-    this.primary = primary;
-  }
-
-  /**
-   * {@link SnapshotDeletionPolicy} wraps another {@link IndexDeletionPolicy} to
-   * enable flexible snapshotting.
-   * 
-   * @param primary
-   *          the {@link IndexDeletionPolicy} that is used on non-snapshotted
-   *          commits. Snapshotted commits, are not deleted until explicitly
-   *          released via {@link #release(String)}
-   * @param snapshotsInfo
-   *          A mapping of snapshot ID to the segments filename that is being
-   *          snapshotted. The expected input would be the output of
-   *          {@link #getSnapshots()}. A null value signals that there are no
-   *          initial snapshots to maintain.
-   */
-  public SnapshotDeletionPolicy(IndexDeletionPolicy primary,
-      Map<String, String> snapshotsInfo) {
-    this(primary);
-
-    if (snapshotsInfo != null) {
-      // Add the ID->segmentIDs here - the actual IndexCommits will be
-      // reconciled on the call to onInit()
-      for (Entry<String, String> e : snapshotsInfo.entrySet()) {
-        registerSnapshotInfo(e.getKey(), e.getValue(), null);
-      }
-    }
-  }
-
-  /**
-   * Checks if the given id is already used by another snapshot, and throws
-   * {@link IllegalStateException} if it is.
-   */
-  protected void checkSnapshotted(String id) {
-    if (isSnapshotted(id)) {
-      throw new IllegalStateException("Snapshot ID " + id
-          + " is already used - must be unique");
-    }
-  }
-
-  /** Registers the given snapshot information. */
-  protected void registerSnapshotInfo(String id, String segment, IndexCommit commit) {
-    idToSnapshot.put(id, new SnapshotInfo(id, segment, commit));
-    Set<String> ids = segmentsFileToIDs.get(segment);
-    if (ids == null) {
-      ids = new HashSet<String>();
-      segmentsFileToIDs.put(segment, ids);
-    }
-    ids.add(id);
-  }
-
-  /** Wraps each {@link IndexCommit} as a {@link
-   *  SnapshotCommitPoint}. */
-  protected List<IndexCommit> wrapCommits(List<? extends IndexCommit> commits) {
-    List<IndexCommit> wrappedCommits = new ArrayList<IndexCommit>(commits.size());
-    for (IndexCommit ic : commits) {
-      wrappedCommits.add(new SnapshotCommitPoint(ic));
-    }
-    return wrappedCommits;
-  }
-
-  /**
-   * Get a snapshotted IndexCommit by ID. The IndexCommit can then be used to
-   * open an IndexReader on a specific commit point, or rollback the index by
-   * opening an IndexWriter with the IndexCommit specified in its
-   * {@link IndexWriterConfig}.
-   * 
-   * @param id
-   *          a unique identifier of the commit that was snapshotted.
-   * @throws IllegalStateException
-   *           if no snapshot exists by the specified ID.
-   * @return The {@link IndexCommit} for this particular snapshot.
-   */
-  public synchronized IndexCommit getSnapshot(String id) {
-    SnapshotInfo snapshotInfo = idToSnapshot.get(id);
-    if (snapshotInfo == null) {
-      throw new IllegalStateException("No snapshot exists by ID: " + id);
-    }
-    return snapshotInfo.commit;
-  }
-
-  /**
-   * Get all the snapshots in a map of snapshot IDs to the segments they
-   * 'cover.' This can be passed to
-   * {@link #SnapshotDeletionPolicy(IndexDeletionPolicy, Map)} in order to
-   * initialize snapshots at construction.
-   */
-  public synchronized Map<String, String> getSnapshots() {
-    Map<String, String> snapshots = new HashMap<String, String>();
-    for (Entry<String, SnapshotInfo> e : idToSnapshot.entrySet()) {
-      snapshots.put(e.getKey(), e.getValue().segmentsFileName);
-    }
-    return snapshots;
-  }
-
-  /**
-   * Returns true if the given ID is already used by a snapshot. You can call
-   * this method before {@link #snapshot(String)} if you are not sure whether
-   * the ID is already used or not.
-   */
-  public boolean isSnapshotted(String id) {
-    return idToSnapshot.containsKey(id);
-  }
-
-  @Override
-  public synchronized void onCommit(List<? extends IndexCommit> commits)
-      throws IOException {
-    primary.onCommit(wrapCommits(commits));
-    lastCommit = commits.get(commits.size() - 1);
-  }
-
-  @Override
-  public synchronized void onInit(List<? extends IndexCommit> commits)
-      throws IOException {
-    primary.onInit(wrapCommits(commits));
-    lastCommit = commits.get(commits.size() - 1);
-
-    /*
-     * Assign snapshotted IndexCommits to their correct snapshot IDs as
-     * specified in the constructor.
-     */
-    for (IndexCommit commit : commits) {
-      Set<String> ids = segmentsFileToIDs.get(commit.getSegmentsFileName());
-      if (ids != null) {
-        for (String id : ids) {
-          idToSnapshot.get(id).commit = commit;
-        }
-      }
-    }
-
-    /*
-     * Second, see if there are any instances where a snapshot ID was specified
-     * in the constructor but an IndexCommit doesn't exist. In this case, the ID
-     * should be removed.
-     * 
-     * Note: This code is protective for extreme cases where IDs point to
-     * non-existent segments. As the constructor should have received its
-     * information via a call to getSnapshots(), the data should be well-formed.
-     */
-    // Find lost snapshots
-    ArrayList<String> idsToRemove = null;
-    for (Entry<String, SnapshotInfo> e : idToSnapshot.entrySet()) {
-      if (e.getValue().commit == null) {
-        if (idsToRemove == null) {
-          idsToRemove = new ArrayList<String>();
-        }
-        idsToRemove.add(e.getKey());
-      }
-    }
-    // Finally, remove those 'lost' snapshots.
-    if (idsToRemove != null) {
-      for (String id : idsToRemove) {
-        SnapshotInfo info = idToSnapshot.remove(id);
-        segmentsFileToIDs.remove(info.segmentsFileName);
-      }
-    }
-  }
-
-  /**
-   * Release a snapshotted commit by ID.
-   * 
-   * @param id
-   *          a unique identifier of the commit that is un-snapshotted.
-   * @throws IllegalStateException
-   *           if no snapshot exists by this ID.
-   */
-  public synchronized void release(String id) throws IOException {
-    SnapshotInfo info = idToSnapshot.remove(id);
-    if (info == null) {
-      throw new IllegalStateException("Snapshot doesn't exist: " + id);
-    }
-    Set<String> ids = segmentsFileToIDs.get(info.segmentsFileName);
-    if (ids != null) {
-      ids.remove(id);
-      if (ids.size() == 0) {
-        segmentsFileToIDs.remove(info.segmentsFileName);
-      }
-    }
-  }
-
-  /**
-   * Snapshots the last commit. Once a commit is 'snapshotted,' it is protected
-   * from deletion (as long as this {@link IndexDeletionPolicy} is used). The
-   * commit can be removed by calling {@link #release(String)} using the same ID
-   * parameter followed by a call to {@link IndexWriter#deleteUnusedFiles()}.
-   * <p>
-   * <b>NOTE:</b> ID must be unique in the system. If the same ID is used twice,
-   * an {@link IllegalStateException} is thrown.
-   * <p>
-   * <b>NOTE:</b> while the snapshot is held, the files it references will not
-   * be deleted, which will consume additional disk space in your index. If you
-   * take a snapshot at a particularly bad time (say just before you call
-   * forceMerge) then in the worst case this could consume an extra 1X of your
-   * total index size, until you release the snapshot.
-   * 
-   * @param id
-   *          a unique identifier of the commit that is being snapshotted.
-   * @throws IllegalStateException
-   *           if either there is no 'last commit' to snapshot, or if the
-   *           parameter 'ID' refers to an already snapshotted commit.
-   * @return the {@link IndexCommit} that was snapshotted.
-   */
-  public synchronized IndexCommit snapshot(String id) throws IOException {
-    if (lastCommit == null) {
-      // no commit exists. Really shouldn't happen, but might be if SDP is
-      // accessed before onInit or onCommit were called.
-      throw new IllegalStateException("No index commit to snapshot");
-    }
-
-    // Can't use the same snapshot ID twice...
-    checkSnapshotted(id);
-
-    registerSnapshotInfo(id, lastCommit.getSegmentsFileName(), lastCommit);
-    return lastCommit;
-  }
-
-  @Override
-  public IndexDeletionPolicy clone() {
-    SnapshotDeletionPolicy other = (SnapshotDeletionPolicy) super.clone();
-    other.primary = this.primary.clone();
-    other.lastCommit = null;
-    other.segmentsFileToIDs = new HashMap<String, Set<String>>(segmentsFileToIDs);
-    other.idToSnapshot = new HashMap<String, SnapshotInfo>(idToSnapshot);
-    return other;
-  }
-
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java?rev=1478452&r1=1478451&r2=1478452&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java Thu May  2 17:16:07 2013
@@ -898,13 +898,13 @@ public void testFilesOpenClose() throws 
     SnapshotDeletionPolicy sdp = (SnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
     writer.addDocument(new Document());
     writer.commit();
-    sdp.snapshot("c1");
+    sdp.snapshot();
     writer.addDocument(new Document());
     writer.commit();
-    sdp.snapshot("c2");
+    sdp.snapshot();
     writer.addDocument(new Document());
     writer.commit();
-    sdp.snapshot("c3");
+    sdp.snapshot();
     writer.close();
     long currentGen = 0;
     for (IndexCommit ic : DirectoryReader.listCommits(dir)) {

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1478452&r1=1478451&r2=1478452&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java Thu May  2 17:16:07 2013
@@ -1392,7 +1392,7 @@ public class TestIndexWriter extends Luc
     assertEquals(1, DirectoryReader.listCommits(dir).size());
 
     // Keep that commit
-    sdp.snapshot("id");
+    IndexCommit id = sdp.snapshot();
 
     // Second commit - now KeepOnlyLastCommit cannot delete the prev commit.
     doc = new Document();
@@ -1402,7 +1402,7 @@ public class TestIndexWriter extends Luc
     assertEquals(2, DirectoryReader.listCommits(dir).size());
 
     // Should delete the unreferenced commit
-    sdp.release("id");
+    sdp.release(id);
     writer.deleteUnusedFiles();
     assertEquals(1, DirectoryReader.listCommits(dir).size());
 

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestPersistentSnapshotDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestPersistentSnapshotDeletionPolicy.java?rev=1478452&r1=1478451&r2=1478452&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestPersistentSnapshotDeletionPolicy.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestPersistentSnapshotDeletionPolicy.java Thu May  2 17:16:07 2013
@@ -18,8 +18,6 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.util.Map;
-import java.util.Map.Entry;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
@@ -62,25 +60,13 @@ public class TestPersistentSnapshotDelet
         TEST_VERSION_CURRENT);
   }
 
-  @Override
-  protected SnapshotDeletionPolicy getDeletionPolicy(Map<String, String> snapshots) throws IOException {
-    SnapshotDeletionPolicy sdp = getDeletionPolicy();
-    if (snapshots != null) {
-      for (Entry<String, String> e: snapshots.entrySet()) {
-        sdp.registerSnapshotInfo(e.getKey(), e.getValue(), null);
-      }
-    }
-    return sdp;
-  }
-
-  @Override
   @Test
   public void testExistingSnapshots() throws Exception {
     int numSnapshots = 3;
     Directory dir = newDirectory();
     IndexWriter writer = new IndexWriter(dir, getConfig(random(), getDeletionPolicy()));
     PersistentSnapshotDeletionPolicy psdp = (PersistentSnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
-    prepareIndexAndSnapshots(psdp, writer, numSnapshots, "snapshot");
+    prepareIndexAndSnapshots(psdp, writer, numSnapshots);
     writer.close();
     psdp.close();
 
@@ -88,19 +74,16 @@ public class TestPersistentSnapshotDelet
     psdp = new PersistentSnapshotDeletionPolicy(
         new KeepOnlyLastCommitDeletionPolicy(), snapshotDir, OpenMode.APPEND,
         TEST_VERSION_CURRENT);
-    new IndexWriter(dir, getConfig(random(), psdp)).close();
 
-    assertSnapshotExists(dir, psdp, numSnapshots);
-    assertEquals(numSnapshots, psdp.getSnapshots().size());
+    IndexWriter iw = new IndexWriter(dir, getConfig(random(), psdp));
+    psdp = (PersistentSnapshotDeletionPolicy) iw.getConfig().getIndexDeletionPolicy();
+    iw.close();
+
+    assertSnapshotExists(dir, psdp, numSnapshots, false);
     psdp.close();
     dir.close();
   }
 
-  @Test(expected=IllegalArgumentException.class)
-  public void testIllegalSnapshotId() throws Exception {
-    getDeletionPolicy().snapshot("$SNAPSHOTS_DOC$");
-  }
-  
   @Test
   public void testInvalidSnapshotInfos() throws Exception {
     // Add the correct number of documents (1), but without snapshot information
@@ -113,6 +96,7 @@ public class TestPersistentSnapshotDelet
           TEST_VERSION_CURRENT);
       fail("should not have succeeded to read from an invalid Directory");
     } catch (IllegalStateException e) {
+      // expected
     }
   }
 
@@ -144,16 +128,35 @@ public class TestPersistentSnapshotDelet
     Directory dir = newDirectory();
     IndexWriter writer = new IndexWriter(dir, getConfig(random(), getDeletionPolicy()));
     PersistentSnapshotDeletionPolicy psdp = (PersistentSnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
-    prepareIndexAndSnapshots(psdp, writer, 1, "snapshot");
+    prepareIndexAndSnapshots(psdp, writer, 1);
+    writer.close();
+
+    psdp.release(snapshots.get(0));
+    psdp.close();
+
+    psdp = new PersistentSnapshotDeletionPolicy(
+        new KeepOnlyLastCommitDeletionPolicy(), snapshotDir, OpenMode.APPEND,
+        TEST_VERSION_CURRENT);
+    assertEquals("Should have no snapshots !", 0, psdp.getSnapshotCount());
+    psdp.close();
+    dir.close();
+  }
+
+  @Test
+  public void testSnapshotReleaseByGeneration() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter writer = new IndexWriter(dir, getConfig(random(), getDeletionPolicy()));
+    PersistentSnapshotDeletionPolicy psdp = (PersistentSnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
+    prepareIndexAndSnapshots(psdp, writer, 1);
     writer.close();
 
-    psdp.release("snapshot0");
+    psdp.release(snapshots.get(0).getGeneration());
     psdp.close();
 
     psdp = new PersistentSnapshotDeletionPolicy(
         new KeepOnlyLastCommitDeletionPolicy(), snapshotDir, OpenMode.APPEND,
         TEST_VERSION_CURRENT);
-    assertEquals("Should have no snapshots !", 0, psdp.getSnapshots().size());
+    assertEquals("Should have no snapshots !", 0, psdp.getSnapshotCount());
     psdp.close();
     dir.close();
   }
@@ -167,7 +170,7 @@ public class TestPersistentSnapshotDelet
     Directory dir = newDirectory();
     IndexWriter writer = new IndexWriter(dir, getConfig(random(), getDeletionPolicy()));
     PersistentSnapshotDeletionPolicy psdp = (PersistentSnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
-    prepareIndexAndSnapshots(psdp, writer, numSnapshots, "snapshot");
+    prepareIndexAndSnapshots(psdp, writer, numSnapshots);
     writer.close();
     dir.close();
     
@@ -176,16 +179,11 @@ public class TestPersistentSnapshotDelet
       new PersistentSnapshotDeletionPolicy(
           new KeepOnlyLastCommitDeletionPolicy(), snapshotDir, OpenMode.APPEND,
           TEST_VERSION_CURRENT);
-     fail("should not have reached here - the snapshots directory should be locked!");
+      fail("should not have reached here - the snapshots directory should be locked!");
     } catch (LockObtainFailedException e) {
       // expected
     } finally {
       psdp.close();
     }
-    
-    // Reading the snapshots info should succeed though
-    Map<String, String> snapshots = PersistentSnapshotDeletionPolicy.readSnapshotsInfo(snapshotDir);
-    assertEquals("expected " + numSnapshots + " snapshots, got " + snapshots.size(), numSnapshots, snapshots.size());
   }
-  
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java?rev=1478452&r1=1478451&r2=1478452&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java Thu May  2 17:16:07 2013
@@ -18,8 +18,9 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Map;
+import java.util.List;
 import java.util.Random;
 
 import org.apache.lucene.analysis.MockAnalyzer;
@@ -28,7 +29,6 @@ import org.apache.lucene.document.FieldT
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.ThreadInterruptedException;
 import org.junit.Test;
@@ -63,30 +63,33 @@ public class TestSnapshotDeletionPolicy 
     }
   }
 
+  protected List<IndexCommit> snapshots = new ArrayList<IndexCommit>();
+
   protected void prepareIndexAndSnapshots(SnapshotDeletionPolicy sdp,
-      IndexWriter writer, int numSnapshots, String snapshotPrefix)
+      IndexWriter writer, int numSnapshots)
       throws RuntimeException, IOException {
     for (int i = 0; i < numSnapshots; i++) {
       // create dummy document to trigger commit.
       writer.addDocument(new Document());
       writer.commit();
-      sdp.snapshot(snapshotPrefix + i);
+      snapshots.add(sdp.snapshot());
     }
   }
 
   protected SnapshotDeletionPolicy getDeletionPolicy() throws IOException {
-    return getDeletionPolicy(null);
-  }
-
-  protected SnapshotDeletionPolicy getDeletionPolicy(Map<String, String> snapshots) throws IOException {
-    return new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy(), snapshots);
+    return new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy());
   }
 
-  protected void assertSnapshotExists(Directory dir, SnapshotDeletionPolicy sdp, int numSnapshots) throws Exception {
+  protected void assertSnapshotExists(Directory dir, SnapshotDeletionPolicy sdp, int numSnapshots, boolean checkIndexCommitSame) throws Exception {
     for (int i = 0; i < numSnapshots; i++) {
-      IndexCommit snapshot = sdp.getSnapshot("snapshot" + i);
+      IndexCommit snapshot = snapshots.get(i);
       checkMaxDoc(snapshot, i + 1);
       checkSnapshotExists(dir, snapshot);
+      if (checkIndexCommitSame) {
+        assertSame(snapshot, sdp.getIndexCommit(snapshot.getGeneration()));
+      } else {
+        assertEquals(snapshot.getGeneration(), sdp.getIndexCommit(snapshot.getGeneration()).getGeneration());
+      }
     }
   }
   
@@ -177,13 +180,14 @@ public class TestSnapshotDeletionPolicy 
    */
   public void backupIndex(Directory dir, SnapshotDeletionPolicy dp) throws Exception {
     // To backup an index we first take a snapshot:
+    IndexCommit snapshot = dp.snapshot();
     try {
-      copyFiles(dir,  dp.snapshot("id"));
+      copyFiles(dir, snapshot);
     } finally {
       // Make sure to release the snapshot, otherwise these
       // files will never be deleted during this IndexWriter
       // session:
-      dp.release("id");
+      dp.release(snapshot);
     }
   }
 
@@ -240,13 +244,13 @@ public class TestSnapshotDeletionPolicy 
     Directory dir = newDirectory();
     IndexWriter writer = new IndexWriter(dir, getConfig(random(), getDeletionPolicy()));
     SnapshotDeletionPolicy sdp = (SnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
-    prepareIndexAndSnapshots(sdp, writer, numSnapshots, "snapshot");
+    prepareIndexAndSnapshots(sdp, writer, numSnapshots);
     writer.close();
     
-    assertSnapshotExists(dir, sdp, numSnapshots);
+    assertSnapshotExists(dir, sdp, numSnapshots, true);
 
     // open a reader on a snapshot - should succeed.
-    DirectoryReader.open(sdp.getSnapshot("snapshot0")).close();
+    DirectoryReader.open(snapshots.get(0)).close();
 
     // open a new IndexWriter w/ no snapshots to keep and assert that all snapshots are gone.
     sdp = getDeletionPolicy();
@@ -254,15 +258,6 @@ public class TestSnapshotDeletionPolicy 
     writer.deleteUnusedFiles();
     writer.close();
     assertEquals("no snapshots should exist", 1, DirectoryReader.listCommits(dir).size());
-    
-    for (int i = 0; i < numSnapshots; i++) {
-      try {
-        sdp.getSnapshot("snapshot" + i);
-        fail("snapshot shouldn't have existed, but did: snapshot" + i);
-      } catch (IllegalStateException e) {
-        // expected - snapshot should not exist
-      }
-    }
     dir.close();
   }
 
@@ -273,14 +268,16 @@ public class TestSnapshotDeletionPolicy 
     final SnapshotDeletionPolicy sdp = (SnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
 
     Thread[] threads = new Thread[10];
+    final IndexCommit[] snapshots = new IndexCommit[threads.length];
     for (int i = 0; i < threads.length; i++) {
+      final int finalI = i;
       threads[i] = new Thread() {
         @Override
         public void run() {
           try {
             writer.addDocument(new Document());
             writer.commit();
-            sdp.snapshot(getName());
+            snapshots[finalI] = sdp.snapshot();
           } catch (Exception e) {
             throw new RuntimeException(e);
           }
@@ -301,8 +298,8 @@ public class TestSnapshotDeletionPolicy 
     writer.addDocument(new Document());
     writer.commit();
     
-    for (Thread t : threads) {
-      sdp.release(t.getName());
+    for (int i=0;i<threads.length;i++) {
+      sdp.release(snapshots[i]);
       writer.deleteUnusedFiles();
     }
     assertEquals(1, DirectoryReader.listCommits(dir).size());
@@ -317,19 +314,19 @@ public class TestSnapshotDeletionPolicy 
 
     IndexWriter writer = new IndexWriter(dir, getConfig(random(), getDeletionPolicy()));
     SnapshotDeletionPolicy sdp = (SnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
-    prepareIndexAndSnapshots(sdp, writer, numSnapshots, "snapshot");
+    prepareIndexAndSnapshots(sdp, writer, numSnapshots);
     writer.close();
 
     // now open the writer on "snapshot0" - make sure it succeeds
-    writer = new IndexWriter(dir, getConfig(random(), sdp).setIndexCommit(sdp.getSnapshot("snapshot0")));
+    writer = new IndexWriter(dir, getConfig(random(), sdp).setIndexCommit(snapshots.get(0)));
     // this does the actual rollback
     writer.commit();
     writer.deleteUnusedFiles();
-    assertSnapshotExists(dir, sdp, numSnapshots - 1);
+    assertSnapshotExists(dir, sdp, numSnapshots - 1, true);
     writer.close();
 
     // but 'snapshot1' files will still exist (need to release snapshot before they can be deleted).
-    String segFileName = sdp.getSnapshot("snapshot1").getSegmentsFileName();
+    String segFileName = snapshots.get(1).getSegmentsFileName();
     assertTrue("snapshot files should exist in the directory: " + segFileName, dir.fileExists(segFileName));
 
     dir.close();
@@ -340,7 +337,7 @@ public class TestSnapshotDeletionPolicy 
     Directory dir = newDirectory();
     IndexWriter writer = new IndexWriter(dir, getConfig(random(), getDeletionPolicy()));
     SnapshotDeletionPolicy sdp = (SnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
-    prepareIndexAndSnapshots(sdp, writer, 1, "snapshot");
+    prepareIndexAndSnapshots(sdp, writer, 1);
     
     // Create another commit - we must do that, because otherwise the "snapshot"
     // files will still remain in the index, since it's the last commit.
@@ -348,16 +345,8 @@ public class TestSnapshotDeletionPolicy 
     writer.commit();
     
     // Release
-    String snapId = "snapshot0";
-    String segFileName = sdp.getSnapshot(snapId).getSegmentsFileName();
-    sdp.release(snapId);
-    try {
-      sdp.getSnapshot(snapId);
-      fail("should not have succeeded to get an unsnapshotted id");
-    } catch (IllegalStateException e) {
-      // expected
-    }
-    assertNull(sdp.getSnapshots().get(snapId));
+    String segFileName = snapshots.get(0).getSegmentsFileName();
+    sdp.release(snapshots.get(0));
     writer.deleteUnusedFiles();
     writer.close();
     assertFalse("segments file should not be found in dirctory: " + segFileName, dir.fileExists(segFileName));
@@ -365,27 +354,6 @@ public class TestSnapshotDeletionPolicy 
   }
 
   @Test
-  public void testExistingSnapshots() throws Exception {
-    // Tests the ability to construct a SDP from existing snapshots, and
-    // asserts that those snapshots/commit points are protected.
-    int numSnapshots = 3;
-    Directory dir = newDirectory();
-    IndexWriter writer = new IndexWriter(dir, getConfig(random(), getDeletionPolicy()));
-    SnapshotDeletionPolicy sdp = (SnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
-    prepareIndexAndSnapshots(sdp, writer, numSnapshots, "snapshot");
-    writer.close();
-
-    // Make a new policy and initialize with snapshots.
-    sdp = getDeletionPolicy(sdp.getSnapshots());
-    writer = new IndexWriter(dir, getConfig(random(), sdp));
-    // attempt to delete unused files - the snapshotted files should not be deleted
-    writer.deleteUnusedFiles();
-    writer.close();
-    assertSnapshotExists(dir, sdp, numSnapshots);
-    dir.close();
-  }
-
-  @Test
   public void testSnapshotLastCommitTwice() throws Exception {
     Directory dir = newDirectory();
 
@@ -394,11 +362,9 @@ public class TestSnapshotDeletionPolicy 
     writer.addDocument(new Document());
     writer.commit();
     
-    String s1 = "s1";
-    String s2 = "s2";
-    IndexCommit ic1 = sdp.snapshot(s1);
-    IndexCommit ic2 = sdp.snapshot(s2);
-    assertTrue(ic1 == ic2); // should be the same instance
+    IndexCommit s1 = sdp.snapshot();
+    IndexCommit s2 = sdp.snapshot();
+    assertSame(s1, s2); // should be the same instance
     
     // create another commit
     writer.addDocument(new Document());
@@ -407,7 +373,7 @@ public class TestSnapshotDeletionPolicy 
     // release "s1" should not delete "s2"
     sdp.release(s1);
     writer.deleteUnusedFiles();
-    checkSnapshotExists(dir, ic2);
+    checkSnapshotExists(dir, s2);
     
     writer.close();
     dir.close();
@@ -422,7 +388,7 @@ public class TestSnapshotDeletionPolicy 
     SnapshotDeletionPolicy sdp = (SnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
     writer.addDocument(new Document());
     writer.commit();
-    IndexCommit ic = sdp.snapshot("s1");
+    IndexCommit s1 = sdp.snapshot();
 
     // create another commit, not snapshotted.
     writer.addDocument(new Document());
@@ -432,21 +398,7 @@ public class TestSnapshotDeletionPolicy 
     // commit.
     new IndexWriter(dir, getConfig(random(), null)).close();
     
-    assertFalse("snapshotted commit should not exist", dir.fileExists(ic.getSegmentsFileName()));
-    
-    // Now reinit SDP from the commits in the index - the snapshot id should not
-    // exist anymore.
-    writer = new IndexWriter(dir, getConfig(random(), getDeletionPolicy(sdp.getSnapshots())));
-    sdp = (SnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
-    writer.close();
-    
-    try {
-      sdp.getSnapshot("s1");
-      fail("snapshot s1 should not exist");
-    } catch (IllegalStateException e) {
-      // expected.
-    }
+    assertFalse("snapshotted commit should not exist", dir.fileExists(s1.getSegmentsFileName()));
     dir.close();
   }
-
 }