You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by eo...@apache.org on 2022/01/17 08:52:09 UTC

[bookkeeper] branch master updated: suport dynamic enable/disable health check (#2947)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 7a770dc  suport dynamic enable/disable health check (#2947)
7a770dc is described below

commit 7a770dcf44213eeacecfdb60083141bcb891b056
Author: chenlin <15...@qq.com>
AuthorDate: Mon Jan 17 16:52:01 2022 +0800

    suport dynamic enable/disable health check (#2947)
---
 .../org/apache/bookkeeper/client/BookKeeper.java   |  19 ++++
 .../bookkeeper/conf/AbstractConfiguration.java     |   1 +
 .../bookkeeper/meta/MetadataBookieDriver.java      |  28 +++++
 .../bookkeeper/meta/MetadataClientDriver.java      |  11 ++
 .../bookkeeper/meta/zk/ZKMetadataDriverBase.java   |  66 ++++++++++++
 .../health/SwitchOfHealthCheckCommand.java         | 116 +++++++++++++++++++++
 .../tools/cli/commands/health/package-info.java    |  18 ++++
 .../bookkeeper/util/BookKeeperConstants.java       |   1 +
 .../cli/commands/HealthCheckCommandGroup.java      |  46 ++++++++
 ....apache.bookkeeper.tools.framework.CommandGroup |   1 +
 tools/ledger/src/main/resources/commands           |   1 +
 .../health/SwitchOfHealthCheckCommandTest.java     |  91 ++++++++++++++++
 12 files changed, 399 insertions(+)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
index 1602490..f08f47b 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
@@ -41,6 +41,7 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
@@ -616,6 +617,24 @@ public class BookKeeper implements org.apache.bookkeeper.client.api.BookKeeper {
 
     void checkForFaultyBookies() {
         List<BookieId> faultyBookies = bookieClient.getFaultyBookies();
+        if (faultyBookies.isEmpty()) {
+            return;
+        }
+
+        boolean isEnabled = false;
+        try {
+            isEnabled = metadataDriver.isHealthCheckEnabled().get();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            LOG.error("Cannot verify if healthcheck is enabled", e);
+        } catch (ExecutionException e) {
+            LOG.error("Cannot verify if healthcheck is enabled", e.getCause());
+        }
+        if (!isEnabled) {
+            LOG.info("Health checks is currently disabled!");
+            return;
+        }
+
         for (BookieId faultyBookie : faultyBookies) {
             if (Math.random() <= bookieQuarantineRatio) {
                 bookieWatcher.quarantineBookie(faultyBookie);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java
index 414ac99..3b80b52 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java
@@ -93,6 +93,7 @@ public abstract class AbstractConfiguration<T extends AbstractConfiguration>
     protected static final String STORE_SYSTEMTIME_AS_LEDGER_CREATION_TIME = "storeSystemTimeAsLedgerCreationTime";
 
     protected static final String ENABLE_BUSY_WAIT = "enableBusyWait";
+    protected static final String ENABLE_HEALTH_CHECK = "enableHealthCheck";
 
     // Metastore settings, only being used when LEDGER_MANAGER_FACTORY_CLASS is MSLedgerManagerFactory
     protected static final String METASTORE_IMPL_CLASS = "metastoreImplClass";
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataBookieDriver.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataBookieDriver.java
index dbbab2e..a397297 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataBookieDriver.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataBookieDriver.java
@@ -18,6 +18,9 @@
  */
 package org.apache.bookkeeper.meta;
 
+import java.util.concurrent.CompletableFuture;
+
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationManager;
 import org.apache.bookkeeper.meta.exceptions.MetadataException;
@@ -68,6 +71,31 @@ public interface MetadataBookieDriver extends AutoCloseable {
      */
     LayoutManager getLayoutManager();
 
+    /**
+     * Return health check is enable or disable.
+     *
+     * @return true if health check is enable, otherwise false.
+     */
+    default CompletableFuture<Boolean>  isHealthCheckEnabled() {
+        return FutureUtils.value(true);
+    }
+
+    /**
+     * Disable health check.
+     */
+    default CompletableFuture<Void> disableHealthCheck() {
+        CompletableFuture<Void> result = new CompletableFuture<>();
+        result.completeExceptionally(new Exception("disableHealthCheck is not supported by this metadata driver"));
+        return result;
+    }
+
+    /**
+     * Enable health check.
+     */
+    default CompletableFuture<Void>  enableHealthCheck() {
+        return FutureUtils.Void();
+    }
+
     @Override
     void close();
 
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java
index 7647a5b..1615f04 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java
@@ -19,9 +19,11 @@
 package org.apache.bookkeeper.meta;
 
 import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ScheduledExecutorService;
 import org.apache.bookkeeper.common.annotation.InterfaceAudience.LimitedPrivate;
 import org.apache.bookkeeper.common.annotation.InterfaceStability.Evolving;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.discover.RegistrationClient;
 import org.apache.bookkeeper.meta.exceptions.MetadataException;
@@ -103,4 +105,13 @@ public interface MetadataClientDriver extends AutoCloseable {
      *            listener listening on metadata client session states.
      */
     void setSessionStateListener(SessionStateListener sessionStateListener);
+
+    /**
+     * Return health check is enable or disable.
+     *
+     * @return true if health check is enable, otherwise false.
+     */
+    default CompletableFuture<Boolean> isHealthCheckEnabled() {
+        return FutureUtils.value(true);
+    }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java
index 11f0877..58f26bf 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java
@@ -21,6 +21,7 @@ package org.apache.bookkeeper.meta.zk;
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE;
+import static org.apache.bookkeeper.util.BookKeeperConstants.DISABLE_HEALTH_CHECK;
 import static org.apache.bookkeeper.util.BookKeeperConstants.EMPTY_BYTE_ARRAY;
 import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY;
 
@@ -28,6 +29,8 @@ import java.io.IOException;
 import java.net.URI;
 import java.util.List;
 import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+
 import lombok.Getter;
 import lombok.Setter;
 import lombok.SneakyThrows;
@@ -42,15 +45,18 @@ import org.apache.bookkeeper.meta.ZkLayoutManager;
 import org.apache.bookkeeper.meta.exceptions.Code;
 import org.apache.bookkeeper.meta.exceptions.MetadataException;
 import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.util.BookKeeperConstants;
 import org.apache.bookkeeper.util.ZkUtils;
 import org.apache.bookkeeper.zookeeper.RetryPolicy;
 import org.apache.bookkeeper.zookeeper.ZooKeeperClient;
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
 
 /**
  * This is a mixin class for supporting zookeeper based metadata driver.
@@ -140,6 +146,9 @@ public class ZKMetadataDriverBase implements AutoCloseable {
     // instantiated us
     protected boolean ownZKHandle = false;
 
+    // disable health check path
+    String disableHealthCheckPath;
+
     // ledgers root path
     protected String ledgersRootPath;
 
@@ -230,6 +239,7 @@ public class ZKMetadataDriverBase implements AutoCloseable {
             this.ownZKHandle = true;
         }
 
+        disableHealthCheckPath = ledgersRootPath + "/" + DISABLE_HEALTH_CHECK;
         // once created the zookeeper client, create the layout manager and registration client
         this.layoutManager = new ZkLayoutManager(
             zk,
@@ -260,6 +270,62 @@ public class ZKMetadataDriverBase implements AutoCloseable {
         return lmFactory;
     }
 
+    public CompletableFuture<Void> disableHealthCheck() {
+        CompletableFuture<Void> createResult = new CompletableFuture<>();
+        zk.create(disableHealthCheckPath, BookKeeperConstants.EMPTY_BYTE_ARRAY, acls,
+                CreateMode.PERSISTENT, new AsyncCallback.StringCallback() {
+            @Override
+            public void processResult(int rc, String path, Object ctx, String name) {
+                if (KeeperException.Code.OK.intValue() == rc) {
+                    createResult.complete(null);
+                } else if (KeeperException.Code.NODEEXISTS.intValue() == rc) {
+                    log.debug("health check already disabled!");
+                    createResult.complete(null);
+                } else {
+                    createResult.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc), path));
+                }
+            }
+        }, null);
+
+        return createResult;
+    }
+
+    public CompletableFuture<Void> enableHealthCheck() {
+        CompletableFuture<Void> deleteResult = new CompletableFuture<>();
+
+        zk.delete(disableHealthCheckPath, -1, new AsyncCallback.VoidCallback() {
+            @Override
+            public void processResult(int rc, String path, Object ctx) {
+                if (KeeperException.Code.OK.intValue() == rc) {
+                    deleteResult.complete(null);
+                } else if (KeeperException.Code.NONODE.intValue() == rc) {
+                    log.debug("health check already enabled!");
+                    deleteResult.complete(null);
+                } else {
+                    deleteResult.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc), path));
+                }
+            }
+        }, null);
+
+        return deleteResult;
+    }
+
+    public CompletableFuture<Boolean> isHealthCheckEnabled() {
+        CompletableFuture<Boolean> enableResult = new CompletableFuture<>();
+        zk.exists(disableHealthCheckPath, false, new AsyncCallback.StatCallback() {
+            @Override
+            public void processResult(int rc, String path, Object ctx, Stat stat) {
+                if (KeeperException.Code.OK.intValue() == rc) {
+                    enableResult.complete(false);
+                } else {
+                    enableResult.complete(true);
+                }
+            }
+        }, null);
+
+        return enableResult;
+    }
+
     @Override
     public void close() {
         if (null != lmFactory) {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/health/SwitchOfHealthCheckCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/health/SwitchOfHealthCheckCommand.java
new file mode 100644
index 0000000..fd71425
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/health/SwitchOfHealthCheckCommand.java
@@ -0,0 +1,116 @@
+/*
+ * 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.bookkeeper.tools.cli.commands.health;
+
+import com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.concurrent.ExecutionException;
+import lombok.Setter;
+import lombok.experimental.Accessors;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.meta.exceptions.MetadataException;
+import org.apache.bookkeeper.tools.cli.helpers.BookieCommand;
+import org.apache.bookkeeper.tools.framework.CliFlags;
+import org.apache.bookkeeper.tools.framework.CliSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+
+/**
+ * Command to enable or disable health check in the cluster.
+ */
+public class SwitchOfHealthCheckCommand extends BookieCommand<SwitchOfHealthCheckCommand.HealthCheckFlags> {
+
+    static final Logger LOG = LoggerFactory.getLogger(SwitchOfHealthCheckCommand.class);
+
+    private static final String NAME = "switch";
+    private static final String DESC = "Enables or disables health check in the cluster. Default is enabled.";
+
+    public SwitchOfHealthCheckCommand() {
+        this(new HealthCheckFlags());
+    }
+
+    private SwitchOfHealthCheckCommand(HealthCheckFlags flags) {
+        super(CliSpec.<HealthCheckFlags>newBuilder()
+                .withName(NAME).withDescription(DESC)
+                .withFlags(flags).build());
+    }
+
+    /**
+     * Flags for health check command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class HealthCheckFlags extends CliFlags {
+
+        @Parameter(names = { "-e", "--enable" }, description = "Enable or disable health check.")
+        private boolean enable;
+
+        @Parameter(names = {"-s", "--status"}, description = "Check the health check status.")
+        private boolean status;
+
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, HealthCheckFlags cmdFlags) {
+        try {
+            return handler(conf, cmdFlags);
+        } catch (MetadataException | ExecutionException e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    private boolean handler(ServerConfiguration conf, HealthCheckFlags flags)
+            throws MetadataException, ExecutionException {
+
+        MetadataDrivers.runFunctionWithMetadataBookieDriver(conf, driver -> {
+            try {
+                boolean isEnable = driver.isHealthCheckEnabled().get();
+
+                if (flags.status) {
+                    LOG.info("EnableHealthCheck is " + (isEnable ? "enabled." : "disabled."));
+                    return null;
+                }
+
+                if (flags.enable) {
+                    if (isEnable) {
+                        LOG.warn("HealthCheck already enabled. Doing nothing");
+                    } else {
+                        LOG.info("Enable HealthCheck");
+                        driver.enableHealthCheck().get();
+                    }
+                } else {
+                    if (!isEnable) {
+                        LOG.warn("HealthCheck already disabled. Doing nothing");
+                    } else {
+                        LOG.info("Disable HealthCheck");
+                        driver.disableHealthCheck().get();
+                    }
+                }
+            } catch (Exception e) {
+                LOG.error("exception", e);
+                throw new UncheckedExecutionException(e);
+            }
+            return null;
+        });
+        return true;
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/health/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/health/package-info.java
new file mode 100644
index 0000000..8966724
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/health/package-info.java
@@ -0,0 +1,18 @@
+/**
+ * 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.bookkeeper.tools.cli.commands.health;
\ No newline at end of file
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/BookKeeperConstants.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/BookKeeperConstants.java
index a08ff59..8e10a15 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/BookKeeperConstants.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/BookKeeperConstants.java
@@ -49,6 +49,7 @@ public class BookKeeperConstants {
     public static final String DEFAULT_ZK_LEDGERS_ROOT_PATH = "/ledgers";
     public static final String LAYOUT_ZNODE = "LAYOUT";
     public static final String INSTANCEID = "INSTANCEID";
+    public static final String DISABLE_HEALTH_CHECK = "disableHealthCheck";
 
     /**
      * Set the max log size limit to 1GB. It makes extra room for entry log file before
diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/HealthCheckCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/HealthCheckCommandGroup.java
new file mode 100644
index 0000000..f3b3dc2
--- /dev/null
+++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/HealthCheckCommandGroup.java
@@ -0,0 +1,46 @@
+/*
+ * 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.bookkeeper.tools.cli.commands;
+
+import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_INFRA_SERVICE;
+import org.apache.bookkeeper.tools.cli.commands.health.SwitchOfHealthCheckCommand;
+import org.apache.bookkeeper.tools.common.BKFlags;
+import org.apache.bookkeeper.tools.framework.CliCommandGroup;
+import org.apache.bookkeeper.tools.framework.CliSpec;
+
+
+/**
+ * Commands on some specific operation.
+ */
+public class HealthCheckCommandGroup extends CliCommandGroup<BKFlags> {
+
+    private static final String NAME = "healthCheck";
+    private static final String DESC = "Command on some specific operation.";
+
+    private static final CliSpec<BKFlags> spec = CliSpec.<BKFlags>newBuilder()
+        .withName(NAME)
+        .withDescription(DESC)
+        .withCategory(CATEGORY_INFRA_SERVICE)
+        .addCommand(new SwitchOfHealthCheckCommand())
+        .build();
+
+    public HealthCheckCommandGroup() {
+        super(spec);
+    }
+}
diff --git a/tools/ledger/src/main/resources/META-INF/services/org.apache.bookkeeper.tools.framework.CommandGroup b/tools/ledger/src/main/resources/META-INF/services/org.apache.bookkeeper.tools.framework.CommandGroup
index d81ba5a..63cfb10 100644
--- a/tools/ledger/src/main/resources/META-INF/services/org.apache.bookkeeper.tools.framework.CommandGroup
+++ b/tools/ledger/src/main/resources/META-INF/services/org.apache.bookkeeper.tools.framework.CommandGroup
@@ -22,3 +22,4 @@ org.apache.bookkeeper.tools.cli.commands.BookiesCommandGroup
 org.apache.bookkeeper.tools.cli.commands.CookieCommandGroup
 org.apache.bookkeeper.tools.cli.commands.LedgerCommandGroup
 org.apache.bookkeeper.tools.cli.commands.AutoRecoveryCommandGroup
+org.apache.bookkeeper.tools.cli.commands.HealthCheckCommandGroup
diff --git a/tools/ledger/src/main/resources/commands b/tools/ledger/src/main/resources/commands
index d81ba5a..63cfb10 100644
--- a/tools/ledger/src/main/resources/commands
+++ b/tools/ledger/src/main/resources/commands
@@ -22,3 +22,4 @@ org.apache.bookkeeper.tools.cli.commands.BookiesCommandGroup
 org.apache.bookkeeper.tools.cli.commands.CookieCommandGroup
 org.apache.bookkeeper.tools.cli.commands.LedgerCommandGroup
 org.apache.bookkeeper.tools.cli.commands.AutoRecoveryCommandGroup
+org.apache.bookkeeper.tools.cli.commands.HealthCheckCommandGroup
diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/health/SwitchOfHealthCheckCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/health/SwitchOfHealthCheckCommandTest.java
new file mode 100644
index 0000000..c8ef808
--- /dev/null
+++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/health/SwitchOfHealthCheckCommandTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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.bookkeeper.tools.cli.commands.health;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.powermock.api.mockito.PowerMockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+import java.util.function.Function;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+
+/**
+ * Unit test for {@link SwitchOfHealthCheckCommand}.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({SwitchOfHealthCheckCommand.class, MetadataDrivers.class})
+public class SwitchOfHealthCheckCommandTest extends BookieCommandTestBase {
+    MetadataClientDriver metadataClientDriver;
+    private MetadataBookieDriver metadataBookieDriver;
+
+
+    public SwitchOfHealthCheckCommandTest() {
+        super(3, 0);
+    }
+
+    @Override
+    public void setup() throws Exception {
+        super.setup();
+        PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf);
+        metadataBookieDriver = mock(MetadataBookieDriver.class);
+        PowerMockito.mockStatic(MetadataDrivers.class);
+        PowerMockito.doAnswer(invocationOnMock -> {
+            Function<MetadataBookieDriver, ?> function = invocationOnMock.getArgument(1);
+            function.apply(metadataBookieDriver);
+            return true;
+        }).when(MetadataDrivers.class, "runFunctionWithMetadataBookieDriver", any(ServerConfiguration.class),
+                any(Function.class));
+        metadataClientDriver = mock(MetadataClientDriver.class);
+        when(metadataBookieDriver.isHealthCheckEnabled()).thenReturn(FutureUtils.value(true));
+        when(metadataBookieDriver.disableHealthCheck()).thenReturn(FutureUtils.value(null));
+    }
+
+    @Test
+    public void testWithEnable() {
+        testCommand("-e");
+    }
+
+    @Test
+    public void testWithStatus() {
+        testCommand("-s");
+    }
+
+
+    @Test
+    public void testWithNoArgsDisable() {
+        testCommand("");
+    }
+
+    private void testCommand(String... args) {
+        SwitchOfHealthCheckCommand cmd = new SwitchOfHealthCheckCommand();
+        Assert.assertTrue(cmd.apply(bkFlags, args));
+    }
+}