You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ae...@apache.org on 2017/06/15 17:06:44 UTC

hadoop git commit: HDFS-11961. Ozone: Add start-ozone.sh to quickly start ozone. Contributed by Weiwei Yang.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 a3c75be58 -> 150ac9886


HDFS-11961. Ozone: Add start-ozone.sh to quickly start ozone. Contributed by Weiwei Yang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/150ac988
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/150ac988
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/150ac988

Branch: refs/heads/HDFS-7240
Commit: 150ac988638cda999ef0d5598f24eb813e385533
Parents: a3c75be
Author: Anu Engineer <ae...@apache.org>
Authored: Thu Jun 15 10:02:59 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Thu Jun 15 10:02:59 2017 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/bin/start-ozone.sh     | 94 ++++++++++++++++++++
 .../org/apache/hadoop/hdfs/tools/GetConf.java   | 49 +++++++++-
 2 files changed, 139 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/150ac988/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-ozone.sh
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-ozone.sh b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-ozone.sh
new file mode 100644
index 0000000..5291f57
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-ozone.sh
@@ -0,0 +1,94 @@
+#!/usr/bin/env bash
+
+# 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.
+
+# Start hadoop hdfs and ozone daemons.
+# Run this on master node.
+
+function hadoop_usage
+{
+  echo "Usage: start-ozone.sh"
+}
+
+this="${BASH_SOURCE-$0}"
+bin=$(cd -P -- "$(dirname -- "${this}")" >/dev/null && pwd -P)
+
+# let's locate libexec...
+if [[ -n "${HADOOP_HOME}" ]]; then
+  HADOOP_DEFAULT_LIBEXEC_DIR="${HADOOP_HOME}/libexec"
+else
+  HADOOP_DEFAULT_LIBEXEC_DIR="${bin}/../libexec"
+fi
+
+HADOOP_LIBEXEC_DIR="${HADOOP_LIBEXEC_DIR:-$HADOOP_DEFAULT_LIBEXEC_DIR}"
+# shellcheck disable=SC2034
+HADOOP_NEW_CONFIG=true
+if [[ -f "${HADOOP_LIBEXEC_DIR}/hdfs-config.sh" ]]; then
+  # shellcheck disable=SC1090
+  . "${HADOOP_LIBEXEC_DIR}/hdfs-config.sh"
+else
+  echo "ERROR: Cannot execute ${HADOOP_LIBEXEC_DIR}/hdfs-config.sh." 2>&1
+  exit 1
+fi
+
+#---------------------------------------------------------
+# Check if ozone is enabled
+OZONE_ENABLED=$("${HADOOP_HDFS_HOME}/bin/hdfs" getconf -confKey ozone.enabled | tr '[:upper:]' '[:lower:]' 2>&-)
+if [[ "${OZONE_ENABLED}" != "true" ]]; then
+  echo "Operation is not supported because ozone is not enabled."
+  exit -1
+fi
+
+#---------------------------------------------------------
+# Start hdfs before starting ozone daemons
+if [[ -f "${bin}/start-dfs.sh" ]]; then
+  "${bin}/start-dfs.sh"
+else
+  echo "ERROR: Cannot execute ${bin}/start-dfs.sh." 2>&1
+  exit 1
+fi
+
+#---------------------------------------------------------
+# Ozone keyspacemanager nodes
+KSM_NODES=$("${HADOOP_HDFS_HOME}/bin/hdfs" getconf -keyspacemanagers 2>/dev/null)
+echo "Starting key space manager nodes [${KSM_NODES}]"
+if [[ "${KSM_NODES}" == "0.0.0.0" ]]; then
+  KSM_NODES=$(hostname)
+fi
+
+hadoop_uservar_su hdfs ksm "${HADOOP_HDFS_HOME}/bin/hdfs" \
+  --workers \
+  --config "${HADOOP_CONF_DIR}" \
+  --hostnames "${KSM_NODES}" \
+  --daemon start \
+  ksm
+
+HADOOP_JUMBO_RETCOUNTER=$?
+
+#---------------------------------------------------------
+# Ozone storagecontainermanager nodes
+SCM_NODES=$("${HADOOP_HDFS_HOME}/bin/hdfs" getconf -storagecontainermanagers 2>/dev/null)
+echo "Starting storage container manager nodes [${SCM_NODES}]"
+hadoop_uservar_su hdfs scm "${HADOOP_HDFS_HOME}/bin/hdfs" \
+  --workers \
+  --config "${HADOOP_CONF_DIR}" \
+  --hostnames "${SCM_NODES}" \
+  --daemon start \
+  scm
+
+(( HADOOP_JUMBO_RETCOUNTER=HADOOP_JUMBO_RETCOUNTER + $? ))
+
+exit ${HADOOP_JUMBO_RETCOUNTER}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/150ac988/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
index e6cf16c..725b20f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
@@ -25,6 +25,7 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Collection;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
@@ -33,6 +34,8 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.DFSUtil.ConfiguredNNAddress;
+import org.apache.hadoop.ozone.OzoneClientUtils;
+import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
@@ -75,6 +78,10 @@ public class GetConf extends Configured implements Tool {
         "gets the exclude file path that defines the datanodes " +
         "that need to decommissioned."),
     NNRPCADDRESSES("-nnRpcAddresses", "gets the namenode rpc addresses"),
