@Test(timeout = 60000) @SetupRpcConnection(setupRpcClient = false) public void testIdleTimeoutWithActiveRequest() throws Exception { RpcConnectionKey rpcConnectionKey = new RpcConnectionKey(server.getListenAddress(), DummyProtocol.class, false); // 500 millis socket timeout Properties connParams = new Properties(); connParams.setProperty(RpcConstants.CLIENT_RETRY_NUM, retries + ""); connParams.setProperty(RpcConstants.CLIENT_SOCKET_TIMEOUT, String.valueOf(500)); BlockingRpcClient client = manager.newClient(rpcConnectionKey, connParams); assertTrue(client.isConnected()); BlockingInterface stub = client.getStub(); EchoMessage echoMessage = EchoMessage.newBuilder().setMessage(MESSAGE).build(); EchoMessage message = stub.delay(null, echoMessage); // 3 sec delay assertEquals(message, echoMessage); assertTrue(client.isConnected()); assertTrue(client.getActiveRequests() == 0); Thread.sleep(600); // timeout assertFalse(client.isConnected()); }
@Test(timeout = 60000) @SetupRpcConnection(setupRpcClient = false) public void testRequestTimeoutOnBusy() throws Exception { RpcConnectionKey rpcConnectionKey = new RpcConnectionKey(server.getListenAddress(), DummyProtocol.class, false); // 500 millis socket timeout Properties connParams = new Properties(); connParams.setProperty(RpcConstants.CLIENT_RETRY_NUM, retries + ""); connParams.setProperty(RpcConstants.CLIENT_SOCKET_TIMEOUT, String.valueOf(500)); connParams.setProperty(RpcConstants.CLIENT_HANG_DETECTION, "true"); BlockingRpcClient client = manager.newClient(rpcConnectionKey, connParams); assertTrue(client.isConnected()); BlockingInterface stub = client.getStub(); EchoMessage echoMessage = EchoMessage.newBuilder().setMessage(MESSAGE).build(); boolean expected = false; try { EchoMessage message = stub.busy(null, echoMessage); // 30 sec delay fail(); } catch (TajoServiceException e) { expected = true; } finally { client.close(); } assertTrue(expected); }
@Test(timeout = 60000) @SetupRpcConnection(setupRpcClient = false) public void testStubRecovery() throws Exception { RpcConnectionKey rpcConnectionKey = new RpcConnectionKey(server.getListenAddress(), DummyProtocol.class, false); Properties connParams = new Properties(); connParams.setProperty(RpcConstants.CLIENT_RETRY_NUM, String.valueOf(1)); BlockingRpcClient client = manager.newClient(rpcConnectionKey, connParams); EchoMessage echoMessage = EchoMessage.newBuilder().setMessage(MESSAGE).build(); int repeat = 5; assertTrue(client.isConnected()); BlockingInterface stub = client.getStub(); client.close(); // close connection assertFalse(client.isConnected()); for (int i = 0; i < repeat; i++) { try { EchoMessage response = stub.echo(null, echoMessage); assertEquals(MESSAGE, response.getMessage()); assertTrue(client.isConnected()); } finally { client.close(); // close connection assertFalse(client.isConnected()); } } }
@Test(timeout = 60000) @SetupRpcConnection(setupRpcServer = false, setupRpcClient = false) public void testClientRetryFailureOnStartup() throws Exception { retries = 2; ServerSocket serverSocket = new ServerSocket(0); final InetSocketAddress address = new InetSocketAddress("127.0.0.1", serverSocket.getLocalPort()); serverSocket.close(); EchoMessage message = EchoMessage.newBuilder().setMessage(MESSAGE).build(); RpcConnectionKey rpcConnectionKey = new RpcConnectionKey(address, DummyProtocol.class, false); Properties connParams = new Properties(); connParams.setProperty(RpcConstants.CLIENT_RETRY_NUM, retries + ""); BlockingRpcClient client = new BlockingRpcClient(NettyUtils.getDefaultEventLoopGroup(), rpcConnectionKey, connParams); try { client.connect(); fail(); } catch (ConnectTimeoutException e) { assertFalse(e.getMessage(), client.isConnected()); } BlockingInterface stub = client.getStub(); try { EchoMessage response = stub.echo(null, message); fail(); } catch (TajoServiceException e) { assertFalse(e.getMessage(), client.isConnected()); } RpcClientManager.cleanup(client); }
public void setUpRpcClient() throws Exception { retries = 1; Properties connParams = new Properties(); connParams.setProperty(RpcConstants.CLIENT_RETRY_NUM, "1"); connParams.setProperty( RpcConstants.CLIENT_SOCKET_TIMEOUT, String.valueOf(TimeUnit.SECONDS.toMillis(10))); connParams.setProperty(RpcConstants.CLIENT_HANG_DETECTION, "true"); RpcConnectionKey rpcConnectionKey = new RpcConnectionKey( RpcUtils.getConnectAddress(server.getListenAddress()), DummyProtocol.class, false); client = manager.newClient(rpcConnectionKey, connParams); assertTrue(client.isConnected()); stub = client.getStub(); }
protected LinkedList<TaskRequestEvent> createTaskRequest(final int incompleteTaskNum) throws Exception { LinkedList<TaskRequestEvent> taskRequestEvents = new LinkedList<>(); // If scheduled tasks is long-term task, cluster resource can be the worst load balance. // This part is to throttle the maximum required container per request int requestContainerNum = Math.min(incompleteTaskNum, maximumRequestContainer); if (LOG.isDebugEnabled()) { LOG.debug("Try to schedule task resources: " + requestContainerNum); } ServiceTracker serviceTracker = context.getMasterContext().getQueryMasterContext().getWorkerContext().getServiceTracker(); NettyClientBase tmClient = RpcClientManager.getInstance() .getClient( serviceTracker.getUmbilicalAddress(), QueryCoordinatorProtocol.class, true, rpcParams); QueryCoordinatorProtocolService masterClientService = tmClient.getStub(); CallFuture<NodeResourceResponse> callBack = new CallFuture<>(); NodeResourceRequest.Builder request = NodeResourceRequest.newBuilder(); request .setCapacity(NodeResources.createResource(minTaskMemory, isLeaf ? 1 : 0).getProto()) .setNumContainers(requestContainerNum) .setPriority(stage.getPriority()) .setQueryId(context.getMasterContext().getQueryId().getProto()) .setType(isLeaf ? ResourceType.LEAF : ResourceType.INTERMEDIATE) .setUserId(context.getMasterContext().getQueryContext().getUser()) .setRunningTasks(stage.getTotalScheduledObjectsCount() - stage.getCompletedTaskCount()) .addAllCandidateNodes(candidateWorkers) .setQueue( context.getMasterContext().getQueryContext().get("queue", "default")); // TODO set queue masterClientService.reserveNodeResources(callBack.getController(), request.build(), callBack); NodeResourceResponse response = callBack.get(RpcConstants.FUTURE_TIMEOUT_SECONDS_DEFAULT, TimeUnit.SECONDS); for (AllocationResourceProto resource : response.getResourceList()) { taskRequestEvents.add( new TaskRequestEvent(resource.getWorkerId(), resource, context.getBlockId())); } return taskRequestEvents; }
@Test(timeout = 60000) @SetupRpcConnection(setupRpcServer = false, setupRpcClient = false) public void testClientRetryOnStartup() throws Exception { retries = 10; ServerSocket serverSocket = new ServerSocket(0); final InetSocketAddress address = new InetSocketAddress("127.0.0.1", serverSocket.getLocalPort()); serverSocket.close(); EchoMessage message = EchoMessage.newBuilder().setMessage(MESSAGE).build(); // lazy startup Thread serverThread = new Thread( new Runnable() { @Override public void run() { try { Thread.sleep(1000); server = new BlockingRpcServer( DummyProtocol.class, new DummyProtocolBlockingImpl(), address, 2); } catch (Exception e) { fail(e.getMessage()); } server.start(); } }); serverThread.start(); RpcConnectionKey rpcConnectionKey = new RpcConnectionKey(address, DummyProtocol.class, false); Properties connParams = new Properties(); connParams.setProperty(RpcConstants.CLIENT_RETRY_NUM, retries + ""); BlockingRpcClient client = manager.newClient(rpcConnectionKey, connParams); assertTrue(client.isConnected()); BlockingInterface stub = client.getStub(); EchoMessage response = stub.echo(null, message); assertEquals(MESSAGE, response.getMessage()); client.close(); server.shutdown(); }
@Test(timeout = 60000) @SetupRpcConnection(setupRpcClient = false) public void testPingOnIdle() throws Exception { RpcConnectionKey rpcConnectionKey = new RpcConnectionKey(server.getListenAddress(), DummyProtocol.class, false); // 500 millis socket timeout Properties connParams = new Properties(); connParams.setProperty(RpcConstants.CLIENT_RETRY_NUM, retries + ""); connParams.setProperty(RpcConstants.CLIENT_SOCKET_TIMEOUT, String.valueOf(500)); connParams.setProperty(RpcConstants.CLIENT_HANG_DETECTION, "true"); BlockingRpcClient client = manager.newClient(rpcConnectionKey, connParams); assertTrue(client.isConnected()); Thread.sleep(600); assertTrue(client.isConnected()); Thread.sleep(600); assertTrue(client.isConnected()); client.close(); }
public void assignToLeafTasks(LinkedList<TaskRequestEvent> taskRequests) { Collections.shuffle(taskRequests); LinkedList<TaskRequestEvent> remoteTaskRequests = new LinkedList<>(); String queryMasterHostAndPort = context .getMasterContext() .getQueryMasterContext() .getWorkerContext() .getConnectionInfo() .getHostAndQMPort(); TaskRequestEvent taskRequest; while (leafTasks.size() > 0 && (!taskRequests.isEmpty() || !remoteTaskRequests.isEmpty())) { int localAssign = 0; int rackAssign = 0; taskRequest = taskRequests.pollFirst(); if (taskRequest == null) { // if there are only remote task requests taskRequest = remoteTaskRequests.pollFirst(); } // checking if this container is still alive. // If not, ignore the task request and stop the task runner WorkerConnectionInfo connectionInfo = context.getMasterContext().getWorkerMap().get(taskRequest.getWorkerId()); if (connectionInfo == null) continue; // getting the hostname of requested node String host = connectionInfo.getHost(); // if there are no worker matched to the hostname a task request if (!leafTaskHostMapping.containsKey(host) && !taskRequests.isEmpty()) { String normalizedHost = NetUtils.normalizeHost(host); if (!leafTaskHostMapping.containsKey(normalizedHost)) { // this case means one of either cases: // * there are no blocks which reside in this node. // * all blocks which reside in this node are consumed, and this task runner requests a // remote task. // In this case, we transfer the task request to the remote task request list, and skip // the followings. remoteTaskRequests.add(taskRequest); continue; } else { host = normalizedHost; } } if (LOG.isDebugEnabled()) { LOG.debug( "assignToLeafTasks: " + taskRequest.getExecutionBlockId() + "," + "worker=" + connectionInfo.getHostAndPeerRpcPort()); } ////////////////////////////////////////////////////////////////////// // disk or host-local allocation ////////////////////////////////////////////////////////////////////// TaskAttemptId attemptId = allocateLocalTask(host); if (attemptId == null) { // if a local task cannot be found HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host); if (!taskRequests .isEmpty()) { // if other requests remains, move to remote list for better locality remoteTaskRequests.add(taskRequest); candidateWorkers.remove(connectionInfo.getId()); continue; } else { if (hostVolumeMapping != null) { int nodes = context.getMasterContext().getWorkerMap().size(); // this part is to control the assignment of tail and remote task balancing per node int tailLimit = 1; if (remainingScheduledObjectNum() > 0 && nodes > 0) { tailLimit = Math.max(remainingScheduledObjectNum() / nodes, 1); } if (hostVolumeMapping.getRemoteConcurrency() >= tailLimit) { // remote task throttling per node continue; } else { // assign to remote volume hostVolumeMapping.increaseConcurrency(HostVolumeMapping.REMOTE); } } } ////////////////////////////////////////////////////////////////////// // rack-local allocation ////////////////////////////////////////////////////////////////////// attemptId = allocateRackTask(host); ////////////////////////////////////////////////////////////////////// // random node allocation ////////////////////////////////////////////////////////////////////// if (attemptId == null && leafTaskNum() > 0) { synchronized (leafTasks) { attemptId = leafTasks.iterator().next(); leafTasks.remove(attemptId); } } if (attemptId != null && hostVolumeMapping != null) { hostVolumeMapping.lastAssignedVolumeId.put(attemptId, HostVolumeMapping.REMOTE); } rackAssign++; } else { localAssign++; } if (attemptId != null) { Task task = stage.getTask(attemptId.getTaskId()); TaskRequest taskAssign = new TaskRequestImpl( attemptId, new ArrayList<>(task.getAllFragments()), "", false, LogicalNodeSerializer.serialize(task.getLogicalPlan()), context.getMasterContext().getQueryContext(), stage.getDataChannel(), stage.getBlock().getEnforcer(), queryMasterHostAndPort); if (checkIfInterQuery(stage.getMasterPlan(), stage.getBlock())) { taskAssign.setInterQuery(); } // TODO send batch request BatchAllocationRequest.Builder requestProto = BatchAllocationRequest.newBuilder(); requestProto.addTaskRequest( TaskAllocationProto.newBuilder() .setResource(taskRequest.getResponseProto().getResource()) .setTaskRequest(taskAssign.getProto()) .build()); requestProto.setExecutionBlockId(attemptId.getTaskId().getExecutionBlockId().getProto()); context .getMasterContext() .getEventHandler() .handle(new TaskAttemptAssignedEvent(attemptId, connectionInfo)); InetSocketAddress addr = stage.getAssignedWorkerMap().get(connectionInfo.getId()); if (addr == null) addr = new InetSocketAddress(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()); AsyncRpcClient tajoWorkerRpc = null; CallFuture<BatchAllocationResponse> callFuture = new CallFuture<>(); totalAttempts++; try { tajoWorkerRpc = RpcClientManager.getInstance() .getClient(addr, TajoWorkerProtocol.class, true, rpcParams); TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); tajoWorkerRpcClient.allocateTasks( callFuture.getController(), requestProto.build(), callFuture); BatchAllocationResponse responseProto = callFuture.get(RpcConstants.FUTURE_TIMEOUT_SECONDS_DEFAULT, TimeUnit.SECONDS); if (responseProto.getCancellationTaskCount() > 0) { for (TaskAllocationProto proto : responseProto.getCancellationTaskList()) { cancel(task.getAttempt(new TaskAttemptId(proto.getTaskRequest().getId()))); cancellation++; } if (LOG.isDebugEnabled()) { LOG.debug( "Canceled requests: " + responseProto.getCancellationTaskCount() + " from " + addr); } continue; } } catch (Exception e) { LOG.error(e); } scheduledObjectNum--; totalAssigned++; hostLocalAssigned += localAssign; rackLocalAssigned += rackAssign; if (rackAssign > 0) { LOG.info( String.format( "Assigned Local/Rack/Total: (%d/%d/%d), " + "Attempted Cancel/Assign/Total: (%d/%d/%d), " + "Locality: %.2f%%, Rack host: %s", hostLocalAssigned, rackLocalAssigned, totalAssigned, cancellation, totalAssigned, totalAttempts, ((double) hostLocalAssigned / (double) totalAssigned) * 100, host)); } } else { throw new RuntimeException("Illegal State!!!!!!!!!!!!!!!!!!!!!"); } } }
public void assignToNonLeafTasks(LinkedList<TaskRequestEvent> taskRequests) { Collections.shuffle(taskRequests); String queryMasterHostAndPort = context .getMasterContext() .getQueryMasterContext() .getWorkerContext() .getConnectionInfo() .getHostAndQMPort(); TaskRequestEvent taskRequest; while (!taskRequests.isEmpty()) { taskRequest = taskRequests.pollFirst(); LOG.debug("assignToNonLeafTasks: " + taskRequest.getExecutionBlockId()); TaskAttemptId attemptId; // random allocation if (nonLeafTasks.size() > 0) { synchronized (nonLeafTasks) { attemptId = nonLeafTasks.iterator().next(); nonLeafTasks.remove(attemptId); } LOG.debug("Assigned based on * match"); Task task; task = stage.getTask(attemptId.getTaskId()); TaskRequest taskAssign = new TaskRequestImpl( attemptId, Lists.newArrayList(task.getAllFragments()), "", false, LogicalNodeSerializer.serialize(task.getLogicalPlan()), context.getMasterContext().getQueryContext(), stage.getDataChannel(), stage.getBlock().getEnforcer(), queryMasterHostAndPort); if (checkIfInterQuery(stage.getMasterPlan(), stage.getBlock())) { taskAssign.setInterQuery(); } for (Map.Entry<String, Set<FetchImpl>> entry : task.getFetchMap().entrySet()) { Collection<FetchImpl> fetches = entry.getValue(); if (fetches != null) { for (FetchImpl fetch : fetches) { taskAssign.addFetch(entry.getKey(), fetch); } } } WorkerConnectionInfo connectionInfo = context.getMasterContext().getWorkerMap().get(taskRequest.getWorkerId()); // TODO send batch request BatchAllocationRequest.Builder requestProto = BatchAllocationRequest.newBuilder(); requestProto.addTaskRequest( TaskAllocationProto.newBuilder() .setResource(taskRequest.getResponseProto().getResource()) .setTaskRequest(taskAssign.getProto()) .build()); requestProto.setExecutionBlockId(attemptId.getTaskId().getExecutionBlockId().getProto()); context .getMasterContext() .getEventHandler() .handle(new TaskAttemptAssignedEvent(attemptId, connectionInfo)); CallFuture<BatchAllocationResponse> callFuture = new CallFuture<>(); InetSocketAddress addr = stage.getAssignedWorkerMap().get(connectionInfo.getId()); if (addr == null) addr = new InetSocketAddress(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()); AsyncRpcClient tajoWorkerRpc; try { tajoWorkerRpc = RpcClientManager.getInstance() .getClient(addr, TajoWorkerProtocol.class, true, rpcParams); TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); tajoWorkerRpcClient.allocateTasks( callFuture.getController(), requestProto.build(), callFuture); BatchAllocationResponse responseProto = callFuture.get(RpcConstants.FUTURE_TIMEOUT_SECONDS_DEFAULT, TimeUnit.SECONDS); if (responseProto.getCancellationTaskCount() > 0) { for (TaskAllocationProto proto : responseProto.getCancellationTaskList()) { cancel(task.getAttempt(new TaskAttemptId(proto.getTaskRequest().getId()))); cancellation++; } if (LOG.isDebugEnabled()) { LOG.debug( "Canceled requests: " + responseProto.getCancellationTaskCount() + " from " + addr); } continue; } totalAssigned++; scheduledObjectNum--; } catch (Exception e) { LOG.error(e); } } } }
public class TestBlockingRpc { public static final String MESSAGE = "TestBlockingRpc"; private BlockingRpcServer server; private BlockingRpcClient client; private BlockingInterface stub; private DummyProtocolBlockingImpl service; RpcClientManager manager = RpcClientManager.getInstance(); private int retries; @Retention(RetentionPolicy.RUNTIME) @Target(ElementType.METHOD) @interface SetupRpcConnection { boolean setupRpcServer() default true; boolean setupRpcClient() default true; } @Rule public ExternalResource resource = new ExternalResource() { private Description description; @Override public Statement apply(Statement base, Description description) { this.description = description; return super.apply(base, description); } @Override protected void before() throws Throwable { SetupRpcConnection setupRpcConnection = description.getAnnotation(SetupRpcConnection.class); if (setupRpcConnection == null || setupRpcConnection.setupRpcServer()) { setUpRpcServer(); } if (setupRpcConnection == null || setupRpcConnection.setupRpcClient()) { setUpRpcClient(); } } @Override protected void after() { SetupRpcConnection setupRpcConnection = description.getAnnotation(SetupRpcConnection.class); if (setupRpcConnection == null || setupRpcConnection.setupRpcClient()) { try { tearDownRpcClient(); } catch (Exception e) { fail(e.getMessage()); } } if (setupRpcConnection == null || setupRpcConnection.setupRpcServer()) { try { tearDownRpcServer(); } catch (Exception e) { fail(e.getMessage()); } } } }; public void setUpRpcServer() throws Exception { service = new DummyProtocolBlockingImpl(); server = new BlockingRpcServer( DummyProtocol.class, service, new InetSocketAddress("127.0.0.1", 0), 2); server.start(); } public void setUpRpcClient() throws Exception { retries = 1; Properties connParams = new Properties(); connParams.setProperty(RpcConstants.CLIENT_RETRY_NUM, "1"); connParams.setProperty( RpcConstants.CLIENT_SOCKET_TIMEOUT, String.valueOf(TimeUnit.SECONDS.toMillis(10))); connParams.setProperty(RpcConstants.CLIENT_HANG_DETECTION, "true"); RpcConnectionKey rpcConnectionKey = new RpcConnectionKey( RpcUtils.getConnectAddress(server.getListenAddress()), DummyProtocol.class, false); client = manager.newClient(rpcConnectionKey, connParams); assertTrue(client.isConnected()); stub = client.getStub(); } @AfterClass public static void tearDownClass() throws Exception { RpcClientManager.shutdown(); } public void tearDownRpcServer() throws Exception { if (server != null) { server.shutdown(); server = null; } } public void tearDownRpcClient() throws Exception { if (client != null) { client.close(); client = null; } } @Test public void testRpc() throws Exception { SumRequest request = SumRequest.newBuilder().setX1(1).setX2(2).setX3(3.15d).setX4(2.0f).build(); SumResponse response1 = stub.sum(null, request); assertEquals(8.15d, response1.getResult(), 1e-15); EchoMessage message = EchoMessage.newBuilder().setMessage(MESSAGE).build(); EchoMessage response2 = stub.echo(null, message); assertEquals(MESSAGE, response2.getMessage()); } @Test public void testGetNull() throws Exception { assertNull(stub.getNull(null, null)); assertTrue(service.getNullCalled); } @Test public void testThrowException() throws Exception { EchoMessage message = EchoMessage.newBuilder().setMessage(MESSAGE).build(); try { stub.throwException(null, message); fail("RpcCall should throw exception"); } catch (TajoServiceException te) { assertEquals("Exception Test", te.getMessage()); assertEquals("org.apache.tajo.rpc.test.DummyProtocol", te.getProtocol()); assertEquals( server.getListenAddress().getAddress().getHostAddress() + ":" + server.getListenAddress().getPort(), te.getRemoteAddress()); } } @Test public void testThrowException2() throws Exception { EchoMessage message = EchoMessage.newBuilder().setMessage(MESSAGE).build(); DefaultRpcController controller = new DefaultRpcController(); try { stub.throwException(controller, message); fail("RpcCall should throw exception"); } catch (TajoServiceException t) { assertTrue(controller.failed()); assertNotNull(controller.errorText()); } controller.reset(); EchoMessage message1 = stub.delay(controller, message); assertEquals(message, message1); assertFalse(controller.failed()); assertNull(controller.errorText()); } @Test(timeout = 60000) public void testServerShutdown1() throws Exception { EchoMessage message = EchoMessage.newBuilder().setMessage(MESSAGE).build(); tearDownRpcServer(); boolean expect = false; try { EchoMessage response = stub.echo(null, message); fail(response.getMessage()); } catch (TajoServiceException e) { expect = true; } assertTrue(expect); } @Test(timeout = 60000) public void testServerShutdown2() throws Exception { EchoMessage message = EchoMessage.newBuilder().setMessage(MESSAGE).build(); tearDownRpcServer(); boolean expect = false; try { BlockingInterface stub = client.getStub(); EchoMessage response = stub.echo(null, message); fail(response.getMessage()); } catch (TajoServiceException e) { expect = true; } assertTrue(expect); } @Test public void testServerShutdown3() throws Exception { final StringBuilder error = new StringBuilder(); Thread callThread = new Thread() { public void run() { try { EchoMessage message = EchoMessage.newBuilder().setMessage(MESSAGE).build(); stub.delay(null, message); } catch (Exception e) { error.append(e.getMessage()); } synchronized (error) { error.notifyAll(); } } }; callThread.start(); final CountDownLatch latch = new CountDownLatch(1); Thread shutdownThread = new Thread() { public void run() { try { Thread.sleep(1000); } catch (InterruptedException e) { } try { server.shutdown(); server = null; latch.countDown(); } catch (Throwable e) { e.printStackTrace(); } } }; shutdownThread.start(); assertTrue(latch.await(5 * 1000, TimeUnit.MILLISECONDS)); assertTrue(latch.getCount() == 0); synchronized (error) { error.wait(5 * 1000); } if (!error.toString().isEmpty()) { fail(error.toString()); } } @Test(timeout = 60000) @SetupRpcConnection(setupRpcServer = false, setupRpcClient = false) public void testClientRetryOnStartup() throws Exception { retries = 10; ServerSocket serverSocket = new ServerSocket(0); final InetSocketAddress address = new InetSocketAddress("127.0.0.1", serverSocket.getLocalPort()); serverSocket.close(); EchoMessage message = EchoMessage.newBuilder().setMessage(MESSAGE).build(); // lazy startup Thread serverThread = new Thread( new Runnable() { @Override public void run() { try { Thread.sleep(1000); server = new BlockingRpcServer( DummyProtocol.class, new DummyProtocolBlockingImpl(), address, 2); } catch (Exception e) { fail(e.getMessage()); } server.start(); } }); serverThread.start(); RpcConnectionKey rpcConnectionKey = new RpcConnectionKey(address, DummyProtocol.class, false); Properties connParams = new Properties(); connParams.setProperty(RpcConstants.CLIENT_RETRY_NUM, retries + ""); BlockingRpcClient client = manager.newClient(rpcConnectionKey, connParams); assertTrue(client.isConnected()); BlockingInterface stub = client.getStub(); EchoMessage response = stub.echo(null, message); assertEquals(MESSAGE, response.getMessage()); client.close(); server.shutdown(); } @Test(timeout = 60000) @SetupRpcConnection(setupRpcServer = false, setupRpcClient = false) public void testClientRetryFailureOnStartup() throws Exception { retries = 2; ServerSocket serverSocket = new ServerSocket(0); final InetSocketAddress address = new InetSocketAddress("127.0.0.1", serverSocket.getLocalPort()); serverSocket.close(); EchoMessage message = EchoMessage.newBuilder().setMessage(MESSAGE).build(); RpcConnectionKey rpcConnectionKey = new RpcConnectionKey(address, DummyProtocol.class, false); Properties connParams = new Properties(); connParams.setProperty(RpcConstants.CLIENT_RETRY_NUM, retries + ""); BlockingRpcClient client = new BlockingRpcClient(NettyUtils.getDefaultEventLoopGroup(), rpcConnectionKey, connParams); try { client.connect(); fail(); } catch (ConnectTimeoutException e) { assertFalse(e.getMessage(), client.isConnected()); } BlockingInterface stub = client.getStub(); try { EchoMessage response = stub.echo(null, message); fail(); } catch (TajoServiceException e) { assertFalse(e.getMessage(), client.isConnected()); } RpcClientManager.cleanup(client); } @Test(timeout = 120000) @SetupRpcConnection(setupRpcServer = false, setupRpcClient = false) public void testUnresolvedAddress() throws Exception { InetSocketAddress address = new InetSocketAddress("test", 0); boolean expected = false; BlockingRpcClient client = null; try { RpcConnectionKey rpcConnectionKey = new RpcConnectionKey(address, DummyProtocol.class, true); Properties connParams = new Properties(); connParams.setProperty(RpcConstants.CLIENT_RETRY_NUM, retries + ""); client = new BlockingRpcClient( NettyUtils.getDefaultEventLoopGroup(), rpcConnectionKey, connParams); client.connect(); fail(); } catch (ConnectException e) { expected = true; } catch (Throwable throwable) { fail(throwable.getMessage()); } finally { client.close(); } assertTrue(expected); } @Test(timeout = 120000) @SetupRpcConnection(setupRpcClient = false) public void testUnresolvedAddress2() throws Exception { String hostAndPort = RpcUtils.normalizeInetSocketAddress(server.getListenAddress()); RpcConnectionKey rpcConnectionKey = new RpcConnectionKey(RpcUtils.createUnresolved(hostAndPort), DummyProtocol.class, false); Properties connParams = new Properties(); connParams.setProperty(RpcConstants.CLIENT_RETRY_NUM, retries + ""); BlockingRpcClient client = new BlockingRpcClient(NettyUtils.getDefaultEventLoopGroup(), rpcConnectionKey, connParams); client.connect(); assertTrue(client.isConnected()); try { BlockingInterface stub = client.getStub(); EchoMessage message = EchoMessage.newBuilder().setMessage(MESSAGE).build(); EchoMessage response2 = stub.echo(null, message); assertEquals(MESSAGE, response2.getMessage()); } finally { client.close(); } } @Test(timeout = 60000) @SetupRpcConnection(setupRpcClient = false) public void testStubRecovery() throws Exception { RpcConnectionKey rpcConnectionKey = new RpcConnectionKey(server.getListenAddress(), DummyProtocol.class, false); Properties connParams = new Properties(); connParams.setProperty(RpcConstants.CLIENT_RETRY_NUM, String.valueOf(1)); BlockingRpcClient client = manager.newClient(rpcConnectionKey, connParams); EchoMessage echoMessage = EchoMessage.newBuilder().setMessage(MESSAGE).build(); int repeat = 5; assertTrue(client.isConnected()); BlockingInterface stub = client.getStub(); client.close(); // close connection assertFalse(client.isConnected()); for (int i = 0; i < repeat; i++) { try { EchoMessage response = stub.echo(null, echoMessage); assertEquals(MESSAGE, response.getMessage()); assertTrue(client.isConnected()); } finally { client.close(); // close connection assertFalse(client.isConnected()); } } } @Test(timeout = 60000) @SetupRpcConnection(setupRpcClient = false) public void testIdleTimeout() throws Exception { RpcConnectionKey rpcConnectionKey = new RpcConnectionKey(server.getListenAddress(), DummyProtocol.class, false); // 500 millis socket timeout Properties connParams = new Properties(); connParams.setProperty(RpcConstants.CLIENT_RETRY_NUM, retries + ""); connParams.setProperty(RpcConstants.CLIENT_SOCKET_TIMEOUT, String.valueOf(500)); BlockingRpcClient client = manager.newClient(rpcConnectionKey, connParams); assertTrue(client.isConnected()); Thread.sleep(600); // timeout assertFalse(client.isConnected()); client.connect(); // try to reconnect assertTrue(client.isConnected()); Thread.sleep(600); // timeout assertFalse(client.isConnected()); client.close(); } @Test(timeout = 60000) @SetupRpcConnection(setupRpcClient = false) public void testPingOnIdle() throws Exception { RpcConnectionKey rpcConnectionKey = new RpcConnectionKey(server.getListenAddress(), DummyProtocol.class, false); // 500 millis socket timeout Properties connParams = new Properties(); connParams.setProperty(RpcConstants.CLIENT_RETRY_NUM, retries + ""); connParams.setProperty(RpcConstants.CLIENT_SOCKET_TIMEOUT, String.valueOf(500)); connParams.setProperty(RpcConstants.CLIENT_HANG_DETECTION, "true"); BlockingRpcClient client = manager.newClient(rpcConnectionKey, connParams); assertTrue(client.isConnected()); Thread.sleep(600); assertTrue(client.isConnected()); Thread.sleep(600); assertTrue(client.isConnected()); client.close(); } @Test(timeout = 60000) @SetupRpcConnection(setupRpcClient = false) public void testIdleTimeoutWithActiveRequest() throws Exception { RpcConnectionKey rpcConnectionKey = new RpcConnectionKey(server.getListenAddress(), DummyProtocol.class, false); // 500 millis socket timeout Properties connParams = new Properties(); connParams.setProperty(RpcConstants.CLIENT_RETRY_NUM, retries + ""); connParams.setProperty(RpcConstants.CLIENT_SOCKET_TIMEOUT, String.valueOf(500)); BlockingRpcClient client = manager.newClient(rpcConnectionKey, connParams); assertTrue(client.isConnected()); BlockingInterface stub = client.getStub(); EchoMessage echoMessage = EchoMessage.newBuilder().setMessage(MESSAGE).build(); EchoMessage message = stub.delay(null, echoMessage); // 3 sec delay assertEquals(message, echoMessage); assertTrue(client.isConnected()); assertTrue(client.getActiveRequests() == 0); Thread.sleep(600); // timeout assertFalse(client.isConnected()); } @Test(timeout = 60000) @SetupRpcConnection(setupRpcClient = false) public void testRequestTimeoutOnBusy() throws Exception { RpcConnectionKey rpcConnectionKey = new RpcConnectionKey(server.getListenAddress(), DummyProtocol.class, false); // 500 millis socket timeout Properties connParams = new Properties(); connParams.setProperty(RpcConstants.CLIENT_RETRY_NUM, retries + ""); connParams.setProperty(RpcConstants.CLIENT_SOCKET_TIMEOUT, String.valueOf(500)); connParams.setProperty(RpcConstants.CLIENT_HANG_DETECTION, "true"); BlockingRpcClient client = manager.newClient(rpcConnectionKey, connParams); assertTrue(client.isConnected()); BlockingInterface stub = client.getStub(); EchoMessage echoMessage = EchoMessage.newBuilder().setMessage(MESSAGE).build(); boolean expected = false; try { EchoMessage message = stub.busy(null, echoMessage); // 30 sec delay fail(); } catch (TajoServiceException e) { expected = true; } finally { client.close(); } assertTrue(expected); } }
@AfterClass public static void tearDownClass() throws Exception { RpcClientManager.shutdown(); }