You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by nk...@apache.org on 2019/05/27 08:55:27 UTC

[zookeeper] branch master updated: ZOOKEEPER-3392: Add admin command to display last snapshot information

This is an automated email from the ASF dual-hosted git repository.

nkalmar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/zookeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new ea85369  ZOOKEEPER-3392: Add admin command to display last snapshot information
ea85369 is described below

commit ea8536982c39fd1127a4e6e7a8229b64a5798447
Author: Brian Nixon <ni...@fb.com>
AuthorDate: Mon May 27 10:54:58 2019 +0200

    ZOOKEEPER-3392: Add admin command to display last snapshot information
    
    Author: Brian Nixon <ni...@fb.com>
    
    Reviewers: Enrico Olivelli <eo...@apache.org>, Norbert Kalmar <nk...@apache.org>
    
    Closes #947 from enixon/cmd-last-snap
---
 .../apache/zookeeper/server/admin/Commands.java    | 26 +++++++++++++++++
 .../zookeeper/server/persistence/FileSnap.java     | 13 +++++++++
 .../server/persistence/FileTxnSnapLog.java         |  8 ++++++
 .../zookeeper/server/persistence/SnapShot.java     |  8 +++++-
 .../zookeeper/server/persistence/SnapshotInfo.java | 33 ++++++++++++++++++++++
 .../zookeeper/server/admin/CommandsTest.java       |  7 +++++
 6 files changed, 94 insertions(+), 1 deletion(-)

diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java
index 72dfe7f..58f48e7 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java
@@ -39,6 +39,7 @@ import org.apache.zookeeper.server.ServerStats;
 import org.apache.zookeeper.server.ZKDatabase;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooTrace;
+import org.apache.zookeeper.server.persistence.SnapshotInfo;
 import org.apache.zookeeper.server.quorum.Follower;
 import org.apache.zookeeper.server.quorum.FollowerZooKeeperServer;
 import org.apache.zookeeper.server.quorum.Leader;
@@ -128,6 +129,7 @@ public class Commands {
         registerCommand(new EnvCommand());
         registerCommand(new GetTraceMaskCommand());
         registerCommand(new IsroCommand());
+        registerCommand(new LastSnapshotCommand());
         registerCommand(new MonitorCommand());
         registerCommand(new RuokCommand());
         registerCommand(new SetTraceMaskCommand());
@@ -297,6 +299,30 @@ public class Commands {
     }
 
     /**
+     * Command returns information of the last snapshot that zookeeper server
+     * has finished saving to disk. During the time between the server starts up
+     * and it finishes saving its first snapshot, the command returns the zxid
+     * and last modified time of the snapshot file used for restoration at
+     * server startup. Returned map contains:
+     *   - "zxid": String
+     *   - "timestamp": Long
+     */
+    public static class LastSnapshotCommand extends CommandBase {
+        public LastSnapshotCommand() {
+            super(Arrays.asList("last_snapshot", "lsnp"));
+        }
+
+        @Override
+        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
+            CommandResponse response = initializeResponse();
+            SnapshotInfo info = zkServer.getTxnLogFactory().getLastSnapshotInfo();
+            response.put("zxid", Long.toHexString(info == null ? -1L : info.zxid));
+            response.put("timestamp", info == null ? -1L : info.timestamp);
+            return response;
+        }
+    }
+
+    /**
      * Some useful info for monitoring. Returned map contains:
      *   - "version": String
      *                server version
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java
index 3b9d030..fc6af85 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java
@@ -46,6 +46,7 @@ import org.apache.zookeeper.server.util.SerializeUtils;
  */
 public class FileSnap implements SnapShot {
     File snapDir;
+    SnapshotInfo lastSnapshotInfo = null;
     private volatile boolean close = false;
     private static final int VERSION = 2;
     private static final long dbId = -1;
@@ -60,6 +61,14 @@ public class FileSnap implements SnapShot {
     }
 
     /**
+     * get information of the last saved/restored snapshot
+     * @return info of last snapshot
+     */
+    public SnapshotInfo getLastSnapshotInfo() {
+        return this.lastSnapshotInfo;
+    }
+
+    /**
      * deserialize a data tree from the most recent snapshot
      * @return the zxid of the snapshot
      */
@@ -91,6 +100,7 @@ public class FileSnap implements SnapShot {
             throw new IOException("Not able to find valid snapshots in " + snapDir);
         }
         dt.lastProcessedZxid = Util.getZxidFromName(snap.getName(), SNAPSHOT_FILE_PREFIX);
+        lastSnapshotInfo = new SnapshotInfo(dt.lastProcessedZxid, snap.lastModified() / 1000);
         return dt.lastProcessedZxid;
     }
 
@@ -216,6 +226,9 @@ public class FileSnap implements SnapShot {
                 FileHeader header = new FileHeader(SNAP_MAGIC, VERSION, dbId);
                 serialize(dt, sessions, oa, header);
                 SnapStream.sealStream(snapOS, oa);
+                lastSnapshotInfo = new SnapshotInfo(
+                        Util.getZxidFromName(snapShot.getName(), SNAPSHOT_FILE_PREFIX),
+                        snapShot.lastModified() / 1000);
             }
         }
     }
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
index b0df373..26fe4f2 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
@@ -201,6 +201,14 @@ public class FileTxnSnapLog {
     }
 
     /**
+     * get information of the last saved/restored snapshot
+     * @return info of last snapshot
+     */
+    public SnapshotInfo getLastSnapshotInfo() {
+        return this.snapLog.getLastSnapshotInfo();
+    }
+
+    /**
      * this function restores the server
      * database after reading from the
      * snapshots and transaction logs
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/SnapShot.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/SnapShot.java
index 257c12d..534e3d3 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/SnapShot.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/SnapShot.java
@@ -59,7 +59,13 @@ public interface SnapShot {
      * @throws IOException
      */
     File findMostRecentSnapshot() throws IOException;
-    
+
+    /**
+     * get information of the last saved/restored snapshot
+     * @return info of last snapshot
+     */
+    SnapshotInfo getLastSnapshotInfo();
+
     /**
      * free resources from this snapshot immediately
      * @throws IOException
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/SnapshotInfo.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/SnapshotInfo.java
new file mode 100644
index 0000000..87e1641
--- /dev/null
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/SnapshotInfo.java
@@ -0,0 +1,33 @@
+/**
+ * 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.persistence;
+
+/**
+ * stores the zxid (as in its file name) and the last modified timestamp
+ * of a snapshot file
+ */
+public class SnapshotInfo {
+    public long zxid;
+    public long timestamp;
+
+    SnapshotInfo(long zxid, long timestamp) {
+        this.zxid = zxid;
+        this.timestamp = timestamp;
+    }
+}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/CommandsTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/CommandsTest.java
index 3293a7d..e4ab05e 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/CommandsTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/CommandsTest.java
@@ -170,6 +170,13 @@ public class CommandsTest extends ClientBase {
     }
 
     @Test
+    public void testLastSnapshot() throws IOException, InterruptedException {
+        testCommand("last_snapshot",
+                    new Field("zxid", String.class),
+                    new Field("timestamp", Long.class));
+    }
+
+    @Test
     public void testMonitor() throws IOException, InterruptedException {
         ArrayList<Field> fields = new ArrayList<>(Arrays.asList(
                 new Field("version", String.class),