From c27e2a77005db355da9e72656af8b0df8b1dfe75 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 16 Feb 2018 18:46:42 +0100 Subject: [PATCH 01/10] [FLINK-8678] [flip6] Make RpcService shut down non blocking Changes the RpcService#stopService method to be non blocking. Instead of waiting until the RpcService has stopped, it returns the termination future which is completed once the RpcService has been completelyshut down. This closes #5517. --- .../MesosResourceManagerTest.java | 2 +- .../runtime/entrypoint/ClusterEntrypoint.java | 2 +- .../runtime/minicluster/MiniCluster.java | 4 +- .../apache/flink/runtime/rpc/RpcService.java | 7 +- .../apache/flink/runtime/rpc/RpcUtils.java | 17 ++- .../runtime/rpc/akka/AkkaRpcService.java | 40 ++++--- .../taskexecutor/TaskManagerRunner.java | 10 +- .../clusterframework/ResourceManagerTest.java | 5 +- .../runtime/dispatcher/DispatcherTest.java | 4 +- .../dispatcher/MiniDispatcherTest.java | 6 +- .../scheduler/SchedulerTestBase.java | 2 +- .../jobmaster/slotpool/SlotPoolRpcTest.java | 4 +- .../slotpool/SlotPoolSchedulingTestBase.java | 4 +- .../jobmaster/slotpool/SlotPoolTest.java | 2 +- .../RegisteredRpcConnectionTest.java | 88 +++++++------- .../RetryingRegistrationTest.java | 108 +++++++++--------- .../ResourceManagerHATest.java | 2 +- .../ResourceManagerJobMasterTest.java | 3 +- .../ResourceManagerTaskExecutorTest.java | 3 +- .../resourcemanager/ResourceManagerTest.java | 10 +- .../flink/runtime/rpc/AsyncCallsTest.java | 13 ++- .../runtime/rpc/FencedRpcEndpointTest.java | 3 +- .../flink/runtime/rpc/RpcConnectionTest.java | 25 +++- .../flink/runtime/rpc/RpcEndpointTest.java | 23 ++-- .../flink/runtime/rpc/TestingRpcService.java | 12 +- .../runtime/rpc/akka/AkkaRpcActorTest.java | 14 ++- .../runtime/rpc/akka/AkkaRpcServiceTest.java | 13 ++- .../rpc/akka/MainThreadValidationTest.java | 2 +- .../rpc/akka/MessageSerializationTest.java | 29 +++-- .../taskexecutor/TaskExecutorITCase.java | 3 +- .../taskexecutor/TaskExecutorTest.java | 2 +- .../impl/RpcGatewayRetrieverTest.java | 5 +- .../flink/yarn/YarnResourceManagerTest.java | 2 +- 33 files changed, 271 insertions(+), 198 deletions(-) diff --git a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerTest.java b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerTest.java index e23e0cbebd326..2b38b8587b69a 100644 --- a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerTest.java +++ b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerTest.java @@ -510,7 +510,7 @@ public Protos.Offer.Operation launch(Protos.TaskInfo... taskInfo) { @Override public void close() throws Exception { - rpcService.stopService(); + rpcService.stopService().get(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index ccb3ae4ae7338..f347d05881c7a 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -437,7 +437,7 @@ protected void stopClusterServices(boolean cleanupHaData) throws FlinkException if (commonRpcService != null) { try { - commonRpcService.stopService(); + commonRpcService.stopService().get(); } catch (Throwable t) { exception = ExceptionUtils.firstOrSuppressed(t, exception); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 5046ae7b0e4c6..3f019f44326d6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -756,7 +756,7 @@ private void initializeIOFormatClasses(Configuration configuration) { private static Throwable shutDownRpc(RpcService rpcService, Throwable priorException) { if (rpcService != null) { try { - rpcService.stopService(); + rpcService.stopService().get(); } catch (Throwable t) { return ExceptionUtils.firstOrSuppressed(t, priorException); @@ -773,7 +773,7 @@ private static Throwable shutDownRpcs(RpcService[] rpcServices, Throwable priorE for (RpcService service : rpcServices) { try { if (service != null) { - service.stopService(); + service.stopService().get(); } } catch (Throwable t) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java index 089e4b0161b56..9aa31195fa946 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java @@ -116,9 +116,12 @@ > CompletableFuture con void stopServer(RpcServer selfGateway); /** - * Stop the rpc service shutting down all started rpc servers. + * Trigger the asynchronous stopping of the {@link RpcService}. + * + * @return Future which is completed once the {@link RpcService} has been + * fully stopped. */ - void stopService(); + CompletableFuture stopService(); /** * Returns a future indicating when the RPC service has been shut down. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java index f87d33cf5b991..c90a8b5bbbc6e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java @@ -50,7 +50,7 @@ public static Set> extractImplementedRpcGateways(Cla while (clazz != null) { for (Class interfaze : clazz.getInterfaces()) { if (RpcGateway.class.isAssignableFrom(interfaze)) { - interfaces.add((Class)interfaze); + interfaces.add((Class) interfaze); } } @@ -65,7 +65,7 @@ public static Set> extractImplementedRpcGateways(Cla * * @param rpcEndpoint to terminate * @param timeout for this operation - * @throws ExecutionException if a problem occurs + * @throws ExecutionException if a problem occurred * @throws InterruptedException if the operation has been interrupted * @throws TimeoutException if a timeout occurred */ @@ -74,6 +74,19 @@ public static void terminateRpcEndpoint(RpcEndpoint rpcEndpoint, Time timeout) t rpcEndpoint.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } + /** + * Shuts the given rpc service down and waits for its termination. + * + * @param rpcService to shut down + * @param timeout for this operation + * @throws InterruptedException if the operation has been interrupted + * @throws ExecutionException if a problem occurred + * @throws TimeoutException if a timeout occurred + */ + public static void terminateRpcService(RpcService rpcService, Time timeout) throws InterruptedException, ExecutionException, TimeoutException { + rpcService.stopService().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + } + // We don't want this class to be instantiable private RpcUtils() {} } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index d2d4bf2322bc9..a65fe4611846d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -42,6 +42,7 @@ import akka.actor.Identify; import akka.actor.PoisonPill; import akka.actor.Props; +import akka.actor.Terminated; import akka.dispatch.Futures; import akka.dispatch.Mapper; import akka.pattern.Patterns; @@ -98,6 +99,8 @@ public class AkkaRpcService implements RpcService { private final ScheduledExecutor internalScheduledExecutor; + private final CompletableFuture terminationFuture; + private volatile boolean stopped; public AkkaRpcService(final ActorSystem actorSystem, final Time timeout) { @@ -127,6 +130,8 @@ public AkkaRpcService(final ActorSystem actorSystem, final Time timeout) { internalScheduledExecutor = new ActorSystemScheduledExecutorAdapter(actorSystem); + terminationFuture = new CompletableFuture<>(); + stopped = false; } @@ -311,33 +316,40 @@ public void stopServer(RpcServer selfGateway) { } @Override - public void stopService() { - LOG.info("Stopping Akka RPC service."); - + public CompletableFuture stopService() { synchronized (lock) { if (stopped) { - return; + return terminationFuture; } stopped = true; - } - actorSystem.shutdown(); - actorSystem.awaitTermination(); + LOG.info("Stopping Akka RPC service."); - synchronized (lock) { - actors.clear(); - } + final CompletableFuture actorSytemTerminationFuture = FutureUtils.toJava(actorSystem.terminate()); + + actorSytemTerminationFuture.whenComplete( + (Terminated ignored, Throwable throwable) -> { + synchronized (lock) { + actors.clear(); + } + + if (throwable != null) { + terminationFuture.completeExceptionally(throwable); + } else { + terminationFuture.complete(null); + } + + LOG.info("Stopped Akka RPC service."); + }); - LOG.info("Stopped Akka RPC service."); + return terminationFuture; } @Override public CompletableFuture getTerminationFuture() { - return CompletableFuture.runAsync( - actorSystem::awaitTermination, - getExecutor()); + return terminationFuture; } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java index 4cb1beb5d7d11..46205855945e5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java @@ -173,7 +173,15 @@ protected void shutDownInternally() throws Exception { exception = ExceptionUtils.firstOrSuppressed(e, exception); } - rpcService.stopService(); + try { + rpcService.stopService().get(); + } catch (InterruptedException ie) { + exception = ExceptionUtils.firstOrSuppressed(ie, exception); + + Thread.currentThread().interrupt(); + } catch (Exception e) { + exception = ExceptionUtils.firstOrSuppressed(e, exception); + } try { highAvailabilityServices.close(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java index 096ba5e85aefd..241da8f1bd6db 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java @@ -53,6 +53,7 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; +import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.taskexecutor.SlotReport; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; @@ -578,7 +579,7 @@ public void testHeartbeatTimeoutWithTaskExecutor() throws Exception { verify(taskExecutorGateway, Mockito.timeout(timeout.toMilliseconds())).disconnectResourceManager(any(TimeoutException.class)); } finally { - rpcService.stopService(); + RpcUtils.terminateRpcService(rpcService, timeout); } } @@ -680,7 +681,7 @@ public void testHeartbeatTimeoutWithJobManager() throws Exception { verify(jobMasterGateway, Mockito.timeout(timeout.toMilliseconds())).disconnectResourceManager(eq(rmLeaderId), any(TimeoutException.class)); } finally { - rpcService.stopService(); + RpcUtils.terminateRpcService(rpcService, timeout); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index 46cd9e26773e0..5d264e2010f99 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -134,9 +134,9 @@ public static void setupClass() { } @AfterClass - public static void teardownClass() { + public static void teardownClass() throws Exception { if (rpcService != null) { - rpcService.stopService(); + RpcUtils.terminateRpcService(rpcService, TIMEOUT); rpcService = null; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java index dfabc611f301e..4291ef21d7e28 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java @@ -61,6 +61,8 @@ import java.io.IOException; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; @@ -143,13 +145,13 @@ public void teardown() throws Exception { } @AfterClass - public static void teardownClass() throws IOException { + public static void teardownClass() throws IOException, InterruptedException, ExecutionException, TimeoutException { if (blobServer != null) { blobServer.close(); } if (rpcService != null) { - rpcService.stopService(); + RpcUtils.terminateRpcService(rpcService, timeout); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java index 949ff9621adef..76a5642ec0313 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java @@ -117,7 +117,7 @@ public void teardown() throws Exception { } if (rpcService != null) { - rpcService.stopService(); + rpcService.stopService().get(); rpcService = null; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolRpcTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolRpcTest.java index 172876d202a49..b2be97ef29c85 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolRpcTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolRpcTest.java @@ -92,9 +92,9 @@ public static void setup() { } @AfterClass - public static void shutdown() { + public static void shutdown() throws InterruptedException, ExecutionException, TimeoutException { if (rpcService != null) { - rpcService.stopService(); + RpcUtils.terminateRpcService(rpcService, timeout); rpcService = null; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSchedulingTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSchedulingTestBase.java index 4cd7782a722b9..b9036c19ced96 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSchedulingTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSchedulingTestBase.java @@ -64,9 +64,9 @@ public static void setup() { } @AfterClass - public static void teardown() { + public static void teardown() throws ExecutionException, InterruptedException { if (testingRpcService != null) { - testingRpcService.stopService(); + testingRpcService.stopService().get(); testingRpcService = null; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolTest.java index e6446ad9e48d5..d6e05217f47a9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolTest.java @@ -104,7 +104,7 @@ public void setUp() throws Exception { @After public void tearDown() throws Exception { - rpcService.stopService(); + RpcUtils.terminateRpcService(rpcService, timeout); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RegisteredRpcConnectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RegisteredRpcConnectionTest.java index 650a0f2112b9f..25f976a1e08a2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RegisteredRpcConnectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RegisteredRpcConnectionTest.java @@ -23,6 +23,8 @@ import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.util.TestLogger; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import org.slf4j.LoggerFactory; @@ -46,6 +48,20 @@ */ public class RegisteredRpcConnectionTest extends TestLogger { + private TestingRpcService rpcService; + + @Before + public void setup() { + rpcService = new TestingRpcService(); + } + + @After + public void tearDown() throws ExecutionException, InterruptedException { + if (rpcService != null) { + rpcService.stopService().get(); + } + } + @Test public void testSuccessfulRpcConnection() throws Exception { final String testRpcConnectionEndpointAddress = ""; @@ -54,7 +70,6 @@ public void testSuccessfulRpcConnection() throws Exception { // an endpoint that immediately returns success TestRegistrationGateway testGateway = new TestRegistrationGateway(new RetryingRegistrationTest.TestRegistrationSuccess(connectionID)); - TestingRpcService rpcService = new TestingRpcService(); try { rpcService.registerGateway(testRpcConnectionEndpointAddress, testGateway); @@ -74,7 +89,6 @@ public void testSuccessfulRpcConnection() throws Exception { } finally { testGateway.stop(); - rpcService.stopService(); } } @@ -84,37 +98,30 @@ public void testRpcConnectionFailures() throws Exception { final String testRpcConnectionEndpointAddress = ""; final UUID leaderId = UUID.randomUUID(); - TestingRpcService rpcService = new TestingRpcService(); + // gateway that upon calls Throw an exception + TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class); + final RuntimeException registrationException = new RuntimeException(connectionFailureMessage); + when(testGateway.registrationCall(any(UUID.class), anyLong())).thenThrow( + registrationException); - try { - // gateway that upon calls Throw an exception - TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class); - final RuntimeException registrationException = new RuntimeException(connectionFailureMessage); - when(testGateway.registrationCall(any(UUID.class), anyLong())).thenThrow( - registrationException); + rpcService.registerGateway(testRpcConnectionEndpointAddress, testGateway); - rpcService.registerGateway(testRpcConnectionEndpointAddress, testGateway); - - TestRpcConnection connection = new TestRpcConnection(testRpcConnectionEndpointAddress, leaderId, rpcService.getExecutor(), rpcService); - connection.start(); - - //wait for connection failure - try { - connection.getConnectionFuture().get(); - fail("expected failure."); - } catch (ExecutionException ee) { - assertEquals(registrationException, ee.getCause()); - } + TestRpcConnection connection = new TestRpcConnection(testRpcConnectionEndpointAddress, leaderId, rpcService.getExecutor(), rpcService); + connection.start(); - // validate correct invocation and result - assertFalse(connection.isConnected()); - assertEquals(testRpcConnectionEndpointAddress, connection.getTargetAddress()); - assertEquals(leaderId, connection.getTargetLeaderId()); - assertNull(connection.getTargetGateway()); - } - finally { - rpcService.stopService(); + //wait for connection failure + try { + connection.getConnectionFuture().get(); + fail("expected failure."); + } catch (ExecutionException ee) { + assertEquals(registrationException, ee.getCause()); } + + // validate correct invocation and result + assertFalse(connection.isConnected()); + assertEquals(testRpcConnectionEndpointAddress, connection.getTargetAddress()); + assertEquals(leaderId, connection.getTargetLeaderId()); + assertNull(connection.getTargetGateway()); } @Test @@ -124,7 +131,6 @@ public void testRpcConnectionClose() throws Exception { final String connectionID = "Test RPC Connection ID"; TestRegistrationGateway testGateway = new TestRegistrationGateway(new RetryingRegistrationTest.TestRegistrationSuccess(connectionID)); - TestingRpcService rpcService = new TestingRpcService(); try { rpcService.registerGateway(testRpcConnectionEndpointAddress, testGateway); @@ -141,7 +147,6 @@ public void testRpcConnectionClose() throws Exception { } finally { testGateway.stop(); - rpcService.stopService(); } } @@ -149,31 +154,26 @@ public void testRpcConnectionClose() throws Exception { public void testReconnect() throws Exception { final String connectionId1 = "Test RPC Connection ID 1"; final String connectionId2 = "Test RPC Connection ID 2"; - final TestingRpcService rpcService = new TestingRpcService(); final String testRpcConnectionEndpointAddress = ""; final UUID leaderId = UUID.randomUUID(); final TestRegistrationGateway testGateway = new TestRegistrationGateway( new RetryingRegistrationTest.TestRegistrationSuccess(connectionId1), new RetryingRegistrationTest.TestRegistrationSuccess(connectionId2)); - try { - rpcService.registerGateway(testRpcConnectionEndpointAddress, testGateway); + rpcService.registerGateway(testRpcConnectionEndpointAddress, testGateway); - TestRpcConnection connection = new TestRpcConnection(testRpcConnectionEndpointAddress, leaderId, rpcService.getExecutor(), rpcService); - connection.start(); + TestRpcConnection connection = new TestRpcConnection(testRpcConnectionEndpointAddress, leaderId, rpcService.getExecutor(), rpcService); + connection.start(); - final String actualConnectionId1 = connection.getConnectionFuture().get(); + final String actualConnectionId1 = connection.getConnectionFuture().get(); - assertEquals(actualConnectionId1, connectionId1); + assertEquals(actualConnectionId1, connectionId1); - assertTrue(connection.tryReconnect()); + assertTrue(connection.tryReconnect()); - final String actualConnectionId2 = connection.getConnectionFuture().get(); + final String actualConnectionId2 = connection.getConnectionFuture().get(); - assertEquals(actualConnectionId2, connectionId2); - } finally { - rpcService.stopService(); - } + assertEquals(actualConnectionId2, connectionId2); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java index 885a7f5dd8548..ff5a74818b68b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java @@ -25,6 +25,8 @@ import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.util.TestLogger; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.slf4j.LoggerFactory; @@ -53,6 +55,20 @@ */ public class RetryingRegistrationTest extends TestLogger { + private TestingRpcService rpcService; + + @Before + public void setup() { + rpcService = new TestingRpcService(); + } + + @After + public void tearDown() throws ExecutionException, InterruptedException { + if (rpcService != null) { + rpcService.stopService().get(); + } + } + @Test public void testSimpleSuccessfulRegistration() throws Exception { final String testId = "laissez les bon temps roulez"; @@ -61,12 +77,11 @@ public void testSimpleSuccessfulRegistration() throws Exception { // an endpoint that immediately returns success TestRegistrationGateway testGateway = new TestRegistrationGateway(new TestRegistrationSuccess(testId)); - TestingRpcService rpc = new TestingRpcService(); try { - rpc.registerGateway(testEndpointAddress, testGateway); + rpcService.registerGateway(testEndpointAddress, testGateway); - TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId); + TestRetryingRegistration registration = new TestRetryingRegistration(rpcService, testEndpointAddress, leaderId); registration.startRegistration(); CompletableFuture> future = registration.getFuture(); @@ -84,7 +99,6 @@ public void testSimpleSuccessfulRegistration() throws Exception { } finally { testGateway.stop(); - rpc.stopService(); } } @@ -173,14 +187,12 @@ public void testRetriesOnTimeouts() throws Exception { new TestRegistrationSuccess(testId) // success ); - TestingRpcService rpc = new TestingRpcService(); - try { - rpc.registerGateway(testEndpointAddress, testGateway); + rpcService.registerGateway(testEndpointAddress, testGateway); final long initialTimeout = 20L; TestRetryingRegistration registration = new TestRetryingRegistration( - rpc, + rpcService, testEndpointAddress, leaderId, initialTimeout, @@ -206,7 +218,6 @@ public void testRetriesOnTimeouts() throws Exception { assertTrue("retries did not properly back off", elapsedMillis >= 3 * initialTimeout); } finally { - rpc.stopService(); testGateway.stop(); } } @@ -217,8 +228,6 @@ public void testDecline() throws Exception { final String testEndpointAddress = ""; final UUID leaderId = UUID.randomUUID(); - TestingRpcService rpc = new TestingRpcService(); - TestRegistrationGateway testGateway = new TestRegistrationGateway( null, // timeout new RegistrationResponse.Decline("no reason "), @@ -227,9 +236,9 @@ public void testDecline() throws Exception { ); try { - rpc.registerGateway(testEndpointAddress, testGateway); + rpcService.registerGateway(testEndpointAddress, testGateway); - TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId); + TestRetryingRegistration registration = new TestRetryingRegistration(rpcService, testEndpointAddress, leaderId); long started = System.nanoTime(); registration.startRegistration(); @@ -251,7 +260,6 @@ public void testDecline() throws Exception { } finally { testGateway.stop(); - rpc.stopService(); } } @@ -262,39 +270,32 @@ public void testRetryOnError() throws Exception { final String testEndpointAddress = ""; final UUID leaderId = UUID.randomUUID(); - TestingRpcService rpc = new TestingRpcService(); - - try { - // gateway that upon calls first responds with a failure, then with a success - TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class); + // gateway that upon calls first responds with a failure, then with a success + TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class); - when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn( - FutureUtils.completedExceptionally(new Exception("test exception")), - CompletableFuture.completedFuture(new TestRegistrationSuccess(testId))); + when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn( + FutureUtils.completedExceptionally(new Exception("test exception")), + CompletableFuture.completedFuture(new TestRegistrationSuccess(testId))); - rpc.registerGateway(testEndpointAddress, testGateway); + rpcService.registerGateway(testEndpointAddress, testGateway); - TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId); + TestRetryingRegistration registration = new TestRetryingRegistration(rpcService, testEndpointAddress, leaderId); - long started = System.nanoTime(); - registration.startRegistration(); + long started = System.nanoTime(); + registration.startRegistration(); - CompletableFuture> future = registration.getFuture(); - Tuple2 success = - future.get(10, TimeUnit.SECONDS); + CompletableFuture> future = registration.getFuture(); + Tuple2 success = + future.get(10, TimeUnit.SECONDS); - long finished = System.nanoTime(); - long elapsedMillis = (finished - started) / 1000000; + long finished = System.nanoTime(); + long elapsedMillis = (finished - started) / 1000000; - assertEquals(testId, success.f1.getCorrelationId()); + assertEquals(testId, success.f1.getCorrelationId()); - // validate that some retry-delay / back-off behavior happened - assertTrue("retries did not properly back off", - elapsedMillis >= TestRetryingRegistration.DELAY_ON_ERROR); - } - finally { - rpc.stopService(); - } + // validate that some retry-delay / back-off behavior happened + assertTrue("retries did not properly back off", + elapsedMillis >= TestRetryingRegistration.DELAY_ON_ERROR); } @Test @@ -302,29 +303,22 @@ public void testCancellation() throws Exception { final String testEndpointAddress = "my-test-address"; final UUID leaderId = UUID.randomUUID(); - TestingRpcService rpc = new TestingRpcService(); + CompletableFuture result = new CompletableFuture<>(); - try { - CompletableFuture result = new CompletableFuture<>(); - - TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class); - when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(result); + TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class); + when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(result); - rpc.registerGateway(testEndpointAddress, testGateway); + rpcService.registerGateway(testEndpointAddress, testGateway); - TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId); - registration.startRegistration(); + TestRetryingRegistration registration = new TestRetryingRegistration(rpcService, testEndpointAddress, leaderId); + registration.startRegistration(); - // cancel and fail the current registration attempt - registration.cancel(); - result.completeExceptionally(new TimeoutException()); + // cancel and fail the current registration attempt + registration.cancel(); + result.completeExceptionally(new TimeoutException()); - // there should not be a second registration attempt - verify(testGateway, atMost(1)).registrationCall(any(UUID.class), anyLong()); - } - finally { - rpc.stopService(); - } + // there should not be a second registration attempt + verify(testGateway, atMost(1)).registrationCall(any(UUID.class), anyLong()); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java index 57ce3497a965c..a3d17f2433489 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java @@ -126,7 +126,7 @@ public void revokeLeadership() { testingFatalErrorHandler.rethrowError(); } } finally { - rpcService.stopService(); + rpcService.stopService().get(); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java index 4440e380cc1de..acd87748f7e68 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java @@ -37,6 +37,7 @@ import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.rpc.exceptions.FencingTokenException; @@ -73,7 +74,7 @@ public void setup() throws Exception { @After public void teardown() throws Exception { - rpcService.stopService(); + RpcUtils.terminateRpcService(rpcService, timeout); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java index 2af6632395580..a0c4b43efa863 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java @@ -31,6 +31,7 @@ import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerInfo; import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.rpc.exceptions.FencingTokenException; @@ -105,7 +106,7 @@ public void setup() throws Exception { @After public void teardown() throws Exception { - rpcService.stopService(); + RpcUtils.terminateRpcService(rpcService, timeout); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java index 8f35b1310ef6b..f5fa89992e152 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java @@ -46,6 +46,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; @Category(Flip6.class) public class ResourceManagerTest extends TestLogger { @@ -54,18 +55,13 @@ public class ResourceManagerTest extends TestLogger { @Before public void setUp() { - if (rpcService != null) { - rpcService.stopService(); - rpcService = null; - } - rpcService = new TestingRpcService(); } @After - public void tearDown() { + public void tearDown() throws ExecutionException, InterruptedException { if (rpcService != null) { - rpcService.stopService(); + rpcService.stopService().get(); rpcService = null; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java index 9b72cb497b981..8ba0ccdb5d96e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; import org.apache.flink.runtime.rpc.exceptions.FencingTokenException; @@ -30,10 +31,12 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; +import akka.actor.Terminated; import org.junit.AfterClass; import org.junit.Test; import org.junit.experimental.categories.Category; +import java.util.Arrays; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -60,9 +63,13 @@ public class AsyncCallsTest extends TestLogger { new AkkaRpcService(actorSystem, Time.milliseconds(10000L)); @AfterClass - public static void shutdown() { - akkaRpcService.stopService(); - actorSystem.shutdown(); + public static void shutdown() throws InterruptedException, ExecutionException, TimeoutException { + final CompletableFuture rpcTerminationFuture = akkaRpcService.stopService(); + final CompletableFuture actorSystemTerminationFuture = FutureUtils.toJava(actorSystem.terminate()); + + FutureUtils + .waitForAll(Arrays.asList(rpcTerminationFuture, actorSystemTerminationFuture)) + .get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java index b8040510f8cd8..3d99c3f8127fd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java @@ -60,8 +60,7 @@ public static void setup() { @AfterClass public static void teardown() throws ExecutionException, InterruptedException, TimeoutException { if (rpcService != null) { - rpcService.stopService(); - rpcService.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + RpcUtils.terminateRpcService(rpcService, timeout); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java index 26630c824ceb6..017c1f58029c4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java @@ -23,17 +23,21 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.util.TestLogger; +import akka.actor.Terminated; import org.junit.Test; import org.junit.experimental.categories.Category; import scala.Option; import scala.Tuple2; +import java.util.Arrays; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -46,10 +50,10 @@ * connect to an RpcEndpoint. */ @Category(Flip6.class) -public class RpcConnectionTest { +public class RpcConnectionTest extends TestLogger { @Test - public void testConnectFailure() { + public void testConnectFailure() throws Exception { ActorSystem actorSystem = null; RpcService rpcService = null; try { @@ -77,12 +81,25 @@ public void testConnectFailure() { fail("wrong exception: " + t); } finally { + final CompletableFuture rpcTerminationFuture; + if (rpcService != null) { - rpcService.stopService(); + rpcTerminationFuture = rpcService.stopService(); + } else { + rpcTerminationFuture = CompletableFuture.completedFuture(null); } + + final CompletableFuture actorSystemTerminationFuture; + if (actorSystem != null) { - actorSystem.shutdown(); + actorSystemTerminationFuture = FutureUtils.toJava(actorSystem.terminate()); + } else { + actorSystemTerminationFuture = CompletableFuture.completedFuture(null); } + + FutureUtils + .waitForAll(Arrays.asList(rpcTerminationFuture, actorSystemTerminationFuture)) + .get(); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcEndpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcEndpointTest.java index 6d60de922febc..d52aadb72c31d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcEndpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcEndpointTest.java @@ -20,21 +20,22 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; import org.apache.flink.testutils.category.Flip6; import org.apache.flink.util.TestLogger; import akka.actor.ActorSystem; +import akka.actor.Terminated; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; +import java.util.Arrays; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; -import scala.concurrent.duration.FiniteDuration; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -56,21 +57,13 @@ public static void setup() { @AfterClass public static void teardown() throws Exception { - if (rpcService != null) { - rpcService.stopService(); - } - - if (actorSystem != null) { - actorSystem.shutdown(); - } - if (rpcService != null) { - rpcService.getTerminationFuture().get(TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS); - } + final CompletableFuture rpcTerminationFuture = rpcService.stopService(); + final CompletableFuture actorSystemTerminationFuture = FutureUtils.toJava(actorSystem.terminate()); - if (actorSystem != null) { - actorSystem.awaitTermination(new FiniteDuration(TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS)); - } + FutureUtils + .waitForAll(Arrays.asList(rpcTerminationFuture, actorSystemTerminationFuture)) + .get(TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java index 4b9f3977fa6c1..db70a0f165f88 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java @@ -74,9 +74,15 @@ public TestingRpcService(Configuration configuration) { // ------------------------------------------------------------------------ @Override - public void stopService() { - super.stopService(); - registeredConnections.clear(); + public CompletableFuture stopService() { + final CompletableFuture terminationFuture = super.stopService(); + + terminationFuture.whenComplete( + (Void ignored, Throwable throwable) -> { + registeredConnections.clear(); + }); + + return terminationFuture; } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java index 3ff1b80214598..1b4500691f2d5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java @@ -31,14 +31,17 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; +import akka.actor.Terminated; import org.hamcrest.core.Is; import org.junit.AfterClass; import org.junit.Test; import org.junit.experimental.categories.Category; +import java.util.Arrays; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -61,10 +64,13 @@ public class AkkaRpcActorTest extends TestLogger { new AkkaRpcService(actorSystem, timeout); @AfterClass - public static void shutdown() { - akkaRpcService.stopService(); - actorSystem.shutdown(); - actorSystem.awaitTermination(); + public static void shutdown() throws InterruptedException, ExecutionException, TimeoutException { + final CompletableFuture rpcTerminationFuture = akkaRpcService.stopService(); + final CompletableFuture actorSystemTerminationFuture = FutureUtils.toJava(actorSystem.terminate()); + + FutureUtils + .waitForAll(Arrays.asList(rpcTerminationFuture, actorSystemTerminationFuture)) + .get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java index c4259f402413f..d92e496f20573 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java @@ -27,10 +27,12 @@ import org.apache.flink.util.TestLogger; import akka.actor.ActorSystem; +import akka.actor.Terminated; import org.junit.AfterClass; import org.junit.Test; import org.junit.experimental.categories.Category; +import java.util.Arrays; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; @@ -59,10 +61,13 @@ public class AkkaRpcServiceTest extends TestLogger { new AkkaRpcService(actorSystem, timeout); @AfterClass - public static void shutdown() { - akkaRpcService.stopService(); - actorSystem.shutdown(); - actorSystem.awaitTermination(FutureUtils.toFiniteDuration(timeout)); + public static void shutdown() throws InterruptedException, ExecutionException, TimeoutException { + final CompletableFuture rpcTerminationFuture = akkaRpcService.stopService(); + final CompletableFuture actorSystemTerminationFuture = FutureUtils.toJava(actorSystem.terminate()); + + FutureUtils + .waitForAll(Arrays.asList(rpcTerminationFuture, actorSystemTerminationFuture)) + .get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java index 8f35c0f6afdd5..a69bd84c9bfe7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java @@ -70,7 +70,7 @@ public void failIfNotInMainThread() throws Exception { testEndpoint.shutDown(); } finally { - akkaRpcService.stopService(); + akkaRpcService.stopService().get(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java index bb46bec23a70d..061145c3f4334 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java @@ -18,16 +18,18 @@ package org.apache.flink.runtime.rpc.akka; -import akka.actor.ActorSystem; -import com.typesafe.config.Config; -import com.typesafe.config.ConfigValueFactory; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.testutils.category.Flip6; import org.apache.flink.util.TestLogger; + +import akka.actor.ActorSystem; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigValueFactory; import org.hamcrest.core.Is; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -35,8 +37,13 @@ import org.junit.experimental.categories.Category; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; @@ -67,15 +74,17 @@ public static void setup() { } @AfterClass - public static void teardown() { - akkaRpcService1.stopService(); - akkaRpcService2.stopService(); + public static void teardown() throws InterruptedException, ExecutionException, TimeoutException { + final Collection> terminationFutures = new ArrayList<>(4); - actorSystem1.shutdown(); - actorSystem2.shutdown(); + terminationFutures.add(akkaRpcService1.stopService()); + terminationFutures.add(FutureUtils.toJava(actorSystem1.terminate())); + terminationFutures.add(akkaRpcService2.stopService()); + terminationFutures.add(FutureUtils.toJava(actorSystem2.terminate())); - actorSystem1.awaitTermination(); - actorSystem2.awaitTermination(); + FutureUtils + .waitForAll(terminationFutures) + .get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java index dc1d09f3aedc3..8f4ec5d7af073 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java @@ -48,6 +48,7 @@ import org.apache.flink.runtime.resourcemanager.SlotRequest; import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; +import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; @@ -220,7 +221,7 @@ public void testSlotAllocation() throws Exception { testingFatalErrorHandler.rethrowError(); } - rpcService.stopService(); + RpcUtils.terminateRpcService(rpcService, timeout); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index e894e48af4511..d7a1860697147 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -189,7 +189,7 @@ public void setup() throws IOException { @After public void teardown() throws Exception { if (rpc != null) { - rpc.stopService(); + RpcUtils.terminateRpcService(rpc, timeout); rpc = null; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/impl/RpcGatewayRetrieverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/impl/RpcGatewayRetrieverTest.java index 847250cf29e1e..ae530f7d19070 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/impl/RpcGatewayRetrieverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/impl/RpcGatewayRetrieverTest.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.RpcTimeout; +import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.util.TestLogger; @@ -58,9 +59,7 @@ public static void setup() { @AfterClass public static void teardown() throws InterruptedException, ExecutionException, TimeoutException { if (rpcService != null) { - rpcService.stopService(); - rpcService.getTerminationFuture().get(TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS); - + RpcUtils.terminateRpcService(rpcService, TIMEOUT); rpcService = null; } } diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java index 8743380d507ab..455abc9596e9c 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java @@ -308,7 +308,7 @@ public void startResourceManager() throws Exception { * Stop the Akka actor system. */ public void stopResourceManager() throws Exception { - rpcService.stopService(); + rpcService.stopService().get(); } } From c131546eaadd07baf950bd6a44d07ee42d109e4c Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 15 Feb 2018 18:43:39 +0100 Subject: [PATCH 02/10] [FLINK-8664] [rest] Change RpcEndpoint#TerminationFuture value type to Void This closes #5496. --- .../flink/runtime/entrypoint/ClusterEntrypoint.java | 2 +- .../apache/flink/runtime/jobmaster/JobManagerRunner.java | 4 ++-- .../org/apache/flink/runtime/jobmaster/JobMaster.java | 2 +- .../java/org/apache/flink/runtime/rpc/RpcEndpoint.java | 2 +- .../main/java/org/apache/flink/runtime/rpc/RpcServer.java | 2 +- .../flink/runtime/rpc/akka/AkkaInvocationHandler.java | 6 +++--- .../org/apache/flink/runtime/rpc/akka/AkkaRpcService.java | 2 +- .../runtime/rpc/akka/FencedAkkaInvocationHandler.java | 2 +- .../flink/runtime/taskexecutor/TaskManagerRunner.java | 2 +- .../flink/runtime/dispatcher/MiniDispatcherTest.java | 2 +- .../apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java | 8 ++++---- 11 files changed, 17 insertions(+), 17 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index f347d05881c7a..8e4f93615aa28 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -213,7 +213,7 @@ protected void runCluster(Configuration configuration) throws Exception { // TODO: Make shutDownAndTerminate non blocking to not use the global executor dispatcher.getTerminationFuture().whenCompleteAsync( - (Boolean success, Throwable throwable) -> { + (Void value, Throwable throwable) -> { if (throwable != null) { LOG.info("Could not properly terminate the Dispatcher.", throwable); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java index 5740bd7c28a86..42692434b2f65 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java @@ -225,10 +225,10 @@ private CompletableFuture shutdownInternally() { jobManager.shutDown(); - final CompletableFuture jobManagerTerminationFuture = jobManager.getTerminationFuture(); + final CompletableFuture jobManagerTerminationFuture = jobManager.getTerminationFuture(); jobManagerTerminationFuture.whenComplete( - (Boolean ignored, Throwable throwable) -> { + (Void ignored, Throwable throwable) -> { try { leaderElectionService.stop(); } catch (Throwable t) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index dd2a7eaef6411..015751b1ab707 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -407,7 +407,7 @@ public void postStop() throws Exception { // shut down will internally release all registered slots slotPool.shutDown(); - CompletableFuture terminationFuture = slotPool.getTerminationFuture(); + CompletableFuture terminationFuture = slotPool.getTerminationFuture(); Exception exception = null; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java index 9c27c95170718..9c2ed83e999c1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -228,7 +228,7 @@ public RpcService getRpcService() { * * @return Future which is completed when the rpc endpoint has been terminated. */ - public CompletableFuture getTerminationFuture() { + public CompletableFuture getTerminationFuture() { return rpcServer.getTerminationFuture(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServer.java index ac2f7eb2656e8..14d0cc9b8791a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServer.java @@ -30,5 +30,5 @@ public interface RpcServer extends StartStoppable, MainThreadExecutable, RpcGate * * @return Future indicating when the rpc endpoint has been terminated */ - CompletableFuture getTerminationFuture(); + CompletableFuture getTerminationFuture(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java index 863b780958d77..cc54f2ed7ae66 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java @@ -84,7 +84,7 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaBasedEndpoint, Rpc // null if gateway; otherwise non-null @Nullable - private final CompletableFuture terminationFuture; + private final CompletableFuture terminationFuture; AkkaInvocationHandler( String address, @@ -92,7 +92,7 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaBasedEndpoint, Rpc ActorRef rpcEndpoint, Time timeout, long maximumFramesize, - @Nullable CompletableFuture terminationFuture) { + @Nullable CompletableFuture terminationFuture) { this.address = Preconditions.checkNotNull(address); this.hostname = Preconditions.checkNotNull(hostname); @@ -341,7 +341,7 @@ public String getHostname() { } @Override - public CompletableFuture getTerminationFuture() { + public CompletableFuture getTerminationFuture() { return terminationFuture; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index a65fe4611846d..8e96492320f39 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -195,7 +195,7 @@ public > CompletableFuture public RpcServer startServer(C rpcEndpoint) { checkNotNull(rpcEndpoint, "rpc endpoint"); - CompletableFuture terminationFuture = new CompletableFuture<>(); + CompletableFuture terminationFuture = new CompletableFuture<>(); final Props akkaRpcActorProps; if (rpcEndpoint instanceof FencedRpcEndpoint) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaInvocationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaInvocationHandler.java index 3ca75e23cdfc6..564b1efa7c72f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaInvocationHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaInvocationHandler.java @@ -60,7 +60,7 @@ public FencedAkkaInvocationHandler( ActorRef rpcEndpoint, Time timeout, long maximumFramesize, - @Nullable CompletableFuture terminationFuture, + @Nullable CompletableFuture terminationFuture, Supplier fencingTokenSupplier) { super(address, hostname, rpcEndpoint, timeout, maximumFramesize, terminationFuture); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java index 46205855945e5..2de1be81c98bd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java @@ -198,7 +198,7 @@ protected void shutDownInternally() throws Exception { } // export the termination future for caller to know it is terminated - public CompletableFuture getTerminationFuture() { + public CompletableFuture getTerminationFuture() { return taskManager.getTerminationFuture(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java index 4291ef21d7e28..2b989398ea71f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java @@ -222,7 +222,7 @@ public void testJobResultRetrieval() throws Exception { resultFuture.complete(archivedExecutionGraph); - final CompletableFuture terminationFuture = miniDispatcher.getTerminationFuture(); + final CompletableFuture terminationFuture = miniDispatcher.getTerminationFuture(); assertThat(terminationFuture.isDone(), is(false)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java index 1b4500691f2d5..2a65cac125fad 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java @@ -185,7 +185,7 @@ public void testRpcEndpointTerminationFuture() throws Exception { final DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(akkaRpcService); rpcEndpoint.start(); - CompletableFuture terminationFuture = rpcEndpoint.getTerminationFuture(); + CompletableFuture terminationFuture = rpcEndpoint.getTerminationFuture(); assertFalse(terminationFuture.isDone()); @@ -246,7 +246,7 @@ public void testPostStopExceptionPropagation() throws Exception { rpcEndpoint.shutDown(); - CompletableFuture terminationFuture = rpcEndpoint.getTerminationFuture(); + CompletableFuture terminationFuture = rpcEndpoint.getTerminationFuture(); try { terminationFuture.get(); @@ -265,7 +265,7 @@ public void testPostStopExecutedByMainThread() throws Exception { simpleRpcEndpoint.shutDown(); - CompletableFuture terminationFuture = simpleRpcEndpoint.getTerminationFuture(); + CompletableFuture terminationFuture = simpleRpcEndpoint.getTerminationFuture(); // check that we executed the postStop method in the main thread, otherwise an exception // would be thrown here. @@ -285,7 +285,7 @@ public void testActorTerminationWhenServiceShutdown() throws Exception { rpcEndpoint.start(); - CompletableFuture terminationFuture = rpcEndpoint.getTerminationFuture(); + CompletableFuture terminationFuture = rpcEndpoint.getTerminationFuture(); rpcService.stopService(); From 1897e238eb9556e6c3e1ffd1169109ec9ed8f217 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 15 Feb 2018 17:58:35 +0100 Subject: [PATCH 03/10] [hotfix] Remove unused method MiniCluster#waitUntilTaskManagerRegistrationsComplete --- .../runtime/minicluster/MiniCluster.java | 47 ------------------- 1 file changed, 47 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 3f019f44326d6..c98387d25e466 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -42,7 +42,6 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobResult; -import org.apache.flink.runtime.leaderelection.LeaderAddressAndId; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.messages.Acknowledge; @@ -508,52 +507,6 @@ private void shutdownInternally() throws Exception { } } - public void waitUntilTaskManagerRegistrationsComplete() throws Exception { - LeaderRetrievalService rmMasterListener = null; - CompletableFuture addressAndIdFuture; - - try { - synchronized (lock) { - checkState(running, "FlinkMiniCluster is not running"); - - OneTimeLeaderListenerFuture listenerFuture = new OneTimeLeaderListenerFuture(); - rmMasterListener = haServices.getResourceManagerLeaderRetriever(); - rmMasterListener.start(listenerFuture); - addressAndIdFuture = listenerFuture.future(); - } - - final LeaderAddressAndId addressAndId = addressAndIdFuture.get(); - - final ResourceManagerGateway resourceManager = commonRpcService - .connect( - addressAndId.leaderAddress(), - new ResourceManagerId(addressAndId.leaderId()), - ResourceManagerGateway.class) - .get(); - - final int numTaskManagersToWaitFor = taskManagers.length; - - // poll and wait until enough TaskManagers are available - while (true) { - int numTaskManagersAvailable = resourceManager.getNumberOfRegisteredTaskManagers().get(); - - if (numTaskManagersAvailable >= numTaskManagersToWaitFor) { - break; - } - Thread.sleep(2); - } - } - finally { - try { - if (rmMasterListener != null) { - rmMasterListener.stop(); - } - } catch (Exception e) { - LOG.warn("Error shutting down leader listener for ResourceManager"); - } - } - } - // ------------------------------------------------------------------------ // running jobs // ------------------------------------------------------------------------ From 6868cacdf309404a553831409734c2f65f16587c Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 15 Feb 2018 18:14:55 +0100 Subject: [PATCH 04/10] [hotfix] Don't fail LeaderContender and Listener when closing EmbeddedLeaderService --- .../nonha/embedded/EmbeddedLeaderService.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderService.java index 5eb43750656fd..ce964a3c7fb91 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderService.java @@ -28,6 +28,7 @@ import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; + import java.util.HashSet; import java.util.Set; import java.util.UUID; @@ -38,7 +39,7 @@ /** * A simple leader election service, which selects a leader among contenders and notifies listeners. - * + * *

