|
@@ -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() {}
|
|
|
}
|