You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2011/11/17 02:50:34 UTC

svn commit: r1203003 - in /lucene/dev/branches/branch_3x: ./ lucene/ lucene/backwards/src/test/ solr/ solr/core/ solr/core/src/java/org/apache/solr/handler/ solr/core/src/test/org/apache/solr/handler/ solr/solrj/

Author: hossman
Date: Thu Nov 17 01:50:33 2011
New Revision: 1203003

URL: http://svn.apache.org/viewvc?rev=1203003&view=rev
Log:
SOLR-2578: merge 1202969 from trunk (ReplicationHandler's backup command now supports a 'numberToKeep' param that can be used to delete all but the most recent N backups)

Modified:
    lucene/dev/branches/branch_3x/   (props changed)
    lucene/dev/branches/branch_3x/lucene/   (props changed)
    lucene/dev/branches/branch_3x/lucene/backwards/src/test/   (props changed)
    lucene/dev/branches/branch_3x/solr/   (props changed)
    lucene/dev/branches/branch_3x/solr/CHANGES.txt
    lucene/dev/branches/branch_3x/solr/core/   (props changed)
    lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
    lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
    lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
    lucene/dev/branches/branch_3x/solr/solrj/   (props changed)

Modified: lucene/dev/branches/branch_3x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/CHANGES.txt?rev=1203003&r1=1203002&r2=1203003&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_3x/solr/CHANGES.txt Thu Nov 17 01:50:33 2011
@@ -68,6 +68,10 @@ New Features
 * SOLR-1023: StatsComponent now supports date fields and string fields.
   (Chris Male, Mark Holland, Gunnlaugur Thor Briem, Ryan McKinley)
 
+* SOLR-2578: ReplicationHandler's backup command now supports a 'numberToKeep' 
+  param that can be used to delete all but the most recent N backups.
+  (James Dyer via hossman)
+
 Optimizations
 ----------------------
 

Modified: lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java?rev=1203003&r1=1203002&r2=1203003&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java (original)
+++ lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java Thu Nov 17 01:50:33 2011
@@ -110,7 +110,7 @@ public class ReplicationHandler extends 
     }
     // This command does not give the current index version of the master
     // It gives the current 'replicateable' index version
-   if (command.equals(CMD_INDEX_VERSION)) {
+    if (command.equals(CMD_INDEX_VERSION)) {
       IndexCommit commitPoint = indexCommitPoint;  // make a copy so it won't change
       if (commitPoint != null && replicationEnabled.get()) {
         //
@@ -186,10 +186,10 @@ public class ReplicationHandler extends 
     } else if (CMD_ENABLE_REPL.equalsIgnoreCase(command)) {
       replicationEnabled.set(true);
       rsp.add(STATUS, OK_STATUS);
-   } else if (CMD_DISABLE_REPL.equalsIgnoreCase(command)) {
-     replicationEnabled.set(false);
-     rsp.add(STATUS, OK_STATUS);
-   }
+    } else if (CMD_DISABLE_REPL.equalsIgnoreCase(command)) {
+      replicationEnabled.set(false);
+      rsp.add(STATUS, OK_STATUS);
+    }
   }
 
   private List<NamedList> getCommits() {
@@ -280,16 +280,17 @@ public class ReplicationHandler extends 
 
   private void doSnapShoot(SolrParams params, SolrQueryResponse rsp, SolrQueryRequest req) {
     try {
+      int numberToKeep = params.getInt(NUMBER_BACKUPS_TO_KEEP, Integer.MAX_VALUE);
       IndexDeletionPolicyWrapper delPolicy = core.getDeletionPolicy();
       IndexCommit indexCommit = delPolicy.getLatestCommit();
-
+      
       if(indexCommit == null) {
         indexCommit = req.getSearcher().getReader().getIndexCommit();
       }
-
+      
       // small race here before the commit point is saved
-      new SnapShooter(core, params.get("location")).createSnapAsync(indexCommit, this);
-
+      new SnapShooter(core, params.get("location")).createSnapAsync(indexCommit, numberToKeep, this);
+      
     } catch (Exception e) {
       LOG.warn("Exception during creating a snapshot", e);
       rsp.add("exception", e);
@@ -338,7 +339,7 @@ public class ReplicationHandler extends 
       rsp.add("status", "unable to get file names for given indexversion");
       rsp.add("exception", e);
       LOG.warn("Unable to get file names for indexCommit version: "
-              + version, e);
+               + version, e);
     }
     rsp.add(CMD_GET_FILE_LIST, result);
     if (confFileNameAlias.size() < 1)
@@ -1153,4 +1154,6 @@ public class ReplicationHandler extends 
   public static final String OK_STATUS = "OK";
 
   public static final String NEXT_EXECUTION_AT = "nextExecutionAt";
+  
+  public static final String NUMBER_BACKUPS_TO_KEEP = "numberToKeep";
 }

