You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ph...@apache.org on 2011/09/02 22:50:38 UTC

svn commit: r1164708 - in /zookeeper/trunk: ./ conf/ src/docs/src/documentation/content/xdocs/ src/java/main/org/apache/zookeeper/server/ src/java/main/org/apache/zookeeper/server/quorum/ src/java/test/org/apache/zookeeper/server/

Author: phunt
Date: Fri Sep  2 20:50:38 2011
New Revision: 1164708

URL: http://svn.apache.org/viewvc?rev=1164708&view=rev
Log:
ZOOKEEPER-1107. automating log and snapshot cleaning (Laxman via phunt)

Added:
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DatadirCleanupManager.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DatadirCleanupManagerTest.java
Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/conf/zoo_sample.cfg
    zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1164708&r1=1164707&r2=1164708&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Fri Sep  2 20:50:38 2011
@@ -446,6 +446,8 @@ NEW FEATURES:
 
   ZOOKEEPER-999. Create an package integration project (Eric Yang via phunt)
 
+  ZOOKEEPER-1107. automating log and snapshot cleaning (Laxman via phunt)
+
 DEPRECATION:
   ZOOKEEPER-1153. Deprecate AuthFLE and LE. (Flavio Junqueira via mahadev)
   

Modified: zookeeper/trunk/conf/zoo_sample.cfg
URL: http://svn.apache.org/viewvc/zookeeper/trunk/conf/zoo_sample.cfg?rev=1164708&r1=1164707&r2=1164708&view=diff
==============================================================================
--- zookeeper/trunk/conf/zoo_sample.cfg (original)
+++ zookeeper/trunk/conf/zoo_sample.cfg Fri Sep  2 20:50:38 2011
@@ -12,3 +12,14 @@ syncLimit=5
 dataDir=/tmp/zookeeper
 # the port at which the clients will connect
 clientPort=2181
+#
+# Be sure to read the maintenance section of the 
+# administrator guide before turning on autopurge.
+#
+# http://zookeeper.apache.org/doc/current/zookeeperAdmin.html#sc_maintenance
+#
+# The number of snapshots to retain in dataDir
+#autopurge.snapRetainCount=3
+# Purge task interval in hours
+# Set to "0" to disable auto purge feature
+#autopurge.purgeInterval=1

Modified: zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml?rev=1164708&r1=1164707&r2=1164708&view=diff
==============================================================================
--- zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml (original)
+++ zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml Fri Sep  2 20:50:38 2011
@@ -460,10 +460,11 @@ server.3=zoo3:2888:3888</programlisting>
         </para>
 
         <para>A ZooKeeper server <emphasis role="bold">will not remove
-          old snapshots and log files</emphasis>, this is the
-          responsibility of the operator. Every serving environment is
-          different and therefore the requirements of managing these
-          files may differ from install to install (backup for example).
+        old snapshots and log files</emphasis> when using the default
+        configuration (see autopurge below), this is the
+        responsibility of the operator. Every serving environment is
+        different and therefore the requirements of managing these
+        files may differ from install to install (backup for example).
         </para>
 
         <para>The PurgeTxnLog utility implements a simple retention
@@ -482,6 +483,13 @@ server.3=zoo3:2888:3888</programlisting>
 
         <programlisting> java -cp zookeeper.jar:lib/slf4j-api-1.6.1.jar:lib/slf4j-log4j12-1.6.1.jar:lib/log4j-1.2.15.jar:conf org.apache.zookeeper.server.PurgeTxnLog &lt;dataDir&gt; &lt;snapDir&gt; -n &lt;count&gt;</programlisting>
 
+        <para>Automatic purging of the snapshots and corresponding
+        transaction logs was introduced in version 3.4.0 and can be
+        enabled via the following configuration parameters <emphasis
+        role="bold">autopurge.snapRetainCount</emphasis> and <emphasis
+        role="bold">autopurge.purgeInterval</emphasis>. For more on
+        this, see <xref linkend="sc_advancedConfiguration"/>
+        below.</para>
       </section>
 
       <section>
@@ -774,6 +782,35 @@ server.3=zoo3:2888:3888</programlisting>
               the <emphasis role="bold">tickTime</emphasis>.</para>
             </listitem>
            </varlistentry>
