summaryrefslogtreecommitdiff
path: root/zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java
blob: 236c7ec24525dd16643a3dc99450b27d0b5008bf (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
/*
 * 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.admin;

import com.fasterxml.jackson.annotation.JsonProperty;
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
import java.net.InetSocketAddress;
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 java.util.stream.Collectors;
import org.apache.zookeeper.Environment;
import org.apache.zookeeper.Environment.Entry;
import org.apache.zookeeper.Version;
import org.apache.zookeeper.server.DataTree;
import org.apache.zookeeper.server.ServerCnxnFactory;
import org.apache.zookeeper.server.ServerMetrics;
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;
import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer;
import org.apache.zookeeper.server.quorum.MultipleAddresses;
import org.apache.zookeeper.server.quorum.QuorumPeer;
import org.apache.zookeeper.server.quorum.QuorumPeer.LearnerType;
import org.apache.zookeeper.server.quorum.QuorumZooKeeperServer;
import org.apache.zookeeper.server.quorum.ReadOnlyZooKeeperServer;
import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
import org.apache.zookeeper.server.util.ZxidUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
 * Class containing static methods for registering and running Commands, as well
 * as default Command definitions.
 *
 * @see Command
 * @see JettyAdminServer
 */
public class Commands {

    static final Logger LOG = LoggerFactory.getLogger(Commands.class);

    /** Maps command names to Command instances */
    private static Map<String, Command> commands = new HashMap<String, Command>();
    private static Set<String> primaryNames = new HashSet<String>();

    /**
     * Registers the given command. Registered commands can be run by passing
     * any of their names to runCommand.
     */
    public static void registerCommand(Command command) {
        for (String name : command.getNames()) {
            Command prev = commands.put(name, command);
            if (prev != null) {
                LOG.warn("Re-registering command {} (primary name = {})", name, command.getPrimaryName());
            }
        }
        primaryNames.add(command.getPrimaryName());
    }

    /**
     * Run the registered command with name cmdName. Commands should not produce
     * any exceptions; any (anticipated) errors should be reported in the
     * "error" entry of the returned map. Likewise, if no command with the given
     * name is registered, this will be noted in the "error" entry.
     *
     * @param cmdName
     * @param zkServer
     * @param kwargs String-valued keyword arguments to the command
     *        (may be null if command requires no additional arguments)
     * @return Map representing response to command containing at minimum:
     *    - "command" key containing the command's primary name
     *    - "error" key containing a String error message or null if no error
     */
    public static CommandResponse runCommand(
        String cmdName,
        ZooKeeperServer zkServer,
        Map<String, String> kwargs) {
        Command command = getCommand(cmdName);
        if (command == null) {
            return new CommandResponse(cmdName, "Unknown command: " + cmdName);
        }
        if (command.isServerRequired() && (zkServer == null || !zkServer.isRunning())) {
            return new CommandResponse(cmdName, "This ZooKeeper instance is not currently serving requests");
        }
        return command.run(zkServer, kwargs);
    }

    /**
     * Returns the primary names of all registered commands.
     */
    public static Set<String> getPrimaryNames() {
        return primaryNames;
    }

    /**
     * Returns the commands registered under cmdName with registerCommand, or
     * null if no command is registered with that name.
     */
    public static Command getCommand(String cmdName) {
        return commands.get(cmdName);
    }

    static {
        registerCommand(new CnxnStatResetCommand());
        registerCommand(new ConfCommand());
        registerCommand(new ConsCommand());
        registerCommand(new DigestCommand());
        registerCommand(new DirsCommand());
        registerCommand(new DumpCommand());
        registerCommand(new EnvCommand());
        registerCommand(new GetTraceMaskCommand());
        registerCommand(new InitialConfigurationCommand());
        registerCommand(new IsroCommand());
        registerCommand(new LastSnapshotCommand());
        registerCommand(new LeaderCommand());
        registerCommand(new MonitorCommand());
        registerCommand(new ObserverCnxnStatResetCommand());
        registerCommand(new RuokCommand());
        registerCommand(new SetTraceMaskCommand());
        registerCommand(new SrvrCommand());
        registerCommand(new StatCommand());
        registerCommand(new StatResetCommand());
        registerCommand(new SyncedObserverConsCommand());
        registerCommand(new SystemPropertiesCommand());
        registerCommand(new VotingViewCommand());
        registerCommand(new WatchCommand());
        registerCommand(new WatchesByPathCommand());
        registerCommand(new WatchSummaryCommand());
        registerCommand(new ZabStateCommand());
    }

    /**
     * Reset all connection statistics.
     */
    public static class CnxnStatResetCommand extends CommandBase {

        public CnxnStatResetCommand() {
            super(Arrays.asList("connection_stat_reset", "crst"));
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = initializeResponse();
            zkServer.getServerCnxnFactory().resetAllConnectionStats();
            return response;

        }

    }

    /**
     * Server configuration parameters.
     * @see ZooKeeperServer#getConf()
     */
    public static class ConfCommand extends CommandBase {

        public ConfCommand() {
            super(Arrays.asList("configuration", "conf", "config"));
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = initializeResponse();
            response.putAll(zkServer.getConf().toMap());
            return response;
        }

    }

    /**
     * Information on client connections to server. Returned Map contains:
     *   - "connections": list of connection info objects
     * @see org.apache.zookeeper.server.ServerCnxn#getConnectionInfo(boolean)
     */
    public static class ConsCommand extends CommandBase {

        public ConsCommand() {
            super(Arrays.asList("connections", "cons"));
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = initializeResponse();
            ServerCnxnFactory serverCnxnFactory = zkServer.getServerCnxnFactory();
            if (serverCnxnFactory != null) {
                response.put("connections", serverCnxnFactory.getAllConnectionInfo(false));
            } else {
                response.put("connections", Collections.emptyList());
            }
            ServerCnxnFactory secureServerCnxnFactory = zkServer.getSecureServerCnxnFactory();
            if (secureServerCnxnFactory != null) {
                response.put("secure_connections", secureServerCnxnFactory.getAllConnectionInfo(false));
            } else {
                response.put("secure_connections", Collections.emptyList());
            }
            return response;
        }

    }

    /**
     * Information on ZK datadir and snapdir size in bytes
     */
    public static class DirsCommand extends CommandBase {

        public DirsCommand() {
            super(Arrays.asList("dirs"));
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = initializeResponse();
            response.put("datadir_size", zkServer.getDataDirSize());
            response.put("logdir_size", zkServer.getLogDirSize());
            return response;
        }

    }

    /**
     * Information on session expirations and ephemerals. Returned map contains:
     *   - "expiry_time_to_session_ids": Map&lt;Long, Set&lt;Long&gt;&gt;
     *                                   time -&gt; sessions IDs of sessions that expire at time
     *   - "session_id_to_ephemeral_paths": Map&lt;Long, Set&lt;String&gt;&gt;
     *                                       session ID -&gt; ephemeral paths created by that session
     * @see ZooKeeperServer#getSessionExpiryMap()
     * @see ZooKeeperServer#getEphemerals()
     */
    public static class DumpCommand extends CommandBase {

        public DumpCommand() {
            super(Arrays.asList("dump"));
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = initializeResponse();
            response.put("expiry_time_to_session_ids", zkServer.getSessionExpiryMap());
            response.put("session_id_to_ephemeral_paths", zkServer.getEphemerals());
            return response;
        }

    }

    /**
     * All defined environment variables.
     */
    public static class EnvCommand extends CommandBase {

        public EnvCommand() {
            super(Arrays.asList("environment", "env", "envi"), false);
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = initializeResponse();
            for (Entry e : Environment.list()) {
                response.put(e.getKey(), e.getValue());
            }
            return response;
        }

    }

    /**
     * Digest histories for every specific number of txns.
     */
    public static class DigestCommand extends CommandBase {

        public DigestCommand() {
            super(Arrays.asList("hash"));
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = initializeResponse();
            response.put("digests", zkServer.getZKDatabase().getDataTree().getDigestLog());
            return response;
        }

    }

    /**
     * The current trace mask. Returned map contains:
     *   - "tracemask": Long
     */
    public static class GetTraceMaskCommand extends CommandBase {

        public GetTraceMaskCommand() {
            super(Arrays.asList("get_trace_mask", "gtmk"), false);
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = initializeResponse();
            response.put("tracemask", ZooTrace.getTextTraceLevel());
            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;
        }

    }

    /**
     * Is this server in read-only mode. Returned map contains:
     *   - "is_read_only": Boolean
     */
    public static class IsroCommand extends CommandBase {

        public IsroCommand() {
            super(Arrays.asList("is_read_only", "isro"));
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = initializeResponse();
            response.put("read_only", zkServer instanceof ReadOnlyZooKeeperServer);
            return response;
        }

    }

    /**
     * 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;
        }

    }

    /**
     * Returns the leader status of this instance and the leader host string.
     */
    public static class LeaderCommand extends CommandBase {

        public LeaderCommand() {
            super(Arrays.asList("leader", "lead"));
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = initializeResponse();
            if (zkServer instanceof QuorumZooKeeperServer) {
                response.put("is_leader", zkServer instanceof LeaderZooKeeperServer);
                QuorumPeer peer = ((QuorumZooKeeperServer) zkServer).self;
                response.put("leader_id", peer.getLeaderId());
                String leaderAddress = peer.getLeaderAddress();
                response.put("leader_ip", leaderAddress != null ? leaderAddress : "");
            } else {
                response.put("error", "server is not initialized");
            }
            return response;
        }

    }

    /**
     * Some useful info for monitoring. Returned map contains:
     *   - "version": String
     *                server version
     *   - "avg_latency": Long
     *   - "max_latency": Long
     *   - "min_latency": Long
     *   - "packets_received": Long
     *   - "packets_sents": Long
     *   - "num_alive_connections": Integer
     *   - "outstanding_requests": Long
     *                             number of unprocessed requests
     *   - "server_state": "leader", "follower", or "standalone"
     *   - "znode_count": Integer
     *   - "watch_count": Integer
     *   - "ephemerals_count": Integer
     *   - "approximate_data_size": Long
     *   - "open_file_descriptor_count": Long (unix only)
     *   - "max_file_descriptor_count": Long (unix only)
     *   - "fsync_threshold_exceed_count": Long
     *   - "non_mtls_conn_count": Long
     *   - "non_mtls_remote_conn_count": Long
     *   - "non_mtls_local_conn_count": Long
     *   - "followers": Integer (leader only)
     *   - "synced_followers": Integer (leader only)
     *   - "pending_syncs": Integer (leader only)
     */
    public static class MonitorCommand extends CommandBase {

        public MonitorCommand() {
            super(Arrays.asList("monitor", "mntr"), false);
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = initializeResponse();
            zkServer.dumpMonitorValues(response::put);
            ServerMetrics.getMetrics().getMetricsProvider().dump(response::put);
            return response;

        }

    }

    /**
     * Reset all observer connection statistics.
     */
    public static class ObserverCnxnStatResetCommand extends CommandBase {

        public ObserverCnxnStatResetCommand() {
            super(Arrays.asList("observer_connection_stat_reset", "orst"));
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = initializeResponse();
            if (zkServer instanceof LeaderZooKeeperServer) {
                Leader leader = ((LeaderZooKeeperServer) zkServer).getLeader();
                leader.resetObserverConnectionStats();
            } else if (zkServer instanceof FollowerZooKeeperServer) {
                Follower follower = ((FollowerZooKeeperServer) zkServer).getFollower();
                follower.resetObserverConnectionStats();
            }
            return response;
        }

    }

    /**
     * No-op command, check if the server is running
     */
    public static class RuokCommand extends CommandBase {

        public RuokCommand() {
            super(Arrays.asList("ruok"));
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            return initializeResponse();
        }

    }

    /**
     * Sets the trace mask. Required arguments:
     *   - "traceMask": Long
     *  Returned Map contains:
     *   - "tracemask": Long
     */
    public static class SetTraceMaskCommand extends CommandBase {

        public SetTraceMaskCommand() {
            super(Arrays.asList("set_trace_mask", "stmk"), false);
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = initializeResponse();
            long traceMask;
            if (!kwargs.containsKey("traceMask")) {
                response.put("error", "setTraceMask requires long traceMask argument");
                return response;
            }
            try {
                traceMask = Long.parseLong(kwargs.get("traceMask"));
            } catch (NumberFormatException e) {
                response.put("error", "setTraceMask requires long traceMask argument, got " + kwargs.get("traceMask"));
                return response;
            }

            ZooTrace.setTextTraceLevel(traceMask);
            response.put("tracemask", traceMask);
            return response;
        }

    }

    /**
     * Server information. Returned map contains:
     *   - "version": String
     *                version of server
     *   - "read_only": Boolean
     *                  is server in read-only mode
     *   - "server_stats": ServerStats object
     *   - "node_count": Integer
     */
    public static class SrvrCommand extends CommandBase {

        public SrvrCommand() {
            super(Arrays.asList("server_stats", "srvr"));
        }

        // Allow subclasses (e.g. StatCommand) to specify their own names
        protected SrvrCommand(List<String> names) {
            super(names);
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = initializeResponse();
            LOG.info("running stat");
            response.put("version", Version.getFullVersion());
            response.put("read_only", zkServer instanceof ReadOnlyZooKeeperServer);
            response.put("server_stats", zkServer.serverStats());
            response.put("client_response", zkServer.serverStats().getClientResponseStats());
            if (zkServer instanceof LeaderZooKeeperServer) {
                Leader leader = ((LeaderZooKeeperServer) zkServer).getLeader();
                response.put("proposal_stats", leader.getProposalStats());
            }
            response.put("node_count", zkServer.getZKDatabase().getNodeCount());
            return response;
        }

    }

    /**
     * Same as SrvrCommand but has extra "connections" entry.
     */
    public static class StatCommand extends SrvrCommand {

        public StatCommand() {
            super(Arrays.asList("stats", "stat"));
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = super.run(zkServer, kwargs);

            final Iterable<Map<String, Object>> connections;
            if (zkServer.getServerCnxnFactory() != null) {
                connections = zkServer.getServerCnxnFactory().getAllConnectionInfo(true);
            } else {
                connections = Collections.emptyList();
            }
            response.put("connections", connections);

            final Iterable<Map<String, Object>> secureConnections;
            if (zkServer.getSecureServerCnxnFactory() != null) {
                secureConnections = zkServer.getSecureServerCnxnFactory().getAllConnectionInfo(true);
            } else {
                secureConnections = Collections.emptyList();
            }
            response.put("secure_connections", secureConnections);
            return response;
        }

    }

    /**
     * Resets server statistics.
     */
    public static class StatResetCommand extends CommandBase {

        public StatResetCommand() {
            super(Arrays.asList("stat_reset", "srst"));
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = initializeResponse();
            zkServer.serverStats().reset();
            return response;
        }

    }

    /**
     * Information on observer connections to server. Returned Map contains:
     *   - "synced_observers": Integer (leader/follower only)
     *   - "observers": list of observer learner handler info objects (leader/follower only)
     * @see org.apache.zookeeper.server.quorum.LearnerHandler#getLearnerHandlerInfo()
     */
    public static class SyncedObserverConsCommand extends CommandBase {

        public SyncedObserverConsCommand() {
            super(Arrays.asList("observers", "obsr"));
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {

            CommandResponse response = initializeResponse();

            if (zkServer instanceof LeaderZooKeeperServer) {
                Leader leader = ((LeaderZooKeeperServer) zkServer).getLeader();

                response.put("synced_observers", leader.getObservingLearners().size());
                response.put("observers", leader.getObservingLearnersInfo());
                return response;
            } else if (zkServer instanceof FollowerZooKeeperServer) {
                Follower follower = ((FollowerZooKeeperServer) zkServer).getFollower();
                Integer syncedObservers = follower.getSyncedObserverSize();
                if (syncedObservers != null) {
                    response.put("synced_observers", syncedObservers);
                    response.put("observers", follower.getSyncedObserversInfo());
                    return response;
                }
            }

            response.put("synced_observers", 0);
            response.put("observers", Collections.emptySet());
            return response;
        }

    }

    /**
     * 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;
        }

    }

    /**
     * Returns the current ensemble configuration information.
     * It provides list of current voting members in the ensemble.
     */
    public static class VotingViewCommand extends CommandBase {

        public VotingViewCommand() {
            super(Arrays.asList("voting_view"));
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = initializeResponse();
            if (zkServer instanceof QuorumZooKeeperServer) {
                QuorumPeer peer = ((QuorumZooKeeperServer) zkServer).self;
                Map<Long, QuorumServerView> votingView = peer.getVotingView().entrySet().stream()
                        .collect(Collectors.toMap(Map.Entry::getKey, e -> new QuorumServerView(e.getValue())));
                response.put("current_config", votingView);
            } else {
                response.put("current_config", Collections.emptyMap());
            }
            return response;
        }

        @SuppressFBWarnings(value = "URF_UNREAD_FIELD", justification = "class is used only for JSON serialization")
        private static class QuorumServerView {

            @JsonProperty
            private List<String> serverAddresses;

            @JsonProperty
            private List<String> electionAddresses;

            @JsonProperty
            private String clientAddress;

            @JsonProperty
            private String learnerType;

            public QuorumServerView(QuorumPeer.QuorumServer quorumServer) {
                this.serverAddresses = getMultiAddressString(quorumServer.addr);
                this.electionAddresses = getMultiAddressString(quorumServer.electionAddr);
                this.learnerType = quorumServer.type.equals(LearnerType.PARTICIPANT) ? "participant" : "observer";
                this.clientAddress = getAddressString(quorumServer.clientAddr);
            }

            private static List<String> getMultiAddressString(MultipleAddresses multipleAddresses) {
                if (multipleAddresses == null) {
                    return Collections.emptyList();
                }

                return multipleAddresses.getAllAddresses().stream()
                        .map(QuorumServerView::getAddressString)
                        .collect(Collectors.toList());
            }

            private static String getAddressString(InetSocketAddress address) {
                if (address == null) {
                    return "";
                }
                return String.format("%s:%d", QuorumPeer.QuorumServer.delimitedHostString(address), address.getPort());
            }
       }

    }

    /**
     * Watch information aggregated by session. Returned Map contains:
     *   - "session_id_to_watched_paths": Map&lt;Long, Set&lt;String&gt;&gt; session ID -&gt; watched paths
     * @see DataTree#getWatches()
     * @see DataTree#getWatches()
     */
    public static class WatchCommand extends CommandBase {

        public WatchCommand() {
            super(Arrays.asList("watches", "wchc"));
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            DataTree dt = zkServer.getZKDatabase().getDataTree();
            CommandResponse response = initializeResponse();
            response.put("session_id_to_watched_paths", dt.getWatches().toMap());
            return response;
        }

    }

    /**
     * Watch information aggregated by path. Returned Map contains:
     *   - "path_to_session_ids": Map&lt;String, Set&lt;Long&gt;&gt; path -&gt; session IDs of sessions watching path
     * @see DataTree#getWatchesByPath()
     */
    public static class WatchesByPathCommand extends CommandBase {

        public WatchesByPathCommand() {
            super(Arrays.asList("watches_by_path", "wchp"));
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            DataTree dt = zkServer.getZKDatabase().getDataTree();
            CommandResponse response = initializeResponse();
            response.put("path_to_session_ids", dt.getWatchesByPath().toMap());
            return response;
        }

    }

    /**
     * Summarized watch information.
     * @see DataTree#getWatchesSummary()
     */
    public static class WatchSummaryCommand extends CommandBase {

        public WatchSummaryCommand() {
            super(Arrays.asList("watch_summary", "wchs"));
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            DataTree dt = zkServer.getZKDatabase().getDataTree();
            CommandResponse response = initializeResponse();
            response.putAll(dt.getWatchesSummary().toMap());
            return response;
        }

    }

    /**
     * Returns the current phase of Zab protocol that peer is running.
     * It can be in one of these phases: ELECTION, DISCOVERY, SYNCHRONIZATION, BROADCAST
     */
    public static class ZabStateCommand extends CommandBase {

        public ZabStateCommand() {
            super(Arrays.asList("zabstate"), false);
        }

        @Override
        public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs) {
            CommandResponse response = initializeResponse();
            if (zkServer instanceof QuorumZooKeeperServer) {
                QuorumPeer peer = ((QuorumZooKeeperServer) zkServer).self;
                QuorumPeer.ZabState zabState = peer.getZabState();
                QuorumVerifier qv = peer.getQuorumVerifier();

                QuorumPeer.QuorumServer voter = qv.getVotingMembers().get(peer.getMyId());
                boolean voting = (
                        voter != null
                                && voter.addr.equals(peer.getQuorumAddress())
                                && voter.electionAddr.equals(peer.getElectionAddress())
                );
                response.put("myid", zkServer.getConf().getServerId());
                response.put("is_leader", zkServer instanceof LeaderZooKeeperServer);
                response.put("quorum_address", peer.getQuorumAddress());
                response.put("election_address", peer.getElectionAddress());
                response.put("client_address", peer.getClientAddress());
                response.put("voting", voting);
                long lastProcessedZxid = zkServer.getZKDatabase().getDataTreeLastProcessedZxid();
                response.put("last_zxid", "0x" + ZxidUtils.zxidToString(lastProcessedZxid));
                response.put("zab_epoch", ZxidUtils.getEpochFromZxid(lastProcessedZxid));
                response.put("zab_counter", ZxidUtils.getCounterFromZxid(lastProcessedZxid));
                response.put("zabstate", zabState.name().toLowerCase());
            } else {
                response.put("voting", false);
                response.put("zabstate", "");
            }
            return response;
        }

    }

    private Commands() {
    }

}