Modified: lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/SnapShooter.java?rev=1203003&r1=1203002&r2=1203003&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/SnapShooter.java (original)
+++ lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/SnapShooter.java Thu Nov 17 01:50:33 2011
@@ -22,9 +22,14 @@ import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.text.SimpleDateFormat;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Date;
+import java.util.List;
 import java.util.Locale;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.lucene.index.IndexCommit;
@@ -59,25 +64,32 @@ public class SnapShooter {
     }
     lockFactory = new SimpleFSLockFactory(snapDir);
   }
-
+  
   void createSnapAsync(final IndexCommit indexCommit, final ReplicationHandler replicationHandler) {
+    createSnapAsync(indexCommit, Integer.MAX_VALUE, replicationHandler);
+  }
+
+  void createSnapAsync(final IndexCommit indexCommit, final int numberToKeep, final ReplicationHandler replicationHandler) {
     replicationHandler.core.getDeletionPolicy().saveCommitPoint(indexCommit.getVersion());
 
     new Thread() {
       @Override
       public void run() {
-        createSnapshot(indexCommit, replicationHandler);
+        createSnapshot(indexCommit, numberToKeep, replicationHandler);
       }
     }.start();
   }
 
-  void createSnapshot(final IndexCommit indexCommit, ReplicationHandler replicationHandler) {
+  void createSnapshot(final IndexCommit indexCommit, int numberToKeep, ReplicationHandler replicationHandler) {
     NamedList details = new NamedList();
     details.add("startTime", new Date().toString());
     File snapShotDir = null;
     String directoryName = null;
     Lock lock = null;
     try {
+      if(numberToKeep<Integer.MAX_VALUE) {
+        deleteOldBackups(numberToKeep);
+      }
       SimpleDateFormat fmt = new SimpleDateFormat(DATE_FMT, Locale.US);
       directoryName = "snapshot." + fmt.format(new Date());
       lock = lockFactory.makeLock(directoryName + ".lock");
@@ -99,8 +111,8 @@ public class SnapShooter {
       LOG.error("Exception while creating snapshot", e);
       details.add("snapShootException", e.getMessage());
     } finally {
-        replicationHandler.core.getDeletionPolicy().releaseCommitPoint(indexCommit.getVersion());   
-        replicationHandler.snapShootDetails = details;
+      replicationHandler.core.getDeletionPolicy().releaseCommitPoint(indexCommit.getVersion());   
+      replicationHandler.snapShootDetails = details;
       if (lock != null) {
         try {
           lock.release();
@@ -110,6 +122,46 @@ public class SnapShooter {
       }
     }
   }
+  private void deleteOldBackups(int numberToKeep) {
+    File[] files = new File(snapDir).listFiles();
+    List<OldBackupDirectory> dirs = new ArrayList<OldBackupDirectory>();
+    for(File f : files) {
+      OldBackupDirectory obd = new OldBackupDirectory(f);
+      if(obd.dir != null) {
+        dirs.add(obd);
+      }
+    }
+    Collections.sort(dirs);
+    int i=1;
+    for(OldBackupDirectory dir : dirs) {
+      if( i > numberToKeep-1 ) {
+        SnapPuller.delTree(dir.dir);
+      }
+    }   
+  }
+  private class OldBackupDirectory implements Comparable<OldBackupDirectory>{
+    File dir;
+    Date timestamp;
+    final Pattern dirNamePattern = Pattern.compile("^snapshot[.](.*)$");
+    
+    OldBackupDirectory(File dir) {
+      if(dir.isDirectory()) {
+        Matcher m = dirNamePattern.matcher(dir.getName());
+        if(m.find()) {
+          try {
+            this.dir = dir;
+            this.timestamp = new SimpleDateFormat(DATE_FMT).parse(m.group(1));
+          } catch(Exception e) {
+            this.dir = null;
+            this.timestamp = null;
+          }
+        }
+      }
+    }
+    public int compareTo(OldBackupDirectory that) {
+      return that.timestamp.compareTo(this.timestamp);
+    }
+  }
 
   public static final String SNAP_DIR = "snapDir";
   public static final String DATE_FMT = "yyyyMMddHHmmss";
@@ -134,7 +186,7 @@ public class SnapShooter {
     }
     
     public void copyFile(File source, File destination, boolean preserveFileDate)
-        throws IOException {
+      throws IOException {
       // check source exists
       if (!source.exists()) {
         String message = "File " + source + " does not exist";
@@ -184,7 +236,7 @@ public class SnapShooter {
 
       if (source.length() != destination.length()) {
         String message = "Failed to copy full contents from " + source + " to "
-            + destination;
+          + destination;
         throw new IOException(message);
       }
 

Modified: lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java?rev=1203003&r1=1203002&r2=1203003&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java (original)
+++ lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java Thu Nov 17 01:50:33 2011
@@ -16,6 +16,20 @@
  */
 package org.apache.solr.handler;
 
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Writer;
+import java.net.URL;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.Map;
+import java.util.HashMap;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.IndexSearcher;
@@ -43,11 +57,6 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.io.*;
-import java.net.URL;
-import java.util.Map;
-import java.util.HashMap;
-
 /**
  * Test for ReplicationHandler
  *
@@ -629,7 +638,7 @@ public class TestReplicationHandler exte
       volatile String fail = null;
       @Override
       public void run() {
-        String masterUrl = "http://localhost:" + masterJetty.getLocalPort() + "/solr/replication?command=" + ReplicationHandler.CMD_BACKUP;
+        String masterUrl = "http://localhost:" + masterJetty.getLocalPort() + "/solr/replication?command=" + ReplicationHandler.CMD_BACKUP + "&" + ReplicationHandler.NUMBER_BACKUPS_TO_KEEP + "=1";
         URL url;
         InputStream stream = null;
         try {
@@ -644,14 +653,18 @@ public class TestReplicationHandler exte
 
       };
     };
-    BackupThread backupThread = new BackupThread();
-    backupThread.start();
     
-    File dataDir = new File(master.getDataDir());
     class CheckStatus extends Thread {
       volatile String fail = null;
       volatile String response = null;
       volatile boolean success = false;
+      volatile String backupTimestamp = null;
+      final String lastBackupTimestamp;
+      final Pattern p = Pattern.compile("<str name=\"snapshotCompletedAt\">(.*?)</str>");
+      
+      CheckStatus(String lastBackupTimestamp) {
+        this.lastBackupTimestamp = lastBackupTimestamp;
+      }
       @Override
       public void run() {
         String masterUrl = "http://localhost:" + masterJetty.getLocalPort() + "/solr/replication?command=" + ReplicationHandler.CMD_DETAILS;
@@ -662,7 +675,14 @@ public class TestReplicationHandler exte
           stream = url.openStream();
           response = IOUtils.toString(stream, "UTF-8");
           if(response.contains("<str name=\"status\">success</str>")) {
-            success = true;
+            Matcher m = p.matcher(response);
+            if(!m.find()) {
+              fail("could not find the completed timestamp in response.");
+            }
+            backupTimestamp = m.group(1);   
+            if(!backupTimestamp.equals(lastBackupTimestamp)) {
+              success = true;
+            }
           }
           stream.close();
         } catch (Exception e) {
@@ -673,47 +693,67 @@ public class TestReplicationHandler exte
 
       };
     };
-    int waitCnt = 0;
-    CheckStatus checkStatus = new CheckStatus();
-    while(true) {
-      checkStatus.run();
-      if(checkStatus.fail != null) {
-        fail(checkStatus.fail);
-      }
-      if(checkStatus.success) {
-        break;
+    
+    File[] snapDir = new File[2];
+    String firstBackupTimestamp = null;
+    for(int i=0 ; i<2 ; i++) {
+      BackupThread backupThread = new BackupThread();
+      backupThread.start();
+      
+      File dataDir = new File(master.getDataDir());
+      
+      int waitCnt = 0;
+      CheckStatus checkStatus = new CheckStatus(firstBackupTimestamp);
+      while(true) {
+        checkStatus.run();
+        if(checkStatus.fail != null) {
+          fail(checkStatus.fail);
+        }
+        if(checkStatus.success) {
+          if(i==0) {
+            firstBackupTimestamp = checkStatus.backupTimestamp;
+            Thread.sleep(1000); //ensure the next backup will have a different timestamp.
+          }
+          break;
+        }
+        Thread.sleep(200);
+        if(waitCnt == 10) {
+          fail("Backup success not detected:" + checkStatus.response);
+        }
+        waitCnt++;
       }
-      Thread.sleep(200);
-      if(waitCnt == 10) {
-        fail("Backup success not detected:" + checkStatus.response);
+      
+      if(backupThread.fail != null) {
+        fail(backupThread.fail);
       }
-      waitCnt++;
+  
+      File[] files = dataDir.listFiles(new FilenameFilter() {
+        
+          public boolean accept(File dir, String name) {
+            if(name.startsWith("snapshot")) {
+              return true;
+            }
+            return false;
+          }
+        });
+      assertEquals(1, files.length);
+      snapDir[i] = files[0];
+      Directory dir = new SimpleFSDirectory(snapDir[i].getAbsoluteFile());
+      IndexReader reader = IndexReader.open(dir);
+      IndexSearcher searcher = new IndexSearcher(reader);
+      TopDocs hits = searcher.search(new MatchAllDocsQuery(), 1);
+      assertEquals(nDocs, hits.totalHits);
+      reader.close();
+      searcher.close();
+      dir.close();
+    }
+    if(snapDir[0].exists()) {
+      fail("The first backup should have been cleaned up because " + ReplicationHandler.NUMBER_BACKUPS_TO_KEEP + " was set to 1");
     }
     
-    if(backupThread.fail != null) {
-      fail(backupThread.fail);
+    for(int i=0 ; i< snapDir.length ; i++) {
+      AbstractSolrTestCase.recurseDelete(snapDir[i]); // clean up the snap dir
     }
-
-    File[] files = dataDir.listFiles(new FilenameFilter() {
-      
-      public boolean accept(File dir, String name) {
-        if(name.startsWith("snapshot")) {
-          return true;
-        }
-        return false;
-      }
-    });
-    assertEquals(1, files.length);
-    File snapDir = files[0];
-    Directory dir = new SimpleFSDirectory(snapDir.getAbsoluteFile());
-    IndexReader reader = IndexReader.open(dir);
-    IndexSearcher searcher = new IndexSearcher(reader);
-    TopDocs hits = searcher.search(new MatchAllDocsQuery(), 1);
-
-    assertEquals(nDocs, hits.totalHits);
-    searcher.close();
-    reader.close();
-    dir.close();
   }
 
   /* character copy of file using UTF-8 */