You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2015/09/25 02:39:39 UTC

hbase git commit: HBASE-14378 Get TestAccessController* passing again on branch-1

Repository: hbase
Updated Branches:
  refs/heads/branch-1 8290139d4 -> 68cb682c0


HBASE-14378 Get TestAccessController* passing again on branch-1


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/68cb682c
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/68cb682c
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/68cb682c

Branch: refs/heads/branch-1
Commit: 68cb682c0719ff312591d32dfce79d3a0d4b2b6d
Parents: 8290139
Author: stack <st...@apache.org>
Authored: Thu Sep 24 17:39:09 2015 -0700
Committer: stack <st...@apache.org>
Committed: Thu Sep 24 17:39:09 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/TestHCM.java |  32 ++---
 .../client/TestSnapshotCloneIndependence.java   |   4 +-
 .../hbase/mapreduce/TestImportExport.java       |   3 +
 .../mapreduce/TestMultithreadedTableMapper.java |   3 +
 .../master/TestDistributedLogSplitting.java     |   2 +-
 .../access/TestAccessControlFilter.java         |   5 +-
 .../security/access/TestAccessController.java   | 116 ++++++++++---------
 .../security/access/TestAccessController2.java  |  13 ++-
 .../TestWithDisabledAuthorization.java          |   7 +-
 9 files changed, 102 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/68cb682c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
index 49b2dbf..4f1fa6b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
@@ -136,6 +136,8 @@ public class TestHCM {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.getConfiguration().setBoolean(HConstants.STATUS_PUBLISHED, true);
+    // Up the handlers; this test needs more than usual.
+    TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10);
     TEST_UTIL.startMiniCluster(2);
   }
 
@@ -148,7 +150,7 @@ public class TestHCM {
     return HConnectionTestingUtility.getConnectionCount();
   }
 
