HBASE-23752: Fix remaining test failures from nightly runs (#1102)

TestFromClientSideWithCoprocessor: Initialization bug causing parameterized
runs to fail.
TestCustomSaslAuthenticationProvider: Test config had to be fixed because
it was written pre-master registry implementation.
TestSnapshotScannerHDFSAclController: Cluster restart did not reset the
cached connection state.

Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
Signed-off-by: Josh Elser <elserj@apache.org>
(cherry picked from commit 02bd0eca53)
This commit is contained in:
Bharath Vissapragada 2020-02-03 10:15:32 -08:00
parent bd13416275
commit 09ca6bdcd4
14 changed files with 107 additions and 44 deletions

View File

@ -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);

View File

@ -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);

View File

@ -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<Message> 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.

View File

@ -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();

View File

@ -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);
}
}

View File

@ -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();
/**

View File

@ -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 " +

View File

@ -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)

View File

@ -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);

View File

@ -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);

View File

@ -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);

View File

@ -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<Object[]> parameters() {
List<Object[]> params = new ArrayList<>();
List<String> rpcClientImpls = Arrays.asList(
BlockingRpcClient.class.getName(), NettyRpcClient.class.getName());
List<String> 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<String,String> createUserDatabase() {
Map<String,String> 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<Void>() {
@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;
}
});
}

View File

@ -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());

View File

@ -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);