|
@@ -22,15 +22,25 @@ import java.io.IOException;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.util.ArrayList;
|
|
|
|
|
|
+import com.google.protobuf.BlockingService;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
|
|
+import org.apache.hadoop.ha.protocolPB.HAServiceProtocolPB;
|
|
|
+import org.apache.hadoop.ha.protocolPB.HAServiceProtocolServerSideTranslatorPB;
|
|
|
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.HAServiceProtocolService;
|
|
|
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
|
|
+import org.apache.hadoop.ipc.RPC;
|
|
|
+import org.apache.hadoop.ipc.Server;
|
|
|
+import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.security.AccessControlException;
|
|
|
import org.mockito.Mockito;
|
|
|
|
|
|
import com.google.common.collect.Lists;
|
|
|
|
|
|
+import static org.apache.hadoop.fs.CommonConfigurationKeys.HA_HM_RPC_TIMEOUT_DEFAULT;
|
|
|
+
|
|
|
/**
|
|
|
* Test-only implementation of {@link HAServiceTarget}, which returns
|
|
|
* a mock implementation.
|
|
@@ -50,22 +60,33 @@ class DummyHAService extends HAServiceTarget {
|
|
|
DummySharedResource sharedResource;
|
|
|
public int fenceCount = 0;
|
|
|
public int activeTransitionCount = 0;
|
|
|
+ boolean testWithProtoBufRPC = false;
|
|
|
|
|
|
static ArrayList<DummyHAService> instances = Lists.newArrayList();
|
|
|
int index;
|
|
|
|
|
|
DummyHAService(HAServiceState state, InetSocketAddress address) {
|
|
|
+ this(state, address, false);
|
|
|
+ }
|
|
|
+
|
|
|
+ DummyHAService(HAServiceState state, InetSocketAddress address,
|
|
|
+ boolean testWithProtoBufRPC) {
|
|
|
this.state = state;
|
|
|
- this.proxy = makeMock();
|
|
|
+ this.testWithProtoBufRPC = testWithProtoBufRPC;
|
|
|
+ if (testWithProtoBufRPC) {
|
|
|
+ this.address = startAndGetRPCServerAddress(address);
|
|
|
+ } else {
|
|
|
+ this.address = address;
|
|
|
+ }
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ this.proxy = makeMock(conf, HA_HM_RPC_TIMEOUT_DEFAULT);
|
|
|
try {
|
|
|
- Configuration conf = new Configuration();
|
|
|
- conf.set(DUMMY_FENCE_KEY, DummyFencer.class.getName());
|
|
|
+ conf.set(DUMMY_FENCE_KEY, DummyFencer.class.getName());
|
|
|
this.fencer = Mockito.spy(
|
|
|
NodeFencer.create(conf, DUMMY_FENCE_KEY));
|
|
|
} catch (BadFencingConfigurationException e) {
|
|
|
throw new RuntimeException(e);
|
|
|
}
|
|
|
- this.address = address;
|
|
|
synchronized (instances) {
|
|
|
instances.add(this);
|
|
|
this.index = instances.size();
|
|
@@ -75,9 +96,42 @@ class DummyHAService extends HAServiceTarget {
|
|
|
public void setSharedResource(DummySharedResource rsrc) {
|
|
|
this.sharedResource = rsrc;
|
|
|
}
|
|
|
-
|
|
|
- private HAServiceProtocol makeMock() {
|
|
|
- return Mockito.spy(new MockHAProtocolImpl());
|
|
|
+
|
|
|
+ private InetSocketAddress startAndGetRPCServerAddress(InetSocketAddress serverAddress) {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+
|
|
|
+ try {
|
|
|
+ RPC.setProtocolEngine(conf,
|
|
|
+ HAServiceProtocolPB.class, ProtobufRpcEngine.class);
|
|
|
+ HAServiceProtocolServerSideTranslatorPB haServiceProtocolXlator =
|
|
|
+ new HAServiceProtocolServerSideTranslatorPB(new MockHAProtocolImpl());
|
|
|
+ BlockingService haPbService = HAServiceProtocolService
|
|
|
+ .newReflectiveBlockingService(haServiceProtocolXlator);
|
|
|
+
|
|
|
+ Server server = new RPC.Builder(conf)
|
|
|
+ .setProtocol(HAServiceProtocolPB.class)
|
|
|
+ .setInstance(haPbService)
|
|
|
+ .setBindAddress(serverAddress.getHostName())
|
|
|
+ .setPort(serverAddress.getPort()).build();
|
|
|
+ server.start();
|
|
|
+ return NetUtils.getConnectAddress(server);
|
|
|
+ } catch (IOException e) {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private HAServiceProtocol makeMock(Configuration conf, int timeoutMs) {
|
|
|
+ HAServiceProtocol service;
|
|
|
+ if (!testWithProtoBufRPC) {
|
|
|
+ service = new MockHAProtocolImpl();
|
|
|
+ } else {
|
|
|
+ try {
|
|
|
+ service = super.getProxy(conf, timeoutMs);
|
|
|
+ } catch (IOException e) {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return Mockito.spy(service);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -93,6 +147,9 @@ class DummyHAService extends HAServiceTarget {
|
|
|
@Override
|
|
|
public HAServiceProtocol getProxy(Configuration conf, int timeout)
|
|
|
throws IOException {
|
|
|
+ if (testWithProtoBufRPC) {
|
|
|
+ proxy = makeMock(conf, timeout);
|
|
|
+ }
|
|
|
return proxy;
|
|
|
}
|
|
|
|
|
@@ -168,7 +225,7 @@ class DummyHAService extends HAServiceTarget {
|
|
|
public HAServiceStatus getServiceStatus() throws IOException {
|
|
|
checkUnreachable();
|
|
|
HAServiceStatus ret = new HAServiceStatus(state);
|
|
|
- if (state == HAServiceState.STANDBY) {
|
|
|
+ if (state == HAServiceState.STANDBY || state == HAServiceState.ACTIVE) {
|
|
|
ret.setReadyToBecomeActive();
|
|
|
}
|
|
|
return ret;
|