+           
+          <varlistentry>
+            <term>autopurge.snapRetainCount</term>
+
+            <listitem>
+              <para>(No Java system property)</para>
+
+              <para><emphasis role="bold">New in 3.4.0:</emphasis> 
+              When enabled, ZooKeeper auto purge feature retains
+              the <emphasis role="bold">autopurge.snapRetainCount</emphasis> most
+              recent snapshots and the corresponding transaction logs in the 
+              <emphasis role="bold">dataDir</emphasis> and <emphasis 
+              role="bold">dataLogDir</emphasis> respectively and deletes the rest.
+              Defaults to 3. Minimum value is 3.</para>
+            </listitem>
+          </varlistentry>
+          
+          <varlistentry>
+            <term>autopurge.purgeInterval</term>
+
+            <listitem>
+              <para>(No Java system property)</para>
+
+              <para><emphasis role="bold">New in 3.4.0:</emphasis> The
+              time interval in hours for which the purge task has to
+              be triggered. Set to a positive integer (1 and above)
+              to enable the auto purging. Defaults to 0.</para>
+            </listitem>
+          </varlistentry>
         </variablelist>
       </section>
 

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DatadirCleanupManager.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DatadirCleanupManager.java?rev=1164708&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DatadirCleanupManager.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DatadirCleanupManager.java Fri Sep  2 20:50:38 2011
@@ -0,0 +1,192 @@
+/**
+ * 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.
+ */
+
+package org.apache.zookeeper.server;
+
+import java.io.File;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.TimeUnit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class manages the cleanup of snapshots and corresponding transaction
+ * logs by scheduling the auto purge task with the specified
+ * 'autopurge.purgeInterval'. It keeps the most recent
+ * 'autopurge.snapRetainCount' number of snapshots and corresponding transaction
+ * logs.
+ */
+public class DatadirCleanupManager {
+
+    private static final Logger LOG = LoggerFactory.getLogger(DatadirCleanupManager.class);
+
+    /**
+     * Status of the dataDir purge task
+     */
+    public enum PurgeTaskStatus {
+        NOT_STARTED, STARTED, COMPLETED;
+    }
+
+    private PurgeTaskStatus purgeTaskStatus = PurgeTaskStatus.NOT_STARTED;
+
+    private final String snapDir;
+
+    private final String dataLogDir;
+
+    private final int snapRetainCount;
+
+    private final int purgeInterval;
+
+    private Timer timer;
+
+    /**
+     * Constructor of DatadirCleanupManager. It takes the parameters to schedule
+     * the purge task.
+     * 
+     * @param snapDir
+     *            snapshot directory
+     * @param dataLogDir
+     *            transaction log directory
+     * @param snapRetainCount
+     *            number of snapshots to be retained after purge
+     * @param purgeInterval
+     *            purge interval in hours
+     */
+    public DatadirCleanupManager(String snapDir, String dataLogDir, int snapRetainCount,
+            int purgeInterval) {
+        this.snapDir = snapDir;
+        this.dataLogDir = dataLogDir;
+        this.snapRetainCount = snapRetainCount;
+        this.purgeInterval = purgeInterval;
+        LOG.info("autopurge.snapRetainCount set to " + snapRetainCount);
+        LOG.info("autopurge.purgeInterval set to " + purgeInterval);
+    }
+
+    /**
+     * Validates the purge configuration and schedules the purge task. Purge
+     * task keeps the most recent <code>snapRetainCount</code> number of
+     * snapshots and deletes the remaining for every <code>purgeInterval</code>
+     * hour(s).
+     * <p>
+     * <code>purgeInterval</code> of <code>0</code> or
+     * <code>negative integer</code> will not schedule the purge task.
+     * </p>
+     * 
+     * @see PurgeTxnLog#purge(File, File, int)
+     */
+    public void start() {
+        if (PurgeTaskStatus.STARTED == purgeTaskStatus) {
+            LOG.warn("Purge task is already running.");
+            return;
+        }
+        // Don't schedule the purge task with zero or negative purge interval.
+        if (purgeInterval <= 0) {
+            LOG.info("Purge task is not scheduled.");
+            return;
+        }
+
+        timer = new Timer("PurgeTask", true);
+        TimerTask task = new PurgeTask(dataLogDir, snapDir, snapRetainCount);
+        timer.scheduleAtFixedRate(task, 0, TimeUnit.HOURS.toMillis(purgeInterval));
+
+        purgeTaskStatus = PurgeTaskStatus.STARTED;
+    }
+
+    /**
+     * Shutdown the purge task.
+     */
+    public void shutdown() {
+        if (PurgeTaskStatus.STARTED == purgeTaskStatus) {
+            LOG.info("Shutting down purge task.");
+            timer.cancel();
+            purgeTaskStatus = PurgeTaskStatus.COMPLETED;
+        } else {
+            LOG.warn("Purge task not started. Ignoring shutdown!");
+        }
+    }
+
+    static class PurgeTask extends TimerTask {
+        private String logsDir;
+        private String snapsDir;
+        private int snapRetainCount;
+
+        public PurgeTask(String dataDir, String snapDir, int count) {
+            logsDir = dataDir;
+            snapsDir = snapDir;
+            snapRetainCount = count;
+        }
+
+        @Override
+        public void run() {
+            LOG.info("Purge task started.");
+            try {
+                PurgeTxnLog.purge(new File(logsDir), new File(snapsDir), snapRetainCount);
+            } catch (Exception e) {
+                LOG.error("Error occured while purging.", e);
+            }
+            LOG.info("Purge task completed.");
+        }
+    }
+
+    /**
+     * Returns the status of the purge task.
+     * 
+     * @return the status of the purge task
+     */
+    public PurgeTaskStatus getPurgeTaskStatus() {
+        return purgeTaskStatus;
+    }
+
+    /**
+     * Returns the snapshot directory.
+     * 
+     * @return the snapshot directory.
+     */
+    public String getSnapDir() {
+        return snapDir;
+    }
+
+    /**
+     * Returns transaction log directory.
+     * 
+     * @return the transaction log directory.
+     */
+    public String getDataLogDir() {
+        return dataLogDir;
+    }
+
+    /**
+     * Returns purge interval in hours.
+     * 
+     * @return the purge interval in hours.
+     */
+    public int getPurgeInterval() {
+        return purgeInterval;
+    }
+
+    /**
+     * Returns the number of snapshots to be retained after purge.
+     * 
+     * @return the number of snapshots to be retained after purge.
+     */
+    public int getSnapRetainCount() {
+        return snapRetainCount;
+    }
+}

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java?rev=1164708&r1=1164707&r2=1164708&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java Fri Sep  2 20:50:38 2011
@@ -69,8 +69,16 @@ public class QuorumPeerConfig {
     protected HashMap<Long, Long> serverGroup = new HashMap<Long, Long>();
     protected int numGroups = 0;
     protected QuorumVerifier quorumVerifier;
+    protected int snapRetainCount = 3;
+    protected int purgeInterval = 0;
 
     protected LearnerType peerType = LearnerType.PARTICIPANT;
+    
+    /**
+     * Minimum snapshot retain count.
+     * @see org.apache.zookeeper.server.PurgeTxnLog#purge(File, File, int)
+     */
+    private final int MIN_SNAP_RETAIN_COUNT = 3;
 
     @SuppressWarnings("serial")
     public static class ConfigException extends Exception {
@@ -158,6 +166,10 @@ public class QuorumPeerConfig {
                 {
                     throw new ConfigException("Unrecognised peertype: " + value);
                 }
+            } else if (key.equals("autopurge.snapRetainCount")) {
+                snapRetainCount = Integer.parseInt(value);
+            } else if (key.equals("autopurge.purgeInterval")) {
+                purgeInterval = Integer.parseInt(value);
             } else if (key.startsWith("server.")) {
                 int dot = key.indexOf('.');
                 long sid = Long.parseLong(key.substring(dot + 1));
@@ -215,6 +227,15 @@ public class QuorumPeerConfig {
                 System.setProperty("zookeeper." + key, value);
             }
         }
+        
+        // Reset to MIN_SNAP_RETAIN_COUNT if invalid (less than 3)
+        // PurgeTxnLog.purge(File, File, int) will not allow to purge less
+        // than 3.
+        if (snapRetainCount < MIN_SNAP_RETAIN_COUNT) {
+            LOG.warn("Invalid autopurge.snapRetainCount: " + snapRetainCount
+                    + ". Defaulting to " + MIN_SNAP_RETAIN_COUNT);
+            snapRetainCount = MIN_SNAP_RETAIN_COUNT;
+        }
 
         if (dataDir == null) {
             throw new IllegalArgumentException("dataDir is not set");
@@ -364,6 +385,14 @@ public class QuorumPeerConfig {
     public int getElectionAlg() { return electionAlg; }
     public int getElectionPort() { return electionPort; }    
     
+    public int getSnapRetainCount() {
+        return snapRetainCount;
+    }
+
+    public int getPurgeInterval() {
+        return purgeInterval;
+    }
+
     public QuorumVerifier getQuorumVerifier() {   
         return quorumVerifier;
     }

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java?rev=1164708&r1=1164707&r2=1164708&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java Fri Sep  2 20:50:38 2011
@@ -27,6 +27,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.zookeeper.jmx.ManagedUtil;
 import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.ZKDatabase;
+import org.apache.zookeeper.server.DatadirCleanupManager;
 import org.apache.zookeeper.server.ZooKeeperServerMain;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
 import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
@@ -100,6 +101,12 @@ public class QuorumPeerMain {
             config.parse(args[0]);
         }
 
+        // Start and schedule the the purge task
+        DatadirCleanupManager purgeMgr = new DatadirCleanupManager(config
+                .getDataDir(), config.getDataLogDir(), config
+                .getSnapRetainCount(), config.getPurgeInterval());
+        purgeMgr.start();
+
         if (args.length == 1 && config.servers.size() > 0) {
             runFromConfig(config);
         } else {

Added: zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DatadirCleanupManagerTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DatadirCleanupManagerTest.java?rev=1164708&view=auto
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DatadirCleanupManagerTest.java (added)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DatadirCleanupManagerTest.java Fri Sep  2 20:50:38 2011
@@ -0,0 +1,91 @@
+/**
+ * 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.
+ */
+
+package org.apache.zookeeper.server;
+
+import static org.apache.zookeeper.server.DatadirCleanupManager.PurgeTaskStatus.COMPLETED;
+import static org.apache.zookeeper.server.DatadirCleanupManager.PurgeTaskStatus.NOT_STARTED;
+import static org.apache.zookeeper.server.DatadirCleanupManager.PurgeTaskStatus.STARTED;
+
+import java.io.File;
+
+import junit.framework.Assert;
+
+import org.apache.zookeeper.test.ClientBase;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class DatadirCleanupManagerTest {
+
+    private DatadirCleanupManager purgeMgr;
+    private String snapDir;
+    private String dataLogDir;
+
+    @Before
+    public void setUp() throws Exception {
+        File dataDir = ClientBase.createTmpDir();
+        snapDir = dataDir.getAbsolutePath();
+        dataLogDir = dataDir.getAbsolutePath();
+    }
+
+    @Test
+    public void testPurgeTask() throws Exception {
+        purgeMgr = new DatadirCleanupManager(snapDir, dataLogDir, 3, 1);
+        purgeMgr.start();
+        Assert.assertEquals("Data log directory is not set as configured",
+                dataLogDir, purgeMgr.getDataLogDir());
+        Assert.assertEquals("Snapshot directory is not set as configured",
+                snapDir, purgeMgr.getSnapDir());
+        Assert.assertEquals("Snapshot retain count is not set as configured",
+                3, purgeMgr.getSnapRetainCount());
+        Assert.assertEquals("Purge task is not started", STARTED, purgeMgr.getPurgeTaskStatus());
+        purgeMgr.shutdown();
+        Assert.assertEquals("Purge task is still running after shutdown", COMPLETED,
+                purgeMgr.getPurgeTaskStatus());
+    }
+
+    @Test
+    public void testWithZeroPurgeInterval() throws Exception {
+        purgeMgr = new DatadirCleanupManager(snapDir, dataLogDir, 3, 0);
+        purgeMgr.start();
+        Assert.assertEquals("Purge task is scheduled with zero purge interval", NOT_STARTED,
+                purgeMgr.getPurgeTaskStatus());
+        purgeMgr.shutdown();
+        Assert.assertEquals("Purge task is scheduled with zero purge interval", NOT_STARTED,
+                purgeMgr.getPurgeTaskStatus());
+    }
+
+    @Test
+    public void testWithNegativePurgeInterval() throws Exception {
+        purgeMgr = new DatadirCleanupManager(snapDir, dataLogDir, 3, -1);
+        purgeMgr.start();
+        Assert.assertEquals("Purge task is scheduled with negative purge interval",
+                NOT_STARTED, purgeMgr.getPurgeTaskStatus());
+        purgeMgr.shutdown();
+        Assert.assertEquals("Purge task is scheduled with negative purge interval", NOT_STARTED,
+                purgeMgr.getPurgeTaskStatus());
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        if (purgeMgr != null) {
+            purgeMgr.shutdown();
+        }
+    }
+}