HBASE-13040 Possible failure of TestHMasterRPCException (Zhang Duo)
This commit is contained in:
parent
ca25a6a870
commit
54d70e61bf
|
@ -19,15 +19,18 @@
|
|||
|
||||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.SocketTimeoutException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.ipc.RpcClient;
|
||||
import org.apache.hadoop.hbase.ipc.RpcClientFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
|
@ -36,60 +39,80 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRe
|
|||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import com.google.protobuf.BlockingRpcChannel;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
@Category({MasterTests.class, MediumTests.class})
|
||||
@Category({ MasterTests.class, MediumTests.class })
|
||||
public class TestHMasterRPCException {
|
||||
|
||||
@Test
|
||||
public void testRPCException() throws Exception {
|
||||
HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
TEST_UTIL.startMiniZKCluster();
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
private static final Log LOG = LogFactory.getLog(TestHMasterRPCException.class);
|
||||
|
||||
private final HBaseTestingUtility testUtil = HBaseTestingUtility.createLocalHTU();
|
||||
|
||||
private HMaster master;
|
||||
|
||||
private RpcClient rpcClient;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
Configuration conf = testUtil.getConfiguration();
|
||||
conf.set(HConstants.MASTER_PORT, "0");
|
||||
conf.setInt(HConstants.ZK_SESSION_TIMEOUT, 2000);
|
||||
testUtil.startMiniZKCluster();
|
||||
|
||||
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(conf);
|
||||
HMaster hm = new HMaster(conf, cp);
|
||||
ServerName sm = hm.getServerName();
|
||||
RpcClient rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT);
|
||||
try {
|
||||
int i = 0;
|
||||
//retry the RPC a few times; we have seen SocketTimeoutExceptions if we
|
||||
//try to connect too soon. Retry on SocketTimeoutException.
|
||||
while (i < 20) {
|
||||
try {
|
||||
BlockingRpcChannel channel =
|
||||
rpcClient.createBlockingRpcChannel(sm, User.getCurrent(), 0);
|
||||
MasterProtos.MasterService.BlockingInterface stub =
|
||||
MasterProtos.MasterService.newBlockingStub(channel);
|
||||
stub.isMasterRunning(null, IsMasterRunningRequest.getDefaultInstance());
|
||||
fail();
|
||||
} catch (ServiceException ex) {
|
||||
IOException ie = ProtobufUtil.getRemoteException(ex);
|
||||
if (!(ie instanceof SocketTimeoutException)) {
|
||||
if (ie.getMessage().startsWith("org.apache.hadoop.hbase.ipc." +
|
||||
"ServerNotRunningYetException: Server is not running yet")) {
|
||||
// Done. Got the exception we wanted.
|
||||
System.out.println("Expected exception: " + ie.getMessage());
|
||||
return;
|
||||
} else {
|
||||
throw ex;
|
||||
}
|
||||
} else {
|
||||
System.err.println("Got SocketTimeoutException. Will retry. ");
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
fail("Unexpected throwable: " + t);
|
||||
}
|
||||
Thread.sleep(100);
|
||||
i++;
|
||||
}
|
||||
fail();
|
||||
} finally {
|
||||
ZooKeeperWatcher watcher = testUtil.getZooKeeperWatcher();
|
||||
ZKUtil.createWithParents(watcher, watcher.getMasterAddressZNode(), Bytes.toBytes("fake:123"));
|
||||
master = new HMaster(conf, cp);
|
||||
rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws IOException {
|
||||
if (rpcClient != null) {
|
||||
rpcClient.close();
|
||||
}
|
||||
if (master != null) {
|
||||
master.stopMaster();
|
||||
}
|
||||
testUtil.shutdownMiniZKCluster();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRPCException() throws IOException, InterruptedException, KeeperException {
|
||||
ServerName sm = master.getServerName();
|
||||
boolean fakeZNodeDelete = false;
|
||||
for (int i = 0; i < 20; i++) {
|
||||
try {
|
||||
BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(sm, User.getCurrent(), 0);
|
||||
MasterProtos.MasterService.BlockingInterface stub =
|
||||
MasterProtos.MasterService.newBlockingStub(channel);
|
||||
assertTrue(stub.isMasterRunning(null, IsMasterRunningRequest.getDefaultInstance())
|
||||
.getIsMasterRunning());
|
||||
return;
|
||||
} catch (ServiceException ex) {
|
||||
IOException ie = ProtobufUtil.getRemoteException(ex);
|
||||
// No SocketTimeoutException here. RpcServer is already started after the construction of
|
||||
// HMaster.
|
||||
assertTrue(ie.getMessage().startsWith(
|
||||
"org.apache.hadoop.hbase.ipc.ServerNotRunningYetException: Server is not running yet"));
|
||||
LOG.info("Expected exception: ", ie);
|
||||
if (!fakeZNodeDelete) {
|
||||
testUtil.getZooKeeperWatcher().getRecoverableZooKeeper()
|
||||
.delete(testUtil.getZooKeeperWatcher().getMasterAddressZNode(), -1);
|
||||
fakeZNodeDelete = true;
|
||||
}
|
||||
}
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue