You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by sm...@apache.org on 2014/02/04 20:22:58 UTC

[2/2] git commit: AMBARI-4499. Add support for marking hbase RS instances as DRAINING before decommission

AMBARI-4499. Add support for marking hbase RS instances as DRAINING before decommission


Project: http://git-wip-us.apache.org/repos/asf/ambari/repo
Commit: http://git-wip-us.apache.org/repos/asf/ambari/commit/1d80ca0b
Tree: http://git-wip-us.apache.org/repos/asf/ambari/tree/1d80ca0b
Diff: http://git-wip-us.apache.org/repos/asf/ambari/diff/1d80ca0b

Branch: refs/heads/trunk
Commit: 1d80ca0bc3f7eb82f0f2bbd2625a72ba7b64c1cc
Parents: aaea21f
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Tue Feb 4 11:22:44 2014 -0800
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Tue Feb 4 11:22:44 2014 -0800

----------------------------------------------------------------------
 .../AmbariCustomCommandExecutionHelper.java     |  10 +-
 .../HBASE/package/files/draining_servers.rb     | 164 +++++
 .../HBASE/package/scripts/hbase_decommission.py |  27 +-
 .../services/HBASE/package/scripts/params.py    |  11 +-
 .../HBASE/package/files/draining_servers.rb     | 164 +++++
 .../HBASE/package/scripts/hbase_decommission.py |  27 +-
 .../services/HBASE/package/scripts/params.py    |   8 +-
 .../AmbariManagementControllerTest.java         | 114 ++++
 .../stacks/1.3.2/HBASE/test_hbase_master.py     |  37 ++
 .../1.3.2/configs/default.hbasedecom.json       | 444 ++++++++++++++
 .../stacks/2.0.6/HBASE/test_hbase_master.py     |  37 ++
 .../2.0.6/configs/default.hbasedecom.json       | 612 +++++++++++++++++++
 .../HDP/2.0.7/services/HBASE/metainfo.xml       |   8 +
 13 files changed, 1648 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/1d80ca0b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java
index 55022cf..602bf57 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java
@@ -502,9 +502,16 @@ public class AmbariCustomCommandExecutionHelper {
       throw new AmbariException("Component " + slaveCompType + " is not supported for decommissioning.");
     }
 
