Переглянути джерело

ZOOKEEPER-3353: Admin commands for showing initial settings

Author: Brian Nixon <nixon@fb.com>

Reviewers: eolivelli@apache.org, andor@apache.org

Closes #898 from enixon/properties-view and squashes the following commits:

03eccb902 [Brian Nixon] move parameter determining whether a Command needs a ZooKeeperServer into the CommandBase constructor
f1eedd385 [Brian Nixon] remove four letter word version of commands
4e73b931e [Brian Nixon] ZOOKEEPER-3353: Admin commands for showing initial settings
Brian Nixon 6 роки тому
батько
коміт
e45551fc7c

+ 2 - 0
zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerConfig.java

@@ -54,6 +54,7 @@ public class ServerConfig {
     protected Properties metricsProviderConfiguration = new Properties();
     /** defaults to -1 if not set explicitly */
     protected int listenBacklog = -1;
+    protected String initialConfig;
 
     /** JVM Pause Monitor feature switch */
     protected boolean jvmPauseMonitorToRun = false;
@@ -121,6 +122,7 @@ public class ServerConfig {
         metricsProviderClassName = config.getMetricsProviderClassName();
         metricsProviderConfiguration = config.getMetricsProviderConfiguration();
         listenBacklog = config.getClientPortListenBacklog();
+        initialConfig = config.getInitialConfig();
     }
 
     public InetSocketAddress getClientPortAddress() {

+ 17 - 8
zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java

@@ -116,6 +116,8 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
     protected JvmPauseMonitor jvmPauseMonitor;
     protected volatile State state = State.INITIAL;
     private boolean isResponseCachingEnabled = true;
+    /* contains the configuration file content read at startup */
+    protected String initialConfig;
 
     protected enum State {
         INITIAL, RUNNING, SHUTDOWN, ERROR
@@ -194,7 +196,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
      */
     public ZooKeeperServer(FileTxnSnapLog txnLogFactory, int tickTime,
             int minSessionTimeout, int maxSessionTimeout, int clientPortListenBacklog,
-            ZKDatabase zkDb) {
+            ZKDatabase zkDb, String initialConfig) {
         serverStats = new ServerStats(this);
         this.txnLogFactory = txnLogFactory;
         this.txnLogFactory.setServerStats(this.serverStats);
@@ -210,6 +212,8 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
 
         connThrottle = new BlueThrottle();
 
+        this.initialConfig = initialConfig;
+
         LOG.info("Created server with tickTime " + tickTime
                 + " minSessionTimeout " + getMinSessionTimeout()
                 + " maxSessionTimeout " + getMaxSessionTimeout()
@@ -218,14 +222,19 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
                 + " snapdir " + txnLogFactory.getSnapDir());
     }
 
+    public String getInitialConfig() {
+        return initialConfig;
+    }
+
     /**
      * Adds JvmPauseMonitor and calls
-     * {@link #ZooKeeperServer(FileTxnSnapLog, int, int, int, int, ZKDatabase)}
+     * {@link #ZooKeeperServer(FileTxnSnapLog, int, int, int, int, ZKDatabase, String)}
      *
      */
     public ZooKeeperServer(JvmPauseMonitor jvmPauseMonitor, FileTxnSnapLog txnLogFactory, int tickTime,
-                           int minSessionTimeout, int maxSessionTimeout, int clientPortListenBacklog, ZKDatabase zkDb) {
-        this(txnLogFactory, tickTime, minSessionTimeout, maxSessionTimeout, clientPortListenBacklog, zkDb);
+                           int minSessionTimeout, int maxSessionTimeout, int clientPortListenBacklog,
+                           ZKDatabase zkDb, String initialConfig) {
+        this(txnLogFactory, tickTime, minSessionTimeout, maxSessionTimeout, clientPortListenBacklog, zkDb, initialConfig);
         this.jvmPauseMonitor = jvmPauseMonitor;
         if(jvmPauseMonitor != null) {
             LOG.info("Added JvmPauseMonitor to server");
@@ -238,9 +247,9 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
      * @param tickTime the ticktime for the server
      * @throws IOException
      */
-    public ZooKeeperServer(FileTxnSnapLog txnLogFactory, int tickTime)
+    public ZooKeeperServer(FileTxnSnapLog txnLogFactory, int tickTime, String initialConfig)
             throws IOException {
-        this(txnLogFactory, tickTime, -1, -1, -1, new ZKDatabase(txnLogFactory));
+        this(txnLogFactory, tickTime, -1, -1, -1, new ZKDatabase(txnLogFactory), initialConfig);
     }
 
     public ServerStats serverStats() {
@@ -300,7 +309,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
     public ZooKeeperServer(File snapDir, File logDir, int tickTime)
             throws IOException {
         this( new FileTxnSnapLog(snapDir, logDir),
-                tickTime);
+                tickTime, "");
     }
 
     /**
@@ -311,7 +320,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
     public ZooKeeperServer(FileTxnSnapLog txnLogFactory)
         throws IOException
     {
-        this(txnLogFactory, DEFAULT_TICK_TIME, -1, -1, -1, new ZKDatabase(txnLogFactory));
+        this(txnLogFactory, DEFAULT_TICK_TIME, -1, -1, -1, new ZKDatabase(txnLogFactory), "");
     }
 
     /**

+ 1 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMain.java

@@ -142,7 +142,7 @@ public class ZooKeeperServerMain {
             }
             final ZooKeeperServer zkServer = new ZooKeeperServer(jvmPauseMonitor, txnLog,
                     config.tickTime, config.minSessionTimeout, config.maxSessionTimeout,
-                    config.listenBacklog, null);
+                    config.listenBacklog, null, config.initialConfig);
             txnLog.setServerStats(zkServer.serverStats());
 
             // Registers shutdown handler which will be used to know the

+ 6 - 0
zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Command.java

@@ -50,6 +50,12 @@ public interface Command {
      */
     String getDoc();
 
+    /**
+     * @return true if the command requires an active ZooKeeperServer or a
+     *     synced peer in order to resolve
+     */
+    boolean isServerRequired();
+
     /**
      * Run this command. Commands take a ZooKeeperServer and String-valued
      * keyword arguments and return a map containing any information

+ 12 - 2
zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/CommandBase.java

@@ -26,18 +26,23 @@ public abstract class CommandBase implements Command {
     private final String primaryName;
     private final Set<String> names;
     private final String doc;
+    private final boolean serverRequired;
 
     /**
      * @param names The possible names of this command, with the primary name first.
      */
     protected CommandBase(List<String> names) {
-        this(names, null);
+        this(names, true, null);
+    }
+    protected CommandBase(List<String> names, boolean serverRequired) {
+        this(names, serverRequired, null);
     }
 
-    protected CommandBase(List<String> names, String doc) {
+    protected CommandBase(List<String> names, boolean serverRequired, String doc) {
         this.primaryName = names.get(0);
         this.names = new HashSet<String>(names);
         this.doc = doc;
+        this.serverRequired = serverRequired;
     }
 
     @Override
@@ -55,6 +60,11 @@ public abstract class CommandBase implements Command {
         return doc;
     }
 
+    @Override
+    public boolean isServerRequired() {
+        return serverRequired;
+    }
+
     /**
      * @return A response with the command set to the primary name and the
      *         error set to null (these are the two entries that all command

+ 99 - 60
zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java

@@ -18,14 +18,16 @@
 
 package org.apache.zookeeper.server.admin;
 
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
 
 import org.apache.zookeeper.Environment;
 import org.apache.zookeeper.Environment.Entry;
@@ -92,13 +94,14 @@ public class Commands {
      *    - "error" key containing a String error message or null if no error
      */
     public static CommandResponse runCommand(String cmdName, ZooKeeperServer zkServer, Map<String, String> kwargs) {
-        if (!commands.containsKey(cmdName)) {
+        Command command = getCommand(cmdName);
+        if (command == null) {
             return new CommandResponse(cmdName, "Unknown command: " + cmdName);
         }
-        if (zkServer == null || !zkServer.isRunning()) {
+        if (command.isServerRequired() && (zkServer == null || !zkServer.isRunning())) {
             return new CommandResponse(cmdName, "This ZooKeeper instance is not currently serving requests");
         }
-        return commands.get(cmdName).run(zkServer, kwargs);
+        return command.run(zkServer, kwargs);
     }
 
     /**
@@ -134,6 +137,8 @@ public class Commands {
         registerCommand(new WatchCommand());
         registerCommand(new WatchesByPathCommand());
         registerCommand(new WatchSummaryCommand());
+        registerCommand(new SystemPropertiesCommand());
+        registerCommand(new InitialConfigurationCommand());
     }
 
     /**
@@ -244,7 +249,7 @@ public class Commands {
      */
     public static class EnvCommand extends CommandBase {
         public EnvCommand() {
-            super(Arrays.asList("environment", "env", "envi"));
+            super(Arrays.asList("environment", "env", "envi"), false);
         }
 
         @Override
@@ -263,7 +268,7 @@ public class Commands {
      */
     public static class GetTraceMaskCommand extends CommandBase {
         public GetTraceMaskCommand() {
-            super(Arrays.asList("get_trace_mask", "gtmk"));
+            super(Arrays.asList("get_trace_mask", "gtmk"), false);
         }
 
         @Override
@@ -317,79 +322,81 @@ public class Commands {
      */
     public static class MonitorCommand extends CommandBase {
         public MonitorCommand() {
-            super(Arrays.asList("monitor", "mntr"));
+            super(Arrays.asList("monitor", "mntr"), false);
         }
 
         @Override
         public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
-            ZKDatabase zkdb = zkServer.getZKDatabase();
-            ServerStats stats = zkServer.serverStats();
-
             CommandResponse response = initializeResponse();
 
             response.put("version", Version.getFullVersion());
 
-            response.put("avg_latency", stats.getAvgLatency());
-            response.put("max_latency", stats.getMaxLatency());
-            response.put("min_latency", stats.getMinLatency());
+            OSMXBean osMbean = new OSMXBean();
+            response.put("open_file_descriptor_count", osMbean.getOpenFileDescriptorCount());
+            response.put("max_file_descriptor_count", osMbean.getMaxFileDescriptorCount());
 
-            response.put("packets_received", stats.getPacketsReceived());
-            response.put("packets_sent", stats.getPacketsSent());
-            response.put("num_alive_connections", stats.getNumAliveClientConnections());
+            if (zkServer != null) {
+                ZKDatabase zkdb = zkServer.getZKDatabase();
+                ServerStats stats = zkServer.serverStats();
+                response.put("avg_latency", stats.getAvgLatency());
+                response.put("max_latency", stats.getMaxLatency());
+                response.put("min_latency", stats.getMinLatency());
 
-            response.put("outstanding_requests", stats.getOutstandingRequests());
-            response.put("uptime", stats.getUptime());
+                response.put("packets_received", stats.getPacketsReceived());
+                response.put("packets_sent", stats.getPacketsSent());
+                response.put("num_alive_connections", stats.getNumAliveClientConnections());
 
-            response.put("server_state", stats.getServerState());
-            response.put("znode_count", zkdb.getNodeCount());
+                response.put("outstanding_requests", stats.getOutstandingRequests());
+                response.put("uptime", stats.getUptime());
 
-            response.put("watch_count", zkdb.getDataTree().getWatchCount());
-            response.put("ephemerals_count", zkdb.getDataTree().getEphemeralsCount());
-            response.put("approximate_data_size", zkdb.getDataTree().cachedApproximateDataSize());
+                response.put("server_state", stats.getServerState());
+                response.put("znode_count", zkdb.getNodeCount());
 
-            response.put("global_sessions", zkdb.getSessionCount());
-            response.put("local_sessions",
-                    zkServer.getSessionTracker().getLocalSessionCount());
+                response.put("watch_count", zkdb.getDataTree().getWatchCount());
+                response.put("ephemerals_count", zkdb.getDataTree().getEphemeralsCount());
+                response.put("approximate_data_size", zkdb.getDataTree().cachedApproximateDataSize());
 
-            OSMXBean osMbean = new OSMXBean();
-            response.put("open_file_descriptor_count", osMbean.getOpenFileDescriptorCount());
-            response.put("max_file_descriptor_count", osMbean.getMaxFileDescriptorCount());
-            response.put("connection_drop_probability", zkServer.getConnectionDropChance());
+                response.put("global_sessions", zkdb.getSessionCount());
+                response.put("local_sessions",
+                        zkServer.getSessionTracker().getLocalSessionCount());
 
-            response.put("last_client_response_size", stats.getClientResponseStats().getLastBufferSize());
-            response.put("max_client_response_size", stats.getClientResponseStats().getMaxBufferSize());
-            response.put("min_client_response_size", stats.getClientResponseStats().getMinBufferSize());
+                response.put("connection_drop_probability", zkServer.getConnectionDropChance());
 
-            if (zkServer instanceof QuorumZooKeeperServer) {
-                QuorumPeer peer = ((QuorumZooKeeperServer) zkServer).self;
-                response.put("quorum_size", peer.getQuorumSize());
-            }
+                response.put("last_client_response_size", stats.getClientResponseStats().getLastBufferSize());
+                response.put("max_client_response_size", stats.getClientResponseStats().getMaxBufferSize());
+                response.put("min_client_response_size", stats.getClientResponseStats().getMinBufferSize());
 
-            if (zkServer instanceof LeaderZooKeeperServer) {
-                Leader leader = ((LeaderZooKeeperServer) zkServer).getLeader();
-
-                response.put("learners", leader.getLearners().size());
-                response.put("synced_followers", leader.getForwardingFollowers().size());
-                response.put("synced_non_voting_followers", leader.getNonVotingFollowers().size());
-                response.put("synced_observers", leader.getObservingLearners().size());
-                response.put("pending_syncs", leader.getNumPendingSyncs());
-                response.put("leader_uptime", leader.getUptime());
-
-                response.put("last_proposal_size", leader.getProposalStats().getLastBufferSize());
-                response.put("max_proposal_size", leader.getProposalStats().getMaxBufferSize());
-                response.put("min_proposal_size", leader.getProposalStats().getMinBufferSize());
-            }
+                if (zkServer instanceof QuorumZooKeeperServer) {
+                    QuorumPeer peer = ((QuorumZooKeeperServer) zkServer).self;
+                    response.put("quorum_size", peer.getQuorumSize());
+                }
+
+                if (zkServer instanceof LeaderZooKeeperServer) {
+                    Leader leader = ((LeaderZooKeeperServer) zkServer).getLeader();
+
+                    response.put("learners", leader.getLearners().size());
+                    response.put("synced_followers", leader.getForwardingFollowers().size());
+                    response.put("synced_non_voting_followers", leader.getNonVotingFollowers().size());
+                    response.put("synced_observers", leader.getObservingLearners().size());
+                    response.put("pending_syncs", leader.getNumPendingSyncs());
+                    response.put("leader_uptime", leader.getUptime());
 
-            if (zkServer instanceof FollowerZooKeeperServer) {
-                Follower follower = ((FollowerZooKeeperServer) zkServer).getFollower();
-                Integer syncedObservers = follower.getSyncedObserverSize();
-                if (syncedObservers != null) {
-                    response.put("synced_observers", syncedObservers);
+                    response.put("last_proposal_size", leader.getProposalStats().getLastBufferSize());
+                    response.put("max_proposal_size", leader.getProposalStats().getMaxBufferSize());
+                    response.put("min_proposal_size", leader.getProposalStats().getMinBufferSize());
                 }
-            }
 
-            if (zkServer instanceof ObserverZooKeeperServer) {
-                response.put("observer_master_id", ((ObserverZooKeeperServer)zkServer).getObserver().getLearnerMasterId());
+                if (zkServer instanceof FollowerZooKeeperServer) {
+                    Follower follower = ((FollowerZooKeeperServer) zkServer).getFollower();
+                    Integer syncedObservers = follower.getSyncedObserverSize();
+                    if (syncedObservers != null) {
+                        response.put("synced_observers", syncedObservers);
+                    }
+                }
+
+                if (zkServer instanceof ObserverZooKeeperServer) {
+                    response.put("observer_master_id", ((ObserverZooKeeperServer) zkServer).getObserver().getLearnerMasterId());
+                }
             }
 
             ServerMetrics.getMetrics()
@@ -424,7 +431,7 @@ public class Commands {
      */
     public static class SetTraceMaskCommand extends CommandBase {
         public SetTraceMaskCommand() {
-            super(Arrays.asList("set_trace_mask", "stmk"));
+            super(Arrays.asList("set_trace_mask", "stmk"), false);
         }
 
         @Override
@@ -573,5 +580,37 @@ public class Commands {
         }
     }
 
+    /**
+     * All defined system properties.
+     */
+    public static class SystemPropertiesCommand extends CommandBase {
+        public SystemPropertiesCommand() {
+            super(Arrays.asList("system_properties", "sysp"), false);
+        }
+
+        @Override
+        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
+            CommandResponse response = initializeResponse();
+            Properties systemProperties = System.getProperties();
+            SortedMap<String, String> sortedSystemProperties = new TreeMap<>();
+            systemProperties.forEach((k, v) -> sortedSystemProperties.put(k.toString(), v.toString()));
+            response.putAll(sortedSystemProperties);
+            return response;
+        }
+    }
+
+    public static class InitialConfigurationCommand extends CommandBase {
+        public InitialConfigurationCommand() {
+            super(Arrays.asList("initial_configuration", "icfg"));
+        }
+
+        @Override
+        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
+            CommandResponse response = initializeResponse();
+            response.put("initial_configuration", zkServer.getInitialConfig());
+            return response;
+        }
+    }
+
     private Commands() {}
 }

+ 10 - 0
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java

@@ -535,6 +535,8 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
 
     private volatile boolean running = true;
 
+    private String initialConfig;
+
     /**
      * The number of milliseconds of each tick
      */
@@ -1461,6 +1463,14 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
         this.myid = myid;
     }
 
+    public void setInitialConfig(String initialConfig) {
+        this.initialConfig = initialConfig;
+    }
+
+    public String getInitialConfig() {
+        return initialConfig;
+    }
+
     /**
      * Get the number of milliseconds of each tick
      */

+ 12 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java

@@ -29,6 +29,7 @@ import java.io.StringReader;
 import java.io.Writer;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
+import java.nio.file.Files;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -104,6 +105,8 @@ public class QuorumPeerConfig {
     protected int purgeInterval = 0;
     protected boolean syncEnabled = true;
 
+    protected String initialConfig;
+
     protected LearnerType peerType = LearnerType.PARTICIPANT;
 
     /**
@@ -172,7 +175,10 @@ public class QuorumPeerConfig {
             } finally {
                 in.close();
             }
-            
+
+            /* Read entire config file as initial configuration */
+            initialConfig = new String(Files.readAllBytes(configFile.toPath()));
+
             parseProperties(cfg);
         } catch (IOException e) {
             throw new ConfigException("Error processing " + path, e);
@@ -802,6 +808,11 @@ public class QuorumPeerConfig {
     public int getObserverMasterPort() { return observerMasterPort; }
     public File getDataDir() { return dataDir; }
     public File getDataLogDir() { return dataLogDir; }
+
+    public String getInitialConfig() {
+        return initialConfig;
+    }
+
     public int getTickTime() { return tickTime; }
     public int getMaxClientCnxns() { return maxClientCnxns; }
     public int getMinSessionTimeout() { return minSessionTimeout; }

+ 1 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java

@@ -46,7 +46,7 @@ public abstract class QuorumZooKeeperServer extends ZooKeeperServer {
             int minSessionTimeout, int maxSessionTimeout, int listenBacklog,
             ZKDatabase zkDb, QuorumPeer self)
     {
-        super(logFactory, tickTime, minSessionTimeout, maxSessionTimeout, listenBacklog, zkDb);
+        super(logFactory, tickTime, minSessionTimeout, maxSessionTimeout, listenBacklog, zkDb, self.getInitialConfig());
         this.self = self;
     }
 

+ 1 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java

@@ -46,7 +46,7 @@ public class ReadOnlyZooKeeperServer extends ZooKeeperServer {
     ReadOnlyZooKeeperServer(FileTxnSnapLog logFactory, QuorumPeer self,
                             ZKDatabase zkDb) {
         super(logFactory, self.tickTime, self.minSessionTimeout,
-              self.maxSessionTimeout, self.clientPortListenBacklog, zkDb);
+              self.maxSessionTimeout, self.clientPortListenBacklog, zkDb, self.getInitialConfig());
         this.self = self;
     }