+    KEYSPACEMANAGER("-keyspacemanagers",
+        "gets list of ozone key space manager nodes in the cluster"),
+    STORAGECONTAINERMANAGER("-storagecontainermanagers",
+        "gets list of ozone storage container manager nodes in the cluster"),
     CONFKEY("-confKey [key]", "gets a specific key from the configuration");
 
     private static final Map<String, CommandHandler> map;
@@ -92,6 +99,10 @@ public class GetConf extends Configured implements Tool {
           new CommandHandler(DFSConfigKeys.DFS_HOSTS_EXCLUDE));
       map.put(StringUtils.toLowerCase(NNRPCADDRESSES.getName()),
           new NNRpcAddressesCommandHandler());
+      map.put(StringUtils.toLowerCase(KEYSPACEMANAGER.getName()),
+          new KeySpaceManagersCommandHandler());
+      map.put(StringUtils.toLowerCase(STORAGECONTAINERMANAGER.getName()),
+          new StorageContainerManagersCommandHandler());
       map.put(StringUtils.toLowerCase(CONFKEY.getName()),
           new PrintConfKeyCommandHandler());
     }
@@ -207,12 +218,39 @@ public class GetConf extends Configured implements Tool {
    * Handler for {@link Command#SECONDARY}
    */
   static class SecondaryNameNodesCommandHandler extends CommandHandler {
-    @Override
-    public int doWorkInternal(GetConf tool, String []args) throws IOException {
+    @Override public int doWorkInternal(GetConf tool, String[] args)
+        throws IOException {
       tool.printMap(DFSUtil.getSecondaryNameNodeAddresses(tool.getConf()));
       return 0;
     }
   }
+
+  /**
+   * Handler for {@link Command#STORAGECONTAINERMANAGER}.
+   */
+  static class StorageContainerManagersCommandHandler extends CommandHandler {
+    @Override
+    public int doWorkInternal(GetConf tool, String[] args) throws IOException {
+      Collection<InetSocketAddress> addresses =
+          OzoneClientUtils.getSCMAddresses(tool.getConf());
+      for (InetSocketAddress addr : addresses) {
+        tool.printOut(addr.getHostName());
+      }
+      return 0;
+    }
+  }
+
+  /**
+   * Handler for {@link Command#KEYSPACEMANAGER}.
+   */
+  static class KeySpaceManagersCommandHandler extends CommandHandler {
+    @Override
+    public int doWorkInternal(GetConf tool, String[] args) throws IOException {
+      tool.printOut(OzoneClientUtils.getKsmAddress(tool.getConf())
+          .getHostName());
+      return 0;
+    }
+  }
   
   /**
    * Handler for {@link Command#NNRPCADDRESSES}
@@ -328,8 +366,11 @@ public class GetConf extends Configured implements Tool {
     if (DFSUtil.parseHelpArgument(args, USAGE, System.out, true)) {
       System.exit(0);
     }
-    
-    int res = ToolRunner.run(new GetConf(new HdfsConfiguration()), args);
+
+    Configuration conf = new Configuration();
+    conf.addResource(new HdfsConfiguration());
+    conf.addResource(new OzoneConfiguration());
+    int res = ToolRunner.run(new GetConf(conf), args);
     System.exit(res);
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org