+    String isDrainOnlyRequest = request.getParameters().get(HBASE_MARK_DRAINING_ONLY);
+    if (isDrainOnlyRequest != null && !slaveCompType.equals(Role.HBASE_REGIONSERVER.name())) {
+      throw new AmbariException(HBASE_MARK_DRAINING_ONLY + " is not a valid parameter for " + masterCompType);
+    }
+
     // Decommission only if the sch is in state STARTED or INSTALLED
     for (ServiceComponentHost sch : svcComponents.get(slaveCompType).getServiceComponentHosts().values()) {
-      if (excludedHosts.contains(sch.getHostName()) && sch.getState() != State.STARTED) {
+      if (excludedHosts.contains(sch.getHostName())
+          && !"true".equals(isDrainOnlyRequest)
+          && sch.getState() != State.STARTED) {
         throw new AmbariException("Component " + slaveCompType + " on host " + sch.getHostName() + " cannot be " +
             "decommissioned as its not in STARTED state. Aborting the whole request.");
       }
@@ -541,7 +548,6 @@ public class AmbariCustomCommandExecutionHelper {
     if (serviceName.equals(Service.Type.HBASE.name()) && listOfExcludedHosts.size() > 0) {
       commandParams = new HashMap<String, String>();
       commandParams.put(DECOM_EXCLUDED_HOSTS, StringUtils.join(listOfExcludedHosts, ','));
-      String isDrainOnlyRequest = request.getParameters().get(HBASE_MARK_DRAINING_ONLY);
       if (isDrainOnlyRequest != null) {
         if (isDrainOnlyRequest.equals("true") || isDrainOnlyRequest.equals("false")) {
           commandParams.put(HBASE_MARK_DRAINING_ONLY, isDrainOnlyRequest);

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d80ca0b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/package/files/draining_servers.rb
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/package/files/draining_servers.rb b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/package/files/draining_servers.rb
new file mode 100644
index 0000000..5bcb5b6
--- /dev/null
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/package/files/draining_servers.rb
@@ -0,0 +1,164 @@
+#
+# 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.
+#
+
+# Add or remove servers from draining mode via zookeeper 
+
+require 'optparse'
+include Java
+
+import org.apache.hadoop.hbase.HBaseConfiguration
+import org.apache.hadoop.hbase.client.HBaseAdmin
+import org.apache.hadoop.hbase.zookeeper.ZKUtil
+import org.apache.commons.logging.Log
+import org.apache.commons.logging.LogFactory
+
+# Name of this script
+NAME = "draining_servers"
+
+# Do command-line parsing
+options = {}
+optparse = OptionParser.new do |opts|
+  opts.banner = "Usage: ./hbase org.jruby.Main #{NAME}.rb [options] add|remove|list <hostname>|<host:port>|<servername> ..."
+  opts.separator 'Add remove or list servers in draining mode. Can accept either hostname to drain all region servers' +
+                 'in that host, a host:port pair or a host,port,startCode triplet. More than one server can be given separated by space'
+  opts.on('-h', '--help', 'Display usage information') do
+    puts opts
+    exit
+  end
+  options[:debug] = false
+  opts.on('-d', '--debug', 'Display extra debug logging') do
+    options[:debug] = true
+  end
+end
+optparse.parse!
+
+# Return array of servernames where servername is hostname+port+startcode
+# comma-delimited
+def getServers(admin)
+  serverInfos = admin.getClusterStatus().getServerInfo()
+  servers = []
+  for server in serverInfos
+    servers << server.getServerName()
+  end
+  return servers
+end
+
+def getServerNames(hostOrServers, config)
+  ret = []
+  
+  for hostOrServer in hostOrServers
+    # check whether it is already serverName. No need to connect to cluster
+    parts = hostOrServer.split(',')
+    if parts.size() == 3
+      ret << hostOrServer
+    else 
+      admin = HBaseAdmin.new(config) if not admin
+      servers = getServers(admin)
+
+      hostOrServer = hostOrServer.gsub(/:/, ",")
+      for server in servers 
+        ret << server if server.start_with?(hostOrServer)
+      end
+    end
+  end
+  
+  admin.close() if admin
+  return ret
+end
+
+def addServers(options, hostOrServers)
+  config = HBaseConfiguration.create()
+  servers = getServerNames(hostOrServers, config)
+  
+  zkw = org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.new(config, "draining_servers", nil)
+  parentZnode = zkw.drainingZNode
+  
+  begin
+    for server in servers
+      node = ZKUtil.joinZNode(parentZnode, server)
+      ZKUtil.createAndFailSilent(zkw, node)
+    end
+  ensure
+    zkw.close()
+  end
+end
+
+def removeServers(options, hostOrServers)
+  config = HBaseConfiguration.create()
+  servers = getServerNames(hostOrServers, config)
+  
+  zkw = org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.new(config, "draining_servers", nil)
+  parentZnode = zkw.drainingZNode
+  
+  begin
+    for server in servers
+      node = ZKUtil.joinZNode(parentZnode, server)
+      ZKUtil.deleteNodeFailSilent(zkw, node)
+    end
+  ensure
+    zkw.close()
+  end
+end
+
+# list servers in draining mode
+def listServers(options)
+  config = HBaseConfiguration.create()
+  
+  zkw = org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.new(config, "draining_servers", nil)
+  parentZnode = zkw.drainingZNode
+
+  servers = ZKUtil.listChildrenNoWatch(zkw, parentZnode)
+  servers.each {|server| puts server}
+end
+
+hostOrServers = ARGV[1..ARGV.size()]
+
+# Create a logger and disable the DEBUG-level annoying client logging
+def configureLogging(options)
+  apacheLogger = LogFactory.getLog(NAME)
+  # Configure log4j to not spew so much
+  unless (options[:debug]) 
+    logger = org.apache.log4j.Logger.getLogger("org.apache.hadoop.hbase")
+    logger.setLevel(org.apache.log4j.Level::WARN)
+    logger = org.apache.log4j.Logger.getLogger("org.apache.zookeeper")
+    logger.setLevel(org.apache.log4j.Level::WARN)
+  end
+  return apacheLogger
+end
+
+# Create a logger and save it to ruby global
+$LOG = configureLogging(options)
+case ARGV[0]
+  when 'add'
+    if ARGV.length < 2
+      puts optparse
+      exit 1
+    end
+    addServers(options, hostOrServers)
+  when 'remove'
+    if ARGV.length < 2
+      puts optparse
+      exit 1
+    end
+    removeServers(options, hostOrServers)
+  when 'list'
+    listServers(options)
+  else
+    puts optparse
+    exit 3
+end

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d80ca0b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/package/scripts/hbase_decommission.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/package/scripts/hbase_decommission.py b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/package/scripts/hbase_decommission.py
index dba89fa..0bec157 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/package/scripts/hbase_decommission.py
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/package/scripts/hbase_decommission.py
@@ -25,22 +25,41 @@ def hbase_decommission(env):
   import params
 
   env.set_params(params)
+  kinit_cmd = params.kinit_cmd
+
+  File(params.region_drainer,
+       content=StaticFile("draining_servers.rb"),
+       mode=0755
+  )
 
   if params.hbase_drain_only == True:
-    print "TBD: Remove host from draining"
+    hosts = params.hbase_excluded_hosts.split(",")
+    for host in hosts:
+      if host:
+        regiondrainer_cmd = format(
+          "{kinit_cmd} {hbase_cmd} --config {conf_dir} org.jruby.Main {region_drainer} remove {host}")
+        Execute(regiondrainer_cmd,
+                user=params.hbase_user,
+                logoutput=True
+        )
+        pass
     pass
 
   else:
 
-    kinit_cmd = format("{kinit_path_local} -kt {hbase_user_keytab} {hbase_user};") if params.security_enabled else ""
-
     hosts = params.hbase_excluded_hosts.split(",")
     for host in hosts:
       if host:
-        print "TBD: Add host to draining"
+        regiondrainer_cmd = format(
+          "{kinit_cmd} {hbase_cmd} --config {conf_dir} org.jruby.Main {region_drainer} add {host}")
         regionmover_cmd = format(
           "{kinit_cmd} {hbase_cmd} --config {conf_dir} org.jruby.Main {region_mover} unload {host}")
 
+        Execute(regiondrainer_cmd,
+                user=params.hbase_user,
+                logoutput=True
+        )
+
         Execute(regionmover_cmd,
                 user=params.hbase_user,
                 logoutput=True

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d80ca0b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/package/scripts/params.py b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/package/scripts/params.py
index 1d57fc9..5c6576a 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/package/scripts/params.py
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/package/scripts/params.py
@@ -28,10 +28,10 @@ config = Script.get_config()
 conf_dir = "/etc/hbase/conf"
 daemon_script = "/usr/lib/hbase/bin/hbase-daemon.sh"
 region_mover = "/usr/lib/hbase/bin/region_mover.rb"
-region_drainer = "/usr/lib/hbase/bin/region_drainer.rb"
+region_drainer = "/usr/lib/hbase/bin/draining_servers.rb"
 hbase_cmd = "/usr/lib/hbase/bin/hbase"
 hbase_excluded_hosts = default("/commandParams/excluded_hosts", "")
-hbase_drain_only = default("/commandParams/mark_draining_only", "")
+hbase_drain_only = default("/commandParams/mark_draining_only", False)
 
 hbase_user = config['configurations']['global']['hbase_user']
 smokeuser = config['configurations']['global']['smokeuser']
@@ -74,7 +74,8 @@ if security_enabled:
   _kerberos_domain = config['configurations']['global']['kerberos_domain']
   _master_principal_name = config['configurations']['global']['hbase_master_principal_name']
   _regionserver_primary_name = config['configurations']['global']['hbase_regionserver_primary_name']
-  
+
+
   if _use_hostname_in_principal:
     master_jaas_princ = format("{_master_primary_name}/{_hostname}@{_kerberos_domain}")
     regionserver_jaas_princ = format("{_regionserver_primary_name}/{_hostname}@{_kerberos_domain}")
@@ -87,6 +88,10 @@ regionserver_keytab_path = config['configurations']['hbase-site']['hbase.regions
 smoke_user_keytab = config['configurations']['global']['smokeuser_keytab']
 hbase_user_keytab = config['configurations']['global']['hbase_user_keytab']
 kinit_path_local = functions.get_kinit_path([default("kinit_path_local",None), "/usr/bin", "/usr/kerberos/bin", "/usr/sbin"])
+if security_enabled:
+  kinit_cmd = format("{kinit_path_local} -kt {hbase_user_keytab} {hbase_user};")
+else:
+  kinit_cmd = ""
 
 #log4j.properties
 if ('hbase-log4j' in config['configurations']):

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d80ca0b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/package/files/draining_servers.rb
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/package/files/draining_servers.rb b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/package/files/draining_servers.rb
new file mode 100644
index 0000000..5bcb5b6
--- /dev/null
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/package/files/draining_servers.rb
@@ -0,0 +1,164 @@
+#
+# 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.
+#
+
+# Add or remove servers from draining mode via zookeeper 
+
+require 'optparse'
+include Java
+
+import org.apache.hadoop.hbase.HBaseConfiguration
+import org.apache.hadoop.hbase.client.HBaseAdmin
+import org.apache.hadoop.hbase.zookeeper.ZKUtil
+import org.apache.commons.logging.Log
+import org.apache.commons.logging.LogFactory
+
+# Name of this script
+NAME = "draining_servers"
+
+# Do command-line parsing
+options = {}
+optparse = OptionParser.new do |opts|
+  opts.banner = "Usage: ./hbase org.jruby.Main #{NAME}.rb [options] add|remove|list <hostname>|<host:port>|<servername> ..."
+  opts.separator 'Add remove or list servers in draining mode. Can accept either hostname to drain all region servers' +
+                 'in that host, a host:port pair or a host,port,startCode triplet. More than one server can be given separated by space'
+  opts.on('-h', '--help', 'Display usage information') do
+    puts opts
+    exit
+  end
+  options[:debug] = false
+  opts.on('-d', '--debug', 'Display extra debug logging') do
+    options[:debug] = true
+  end
+end
+optparse.parse!
+
+# Return array of servernames where servername is hostname+port+startcode
+# comma-delimited
+def getServers(admin)
+  serverInfos = admin.getClusterStatus().getServerInfo()
+  servers = []
+  for server in serverInfos
+    servers << server.getServerName()
+  end
+  return servers
+end
+
+def getServerNames(hostOrServers, config)
+  ret = []
+  
+  for hostOrServer in hostOrServers
+    # check whether it is already serverName. No need to connect to cluster
+    parts = hostOrServer.split(',')
+    if parts.size() == 3
+      ret << hostOrServer
+    else 
+      admin = HBaseAdmin.new(config) if not admin
+      servers = getServers(admin)
+
+      hostOrServer = hostOrServer.gsub(/:/, ",")
+      for server in servers 
+        ret << server if server.start_with?(hostOrServer)
+      end
+    end
+  end
+  
+  admin.close() if admin
+  return ret
+end
+
+def addServers(options, hostOrServers)
+  config = HBaseConfiguration.create()
+  servers = getServerNames(hostOrServers, config)
+  
+  zkw = org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.new(config, "draining_servers", nil)
+  parentZnode = zkw.drainingZNode
+  
+  begin
+    for server in servers
+      node = ZKUtil.joinZNode(parentZnode, server)
+      ZKUtil.createAndFailSilent(zkw, node)
+    end
+  ensure
+    zkw.close()
+  end
+end
+
+def removeServers(options, hostOrServers)
+  config = HBaseConfiguration.create()
+  servers = getServerNames(hostOrServers, config)
+  
+  zkw = org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.new(config, "draining_servers", nil)
+  parentZnode = zkw.drainingZNode
+  
+  begin
+    for server in servers
+      node = ZKUtil.joinZNode(parentZnode, server)
+      ZKUtil.deleteNodeFailSilent(zkw, node)
+    end
+  ensure
+    zkw.close()
+  end
+end
+
+# list servers in draining mode
+def listServers(options)
+  config = HBaseConfiguration.create()
+  
+  zkw = org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.new(config, "draining_servers", nil)
+  parentZnode = zkw.drainingZNode
+
+  servers = ZKUtil.listChildrenNoWatch(zkw, parentZnode)
+  servers.each {|server| puts server}
+end
+
+hostOrServers = ARGV[1..ARGV.size()]
+
+# Create a logger and disable the DEBUG-level annoying client logging
+def configureLogging(options)
+  apacheLogger = LogFactory.getLog(NAME)
+  # Configure log4j to not spew so much
+  unless (options[:debug]) 
+    logger = org.apache.log4j.Logger.getLogger("org.apache.hadoop.hbase")
+    logger.setLevel(org.apache.log4j.Level::WARN)
+    logger = org.apache.log4j.Logger.getLogger("org.apache.zookeeper")
+    logger.setLevel(org.apache.log4j.Level::WARN)
+  end
+  return apacheLogger
+end
+
+# Create a logger and save it to ruby global
+$LOG = configureLogging(options)
+case ARGV[0]
+  when 'add'
+    if ARGV.length < 2
+      puts optparse
+      exit 1
+    end
+    addServers(options, hostOrServers)
+  when 'remove'
+    if ARGV.length < 2
+      puts optparse
+      exit 1
+    end
+    removeServers(options, hostOrServers)
+  when 'list'
+    listServers(options)
+  else
+    puts optparse
+    exit 3
+end

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d80ca0b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/package/scripts/hbase_decommission.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/package/scripts/hbase_decommission.py b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/package/scripts/hbase_decommission.py
index dba89fa..0bec157 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/package/scripts/hbase_decommission.py
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/package/scripts/hbase_decommission.py
@@ -25,22 +25,41 @@ def hbase_decommission(env):
   import params
 
   env.set_params(params)
+  kinit_cmd = params.kinit_cmd
+
+  File(params.region_drainer,
+       content=StaticFile("draining_servers.rb"),
+       mode=0755
+  )
 
   if params.hbase_drain_only == True:
-    print "TBD: Remove host from draining"
+    hosts = params.hbase_excluded_hosts.split(",")
+    for host in hosts:
+      if host:
+        regiondrainer_cmd = format(
+          "{kinit_cmd} {hbase_cmd} --config {conf_dir} org.jruby.Main {region_drainer} remove {host}")
+        Execute(regiondrainer_cmd,
+                user=params.hbase_user,
+                logoutput=True
+        )
+        pass
     pass
 
   else:
 
-    kinit_cmd = format("{kinit_path_local} -kt {hbase_user_keytab} {hbase_user};") if params.security_enabled else ""
-
     hosts = params.hbase_excluded_hosts.split(",")
     for host in hosts:
       if host:
-        print "TBD: Add host to draining"
+        regiondrainer_cmd = format(
+          "{kinit_cmd} {hbase_cmd} --config {conf_dir} org.jruby.Main {region_drainer} add {host}")
         regionmover_cmd = format(
           "{kinit_cmd} {hbase_cmd} --config {conf_dir} org.jruby.Main {region_mover} unload {host}")
 
+        Execute(regiondrainer_cmd,
+                user=params.hbase_user,
+                logoutput=True
+        )
+
         Execute(regionmover_cmd,
                 user=params.hbase_user,
                 logoutput=True

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d80ca0b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/package/scripts/params.py b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/package/scripts/params.py
index 7db6306..5e156cd 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/package/scripts/params.py
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/package/scripts/params.py
@@ -28,10 +28,10 @@ config = Script.get_config()
 conf_dir = "/etc/hbase/conf"
 daemon_script = "/usr/lib/hbase/bin/hbase-daemon.sh"
 region_mover = "/usr/lib/hbase/bin/region_mover.rb"
-region_drainer = "/usr/lib/hbase/bin/region_drainer.rb"
+region_drainer = "/usr/lib/hbase/bin/draining_servers.rb"
 hbase_cmd = "/usr/lib/hbase/bin/hbase"
 hbase_excluded_hosts = default("/commandParams/excluded_hosts", "")
-hbase_drain_only = default("/commandParams/mark_draining_only", "")
+hbase_drain_only = default("/commandParams/mark_draining_only", False)
 
 hbase_user = config['configurations']['global']['hbase_user']
 smokeuser = config['configurations']['global']['smokeuser']
@@ -87,6 +87,10 @@ regionserver_keytab_path = config['configurations']['hbase-site']['hbase.regions
 smoke_user_keytab = config['configurations']['global']['smokeuser_keytab']
 hbase_user_keytab = config['configurations']['global']['hbase_user_keytab']
 kinit_path_local = functions.get_kinit_path([default("kinit_path_local",None), "/usr/bin", "/usr/kerberos/bin", "/usr/sbin"])
+if security_enabled:
+  kinit_cmd = format("{kinit_path_local} -kt {hbase_user_keytab} {hbase_user};")
+else:
+  kinit_cmd = ""
 
 #log4j.properties
 if ('hbase-log4j' in config['configurations']):

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d80ca0b/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerTest.java
index 009e0af..f483068 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerTest.java
@@ -2291,6 +2291,112 @@ public class AmbariManagementControllerTest {
     }
   }
 
+  @Test
+  public void testHbaseDecommission() throws AmbariException {
+    String clusterName = "foo1";
+    createCluster(clusterName);
+    clusters.getCluster(clusterName)
+        .setDesiredStackVersion(new StackId("HDP-2.0.7"));
+    String serviceName = "HBASE";
+    createService(clusterName, serviceName, null);
+    String componentName1 = "HBASE_MASTER";
+    String componentName2 = "HBASE_REGIONSERVER";
+
+    createServiceComponent(clusterName, serviceName, componentName1,
+        State.INIT);
+    createServiceComponent(clusterName, serviceName, componentName2,
+        State.INIT);
+
+    String host1 = "h1";
+    clusters.addHost(host1);
+    clusters.getHost("h1").setOsType("centos5");
+    clusters.getHost("h1").setState(HostState.HEALTHY);
+    clusters.getHost("h1").persist();
+    String host2 = "h2";
+    clusters.addHost(host2);
+    clusters.getHost("h2").setOsType("centos6");
+    clusters.getHost("h2").setState(HostState.HEALTHY);
+    clusters.getHost("h2").persist();
+
+    clusters.mapHostToCluster(host1, clusterName);
+    clusters.mapHostToCluster(host2, clusterName);
+
+    createServiceComponentHost(clusterName, serviceName, componentName1,
+        host1, null);
+    createServiceComponentHost(clusterName, serviceName, componentName2,
+        host1, null);
+    createServiceComponentHost(clusterName, serviceName, componentName2,
+        host2, null);
+
+    // Install
+    installService(clusterName, serviceName, false, false);
+
+    // Start
+    startService(clusterName, serviceName, false, false);
+
+    Cluster cluster = clusters.getCluster(clusterName);
+    Service s = cluster.getService(serviceName);
+    Assert.assertEquals(State.STARTED, s.getDesiredState());
+    ServiceComponentHost scHost = s.getServiceComponent("HBASE_REGIONSERVER").getServiceComponentHost("h2");
+    Assert.assertEquals(HostComponentAdminState.INSERVICE, scHost.getComponentAdminState());
+
+    // Decommission one RS
+    Map<String, String> params = new HashMap<String, String>() {{
+      put("excluded_hosts", "h2");
+    }};
+    ExecuteActionRequest request = new ExecuteActionRequest(clusterName, "DECOMMISSION", null, "HBASE", "HBASE_MASTER",
+        null, params);
+
+    Map<String, String> requestProperties = new HashMap<String, String>();
+    requestProperties.put(REQUEST_CONTEXT_PROPERTY, "Called from a test");
+
+    RequestStatusResponse response = controller.createAction(request,
+        requestProperties);
+
+    List<HostRoleCommand> storedTasks = actionDB.getRequestTasks(response.getRequestId());
+    ExecutionCommand execCmd = storedTasks.get(0).getExecutionCommandWrapper
+        ().getExecutionCommand();
+    Assert.assertNotNull(storedTasks);
+    Assert.assertEquals(1, storedTasks.size());
+    Assert.assertEquals(HostComponentAdminState.DECOMMISSIONED, scHost.getComponentAdminState());
+    Assert.assertEquals(PassiveState.PASSIVE, scHost.getPassiveState());
+    HostRoleCommand command = storedTasks.get(0);
+    Assert.assertEquals(Role.HBASE_MASTER, command.getRole());
+    Assert.assertEquals(RoleCommand.CUSTOM_COMMAND, command.getRoleCommand());
+    Map<String, Set<String>> cInfo = execCmd.getClusterHostInfo();
+    Assert.assertTrue(cInfo.containsKey("decom_hbase_rs_hosts"));
+    Assert.assertTrue(cInfo.get("decom_hbase_rs_hosts").size() == 1);
+    Assert.assertEquals("h2",
+        cInfo.get("all_hosts").toArray()[Integer.parseInt(cInfo.get("decom_hbase_rs_hosts").iterator().next())]);
+    Assert.assertEquals("DECOMMISSION", execCmd.getHostLevelParams().get("custom_command"));
+
+    // RS stops
+    s.getServiceComponent("HBASE_REGIONSERVER").getServiceComponentHost("h2").setState(State.INSTALLED);
+
+    // Remove RS from draining
+    params = new
+        HashMap<String, String>() {{
+          put("excluded_hosts", "h2");
+          put("mark_draining_only", "true");
+          put("slave_type", "HBASE_REGIONSERVER");
+        }};
+    request = new ExecuteActionRequest(clusterName, "DECOMMISSION", null, "HBASE", "HBASE_MASTER", null, params);
+
+    response = controller.createAction(request,
+        requestProperties);
+
+    storedTasks = actionDB.getRequestTasks(response.getRequestId());
+    execCmd = storedTasks.get(0).getExecutionCommandWrapper
+        ().getExecutionCommand();
+    Assert.assertNotNull(storedTasks);
+    Assert.assertEquals(1, storedTasks.size());
+    Assert.assertEquals(HostComponentAdminState.DECOMMISSIONED, scHost.getComponentAdminState());
+    Assert.assertEquals(PassiveState.PASSIVE, scHost.getPassiveState());
+    cInfo = execCmd.getClusterHostInfo();
+    Assert.assertTrue(cInfo.containsKey("decom_hbase_rs_hosts"));
+    Assert.assertEquals("DECOMMISSION", execCmd.getHostLevelParams().get("custom_command"));
+  }
+
   private Cluster setupClusterWithHosts(String clusterName, String stackId, List<String> hosts,
                                         String osType) throws AmbariException {
     ClusterRequest r = new ClusterRequest(null, clusterName, stackId, null);
@@ -3846,6 +3952,14 @@ public class AmbariManagementControllerTest {
     expectActionCreationErrorWithMessage(actionRequest, requestProperties,
         "Component DATANODE on host h1 cannot be decommissioned as its not in STARTED state");
 
+    params2 = new HashMap<String, String>() {{
+      put("excluded_hosts", "h1 ");
+      put("mark_draining_only", "true");
+    }};
+    actionRequest = new ExecuteActionRequest("c1", "DECOMMISSION", null, "HDFS", "NAMENODE", null, params2);
+    expectActionCreationErrorWithMessage(actionRequest, requestProperties,
+        "mark_draining_only is not a valid parameter for NAMENODE");
+
     controller.getAmbariMetaInfo().addActionDefinition(new ActionDefinition(
         "a1", ActionType.SYSTEM, "test,dirName", "", "", "Does file exist",
         TargetHostType.SPECIFIC, Short.valueOf("100")));

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d80ca0b/ambari-server/src/test/python/stacks/1.3.2/HBASE/test_hbase_master.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/1.3.2/HBASE/test_hbase_master.py b/ambari-server/src/test/python/stacks/1.3.2/HBASE/test_hbase_master.py
index 397503e..402d8aa 100644
--- a/ambari-server/src/test/python/stacks/1.3.2/HBASE/test_hbase_master.py
+++ b/ambari-server/src/test/python/stacks/1.3.2/HBASE/test_hbase_master.py
@@ -66,16 +66,45 @@ class TestHBaseMaster(RMFTestCase):
                        config_file="default.json"
     )
 
+    self.assertResourceCalled('File', '/usr/lib/hbase/bin/draining_servers.rb',
+                              content = StaticFile('draining_servers.rb'),
+                              mode = 0755,
+                              )
+    self.assertResourceCalled('Execute', ' /usr/lib/hbase/bin/hbase --config /etc/hbase/conf org.jruby.Main /usr/lib/hbase/bin/draining_servers.rb add host1',
+                              logoutput = True,
+                              user = 'hbase',
+                              )
     self.assertResourceCalled('Execute', ' /usr/lib/hbase/bin/hbase --config /etc/hbase/conf org.jruby.Main /usr/lib/hbase/bin/region_mover.rb unload host1',
                               logoutput = True,
                               user = 'hbase',
                               )
+    self.assertResourceCalled('Execute', ' /usr/lib/hbase/bin/hbase --config /etc/hbase/conf org.jruby.Main /usr/lib/hbase/bin/draining_servers.rb add host2',
+                              logoutput = True,
+                              user = 'hbase',
+                              )
     self.assertResourceCalled('Execute', ' /usr/lib/hbase/bin/hbase --config /etc/hbase/conf org.jruby.Main /usr/lib/hbase/bin/region_mover.rb unload host2',
                               logoutput = True,
                               user = 'hbase',
                               )
     self.assertNoMoreResources()
 
+  def test_decom_default_draining_only(self):
+    self.executeScript("2.0.6/services/HBASE/package/scripts/hbase_master.py",
+                       classname = "HbaseMaster",
+                       command = "decommission",
+                       config_file="default.hbasedecom.json"
+    )
+
+    self.assertResourceCalled('File', '/usr/lib/hbase/bin/draining_servers.rb',
+                              content = StaticFile('draining_servers.rb'),
+                              mode = 0755,
+                              )
+    self.assertResourceCalled('Execute', ' /usr/lib/hbase/bin/hbase --config /etc/hbase/conf org.jruby.Main /usr/lib/hbase/bin/draining_servers.rb remove host1',
+                              logoutput = True,
+                              user = 'hbase',
+                              )
+    self.assertNoMoreResources()
+
   def test_configure_secured(self):
     self.executeScript("1.3.2/services/HBASE/package/scripts/hbase_master.py",
                    classname = "HbaseMaster",
@@ -120,6 +149,14 @@ class TestHBaseMaster(RMFTestCase):
                        config_file="secured.json"
     )
 
+    self.assertResourceCalled('File', '/usr/lib/hbase/bin/draining_servers.rb',
+                              content = StaticFile('draining_servers.rb'),
+                              mode = 0755,
+                              )
+    self.assertResourceCalled('Execute', '/usr/bin/kinit -kt /etc/security/keytabs/hbase.headless.keytab hbase; /usr/lib/hbase/bin/hbase --config /etc/hbase/conf org.jruby.Main /usr/lib/hbase/bin/draining_servers.rb add host1',
+                              logoutput = True,
+                              user = 'hbase',
+                              )
     self.assertResourceCalled('Execute', '/usr/bin/kinit -kt /etc/security/keytabs/hbase.headless.keytab hbase; /usr/lib/hbase/bin/hbase --config /etc/hbase/conf org.jruby.Main /usr/lib/hbase/bin/region_mover.rb unload host1',
                               logoutput = True,
                               user = 'hbase',

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d80ca0b/ambari-server/src/test/python/stacks/1.3.2/configs/default.hbasedecom.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/1.3.2/configs/default.hbasedecom.json b/ambari-server/src/test/python/stacks/1.3.2/configs/default.hbasedecom.json
new file mode 100644
index 0000000..6d12470
--- /dev/null
+++ b/ambari-server/src/test/python/stacks/1.3.2/configs/default.hbasedecom.json
@@ -0,0 +1,444 @@
+{
+    "roleCommand": "INSTALL", 
+    "clusterName": "cl1", 
+    "hostname": "c6402.ambari.apache.org", 
+    "hostLevelParams": {
+        "jdk_location": "http://c6401.ambari.apache.org:8080/resources/", 
+        "ambari_db_rca_password": "mapred", 
+        "ambari_db_rca_url": "jdbc:postgresql://c6401.ambari.apache.org/ambarirca", 
+        "jce_name": "UnlimitedJCEPolicyJDK7.zip", 
+        "oracle_jdbc_url": "http://c6401.ambari.apache.org:8080/resources//ojdbc6.jar", 
+        "repo_info": "[{\"baseUrl\":\"http://public-repo-1.hortonworks.com/HDP/centos6/1.x/updates/1.3.3.0\",\"osType\":\"centos6\",\"repoId\":\"HDP-1.3.4\",\"repoName\":\"HDP\",\"defaultBaseUrl\":\"http://public-repo-1.hortonworks.com/HDP/centos6/1.x/updates/1.3.3.0\"}]", 
+        "package_list": "[{\"type\":\"rpm\",\"name\":\"lzo\"},{\"type\":\"rpm\",\"name\":\"hadoop\"},{\"type\":\"rpm\",\"name\":\"hadoop-libhdfs\"},{\"type\":\"rpm\",\"name\":\"hadoop-native\"},{\"type\":\"rpm\",\"name\":\"hadoop-pipes\"},{\"type\":\"rpm\",\"name\":\"hadoop-sbin\"},{\"type\":\"rpm\",\"name\":\"hadoop-lzo\"},{\"type\":\"rpm\",\"name\":\"hadoop-lzo-native\"},{\"type\":\"rpm\",\"name\":\"snappy\"},{\"type\":\"rpm\",\"name\":\"snappy-devel\"},{\"type\":\"rpm\",\"name\":\"ambari-log4j\"}]", 
+        "stack_version": "1.3.4", 
+        "stack_name": "HDP", 
+        "db_name": "ambari", 
+        "ambari_db_rca_driver": "org.postgresql.Driver", 
+        "jdk_name": "jdk-7u45-linux-x64.tar.gz", 
+        "ambari_db_rca_username": "mapred", 
+        "java_home": "/usr/jdk64/jdk1.7.0_45", 
+        "mysql_jdbc_url": "http://c6401.ambari.apache.org:8080/resources//mysql-connector-java.jar"
+    }, 
+    "commandType": "EXECUTION_COMMAND", 
+    "roleParams": {}, 
+    "serviceName": "HDFS", 
+    "role": "DATANODE", 
+    "commandParams": {
+        "command_timeout": "600", 
+        "service_package_folder": "HDFS",
+        "script_type": "PYTHON", 
+        "schema_version": "2.0", 
+        "script": "scripts/datanode.py",
+        "excluded_hosts": "host1,host2"
+    }, 
+    "taskId": 18, 
+    "public_hostname": "c6402.ambari.apache.org", 
+    "configurations": {
+        "mapred-site": {
+            "ambari.mapred.child.java.opts.memory": "768", 
+            "mapred.job.reduce.input.buffer.percent": "0.0", 
+            "mapred.job.map.memory.mb": "1536", 
+            "mapred.output.compression.type": "BLOCK", 
+            "mapred.jobtracker.maxtasks.per.job": "-1", 
+            "mapred.hosts": "/etc/hadoop/conf/mapred.include", 
+            "mapred.map.output.compression.codec": "org.apache.hadoop.io.compress.SnappyCodec", 
+            "mapred.child.root.logger": "INFO,TLA", 
+            "mapred.tasktracker.tasks.sleeptime-before-sigkill": "250", 
+            "io.sort.spill.percent": "0.9", 
+            "mapred.reduce.parallel.copies": "30", 
+            "mapred.userlog.retain.hours": "24", 
+            "mapred.reduce.tasks.speculative.execution": "false", 
+            "io.sort.mb": "200", 
+            "mapreduce.cluster.administrators": " hadoop", 
+            "mapred.jobtracker.blacklist.fault-timeout-window": "180", 
+            "mapred.job.tracker.history.completed.location": "/mapred/history/done", 
+            "mapred.job.shuffle.input.buffer.percent": "0.7", 
+            "io.sort.record.percent": ".2", 
+            "mapred.cluster.max.reduce.memory.mb": "4096", 
+            "mapred.job.reuse.jvm.num.tasks": "1", 
+            "mapreduce.jobhistory.intermediate-done-dir": "/mr-history/tmp", 
+            "mapred.job.tracker.http.address": "c6402.ambari.apache.org:50030", 
+            "mapred.job.tracker.persist.jobstatus.hours": "1", 
+            "mapred.healthChecker.script.path": "/etc/hadoop/conf/health_check", 
+            "mapreduce.jobtracker.staging.root.dir": "/user", 
+            "mapred.job.shuffle.merge.percent": "0.66", 
+            "mapred.cluster.reduce.memory.mb": "2048", 
+            "mapred.job.tracker.persist.jobstatus.dir": "/mapred/jobstatus", 
+            "mapreduce.tasktracker.group": "hadoop", 
+            "mapred.tasktracker.map.tasks.maximum": "4", 
+            "mapred.child.java.opts": "-server -Xmx${ambari.mapred.child.java.opts.memory}m -Djava.net.preferIPv4Stack=true", 
+            "mapred.jobtracker.retirejob.check": "10000", 
+            "mapred.job.tracker": "c6402.ambari.apache.org:50300", 
+            "mapreduce.history.server.embedded": "false", 
+            "io.sort.factor": "100", 
+            "hadoop.job.history.user.location": "none", 
+            "mapreduce.reduce.input.limit": "10737418240", 
+            "mapred.reduce.slowstart.completed.maps": "0.05", 
+            "mapred.cluster.max.map.memory.mb": "6144", 
+            "mapreduce.history.server.http.address": "c6402.ambari.apache.org:51111", 
+            "mapred.jobtracker.taskScheduler": "org.apache.hadoop.mapred.CapacityTaskScheduler", 
+            "mapred.max.tracker.blacklists": "16", 
+            "mapred.local.dir": "/hadoop/mapred", 
+            "mapred.healthChecker.interval": "135000", 
+            "mapred.jobtracker.restart.recover": "false", 
+            "mapred.jobtracker.blacklist.fault-bucket-width": "15", 
+            "mapred.jobtracker.retirejob.interval": "21600000", 
+            "tasktracker.http.threads": "50", 
+            "mapred.job.tracker.persist.jobstatus.active": "false", 
+            "mapred.system.dir": "/mapred/system", 
+            "mapred.tasktracker.reduce.tasks.maximum": "2", 
+            "mapred.cluster.map.memory.mb": "1536", 
+            "mapred.hosts.exclude": "/etc/hadoop/conf/mapred.exclude", 
+            "mapred.queue.names": "default", 
+            "mapreduce.jobhistory.webapp.address": "c6402.ambari.apache.org:19888", 
+            "mapreduce.fileoutputcommitter.marksuccessfuljobs": "false", 
+            "mapred.job.reduce.memory.mb": "2048", 
+            "mapreduce.jobhistory.done-dir": "/mr-history/done", 
+            "mapred.healthChecker.script.timeout": "60000", 
+            "jetty.connector": "org.mortbay.jetty.nio.SelectChannelConnector", 
+            "mapreduce.jobtracker.split.metainfo.maxsize": "50000000", 
+            "mapred.job.tracker.handler.count": "50", 
+            "mapred.inmem.merge.threshold": "1000", 
+            "mapred.task.tracker.task-controller": "org.apache.hadoop.mapred.DefaultTaskController", 
+            "mapred.jobtracker.completeuserjobs.maximum": "0", 
+            "mapred.task.timeout": "600000", 
+            "mapred.map.tasks.speculative.execution": "false"
+        }, 
+        "oozie-site": {
+            "oozie.service.PurgeService.purge.interval": "3600", 
+            "oozie.service.CallableQueueService.queue.size": "1000", 
+            "oozie.service.SchemaService.wf.ext.schemas": "shell-action-0.1.xsd,email-action-0.1.xsd,hive-action-0.2.xsd,sqoop-action-0.2.xsd,ssh-action-0.1.xsd,distcp-action-0.1.xsd", 
+            "oozie.service.JPAService.jdbc.url": "jdbc:derby:${oozie.data.dir}/${oozie.db.schema.name}-db;create=true", 
+            "oozie.service.HadoopAccessorService.nameNode.whitelist": " ", 
+            "use.system.libpath.for.mapreduce.and.pig.jobs": "false", 
+            "oozie.service.JPAService.create.db.schema": "false", 
+            "oozie.authentication.kerberos.name.rules": "DEFAULT", 
+            "oozie.service.ActionService.executor.ext.classes": "org.apache.oozie.action.email.EmailActionExecutor,\norg.apache.oozie.action.hadoop.HiveActionExecutor,\norg.apache.oozie.action.hadoop.ShellActionExecutor,\norg.apache.oozie.action.hadoop.SqoopActionExecutor,\norg.apache.oozie.action.hadoop.DistcpActionExecutor", 
+            "oozie.service.AuthorizationService.authorization.enabled": "true", 
+            "oozie.base.url": "http://c6402.ambari.apache.org:11000/oozie", 
+            "oozie.service.JPAService.jdbc.password": "q", 
+            "oozie.service.coord.normal.default.timeout": "120", 
+            "oozie.service.JPAService.pool.max.active.conn": "10", 
+            "oozie.service.PurgeService.older.than": "30", 
+            "oozie.db.schema.name": "oozie", 
+            "oozie.service.HadoopAccessorService.hadoop.configurations": "*=/etc/hadoop/conf", 
+            "oozie.service.HadoopAccessorService.jobTracker.whitelist": " ", 
+            "oozie.service.CallableQueueService.callable.concurrency": "3", 
+            "oozie.service.JPAService.jdbc.username": "oozie", 
+            "oozie.service.CallableQueueService.threads": "10", 
+            "oozie.systemmode": "NORMAL", 
+            "oozie.service.WorkflowAppService.system.libpath": "/user/${user.name}/share/lib", 
+            "oozie.authentication.type": "simple", 
+            "oozie.service.JPAService.jdbc.driver": "org.apache.derby.jdbc.EmbeddedDriver", 
+            "oozie.system.id": "oozie-${user.name}"
+        }, 
+        "webhcat-site": {
+            "templeton.pig.path": "pig.tar.gz/pig/bin/pig", 
+            "templeton.exec.timeout": "60000", 
+            "templeton.override.enabled": "false", 
+            "templeton.jar": "/usr/lib/hcatalog/share/webhcat/svr/webhcat.jar", 
+            "templeton.zookeeper.hosts": "c6401.ambari.apache.org:2181", 
+            "templeton.hive.properties": "hive.metastore.local=false,hive.metastore.uris=thrift://c6402.ambari.apache.org:9083,hive.metastore.sasl.enabled=yes,hive.metastore.execute.setugi=true,hive.metastore.warehouse.dir=/apps/hive/warehouse", 
+            "templeton.storage.class": "org.apache.hcatalog.templeton.tool.ZooKeeperStorage", 
+            "templeton.hive.archive": "hdfs:///apps/webhcat/hive.tar.gz", 
+            "templeton.streaming.jar": "hdfs:///apps/webhcat/hadoop-streaming.jar", 
+            "templeton.port": "50111", 
+            "templeton.libjars": "/usr/lib/zookeeper/zookeeper.jar", 
+            "templeton.hadoop": "/usr/bin/hadoop", 
+            "templeton.hive.path": "hive.tar.gz/hive/bin/hive", 
+            "templeton.hadoop.conf.dir": "/etc/hadoop/conf", 
+            "templeton.hcat": "/usr/bin/hcat", 
+            "templeton.pig.archive": "hdfs:///apps/webhcat/pig.tar.gz"
+        }, 
+        "global": {
+            "security_enabled": "false", 
+            "hbase_pid_dir": "/var/run/hbase", 
+            "proxyuser_group": "users", 
+            "zk_user": "zookeeper", 
+            "namenode_formatted_mark_dir": "/var/run/hadoop/hdfs/namenode/formatted/", 
+            "rrdcached_base_dir": "/var/lib/ganglia/rrds", 
+            "syncLimit": "5", 
+            "oozie_pid_dir": "/var/run/oozie", 
+            "hbase_regionserver_heapsize": "1024m", 
+            "dtnode_heapsize": "1024m", 
+            "jtnode_heapsize": "1024m", 
+            "hcat_log_dir": "/var/log/webhcat", 
+            "oozie_hostname": "c6402.ambari.apache.org", 
+            "hive_aux_jars_path": "/usr/lib/hcatalog/share/hcatalog/hcatalog-core.jar", 
+            "tickTime": "2000", 
+            "hive_ambari_database": "MySQL", 
+            "rca_enabled": "true", 
+            "namenode_heapsize": "1024m", 
+            "oozie_log_dir": "/var/log/oozie", 
+            "hive_jdbc_driver": "com.mysql.jdbc.Driver", 
+            "oozie_user": "oozie", 
+            "oozie_data_dir": "/hadoop/oozie/data", 
+            "ganglia_runtime_dir": "/var/run/ganglia/hdp", 
+            "lzo_enabled": "true", 
+            "namenode_opt_maxnewsize": "200m", 
+            "smokeuser": "ambari-qa", 
+            "hdfs_log_dir_prefix": "/var/log/hadoop", 
+            "hive_hostname": "c6402.ambari.apache.org", 
+            "hive_metastore_port": "9083", 
+            "hbase_master_heapsize": "1024m", 
+            "zk_data_dir": "/hadoop/zookeeper", 
+            "hcat_pid_dir": "/etc/run/webhcat", 
+            "oozie_jdbc_driver": "org.apache.derby.jdbc.EmbeddedDriver", 
+            "initLimit": "10", 
+            "hive_database_type": "mysql", 
+            "oozie_database": "New Derby Database", 
+            "zk_pid_dir": "/var/run/zookeeper", 
+            "user_group": "hadoop", 
+            "hive_user": "hive", 
+            "gmond_user": "nobody", 
+            "nagios_web_login": "nagiosadmin", 
+            "nagios_contact": "q@q.q", 
+            "hive_database": "New MySQL Database", 
+            "nagios_web_password": "q", 
+            "clientPort": "2181", 
+            "oozie_derby_database": "Derby", 
+            "snappy_enabled": "true", 
+            "ganglia_conf_dir": "/etc/ganglia/hdp", 
+            "hdfs_user": "hdfs", 
+            "hbase_user": "hbase", 
+            "oozie_database_type": "derby", 
+            "webhcat_user": "hcat", 
+            "zk_log_dir": "/var/log/zookeeper", 
+            "jtnode_opt_maxnewsize": "200m", 
+            "mysql_connector_url": "${download_url}/mysql-connector-java-5.1.18.zip", 
+            "gmetad_user": "nobody", 
+            "hive_log_dir": "/var/log/hive", 
+            "jtnode_opt_newsize": "200m", 
+            "namenode_opt_newsize": "200m", 
+            "mapred_user": "mapred", 
+            "nagios_group": "nagios", 
+            "hive_pid_dir": "/var/run/hive", 
+            "hcat_user": "hcat", 
+            "hadoop_heapsize": "1024", 
+            "hadoop_pid_dir_prefix": "/var/run/hadoop", 
+            "nagios_user": "nagios", 
+            "hbase_log_dir": "/var/log/hbase"
+        }, 
+        "hdfs-site": {
+            "dfs.namenode.avoid.write.stale.datanode": "true", 
+            "dfs.access.time.precision": "0", 
+            "ipc.server.max.response.size": "5242880", 
+            "dfs.web.ugi": "gopher,gopher", 
+            "dfs.support.append": "true", 
+            "dfs.cluster.administrators": " hdfs", 
+            "dfs.replication": "3", 
+            "ambari.dfs.datanode.http.port": "50075", 
+            "dfs.block.size": "134217728", 
+            "dfs.data.dir": "/hadoop/hdfs/data", 
+            "dfs.datanode.du.reserved": "1073741824", 
+            "dfs.webhdfs.enabled": "true", 
+            "dfs.namenode.handler.count": "100", 
+            "dfs.datanode.http.address": "0.0.0.0:${ambari.dfs.datanode.http.port}", 
+            "dfs.datanode.socket.write.timeout": "0", 
+            "ipc.server.read.threadpool.size": "5", 
+            "dfs.balance.bandwidthPerSec": "6250000", 
+            "dfs.datanode.address": "0.0.0.0:${ambari.dfs.datanode.port}", 
+            "dfs.blockreport.initialDelay": "120", 
+            "dfs.datanode.failed.volumes.tolerated": "0", 
+            "dfs.permissions.supergroup": "hdfs", 
+            "dfs.https.address": "c6401.ambari.apache.org:50470", 
+            "ambari.dfs.datanode.port": "50010", 
+            "dfs.namenode.avoid.read.stale.datanode": "true", 
+            "dfs.name.dir": "/hadoop/hdfs/namenode", 
+            "dfs.hosts": "/etc/hadoop/conf/dfs.include", 
+            "dfs.namenode.stale.datanode.interval": "30000", 
+            "dfs.heartbeat.interval": "3", 
+            "dfs.secondary.https.port": "50490", 
+            "dfs.permissions": "true", 
+            "dfs.datanode.ipc.address": "0.0.0.0:8010", 
+            "dfs.block.local-path-access.user": "hbase", 
+            "dfs.block.access.token.enable": "true", 
+            "dfs.datanode.data.dir.perm": "750", 
+            "dfs.secondary.http.address": "c6402.ambari.apache.org:50090", 
+            "dfs.http.address": "c6401.ambari.apache.org:50070", 
+            "dfs.https.port": "50070", 
+            "dfs.replication.max": "50", 
+            "dfs.datanode.max.xcievers": "4096", 
+            "dfs.namenode.write.stale.datanode.ratio": "1.0f", 
+            "dfs.hosts.exclude": "/etc/hadoop/conf/dfs.exclude", 
+            "dfs.datanode.du.pct": "0.85f", 
+            "dfs.safemode.threshold.pct": "1.0f", 
+            "dfs.umaskmode": "077"
+        }, 
+        "hbase-site": {
+            "hbase.client.keyvalue.maxsize": "10485760", 
+            "hbase.hstore.compactionThreshold": "3", 
+            "hbase.rootdir": "hdfs://c6401.ambari.apache.org:8020/apps/hbase/data", 
+            "hbase.regionserver.handler.count": "60", 
+            "dfs.client.read.shortcircuit": "true", 
+            "hbase.regionserver.global.memstore.lowerLimit": "0.38", 
+            "hbase.hregion.memstore.block.multiplier": "2", 
+            "hbase.hregion.memstore.flush.size": "134217728", 
+            "hbase.superuser": "hbase", 
+            "hbase.zookeeper.property.clientPort": "2181", 
+            "hbase.rpc.engine": "org.apache.hadoop.hbase.ipc.WritableRpcEngine", 
+            "hbase.regionserver.global.memstore.upperLimit": "0.4", 
+            "zookeeper.session.timeout": "60000", 
+            "hbase.tmp.dir": "/hadoop/hbase", 
+            "hbase.hregion.max.filesize": "10737418240", 
+            "hfile.block.cache.size": "0.40", 
+            "hbase.security.authentication": "simple", 
+            "hbase.zookeeper.quorum": "c6401.ambari.apache.org", 
+            "zookeeper.znode.parent": "/hbase-unsecure", 
+            "hbase.hstore.blockingStoreFiles": "10", 
+            "hbase.hregion.majorcompaction": "86400000", 
+            "hbase.security.authorization": "false", 
+            "hbase.cluster.distributed": "true", 
+            "hbase.hregion.memstore.mslab.enabled": "true", 
+            "hbase.client.scanner.caching": "100", 
+            "hbase.zookeeper.useMulti": "true"
+        }, 
+        "core-site": {
+            "io.serializations": "org.apache.hadoop.io.serializer.WritableSerialization", 
+            "hadoop.proxyuser.hcat.groups": "users", 
+            "fs.checkpoint.size": "67108864", 
+            "hadoop.proxyuser.oozie.groups": "users", 
+            "fs.default.name": "hdfs://c6401.ambari.apache.org:8020", 
+            "io.file.buffer.size": "131072", 
+            "hadoop.proxyuser.hive.groups": "users", 
+            "webinterface.private.actions": "false", 
+            "hadoop.proxyuser.oozie.hosts": "c6402.ambari.apache.org", 
+            "io.compression.codecs": "org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.DefaultCodec,com.hadoop.compression.lzo.LzoCodec,com.hadoop.compression.lzo.LzopCodec,org.apache.hadoop.io.compress.SnappyCodec", 
+            "hadoop.security.authentication": "simple", 
+            "fs.checkpoint.edits.dir": "${fs.checkpoint.dir}", 
+            "fs.checkpoint.dir": "/hadoop/hdfs/namesecondary", 
+            "fs.trash.interval": "360", 
+            "ipc.client.idlethreshold": "8000", 
+            "hadoop.proxyuser.hcat.hosts": "c6402.ambari.apache.org", 
+            "hadoop.proxyuser.hive.hosts": "c6402.ambari.apache.org", 
+            "io.compression.codec.lzo.class": "com.hadoop.compression.lzo.LzoCodec", 
+            "fs.checkpoint.period": "21600", 
+            "ipc.client.connection.maxidletime": "30000", 
+            "ipc.client.connect.max.retries": "50"
+        }, 
+        "hive-site": {
+            "hive.enforce.sorting": "true", 
+            "javax.jdo.option.ConnectionPassword": "q", 
+            "javax.jdo.option.ConnectionDriverName": "com.mysql.jdbc.Driver", 
+            "hive.optimize.bucketmapjoin.sortedmerge": "true", 
+            "fs.file.impl.disable.cache": "true", 
+            "hive.auto.convert.join.noconditionaltask": "true", 
+            "hive.map.aggr": "true", 
+            "hive.security.authorization.enabled": "false", 
+            "hive.optimize.reducededuplication.min.reducer": "1", 
+            "hive.optimize.bucketmapjoin": "true", 
+            "hive.metastore.uris": "thrift://c6402.ambari.apache.org:9083", 
+            "hive.mapjoin.bucket.cache.size": "10000", 
+            "hive.auto.convert.join.noconditionaltask.size": "1000000000", 
+            "javax.jdo.option.ConnectionUserName": "hive", 
+            "hive.metastore.cache.pinobjtypes": "Table,Database,Type,FieldSchema,Order", 
+            "hive.metastore.warehouse.dir": "/apps/hive/warehouse", 
+            "hive.metastore.client.socket.timeout": "60", 
+            "hive.semantic.analyzer.factory.impl": "org.apache.hivealog.cli.HCatSemanticAnalyzerFactory", 
+            "hive.auto.convert.join": "true", 
+            "hive.enforce.bucketing": "true", 
+            "hive.mapred.reduce.tasks.speculative.execution": "false", 
+            "javax.jdo.option.ConnectionURL": "jdbc:mysql://c6402.ambari.apache.org/hive?createDatabaseIfNotExist=true", 
+            "hive.auto.convert.sortmerge.join": "true", 
+            "fs.hdfs.impl.disable.cache": "true", 
+            "hive.security.authorization.manager": "org.apache.hcatalog.security.HdfsAuthorizationProvider", 
+            "ambari.hive.db.schema.name": "hive", 
+            "hive.metastore.execute.setugi": "true", 
+            "hive.auto.convert.sortmerge.join.noconditionaltask": "true", 
+            "hive.server2.enable.doAs": "true", 
+            "hive.optimize.mapjoin.mapreduce": "true"
+        }
+    }, 
+    "configurationTags": {
+        "mapred-site": {
+            "tag": "version1"
+        }, 
+        "oozie-site": {
+            "tag": "version1"
+        }, 
+        "webhcat-site": {
+            "tag": "version1"
+        }, 
+        "global": {
+            "tag": "version1"
+        }, 
+        "hdfs-site": {
+            "tag": "version1"
+        }, 
+        "hbase-site": {
+            "tag": "version1"
+        }, 
+        "core-site": {
+            "tag": "version1"
+        }, 
+        "hive-site": {
+            "tag": "version1"
+        }
+    }, 
+    "commandId": "1-1", 
+    "clusterHostInfo": {
+        "snamenode_host": [
+            "c6402.ambari.apache.org"
+        ], 
+        "ganglia_monitor_hosts": [
+            "c6401.ambari.apache.org", 
+            "c6402.ambari.apache.org"
+        ], 
+        "nagios_server_host": [
+            "c6402.ambari.apache.org"
+        ], 
+        "hive_metastore_hosts": [
+            "c6402.ambari.apache.org"
+        ], 
+        "all_ping_ports": [
+            "8670", 
+            "8670"
+        ], 
+        "mapred_tt_hosts": [
+            "c6401.ambari.apache.org", 
+            "c6402.ambari.apache.org"
+        ], 
+        "all_hosts": [
+            "c6401.ambari.apache.org", 
+            "c6402.ambari.apache.org"
+        ], 
+        "hbase_rs_hosts": [
+            "c6401.ambari.apache.org", 
+            "c6402.ambari.apache.org"
+        ], 
+        "slave_hosts": [
+            "c6401.ambari.apache.org", 
+            "c6402.ambari.apache.org"
+        ], 
+        "namenode_host": [
+            "c6401.ambari.apache.org"
+        ], 
+        "ganglia_server_host": [
+            "c6402.ambari.apache.org"
+        ], 
+        "hbase_master_hosts": [
+            "c6401.ambari.apache.org"
+        ], 
+        "hive_mysql_host": [
+            "c6402.ambari.apache.org"
+        ], 
+        "oozie_server": [
+            "c6402.ambari.apache.org"
+        ], 
+        "webhcat_server_host": [
+            "c6402.ambari.apache.org"
+        ], 
+        "jtnode_host": [
+            "c6402.ambari.apache.org"
+        ], 
+        "zookeeper_hosts": [
+            "c6402.ambari.apache.org"
+        ], 
+        "hs_host": [
+            "c6402.ambari.apache.org"
+        ], 
+        "hive_server_host": [
+            "c6402.ambari.apache.org"
+        ]
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d80ca0b/ambari-server/src/test/python/stacks/2.0.6/HBASE/test_hbase_master.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.0.6/HBASE/test_hbase_master.py b/ambari-server/src/test/python/stacks/2.0.6/HBASE/test_hbase_master.py
index 8616fa5..6959141 100644
--- a/ambari-server/src/test/python/stacks/2.0.6/HBASE/test_hbase_master.py
+++ b/ambari-server/src/test/python/stacks/2.0.6/HBASE/test_hbase_master.py
@@ -66,16 +66,45 @@ class TestHBaseMaster(RMFTestCase):
                        config_file="default.json"
     )
 
+    self.assertResourceCalled('File', '/usr/lib/hbase/bin/draining_servers.rb',
+                              content = StaticFile('draining_servers.rb'),
+                              mode = 0755,
+                              )
+    self.assertResourceCalled('Execute', ' /usr/lib/hbase/bin/hbase --config /etc/hbase/conf org.jruby.Main /usr/lib/hbase/bin/draining_servers.rb add host1',
+                              logoutput = True,
+                              user = 'hbase',
+                              )
     self.assertResourceCalled('Execute', ' /usr/lib/hbase/bin/hbase --config /etc/hbase/conf org.jruby.Main /usr/lib/hbase/bin/region_mover.rb unload host1',
                               logoutput = True,
                               user = 'hbase',
                               )
+    self.assertResourceCalled('Execute', ' /usr/lib/hbase/bin/hbase --config /etc/hbase/conf org.jruby.Main /usr/lib/hbase/bin/draining_servers.rb add host2',
+                              logoutput = True,
+                              user = 'hbase',
+                              )
     self.assertResourceCalled('Execute', ' /usr/lib/hbase/bin/hbase --config /etc/hbase/conf org.jruby.Main /usr/lib/hbase/bin/region_mover.rb unload host2',
                               logoutput = True,
                               user = 'hbase',
                               )
     self.assertNoMoreResources()
 
+  def test_decom_default_draining_only(self):
+    self.executeScript("2.0.6/services/HBASE/package/scripts/hbase_master.py",
+                       classname = "HbaseMaster",
+                       command = "decommission",
+                       config_file="default.hbasedecom.json"
+    )
+
+    self.assertResourceCalled('File', '/usr/lib/hbase/bin/draining_servers.rb',
+                              content = StaticFile('draining_servers.rb'),
+                              mode = 0755,
+                              )
+    self.assertResourceCalled('Execute', ' /usr/lib/hbase/bin/hbase --config /etc/hbase/conf org.jruby.Main /usr/lib/hbase/bin/draining_servers.rb remove host1',
+                              logoutput = True,
+                              user = 'hbase',
+                              )
+    self.assertNoMoreResources()
+
   def test_configure_secured(self):
     self.executeScript("2.0.6/services/HBASE/package/scripts/hbase_master.py",
                    classname = "HbaseMaster",
@@ -120,6 +149,14 @@ class TestHBaseMaster(RMFTestCase):
                        config_file="secured.json"
     )
 
+    self.assertResourceCalled('File', '/usr/lib/hbase/bin/draining_servers.rb',
+                              content = StaticFile('draining_servers.rb'),
+                              mode = 0755,
+                              )
+    self.assertResourceCalled('Execute', '/usr/bin/kinit -kt /etc/security/keytabs/hbase.headless.keytab hbase; /usr/lib/hbase/bin/hbase --config /etc/hbase/conf org.jruby.Main /usr/lib/hbase/bin/draining_servers.rb add host1',
+                              logoutput = True,
+                              user = 'hbase',
+                              )
     self.assertResourceCalled('Execute', '/usr/bin/kinit -kt /etc/security/keytabs/hbase.headless.keytab hbase; /usr/lib/hbase/bin/hbase --config /etc/hbase/conf org.jruby.Main /usr/lib/hbase/bin/region_mover.rb unload host1',
                               logoutput = True,
                               user = 'hbase',