You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2021/12/16 12:59:50 UTC

[GitHub] [bookkeeper] eolivelli commented on a change in pull request #2947: suport dynamic enable/disable health check

eolivelli commented on a change in pull request #2947:
URL: https://github.com/apache/bookkeeper/pull/2947#discussion_r770513971



##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
##########
@@ -614,7 +615,19 @@ public void safeRun() {
         }
     }
 
+    boolean enableHealthCheck() {
+        if (metadataDriver instanceof ZKMetadataClientDriver) {

Review comment:
       why this "instanceof" ?
   
   please add the method in the MetadataClientDriver class
   
   
   

##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java
##########
@@ -260,6 +261,30 @@ public synchronized LedgerManagerFactory getLedgerManagerFactory()
         return lmFactory;
     }
 
+    public boolean isEnableHealthCheck() {
+        try {
+            return null == zk.exists(conf.getEnableHealthPath(), false);
+        } catch (Exception e) {
+            return true;
+        }
+    }
+
+    public void disableHealthCheck(String enableHealthPath) throws Exception{
+        zk.create(enableHealthPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);

Review comment:
       there should be a configuration flag to define which ACL we should use, please check

##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
##########
@@ -614,7 +615,19 @@ public void safeRun() {
         }
     }
 
+    boolean enableHealthCheck() {
+        if (metadataDriver instanceof ZKMetadataClientDriver) {
+            return ((ZKMetadataClientDriver) metadataDriver).enableHealthCheck();
+        }
+        return true;
+    }
+
     void checkForFaultyBookies() {
+        if (!enableHealthCheck()) {
+            LOG.info("enableHealthCheck is false!");

Review comment:
       What about: "Health checks is currently disabled"

##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java
##########
@@ -260,6 +261,30 @@ public synchronized LedgerManagerFactory getLedgerManagerFactory()
         return lmFactory;
     }
 
+    public boolean isEnableHealthCheck() {
+        try {
+            return null == zk.exists(conf.getEnableHealthPath(), false);
+        } catch (Exception e) {
+            return true;
+        }
+    }
+
+    public void disableHealthCheck(String enableHealthPath) throws Exception{
+        zk.create(enableHealthPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+    }
+
+    public void enableHealthCheck(String enableHealthPath) throws KeeperException, InterruptedException {
+        zk.delete(enableHealthPath, -1);
+    }
+
+    public boolean enableHealthCheck() {
+        try {
+            return null == zk.exists(conf.getEnableHealthPath(), false);
+        } catch (Exception e) {

Review comment:
       do not catch Exception.
   in this case it is better to let the error bubble out

##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/health/SwitchOfHealthCheckCommand.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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 lombok.Setter;
+import lombok.experimental.Accessors;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.LedgerUnderreplicationManager;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.meta.exceptions.MetadataException;
+import org.apache.bookkeeper.meta.zk.ZKMetadataBookieDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.replication.ReplicationException;
+import org.apache.bookkeeper.tools.cli.helpers.BookieCommand;
+import org.apache.bookkeeper.tools.framework.CliFlags;
+import org.apache.bookkeeper.tools.framework.CliSpec;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ExecutionException;
+
+
+/**
+ * Command to enable or disable auto recovery 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 = "Enable or disable health check in the cluster. Default is enable.";
+
+    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 = { "-d", "--disable" }, description = "Enable or disable health check.")

Review comment:
       usually it is better to have "positive" options, like "--enable" 

##########
File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/health/SwitchOfHealthCheckCommand.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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 lombok.Setter;
+import lombok.experimental.Accessors;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.LedgerUnderreplicationManager;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.meta.exceptions.MetadataException;
+import org.apache.bookkeeper.meta.zk.ZKMetadataBookieDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.replication.ReplicationException;
+import org.apache.bookkeeper.tools.cli.helpers.BookieCommand;
+import org.apache.bookkeeper.tools.framework.CliFlags;
+import org.apache.bookkeeper.tools.framework.CliSpec;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ExecutionException;
+
+
+/**
+ * Command to enable or disable auto recovery 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 = "Enable or disable health check in the cluster. Default is enable.";
+
+    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 = { "-d", "--disable" }, description = "Enable or disable health check.")
+        private boolean disable;
+
+        @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 {
+                String enableHealthPath = conf.getEnableHealthPath();
+
+                if(!(driver instanceof ZKMetadataBookieDriver)){

Review comment:
       please remove this line, you can add a dummy implementation in the MetadataBookieDriver (using "default" methods)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org