You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by sy...@apache.org on 2022/05/16 16:34:18 UTC

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

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

symat pushed a commit to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/zookeeper.git


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

commit 22d0c856aeb1eb19da2810dacb5585c055365ed6
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
    
    (cherry picked from commit ea8536982c39fd1127a4e6e7a8229b64a5798447)
---
 .../apache/zookeeper/server/admin/Commands.java    | 26 +++++++++++++++++
 .../zookeeper/server/persistence/FileSnap.java     | 19 +++++++++++--
 .../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, 97 insertions(+), 4 deletions(-)

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 108c0f2c9..7f338091c 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
@@ -36,6 +36,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.Leader;
 import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer;
 import org.apache.zookeeper.server.quorum.ReadOnlyZooKeeperServer;
@@ -119,6 +120,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());
@@ -285,6 +287,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
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 9d9a3e2f2..1b083ffe6 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
@@ -51,6 +51,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;
@@ -64,6 +65,14 @@ public class FileSnap implements SnapShot {
         this.snapDir = snapDir;
     }
 
+    /**
+     * 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
@@ -101,6 +110,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;
     }
 
@@ -136,12 +146,12 @@ public class FileSnap implements SnapShot {
     }
 
     /**
-     * find the last (maybe) valid n snapshots. this does some 
+     * find the last (maybe) valid n snapshots. this does some
      * minor checks on the validity of the snapshots. It just
      * checks for / at the end of the snapshot. This does
      * not mean that the snapshot is truly valid but is
-     * valid with a high probability. also, the most recent 
-     * will be first on the list. 
+     * valid with a high probability. also, the most recent
+     * will be first on the list.
      * @param n the number of most recent snapshots
      * @return the last n snapshots (the number might be
      * less than n in case enough snapshots are not available).
@@ -233,6 +243,9 @@ public class FileSnap implements SnapShot {
                 oa.writeLong(val, "val");
                 oa.writeString("/", "path");
                 crcOut.flush();
+                lastSnapshotInfo = new SnapshotInfo(
+                        Util.getZxidFromName(snapShot.getName(), SNAPSHOT_FILE_PREFIX),
+                        snapShot.lastModified() / 1000);
             }
         } else {
             throw new IOException("FileSnap has already been closed");
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 4881e2844..0267eb480 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
@@ -209,6 +209,14 @@ public class FileTxnSnapLog {
         return this.snapDir;
     }
 
+    /**
+     * 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
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 257c12d95..534e3d37f 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 000000000..87e164157
--- /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 bf1fa6929..321d15693 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
@@ -165,6 +165,13 @@ public class CommandsTest extends ClientBase {
                     new Field("read_only", Boolean.class));
     }
 
+    @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 {
         testCommand("monitor",