瀏覽代碼

HDDS-528. add cli command to checkChill mode status and exit chill mode. Contributed by chencan.

Márton Elek 6 年之前
父節點
當前提交
fb1e8bb0cc

+ 20 - 0
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java

@@ -440,4 +440,24 @@ public class ContainerOperationClient implements ScmClient {
     }
     return size;
   }
+
+  /**
+   * Check if SCM is in chill mode.
+   *
+   * @return Returns true if SCM is in chill mode else returns false.
+   * @throws IOException
+   */
+  public boolean inChillMode() throws IOException {
+    return storageContainerLocationClient.inChillMode();
+  }
+
+  /**
+   * Force SCM out of chill mode.
+   *
+   * @return returns true if operation is successful.
+   * @throws IOException
+   */
+  public boolean forceExitChillMode() throws IOException {
+    return storageContainerLocationClient.forceExitChillMode();
+  }
 }

+ 16 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java

@@ -171,4 +171,20 @@ public interface ScmClient extends Closeable {
   Pipeline createReplicationPipeline(HddsProtos.ReplicationType type,
       HddsProtos.ReplicationFactor factor, HddsProtos.NodePool nodePool)
       throws IOException;
+
+  /**
+   * Check if SCM is in chill mode.
+   *
+   * @return Returns true if SCM is in chill mode else returns false.
+   * @throws IOException
+   */
+  boolean inChillMode() throws IOException;
+
+  /**
+   * Force SCM out of chill mode.
+   *
+   * @return returns true if operation is successful.
+   * @throws IOException
+   */
+  boolean forceExitChillMode() throws IOException;
 }

+ 61 - 0
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ChillModeCheckSubcommand.java

@@ -0,0 +1,61 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.hdds.scm.cli;
+
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.ParentCommand;
+
+/**
+ * This is the handler that process chill mode check command.
+ */
+@Command(
+    name = "status",
+    description = "Check if SCM is in chill mode",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class ChillModeCheckSubcommand implements Callable<Void> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ChillModeCheckSubcommand.class);
+
+  @ParentCommand
+  private ChillModeCommands parent;
+
+  @Override
+  public Void call() throws Exception {
+    try (ScmClient scmClient = parent.getParent().createScmClient()) {
+
+      boolean execReturn = scmClient.inChillMode();
+
+      // Output data list
+      if(execReturn){
+        LOG.info("SCM is in chill mode.");
+      } else {
+        LOG.info("SCM is out of chill mode.");
+      }
+      return null;
+    }
+  }
+}

+ 60 - 0
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ChillModeCommands.java

@@ -0,0 +1,60 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.hdds.scm.cli;
+
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.cli.MissingSubcommandException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.ParentCommand;
+
+/**
+ * Subcommand to group chill mode related operations.
+ */
+@Command(
+    name = "chillmode",
+    description = "Chill mode specific operations",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class,
+    subcommands = {
+        ChillModeCheckSubcommand.class,
+        ChillModeExitSubcommand.class,
+    })
+public class ChillModeCommands implements Callable<Void> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ChillModeCommands.class);
+
+  @ParentCommand
+  private SCMCLI parent;
+
+  public SCMCLI getParent() {
+    return parent;
+  }
+
+  @Override
+  public Void call() throws Exception {
+    throw new MissingSubcommandException(
+        this.parent.getCmd().getSubcommands().get("chillmode").
+        getUsageMessage());
+  }
+}

+ 57 - 0
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ChillModeExitSubcommand.java

@@ -0,0 +1,57 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.hdds.scm.cli;
+
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.ParentCommand;
+
+/**
+ * This is the handler that process chill mode exit command.
+ */
+@Command(
+    name = "exit",
+    description = "Force SCM out of chill mode",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class ChillModeExitSubcommand implements Callable<Void> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ChillModeExitSubcommand.class);
+
+  @ParentCommand
+  private ChillModeCommands parent;
+
+  @Override
+  public Void call() throws Exception {
+    try (ScmClient scmClient = parent.getParent().createScmClient()) {
+
+      boolean execReturn = scmClient.forceExitChillMode();
+      if(execReturn){
+        LOG.info("SCM exit chill mode successfully.");
+      }
+      return null;
+    }
+  }
+}

+ 1 - 0
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java

@@ -73,6 +73,7 @@ import picocli.CommandLine.Option;
         + "operations.",
     versionProvider = HddsVersionProvider.class,
     subcommands = {
+        ChillModeCommands.class,
         ListSubcommand.class,
         InfoSubcommand.class,
         DeleteSubcommand.class,