diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java index f4a651f3397..5a5de4222d0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java @@ -74,6 +74,10 @@ public class TestZooKeeper { public static void setUpBeforeClass() throws Exception { // Test we can first start the ZK cluster by itself Configuration conf = TEST_UTIL.getConfiguration(); + // A couple of tests rely on master expiring ZK session, hence killing the only master. So it + // makes sense only for ZK registry. Enforcing it. + conf.set(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, + HConstants.ZK_CONNECTION_REGISTRY_CLASS); TEST_UTIL.startMiniDFSCluster(2); TEST_UTIL.startMiniZKCluster(); conf.setInt(HConstants.ZK_SESSION_TIMEOUT, 1000); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index 3eaadfc575f..5beef533c7f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -521,6 +521,9 @@ public class TestAdmin2 extends TestAdminBase { public void testCheckHBaseAvailableWithoutCluster() { Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); + // Test makes sense only when ZK connection registry is in use. + conf.set(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, + HConstants.ZK_CONNECTION_REGISTRY_CLASS); // Change the ZK address to go to something not used. conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, 9999)+10); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java index c376df9f3aa..e0e7b7111e1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java @@ -114,7 +114,7 @@ public class TestClientTimeouts { } catch (MasterNotRunningException ex) { // Since we are randomly throwing SocketTimeoutExceptions, it is possible to get // a MasterNotRunningException. It's a bug if we get other exceptions. - lastFailed = true; + lastFailed = true; } finally { if(admin != null) { admin.close(); @@ -206,7 +206,8 @@ public class TestClientTimeouts { public void callMethod(MethodDescriptor md, RpcController controller, Message param, Message returnType, RpcCallback done) { RandomTimeoutBlockingRpcChannel.invokations.getAndIncrement(); - if (ThreadLocalRandom.current().nextFloat() < RandomTimeoutBlockingRpcChannel.CHANCE_OF_TIMEOUT) { + if (ThreadLocalRandom.current().nextFloat() < + RandomTimeoutBlockingRpcChannel.CHANCE_OF_TIMEOUT) { // throw a ServiceException, because that is the only exception type that // {@link ProtobufRpcEngine} throws. If this RpcEngine is used with a different // "actual" type, this may not properly mimic the underlying RpcEngine. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java index 98344fe3fcb..89a954020f7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java @@ -842,11 +842,13 @@ public class TestConnectionImplementation { public void testConnection() throws Exception{ // We create an empty config and add the ZK address. Configuration c = new Configuration(); + // This test only makes sense for ZK based connection registry. + c.set(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, + HConstants.ZK_CONNECTION_REGISTRY_CLASS); c.set(HConstants.ZOOKEEPER_QUORUM, TEST_UTIL.getConfiguration().get(HConstants.ZOOKEEPER_QUORUM)); c.set(HConstants.ZOOKEEPER_CLIENT_PORT, TEST_UTIL.getConfiguration().get(HConstants.ZOOKEEPER_CLIENT_PORT)); - // This should be enough to connect ClusterConnection conn = (ClusterConnection) ConnectionFactory.createConnection(c); assertTrue(conn.isMasterRunning()); @@ -1028,6 +1030,9 @@ public class TestConnectionImplementation { @Test public void testConnectionRideOverClusterRestart() throws IOException, InterruptedException { Configuration config = new Configuration(TEST_UTIL.getConfiguration()); + // This test only makes sense for ZK based connection registry. + config.set(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, + HConstants.ZK_CONNECTION_REGISTRY_CLASS); final TableName tableName = TableName.valueOf(name.getMethodName()); TEST_UTIL.createTable(tableName, new byte[][] {FAM_NAM}).close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java index d78976e63af..723b9971add 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java @@ -49,10 +49,7 @@ public class TestFromClientSideWithCoprocessor extends TestFromClientSide { } public TestFromClientSideWithCoprocessor(Class registry, int numHedgedReqs) throws Exception { - if (TEST_UTIL == null) { - // It is ok to initialize once because the test is parameterized for a single dimension. - initialize(registry, numHedgedReqs, NoOpScanPolicyObserver.class, - MultiRowMutationEndpoint.class); - } + initialize(registry, numHedgedReqs, NoOpScanPolicyObserver.class, + MultiRowMutationEndpoint.class); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java index eaf5f82e56d..7bd9074b49f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java @@ -64,7 +64,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.junit.AfterClass; import org.junit.Before; -import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; @@ -95,16 +94,6 @@ public class TestScannersFromClientSide { private static byte [] QUALIFIER = Bytes.toBytes("testQualifier"); private static byte [] VALUE = Bytes.toBytes("testValue"); - /** - * @throws java.lang.Exception - */ - @BeforeClass - public static void setUpBeforeClass() throws Exception { - Configuration conf = TEST_UTIL.getConfiguration(); - conf.setLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY, 10 * 1024 * 1024); - TEST_UTIL.startMiniCluster(3); - } - @Rule public TableNameTestRule name = new TableNameTestRule(); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java index d8a228e2432..0d140ae0ec1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java @@ -101,7 +101,7 @@ public class TestZKConnectionRegistry { public void testIndependentZKConnections() throws IOException { try (ReadOnlyZKClient zk1 = REGISTRY.getZKClient()) { Configuration otherConf = new Configuration(TEST_UTIL.getConfiguration()); - otherConf.set(HConstants.ZOOKEEPER_QUORUM, "127.0.0.1"); + otherConf.set(HConstants.ZOOKEEPER_QUORUM, "localhost"); try (ZKConnectionRegistry otherRegistry = new ZKConnectionRegistry(otherConf)) { ReadOnlyZKClient zk2 = otherRegistry.getZKClient(); assertNotSame("Using a different configuration / quorum should result in different " + 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 f93b2ced2cc..8abca21ede1 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 @@ -227,7 +227,6 @@ public class TestReplicationBase { private static void startClusters() throws Exception { UTIL1.startMiniZKCluster(); MiniZooKeeperCluster miniZK = UTIL1.getZkCluster(); - admin = new ReplicationAdmin(CONF1); LOG.info("Setup first Zk"); UTIL2.setZkCluster(miniZK); @@ -239,6 +238,7 @@ public class TestReplicationBase { // as a component in deciding maximum number of parallel batches to send to the peer cluster. UTIL2.startMiniCluster(NUM_SLAVES2); + admin = new ReplicationAdmin(CONF1); hbaseAdmin = ConnectionFactory.createConnection(CONF1).getAdmin(); TableDescriptor table = TableDescriptorBuilder.newBuilder(tableName) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java index ad99134e13f..dd3b936aba3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java @@ -120,7 +120,6 @@ public class TestReplicationWithTags { // Have to reget conf1 in case zk cluster location different // than default conf1 = utility1.getConfiguration(); - replicationAdmin = new ReplicationAdmin(conf1); LOG.info("Setup first Zk"); // Base conf2 on conf1 so it gets the right zk cluster. @@ -140,6 +139,7 @@ public class TestReplicationWithTags { utility1.startMiniCluster(2); utility2.startMiniCluster(2); + replicationAdmin = new ReplicationAdmin(conf1); ReplicationPeerConfig rpc = new ReplicationPeerConfig(); rpc.setClusterKey(utility2.getClusterKey()); replicationAdmin.addPeer("2", rpc, null); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java index 6078f55e1a0..4b9b967f106 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java @@ -98,13 +98,13 @@ public class TestGlobalReplicationThrottler { utility2.setZkCluster(miniZK); new ZKWatcher(conf2, "cluster2", null, true); - ReplicationAdmin admin1 = new ReplicationAdmin(conf1); ReplicationPeerConfig rpc = new ReplicationPeerConfig(); rpc.setClusterKey(utility2.getClusterKey()); utility1.startMiniCluster(); utility2.startMiniCluster(); + ReplicationAdmin admin1 = new ReplicationAdmin(conf1); admin1.addPeer("peer1", rpc, null); admin1.addPeer("peer2", rpc, null); admin1.addPeer("peer3", rpc, null); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestSnapshotScannerHDFSAclController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestSnapshotScannerHDFSAclController.java index 7ef4cf83ee7..7424bc44676 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestSnapshotScannerHDFSAclController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestSnapshotScannerHDFSAclController.java @@ -910,6 +910,11 @@ public class TestSnapshotScannerHDFSAclController { TEST_UTIL.restartHBaseCluster(1); TEST_UTIL.waitUntilNoRegionsInTransition(); + // reset the cached configs after restart + conf = TEST_UTIL.getConfiguration(); + admin = TEST_UTIL.getAdmin(); + helper = new SnapshotScannerHDFSAclHelper(conf, admin.getConnection()); + Path tmpNsDir = helper.getPathHelper().getTmpNsDir(namespace); assertTrue(fs.exists(tmpNsDir)); // check all regions in tmp table2 dir are archived @@ -917,7 +922,6 @@ public class TestSnapshotScannerHDFSAclController { // create table1 and snapshot TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table); - admin = TEST_UTIL.getAdmin(); aclTable = TEST_UTIL.getConnection().getTable(PermissionStorage.ACL_TABLE_NAME); admin.snapshot(snapshot, table); TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, 6); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/provider/TestCustomSaslAuthenticationProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/provider/TestCustomSaslAuthenticationProvider.java index 957810e692e..09c6ca84c4c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/provider/TestCustomSaslAuthenticationProvider.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/provider/TestCustomSaslAuthenticationProvider.java @@ -17,11 +17,11 @@ */ package org.apache.hadoop.hbase.security.provider; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; - import java.io.ByteArrayInputStream; import java.io.DataInput; import java.io.DataInputStream; @@ -30,13 +30,14 @@ import java.io.File; import java.io.IOException; import java.net.InetAddress; import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; - import javax.security.auth.callback.Callback; import javax.security.auth.callback.CallbackHandler; import javax.security.auth.callback.NameCallback; @@ -47,7 +48,6 @@ import javax.security.sasl.RealmCallback; import javax.security.sasl.RealmChoiceCallback; import javax.security.sasl.Sasl; import javax.security.sasl.SaslClient; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; @@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.LocalHBaseCluster; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; @@ -69,6 +70,8 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.ipc.BlockingRpcClient; +import org.apache.hadoop.hbase.ipc.NettyRpcClient; +import org.apache.hadoop.hbase.ipc.NettyRpcServer; import org.apache.hadoop.hbase.ipc.RpcClientFactory; import org.apache.hadoop.hbase.ipc.RpcServerFactory; import org.apache.hadoop.hbase.ipc.SimpleRpcServer; @@ -93,6 +96,7 @@ import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; +import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; @@ -100,10 +104,11 @@ import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - +import org.apache.hbase.thirdparty.com.google.common.base.Throwables; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation; /** @@ -112,6 +117,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformati * This tests holds a "user database" in memory as a hashmap. Clients provide their password * in the client Hadoop configuration. The servers validate this password via the "user database". */ +@RunWith(Parameterized.class) @Category({MediumTests.class, SecurityTests.class}) public class TestCustomSaslAuthenticationProvider { private static final Logger LOG = LoggerFactory.getLogger( @@ -126,6 +132,27 @@ public class TestCustomSaslAuthenticationProvider { private static final String USER1_PASSWORD = "foobarbaz"; private static final String USER2_PASSWORD = "bazbarfoo"; + @Parameterized.Parameters(name = "{index}: rpcClientImpl={0}, rpcServerImpl={1}") + public static Collection parameters() { + List params = new ArrayList<>(); + List rpcClientImpls = Arrays.asList( + BlockingRpcClient.class.getName(), NettyRpcClient.class.getName()); + List rpcServerImpls = Arrays.asList( + SimpleRpcServer.class.getName(), NettyRpcServer.class.getName()); + for (String rpcClientImpl : rpcClientImpls) { + for (String rpcServerImpl : rpcServerImpls) { + params.add(new Object[] { rpcClientImpl, rpcServerImpl }); + } + } + return params; + } + + @Parameterized.Parameter(0) + public String rpcClientImpl; + + @Parameterized.Parameter(1) + public String rpcServerImpl; + private static Map createUserDatabase() { Map db = new ConcurrentHashMap<>(); db.put("user1", USER1_PASSWORD); @@ -400,7 +427,7 @@ public class TestCustomSaslAuthenticationProvider { } } - static LocalHBaseCluster createCluster(HBaseTestingUtility util, File keytabFile, + static void createBaseCluster(HBaseTestingUtility util, File keytabFile, MiniKdc kdc) throws Exception { String servicePrincipal = "hbase/localhost"; String spnegoPrincipal = "HTTP/localhost"; @@ -416,8 +443,6 @@ public class TestCustomSaslAuthenticationProvider { util.startMiniDFSCluster(1); Path rootdir = util.getDataTestDirOnTestFS("TestGenerateDelegationToken"); FSUtils.setRootDir(util.getConfiguration(), rootdir); - LocalHBaseCluster cluster = new LocalHBaseCluster(util.getConfiguration(), 1); - return cluster; } private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); @@ -431,10 +456,6 @@ public class TestCustomSaslAuthenticationProvider { UTIL.getDataTestDir("keytab").toUri().getPath()); final MiniKdc kdc = UTIL.setupMiniKdc(KEYTAB_FILE); - // Switch back to NIO for now. - CONF.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, BlockingRpcClient.class.getName()); - CONF.set(RpcServerFactory.CUSTOM_RPC_SERVER_IMPL_CONF_KEY, SimpleRpcServer.class.getName()); - // Adds our test impls instead of creating service loader entries which // might inadvertently get them loaded on a real cluster. CONF.setStrings(SaslClientAuthenticationProviders.EXTRA_PROVIDERS_KEY, @@ -443,9 +464,24 @@ public class TestCustomSaslAuthenticationProvider { InMemoryServerProvider.class.getName()); CONF.set(SaslClientAuthenticationProviders.SELECTOR_KEY, InMemoryProviderSelector.class.getName()); + CONF.setLong(HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN, 600); + createBaseCluster(UTIL, KEYTAB_FILE, kdc); + } - CLUSTER = createCluster(UTIL, KEYTAB_FILE, kdc); + @Before + public void setUpBeforeTest() throws Exception { + CONF.unset(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY); + CONF.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, rpcClientImpl); + CONF.set(RpcServerFactory.CUSTOM_RPC_SERVER_IMPL_CONF_KEY, rpcServerImpl); + if (rpcClientImpl.equals(BlockingRpcClient.class.getName())) { + // Set the connection registry to ZKConnectionRegistry since hedging is not supported on + // blocking rpc clients. + CONF.set(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, + HConstants.ZK_CONNECTION_REGISTRY_CLASS); + } + CLUSTER = new LocalHBaseCluster(CONF, 1); CLUSTER.startup(); + createTable(); } @AfterClass @@ -457,14 +493,21 @@ public class TestCustomSaslAuthenticationProvider { UTIL.shutdownMiniZKCluster(); } + @After + public void shutDownCluster() throws IOException { + if (CLUSTER != null) { + UTIL.deleteTable(name.getTableName()); + CLUSTER.shutdown(); + } + } + @Rule - public TestName name = new TestName(); + public TableNameTestRule name = new TableNameTestRule(); TableName tableName; String clusterId; - @Before public void createTable() throws Exception { - tableName = TableName.valueOf(name.getMethodName()); + tableName = name.getTableName(); // Create a table and write a record as the service user (hbase) UserGroupInformation serviceUgi = UserGroupInformation.loginUserFromKeytabAndReturnUGI( @@ -517,22 +560,39 @@ public class TestCustomSaslAuthenticationProvider { }); } - @Test(expected = RetriesExhaustedException.class) + @Test public void testNegativeAuthentication() throws Exception { // Validate that we can read that record back out as the user with our custom auth'n final Configuration clientConf = new Configuration(CONF); + // This test does not work with master registry in branch-2 because of a nuance in the non-async + // connection implementation. See the detail below. + clientConf.set(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, + HConstants.ZK_CONNECTION_REGISTRY_CLASS); clientConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3); UserGroupInformation user1 = UserGroupInformation.createUserForTesting( "user1", new String[0]); user1.addToken(createPasswordToken("user1", "definitely not the password", clusterId)); user1.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { + // There is a slight behavioral difference here in the 3.x vs 2.x branches. 3.x branches + // use async client connection implementation which throws if there is an exception when + // fetching the clusterId(). 2.x branches that use non-async client falls back to using a + // DEFAULT cluster ID in such cases. 3.x behavior makes more sense, especially if the + // exception is of type InvalidToken (digest mis-match), however I did not want to fix it + // since it makes sense only when master registry is in use (which has RPCs to master). + // That is the reason if you see a slight difference in the test between 3.x and 2.x. try (Connection conn = ConnectionFactory.createConnection(clientConf); Table t = conn.getTable(tableName)) { t.get(new Get(Bytes.toBytes("r1"))); fail("Should not successfully authenticate with HBase"); - return null; + } catch (RetriesExhaustedException re) { + assertTrue(re.getMessage(), re.getMessage().contains("SaslException")); + } catch (Exception e) { + // Any other exception is unexpected. + fail("Unexpected exception caught, was expecting a authentication error: " + + Throwables.getStackTraceAsString(e)); } + return null; } }); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java index ba0dee3643e..ceb174cb423 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java @@ -115,7 +115,6 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit TEST_UTIL.startMiniZKCluster(); MiniZooKeeperCluster miniZK = TEST_UTIL.getZkCluster(); zkw1 = new ZKWatcher(conf, "cluster1", null, true); - admin = TEST_UTIL.getAdmin(); // Base conf2 on conf1 so it gets the right zk cluster. conf1 = HBaseConfiguration.create(conf); @@ -135,6 +134,7 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit // Wait for the labels table to become available TEST_UTIL.waitTableEnabled(LABELS_TABLE_NAME.getName(), 50000); TEST_UTIL1.startMiniCluster(1); + admin = TEST_UTIL.getAdmin(); ReplicationPeerConfig rpc = new ReplicationPeerConfig(); rpc.setClusterKey(TEST_UTIL1.getClusterKey()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java index cea037b4b80..e6a35b2b0bc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java @@ -166,7 +166,6 @@ public class TestVisibilityLabelsReplication { TEST_UTIL.startMiniZKCluster(); MiniZooKeeperCluster miniZK = TEST_UTIL.getZkCluster(); zkw1 = new ZKWatcher(conf, "cluster1", null, true); - admin = TEST_UTIL.getAdmin(); // Base conf2 on conf1 so it gets the right zk cluster. conf1 = HBaseConfiguration.create(conf); @@ -184,6 +183,7 @@ public class TestVisibilityLabelsReplication { zkw2 = new ZKWatcher(conf1, "cluster2", null, true); TEST_UTIL.startMiniCluster(1); + admin = TEST_UTIL.getAdmin(); // Wait for the labels table to become available TEST_UTIL.waitTableEnabled(LABELS_TABLE_NAME.getName(), 50000); TEST_UTIL1.startMiniCluster(1);