Author: phunt
Date: Sat Jan 31 01:18:30 2009
New Revision: 739479
URL: http://svn.apache.org/viewvc?rev=739479&view=rev
Log:
ZOOKEEPER-252. PurgeTxnLog is not handling the new dataDir directory structure
Added:
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/PurgeTxnTest.java (with
props)
Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PurgeTxnLog.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileSnap.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=739479&r1=739478&r2=739479&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Sat Jan 31 01:18:30 2009
@@ -81,6 +81,9 @@
ZOOKEEPER-16. Need to do path validation. (pat, mahadev)
+ ZOOKEEPER-252. PurgeTxnLog is not handling the new dataDir directory
+ structure (mahadev via phunt)
+
IMPROVEMENTS:
ZOOKEEPER-64. Log system env information when initializing server and
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PurgeTxnLog.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PurgeTxnLog.java?rev=739479&r1=739478&r2=739479&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PurgeTxnLog.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PurgeTxnLog.java Sat
Jan 31 01:18:30 2009
@@ -28,38 +28,55 @@
import java.util.List;
import java.util.Set;
+import org.apache.log4j.Logger;
import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
import org.apache.zookeeper.server.persistence.Util;
+/**
+ * this class is used to clean up the
+ * snapshot and data log dir's. This is usually
+ * run as a cronjob on the zookeeper server machine.
+ * Invocation of this class will clean up the datalogdir
+ * files and snapdir files keeping the last "-n" snapshot files
+ * and the corresponding logs.
+ */
public class PurgeTxnLog {
+ private static final Logger LOG = Logger.getLogger(PurgeTxnLog.class);
static void printUsage(){
- System.out.println("PurgeTxnLog dataLogDir [snapDir]");
+ System.out.println("PurgeTxnLog dataLogDir [snapDir] -n count");
System.out.println("\tdataLogDir -- path to the txn log directory");
System.out.println("\tsnapDir -- path to the snapshot directory");
+ System.out.println("\tcount -- the number of old snaps/logs you want to keep");
System.exit(1);
}
+
/**
- * @param args PurgeTxnLog dataLogDir
- * dataLogDir -- txn log directory
+ * purges the snapshot and logs keeping the last num snapshots
+ * and the corresponding logs.
+ * @param dataDir the dir that has the logs
+ * @param snapDir the dir that has the snapshots
+ * @param num the number of snapshots to keep
+ * @throws IOException
*/
- public static void main(String[] args) throws IOException {
- if(args.length<1 || args.length>2)
- printUsage();
+ public static void purge(File dataDir, File snapDir, int num) throws IOException {
+ if (num < 3) {
+ throw new IllegalArgumentException("count should be greater than 3");
+ }
- File dataDir=new File(args[0]);
- File snapDir=dataDir;
- if(args.length==2){
- snapDir=new File(args[1]);
- }
FileTxnSnapLog txnLog = new FileTxnSnapLog(dataDir, snapDir);
// found any valid recent snapshots?
// files to exclude from deletion
Set<File> exc=new HashSet<File>();
- File snapShot = txnLog.findMostRecentSnapshot();
- exc.add(txnLog.findMostRecentSnapshot());
+ List<File> snaps = txnLog.findNRecentSnapshots(num);
+ if (snaps.size() == 0)
+ return;
+ File snapShot = snaps.get(snaps.size() -1);
+ for (File f: snaps) {
+ exc.add(f);
+ }
long zxid = Util.getZxidFromName(snapShot.getName(),"snapshot");
exc.addAll(Arrays.asList(txnLog.getSnapshotLogs(zxid)));
@@ -77,9 +94,9 @@
}
// add all non-excluded log files
List<File> files=new ArrayList<File>(
- Arrays.asList(dataDir.listFiles(new MyFileFilter("log."))));
+ Arrays.asList(txnLog.getDataDir().listFiles(new MyFileFilter("log."))));
// add all non-excluded snapshot files to the deletion list
- files.addAll(Arrays.asList(snapDir.listFiles(new MyFileFilter("snapshot."))));
+ files.addAll(Arrays.asList(txnLog.getSnapDir().listFiles(new MyFileFilter("snapshot."))));
// remove the old files
for(File f: files)
{
@@ -90,5 +107,26 @@
System.err.println("Failed to remove "+f.getPath());
}
}
+
+ }
+
+ /**
+ * @param args PurgeTxnLog dataLogDir
+ * dataLogDir -- txn log directory
+ * -n num (number of snapshots to keep)
+ */
+ public static void main(String[] args) throws IOException {
+ if(args.length<3 || args.length>4)
+ printUsage();
+ int i = 0;
+ File dataDir=new File(args[0]);
+ File snapDir=dataDir;
+ if(args.length==4){
+ i++;
+ snapDir=new File(args[i]);
+ }
+ i++; i++;
+ int num = Integer.parseInt(args[i]);
+ purge(dataDir, snapDir, num);
}
}
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileSnap.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileSnap.java?rev=739479&r1=739478&r2=739479&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileSnap.java
(original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileSnap.java
Sat Jan 31 01:18:30 2009
@@ -26,6 +26,7 @@
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
+import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.zip.Adler32;
@@ -121,6 +122,25 @@
}
return null;
}
+
+ /**
+ * find the last n snapshots.
+ * @param the number of most recent snapshots
+ * @return the last n snapshots
+ * @throws IOException
+ */
+ public List<File> findNRecentSnapshots(int n) throws IOException {
+ List<File> files = Util.sortDataDir(snapDir.listFiles(), "snapshot", false);
+ int i = 0;
+ List<File> list = new ArrayList<File>();
+ for (File f: files) {
+ if (i==n)
+ break;
+ i++;
+ list.add(f);
+ }
+ return list;
+ }
/**
* serialize the datatree and sessions
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java?rev=739479&r1=739478&r2=739479&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
(original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
Sat Jan 31 01:18:30 2009
@@ -20,6 +20,7 @@
import java.io.File;
import java.io.IOException;
+import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
@@ -224,6 +225,18 @@
FileSnap snaplog = new FileSnap(snapDir);
return snaplog.findMostRecentSnapshot();
}
+
+ /**
+ * the n most recent snapshots
+ * @param n the number of recent snapshots
+ * @return the list of n most recent snapshots, with
+ * the most recent in front
+ * @throws IOException
+ */
+ public List<File> findNRecentSnapshots(int n) throws IOException {
+ FileSnap snaplog = new FileSnap(snapDir);
+ return snaplog.findNRecentSnapshots(n);
+ }
/**
* get the snapshot logs that are greater than
Added: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/PurgeTxnTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/PurgeTxnTest.java?rev=739479&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/PurgeTxnTest.java (added)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/PurgeTxnTest.java Sat Jan
31 01:18:30 2009
@@ -0,0 +1,90 @@
+/**
+ * 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.test;
+
+import java.io.File;
+import java.io.RandomAccessFile;
+import java.util.ArrayList;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.PurgeTxnLog;
+import org.apache.zookeeper.server.SyncRequestProcessor;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.test.ClientBase;
+
+/**
+ * test the purging of the logs
+ * and purging of the snapshots.
+ */
+public class PurgeTxnTest extends TestCase implements Watcher {
+ private static final Logger LOG = Logger.getLogger(PurgeTxnTest.class);
+ private static String HOSTPORT = "127.0.0.1:2357";
+ ZooKeeperServer zks = null;
+ private static final int CONNECTION_TIMEOUT = 3000;
+ /**
+ * test the purge
+ * @throws Exception
+ */
+ public void testPurge() throws Exception {
+ File tmpDir = ClientBase.createTmpDir();
+ ClientBase.setupTestEnv();
+ zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+ SyncRequestProcessor.snapCount = 100;
+ final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
+ NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
+ f.startup(zks);
+ assertTrue("waiting for server being up ",
+ ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
+ ZooKeeper zk = new ZooKeeper(HOSTPORT, 20000, this);
+ for (int i=0; i< 2000; i++) {
+ zk.create("/invalidsnap-" + i, new byte[0], Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+ }
+ f.shutdown();
+ assertTrue("waiting for server to shutdown",
+ ClientBase.waitForServerDown(HOSTPORT, CONNECTION_TIMEOUT));
+ // now corrupt the snapshot
+ PurgeTxnLog.purge(tmpDir, tmpDir, 3);
+ FileTxnSnapLog snaplog = new FileTxnSnapLog(tmpDir, tmpDir);
+ List<File> listLogs = snaplog.findNRecentSnapshots(4);
+ int numSnaps = 0;
+ for (File ff: listLogs) {
+ if (ff.getName().startsWith("snapshot")) {
+ numSnaps++;
+ }
+ }
+ assertTrue("exactly 3 snapshots ", (numSnaps == 3));
+ }
+
+ @Override
+ public void process(WatchedEvent event) {
+ // do nothing
+ }
+
+}
\ No newline at end of file
Propchange: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/PurgeTxnTest.java
------------------------------------------------------------------------------
svn:eol-style = native
|