You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zg...@apache.org on 2020/08/14 01:31:03 UTC

[hbase] branch branch-2.2 updated: HBASE-24847 Backport HBASE-23956: Use less resources running tests to branch-2.2 (#2247)

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

zghao pushed a commit to branch branch-2.2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.2 by this push:
     new 8fad2c2  HBASE-24847 Backport HBASE-23956: Use less resources running tests to branch-2.2 (#2247)
8fad2c2 is described below

commit 8fad2c210c4f67a4f8c2285d76999a94c770a10d
Author: Guanghao Zhang <zg...@apache.org>
AuthorDate: Fri Aug 14 09:30:35 2020 +0800

    HBASE-24847 Backport HBASE-23956: Use less resources running tests to branch-2.2 (#2247)
    
    Add being able to configure netty thread counts. Enable socket reuse
    (should not have any impact).
    
    hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
     Rename the threads we create in here so they are NOT named same was
     threads created by Hadoop RPC.
    
    hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DefaultNettyEventLoopConfig.java
    hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java
    hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
     Allow configuring eventloopgroup thread count (so can override for
     tests)
    
    hbase-examples/src/main/java/org/apache/hadoop/hbase/client/example/HttpProxyExample.java
     Enable socket resuse.
    
    hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
     Enable socket resuse and config for how many threads to use.
    
    hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
    hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
     Thread name edit; drop the redundant 'Thread' suffix.
    
    hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java
     Make closeable and shutdown executor when called.
    
    hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
     Call close on HFileReplicator
    
    hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
     HDFS creates lots of threads. Use less of it so less threads overall.
    
    hbase-server/src/test/resources/hbase-site.xml
    hbase-server/src/test/resources/hdfs-site.xml
     Constrain resources when running in test context.
    
    hbase-server/src/test/resources/log4j.properties
     Enable debug on netty to see netty configs in our log
    
    pom.xml
     Add system properties when we launch JVMs to constrain thread counts in
     tests
    
    Addendum to cleanup pom warning.
    
    Addendum: This test was depending on hard-coded number for
    log cleaners; the parent patch made them less in test context.
    
    Addendum: Two more tests that had issues because parent cut
    down the number of default RegionServers launched.
    
    Addendum: TestMultiParallel thread counts were off after
    parent change.
    
    Addendum: TestNamespaceCommands was using up the limit on
    nioeventloopgroup.... so create them per run of client.
---
 .../hadoop/hbase/ipc/BlockingRpcConnection.java    |   2 +-
 .../hbase/ipc/DefaultNettyEventLoopConfig.java     |  40 -------
 .../apache/hadoop/hbase/ipc/NettyRpcClient.java    |  13 +-
 .../hbase/ipc/NettyRpcClientConfigHelper.java      |  36 ++++--
 hbase-client/src/test/resources/hbase-site.xml     | 131 +++++++++++++++++++++
 .../src/test/resources/hdfs-site.xml               |  26 +++-
 hbase-endpoint/src/test/resources/hbase-site.xml   | 131 +++++++++++++++++++++
 .../src/test/resources/hdfs-site.xml               |  26 +++-
 .../hbase/client/example/HttpProxyExample.java     |   1 +
 hbase-examples/src/test/resources/hbase-site.xml   | 131 +++++++++++++++++++++
 .../src/test/resources/hdfs-site.xml               |  26 +++-
 hbase-mapreduce/src/test/resources/hbase-site.xml  | 131 +++++++++++++++++++++
 hbase-mapreduce/src/test/resources/hdfs-site.xml   |  26 +++-
 hbase-procedure/src/test/resources/hbase-site.xml  | 131 +++++++++++++++++++++
 .../src/test/resources/hdfs-site.xml               |  26 +++-
 hbase-rest/src/test/resources/hdfs-site.xml        |  26 +++-
 hbase-rsgroup/src/test/resources/hbase-site.xml    | 131 +++++++++++++++++++++
 .../src/test/resources/hdfs-site.xml               |  26 +++-
 .../apache/hadoop/hbase/ipc/NettyRpcServer.java    |  61 ++++++----
 .../apache/hadoop/hbase/regionserver/HRegion.java  |   2 +-
 .../apache/hadoop/hbase/regionserver/HStore.java   |   4 +-
 .../hadoop/hbase/regionserver/wal/AsyncFSWAL.java  |   6 +-
 .../replication/regionserver/HFileReplicator.java  |  35 +++---
 .../replication/regionserver/ReplicationSink.java  |  18 +--
 .../hadoop/hbase/util/ModifyRegionUtils.java       |   6 +-
 .../hbase/util/NettyEventLoopGroupConfig.java      |   3 -
 .../hadoop/hbase/client/TestMultiParallel.java     |   3 +
 .../hbase/master/cleaner/TestLogsCleaner.java      |   6 +-
 .../hbase/replication/TestReplicationBase.java     |   4 +-
 .../TestReplicationChangingPeerRegionservers.java  |  18 +--
 .../hbase/replication/TestReplicationStatus.java   |   9 +-
 .../security/access/TestNamespaceCommands.java     |  18 ++-
 hbase-server/src/test/resources/hbase-site.xml     | 126 +++++++++++++++++++-
 hbase-server/src/test/resources/hdfs-site.xml      |  26 +++-
 hbase-server/src/test/resources/log4j.properties   |   1 +
 hbase-shell/src/test/resources/hbase-site.xml      | 131 +++++++++++++++++++++
 .../src/test/resources/hdfs-site.xml               |  26 +++-
 hbase-thrift/src/test/resources/hbase-site.xml     | 131 +++++++++++++++++++++
 .../src/test/resources/hdfs-site.xml               |  26 +++-
 pom.xml                                            |   5 +
 40 files changed, 1567 insertions(+), 158 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
index be37553..e04cfe1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
@@ -232,7 +232,7 @@ class BlockingRpcConnection extends RpcConnection implements Runnable {
     this.connectionHeaderWithLength = baos.getBuffer();
 
     UserGroupInformation ticket = remoteId.ticket.getUGI();
-    this.threadName = "IPC Client (" + this.rpcClient.socketFactory.hashCode() + ") connection to "
+    this.threadName = "BRPC Connection (" + this.rpcClient.socketFactory.hashCode() + ") to "
         + remoteId.getAddress().toString()
         + ((ticket == null) ? " from an unknown user" : (" from " + ticket.getUserName()));
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DefaultNettyEventLoopConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DefaultNettyEventLoopConfig.java
deleted file mode 100644
index 87e5540..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DefaultNettyEventLoopConfig.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * 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.hadoop.hbase.ipc;
-
-import org.apache.hbase.thirdparty.io.netty.channel.Channel;
-import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
-import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
-import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
-import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.util.Pair;
-
-/**
- * The default netty event loop config
- */
-@InterfaceAudience.Private
-class DefaultNettyEventLoopConfig {
-
-  public static final Pair<EventLoopGroup, Class<? extends Channel>> GROUP_AND_CHANNEL_CLASS = Pair
-      .<EventLoopGroup, Class<? extends Channel>> newPair(
-        new NioEventLoopGroup(0,
-            new DefaultThreadFactory("Default-IPC-NioEventLoopGroup", true, Thread.MAX_PRIORITY)),
-        NioSocketChannel.class);
-}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java
index 61dedbb..c8a6104 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -49,12 +49,15 @@ public class NettyRpcClient extends AbstractRpcClient<NettyRpcConnection> {
   public NettyRpcClient(Configuration configuration, String clusterId, SocketAddress localAddress,
       MetricsConnection metrics) {
     super(configuration, clusterId, localAddress, metrics);
-    Pair<EventLoopGroup, Class<? extends Channel>> groupAndChannelClass = NettyRpcClientConfigHelper
-        .getEventLoopConfig(conf);
+    Pair<EventLoopGroup, Class<? extends Channel>> groupAndChannelClass =
+      NettyRpcClientConfigHelper.getEventLoopConfig(conf);
     if (groupAndChannelClass == null) {
       // Use our own EventLoopGroup.
-      this.group = new NioEventLoopGroup(0,
-          new DefaultThreadFactory("IPC-NioEventLoopGroup", true, Thread.MAX_PRIORITY));
+      int threadCount = conf.getInt(
+        NettyRpcClientConfigHelper.HBASE_NETTY_EVENTLOOP_RPCCLIENT_THREADCOUNT_KEY, 0);
+      this.group = new NioEventLoopGroup(threadCount,
+          new DefaultThreadFactory("RPCClient(own)-NioEventLoopGroup", true,
+            Thread.NORM_PRIORITY));
       this.channelClass = NioSocketChannel.class;
       this.shutdownGroupWhenClose = true;
     } else {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java
index e779339..6107183 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,18 +17,18 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
-
-import org.apache.hbase.thirdparty.io.netty.channel.Channel;
-import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
-
 import java.util.HashMap;
 import java.util.Map;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.io.netty.channel.Channel;
+import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory;
 
 /**
  * Helper class for passing config to {@link NettyRpcClient}.
@@ -39,16 +39,28 @@ import org.apache.hadoop.hbase.util.Pair;
  * @since 2.0.0
  */
 @InterfaceAudience.Public
-public class NettyRpcClientConfigHelper {
+public final class NettyRpcClientConfigHelper {
 
   public static final String EVENT_LOOP_CONFIG = "hbase.rpc.client.event-loop.config";
 
+  /**
+   * Name of property to change netty rpc client eventloop thread count. Default is 0.
+   * Tests may set this down from unlimited.
+   */
+  public static final String HBASE_NETTY_EVENTLOOP_RPCCLIENT_THREADCOUNT_KEY =
+    "hbase.netty.eventloop.rpcclient.thread.count";
+
   private static final String CONFIG_NAME = "global-event-loop";
 
   private static final Map<String, Pair<EventLoopGroup, Class<? extends Channel>>>
     EVENT_LOOP_CONFIG_MAP = new HashMap<>();
 
   /**
+   * Shutdown constructor.
+   */
+  private NettyRpcClientConfigHelper() {}
+
+  /**
    * Set the EventLoopGroup and channel class for {@code AsyncRpcClient}.
    */
   public static void setEventLoopConfig(Configuration conf, EventLoopGroup group,
@@ -71,12 +83,14 @@ public class NettyRpcClientConfigHelper {
   static Pair<EventLoopGroup, Class<? extends Channel>> getEventLoopConfig(Configuration conf) {
     String name = conf.get(EVENT_LOOP_CONFIG);
     if (name == null) {
-      return DefaultNettyEventLoopConfig.GROUP_AND_CHANNEL_CLASS;
+      int threadCount = conf.getInt(HBASE_NETTY_EVENTLOOP_RPCCLIENT_THREADCOUNT_KEY, 0);
+      return new Pair<>(new NioEventLoopGroup(threadCount,
+        new DefaultThreadFactory("RPCClient-NioEventLoopGroup", true,
+          Thread.NORM_PRIORITY)), NioSocketChannel.class);
     }
     if (StringUtils.isBlank(name)) {
       return null;
     }
     return EVENT_LOOP_CONFIG_MAP.get(name);
   }
-
 }
diff --git a/hbase-client/src/test/resources/hbase-site.xml b/hbase-client/src/test/resources/hbase-site.xml
index 99d2ab8..3a2b052 100644
--- a/hbase-client/src/test/resources/hbase-site.xml
+++ b/hbase-client/src/test/resources/hbase-site.xml
@@ -22,6 +22,22 @@
 -->
 <configuration>
   <property>
+    <name>hbase.regionserver.msginterval</name>
+    <value>100</value>
+    <description>Interval between messages from the RegionServer to HMaster
+     in milliseconds.  Default is 15. Set this value low if you want unit
+     tests to be responsive.
+    </description>
+  </property>
+  <property>
+    <name>hbase.server.thread.wakefrequency</name>
+    <value>1000</value>
+    <value>100</value>
+    <description>Time to sleep in between searches for work (in milliseconds).
+     Used as sleep interval by service threads such as hbase:meta scanner and log roller.
+    </description>
+  </property>
+  <property>
     <name>hbase.defaults.for.version.skip</name>
     <value>true</value>
   </property>
@@ -29,4 +45,119 @@
     <name>hbase.hconnection.threads.keepalivetime</name>
     <value>3</value>
   </property>
+  <property>
+    <name>hbase.regionserver.handler.count</name>
+    <value>3</value>
+    <description>Default is 30</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.metahandler.count</name>
+    <value>3</value>
+    <description>Default is 20</description>
+   </property>
+  <property>
+    <name>hbase.netty.worker.count</name>
+    <value>3</value>
+    <description>Default is 0</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.threads.max</name>
+    <value>6</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.htable.threads.max</name>
+    <value>3</value>
+    <description>Default is MAX_INTEGER</description>
+  </property>
+  <property>
+    <name>hbase.region.replica.replication.threads.max</name>
+    <value>7</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.rest.threads.max</name>
+    <value>5</value>
+    <description>Default is 100</description>
+  </property>
+  <property>
+    <name>hbase.replication.bulkload.copy.maxthreads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.loadincremental.threads.max</name>
+    <value>1</value>
+    <description>Default is # of CPUs</description>
+  </property>
+  <property>
+    <name>hbase.hstore.flusher.count</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.oldwals.cleaner.thread.size</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.master.procedure.threads</name>
+    <value>5</value>
+    <description>Default is at least 16</description>
+  </property>
+  <property>
+    <name>hbase.procedure.remote.dispatcher.threadpool.size</name>
+    <value>3</value>
+    <description>Default is 128</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.closeregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openpriorityregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.storescanner.parallel.seek.threads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hfile.compaction.discharger.thread.count</name>
+    <value>1</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.refresh.peer.threads</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.hregion.open.and.init.threads.max</name>
+    <value>3</value>
+    <description>Default is 16 or # of Regions</description>
+  </property>
+  <property>
+    <name>hbase.master.handler.count</name>
+    <value>7</value>
+    <description>Default is 25</description>
+  </property>
+  <property>
+    <name>hbase.replication.source.maxthreads</name>
+    <value></value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.meta.lookup.threads.max</name>
+    <value>5</value>
+    <description>Default is 128</description>
+  </property>
 </configuration>
diff --git a/hbase-rest/src/test/resources/hdfs-site.xml b/hbase-client/src/test/resources/hdfs-site.xml
similarity index 63%
copy from hbase-rest/src/test/resources/hdfs-site.xml
copy to hbase-client/src/test/resources/hdfs-site.xml
index 03be0c7..9230105 100644
--- a/hbase-rest/src/test/resources/hdfs-site.xml
+++ b/hbase-client/src/test/resources/hdfs-site.xml
@@ -29,4 +29,28 @@
     <name>dfs.namenode.fs-limits.min-block-size</name>
     <value>0</value>
   </property>
-</configuration>
\ No newline at end of file
+  <property>
+    <name>dfs.datanode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.service.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <!--
+   Constraining this config makes tests fail.
+  <property>
+    <name>dfs.datanode.max.transfer.threads</name>
+    <value>16</value>
+    <description>Default is 4096. If constrain this
+    too much, tests do not complete.</description>
+  </property>
+  -->
+</configuration>
diff --git a/hbase-endpoint/src/test/resources/hbase-site.xml b/hbase-endpoint/src/test/resources/hbase-site.xml
index 99d2ab8..3a2b052 100644
--- a/hbase-endpoint/src/test/resources/hbase-site.xml
+++ b/hbase-endpoint/src/test/resources/hbase-site.xml
@@ -22,6 +22,22 @@
 -->
 <configuration>
   <property>
+    <name>hbase.regionserver.msginterval</name>
+    <value>100</value>
+    <description>Interval between messages from the RegionServer to HMaster
+     in milliseconds.  Default is 15. Set this value low if you want unit
+     tests to be responsive.
+    </description>
+  </property>
+  <property>
+    <name>hbase.server.thread.wakefrequency</name>
+    <value>1000</value>
+    <value>100</value>
+    <description>Time to sleep in between searches for work (in milliseconds).
+     Used as sleep interval by service threads such as hbase:meta scanner and log roller.
+    </description>
+  </property>
+  <property>
     <name>hbase.defaults.for.version.skip</name>
     <value>true</value>
   </property>
@@ -29,4 +45,119 @@
     <name>hbase.hconnection.threads.keepalivetime</name>
     <value>3</value>
   </property>
+  <property>
+    <name>hbase.regionserver.handler.count</name>
+    <value>3</value>
+    <description>Default is 30</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.metahandler.count</name>
+    <value>3</value>
+    <description>Default is 20</description>
+   </property>
+  <property>
+    <name>hbase.netty.worker.count</name>
+    <value>3</value>
+    <description>Default is 0</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.threads.max</name>
+    <value>6</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.htable.threads.max</name>
+    <value>3</value>
+    <description>Default is MAX_INTEGER</description>
+  </property>
+  <property>
+    <name>hbase.region.replica.replication.threads.max</name>
+    <value>7</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.rest.threads.max</name>
+    <value>5</value>
+    <description>Default is 100</description>
+  </property>
+  <property>
+    <name>hbase.replication.bulkload.copy.maxthreads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.loadincremental.threads.max</name>
+    <value>1</value>
+    <description>Default is # of CPUs</description>
+  </property>
+  <property>
+    <name>hbase.hstore.flusher.count</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.oldwals.cleaner.thread.size</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.master.procedure.threads</name>
+    <value>5</value>
+    <description>Default is at least 16</description>
+  </property>
+  <property>
+    <name>hbase.procedure.remote.dispatcher.threadpool.size</name>
+    <value>3</value>
+    <description>Default is 128</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.closeregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openpriorityregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.storescanner.parallel.seek.threads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hfile.compaction.discharger.thread.count</name>
+    <value>1</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.refresh.peer.threads</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.hregion.open.and.init.threads.max</name>
+    <value>3</value>
+    <description>Default is 16 or # of Regions</description>
+  </property>
+  <property>
+    <name>hbase.master.handler.count</name>
+    <value>7</value>
+    <description>Default is 25</description>
+  </property>
+  <property>
+    <name>hbase.replication.source.maxthreads</name>
+    <value></value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.meta.lookup.threads.max</name>
+    <value>5</value>
+    <description>Default is 128</description>
+  </property>
 </configuration>
diff --git a/hbase-rest/src/test/resources/hdfs-site.xml b/hbase-endpoint/src/test/resources/hdfs-site.xml
similarity index 63%
copy from hbase-rest/src/test/resources/hdfs-site.xml
copy to hbase-endpoint/src/test/resources/hdfs-site.xml
index 03be0c7..9230105 100644
--- a/hbase-rest/src/test/resources/hdfs-site.xml
+++ b/hbase-endpoint/src/test/resources/hdfs-site.xml
@@ -29,4 +29,28 @@
     <name>dfs.namenode.fs-limits.min-block-size</name>
     <value>0</value>
   </property>
-</configuration>
\ No newline at end of file
+  <property>
+    <name>dfs.datanode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.service.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <!--
+   Constraining this config makes tests fail.
+  <property>
+    <name>dfs.datanode.max.transfer.threads</name>
+    <value>16</value>
+    <description>Default is 4096. If constrain this
+    too much, tests do not complete.</description>
+  </property>
+  -->
+</configuration>
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/client/example/HttpProxyExample.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/client/example/HttpProxyExample.java
index 4ada599..00d01a8 100644
--- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/client/example/HttpProxyExample.java
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/client/example/HttpProxyExample.java
@@ -229,6 +229,7 @@ public class HttpProxyExample {
     channelGroup = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE);
     serverChannel = new ServerBootstrap().group(bossGroup, workerGroup)
         .channel(NioServerSocketChannel.class).childOption(ChannelOption.TCP_NODELAY, true)
+        .childOption(ChannelOption.SO_REUSEADDR, true)
         .childHandler(new ChannelInitializer<Channel>() {
 
           @Override
diff --git a/hbase-examples/src/test/resources/hbase-site.xml b/hbase-examples/src/test/resources/hbase-site.xml
index ab4d1cd..6d6f4bf 100644
--- a/hbase-examples/src/test/resources/hbase-site.xml
+++ b/hbase-examples/src/test/resources/hbase-site.xml
@@ -22,7 +22,138 @@
 -->
 <configuration>
   <property>
+    <name>hbase.regionserver.msginterval</name>
+    <value>100</value>
+    <description>Interval between messages from the RegionServer to HMaster
+     in milliseconds.  Default is 15. Set this value low if you want unit
+     tests to be responsive.
+    </description>
+  </property>
+  <property>
+    <name>hbase.server.thread.wakefrequency</name>
+    <value>1000</value>
+    <value>100</value>
+    <description>Time to sleep in between searches for work (in milliseconds).
+     Used as sleep interval by service threads such as hbase:meta scanner and log roller.
+    </description>
+  </property>
+  <property>
     <name>hbase.defaults.for.version.skip</name>
     <value>true</value>
   </property>
+  <property>
+    <name>hbase.regionserver.handler.count</name>
+    <value>3</value>
+    <description>Default is 30</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.metahandler.count</name>
+    <value>3</value>
+    <description>Default is 20</description>
+   </property>
+  <property>
+    <name>hbase.netty.worker.count</name>
+    <value>3</value>
+    <description>Default is 0</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.threads.max</name>
+    <value>6</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.htable.threads.max</name>
+    <value>3</value>
+    <description>Default is MAX_INTEGER</description>
+  </property>
+  <property>
+    <name>hbase.region.replica.replication.threads.max</name>
+    <value>7</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.rest.threads.max</name>
+    <value>5</value>
+    <description>Default is 100</description>
+  </property>
+  <property>
+    <name>hbase.replication.bulkload.copy.maxthreads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.loadincremental.threads.max</name>
+    <value>1</value>
+    <description>Default is # of CPUs</description>
+  </property>
+  <property>
+    <name>hbase.hstore.flusher.count</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.oldwals.cleaner.thread.size</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.master.procedure.threads</name>
+    <value>5</value>
+    <description>Default is at least 16</description>
+  </property>
+  <property>
+    <name>hbase.procedure.remote.dispatcher.threadpool.size</name>
+    <value>3</value>
+    <description>Default is 128</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.closeregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openpriorityregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.storescanner.parallel.seek.threads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hfile.compaction.discharger.thread.count</name>
+    <value>1</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.refresh.peer.threads</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.hregion.open.and.init.threads.max</name>
+    <value>3</value>
+    <description>Default is 16 or # of Regions</description>
+  </property>
+  <property>
+    <name>hbase.master.handler.count</name>
+    <value>7</value>
+    <description>Default is 25</description>
+  </property>
+  <property>
+    <name>hbase.replication.source.maxthreads</name>
+    <value></value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.meta.lookup.threads.max</name>
+    <value>5</value>
+    <description>Default is 128</description>
+  </property>
 </configuration>
diff --git a/hbase-rest/src/test/resources/hdfs-site.xml b/hbase-examples/src/test/resources/hdfs-site.xml
similarity index 63%
copy from hbase-rest/src/test/resources/hdfs-site.xml
copy to hbase-examples/src/test/resources/hdfs-site.xml
index 03be0c7..9230105 100644
--- a/hbase-rest/src/test/resources/hdfs-site.xml
+++ b/hbase-examples/src/test/resources/hdfs-site.xml
@@ -29,4 +29,28 @@
     <name>dfs.namenode.fs-limits.min-block-size</name>
     <value>0</value>
   </property>
-</configuration>
\ No newline at end of file
+  <property>
+    <name>dfs.datanode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.service.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <!--
+   Constraining this config makes tests fail.
+  <property>
+    <name>dfs.datanode.max.transfer.threads</name>
+    <value>16</value>
+    <description>Default is 4096. If constrain this
+    too much, tests do not complete.</description>
+  </property>
+  -->
+</configuration>
diff --git a/hbase-mapreduce/src/test/resources/hbase-site.xml b/hbase-mapreduce/src/test/resources/hbase-site.xml
index 64a1964..a2aee0c 100644
--- a/hbase-mapreduce/src/test/resources/hbase-site.xml
+++ b/hbase-mapreduce/src/test/resources/hbase-site.xml
@@ -23,6 +23,22 @@
 <configuration>
   <property>
     <name>hbase.regionserver.msginterval</name>
+    <value>100</value>
+    <description>Interval between messages from the RegionServer to HMaster
+     in milliseconds.  Default is 15. Set this value low if you want unit
+     tests to be responsive.
+    </description>
+  </property>
+  <property>
+    <name>hbase.server.thread.wakefrequency</name>
+    <value>1000</value>
+    <value>100</value>
+    <description>Time to sleep in between searches for work (in milliseconds).
+     Used as sleep interval by service threads such as hbase:meta scanner and log roller.
+    </description>
+  </property>
+  <property>
+    <name>hbase.regionserver.msginterval</name>
     <value>1000</value>
     <description>Interval between messages from the RegionServer to HMaster
     in milliseconds.  Default is 15. Set this value low if you want unit
@@ -158,4 +174,119 @@
     <name>hbase.hconnection.threads.keepalivetime</name>
     <value>3</value>
   </property>
+  <property>
+    <name>hbase.regionserver.handler.count</name>
+    <value>3</value>
+    <description>Default is 30</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.metahandler.count</name>
+    <value>3</value>
+    <description>Default is 20</description>
+   </property>
+  <property>
+    <name>hbase.netty.worker.count</name>
+    <value>3</value>
+    <description>Default is 0</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.threads.max</name>
+    <value>6</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.htable.threads.max</name>
+    <value>3</value>
+    <description>Default is MAX_INTEGER</description>
+  </property>
+  <property>
+    <name>hbase.region.replica.replication.threads.max</name>
+    <value>7</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.rest.threads.max</name>
+    <value>5</value>
+    <description>Default is 100</description>
+  </property>
+  <property>
+    <name>hbase.replication.bulkload.copy.maxthreads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.loadincremental.threads.max</name>
+    <value>1</value>
+    <description>Default is # of CPUs</description>
+  </property>
+  <property>
+    <name>hbase.hstore.flusher.count</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.oldwals.cleaner.thread.size</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.master.procedure.threads</name>
+    <value>5</value>
+    <description>Default is at least 16</description>
+  </property>
+  <property>
+    <name>hbase.procedure.remote.dispatcher.threadpool.size</name>
+    <value>3</value>
+    <description>Default is 128</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.closeregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openpriorityregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.storescanner.parallel.seek.threads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hfile.compaction.discharger.thread.count</name>
+    <value>1</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.refresh.peer.threads</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.hregion.open.and.init.threads.max</name>
+    <value>3</value>
+    <description>Default is 16 or # of Regions</description>
+  </property>
+  <property>
+    <name>hbase.master.handler.count</name>
+    <value>7</value>
+    <description>Default is 25</description>
+  </property>
+  <property>
+    <name>hbase.replication.source.maxthreads</name>
+    <value></value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.meta.lookup.threads.max</name>
+    <value>5</value>
+    <description>Default is 128</description>
+  </property>
 </configuration>
diff --git a/hbase-mapreduce/src/test/resources/hdfs-site.xml b/hbase-mapreduce/src/test/resources/hdfs-site.xml
index 03be0c7..9230105 100644
--- a/hbase-mapreduce/src/test/resources/hdfs-site.xml
+++ b/hbase-mapreduce/src/test/resources/hdfs-site.xml
@@ -29,4 +29,28 @@
     <name>dfs.namenode.fs-limits.min-block-size</name>
     <value>0</value>
   </property>
-</configuration>
\ No newline at end of file
+  <property>
+    <name>dfs.datanode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.service.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <!--
+   Constraining this config makes tests fail.
+  <property>
+    <name>dfs.datanode.max.transfer.threads</name>
+    <value>16</value>
+    <description>Default is 4096. If constrain this
+    too much, tests do not complete.</description>
+  </property>
+  -->
+</configuration>
diff --git a/hbase-procedure/src/test/resources/hbase-site.xml b/hbase-procedure/src/test/resources/hbase-site.xml
index 3709a71..c938f28 100644
--- a/hbase-procedure/src/test/resources/hbase-site.xml
+++ b/hbase-procedure/src/test/resources/hbase-site.xml
@@ -22,6 +22,22 @@
 -->
 <configuration>
   <property>
+    <name>hbase.regionserver.msginterval</name>
+    <value>100</value>
+    <description>Interval between messages from the RegionServer to HMaster
+     in milliseconds.  Default is 15. Set this value low if you want unit
+     tests to be responsive.
+    </description>
+  </property>
+  <property>
+    <name>hbase.server.thread.wakefrequency</name>
+    <value>1000</value>
+    <value>100</value>
+    <description>Time to sleep in between searches for work (in milliseconds).
+     Used as sleep interval by service threads such as hbase:meta scanner and log roller.
+    </description>
+  </property>
+  <property>
     <name>hbase.defaults.for.version.skip</name>
     <value>true</value>
   </property>
@@ -45,4 +61,119 @@
       WARNING: Doing so may expose you to additional risk of data loss!
     </description>
   </property>
+  <property>
+    <name>hbase.regionserver.handler.count</name>
+    <value>3</value>
+    <description>Default is 30</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.metahandler.count</name>
+    <value>3</value>
+    <description>Default is 20</description>
+   </property>
+  <property>
+    <name>hbase.netty.worker.count</name>
+    <value>3</value>
+    <description>Default is 0</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.threads.max</name>
+    <value>6</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.htable.threads.max</name>
+    <value>3</value>
+    <description>Default is MAX_INTEGER</description>
+  </property>
+  <property>
+    <name>hbase.region.replica.replication.threads.max</name>
+    <value>7</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.rest.threads.max</name>
+    <value>5</value>
+    <description>Default is 100</description>
+  </property>
+  <property>
+    <name>hbase.replication.bulkload.copy.maxthreads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.loadincremental.threads.max</name>
+    <value>1</value>
+    <description>Default is # of CPUs</description>
+  </property>
+  <property>
+    <name>hbase.hstore.flusher.count</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.oldwals.cleaner.thread.size</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.master.procedure.threads</name>
+    <value>5</value>
+    <description>Default is at least 16</description>
+  </property>
+  <property>
+    <name>hbase.procedure.remote.dispatcher.threadpool.size</name>
+    <value>3</value>
+    <description>Default is 128</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.closeregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openpriorityregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.storescanner.parallel.seek.threads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hfile.compaction.discharger.thread.count</name>
+    <value>1</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.refresh.peer.threads</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.hregion.open.and.init.threads.max</name>
+    <value>3</value>
+    <description>Default is 16 or # of Regions</description>
+  </property>
+  <property>
+    <name>hbase.master.handler.count</name>
+    <value>7</value>
+    <description>Default is 25</description>
+  </property>
+  <property>
+    <name>hbase.replication.source.maxthreads</name>
+    <value></value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.meta.lookup.threads.max</name>
+    <value>5</value>
+    <description>Default is 128</description>
+  </property>
 </configuration>
diff --git a/hbase-rest/src/test/resources/hdfs-site.xml b/hbase-procedure/src/test/resources/hdfs-site.xml
similarity index 63%
copy from hbase-rest/src/test/resources/hdfs-site.xml
copy to hbase-procedure/src/test/resources/hdfs-site.xml
index 03be0c7..9230105 100644
--- a/hbase-rest/src/test/resources/hdfs-site.xml
+++ b/hbase-procedure/src/test/resources/hdfs-site.xml
@@ -29,4 +29,28 @@
     <name>dfs.namenode.fs-limits.min-block-size</name>
     <value>0</value>
   </property>
-</configuration>
\ No newline at end of file
+  <property>
+    <name>dfs.datanode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.service.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <!--
+   Constraining this config makes tests fail.
+  <property>
+    <name>dfs.datanode.max.transfer.threads</name>
+    <value>16</value>
+    <description>Default is 4096. If constrain this
+    too much, tests do not complete.</description>
+  </property>
+  -->
+</configuration>
diff --git a/hbase-rest/src/test/resources/hdfs-site.xml b/hbase-rest/src/test/resources/hdfs-site.xml
index 03be0c7..9230105 100644
--- a/hbase-rest/src/test/resources/hdfs-site.xml
+++ b/hbase-rest/src/test/resources/hdfs-site.xml
@@ -29,4 +29,28 @@
     <name>dfs.namenode.fs-limits.min-block-size</name>
     <value>0</value>
   </property>
-</configuration>
\ No newline at end of file
+  <property>
+    <name>dfs.datanode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.service.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <!--
+   Constraining this config makes tests fail.
+  <property>
+    <name>dfs.datanode.max.transfer.threads</name>
+    <value>16</value>
+    <description>Default is 4096. If constrain this
+    too much, tests do not complete.</description>
+  </property>
+  -->
+</configuration>
diff --git a/hbase-rsgroup/src/test/resources/hbase-site.xml b/hbase-rsgroup/src/test/resources/hbase-site.xml
index 99d2ab8..3a2b052 100644
--- a/hbase-rsgroup/src/test/resources/hbase-site.xml
+++ b/hbase-rsgroup/src/test/resources/hbase-site.xml
@@ -22,6 +22,22 @@
 -->
 <configuration>
   <property>
+    <name>hbase.regionserver.msginterval</name>
+    <value>100</value>
+    <description>Interval between messages from the RegionServer to HMaster
+     in milliseconds.  Default is 15. Set this value low if you want unit
+     tests to be responsive.
+    </description>
+  </property>
+  <property>
+    <name>hbase.server.thread.wakefrequency</name>
+    <value>1000</value>
+    <value>100</value>
+    <description>Time to sleep in between searches for work (in milliseconds).
+     Used as sleep interval by service threads such as hbase:meta scanner and log roller.
+    </description>
+  </property>
+  <property>
     <name>hbase.defaults.for.version.skip</name>
     <value>true</value>
   </property>
@@ -29,4 +45,119 @@
     <name>hbase.hconnection.threads.keepalivetime</name>
     <value>3</value>
   </property>
+  <property>
+    <name>hbase.regionserver.handler.count</name>
+    <value>3</value>
+    <description>Default is 30</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.metahandler.count</name>
+    <value>3</value>
+    <description>Default is 20</description>
+   </property>
+  <property>
+    <name>hbase.netty.worker.count</name>
+    <value>3</value>
+    <description>Default is 0</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.threads.max</name>
+    <value>6</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.htable.threads.max</name>
+    <value>3</value>
+    <description>Default is MAX_INTEGER</description>
+  </property>
+  <property>
+    <name>hbase.region.replica.replication.threads.max</name>
+    <value>7</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.rest.threads.max</name>
+    <value>5</value>
+    <description>Default is 100</description>
+  </property>
+  <property>
+    <name>hbase.replication.bulkload.copy.maxthreads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.loadincremental.threads.max</name>
+    <value>1</value>
+    <description>Default is # of CPUs</description>
+  </property>
+  <property>
+    <name>hbase.hstore.flusher.count</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.oldwals.cleaner.thread.size</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.master.procedure.threads</name>
+    <value>5</value>
+    <description>Default is at least 16</description>
+  </property>
+  <property>
+    <name>hbase.procedure.remote.dispatcher.threadpool.size</name>
+    <value>3</value>
+    <description>Default is 128</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.closeregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openpriorityregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.storescanner.parallel.seek.threads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hfile.compaction.discharger.thread.count</name>
+    <value>1</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.refresh.peer.threads</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.hregion.open.and.init.threads.max</name>
+    <value>3</value>
+    <description>Default is 16 or # of Regions</description>
+  </property>
+  <property>
+    <name>hbase.master.handler.count</name>
+    <value>7</value>
+    <description>Default is 25</description>
+  </property>
+  <property>
+    <name>hbase.replication.source.maxthreads</name>
+    <value></value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.meta.lookup.threads.max</name>
+    <value>5</value>
+    <description>Default is 128</description>
+  </property>
 </configuration>
diff --git a/hbase-rest/src/test/resources/hdfs-site.xml b/hbase-rsgroup/src/test/resources/hdfs-site.xml
similarity index 63%
copy from hbase-rest/src/test/resources/hdfs-site.xml
copy to hbase-rsgroup/src/test/resources/hdfs-site.xml
index 03be0c7..9230105 100644
--- a/hbase-rest/src/test/resources/hdfs-site.xml
+++ b/hbase-rsgroup/src/test/resources/hdfs-site.xml
@@ -29,4 +29,28 @@
     <name>dfs.namenode.fs-limits.min-block-size</name>
     <value>0</value>
   </property>
-</configuration>
\ No newline at end of file
+  <property>
+    <name>dfs.datanode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.service.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <!--
+   Constraining this config makes tests fail.
+  <property>
+    <name>dfs.datanode.max.transfer.threads</name>
+    <value>16</value>
+    <description>Default is 4096. If constrain this
+    too much, tests do not complete.</description>
+  </property>
+  -->
+</configuration>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
index 742a728..798e5ee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,44 +17,42 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import org.apache.hbase.thirdparty.io.netty.bootstrap.ServerBootstrap;
-import org.apache.hbase.thirdparty.io.netty.channel.Channel;
-import org.apache.hbase.thirdparty.io.netty.channel.ChannelInitializer;
-import org.apache.hbase.thirdparty.io.netty.channel.ChannelOption;
-import org.apache.hbase.thirdparty.io.netty.channel.ChannelPipeline;
-import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
-import org.apache.hbase.thirdparty.io.netty.channel.ServerChannel;
-import org.apache.hbase.thirdparty.io.netty.channel.group.ChannelGroup;
-import org.apache.hbase.thirdparty.io.netty.channel.group.DefaultChannelGroup;
-import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
-import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioServerSocketChannel;
-import org.apache.hbase.thirdparty.io.netty.handler.codec.FixedLengthFrameDecoder;
-import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory;
-import org.apache.hbase.thirdparty.io.netty.util.concurrent.GlobalEventExecutor;
-
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.net.InetSocketAddress;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.Server;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.security.HBasePolicyProvider;
+import org.apache.hadoop.hbase.util.NettyEventLoopGroupConfig;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
 import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
 import org.apache.hbase.thirdparty.com.google.protobuf.Message;
-import org.apache.hadoop.hbase.util.NettyEventLoopGroupConfig;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
+import org.apache.hbase.thirdparty.io.netty.bootstrap.ServerBootstrap;
+import org.apache.hbase.thirdparty.io.netty.channel.Channel;
+import org.apache.hbase.thirdparty.io.netty.channel.ChannelInitializer;
+import org.apache.hbase.thirdparty.io.netty.channel.ChannelOption;
+import org.apache.hbase.thirdparty.io.netty.channel.ChannelPipeline;
+import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.ServerChannel;
+import org.apache.hbase.thirdparty.io.netty.channel.group.ChannelGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.group.DefaultChannelGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioServerSocketChannel;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.FixedLengthFrameDecoder;
+import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory;
+import org.apache.hbase.thirdparty.io.netty.util.concurrent.GlobalEventExecutor;
 
 /**
  * An RPC server with Netty4 implementation.
@@ -62,9 +60,16 @@ import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.CONFIG})
 public class NettyRpcServer extends RpcServer {
-
   public static final Logger LOG = LoggerFactory.getLogger(NettyRpcServer.class);
 
+  /**
+   * Name of property to change netty rpc server eventloop thread count. Default is 0.
+   * Tests may set this down from unlimited.
+   */
+  public static final String HBASE_NETTY_EVENTLOOP_RPCSERVER_THREADCOUNT_KEY =
+    "hbase.netty.eventloop.rpcserver.thread.count";
+  private static final int EVENTLOOP_THREADCOUNT_DEFAULT = 0;
+
   private final InetSocketAddress bindAddress;
 
   private final CountDownLatch closed = new CountDownLatch(1);
@@ -84,13 +89,17 @@ public class NettyRpcServer extends RpcServer {
       eventLoopGroup = config.group();
       channelClass = config.serverChannelClass();
     } else {
-      eventLoopGroup = new NioEventLoopGroup(0,
-          new DefaultThreadFactory("NettyRpcServer", true, Thread.MAX_PRIORITY));
+      int threadCount = server == null? EVENTLOOP_THREADCOUNT_DEFAULT:
+        server.getConfiguration().getInt(HBASE_NETTY_EVENTLOOP_RPCSERVER_THREADCOUNT_KEY,
+          EVENTLOOP_THREADCOUNT_DEFAULT);
+      eventLoopGroup = new NioEventLoopGroup(threadCount,
+        new DefaultThreadFactory("NettyRpcServer", true, Thread.MAX_PRIORITY));
       channelClass = NioServerSocketChannel.class;
     }
     ServerBootstrap bootstrap = new ServerBootstrap().group(eventLoopGroup).channel(channelClass)
         .childOption(ChannelOption.TCP_NODELAY, tcpNoDelay)
         .childOption(ChannelOption.SO_KEEPALIVE, tcpKeepAlive)
+        .childOption(ChannelOption.SO_REUSEADDR, true)
         .childHandler(new ChannelInitializer<Channel>() {
 
           @Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 6cbd252..5264801 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -1699,7 +1699,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       if (!stores.isEmpty()) {
         // initialize the thread pool for closing stores in parallel.
         ThreadPoolExecutor storeCloserThreadPool =
-          getStoreOpenAndCloseThreadPool("StoreCloserThread-" +
+          getStoreOpenAndCloseThreadPool("StoreCloser-" +
             getRegionInfo().getRegionNameAsString());
         CompletionService<Pair<byte[], Collection<HStoreFile>>> completionService =
           new ExecutorCompletionService<>(storeCloserThreadPool);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 01fb231..96a8afb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -576,7 +576,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
     }
     // initialize the thread pool for opening store files in parallel..
     ThreadPoolExecutor storeFileOpenerThreadPool =
-      this.region.getStoreFileOpenAndCloseThreadPool("StoreFileOpenerThread-" +
+      this.region.getStoreFileOpenAndCloseThreadPool("StoreFileOpener-" +
           this.getColumnFamilyName());
     CompletionService<HStoreFile> completionService = new ExecutorCompletionService<>(storeFileOpenerThreadPool);
 
@@ -970,7 +970,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
       if (!result.isEmpty()) {
         // initialize the thread pool for closing store files in parallel.
         ThreadPoolExecutor storeFileCloserThreadPool = this.region
-            .getStoreFileOpenAndCloseThreadPool("StoreFileCloserThread-"
+            .getStoreFileOpenAndCloseThreadPool("StoreFileCloser-"
                 + this.getColumnFamilyName());
 
         // close each store file in parallel
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
index ef6870b..e31581e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -226,8 +226,10 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
       }
     } else {
       ThreadPoolExecutor threadPool =
-        new ThreadPoolExecutor(1, 1, 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>(),
-            new ThreadFactoryBuilder().setNameFormat("AsyncFSWAL-%d").setDaemon(true).build());
+        new ThreadPoolExecutor(1, 1, 0L,
+          TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>(),
+            new ThreadFactoryBuilder().setNameFormat("AsyncFSWAL-%d-" + rootDir.toString()).
+              setDaemon(true).build());
       hasConsumerTask = () -> threadPool.getQueue().peek() == consumer;
       this.consumeExecutor = threadPool;
     }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java
index 9bbc16d..a81c576 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java
@@ -10,6 +10,7 @@
  */
 package org.apache.hadoop.hbase.replication.regionserver;
 
+import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -27,7 +28,6 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
@@ -39,11 +39,11 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles.LoadQueueItem;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.security.token.FsDelegationToken;
+import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles.LoadQueueItem;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
@@ -51,16 +51,16 @@ import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
  * It is used for replicating HFile entries. It will first copy parallely all the hfiles to a local
  * staging directory and then it will use ({@link LoadIncrementalHFiles} to prepare a collection of
  * {@link LoadQueueItem} which will finally be loaded(replicated) into the table of this cluster.
+ * Call {@link #close()} when done.
  */
 @InterfaceAudience.Private
-public class HFileReplicator {
+public class HFileReplicator implements Closeable {
   /** Maximum number of threads to allow in pool to copy hfiles during replication */
   public static final String REPLICATION_BULKLOAD_COPY_MAXTHREADS_KEY =
       "hbase.replication.bulkload.copy.maxthreads";
@@ -109,7 +109,8 @@ public class HFileReplicator {
           REPLICATION_BULKLOAD_COPY_MAXTHREADS_DEFAULT);
     this.exec = Threads.getBoundedCachedThreadPool(maxCopyThreads, 60, TimeUnit.SECONDS,
         new ThreadFactoryBuilder().setDaemon(true)
-            .setNameFormat("HFileReplicationCallable-%1$d").build());
+            .setNameFormat("HFileReplicationCopier-%1$d-" + this.sourceBaseNamespaceDirPath).
+          build());
     this.copiesPerThread =
         conf.getInt(REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_KEY,
           REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_DEFAULT);
@@ -117,6 +118,13 @@ public class HFileReplicator {
     sinkFs = FileSystem.get(conf);
   }
 
+  @Override
+  public void close() throws IOException {
+    if (this.exec != null) {
+      this.exec.shutdown();
+    }
+  }
+
   public Void replicate() throws IOException {
     // Copy all the hfiles to the local file system
     Map<String, Path> tableStagingDirsMap = copyHFilesToStagingDir();
@@ -132,8 +140,7 @@ public class HFileReplicator {
         loadHFiles = new LoadIncrementalHFiles(conf);
         loadHFiles.setClusterIds(sourceClusterIds);
       } catch (Exception e) {
-        LOG.error("Failed to initialize LoadIncrementalHFiles for replicating bulk loaded"
-            + " data.", e);
+        LOG.error("Failed initialize LoadIncrementalHFiles for replicating bulk loaded data.", e);
         throw new IOException(e);
       }
       Configuration newConf = HBaseConfiguration.create(conf);
@@ -148,19 +155,15 @@ public class HFileReplicator {
       loadHFiles.prepareHFileQueue(stagingDir, table, queue, false);
 
       if (queue.isEmpty()) {
-        LOG.warn("Replication process did not find any files to replicate in directory "
-            + stagingDir.toUri());
+        LOG.warn("Did not find any files to replicate in directory {}", stagingDir.toUri());
         return null;
       }
 
       try (RegionLocator locator = connection.getRegionLocator(tableName)) {
-
         fsDelegationToken.acquireDelegationToken(sinkFs);
-
         // Set the staging directory which will be used by LoadIncrementalHFiles for loading the
         // data
         loadHFiles.setBulkToken(stagingDir.toString());
-
         doBulkLoad(loadHFiles, table, queue, locator, maxRetries);
       } finally {
         cleanup(stagingDir.toString(), table);
@@ -177,13 +180,11 @@ public class HFileReplicator {
       // need to reload split keys each iteration.
       startEndKeys = locator.getStartEndKeys();
       if (count != 0) {
-        LOG.warn("Error occurred while replicating HFiles, retry attempt " + count + " with "
-            + queue.size() + " files still remaining to replicate.");
+        LOG.warn("Error replicating HFiles; retry={} with {} remaining.", count, queue.size());
       }
 
       if (maxRetries != 0 && count >= maxRetries) {
-        throw new IOException("Retry attempted " + count
-            + " times without completing, bailing out.");
+        throw new IOException("Retry attempted " + count + " times without completing, bailing.");
       }
       count++;
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
index 76e22f8..b947c80 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
@@ -30,7 +30,6 @@ import java.util.Map.Entry;
 import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -259,18 +258,13 @@ public class ReplicationSink {
           List<String>>>>> entry : bulkLoadsPerClusters.entrySet()) {
           Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap = entry.getValue();
           if (bulkLoadHFileMap != null && !bulkLoadHFileMap.isEmpty()) {
-            if(LOG.isDebugEnabled()) {
-              LOG.debug("Started replicating bulk loaded data from cluster ids: {}.",
-                entry.getKey().toString());
-            }
-            HFileReplicator hFileReplicator =
-              new HFileReplicator(this.provider.getConf(this.conf, replicationClusterId),
+            LOG.debug("Replicating {} bulk loaded data", entry.getKey().toString());
+            Configuration providerConf = this.provider.getConf(this.conf, replicationClusterId);
+            try (HFileReplicator hFileReplicator = new HFileReplicator(providerConf,
                 sourceBaseNamespaceDirPath, sourceHFileArchiveDirPath, bulkLoadHFileMap, conf,
-                getConnection(), entry.getKey());
-            hFileReplicator.replicate();
-            if(LOG.isDebugEnabled()) {
-              LOG.debug("Finished replicating bulk loaded data from cluster id: {}",
-                entry.getKey().toString());
+                getConnection(), entry.getKey())) {
+              hFileReplicator.replicate();
+              LOG.debug("Finished replicating {} bulk loaded data", entry.getKey().toString());
             }
           }
         }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
index 79544fb..bf8772f4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
@@ -110,7 +110,7 @@ public abstract class ModifyRegionUtils {
     if (newRegions == null) return null;
     int regionNumber = newRegions.length;
     ThreadPoolExecutor exec = getRegionOpenAndInitThreadPool(conf,
-        "RegionOpenAndInitThread-" + tableDescriptor.getTableName(), regionNumber);
+        "RegionOpenAndInit-" + tableDescriptor.getTableName(), regionNumber);
     try {
       return createRegions(exec, conf, rootDir, tableDescriptor, newRegions, task);
     } finally {
@@ -232,8 +232,8 @@ public abstract class ModifyRegionUtils {
       final String threadNamePrefix, int regionNumber) {
     int maxThreads = Math.min(regionNumber, conf.getInt(
         "hbase.hregion.open.and.init.threads.max", 16));
-    ThreadPoolExecutor regionOpenAndInitThreadPool = Threads
-    .getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
+    ThreadPoolExecutor regionOpenAndInitThreadPool = Threads.
+      getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
         Threads.newDaemonThreadFactory(threadNamePrefix));
     return regionOpenAndInitThreadPool;
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/NettyEventLoopGroupConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/NettyEventLoopGroupConfig.java
index 3e7b488..3e247f3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/NettyEventLoopGroupConfig.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/NettyEventLoopGroupConfig.java
@@ -27,9 +27,7 @@ import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
 import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioServerSocketChannel;
 import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
 import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory;
-
 import java.util.concurrent.ThreadFactory;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -38,7 +36,6 @@ import org.apache.yetus.audience.InterfaceAudience;
  */
 @InterfaceAudience.Private
 public class NettyEventLoopGroupConfig {
-
   private final EventLoopGroup group;
 
   private final Class<? extends ServerChannel> serverChannelClass;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
index 7d36e99..84c90de 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
@@ -181,6 +181,9 @@ public class TestMultiParallel {
   @Test
   public void testActiveThreadsCount() throws Exception {
     UTIL.getConfiguration().setLong("hbase.htable.threads.coresize", slaves + 1);
+    // Make sure max is at least as big as coresize; can be smaller in test context where
+    // we tune down thread sizes -- max could be < slaves + 1.
+    UTIL.getConfiguration().setLong("hbase.htable.threads.max", slaves + 1);
     try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration())) {
       ThreadPoolExecutor executor = HTable.getDefaultExecutor(UTIL.getConfiguration());
       try {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index c3a385c..32e6635 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -295,7 +295,7 @@ public class TestLogsCleaner {
 
     FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
     LogCleaner cleaner = new LogCleaner(3000, server, conf, fs, OLD_WALS_DIR, POOL);
-    assertEquals(LogCleaner.DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE, cleaner.getSizeOfCleaners());
+    int size = cleaner.getSizeOfCleaners();
     assertEquals(LogCleaner.DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC,
         cleaner.getCleanerThreadTimeoutMsec());
     // Create dir and files for test
@@ -310,10 +310,10 @@ public class TestLogsCleaner {
     // change size of cleaners dynamically
     int sizeToChange = 4;
     long threadTimeoutToChange = 30 * 1000L;
-    conf.setInt(LogCleaner.OLD_WALS_CLEANER_THREAD_SIZE, sizeToChange);
+    conf.setInt(LogCleaner.OLD_WALS_CLEANER_THREAD_SIZE, size + sizeToChange);
     conf.setLong(LogCleaner.OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC, threadTimeoutToChange);
     cleaner.onConfigurationChange(conf);
-    assertEquals(sizeToChange, cleaner.getSizeOfCleaners());
+    assertEquals(sizeToChange + size, cleaner.getSizeOfCleaners());
     assertEquals(threadTimeoutToChange, cleaner.getCleanerThreadTimeoutMsec());
     // Stop chore
     thread.join();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
index e87b076..ff516da 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
@@ -76,8 +76,8 @@ public class TestReplicationBase {
   protected static final Configuration CONF1 = UTIL1.getConfiguration();
   protected static final Configuration CONF2 = UTIL2.getConfiguration();
 
-  protected static final int NUM_SLAVES1 = 2;
-  protected static final int NUM_SLAVES2 = 4;
+  protected static final int NUM_SLAVES1 = 1;
+  protected static final int NUM_SLAVES2 = 1;
   protected static final int NB_ROWS_IN_BATCH = 100;
   protected static final int NB_ROWS_IN_BIG_BATCH =
       NB_ROWS_IN_BATCH * 10;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
index a7d983c..2fcc2c8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -20,12 +20,11 @@ package org.apache.hadoop.hbase.replication;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.util.List;
-
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
@@ -45,7 +44,6 @@ import org.junit.runners.Parameterized.Parameter;
 import org.junit.runners.Parameterized.Parameters;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
 
 /**
@@ -62,7 +60,7 @@ public class TestReplicationChangingPeerRegionservers extends TestReplicationBas
   private static final Logger LOG =
       LoggerFactory.getLogger(TestReplicationChangingPeerRegionservers.class);
 
-  @Parameter
+  @SuppressWarnings("checkstyle:VisibilityModifier") @Parameter
   public boolean serialPeer;
 
   @Override
@@ -75,9 +73,6 @@ public class TestReplicationChangingPeerRegionservers extends TestReplicationBas
     return ImmutableList.of(true, false);
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @Before
   public void setUp() throws Exception {
     // Starting and stopping replication can make us miss new logs,
@@ -118,6 +113,13 @@ public class TestReplicationChangingPeerRegionservers extends TestReplicationBas
   public void testChangingNumberOfPeerRegionServers() throws IOException, InterruptedException {
     LOG.info("testSimplePutDelete");
     MiniHBaseCluster peerCluster = UTIL2.getMiniHBaseCluster();
+    // This test wants two RS's up. We only run one generally so add one.
+    peerCluster.startRegionServer();
+    Waiter.waitFor(peerCluster.getConfiguration(), 30000, new Waiter.Predicate<Exception>() {
+      @Override public boolean evaluate() throws Exception {
+        return peerCluster.getLiveRegionServerThreads().size() > 1;
+      }
+    });
     int numRS = peerCluster.getRegionServerThreads().size();
 
     doPutTest(Bytes.toBytes(1));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java
index 17c2931..90cff34 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.ServerMetrics;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -69,7 +70,13 @@ public class TestReplicationStatus extends TestReplicationBase {
    */
   @Test
   public void testReplicationStatus() throws Exception {
-    LOG.info("testReplicationStatus");
+    // This test wants two RS's up. We only run one generally so add one.
+    UTIL1.getMiniHBaseCluster().startRegionServer();
+    Waiter.waitFor(UTIL1.getConfiguration(), 30000, new Waiter.Predicate<Exception>() {
+      @Override public boolean evaluate() throws Exception {
+        return UTIL1.getMiniHBaseCluster().getLiveRegionServerThreads().size() > 1;
+      }
+    });
     try (Admin hbaseAdmin = UTIL1.getAdmin()) {
       // disable peer
       hbaseAdmin.disableReplicationPeer(PEER_ID2);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
index 9e696fd..8adb885 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.security.access;
 import static org.apache.hadoop.hbase.AuthUtil.toGroupEntry;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-
 import com.google.protobuf.BlockingRpcChannel;
 import java.util.Arrays;
 import java.util.List;
@@ -42,6 +41,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.ObserverContextImpl;
+import org.apache.hadoop.hbase.ipc.NettyRpcClientConfigHelper;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
@@ -56,7 +56,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
 
 @Category({SecurityTests.class, MediumTests.class})
@@ -164,9 +163,13 @@ public class TestNamespaceCommands extends SecureTestUtil {
         UTIL.getMiniHBaseCluster().getLiveRegionServerThreads()) {
       ACCESS_CONTROLLER = rst.getRegionServer().getRegionServerCoprocessorHost().
         findCoprocessor(AccessController.class);
-      if (ACCESS_CONTROLLER != null) break;
+      if (ACCESS_CONTROLLER != null) {
+        break;
+      }
+    }
+    if (ACCESS_CONTROLLER == null) {
+      throw new NullPointerException();
     }
-    if (ACCESS_CONTROLLER == null) throw new NullPointerException();
 
     UTIL.getAdmin().createNamespace(NamespaceDescriptor.create(TEST_NAMESPACE).build());
     UTIL.getAdmin().createNamespace(NamespaceDescriptor.create(TEST_NAMESPACE2).build());
@@ -354,9 +357,12 @@ public class TestNamespaceCommands extends SecureTestUtil {
     assertEquals(0, ((List)USER_GROUP_WRITE.runAs(listAction)).size());
   }
 
-  @Test
+  @SuppressWarnings("checkstyle:MethodLength") @Test
   public void testGrantRevoke() throws Exception {
     final String testUser = "testUser";
+    // Set this else in test context, with limit on the number of threads for
+    // netty eventloopgroup, we can run out of threads if one group used throughout.
+    NettyRpcClientConfigHelper.createEventLoopPerClient(conf);
     // Test if client API actions are authorized
     AccessTestAction grantAction = new AccessTestAction() {
       @Override
diff --git a/hbase-server/src/test/resources/hbase-site.xml b/hbase-server/src/test/resources/hbase-site.xml
index 64a1964..a317376 100644
--- a/hbase-server/src/test/resources/hbase-site.xml
+++ b/hbase-server/src/test/resources/hbase-site.xml
@@ -23,7 +23,7 @@
 <configuration>
   <property>
     <name>hbase.regionserver.msginterval</name>
-    <value>1000</value>
+    <value>100</value>
     <description>Interval between messages from the RegionServer to HMaster
     in milliseconds.  Default is 15. Set this value low if you want unit
     tests to be responsive.
@@ -35,7 +35,7 @@
   </property>
   <property>
     <name>hbase.server.thread.wakefrequency</name>
-    <value>1000</value>
+    <value>100</value>
     <description>Time to sleep in between searches for work (in milliseconds).
     Used as sleep interval by service threads such as hbase:meta scanner and log roller.
     </description>
@@ -48,11 +48,13 @@
   </property>
   <property>
     <name>hbase.regionserver.handler.count</name>
-    <value>5</value>
+    <value>3</value>
+    <description>Default is 30</description>
   </property>
   <property>
     <name>hbase.regionserver.metahandler.count</name>
-    <value>6</value>
+    <value>3</value>
+    <description>Default is 20</description>
   </property>
   <property>
       <name>hbase.ipc.server.read.threadpool.size</name>
@@ -83,7 +85,6 @@
     <name>hbase.ipc.client.fallback-to-simple-auth-allowed</name>
     <value>true</value>
   </property>
-
   <property>
     <name>hbase.regionserver.info.port</name>
     <value>-1</value>
@@ -158,4 +159,119 @@
     <name>hbase.hconnection.threads.keepalivetime</name>
     <value>3</value>
   </property>
+  <property>
+    <name>hbase.netty.worker.count</name>
+    <value>3</value>
+    <description>Default is 0</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.threads.max</name>
+    <value>6</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.htable.threads.max</name>
+    <value>3</value>
+    <description>Default is MAX_INTEGER</description>
+  </property>
+  <property>
+    <name>hbase.region.replica.replication.threads.max</name>
+    <value>7</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.rest.threads.max</name>
+    <value>5</value>
+    <description>Default is 100</description>
+  </property>
+  <property>
+    <name>hbase.replication.bulkload.copy.maxthreads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.loadincremental.threads.max</name>
+    <value>1</value>
+    <description>Default is # of CPUs</description>
+  </property>
+  <property>
+    <name>hbase.hstore.flusher.count</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.oldwals.cleaner.thread.size</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.master.procedure.threads</name>
+    <value>5</value>
+    <description>Default is at least 16</description>
+  </property>
+  <property>
+    <name>hbase.procedure.remote.dispatcher.threadpool.size</name>
+    <value>3</value>
+    <description>Default is 128</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.closeregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openpriorityregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.storescanner.parallel.seek.threads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hfile.compaction.discharger.thread.count</name>
+    <value>1</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.refresh.peer.threads</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.hregion.open.and.init.threads.max</name>
+    <value>3</value>
+    <description>Default is 16 or # of Regions</description>
+  </property>
+  <property>
+    <name>hbase.master.handler.count</name>
+    <value>7</value>
+    <description>Default is 25</description>
+  </property>
+  <property>
+    <name>hbase.replication.source.maxthreads</name>
+    <value></value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.meta.lookup.threads.max</name>
+    <value>5</value>
+    <description>Default is 128</description>
+  </property>
+  <property>
+    <name>hbase.netty.eventloop.rpclient.thread.count</name>
+    <value>3</value>
+    <description>Default is unbounded</description>
+  </property>
+  <property>
+    <name>hbase.netty.eventloop.rpcserver.thread.count</name>
+    <value>3</value>
+    <description>Default is unbounded</description>
+  </property>
 </configuration>
diff --git a/hbase-server/src/test/resources/hdfs-site.xml b/hbase-server/src/test/resources/hdfs-site.xml
index 03be0c7..9230105 100644
--- a/hbase-server/src/test/resources/hdfs-site.xml
+++ b/hbase-server/src/test/resources/hdfs-site.xml
@@ -29,4 +29,28 @@
     <name>dfs.namenode.fs-limits.min-block-size</name>
     <value>0</value>
   </property>
-</configuration>
\ No newline at end of file
+  <property>
+    <name>dfs.datanode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.service.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <!--
+   Constraining this config makes tests fail.
+  <property>
+    <name>dfs.datanode.max.transfer.threads</name>
+    <value>16</value>
+    <description>Default is 4096. If constrain this
+    too much, tests do not complete.</description>
+  </property>
+  -->
+</configuration>
diff --git a/hbase-server/src/test/resources/log4j.properties b/hbase-server/src/test/resources/log4j.properties
index fcb6600..a699a92 100644
--- a/hbase-server/src/test/resources/log4j.properties
+++ b/hbase-server/src/test/resources/log4j.properties
@@ -66,5 +66,6 @@ log4j.logger.org.apache.hadoop.metrics2.impl.MetricsConfig=WARN
 log4j.logger.org.apache.hadoop.metrics2.impl.MetricsSinkAdapter=WARN
 log4j.logger.org.apache.hadoop.metrics2.impl.MetricsSystemImpl=WARN
 log4j.logger.org.apache.hadoop.metrics2.util.MBeans=WARN
+log4j.logger.io.netty.channel=DEBUG
 # Enable this to get detailed connection error/retry logging.
 # log4j.logger.org.apache.hadoop.hbase.client.ConnectionImplementation=TRACE
diff --git a/hbase-shell/src/test/resources/hbase-site.xml b/hbase-shell/src/test/resources/hbase-site.xml
index 99d2ab8..3a2b052 100644
--- a/hbase-shell/src/test/resources/hbase-site.xml
+++ b/hbase-shell/src/test/resources/hbase-site.xml
@@ -22,6 +22,22 @@
 -->
 <configuration>
   <property>
+    <name>hbase.regionserver.msginterval</name>
+    <value>100</value>
+    <description>Interval between messages from the RegionServer to HMaster
+     in milliseconds.  Default is 15. Set this value low if you want unit
+     tests to be responsive.
+    </description>
+  </property>
+  <property>
+    <name>hbase.server.thread.wakefrequency</name>
+    <value>1000</value>
+    <value>100</value>
+    <description>Time to sleep in between searches for work (in milliseconds).
+     Used as sleep interval by service threads such as hbase:meta scanner and log roller.
+    </description>
+  </property>
+  <property>
     <name>hbase.defaults.for.version.skip</name>
     <value>true</value>
   </property>
@@ -29,4 +45,119 @@
     <name>hbase.hconnection.threads.keepalivetime</name>
     <value>3</value>
   </property>
+  <property>
+    <name>hbase.regionserver.handler.count</name>
+    <value>3</value>
+    <description>Default is 30</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.metahandler.count</name>
+    <value>3</value>
+    <description>Default is 20</description>
+   </property>
+  <property>
+    <name>hbase.netty.worker.count</name>
+    <value>3</value>
+    <description>Default is 0</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.threads.max</name>
+    <value>6</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.htable.threads.max</name>
+    <value>3</value>
+    <description>Default is MAX_INTEGER</description>
+  </property>
+  <property>
+    <name>hbase.region.replica.replication.threads.max</name>
+    <value>7</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.rest.threads.max</name>
+    <value>5</value>
+    <description>Default is 100</description>
+  </property>
+  <property>
+    <name>hbase.replication.bulkload.copy.maxthreads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.loadincremental.threads.max</name>
+    <value>1</value>
+    <description>Default is # of CPUs</description>
+  </property>
+  <property>
+    <name>hbase.hstore.flusher.count</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.oldwals.cleaner.thread.size</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.master.procedure.threads</name>
+    <value>5</value>
+    <description>Default is at least 16</description>
+  </property>
+  <property>
+    <name>hbase.procedure.remote.dispatcher.threadpool.size</name>
+    <value>3</value>
+    <description>Default is 128</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.closeregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openpriorityregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.storescanner.parallel.seek.threads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hfile.compaction.discharger.thread.count</name>
+    <value>1</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.refresh.peer.threads</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.hregion.open.and.init.threads.max</name>
+    <value>3</value>
+    <description>Default is 16 or # of Regions</description>
+  </property>
+  <property>
+    <name>hbase.master.handler.count</name>
+    <value>7</value>
+    <description>Default is 25</description>
+  </property>
+  <property>
+    <name>hbase.replication.source.maxthreads</name>
+    <value></value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.meta.lookup.threads.max</name>
+    <value>5</value>
+    <description>Default is 128</description>
+  </property>
 </configuration>
diff --git a/hbase-rest/src/test/resources/hdfs-site.xml b/hbase-shell/src/test/resources/hdfs-site.xml
similarity index 63%
copy from hbase-rest/src/test/resources/hdfs-site.xml
copy to hbase-shell/src/test/resources/hdfs-site.xml
index 03be0c7..9230105 100644
--- a/hbase-rest/src/test/resources/hdfs-site.xml
+++ b/hbase-shell/src/test/resources/hdfs-site.xml
@@ -29,4 +29,28 @@
     <name>dfs.namenode.fs-limits.min-block-size</name>
     <value>0</value>
   </property>
-</configuration>
\ No newline at end of file
+  <property>
+    <name>dfs.datanode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.service.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <!--
+   Constraining this config makes tests fail.
+  <property>
+    <name>dfs.datanode.max.transfer.threads</name>
+    <value>16</value>
+    <description>Default is 4096. If constrain this
+    too much, tests do not complete.</description>
+  </property>
+  -->
+</configuration>
diff --git a/hbase-thrift/src/test/resources/hbase-site.xml b/hbase-thrift/src/test/resources/hbase-site.xml
index b3fb0d9..a1e89ce 100644
--- a/hbase-thrift/src/test/resources/hbase-site.xml
+++ b/hbase-thrift/src/test/resources/hbase-site.xml
@@ -23,6 +23,22 @@
 <configuration>
   <property>
     <name>hbase.regionserver.msginterval</name>
+    <value>100</value>
+    <description>Interval between messages from the RegionServer to HMaster
+     in milliseconds.  Default is 15. Set this value low if you want unit
+     tests to be responsive.
+    </description>
+  </property>
+  <property>
+    <name>hbase.server.thread.wakefrequency</name>
+    <value>1000</value>
+    <value>100</value>
+    <description>Time to sleep in between searches for work (in milliseconds).
+     Used as sleep interval by service threads such as hbase:meta scanner and log roller.
+    </description>
+  </property>
+  <property>
+    <name>hbase.regionserver.msginterval</name>
     <value>1000</value>
     <description>Interval between messages from the RegionServer to HMaster
     in milliseconds.  Default is 15. Set this value low if you want unit
@@ -154,4 +170,119 @@
       Enable replay sanity checks on procedure tests.
     </description>
   </property>
+  <property>
+    <name>hbase.regionserver.handler.count</name>
+    <value>3</value>
+    <description>Default is 30</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.metahandler.count</name>
+    <value>3</value>
+    <description>Default is 20</description>
+   </property>
+  <property>
+    <name>hbase.netty.worker.count</name>
+    <value>3</value>
+    <description>Default is 0</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.threads.max</name>
+    <value>6</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.htable.threads.max</name>
+    <value>3</value>
+    <description>Default is MAX_INTEGER</description>
+  </property>
+  <property>
+    <name>hbase.region.replica.replication.threads.max</name>
+    <value>7</value>
+    <description>Default is 256</description>
+  </property>
+  <property>
+    <name>hbase.rest.threads.max</name>
+    <value>5</value>
+    <description>Default is 100</description>
+  </property>
+  <property>
+    <name>hbase.replication.bulkload.copy.maxthreads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.loadincremental.threads.max</name>
+    <value>1</value>
+    <description>Default is # of CPUs</description>
+  </property>
+  <property>
+    <name>hbase.hstore.flusher.count</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.oldwals.cleaner.thread.size</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.master.procedure.threads</name>
+    <value>5</value>
+    <description>Default is at least 16</description>
+  </property>
+  <property>
+    <name>hbase.procedure.remote.dispatcher.threadpool.size</name>
+    <value>3</value>
+    <description>Default is 128</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.closeregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.openpriorityregion.threads</name>
+    <value>1</value>
+    <description>Default is 3</description>
+  </property>
+  <property>
+    <name>hbase.storescanner.parallel.seek.threads</name>
+    <value>3</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hfile.compaction.discharger.thread.count</name>
+    <value>1</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.executor.refresh.peer.threads</name>
+    <value>1</value>
+    <description>Default is 2</description>
+  </property>
+  <property>
+    <name>hbase.hregion.open.and.init.threads.max</name>
+    <value>3</value>
+    <description>Default is 16 or # of Regions</description>
+  </property>
+  <property>
+    <name>hbase.master.handler.count</name>
+    <value>7</value>
+    <description>Default is 25</description>
+  </property>
+  <property>
+    <name>hbase.replication.source.maxthreads</name>
+    <value></value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>hbase.hconnection.meta.lookup.threads.max</name>
+    <value>5</value>
+    <description>Default is 128</description>
+  </property>
 </configuration>
diff --git a/hbase-rest/src/test/resources/hdfs-site.xml b/hbase-thrift/src/test/resources/hdfs-site.xml
similarity index 63%
copy from hbase-rest/src/test/resources/hdfs-site.xml
copy to hbase-thrift/src/test/resources/hdfs-site.xml
index 03be0c7..9230105 100644
--- a/hbase-rest/src/test/resources/hdfs-site.xml
+++ b/hbase-thrift/src/test/resources/hdfs-site.xml
@@ -29,4 +29,28 @@
     <name>dfs.namenode.fs-limits.min-block-size</name>
     <value>0</value>
   </property>
-</configuration>
\ No newline at end of file
+  <property>
+    <name>dfs.datanode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <property>
+    <name>dfs.namenode.service.handler.count</name>
+    <value>5</value>
+    <description>Default is 10</description>
+  </property>
+  <!--
+   Constraining this config makes tests fail.
+  <property>
+    <name>dfs.datanode.max.transfer.threads</name>
+    <value>16</value>
+    <description>Default is 4096. If constrain this
+    too much, tests do not complete.</description>
+  </property>
+  -->
+</configuration>
diff --git a/pom.xml b/pom.xml
index c9ebf51..cd21e05 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1480,11 +1480,16 @@
     <surefire.Xmx>2800m</surefire.Xmx>
     <surefire.cygwinXmx>2800m</surefire.cygwinXmx>
     <!--Mark our test runs with '-Dhbase.build.id' so we can identify a surefire test as ours in a process listing
+
+      And for netty eventloops that have no explicit configuration, netty sets
+      nioeventloopgroup thread count to CPU count * 2. Thats too much for mini
+      clusters/tests.
      -->
     <hbase-surefire.argLine>-enableassertions -Dhbase.build.id=${build.id} -Xmx${surefire.Xmx}
       -Djava.security.egd=file:/dev/./urandom -Djava.net.preferIPv4Stack=true
       -Djava.awt.headless=true -Djdk.net.URLClassPath.disableClassPathURLCheck=true
       -Dorg.apache.hbase.thirdparty.io.netty.leakDetection.level=advanced
+      -Dio.netty.eventLoopThreads=3
     </hbase-surefire.argLine>
     <hbase-surefire.cygwin-argLine>-enableassertions -Xmx${surefire.cygwinXmx}
       -Djava.security.egd=file:/dev/./urandom -Djava.net.preferIPv4Stack=true