-  @Test
+  @Test (timeout=120000)
   public void testClusterConnection() throws IOException {
     ThreadPoolExecutor otherPool = new ThreadPoolExecutor(1, 1,
         5, TimeUnit.SECONDS,
@@ -211,7 +213,7 @@ public class TestHCM {
    * Naive test to check that HConnection#getAdmin returns a properly constructed HBaseAdmin object
    * @throws IOException Unable to construct admin
    */
-  @Test
+  @Test (timeout=120000)
   public void testAdminFactory() throws IOException {
     Connection con1 = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
     Admin admin = con1.getAdmin();
@@ -221,7 +223,7 @@ public class TestHCM {
   }
 
   // Fails too often!  Needs work.  HBASE-12558
-  @Ignore @Test(expected = RegionServerStoppedException.class)
+  @Ignore @Test (expected = RegionServerStoppedException.class)
   public void testClusterStatus() throws Exception {
 
     TableName tn =
@@ -283,12 +285,12 @@ public class TestHCM {
    * Test that we can handle connection close: it will trigger a retry, but the calls will
    *  finish.
    */
-  @Test
+  @Test (timeout=120000)
   public void testConnectionCloseAllowsInterrupt() throws Exception {
     testConnectionClose(true);
   }
 
-  @Test
+  @Test (timeout=120000)
   public void testConnectionNotAllowsInterrupt() throws Exception {
     testConnectionClose(false);
   }
@@ -301,7 +303,7 @@ public class TestHCM {
    * succeeds. But the client won't wait that much, because 20 + 20 > 30, so the client
    * timeouted when the server answers.
    */
-  @Test
+  @Test (timeout=120000)
   public void testOperationTimeout() throws Exception {
     HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testOperationTimeout");
     hdt.addCoprocessor(SleepAndFailFirstTime.class.getName());
@@ -410,7 +412,7 @@ public class TestHCM {
   /**
    * Test that connection can become idle without breaking everything.
    */
-  @Test
+  @Test (timeout=120000)
   public void testConnectionIdle() throws Exception {
     TableName tableName = TableName.valueOf("HCM-testConnectionIdle");
     TEST_UTIL.createTable(tableName, FAM_NAM).close();
@@ -468,7 +470,7 @@ public class TestHCM {
      *  notification.
      * @throws Exception
      */
-  @Test
+  @Test (timeout=120000)
   public void testConnectionCut() throws Exception {
 
     TableName tableName = TableName.valueOf("HCM-testConnectionCut");
@@ -555,7 +557,7 @@ public class TestHCM {
     }
   }
 
-  @Test
+  @Test (timeout=120000)
   public void abortingHConnectionRemovesItselfFromHCM() throws Exception {
     // Save off current HConnections
     Map<HConnectionKey, HConnectionImplementation> oldHBaseInstances =
@@ -582,7 +584,7 @@ public class TestHCM {
    * that we really delete it.
    * @throws Exception
    */
-  @Test
+  @Test (timeout=120000)
   public void testRegionCaching() throws Exception{
     TEST_UTIL.createMultiRegionTable(TABLE_NAME, FAM_NAM).close();
     Configuration conf =  new Configuration(TEST_UTIL.getConfiguration());
@@ -768,7 +770,7 @@ public class TestHCM {
    * Test that Connection or Pool are not closed when managed externally
    * @throws Exception
    */
-  @Test
+  @Test (timeout=120000)
   public void testConnectionManagement() throws Exception{
     Table table0 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAM);
     Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
@@ -835,7 +837,7 @@ public class TestHCM {
    * Make sure that {@link Configuration} instances that are essentially the
    * same map to the same {@link HConnection} instance.
    */
-  @Test
+  @Test (timeout=120000)
   public void testConnectionSameness() throws Exception {
     Connection previousConnection = null;
     for (int i = 0; i < 2; i++) {
@@ -867,7 +869,7 @@ public class TestHCM {
    * @deprecated Tests deprecated functionality.  Remove in 1.0.
    */
   @Deprecated
-  @Test
+  @Test (timeout=120000)
   public void testConnectionUniqueness() throws Exception {
     int zkmaxconnections = TEST_UTIL.getConfiguration().
       getInt(HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS,
@@ -913,7 +915,7 @@ public class TestHCM {
     }
   }
 
-  @Test
+  @Test (timeout=120000)
   public void testClosing() throws Exception {
     Configuration configuration =
       new Configuration(TEST_UTIL.getConfiguration());
@@ -953,7 +955,7 @@ public class TestHCM {
    * Trivial test to verify that nobody messes with
    * {@link HConnectionManager#createConnection(Configuration)}
    */
-  @Test
+  @Test (timeout=120000)
   public void testCreateConnection() throws Exception {
     Configuration configuration = TEST_UTIL.getConfiguration();
     Connection c1 = ConnectionFactory.createConnection(configuration);

http://git-wip-us.apache.org/repos/asf/hbase/blob/68cb682c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
index 6b15e77..3122ae4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
@@ -69,8 +69,10 @@ public class TestSnapshotCloneIndependence {
     setupConf(UTIL.getConfiguration());
     UTIL.startMiniCluster(NUM_RS);
   }
-
+ 
   private static void setupConf(Configuration conf) {
+    // Up the handlers; this test needs more than usual.
+    conf.setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10);
     // enable snapshot support
     conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
     // disable the ui

http://git-wip-us.apache.org/repos/asf/hbase/blob/68cb682c/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
index 1073cdb..d18557e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
@@ -37,6 +37,7 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -100,6 +101,8 @@ public class TestImportExport {
 
   @BeforeClass
   public static void beforeClass() throws Exception {
+    // Up the handlers; this test needs more than usual.
+    UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10);
     UTIL.startMiniCluster();
     UTIL.startMiniMapReduceCluster();
     FQ_OUTPUT_DIR =  new Path(OUTPUT_DIR).makeQualified(FileSystem.get(UTIL.getConfiguration())).toString();

http://git-wip-us.apache.org/repos/asf/hbase/blob/68cb682c/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java
index 7ce4a63..6b2ee75 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java
@@ -31,6 +31,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -69,6 +70,8 @@ public class TestMultithreadedTableMapper {
 
   @BeforeClass
   public static void beforeClass() throws Exception {
+    // Up the handlers; this test needs more than usual.
+    UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10);
     UTIL.startMiniCluster();
     HTable table =
         UTIL.createMultiRegionTable(MULTI_REGION_TABLE_NAME, new byte[][] { INPUT_FAMILY,

http://git-wip-us.apache.org/repos/asf/hbase/blob/68cb682c/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
index 2364861..33d4e9b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
@@ -1793,4 +1793,4 @@ public class TestDistributedLogSplitting {
 
     return hrs;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/68cb682c/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java
index 42982d8..f54d42c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java
@@ -29,6 +29,7 @@ import java.util.UUID;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
@@ -70,6 +71,8 @@ public class TestAccessControlFilter extends SecureTestUtil {
   public static void setupBeforeClass() throws Exception {
     TEST_UTIL = new HBaseTestingUtility();
     Configuration conf = TEST_UTIL.getConfiguration();
+    // Up the handlers; this test needs more than usual.
+    conf.setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10);
     enableSecurity(conf);
     verifyConfiguration(conf);
 
@@ -89,7 +92,7 @@ public class TestAccessControlFilter extends SecureTestUtil {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testQualifierAccess() throws Exception {
     final Table table = createTable(TEST_UTIL, TABLE, new byte[][] { FAMILY });
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/68cb682c/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index b11095f..a148bda 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -192,6 +192,8 @@ public class TestAccessController extends SecureTestUtil {
   public static void setupBeforeClass() throws Exception {
     // setup configuration
     conf = TEST_UTIL.getConfiguration();
+    // Up the handlers; this test needs more than usual.
+    conf.setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10);
     // Enable security
     enableSecurity(conf);
     // In this particular test case, we can't use SecureBulkLoadEndpoint because its doAs will fail
@@ -318,7 +320,7 @@ public class TestAccessController extends SecureTestUtil {
             TEST_TABLE.getNamespaceAsString()).size());
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testSecurityCapabilities() throws Exception {
     List<SecurityCapability> capabilities = TEST_UTIL.getConnection().getAdmin()
       .getSecurityCapabilities();
@@ -328,7 +330,7 @@ public class TestAccessController extends SecureTestUtil {
       capabilities.contains(SecurityCapability.CELL_AUTHORIZATION));
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testTableCreate() throws Exception {
     AccessTestAction createTable = new AccessTestAction() {
       @Override
@@ -348,7 +350,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_READ, USER_GROUP_WRITE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testTableModify() throws Exception {
     AccessTestAction modifyTable = new AccessTestAction() {
       @Override
@@ -367,7 +369,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyDenied(modifyTable, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testTableDelete() throws Exception {
     AccessTestAction deleteTable = new AccessTestAction() {
       @Override
@@ -383,7 +385,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyDenied(deleteTable, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testTableTruncate() throws Exception {
     AccessTestAction truncateTable = new AccessTestAction() {
       @Override
@@ -400,7 +402,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyDenied(truncateTable, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testAddColumn() throws Exception {
     final HColumnDescriptor hcd = new HColumnDescriptor("fam_new");
     AccessTestAction action = new AccessTestAction() {
@@ -417,7 +419,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyDenied(action, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testModifyColumn() throws Exception {
     final HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
     hcd.setMaxVersions(10);
@@ -435,7 +437,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyDenied(action, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testDeleteColumn() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override
@@ -451,7 +453,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyDenied(action, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testTableDisable() throws Exception {
     AccessTestAction disableTable = new AccessTestAction() {
       @Override
@@ -480,7 +482,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_CREATE, USER_GROUP_ADMIN, USER_GROUP_READ, USER_GROUP_WRITE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testTableEnable() throws Exception {
     AccessTestAction enableTable = new AccessTestAction() {
       @Override
@@ -496,7 +498,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyDenied(enableTable, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testMove() throws Exception {
     List<HRegionLocation> regions;
     try (RegionLocator locator = systemUserConnection.getRegionLocator(TEST_TABLE)) {
@@ -519,7 +521,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_WRITE, USER_GROUP_CREATE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testAssign() throws Exception {
     List<HRegionLocation> regions;
     try (RegionLocator locator = systemUserConnection.getRegionLocator(TEST_TABLE)) {
@@ -540,7 +542,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_WRITE, USER_GROUP_CREATE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testUnassign() throws Exception {
     List<HRegionLocation> regions;
     try (RegionLocator locator = systemUserConnection.getRegionLocator(TEST_TABLE)) {
@@ -561,7 +563,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_WRITE, USER_GROUP_CREATE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testRegionOffline() throws Exception {
     List<HRegionLocation> regions;
     try (RegionLocator locator = systemUserConnection.getRegionLocator(TEST_TABLE)) {
@@ -582,7 +584,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_WRITE, USER_GROUP_CREATE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testBalance() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override
@@ -597,7 +599,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_WRITE, USER_GROUP_CREATE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testBalanceSwitch() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override
@@ -612,7 +614,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_WRITE, USER_GROUP_CREATE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testShutdown() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override
@@ -627,7 +629,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_WRITE, USER_GROUP_CREATE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testStopMaster() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override
@@ -648,7 +650,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyDenied(action, USER_NONE, USER_RO, USER_GROUP_ADMIN, USER_GROUP_READ, USER_GROUP_CREATE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testSplit() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override
@@ -663,7 +665,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_WRITE, USER_GROUP_CREATE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testSplitWithSplitRow() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override
@@ -680,7 +682,7 @@ public class TestAccessController extends SecureTestUtil {
         USER_GROUP_WRITE, USER_GROUP_CREATE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testMergeRegions() throws Exception {
     final TableName tname = TableName.valueOf("testMergeRegions");
     createTestTable(tname);
@@ -705,7 +707,7 @@ public class TestAccessController extends SecureTestUtil {
     }
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testFlush() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override
@@ -720,7 +722,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyDenied(action, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testCompact() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override
@@ -748,7 +750,7 @@ public class TestAccessController extends SecureTestUtil {
         USER_GROUP_WRITE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testRead() throws Exception {
     // get action
     AccessTestAction getAction = new AccessTestAction() {
@@ -790,7 +792,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyRead(scanAction);
   }
 
-  @Test
+  @Test (timeout=180000)
   // test put, delete, increment
   public void testWrite() throws Exception {
     // put action
@@ -839,7 +841,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyWrite(incrementAction);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testReadWrite() throws Exception {
     // action for checkAndDelete
     AccessTestAction checkAndDeleteAction = new AccessTestAction() {
@@ -874,7 +876,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyReadWrite(checkAndPut);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testBulkLoad() throws Exception {
     try {
       FileSystem fs = TEST_UTIL.getTestFileSystem();
@@ -988,7 +990,7 @@ public class TestAccessController extends SecureTestUtil {
     }
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testAppend() throws Exception {
 
     AccessTestAction appendAction = new AccessTestAction() {
@@ -1015,7 +1017,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_ADMIN);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testGrantRevoke() throws Exception {
     AccessTestAction grantAction = new AccessTestAction() {
       @Override
@@ -1097,7 +1099,7 @@ public class TestAccessController extends SecureTestUtil {
     }
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testPostGrantRevoke() throws Exception {
     final TableName tableName =
         TableName.valueOf("TempTable");
@@ -1350,7 +1352,7 @@ public class TestAccessController extends SecureTestUtil {
     return perms.contains(userPermission);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testPostGrantRevokeAtQualifierLevel() throws Exception {
     final TableName tableName =
         TableName.valueOf("testGrantRevokeAtQualifierLevel");
@@ -1454,7 +1456,7 @@ public class TestAccessController extends SecureTestUtil {
     }
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testPermissionList() throws Exception {
     final TableName tableName =
         TableName.valueOf("testPermissionList");
@@ -1585,7 +1587,7 @@ public class TestAccessController extends SecureTestUtil {
     }
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testGlobalPermissionList() throws Exception {
     List<UserPermission> perms;
     Table acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
@@ -1617,7 +1619,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyDenied(action, USER_CREATE, USER_RW, USER_NONE, USER_RO);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testCheckPermissions() throws Exception {
     // --------------------------------------
     // test global permissions
@@ -1784,7 +1786,7 @@ public class TestAccessController extends SecureTestUtil {
     }
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testStopRegionServer() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override
@@ -1799,7 +1801,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_WRITE, USER_GROUP_CREATE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testRollWALWriterRequest() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override
@@ -1814,7 +1816,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_WRITE, USER_GROUP_CREATE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testOpenRegion() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override
@@ -1829,7 +1831,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_READ, USER_GROUP_WRITE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testCloseRegion() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override
@@ -1844,7 +1846,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_READ, USER_GROUP_WRITE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testSnapshot() throws Exception {
     Admin admin = TEST_UTIL.getHBaseAdmin();
     final HTableDescriptor htd = admin.getTableDescriptor(TEST_TABLE);
@@ -1905,7 +1907,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testSnapshotWithOwner() throws Exception {
     Admin admin = TEST_UTIL.getHBaseAdmin();
     final HTableDescriptor htd = admin.getTableDescriptor(TEST_TABLE);
@@ -1965,7 +1967,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testGlobalAuthorizationForNewRegisteredRS() throws Exception {
     LOG.debug("Test for global authorization for a new registered RegionServer.");
     MiniHBaseCluster hbaseCluster = TEST_UTIL.getHBaseCluster();
@@ -2028,7 +2030,7 @@ public class TestAccessController extends SecureTestUtil {
     }
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testTableDescriptorsEnumeration() throws Exception {
     User TABLE_ADMIN = User.createUserForTesting(conf, "UserA", new String[0]);
 
@@ -2072,7 +2074,7 @@ public class TestAccessController extends SecureTestUtil {
     }
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testTableNameEnumeration() throws Exception {
     AccessTestAction listTablesAction = new AccessTestAction() {
       @Override
@@ -2094,7 +2096,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyIfEmptyList(listTablesAction, USER_NONE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testTableDeletion() throws Exception {
     User TABLE_ADMIN = User.createUserForTesting(conf, "TestUser", new String[0]);
     final TableName tname = TableName.valueOf("testTableDeletion");
@@ -2133,7 +2135,7 @@ public class TestAccessController extends SecureTestUtil {
     createTable(TEST_UTIL, htd, new byte[][]{Bytes.toBytes("s")});
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testNamespaceUserGrant() throws Exception {
     AccessTestAction getAction = new AccessTestAction() {
       @Override
@@ -2157,7 +2159,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyDenied(getAction, USER_NONE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testAccessControlClientGrantRevoke() throws Exception {
     // Create user for testing, who has no READ privileges by default.
     User testGrantRevoke = User.createUserForTesting(conf, "testGrantRevoke", new String[0]);
@@ -2196,7 +2198,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyDenied(getAction, testGrantRevoke);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testAccessControlClientGlobalGrantRevoke() throws Exception {
     // Create user for testing, who has no READ privileges by default.
     User testGlobalGrantRevoke = User.createUserForTesting(conf,
@@ -2240,7 +2242,7 @@ public class TestAccessController extends SecureTestUtil {
     }
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testAccessControlClientGrantRevokeOnNamespace() throws Exception {
     // Create user for testing, who has no READ privileges by default.
     User testNS = User.createUserForTesting(conf, "testNS", new String[0]);
@@ -2330,7 +2332,7 @@ public class TestAccessController extends SecureTestUtil {
     }
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testCoprocessorExec() throws Exception {
     // Set up our ping endpoint service on all regions of our test table
     for (JVMClusterUtil.RegionServerThread thread:
@@ -2381,7 +2383,7 @@ public class TestAccessController extends SecureTestUtil {
     }
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testReservedCellTags() throws Exception {
     AccessTestAction putWithReservedTag = new AccessTestAction() {
       @Override
@@ -2405,7 +2407,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyDenied(putWithReservedTag, USER_OWNER, USER_ADMIN, USER_CREATE, USER_RW, USER_RO);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testSetQuota() throws Exception {
     AccessTestAction setUserQuotaAction = new AccessTestAction() {
       @Override
@@ -2472,7 +2474,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testGetNamespacePermission() throws Exception {
     String namespace = "testGetNamespacePermission";
     NamespaceDescriptor desc = NamespaceDescriptor.create(namespace).build();
@@ -2489,7 +2491,7 @@ public class TestAccessController extends SecureTestUtil {
     deleteNamespace(TEST_UTIL, namespace);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testTruncatePerms() throws Exception {
     try {
       List<UserPermission> existingPerms = AccessControlClient.getUserPermissions(
@@ -2522,7 +2524,7 @@ public class TestAccessController extends SecureTestUtil {
     };
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testAccessControlClientUserPerms() throws Exception {
     TableName tname = TableName.valueOf("testAccessControlClientUserPerms");
     createTestTable(tname);
@@ -2541,7 +2543,7 @@ public class TestAccessController extends SecureTestUtil {
     }
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testAccessControllerUserPermsRegexHandling() throws Exception {
     User testRegexHandler = User.createUserForTesting(conf, "testRegexHandling", new String[0]);
 
@@ -2598,7 +2600,7 @@ public class TestAccessController extends SecureTestUtil {
       USER_GROUP_ADMIN);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testPrepareAndCleanBulkLoad() throws Exception {
     AccessTestAction prepareBulkLoadAction = new AccessTestAction() {
       @Override
@@ -2620,7 +2622,7 @@ public class TestAccessController extends SecureTestUtil {
     verifyAnyCreate(cleanupBulkLoadAction);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testReplicateLogEntries() throws Exception {
     AccessTestAction replicateLogEntriesAction = new AccessTestAction() {
       @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/68cb682c/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
index 0055740..d400fa6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
@@ -106,6 +107,8 @@ public class TestAccessController2 extends SecureTestUtil {
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
     conf = TEST_UTIL.getConfiguration();
+    // Up the handlers; this test needs more than usual.
+    conf.setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10);
     // Enable security
     enableSecurity(conf);
     // Verify enableSecurity sets up what we require
@@ -166,7 +169,7 @@ public class TestAccessController2 extends SecureTestUtil {
     assertEquals(0, AccessControlLists.getNamespacePermissions(conf, namespace).size());
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testCreateWithCorrectOwner() throws Exception {
     // Create a test user
     final User testUser = User.createUserForTesting(TEST_UTIL.getConfiguration(), "TestUser",
@@ -203,7 +206,7 @@ public class TestAccessController2 extends SecureTestUtil {
     assertTrue(perms.get(0).implies(Permission.Action.ADMIN));
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testCreateTableWithGroupPermissions() throws Exception {
     grantGlobal(TEST_UTIL, TESTGROUP_1_NAME, Action.CREATE);
     try {
@@ -228,7 +231,7 @@ public class TestAccessController2 extends SecureTestUtil {
     }
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testACLTableAccess() throws Exception {
     final Configuration conf = TEST_UTIL.getConfiguration();
 
@@ -471,7 +474,7 @@ public class TestAccessController2 extends SecureTestUtil {
   public static class MyAccessController extends AccessController {
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testCoprocessorLoading() throws Exception {
     MasterCoprocessorHost cpHost =
         TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterCoprocessorHost();
@@ -486,7 +489,7 @@ public class TestAccessController2 extends SecureTestUtil {
       MyAccessController.class, ACCESS_CONTROLLER, Coprocessor.PRIORITY_HIGHEST, 1, conf);
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testACLZNodeDeletion() throws Exception {
     String baseAclZNode = "/hbase/acl/";
     String ns = "testACLZNodeDeletionNamespace";

http://git-wip-us.apache.org/repos/asf/hbase/blob/68cb682c/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestWithDisabledAuthorization.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestWithDisabledAuthorization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestWithDisabledAuthorization.java
index f9538ea..a124392 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestWithDisabledAuthorization.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestWithDisabledAuthorization.java
@@ -72,7 +72,8 @@ public class TestWithDisabledAuthorization {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     conf = TEST_UTIL.getConfiguration();
-
+    // Up the handlers; this test needs more than usual.
+    conf.setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10);
     // Set up superuser
     SecureTestUtil.configureSuperuser(conf);
 
@@ -113,7 +114,7 @@ public class TestWithDisabledAuthorization {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testManageUserAuths() throws Throwable {
     // Even though authorization is disabled, we should be able to manage user auths
 
@@ -187,7 +188,7 @@ public class TestWithDisabledAuthorization {
     assertEquals(0, authsList.size());
   }
 
-  @Test
+  @Test (timeout=180000)
   public void testPassiveVisibility() throws Exception {
     // No values should be filtered regardless of authorization if we are passive
     try (Table t = createTableAndWriteDataWithLabels(