An election service for contenders can be created via {@link #createLeaderElectionService()}, * a listener service for leader observers can be created via {@link #createLeaderRetrievalService()}. */ @@ -54,19 +55,19 @@ public class EmbeddedLeaderService { private final Set listeners; - /** proposed leader, which has been notified of leadership grant, but has not confirmed */ + /** proposed leader, which has been notified of leadership grant, but has not confirmed. */ private EmbeddedLeaderElectionService currentLeaderProposed; - /** actual leader that has confirmed leadership and of which listeners have been notified */ + /** actual leader that has confirmed leadership and of which listeners have been notified. */ private EmbeddedLeaderElectionService currentLeaderConfirmed; - /** fencing UID for the current leader (or proposed leader) */ + /** fencing UID for the current leader (or proposed leader). */ private UUID currentLeaderSessionId; - /** the cached address of the current leader */ + /** the cached address of the current leader. */ private String currentLeaderAddress; - /** flag marking the service as terminated */ + /** flag marking the service as terminated. */ private boolean shutdown; // ------------------------------------------------------------------------ @@ -83,7 +84,7 @@ public EmbeddedLeaderService(Executor notificationsDispatcher) { /** * Shuts down this leader election service. - * + * *

This method does not perform a clean revocation of the leader status and * no notification to any leader listeners. It simply notifies all contenders * and listeners that the service is no longer available. @@ -364,7 +365,7 @@ void shutdown(Exception cause) { if (running) { running = false; isLeader = false; - contender.handleError(cause); + contender.revokeLeadership(); contender = null; } } @@ -392,7 +393,6 @@ public void stop() throws Exception { public void shutdown(Exception cause) { if (running) { running = false; - listener.handleError(cause); listener = null; } } From bb306b9d89e41ae0e32bedecacfece25f7a87d3d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 15 Feb 2018 18:22:21 +0100 Subject: [PATCH 05/10] [hotfix] Fix checkstyle violations in RpcEndpoint --- .../apache/flink/runtime/rpc/RpcEndpoint.java | 33 +++++++++---------- 1 file changed, 16 insertions(+), 17 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java index 9c2ed83e999c1..d269e8473c5e1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.util.Preconditions; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,17 +38,15 @@ /** * Base class for RPC endpoints. Distributed components which offer remote procedure calls have to - * extend the RPC endpoint base class. An RPC endpoint is backed by an {@link RpcService}. - * + * extend the RPC endpoint base class. An RPC endpoint is backed by an {@link RpcService}. + * *

Endpoint and Gateway

- * * To be done... - * *

Single Threaded Endpoint Execution

- * + * *

All RPC calls on the same endpoint are called by the same thread * (referred to as the endpoint's main thread). - * Thus, by executing all state changing operations within the main + * Thus, by executing all state changing operations within the main * thread, we don't have to reason about concurrent accesses, in the same way in the Actor Model * of Erlang or Akka. * @@ -60,16 +59,16 @@ public abstract class RpcEndpoint implements RpcGateway { // ------------------------------------------------------------------------ - /** RPC service to be used to start the RPC server and to obtain rpc gateways */ + /** RPC service to be used to start the RPC server and to obtain rpc gateways. */ private final RpcService rpcService; - /** Unique identifier for this rpc endpoint */ + /** Unique identifier for this rpc endpoint. */ private final String endpointId; - /** Interface to access the underlying rpc server */ + /** Interface to access the underlying rpc server. */ protected final RpcServer rpcServer; - /** A reference to the endpoint's main thread, if the current method is called by the main thread */ + /** A reference to the endpoint's main thread, if the current method is called by the main thread. */ final AtomicReference currentMainThread = new AtomicReference<>(null); /** The main thread executor to be used to execute future callbacks in the main thread @@ -78,7 +77,7 @@ public abstract class RpcEndpoint implements RpcGateway { /** * Initializes the RPC endpoint. - * + * * @param rpcService The RPC server that dispatches calls to this RPC endpoint. * @param endpointId Unique identifier for this endpoint */ @@ -117,7 +116,7 @@ public String getEndpointId() { * Starts the rpc endpoint. This tells the underlying rpc server that the rpc endpoint is ready * to process remote procedure calls. * - * IMPORTANT: Whenever you override this method, call the parent implementation to enable + *

IMPORTANT: Whenever you override this method, call the parent implementation to enable * rpc processing. It is advised to make the parent call last. * * @throws Exception indicating that something went wrong while starting the RPC endpoint @@ -140,7 +139,7 @@ protected final void stop() { *

This method is called when the RpcEndpoint is being shut down. The method is guaranteed * to be executed in the main thread context and can be used to clean up internal state. * - * IMPORTANT: This method should never be called directly by the user. + *

IMPORTANT: This method should never be called directly by the user. * * @throws Exception if an error occurs. The exception is returned as result of the termination future. */ @@ -288,15 +287,15 @@ protected CompletableFuture callAsync(Callable callable, Time timeout) /** * Validates that the method call happens in the RPC endpoint's main thread. - * + * *

IMPORTANT: This check only happens when assertions are enabled, * such as when running tests. - * + * *

This can be used for additional checks, like *

{@code
 	 * protected void concurrencyCriticalMethod() {
 	 *     validateRunsInMainThread();
-	 *     
+	 *
 	 *     // some critical stuff
 	 * }
 	 * }
@@ -308,7 +307,7 @@ public void validateRunsInMainThread() { // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ - + /** * Executor which executes runnables in the main thread context. */ From d9b28e817351eb2eb6b4cdd9597061713d9160e8 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 15 Feb 2018 19:19:48 +0100 Subject: [PATCH 06/10] [FLINK-8665] [rest] Let RpcEndpoint#postStop return completion future The RpcEndpoint#postStop method returns a CompletableFuture which is completed once all post stop actions have completed. The termination future of the respective RpcEndpoint is only completed afterwards. This closes #5498. --- .../MesosResourceManager.java | 22 +---- .../flink/runtime/dispatcher/Dispatcher.java | 24 ++--- .../flink/runtime/jobmaster/JobMaster.java | 28 +----- .../runtime/jobmaster/slotpool/SlotPool.java | 4 +- .../resourcemanager/ResourceManager.java | 13 +-- .../flink/runtime/rpc/FencedRpcEndpoint.java | 2 +- .../apache/flink/runtime/rpc/RpcEndpoint.java | 5 +- .../flink/runtime/rpc/akka/AkkaRpcActor.java | 21 ++-- .../runtime/taskexecutor/TaskExecutor.java | 14 +-- .../flink/runtime/rpc/AsyncCallsTest.java | 10 ++ .../runtime/rpc/FencedRpcEndpointTest.java | 5 + .../flink/runtime/rpc/RpcEndpointTest.java | 5 + .../runtime/rpc/akka/AkkaRpcActorTest.java | 98 +++++++++++++++---- .../rpc/akka/MainThreadValidationTest.java | 7 ++ .../rpc/akka/MessageSerializationTest.java | 5 + .../impl/RpcGatewayRetrieverTest.java | 5 + .../flink/yarn/YarnResourceManager.java | 15 +-- 17 files changed, 170 insertions(+), 113 deletions(-) diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManager.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManager.java index d42d8d1e163ce..1f58b119f2768 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManager.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManager.java @@ -333,8 +333,7 @@ private void recoverWorkers() throws Exception { } @Override - public void postStop() throws Exception { - Exception exception = null; + public CompletableFuture postStop() { FiniteDuration stopTimeout = new FiniteDuration(5L, TimeUnit.SECONDS); CompletableFuture stopTaskMonitorFuture = stopActor(taskMonitor, stopTimeout); @@ -355,22 +354,11 @@ public void postStop() throws Exception { stopLaunchCoordinatorFuture, stopReconciliationCoordinatorFuture); - // wait for the future to complete or to time out - try { - stopFuture.get(); - } catch (Exception e) { - exception = e; - } - - try { - super.postStop(); - } catch (Exception e) { - exception = ExceptionUtils.firstOrSuppressed(e, exception); - } + final CompletableFuture terminationFuture = super.postStop(); - if (exception != null) { - throw new ResourceManagerException("Could not properly shut down the ResourceManager.", exception); - } + return stopFuture.thenCombine( + terminationFuture, + (Void voidA, Void voidB) -> null); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index b2d2b6a8b0cf5..e212752295296 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -156,40 +156,40 @@ public Dispatcher( //------------------------------------------------------ @Override - public void postStop() throws Exception { + public CompletableFuture postStop() { log.info("Stopping dispatcher {}.", getAddress()); - Throwable exception = null; - - clearState(); + Exception exception = null; try { - jobManagerSharedServices.shutdown(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); + clearState(); + } catch (Exception e) { + exception = ExceptionUtils.firstOrSuppressed(e, exception); } try { - submittedJobGraphStore.stop(); + jobManagerSharedServices.shutdown(); } catch (Exception e) { exception = ExceptionUtils.firstOrSuppressed(e, exception); } try { - leaderElectionService.stop(); + submittedJobGraphStore.stop(); } catch (Exception e) { exception = ExceptionUtils.firstOrSuppressed(e, exception); } try { - super.postStop(); + leaderElectionService.stop(); } catch (Exception e) { exception = ExceptionUtils.firstOrSuppressed(e, exception); } if (exception != null) { - throw new FlinkException("Could not properly terminate the Dispatcher.", exception); + return FutureUtils.completedExceptionally( + new FlinkException("Could not properly terminate the Dispatcher.", exception)); + } else { + return CompletableFuture.completedFuture(null); } - log.info("Stopped dispatcher {}.", getAddress()); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index 015751b1ab707..425f241c140f5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -99,7 +99,6 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.util.clock.SystemClock; import org.apache.flink.runtime.webmonitor.WebMonitorUtils; -import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; @@ -123,7 +122,6 @@ import java.util.concurrent.CompletionException; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -387,8 +385,8 @@ public CompletableFuture suspend(final Exception cause, final Time * Suspend the job and shutdown all other services including rpc. */ @Override - public void postStop() throws Exception { - log.info("Stopping the JobMaster for job " + jobGraph.getName() + '(' + jobGraph.getJobID() + ")."); + public CompletableFuture postStop() { + log.info("Stopping the JobMaster for job {}({}).", jobGraph.getName(), jobGraph.getJobID()); // disconnect from all registered TaskExecutors final Set taskManagerResourceIds = new HashSet<>(registeredTaskManagers.keySet()); @@ -407,28 +405,8 @@ public void postStop() throws Exception { // shut down will internally release all registered slots slotPool.shutDown(); - CompletableFuture terminationFuture = slotPool.getTerminationFuture(); - Exception exception = null; - - // wait for the slot pool shut down - try { - terminationFuture.get(rpcTimeout.toMilliseconds(), TimeUnit.MILLISECONDS); - } catch (Exception e) { - exception = e; - } - - try { - super.postStop(); - } catch (Exception e) { - exception = ExceptionUtils.firstOrSuppressed(e, exception); - } - - if (exception != null) { - throw exception; - } - - log.info("Stopped the JobMaster for job " + jobGraph.getName() + '(' + jobGraph.getJobID() + ")."); + return slotPool.getTerminationFuture(); } //---------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java index a94a10769da24..6ba9e8a097445 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java @@ -199,7 +199,7 @@ public void start(JobMasterId jobMasterId, String newJobManagerAddress) throws E } @Override - public void postStop() throws Exception { + public CompletableFuture postStop() { // cancel all pending allocations Set allocationIds = pendingRequests.keySetB(); @@ -214,7 +214,7 @@ public void postStop() throws Exception { clear(); - super.postStop(); + return CompletableFuture.completedFuture(null); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index 2d633f1ad6f2c..77e43621dbb20 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -213,7 +213,7 @@ public void start() throws Exception { } @Override - public void postStop() throws Exception { + public CompletableFuture postStop() { Exception exception = null; taskManagerHeartbeatManager.stop(); @@ -240,14 +240,11 @@ public void postStop() throws Exception { clearState(); - try { - super.postStop(); - } catch (Exception e) { - exception = ExceptionUtils.firstOrSuppressed(e, exception); - } - if (exception != null) { - ExceptionUtils.rethrowException(exception, "Error while shutting the ResourceManager down."); + return FutureUtils.completedExceptionally( + new FlinkException("Could not properly shut down the ResourceManager.", exception)); + } else { + return CompletableFuture.completedFuture(null); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedRpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedRpcEndpoint.java index ff74f478d6ad7..d078d58f2d3a7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedRpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedRpcEndpoint.java @@ -35,7 +35,7 @@ * * @param type of the fencing token */ -public class FencedRpcEndpoint extends RpcEndpoint { +public abstract class FencedRpcEndpoint extends RpcEndpoint { private volatile F fencingToken; private volatile MainThreadExecutor fencedMainThreadExecutor; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java index d269e8473c5e1..549e5c2b50160 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -141,9 +141,10 @@ protected final void stop() { * *

IMPORTANT: This method should never be called directly by the user. * - * @throws Exception if an error occurs. The exception is returned as result of the termination future. + * @return Future which is completed once all post stop actions are completed. If an error + * occurs this future is completed exceptionally */ - public void postStop() throws Exception {} + public abstract CompletableFuture postStop(); /** * Triggers the shut down of the rpc endpoint. The shut down is executed asynchronously. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java index da7ce35497819..a7d15d6e46c12 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.rpc.akka; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.rpc.MainThreadValidatorUtil; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; @@ -90,12 +91,11 @@ public void postStop() throws Exception { mainThreadValidator.enterMainThread(); try { - Throwable shutdownThrowable = null; - + CompletableFuture postStopFuture; try { - rpcEndpoint.postStop(); + postStopFuture = rpcEndpoint.postStop(); } catch (Throwable throwable) { - shutdownThrowable = throwable; + postStopFuture = FutureUtils.completedExceptionally(throwable); } super.postStop(); @@ -105,11 +105,14 @@ public void postStop() throws Exception { // future. // Complete the termination future so that others know that we've stopped. - if (shutdownThrowable != null) { - terminationFuture.completeExceptionally(shutdownThrowable); - } else { - terminationFuture.complete(null); - } + postStopFuture.whenComplete( + (Void value, Throwable throwable) -> { + if (throwable != null) { + terminationFuture.completeExceptionally(throwable); + } else { + terminationFuture.complete(null); + } + }); } finally { mainThreadValidator.exitMainThread(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index d8804079caf81..f4c953ddb8d7e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -254,7 +254,7 @@ public void start() throws Exception { * Called to shut down the TaskManager. The method closes all TaskManager services. */ @Override - public void postStop() throws Exception { + public CompletableFuture postStop() { log.info("Stopping TaskManager {}.", getAddress()); Throwable throwable = null; @@ -281,17 +281,11 @@ public void postStop() throws Exception { throwable = ExceptionUtils.firstOrSuppressed(t, throwable); } - try { - super.postStop(); - } catch (Throwable e) { - throwable = ExceptionUtils.firstOrSuppressed(e, throwable); - } - if (throwable != null) { - ExceptionUtils.rethrowException(throwable, "Error while shutting the TaskExecutor down."); + return FutureUtils.completedExceptionally(new FlinkException("Error while shutting the TaskExecutor down.", throwable)); + } else { + return CompletableFuture.completedFuture(null); } - - log.info("Stopped TaskManager {}.", getAddress()); } // ====================================================================== diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java index 8ba0ccdb5d96e..66f8d9fd644f8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java @@ -340,6 +340,11 @@ public void anotherCall() { public boolean hasConcurrentAccess() { return concurrentAccess; } + + @Override + public CompletableFuture postStop() { + return CompletableFuture.completedFuture(null); + } } public interface FencedTestGateway extends FencedRpcGateway { @@ -384,5 +389,10 @@ public CompletableFuture setNewFencingToken(UUID fencingToken, Time return CompletableFuture.completedFuture(Acknowledge.get()); } + + @Override + public CompletableFuture postStop() { + return CompletableFuture.completedFuture(null); + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java index 3d99c3f8127fd..f488308b0b7e6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java @@ -300,6 +300,11 @@ protected FencedTestingEndpoint(RpcService rpcService, String value) { this(rpcService, value, null); } + @Override + public CompletableFuture postStop() { + return CompletableFuture.completedFuture(null); + } + protected FencedTestingEndpoint(RpcService rpcService, String value, UUID initialFencingToken) { super(rpcService); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcEndpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcEndpointTest.java index d52aadb72c31d..b5add60b246aa 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcEndpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcEndpointTest.java @@ -163,6 +163,11 @@ protected BaseEndpoint(RpcService rpcService, int foobarValue) { public CompletableFuture foobar() { return CompletableFuture.completedFuture(foobarValue); } + + @Override + public CompletableFuture postStop() { + return CompletableFuture.completedFuture(null); + } } public static class ExtendedEndpoint extends BaseEndpoint implements ExtendedGateway, DifferentGateway { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java index 2a65cac125fad..2530bcea1ee6e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java @@ -18,26 +18,28 @@ package org.apache.flink.runtime.rpc.akka; -import akka.actor.ActorSystem; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.RpcUtils; +import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcException; import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; import org.apache.flink.testutils.category.Flip6; import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; -import akka.actor.Terminated; +import akka.actor.ActorSystem; import org.hamcrest.core.Is; import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.util.Arrays; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -56,21 +58,19 @@ public class AkkaRpcActorTest extends TestLogger { // shared test members // ------------------------------------------------------------------------ - private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); - private static Time timeout = Time.milliseconds(10000L); - private static AkkaRpcService akkaRpcService = - new AkkaRpcService(actorSystem, timeout); + private static AkkaRpcService akkaRpcService; + + + @BeforeClass + public static void setup() { + akkaRpcService = new TestingRpcService(); + } @AfterClass public static void shutdown() throws InterruptedException, ExecutionException, TimeoutException { - final CompletableFuture rpcTerminationFuture = akkaRpcService.stopService(); - final CompletableFuture actorSystemTerminationFuture = FutureUtils.toJava(actorSystem.terminate()); - - FutureUtils - .waitForAll(Arrays.asList(rpcTerminationFuture, actorSystemTerminationFuture)) - .get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + RpcUtils.terminateRpcService(akkaRpcService, timeout); } /** @@ -191,7 +191,7 @@ public void testRpcEndpointTerminationFuture() throws Exception { CompletableFuture.runAsync( () -> rpcEndpoint.shutDown(), - actorSystem.dispatcher()); + akkaRpcService.getExecutor()); // wait until the rpc endpoint has terminated terminationFuture.get(); @@ -296,6 +296,33 @@ public void testActorTerminationWhenServiceShutdown() throws Exception { } } + /** + * Tests that the {@link AkkaRpcActor} only completes after the asynchronous + * post stop action has completed. + */ + @Test + public void testActorTerminationWithAsynchronousPostStopAction() throws Exception { + final CompletableFuture postStopFuture = new CompletableFuture<>(); + final AsynchronousPostStopEndpoint endpoint = new AsynchronousPostStopEndpoint(akkaRpcService, postStopFuture); + + try { + endpoint.start(); + + final CompletableFuture terminationFuture = endpoint.getTerminationFuture(); + + endpoint.shutDown(); + + assertFalse(terminationFuture.isDone()); + + postStopFuture.complete(null); + + // the postStopFuture completion should allow the endpoint to terminate + terminationFuture.get(); + } finally { + RpcUtils.terminateRpcEndpoint(endpoint, timeout); + } + } + // ------------------------------------------------------------------------ // Test Actors and Interfaces // ------------------------------------------------------------------------ @@ -309,7 +336,19 @@ private interface WrongRpcGateway extends RpcGateway { void tell(String message); } - private static class DummyRpcEndpoint extends RpcEndpoint implements DummyRpcGateway { + private static class TestRpcEndpoint extends RpcEndpoint { + + protected TestRpcEndpoint(RpcService rpcService) { + super(rpcService); + } + + @Override + public CompletableFuture postStop() { + return CompletableFuture.completedFuture(null); + } + } + + private static class DummyRpcEndpoint extends TestRpcEndpoint implements DummyRpcGateway { private volatile int _foobar = 42; @@ -333,7 +372,7 @@ private interface ExceptionalGateway extends RpcGateway { CompletableFuture doStuff(); } - private static class ExceptionalEndpoint extends RpcEndpoint implements ExceptionalGateway { + private static class ExceptionalEndpoint extends TestRpcEndpoint implements ExceptionalGateway { protected ExceptionalEndpoint(RpcService rpcService) { super(rpcService); @@ -345,7 +384,7 @@ public CompletableFuture doStuff() { } } - private static class ExceptionalFutureEndpoint extends RpcEndpoint implements ExceptionalGateway { + private static class ExceptionalFutureEndpoint extends TestRpcEndpoint implements ExceptionalGateway { protected ExceptionalFutureEndpoint(RpcService rpcService) { super(rpcService); @@ -379,8 +418,9 @@ protected SimpleRpcEndpoint(RpcService rpcService, String endpointId) { } @Override - public void postStop() { + public CompletableFuture postStop() { validateRunsInMainThread(); + return CompletableFuture.completedFuture(null); } } @@ -393,8 +433,8 @@ protected FailingPostStopEndpoint(RpcService rpcService, String endpointId) { } @Override - public void postStop() throws Exception { - throw new PostStopException("Test exception."); + public CompletableFuture postStop() { + return FutureUtils.completedExceptionally(new PostStopException("Test exception.")); } private static class PostStopException extends FlinkException { @@ -406,4 +446,22 @@ public PostStopException(String message) { } } } + + // ------------------------------------------------------------------------ + + private static class AsynchronousPostStopEndpoint extends RpcEndpoint { + + private final CompletableFuture postStopFuture; + + protected AsynchronousPostStopEndpoint(RpcService rpcService, CompletableFuture postStopFuture) { + super(rpcService); + + this.postStopFuture = Preconditions.checkNotNull(postStopFuture); + } + + @Override + public CompletableFuture postStop() { + return postStopFuture; + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java index a69bd84c9bfe7..6dacdfd47dffc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java @@ -30,6 +30,8 @@ import org.junit.Test; import org.junit.experimental.categories.Category; +import java.util.concurrent.CompletableFuture; + import static org.junit.Assert.assertTrue; @Category(Flip6.class) @@ -90,6 +92,11 @@ public TestEndpoint(RpcService rpcService) { super(rpcService); } + @Override + public CompletableFuture postStop() { + return CompletableFuture.completedFuture(null); + } + @Override public void someConcurrencyCriticalFunction() { validateRunsInMainThread(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java index 061145c3f4334..60068505fa231 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java @@ -193,6 +193,11 @@ protected TestEndpoint(RpcService rpcService, LinkedBlockingQueue queue) public void foobar(Object object) throws InterruptedException { queue.put(object); } + + @Override + public CompletableFuture postStop() { + return CompletableFuture.completedFuture(null); + } } private static class NonSerializableObject { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/impl/RpcGatewayRetrieverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/impl/RpcGatewayRetrieverTest.java index ae530f7d19070..5f59d59a98cc9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/impl/RpcGatewayRetrieverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/impl/RpcGatewayRetrieverTest.java @@ -138,5 +138,10 @@ public CompletableFuture foobar(Time timeout) { public UUID getFencingToken() { return HighAvailabilityServices.DEFAULT_LEADER_ID; } + + @Override + public CompletableFuture postStop() { + return CompletableFuture.completedFuture(null); + } } } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java index 87324cba91a3b..5380356de7965 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -36,6 +37,7 @@ import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; import org.apache.flink.yarn.configuration.YarnConfigOptions; import org.apache.hadoop.yarn.api.ApplicationConstants; @@ -56,6 +58,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -218,7 +221,7 @@ protected void initialize() throws ResourceManagerException { } @Override - public void postStop() throws Exception { + public CompletableFuture postStop() { // shut down all components Throwable firstException = null; @@ -238,14 +241,12 @@ public void postStop() throws Exception { } } - try { - super.postStop(); - } catch (Throwable t) { - firstException = ExceptionUtils.firstOrSuppressed(t, firstException); - } + final CompletableFuture terminationFuture = super.postStop(); if (firstException != null) { - ExceptionUtils.rethrowException(firstException, "Error while shutting down YARN resource manager"); + return FutureUtils.completedExceptionally(new FlinkException("Error while shutting down YARN resource manager", firstException)); + } else { + return terminationFuture; } } From e29ec0fbd2cb03a42b98142f63ce73b97dc2e915 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 16 Feb 2018 08:12:04 +0100 Subject: [PATCH 07/10] [FLINK-8670] Make MetricRegistryImpl#shutdown non blocking This commit makes the MetricRegistryImpl#shutdown method non blocking. Instead of waiting for the completion of the shutdown procedure, the method returns a future which is completed once the metric registry has completed the shut down. This closes #5504. --- .../org/apache/flink/util/ExecutorUtils.java | 21 +++ .../MesosApplicationMasterRunner.java | 2 +- .../ScheduledDropwizardReporterTest.java | 4 +- .../DropwizardFlinkHistogramWrapperTest.java | 2 +- .../flink/metrics/jmx/JMXReporterTest.java | 8 +- .../PrometheusReporterTaskScopeTest.java | 4 +- .../prometheus/PrometheusReporterTest.java | 20 +-- .../metrics/slf4j/Slf4jReporterTest.java | 4 +- .../metrics/statsd/StatsDReporterTest.java | 8 +- .../apache/flink/runtime/akka/ActorUtils.java | 89 +++++++++++++ .../runtime/metrics/MetricRegistryImpl.java | 122 ++++++++++-------- .../flink/runtime/jobmanager/JobManager.scala | 2 +- .../minicluster/FlinkMiniCluster.scala | 2 +- .../runtime/taskmanager/TaskManager.scala | 2 +- .../metrics/MetricRegistryImplTest.java | 42 +++--- .../metrics/TaskManagerMetricsTest.java | 2 +- .../groups/AbstractMetricGroupTest.java | 10 +- .../metrics/groups/JobManagerGroupTest.java | 16 +-- .../groups/JobManagerJobGroupTest.java | 12 +- .../groups/MetricGroupRegistrationTest.java | 8 +- .../metrics/groups/MetricGroupTest.java | 4 +- .../metrics/groups/OperatorGroupTest.java | 37 +++--- .../metrics/groups/TaskManagerGroupTest.java | 30 +++-- .../groups/TaskManagerJobGroupTest.java | 28 ++-- .../metrics/groups/TaskMetricGroupTest.java | 34 +++-- .../yarn/YarnApplicationMasterRunner.java | 2 +- 26 files changed, 334 insertions(+), 181 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/akka/ActorUtils.java diff --git a/flink-core/src/main/java/org/apache/flink/util/ExecutorUtils.java b/flink-core/src/main/java/org/apache/flink/util/ExecutorUtils.java index d98bdd29cbd80..0a7f1613a8089 100644 --- a/flink-core/src/main/java/org/apache/flink/util/ExecutorUtils.java +++ b/flink-core/src/main/java/org/apache/flink/util/ExecutorUtils.java @@ -21,6 +21,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; @@ -74,4 +75,24 @@ public static void gracefulShutdown(long timeout, TimeUnit unit, ExecutorService } } } + + /** + * Shuts the given {@link ExecutorService} down in a non-blocking fashion. The shut down will + * be executed by a thread from the common fork-join pool. + * + *

The executor services will be shut down gracefully for the given timeout period. Afterwards + * {@link ExecutorService#shutdownNow()} will be called. + * + * @param timeout before {@link ExecutorService#shutdownNow()} is called + * @param unit time unit of the timeout + * @param executorServices to shut down + * @return Future which is completed once the {@link ExecutorService} are shut down + */ + public static CompletableFuture nonBlockingShutdown(long timeout, TimeUnit unit, ExecutorService... executorServices) { + return CompletableFuture.supplyAsync( + () -> { + gracefulShutdown(timeout, unit, executorServices); + return null; + }); + } } diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java index 94804aceaf60f..630fa8390003a 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java @@ -433,7 +433,7 @@ protected int runPrivileged(Configuration config, Configuration dynamicPropertie if (metricRegistry != null) { try { - metricRegistry.shutdown(); + metricRegistry.shutdown().get(); } catch (Throwable t) { LOG.error("Could not shut down metric registry.", t); } diff --git a/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/ScheduledDropwizardReporterTest.java b/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/ScheduledDropwizardReporterTest.java index 4a2ca3ab34d54..b69b8d8e3fa51 100644 --- a/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/ScheduledDropwizardReporterTest.java +++ b/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/ScheduledDropwizardReporterTest.java @@ -74,7 +74,7 @@ public ScheduledReporter getReporter(MetricConfig config) { * Tests that the registered metrics' names don't contain invalid characters. */ @Test - public void testAddingMetrics() throws NoSuchFieldException, IllegalAccessException { + public void testAddingMetrics() throws Exception { Configuration configuration = new Configuration(); String taskName = "test\"Ta\"..sk"; String jobName = "testJ\"ob:-!ax..?"; @@ -131,7 +131,7 @@ public void testAddingMetrics() throws NoSuchFieldException, IllegalAccessExcept assertEquals(expectedCounterName, counters.get(myCounter)); - metricRegistry.shutdown(); + metricRegistry.shutdown().get(); } /** diff --git a/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/metrics/DropwizardFlinkHistogramWrapperTest.java b/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/metrics/DropwizardFlinkHistogramWrapperTest.java index fb21a75b86ddc..d23a22ca30df4 100644 --- a/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/metrics/DropwizardFlinkHistogramWrapperTest.java +++ b/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/metrics/DropwizardFlinkHistogramWrapperTest.java @@ -150,7 +150,7 @@ public void testDropwizardHistogramWrapperReporting() throws Exception { assertEquals(0, testingReporter.getMetrics().size()); } finally { if (registry != null) { - registry.shutdown(); + registry.shutdown().get(); } } } diff --git a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/metrics/jmx/JMXReporterTest.java b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/metrics/jmx/JMXReporterTest.java index 40b7f158639fb..6e4564671c595 100644 --- a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/metrics/jmx/JMXReporterTest.java +++ b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/metrics/jmx/JMXReporterTest.java @@ -145,7 +145,7 @@ public Integer getValue() { rep1.notifyOfRemovedMetric(g2, "rep2", null); mg.close(); - reg.shutdown(); + reg.shutdown().get(); } /** @@ -219,7 +219,7 @@ public Integer getValue() { rep1.close(); rep2.close(); mg.close(); - reg.shutdown(); + reg.shutdown().get(); } /** @@ -266,7 +266,7 @@ public void testHistogramReporting() throws Exception { } finally { if (registry != null) { - registry.shutdown(); + registry.shutdown().get(); } } } @@ -306,7 +306,7 @@ public void testMeterReporting() throws Exception { } finally { if (registry != null) { - registry.shutdown(); + registry.shutdown().get(); } } } diff --git a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTaskScopeTest.java b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTaskScopeTest.java index d4ad1f9ea0d14..724a79b86eb85 100644 --- a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTaskScopeTest.java +++ b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTaskScopeTest.java @@ -90,9 +90,9 @@ public void setupReporter() { } @After - public void shutdownRegistry() { + public void shutdownRegistry() throws Exception { if (registry != null) { - registry.shutdown(); + registry.shutdown().get(); } } diff --git a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java index 6833a0661b931..e9fd985e4ac2c 100644 --- a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java +++ b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java @@ -84,9 +84,9 @@ public void setupReporter() { } @After - public void shutdownRegistry() { + public void shutdownRegistry() throws Exception { if (registry != null) { - registry.shutdown(); + registry.shutdown().get(); } } @@ -237,7 +237,7 @@ class SomeMetricType implements Metric{} } @Test - public void cannotStartTwoReportersOnSamePort() { + public void cannotStartTwoReportersOnSamePort() throws Exception { final MetricRegistryImpl fixedPort1 = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", portRangeProvider.next()))); assertThat(fixedPort1.getReporters(), hasSize(1)); @@ -246,12 +246,12 @@ public void cannotStartTwoReportersOnSamePort() { final MetricRegistryImpl fixedPort2 = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test2", String.valueOf(firstReporter.getPort())))); assertThat(fixedPort2.getReporters(), hasSize(0)); - fixedPort1.shutdown(); - fixedPort2.shutdown(); + fixedPort1.shutdown().get(); + fixedPort2.shutdown().get(); } @Test - public void canStartTwoReportersWhenUsingPortRange() { + public void canStartTwoReportersWhenUsingPortRange() throws Exception { String portRange = portRangeProvider.next(); final MetricRegistryImpl portRange1 = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", portRange))); final MetricRegistryImpl portRange2 = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test2", portRange))); @@ -259,8 +259,8 @@ public void canStartTwoReportersWhenUsingPortRange() { assertThat(portRange1.getReporters(), hasSize(1)); assertThat(portRange2.getReporters(), hasSize(1)); - portRange1.shutdown(); - portRange2.shutdown(); + portRange1.shutdown().get(); + portRange2.shutdown().get(); } private String addMetricAndPollResponse(Metric metric, String metricName) throws UnirestException { @@ -280,8 +280,8 @@ static Configuration createConfigWithOneReporter(String reporterName, String por } @After - public void closeReporterAndShutdownRegistry() { - registry.shutdown(); + public void closeReporterAndShutdownRegistry() throws Exception { + registry.shutdown().get(); } /** diff --git a/flink-metrics/flink-metrics-slf4j/src/test/java/org/apache/flink/metrics/slf4j/Slf4jReporterTest.java b/flink-metrics/flink-metrics-slf4j/src/test/java/org/apache/flink/metrics/slf4j/Slf4jReporterTest.java index b344f45bca608..172c79c775f50 100644 --- a/flink-metrics/flink-metrics-slf4j/src/test/java/org/apache/flink/metrics/slf4j/Slf4jReporterTest.java +++ b/flink-metrics/flink-metrics-slf4j/src/test/java/org/apache/flink/metrics/slf4j/Slf4jReporterTest.java @@ -76,8 +76,8 @@ public static void setUp() { } @AfterClass - public static void tearDown() { - registry.shutdown(); + public static void tearDown() throws Exception { + registry.shutdown().get(); } @Test diff --git a/flink-metrics/flink-metrics-statsd/src/test/java/org/apache/flink/metrics/statsd/StatsDReporterTest.java b/flink-metrics/flink-metrics-statsd/src/test/java/org/apache/flink/metrics/statsd/StatsDReporterTest.java index 08d4998c859f2..c9f5af07a72a1 100644 --- a/flink-metrics/flink-metrics-statsd/src/test/java/org/apache/flink/metrics/statsd/StatsDReporterTest.java +++ b/flink-metrics/flink-metrics-statsd/src/test/java/org/apache/flink/metrics/statsd/StatsDReporterTest.java @@ -73,7 +73,7 @@ public void testReplaceInvalidChars() throws NoSuchMethodException, InvocationTa * Tests that the registered metrics' names don't contain invalid characters. */ @Test - public void testAddingMetrics() throws NoSuchFieldException, IllegalAccessException { + public void testAddingMetrics() throws Exception { Configuration configuration = new Configuration(); String taskName = "testTask"; String jobName = "testJob:-!ax..?"; @@ -124,7 +124,7 @@ public void testAddingMetrics() throws NoSuchFieldException, IllegalAccessExcept assertEquals(expectedCounterName, counters.get(myCounter)); - metricRegistry.shutdown(); + metricRegistry.shutdown().get(); } /** @@ -187,7 +187,7 @@ public void testStatsDHistogramReporting() throws Exception { } finally { if (registry != null) { - registry.shutdown(); + registry.shutdown().get(); } if (receiver != null) { @@ -247,7 +247,7 @@ public void testStatsDMetersReporting() throws Exception { } finally { if (registry != null) { - registry.shutdown(); + registry.shutdown().get(); } if (receiver != null) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/akka/ActorUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/akka/ActorUtils.java new file mode 100644 index 0000000000000..f2f905971c332 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/akka/ActorUtils.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.akka; + +import org.apache.flink.runtime.concurrent.FutureUtils; + +import akka.actor.ActorRef; +import akka.actor.Kill; +import akka.pattern.Patterns; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; + +/** + * Utility functions for the interaction with Akka {@link akka.actor.Actor}. + */ +public class ActorUtils { + + private static final Logger LOG = LoggerFactory.getLogger(ActorUtils.class); + + /** + * Shuts the given {@link akka.actor.Actor} down in a non blocking fashion. The method first tries to + * gracefully shut them down. If this is not successful, then the actors will be terminated by sending + * a {@link akka.actor.Kill} message. + * + * @param gracePeriod for the graceful shutdown + * @param timeUnit time unit of the grace period + * @param actors to shut down + * @return Future which is completed once all actors have been shut down gracefully or forceful + * kill messages have been sent to all actors. Occurring errors will be suppressed into one error. + */ + public static CompletableFuture nonBlockingShutDown(long gracePeriod, TimeUnit timeUnit, ActorRef... actors) { + final Collection> terminationFutures = new ArrayList<>(actors.length); + final FiniteDuration timeout = new FiniteDuration(gracePeriod, timeUnit); + + for (ActorRef actor : actors) { + try { + final Future booleanFuture = Patterns.gracefulStop(actor, timeout); + final CompletableFuture terminationFuture = FutureUtils.toJava(booleanFuture) + .thenApply(ignored -> null) + .exceptionally((Throwable throwable) -> { + if (throwable instanceof TimeoutException) { + // the actor did not gracefully stop within the grace period --> Let's kill him + actor.tell(Kill.getInstance(), ActorRef.noSender()); + return null; + } else { + throw new CompletionException(throwable); + } + }); + + terminationFutures.add(terminationFuture); + } catch (IllegalStateException ignored) { + // this can happen if the underlying actor system has been stopped before shutting + // the actor down + LOG.debug("The actor {} has already been stopped because the " + + "underlying ActorSystem has already been shut down.", actor.path()); + } + } + + return FutureUtils.completeAll(terminationFutures); + } + + private ActorUtils() {} +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java index c8f449022b285..6b3770907a94a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.metrics; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -28,35 +29,36 @@ import org.apache.flink.metrics.View; import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.metrics.reporter.Scheduled; +import org.apache.flink.runtime.akka.ActorUtils; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.metrics.dump.MetricQueryService; import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; import org.apache.flink.runtime.metrics.groups.FrontMetricGroup; import org.apache.flink.runtime.metrics.scope.ScopeFormats; import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.ExecutorUtils; +import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import akka.actor.ActorRef; import akka.actor.ActorSystem; -import akka.actor.Kill; -import akka.pattern.Patterns; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.TimerTask; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import scala.concurrent.Await; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; - /** * A MetricRegistry keeps track of all registered {@link Metric Metrics}. It serves as the * connection between {@link MetricGroup MetricGroups} and {@link MetricReporter MetricReporters}. @@ -66,8 +68,14 @@ public class MetricRegistryImpl implements MetricRegistry { private final Object lock = new Object(); - private List reporters; - private ScheduledExecutorService executor; + private final List reporters; + private final ScheduledExecutorService executor; + + private final ScopeFormats scopeFormats; + private final char globalDelimiter; + private final List delimiters; + + private final CompletableFuture terminationFuture; @Nullable private ActorRef queryService; @@ -77,9 +85,7 @@ public class MetricRegistryImpl implements MetricRegistry { private ViewUpdater viewUpdater; - private final ScopeFormats scopeFormats; - private final char globalDelimiter; - private final List delimiters = new ArrayList<>(); + private boolean isShutdown; /** * Creates a new MetricRegistry and starts the configured reporter. @@ -87,9 +93,12 @@ public class MetricRegistryImpl implements MetricRegistry { public MetricRegistryImpl(MetricRegistryConfiguration config) { this.scopeFormats = config.getScopeFormats(); this.globalDelimiter = config.getDelimiter(); + this.delimiters = new ArrayList<>(10); + this.terminationFuture = new CompletableFuture<>(); + this.isShutdown = false; // second, instantiate any custom configured reporters - this.reporters = new ArrayList<>(); + this.reporters = new ArrayList<>(4); List> reporterConfigurations = config.getReporterConfigurations(); @@ -226,71 +235,72 @@ public List getReporters() { */ public boolean isShutdown() { synchronized (lock) { - return reporters == null && executor.isShutdown(); + return isShutdown; } } /** * Shuts down this registry and the associated {@link MetricReporter}. + * + *

NOTE: This operation is asynchronous and returns a future which is completed + * once the shutdown operation has been completed. + * + * @return Future which is completed once the {@link MetricRegistryImpl} + * is shut down. */ - public void shutdown() { + public CompletableFuture shutdown() { synchronized (lock) { - Future stopFuture = null; - FiniteDuration stopTimeout = null; + if (isShutdown) { + return terminationFuture; + } else { + isShutdown = true; + final Collection> terminationFutures = new ArrayList<>(3); + final Time gracePeriod = Time.seconds(1L); - if (queryService != null) { - stopTimeout = new FiniteDuration(1L, TimeUnit.SECONDS); + if (queryService != null) { + final CompletableFuture queryServiceTerminationFuture = ActorUtils.nonBlockingShutDown( + gracePeriod.toMilliseconds(), + TimeUnit.MILLISECONDS, + queryService); - try { - stopFuture = Patterns.gracefulStop(queryService, stopTimeout); - } catch (IllegalStateException ignored) { - // this can happen if the underlying actor system has been stopped before shutting - // the metric registry down - // TODO: Pull the MetricQueryService actor out of the MetricRegistry - LOG.debug("The metric query service actor has already been stopped because the " + - "underlying ActorSystem has already been shut down."); + terminationFutures.add(queryServiceTerminationFuture); } - } - if (reporters != null) { + Throwable throwable = null; for (MetricReporter reporter : reporters) { try { reporter.close(); } catch (Throwable t) { - LOG.warn("Metrics reporter did not shut down cleanly", t); + throwable = ExceptionUtils.firstOrSuppressed(t, throwable); } } - reporters = null; - } - shutdownExecutor(); - - if (stopFuture != null) { - boolean stopped = false; - - try { - stopped = Await.result(stopFuture, stopTimeout); - } catch (Exception e) { - LOG.warn("Query actor did not properly stop.", e); - } + reporters.clear(); - if (!stopped) { - // the query actor did not stop in time, let's kill him - queryService.tell(Kill.getInstance(), ActorRef.noSender()); + if (throwable != null) { + terminationFutures.add( + FutureUtils.completedExceptionally( + new FlinkException("Could not shut down the metric reporters properly.", throwable))); } - } - } - } - private void shutdownExecutor() { - if (executor != null) { - executor.shutdown(); + final CompletableFuture executorShutdownFuture = ExecutorUtils.nonBlockingShutdown( + gracePeriod.toMilliseconds(), + TimeUnit.MILLISECONDS, + executor); + + terminationFutures.add(executorShutdownFuture); + + FutureUtils + .completeAll(terminationFutures) + .whenComplete( + (Void ignored, Throwable error) -> { + if (error != null) { + terminationFuture.completeExceptionally(error); + } else { + terminationFuture.complete(null); + } + }); - try { - if (!executor.awaitTermination(1L, TimeUnit.SECONDS)) { - executor.shutdownNow(); - } - } catch (InterruptedException e) { - executor.shutdownNow(); + return terminationFuture; } } } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index d0e401cea5db4..1dfaa5d344bf7 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -2054,7 +2054,7 @@ object JobManager { } try { - metricRegistry.shutdown() + metricRegistry.shutdown().get() } catch { case t: Throwable => LOG.warn("Could not properly shut down the metric registry.", t) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala index 7948ba177e22a..6c9ee5ba6d6ae 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala @@ -468,7 +468,7 @@ abstract class FlinkMiniCluster( Await.ready(Future.sequence(jmFutures ++ tmFutures ++ rmFutures), timeout) - metricRegistryOpt.foreach(_.shutdown()) + metricRegistryOpt.foreach(_.shutdown().get()) if (!useSingleActorSystem) { taskManagerActorSystems foreach { diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 106bea14ff8c1..485add57c6cfb 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -1900,7 +1900,7 @@ object TaskManager { // shut down the metric query service try { - metricRegistry.shutdown() + metricRegistry.shutdown().get() } catch { case t: Throwable => LOG.error("Could not properly shut down the metric registry.", t) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java index 2eccc0cb7a04d..adb622d7e1c19 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java @@ -59,12 +59,12 @@ public class MetricRegistryImplTest extends TestLogger { private static final char GLOBAL_DEFAULT_DELIMITER = '.'; @Test - public void testIsShutdown() { + public void testIsShutdown() throws Exception { MetricRegistryImpl metricRegistry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); Assert.assertFalse(metricRegistry.isShutdown()); - metricRegistry.shutdown(); + metricRegistry.shutdown().get(); Assert.assertTrue(metricRegistry.isShutdown()); } @@ -73,7 +73,7 @@ public void testIsShutdown() { * Verifies that the reporter name list is correctly used to determine which reporters should be instantiated. */ @Test - public void testReporterInclusion() { + public void testReporterInclusion() throws Exception { Configuration config = new Configuration(); config.setString(MetricOptions.REPORTERS_LIST, "test"); @@ -87,7 +87,7 @@ public void testReporterInclusion() { Assert.assertTrue(TestReporter1.wasOpened); Assert.assertFalse(TestReporter11.wasOpened); - metricRegistry.shutdown(); + metricRegistry.shutdown().get(); } /** @@ -106,7 +106,7 @@ public void open(MetricConfig config) { * Verifies that multiple reporters are instantiated correctly. */ @Test - public void testMultipleReporterInstantiation() { + public void testMultipleReporterInstantiation() throws Exception { Configuration config = new Configuration(); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter11.class.getName()); @@ -121,7 +121,7 @@ public void testMultipleReporterInstantiation() { Assert.assertTrue(TestReporter12.wasOpened); Assert.assertTrue(TestReporter13.wasOpened); - metricRegistry.shutdown(); + metricRegistry.shutdown().get(); } /** @@ -164,14 +164,14 @@ public void open(MetricConfig config) { * Verifies that configured arguments are properly forwarded to the reporter. */ @Test - public void testReporterArgumentForwarding() { + public void testReporterArgumentForwarding() throws Exception { Configuration config = new Configuration(); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter2.class.getName()); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test.arg1", "hello"); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test.arg2", "world"); - new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)).shutdown(); + new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)).shutdown().get(); Assert.assertEquals("hello", TestReporter2.mc.getString("arg1", null)); Assert.assertEquals("world", TestReporter2.mc.getString("arg2", null)); @@ -190,11 +190,9 @@ public void open(MetricConfig config) { /** * Verifies that reporters implementing the Scheduled interface are regularly called to report the metrics. - * - * @throws InterruptedException */ @Test - public void testReporterScheduling() throws InterruptedException { + public void testReporterScheduling() throws Exception { Configuration config = new Configuration(); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter3.class.getName()); @@ -225,7 +223,7 @@ public void testReporterScheduling() throws InterruptedException { } Assert.assertTrue("No report was triggered.", TestReporter3.reportCount > 0); - registry.shutdown(); + registry.shutdown().get(); } /** @@ -244,7 +242,7 @@ public void report() { * Verifies that reporters are notified of added/removed metrics. */ @Test - public void testReporterNotifications() { + public void testReporterNotifications() throws Exception { Configuration config = new Configuration(); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter6.class.getName()); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter7.class.getName()); @@ -268,7 +266,7 @@ public void testReporterNotifications() { assertTrue(TestReporter7.removedMetric instanceof Counter); assertEquals("rootCounter", TestReporter7.removedMetricName); - registry.shutdown(); + registry.shutdown().get(); } /** @@ -338,7 +336,7 @@ public void testScopeConfig() { } @Test - public void testConfigurableDelimiter() { + public void testConfigurableDelimiter() throws Exception { Configuration config = new Configuration(); config.setString(MetricOptions.SCOPE_DELIMITER, "_"); config.setString(MetricOptions.SCOPE_NAMING_TM, "A.B.C.D.E"); @@ -348,11 +346,11 @@ public void testConfigurableDelimiter() { TaskManagerMetricGroup tmGroup = new TaskManagerMetricGroup(registry, "host", "id"); assertEquals("A_B_C_D_E_name", tmGroup.getMetricIdentifier("name")); - registry.shutdown(); + registry.shutdown().get(); } @Test - public void testConfigurableDelimiterForReporters() { + public void testConfigurableDelimiterForReporters() throws Exception { Configuration config = new Configuration(); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "_"); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter.class.getName()); @@ -370,11 +368,11 @@ public void testConfigurableDelimiterForReporters() { assertEquals(GLOBAL_DEFAULT_DELIMITER, registry.getDelimiter(3)); assertEquals(GLOBAL_DEFAULT_DELIMITER, registry.getDelimiter(-1)); - registry.shutdown(); + registry.shutdown().get(); } @Test - public void testConfigurableDelimiterForReportersInGroup() { + public void testConfigurableDelimiterForReportersInGroup() throws Exception { Configuration config = new Configuration(); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "_"); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter8.class.getName()); @@ -395,7 +393,7 @@ public void testConfigurableDelimiterForReportersInGroup() { TaskManagerMetricGroup group = new TaskManagerMetricGroup(registry, "host", "id"); group.counter("C"); group.close(); - registry.shutdown(); + registry.shutdown().get(); assertEquals(4, TestReporter8.numCorrectDelimitersForRegister); assertEquals(4, TestReporter8.numCorrectDelimitersForUnregister); } @@ -415,7 +413,7 @@ public void testQueryActorShutdown() throws Exception { ActorRef queryServiceActor = registry.getQueryService(); - registry.shutdown(); + registry.shutdown().get(); try { Await.result(actorSystem.actorSelection(queryServiceActor.path()).resolveOne(timeout), timeout); @@ -471,7 +469,7 @@ public void testExceptionIsolation() throws Exception { assertEquals(metric, TestReporter7.removedMetric); assertEquals("counter", TestReporter7.removedMetricName); - registry.shutdown(); + registry.shutdown().get(); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TaskManagerMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TaskManagerMetricsTest.java index fe22095fdb276..179885112b12b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TaskManagerMetricsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TaskManagerMetricsTest.java @@ -162,7 +162,7 @@ protected void run() { highAvailabilityServices.closeAndCleanupAllData(); } - metricRegistry.shutdown(); + metricRegistry.shutdown().get(); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroupTest.java index 325982baa2b3a..f8ed3c6a6d8e0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroupTest.java @@ -44,7 +44,7 @@ public class AbstractMetricGroupTest { * called and the parent is null. */ @Test - public void testGetAllVariables() { + public void testGetAllVariables() throws Exception { MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); AbstractMetricGroup group = new AbstractMetricGroup>(registry, new String[0], null) { @@ -60,7 +60,7 @@ protected String getGroupName(CharacterFilter filter) { }; assertTrue(group.getAllVariables().isEmpty()); - registry.shutdown(); + registry.shutdown().get(); } // ======================================================================== @@ -101,7 +101,7 @@ public void testScopeCachingForMultipleReporters() throws Exception { } } } finally { - testRegistry.shutdown(); + testRegistry.shutdown().get(); } } @@ -176,7 +176,7 @@ public String filterCharacters(String input) { } @Test - public void testScopeGenerationWithoutReporters() { + public void testScopeGenerationWithoutReporters() throws Exception { Configuration config = new Configuration(); config.setString(MetricOptions.SCOPE_NAMING_TM, "A.B.C.D"); MetricRegistryImpl testRegistry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); @@ -193,7 +193,7 @@ public void testScopeGenerationWithoutReporters() { assertEquals("A.X.C.D.1", group.getMetricIdentifier("1", FILTER_B, -1)); assertEquals("A.X.C.D.1", group.getMetricIdentifier("1", FILTER_B, 2)); } finally { - testRegistry.shutdown(); + testRegistry.shutdown().get(); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerGroupTest.java index 05a72acb4d1e4..cb5ec67c97c92 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerGroupTest.java @@ -44,7 +44,7 @@ public class JobManagerGroupTest extends TestLogger { // ------------------------------------------------------------------------ @Test - public void addAndRemoveJobs() { + public void addAndRemoveJobs() throws Exception { MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); final JobManagerMetricGroup group = new JobManagerMetricGroup(registry, "localhost"); @@ -72,11 +72,11 @@ public void addAndRemoveJobs() { assertTrue(jmJobGroup21.isClosed()); assertEquals(0, group.numRegisteredJobMetricGroups()); - registry.shutdown(); + registry.shutdown().get(); } @Test - public void testCloseClosesAll() { + public void testCloseClosesAll() throws Exception { MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); final JobManagerMetricGroup group = new JobManagerMetricGroup(registry, "localhost"); @@ -94,7 +94,7 @@ public void testCloseClosesAll() { assertTrue(jmJobGroup11.isClosed()); assertTrue(jmJobGroup21.isClosed()); - registry.shutdown(); + registry.shutdown().get(); } // ------------------------------------------------------------------------ @@ -102,18 +102,18 @@ public void testCloseClosesAll() { // ------------------------------------------------------------------------ @Test - public void testGenerateScopeDefault() { + public void testGenerateScopeDefault() throws Exception { MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); JobManagerMetricGroup group = new JobManagerMetricGroup(registry, "localhost"); assertArrayEquals(new String[]{"localhost", "jobmanager"}, group.getScopeComponents()); assertEquals("localhost.jobmanager.name", group.getMetricIdentifier("name")); - registry.shutdown(); + registry.shutdown().get(); } @Test - public void testGenerateScopeCustom() { + public void testGenerateScopeCustom() throws Exception { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_JM, "constant..foo."); MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(cfg)); @@ -123,7 +123,7 @@ public void testGenerateScopeCustom() { assertArrayEquals(new String[]{"constant", "host", "foo", "host"}, group.getScopeComponents()); assertEquals("constant.host.foo.host.name", group.getMetricIdentifier("name")); - registry.shutdown(); + registry.shutdown().get(); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerJobGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerJobGroupTest.java index 4373f80c37d9b..6f4751b07d9b7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerJobGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerJobGroupTest.java @@ -38,7 +38,7 @@ public class JobManagerJobGroupTest extends TestLogger { @Test - public void testGenerateScopeDefault() { + public void testGenerateScopeDefault() throws Exception { MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); JobManagerMetricGroup tmGroup = new JobManagerMetricGroup(registry, "theHostName"); @@ -52,11 +52,11 @@ public void testGenerateScopeDefault() { "theHostName.jobmanager.myJobName.name", jmGroup.getMetricIdentifier("name")); - registry.shutdown(); + registry.shutdown().get(); } @Test - public void testGenerateScopeCustom() { + public void testGenerateScopeCustom() throws Exception { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_JM, "abc"); cfg.setString(MetricOptions.SCOPE_NAMING_JM_JOB, "some-constant."); @@ -75,11 +75,11 @@ public void testGenerateScopeCustom() { "some-constant.myJobName.name", jmGroup.getMetricIdentifier("name")); - registry.shutdown(); + registry.shutdown().get(); } @Test - public void testGenerateScopeCustomWildcard() { + public void testGenerateScopeCustomWildcard() throws Exception { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_JM, "peter"); cfg.setString(MetricOptions.SCOPE_NAMING_JM_JOB, "*.some-constant."); @@ -98,7 +98,7 @@ public void testGenerateScopeCustomWildcard() { "peter.some-constant." + jid + ".name", jmGroup.getMetricIdentifier("name")); - registry.shutdown(); + registry.shutdown().get(); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupRegistrationTest.java index bcdcd63ac6dc9..22148db75a776 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupRegistrationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupRegistrationTest.java @@ -44,7 +44,7 @@ public class MetricGroupRegistrationTest extends TestLogger { * Verifies that group methods instantiate the correct metric with the given name. */ @Test - public void testMetricInstantiation() { + public void testMetricInstantiation() throws Exception { Configuration config = new Configuration(); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter1.class.getName()); @@ -85,7 +85,7 @@ public HistogramStatistics getStatistics() { Assert.assertEquals(histogram, TestReporter1.lastPassedMetric); assertEquals("histogram", TestReporter1.lastPassedName); - registry.shutdown(); + registry.shutdown().get(); } /** @@ -107,7 +107,7 @@ public void notifyOfAddedMetric(Metric metric, String metricName, MetricGroup gr * Verifies that when attempting to create a group with the name of an existing one the existing one will be returned instead. */ @Test - public void testDuplicateGroupName() { + public void testDuplicateGroupName() throws Exception { Configuration config = new Configuration(); MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); @@ -119,6 +119,6 @@ public void testDuplicateGroupName() { MetricGroup group3 = root.addGroup("group"); Assert.assertTrue(group1 == group2 && group2 == group3); - registry.shutdown(); + registry.shutdown().get(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupTest.java index 4dc5edf8a17b6..71ae7f18ecb7e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupTest.java @@ -62,8 +62,8 @@ public void createRegistry() { } @After - public void shutdownRegistry() { - this.registry.shutdown(); + public void shutdownRegistry() throws Exception { + this.registry.shutdown().get(); this.registry = null; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/OperatorGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/OperatorGroupTest.java index 820b73efc0010..58198e4f92313 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/OperatorGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/OperatorGroupTest.java @@ -32,6 +32,8 @@ import org.apache.flink.util.AbstractID; import org.apache.flink.util.TestLogger; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.util.Map; @@ -45,10 +47,22 @@ */ public class OperatorGroupTest extends TestLogger { - @Test - public void testGenerateScopeDefault() { - MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + private MetricRegistryImpl registry; + + @Before + public void setup() { + registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + } + + @After + public void teardown() throws Exception { + if (registry != null) { + registry.shutdown().get(); + } + } + @Test + public void testGenerateScopeDefault() throws Exception { TaskManagerMetricGroup tmGroup = new TaskManagerMetricGroup(registry, "theHostName", "test-tm-id"); TaskManagerJobMetricGroup jmGroup = new TaskManagerJobMetricGroup(registry, tmGroup, new JobID(), "myJobName"); TaskMetricGroup taskGroup = new TaskMetricGroup( @@ -62,12 +76,10 @@ public void testGenerateScopeDefault() { assertEquals( "theHostName.taskmanager.test-tm-id.myJobName.myOpName.11.name", opGroup.getMetricIdentifier("name")); - - registry.shutdown(); } @Test - public void testGenerateScopeCustom() { + public void testGenerateScopeCustom() throws Exception { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_OPERATOR, "...."); MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(cfg)); @@ -91,14 +103,12 @@ public void testGenerateScopeCustom() { String.format("%s.%s.%s.%s.%s.name", tmID, jid, vertexId, operatorName, operatorID), operatorGroup.getMetricIdentifier("name")); } finally { - registry.shutdown(); + registry.shutdown().get(); } } @Test - public void testIOMetricGroupInstantiation() { - MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); - + public void testIOMetricGroupInstantiation() throws Exception { TaskManagerMetricGroup tmGroup = new TaskManagerMetricGroup(registry, "theHostName", "test-tm-id"); TaskManagerJobMetricGroup jmGroup = new TaskManagerJobMetricGroup(registry, tmGroup, new JobID(), "myJobName"); TaskMetricGroup taskGroup = new TaskMetricGroup( @@ -108,14 +118,10 @@ public void testIOMetricGroupInstantiation() { assertNotNull(opGroup.getIOMetricGroup()); assertNotNull(opGroup.getIOMetricGroup().getNumRecordsInCounter()); assertNotNull(opGroup.getIOMetricGroup().getNumRecordsOutCounter()); - - registry.shutdown(); } @Test public void testVariables() { - MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); - JobID jid = new JobID(); JobVertexID tid = new JobVertexID(); AbstractID eid = new AbstractID(); @@ -140,8 +146,6 @@ public void testVariables() { testVariable(variables, ScopeFormat.SCOPE_TASK_ATTEMPT_NUM, "0"); testVariable(variables, ScopeFormat.SCOPE_OPERATOR_ID, oid.toString()); testVariable(variables, ScopeFormat.SCOPE_OPERATOR_NAME, "myOpName"); - - registry.shutdown(); } private static void testVariable(Map variables, String key, String expectedValue) { @@ -156,7 +160,6 @@ public void testCreateQueryServiceMetricInfo() { JobVertexID vid = new JobVertexID(); AbstractID eid = new AbstractID(); OperatorID oid = new OperatorID(); - MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); TaskManagerMetricGroup tm = new TaskManagerMetricGroup(registry, "host", "id"); TaskManagerJobMetricGroup job = new TaskManagerJobMetricGroup(registry, tm, jid, "jobname"); TaskMetricGroup task = new TaskMetricGroup(registry, job, vid, eid, "taskName", 4, 5); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskManagerGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskManagerGroupTest.java index 3272f73765a2b..addbcbde66a73 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskManagerGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskManagerGroupTest.java @@ -30,6 +30,8 @@ import org.apache.flink.util.AbstractID; import org.apache.flink.util.TestLogger; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.IOException; @@ -44,14 +46,26 @@ */ public class TaskManagerGroupTest extends TestLogger { + private MetricRegistryImpl registry; + + @Before + public void setup() { + registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + } + + @After + public void teardown() throws Exception { + if (registry != null) { + registry.shutdown().get(); + } + } + // ------------------------------------------------------------------------ // adding and removing jobs // ------------------------------------------------------------------------ @Test public void addAndRemoveJobs() throws IOException { - MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); - final TaskManagerMetricGroup group = new TaskManagerMetricGroup( registry, "localhost", new AbstractID().toString()); @@ -106,13 +120,10 @@ public void addAndRemoveJobs() throws IOException { assertTrue(tmGroup13.parent().isClosed()); assertEquals(0, group.numRegisteredJobMetricGroups()); - - registry.shutdown(); } @Test public void testCloseClosesAll() throws IOException { - MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); final TaskManagerMetricGroup group = new TaskManagerMetricGroup( registry, "localhost", new AbstractID().toString()); @@ -142,8 +153,6 @@ public void testCloseClosesAll() throws IOException { assertTrue(tmGroup11.isClosed()); assertTrue(tmGroup12.isClosed()); assertTrue(tmGroup21.isClosed()); - - registry.shutdown(); } // ------------------------------------------------------------------------ @@ -152,16 +161,14 @@ public void testCloseClosesAll() throws IOException { @Test public void testGenerateScopeDefault() { - MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); TaskManagerMetricGroup group = new TaskManagerMetricGroup(registry, "localhost", "id"); assertArrayEquals(new String[]{"localhost", "taskmanager", "id"}, group.getScopeComponents()); assertEquals("localhost.taskmanager.id.name", group.getMetricIdentifier("name")); - registry.shutdown(); } @Test - public void testGenerateScopeCustom() { + public void testGenerateScopeCustom() throws Exception { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_TM, "constant..foo."); MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(cfg)); @@ -169,12 +176,11 @@ public void testGenerateScopeCustom() { assertArrayEquals(new String[]{"constant", "host", "foo", "host"}, group.getScopeComponents()); assertEquals("constant.host.foo.host.name", group.getMetricIdentifier("name")); - registry.shutdown(); + registry.shutdown().get(); } @Test public void testCreateQueryServiceMetricInfo() { - MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); TaskManagerMetricGroup tm = new TaskManagerMetricGroup(registry, "host", "id"); QueryScopeInfo.TaskManagerQueryScopeInfo info = tm.createQueryServiceMetricInfo(new DummyCharacterFilter()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskManagerJobGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskManagerJobGroupTest.java index b6be31c2dbfb3..52ee578ad5bfe 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskManagerJobGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskManagerJobGroupTest.java @@ -27,6 +27,8 @@ import org.apache.flink.runtime.metrics.util.DummyCharacterFilter; import org.apache.flink.util.TestLogger; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertArrayEquals; @@ -37,10 +39,22 @@ */ public class TaskManagerJobGroupTest extends TestLogger { + private MetricRegistryImpl registry; + + @Before + public void setup() { + registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + } + + @After + public void teardown() throws Exception { + if (registry != null) { + registry.shutdown().get(); + } + } + @Test public void testGenerateScopeDefault() { - MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); - TaskManagerMetricGroup tmGroup = new TaskManagerMetricGroup(registry, "theHostName", "test-tm-id"); JobMetricGroup jmGroup = new TaskManagerJobMetricGroup(registry, tmGroup, new JobID(), "myJobName"); @@ -51,11 +65,10 @@ public void testGenerateScopeDefault() { assertEquals( "theHostName.taskmanager.test-tm-id.myJobName.name", jmGroup.getMetricIdentifier("name")); - registry.shutdown(); } @Test - public void testGenerateScopeCustom() { + public void testGenerateScopeCustom() throws Exception { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_TM, "abc"); cfg.setString(MetricOptions.SCOPE_NAMING_TM_JOB, "some-constant."); @@ -73,11 +86,11 @@ public void testGenerateScopeCustom() { assertEquals( "some-constant.myJobName.name", jmGroup.getMetricIdentifier("name")); - registry.shutdown(); + registry.shutdown().get(); } @Test - public void testGenerateScopeCustomWildcard() { + public void testGenerateScopeCustomWildcard() throws Exception { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_TM, "peter."); cfg.setString(MetricOptions.SCOPE_NAMING_TM_JOB, "*.some-constant."); @@ -95,13 +108,12 @@ public void testGenerateScopeCustomWildcard() { assertEquals( "peter.test-tm-id.some-constant." + jid + ".name", jmGroup.getMetricIdentifier("name")); - registry.shutdown(); + registry.shutdown().get(); } @Test public void testCreateQueryServiceMetricInfo() { JobID jid = new JobID(); - MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); TaskManagerMetricGroup tm = new TaskManagerMetricGroup(registry, "host", "id"); TaskManagerJobMetricGroup job = new TaskManagerJobMetricGroup(registry, tm, jid, "jobname"); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskMetricGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskMetricGroupTest.java index 47ee1a91db113..d9e6158ccc869 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskMetricGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskMetricGroupTest.java @@ -31,7 +31,9 @@ import org.apache.flink.util.AbstractID; import org.apache.flink.util.TestLogger; +import org.junit.After; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertArrayEquals; @@ -43,13 +45,26 @@ */ public class TaskMetricGroupTest extends TestLogger { + private MetricRegistryImpl registry; + + @Before + public void setup() { + registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + } + + @After + public void teardown() throws Exception { + if (registry != null) { + registry.shutdown().get(); + } + } + // ------------------------------------------------------------------------ // scope tests // ----------------------------------------------------------------------- @Test public void testGenerateScopeDefault() { - MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); JobVertexID vertexId = new JobVertexID(); AbstractID executionId = new AbstractID(); @@ -64,11 +79,10 @@ public void testGenerateScopeDefault() { assertEquals( "theHostName.taskmanager.test-tm-id.myJobName.aTaskName.13.name", taskGroup.getMetricIdentifier("name")); - registry.shutdown(); } @Test - public void testGenerateScopeCustom() { + public void testGenerateScopeCustom() throws Exception { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_TM, "abc"); cfg.setString(MetricOptions.SCOPE_NAMING_TM_JOB, "def"); @@ -91,11 +105,11 @@ public void testGenerateScopeCustom() { assertEquals( String.format("test-tm-id.%s.%s.%s.name", jid, vertexId, executionId), taskGroup.getMetricIdentifier("name")); - registry.shutdown(); + registry.shutdown().get(); } @Test - public void testGenerateScopeWilcard() { + public void testGenerateScopeWilcard() throws Exception { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_TASK, "*.."); MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(cfg)); @@ -115,7 +129,7 @@ public void testGenerateScopeWilcard() { assertEquals( "theHostName.taskmanager.test-tm-id.myJobName." + executionId + ".13.name", taskGroup.getMetricIdentifier("name")); - registry.shutdown(); + registry.shutdown().get(); } @Test @@ -123,7 +137,6 @@ public void testCreateQueryServiceMetricInfo() { JobID jid = new JobID(); JobVertexID vid = new JobVertexID(); AbstractID eid = new AbstractID(); - MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); TaskManagerMetricGroup tm = new TaskManagerMetricGroup(registry, "host", "id"); TaskManagerJobMetricGroup job = new TaskManagerJobMetricGroup(registry, tm, jid, "jobname"); TaskMetricGroup task = new TaskMetricGroup(registry, job, vid, eid, "taskName", 4, 5); @@ -136,7 +149,7 @@ public void testCreateQueryServiceMetricInfo() { } @Test - public void testTaskMetricGroupCleanup() { + public void testTaskMetricGroupCleanup() throws Exception { CountingMetricRegistry registry = new CountingMetricRegistry(new Configuration()); TaskManagerMetricGroup taskManagerMetricGroup = new TaskManagerMetricGroup(registry, "localhost", "0"); TaskManagerJobMetricGroup taskManagerJobMetricGroup = new TaskManagerJobMetricGroup(registry, taskManagerMetricGroup, new JobID(), "job"); @@ -150,11 +163,11 @@ public void testTaskMetricGroupCleanup() { // now all registered metrics should have been unregistered assertEquals(0, registry.getNumberRegisteredMetrics()); - registry.shutdown(); + registry.shutdown().get(); } @Test - public void testOperatorNameTruncation() { + public void testOperatorNameTruncation() throws Exception { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_OPERATOR, ScopeFormat.SCOPE_OPERATOR_NAME); MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(cfg)); @@ -168,6 +181,7 @@ public void testOperatorNameTruncation() { String storedName = operatorMetricGroup.getScopeComponents()[0]; Assert.assertEquals(TaskMetricGroup.METRICS_OPERATOR_NAME_MAX_LENGTH, storedName.length()); Assert.assertEquals(originalName.substring(0, TaskMetricGroup.METRICS_OPERATOR_NAME_MAX_LENGTH), storedName); + registry.shutdown().get(); } private static class CountingMetricRegistry extends MetricRegistryImpl { diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java index 9d1af35c38b3e..ed1aad35d7494 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java @@ -457,7 +457,7 @@ protected int runApplicationMaster(Configuration config) { if (metricRegistry != null) { try { - metricRegistry.shutdown(); + metricRegistry.shutdown().get(); } catch (Throwable t) { LOG.error("Could not properly shut down the metric registry.", t); } From 65081ac72d18f610db210c5fe030805141a9b2e5 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 15 Feb 2018 18:07:49 +0100 Subject: [PATCH 08/10] [FLINK-8666] [test] Use testDispatcherConfig in MiniCluster Using the AkkaUtils#testDispatcherConfig reduces the number of started threads. This effectively decreases the resource foot print of the MiniCluster. This closes #5499. --- .../flink/runtime/minicluster/MiniCluster.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index c98387d25e466..01be01dca61a8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -65,6 +65,7 @@ import org.apache.flink.util.ExceptionUtils; import akka.actor.ActorSystem; +import com.typesafe.config.Config; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -634,13 +635,18 @@ protected RpcService createRpcService( boolean remoteEnabled, String bindAddress) { - ActorSystem actorSystem; + final Config akkaConfig; + if (remoteEnabled) { - actorSystem = AkkaUtils.createActorSystem(configuration, bindAddress, 0); + akkaConfig = AkkaUtils.getAkkaConfig(configuration, bindAddress, 0); } else { - actorSystem = AkkaUtils.createLocalActorSystem(configuration); + akkaConfig = AkkaUtils.getAkkaConfig(configuration); } + final Config effectiveAkkaConfig = AkkaUtils.testDispatcherConfig().withFallback(akkaConfig); + + final ActorSystem actorSystem = AkkaUtils.createActorSystem(effectiveAkkaConfig); + return new AkkaRpcService(actorSystem, askTimeout); } From fba655af1a4a543f01eecf222826a0821e6e1c17 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 16 Feb 2018 10:08:25 +0100 Subject: [PATCH 09/10] [FLINK-8677] [flip6] Make ClusterEntrypoint shut down non-blocking Makes the ClusterEntrypoint shut down method non-blocking. This also removes the need to run the Dispatcher#terminationFuture callback in the common Fork-Join pool. This closes #5512. --- .../entrypoint/MesosJobClusterEntrypoint.java | 33 ++- .../MesosSessionClusterEntrypoint.java | 34 ++- .../runtime/entrypoint/ClusterEntrypoint.java | 202 +++++++++--------- 3 files changed, 127 insertions(+), 142 deletions(-) diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java index b0054bb52e598..02f561a860f87 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.clusterframework.ContainerSpecification; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.entrypoint.JobClusterEntrypoint; import org.apache.flink.runtime.heartbeat.HeartbeatServices; @@ -43,7 +44,6 @@ import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.util.JvmShutdownSafeguard; import org.apache.flink.runtime.util.SignalHandler; -import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; @@ -59,6 +59,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.ObjectInputStream; +import java.util.concurrent.CompletableFuture; /** * Entry point for Mesos per-job clusters. @@ -167,26 +168,16 @@ protected JobGraph retrieveJobGraph(Configuration configuration) throws FlinkExc } @Override - protected void stopClusterServices(boolean cleanupHaData) throws FlinkException { - Throwable exception = null; - - try { - super.stopClusterServices(cleanupHaData); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - - if (mesosServices != null) { - try { - mesosServices.close(cleanupHaData); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - } - - if (exception != null) { - throw new FlinkException("Could not properly shut down the Mesos job cluster entry point.", exception); - } + protected CompletableFuture stopClusterServices(boolean cleanupHaData) { + final CompletableFuture serviceShutDownFuture = super.stopClusterServices(cleanupHaData); + + return FutureUtils.runAfterwards( + serviceShutDownFuture, + () -> { + if (mesosServices != null) { + mesosServices.close(cleanupHaData); + } + }); } public static void main(String[] args) { diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java index 306d5b2b2a5aa..5dea936669d90 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.clusterframework.ContainerSpecification; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.entrypoint.SessionClusterEntrypoint; import org.apache.flink.runtime.heartbeat.HeartbeatServices; @@ -42,7 +43,6 @@ import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.util.JvmShutdownSafeguard; import org.apache.flink.runtime.util.SignalHandler; -import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.apache.commons.cli.CommandLine; @@ -52,6 +52,8 @@ import javax.annotation.Nullable; +import java.util.concurrent.CompletableFuture; + /** * Entry point for Mesos session clusters. */ @@ -141,26 +143,16 @@ protected ResourceManager createResourceManager( } @Override - protected void stopClusterServices(boolean cleanupHaData) throws FlinkException { - Throwable exception = null; - - try { - super.stopClusterServices(cleanupHaData); - } catch (Throwable t) { - exception = t; - } - - if (mesosServices != null) { - try { - mesosServices.close(cleanupHaData); - } catch (Throwable t) { - exception = t; - } - } - - if (exception != null) { - throw new FlinkException("Could not properly shut down the Mesos session cluster entry point.", exception); - } + protected CompletableFuture stopClusterServices(boolean cleanupHaData) { + final CompletableFuture serviceShutDownFuture = super.stopClusterServices(cleanupHaData); + + return FutureUtils.runAfterwards( + serviceShutDownFuture, + () -> { + if (mesosServices != null) { + mesosServices.close(cleanupHaData); + } + }); } public static void main(String[] args) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index 8e4f93615aa28..ba286703412c7 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -34,6 +34,7 @@ import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.dispatcher.ArchivedExecutionGraphStore; import org.apache.flink.runtime.dispatcher.Dispatcher; @@ -63,7 +64,6 @@ import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaQueryServiceRetriever; import org.apache.flink.runtime.webmonitor.retriever.impl.RpcGatewayRetriever; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import akka.actor.ActorSystem; @@ -75,6 +75,8 @@ import java.io.IOException; import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Collection; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; @@ -104,10 +106,12 @@ public abstract class ClusterEntrypoint implements FatalErrorHandler { private final Configuration configuration; - private final CompletableFuture terminationFuture; + private final CompletableFuture terminationFuture; private final AtomicBoolean isTerminating = new AtomicBoolean(false); + private final AtomicBoolean isShutDown = new AtomicBoolean(false); + @GuardedBy("lock") private MetricRegistryImpl metricRegistry; @@ -144,12 +148,15 @@ public abstract class ClusterEntrypoint implements FatalErrorHandler { @GuardedBy("lock") private TransientBlobCache transientBlobCache; + @GuardedBy("lock") + private ClusterInformation clusterInformation; + protected ClusterEntrypoint(Configuration configuration) { this.configuration = Preconditions.checkNotNull(configuration); this.terminationFuture = new CompletableFuture<>(); } - public CompletableFuture getTerminationFuture() { + public CompletableFuture getTerminationFuture() { return terminationFuture; } @@ -211,8 +218,7 @@ protected void runCluster(Configuration configuration) throws Exception { heartbeatServices, metricRegistry); - // TODO: Make shutDownAndTerminate non blocking to not use the global executor - dispatcher.getTerminationFuture().whenCompleteAsync( + dispatcher.getTerminationFuture().whenComplete( (Void value, Throwable throwable) -> { if (throwable != null) { LOG.info("Could not properly terminate the Dispatcher.", throwable); @@ -250,6 +256,18 @@ protected void initializeServices(Configuration configuration) throws Exception // start the MetricQueryService final ActorSystem actorSystem = ((AkkaRpcService) commonRpcService).getActorSystem(); metricRegistry.startQueryService(actorSystem, null); + + archivedExecutionGraphStore = createSerializableExecutionGraphStore(configuration, commonRpcService.getScheduledExecutor()); + + clusterInformation = new ClusterInformation( + commonRpcService.getAddress(), + blobServer.getPort()); + + transientBlobCache = new TransientBlobCache( + configuration, + new InetSocketAddress( + clusterInformation.getBlobServerHostname(), + clusterInformation.getBlobServerPort())); } } @@ -261,22 +279,10 @@ protected void startClusterComponents( HeartbeatServices heartbeatServices, MetricRegistry metricRegistry) throws Exception { synchronized (lock) { - archivedExecutionGraphStore = createSerializableExecutionGraphStore(configuration, rpcService.getScheduledExecutor()); - dispatcherLeaderRetrievalService = highAvailabilityServices.getDispatcherLeaderRetriever(); resourceManagerRetrievalService = highAvailabilityServices.getResourceManagerLeaderRetriever(); - final ClusterInformation clusterInformation = new ClusterInformation( - rpcService.getAddress(), - blobServer.getPort()); - - transientBlobCache = new TransientBlobCache( - configuration, - new InetSocketAddress( - clusterInformation.getBlobServerHostname(), - clusterInformation.getBlobServerPort())); - LeaderGatewayRetriever dispatcherGatewayRetriever = new RpcGatewayRetriever<>( rpcService, DispatcherGateway.class, @@ -376,44 +382,11 @@ protected MetricRegistryImpl createMetricRegistry(Configuration configuration) { return new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(configuration)); } - private void shutDown(boolean cleanupHaData) throws FlinkException { - LOG.info("Stopping {}.", getClass().getSimpleName()); - - Throwable exception = null; - + protected CompletableFuture stopClusterServices(boolean cleanupHaData) { synchronized (lock) { + Throwable exception = null; - try { - stopClusterComponents(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - - try { - stopClusterServices(cleanupHaData); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - - terminationFuture.complete(true); - } - - if (exception != null) { - throw new FlinkException("Could not properly shut down the cluster entrypoint.", exception); - } - } - - protected void stopClusterServices(boolean cleanupHaData) throws FlinkException { - Throwable exception = null; - - synchronized (lock) { - if (metricRegistry != null) { - try { - metricRegistry.shutdown(); - } catch (Throwable t) { - exception = t; - } - } + final Collection> terminationFutures = new ArrayList<>(3); if (blobServer != null) { try { @@ -435,79 +408,80 @@ protected void stopClusterServices(boolean cleanupHaData) throws FlinkException } } - if (commonRpcService != null) { + if (archivedExecutionGraphStore != null) { try { - commonRpcService.stopService().get(); + archivedExecutionGraphStore.close(); } catch (Throwable t) { exception = ExceptionUtils.firstOrSuppressed(t, exception); } } - } - if (exception != null) { - throw new FlinkException("Could not properly shut down the cluster services.", exception); + if (transientBlobCache != null) { + try { + transientBlobCache.close(); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + } + + if (metricRegistry != null) { + terminationFutures.add(metricRegistry.shutdown()); + } + + if (commonRpcService != null) { + terminationFutures.add(commonRpcService.stopService()); + } + + if (exception != null) { + terminationFutures.add(FutureUtils.completedExceptionally(exception)); + } + + return FutureUtils.completeAll(terminationFutures); } } - protected void stopClusterComponents() throws Exception { + protected CompletableFuture stopClusterComponents() { synchronized (lock) { - Throwable exception = null; - if (webMonitorEndpoint != null) { - webMonitorEndpoint.shutDownAsync().get(); - } + Exception exception = null; + + final Collection> terminationFutures = new ArrayList<>(4); if (dispatcherLeaderRetrievalService != null) { try { dispatcherLeaderRetrievalService.stop(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - } - - if (dispatcher != null) { - try { - dispatcher.shutDown(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); + } catch (Exception e) { + exception = ExceptionUtils.firstOrSuppressed(e, exception); } } if (resourceManagerRetrievalService != null) { try { resourceManagerRetrievalService.stop(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); + } catch (Exception e) { + exception = ExceptionUtils.firstOrSuppressed(e, exception); } } - if (resourceManager != null) { - try { - resourceManager.shutDown(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } + if (webMonitorEndpoint != null) { + terminationFutures.add(webMonitorEndpoint.shutDownAsync()); } - if (archivedExecutionGraphStore != null) { - try { - archivedExecutionGraphStore.close(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } + if (dispatcher != null) { + dispatcher.shutDown(); + terminationFutures.add(dispatcher.getTerminationFuture()); } - if (transientBlobCache != null) { - try { - transientBlobCache.close(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } + if (resourceManager != null) { + resourceManager.shutDown(); + terminationFutures.add(resourceManager.getTerminationFuture()); } if (exception != null) { - throw new FlinkException("Could not properly shut down the session cluster entry point.", exception); + terminationFutures.add(FutureUtils.completedExceptionally(exception)); } + + return FutureUtils.completeAll(terminationFutures); } } @@ -522,6 +496,33 @@ public void onFatalError(Throwable exception) { // Internal methods // -------------------------------------------------- + private CompletableFuture shutDownAsync(boolean cleanupHaData) { + if (isShutDown.compareAndSet(false, true)) { + LOG.info("Stopping {}.", getClass().getSimpleName()); + + final CompletableFuture componentShutdownFuture = stopClusterComponents(); + + componentShutdownFuture.whenComplete( + (Void ignored1, Throwable componentThrowable) -> { + final CompletableFuture serviceShutdownFuture = stopClusterServices(cleanupHaData); + + serviceShutdownFuture.whenComplete( + (Void ignored2, Throwable serviceThrowable) -> { + if (serviceThrowable != null) { + terminationFuture.completeExceptionally( + ExceptionUtils.firstOrSuppressed(serviceThrowable, componentThrowable)); + } else if (componentThrowable != null) { + terminationFuture.completeExceptionally(componentThrowable); + } else { + terminationFuture.complete(null); + } + }); + }); + } + + return terminationFuture; + } + private void shutDownAndTerminate( int returnCode, ApplicationStatus applicationStatus, @@ -533,13 +534,14 @@ private void shutDownAndTerminate( applicationStatus); if (isTerminating.compareAndSet(false, true)) { - try { - shutDown(cleanupHaData); - } catch (Throwable t) { - LOG.info("Could not properly shut down cluster entrypoint.", t); - } + shutDownAsync(cleanupHaData).whenComplete( + (Void ignored, Throwable t) -> { + if (t != null) { + LOG.info("Could not properly shut down cluster entrypoint.", t); + } - System.exit(returnCode); + System.exit(returnCode); + }); } else { LOG.debug("Concurrent termination call detected. Ignoring termination call with return code {} and application status {}.", returnCode, From 16ec3d7ea12c520c5c86f0721553355cc938c2ae Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 23 Feb 2018 18:13:31 +0100 Subject: [PATCH 10/10] [hotfix] Workaround for shut down deadlock of Netty < 4.0.33 --- .../runtime/rest/RestServerEndpoint.java | 47 ++++++++++++++++--- 1 file changed, 41 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java index bade160f35c35..f131ec186cbcf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.rest.handler.PipelineErrorHandler; import org.apache.flink.runtime.rest.handler.RestHandlerSpecification; import org.apache.flink.runtime.rest.handler.RouterHandler; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FileUtils; import org.apache.flink.util.Preconditions; @@ -59,7 +60,9 @@ import java.util.Comparator; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; /** * An abstract class for netty-based REST server endpoints. @@ -287,11 +290,14 @@ protected CompletableFuture shutDownInternal() { }); serverChannel = null; } - CompletableFuture groupFuture = new CompletableFuture<>(); - CompletableFuture childGroupFuture = new CompletableFuture<>(); - final Time gracePeriod = Time.seconds(10L); + + final CompletableFuture channelTerminationFuture = new CompletableFuture<>(); channelFuture.thenRun(() -> { + CompletableFuture groupFuture = new CompletableFuture<>(); + CompletableFuture childGroupFuture = new CompletableFuture<>(); + final Time gracePeriod = Time.seconds(10L); + if (bootstrap != null) { if (bootstrap.group() != null) { bootstrap.group().shutdownGracefully(0L, gracePeriod.toMilliseconds(), TimeUnit.MILLISECONDS) @@ -302,7 +308,10 @@ protected CompletableFuture shutDownInternal() { groupFuture.completeExceptionally(finished.cause()); } }); + } else { + groupFuture.complete(null); } + if (bootstrap.childGroup() != null) { bootstrap.childGroup().shutdownGracefully(0L, gracePeriod.toMilliseconds(), TimeUnit.MILLISECONDS) .addListener(finished -> { @@ -312,17 +321,43 @@ protected CompletableFuture shutDownInternal() { childGroupFuture.completeExceptionally(finished.cause()); } }); + } else { + childGroupFuture.complete(null); } + bootstrap = null; } else { // complete the group futures since there is nothing to stop groupFuture.complete(null); childGroupFuture.complete(null); } - }); - final CompletableFuture channelTerminationFuture = FutureUtils.completeAll( - Arrays.asList(groupFuture, childGroupFuture)); + CompletableFuture combinedFuture = FutureUtils.completeAll(Arrays.asList(groupFuture, childGroupFuture)); + + // TODO: Temporary fix to circumvent shutdown bug in Netty < 4.0.33 + // See: https://github.com/netty/netty/issues/4357 + FutureUtils.orTimeout(combinedFuture, gracePeriod.toMilliseconds(), TimeUnit.MILLISECONDS); + + combinedFuture + .exceptionally( + (Throwable throwable) -> { + if (throwable instanceof TimeoutException) { + // We ignore timeout exceptions because they indicate that Netty's shut down deadlocked + log.info("Could not properly shut down Netty. Continue shut down of RestServerEndpoint."); + return null; + } else { + throw new CompletionException(ExceptionUtils.stripCompletionException(throwable)); + } + }) + .whenComplete( + (Void ignored, Throwable throwable) -> { + if (throwable != null) { + channelTerminationFuture.completeExceptionally(throwable); + } else { + channelTerminationFuture.complete(null); + } + }); + }); return FutureUtils.runAfterwards( channelTerminationFuture,