From 0e497367d8666a580d033848977b0639b037e554 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 1 Aug 2017 11:33:48 +0200 Subject: [PATCH 1/3] [FLINK-7334] [futures] Replace Flink's futures with Java 8's CompletableFuture in RpcEndpoint, RpcGateways and RpcService Remove Futures from RpcGateways Remove Future usage --- .../MesosResourceManagerTest.java | 6 +- .../handlers/TaskManagerLogHandler.java | 4 +- .../handlers/TaskManagerLogHandlerTest.java | 5 +- .../runtime/dispatcher/DispatcherGateway.java | 6 +- .../runtime/executiongraph/Execution.java | 25 ++-- .../flink/runtime/instance/SlotPool.java | 17 +-- .../runtime/instance/SlotPoolGateway.java | 8 +- .../slots/ActorTaskManagerGateway.java | 38 ++--- .../jobmanager/slots/TaskManagerGateway.java | 19 +-- .../flink/runtime/jobmaster/JobMaster.java | 93 ++++++------ .../runtime/jobmaster/JobMasterGateway.java | 18 +-- .../jobmaster/RpcTaskManagerGateway.java | 25 ++-- .../registration/RetryingRegistration.java | 4 +- .../resourcemanager/JobLeaderIdService.java | 16 +- .../resourcemanager/ResourceManager.java | 29 ++-- .../ResourceManagerGateway.java | 10 +- .../slotmanager/SlotManager.java | 16 +- .../runtime/rpc/MainThreadExecutable.java | 4 +- .../apache/flink/runtime/rpc/RpcEndpoint.java | 6 +- .../apache/flink/runtime/rpc/RpcService.java | 10 +- .../apache/flink/runtime/rpc/SelfGateway.java | 4 +- .../rpc/akka/AkkaInvocationHandler.java | 30 ++-- .../flink/runtime/rpc/akka/AkkaRpcActor.java | 36 ++--- .../runtime/rpc/akka/AkkaRpcService.java | 28 ++-- .../taskexecutor/JobLeaderService.java | 5 +- .../runtime/taskexecutor/TaskExecutor.java | 20 +-- .../taskexecutor/TaskExecutorGateway.java | 16 +- ...skExecutorToResourceManagerConnection.java | 3 +- .../taskexecutor/TaskManagerRunner.java | 4 +- .../rpc/RpcInputSplitProvider.java | 4 +- .../rpc/RpcPartitionStateChecker.java | 3 +- .../RpcResultPartitionConsumableNotifier.java | 22 ++- .../flink/runtime/jobmanager/JobManager.scala | 17 ++- .../runtime/akka/QuarantineMonitorTest.java | 16 +- .../CheckpointCoordinatorMasterHooksTest.java | 5 +- .../clusterframework/ResourceManagerTest.java | 6 +- .../runtime/dispatcher/DispatcherTest.java | 5 +- .../ExecutionGraphMetricsTest.java | 3 +- .../ExecutionGraphSchedulingTest.java | 3 +- .../ExecutionGraphStopTest.java | 7 +- .../utils/NotCancelAckingTaskGateway.java | 9 +- .../utils/SimpleAckingTaskManagerGateway.java | 36 ++--- .../runtime/instance/SlotPoolRpcTest.java | 4 +- .../flink/runtime/instance/SlotPoolTest.java | 20 +-- .../runtime/jobmaster/JobMasterTest.java | 5 +- .../RetryingRegistrationTest.java | 13 +- .../registration/TestRegistrationGateway.java | 7 +- .../JobLeaderIdServiceTest.java | 8 +- .../ResourceManagerJobMasterTest.java | 22 +-- .../ResourceManagerTaskExecutorTest.java | 10 +- .../slotmanager/SlotManagerTest.java | 138 +++++++----------- .../slotmanager/SlotProtocolTest.java | 6 +- .../flink/runtime/rpc/AsyncCallsTest.java | 12 +- .../runtime/rpc/RpcCompletenessTest.java | 6 +- .../flink/runtime/rpc/RpcConnectionTest.java | 4 +- .../flink/runtime/rpc/TestingGatewayBase.java | 9 +- .../flink/runtime/rpc/TestingRpcService.java | 13 +- .../runtime/rpc/TestingSerialRpcService.java | 37 +++-- .../runtime/rpc/akka/AkkaRpcActorTest.java | 46 +++--- .../runtime/rpc/akka/AkkaRpcServiceTest.java | 25 +--- .../rpc/akka/MessageSerializationTest.java | 8 +- .../taskexecutor/TaskExecutorITCase.java | 9 +- .../taskexecutor/TaskExecutorTest.java | 39 +++-- .../runtime/taskmanager/TaskManagerTest.java | 11 +- .../queue/OrderedStreamElementQueueTest.java | 20 +-- .../async/queue/StreamElementQueueTest.java | 52 ++++--- .../UnorderedStreamElementQueueTest.java | 50 ++++--- .../tasks/StreamTaskTerminationTest.java | 14 +- 68 files changed, 559 insertions(+), 670 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 6e6a59c3e24f4..e63b4abe00125 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 @@ -40,7 +40,6 @@ 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.Future; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices; @@ -89,6 +88,7 @@ import java.util.HashSet; import java.util.Set; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; @@ -414,7 +414,7 @@ public void startResourceManager() throws Exception { * Register a job master with the RM. */ public void registerJobMaster(MockJobMaster jobMaster) throws Exception { - Future registration = resourceManager.registerJobManager( + CompletableFuture registration = resourceManager.registerJobManager( rmServices.rmLeaderSessionId, jobMaster.leaderSessionID, jobMaster.resourceID, jobMaster.address, jobMaster.jobID); assertTrue(registration.get() instanceof JobMasterRegistrationSuccess); } @@ -588,7 +588,7 @@ public void testWorkerStarted() throws Exception { assertThat(resourceManager.workersInLaunch, hasEntry(extractResourceID(task1), worker1launched)); // send registration message - Future successfulFuture = + CompletableFuture successfulFuture = resourceManager.registerTaskExecutor(rmServices.rmLeaderSessionId, task1Executor.address, task1Executor.resourceID, slotReport); RegistrationResponse response = successfulFuture.get(5, TimeUnit.SECONDS); assertTrue(response instanceof TaskExecutorRegistrationSuccess); diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java index 9cbb71d40ddff..ce29721879037 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java @@ -196,10 +196,10 @@ public BlobCache checkedApply(Object result) throws IOException { switch (fileMode) { case LOG: - return FutureUtils.toJava(taskManager.getTaskManagerGateway().requestTaskManagerLog(timeTimeout)); + return taskManager.getTaskManagerGateway().requestTaskManagerLog(timeTimeout); case STDOUT: default: - return FutureUtils.toJava(taskManager.getTaskManagerGateway().requestTaskManagerStdout(timeTimeout)); + return taskManager.getTaskManagerGateway().requestTaskManagerStdout(timeTimeout); } } ); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java index bfcaf88e12e6e..e1c3686cb0791 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java @@ -25,9 +25,7 @@ import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.concurrent.CompletableFuture; import org.apache.flink.runtime.concurrent.Executors; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.instance.InstanceID; @@ -50,6 +48,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; import scala.Option; @@ -109,7 +108,7 @@ public void testLogFetchingFailure() throws Exception { when(taskManager.getId()).thenReturn(tmID); when(taskManager.getTaskManagerID()).thenReturn(tmRID); when(taskManager.getTaskManagerGateway()).thenReturn(taskManagerGateway); - CompletableFuture future = new FlinkCompletableFuture<>(); + CompletableFuture future = new CompletableFuture<>(); future.completeExceptionally(new IOException("failure")); when(taskManagerGateway.requestTaskManagerLog(any(Time.class))).thenReturn(future); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherGateway.java index c730bc162f867..33b8a42a364d4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherGateway.java @@ -20,13 +20,13 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; import java.util.Collection; +import java.util.concurrent.CompletableFuture; /** * Gateway for the Dispatcher component. @@ -40,7 +40,7 @@ public interface DispatcherGateway extends RpcGateway { * @param timeout RPC timeout * @return A future acknowledge if the submission succeeded */ - Future submitJob( + CompletableFuture submitJob( JobGraph jobGraph, @RpcTimeout Time timeout); @@ -50,6 +50,6 @@ Future submitJob( * @param timeout RPC timeout * @return A future collection of currently submitted jobs */ - Future> listJobs( + CompletableFuture> listJobs( @RpcTimeout Time timeout); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index 5cb12ea9cf0dd..bd5bc7f5da6cc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -418,8 +418,7 @@ public void deployToSlot(final SimpleSlot slot) throws JobException { final TaskManagerGateway taskManagerGateway = slot.getTaskManagerGateway(); - final CompletableFuture submitResultFuture = FutureUtils.toJava( - taskManagerGateway.submitTask(deployment, timeout)); + final CompletableFuture submitResultFuture = taskManagerGateway.submitTask(deployment, timeout); submitResultFuture.whenCompleteAsync( (ack, failure) -> { @@ -454,7 +453,7 @@ public void stop() { final TaskManagerGateway taskManagerGateway = slot.getTaskManagerGateway(); CompletableFuture stopResultFuture = FutureUtils.retry( - () -> FutureUtils.toJava(taskManagerGateway.stopTask(attemptId, timeout)), + () -> taskManagerGateway.stopTask(attemptId, timeout), NUM_STOP_CALL_TRIES, executor); @@ -679,14 +678,13 @@ public CompletableFuture requestStackTraceSample( if (slot != null) { final TaskManagerGateway taskManagerGateway = slot.getTaskManagerGateway(); - return FutureUtils.toJava( - taskManagerGateway.requestStackTraceSample( - attemptId, - sampleId, - numSamples, - delayBetweenSamples, - maxStrackTraceDepth, - timeout)); + return taskManagerGateway.requestStackTraceSample( + attemptId, + sampleId, + numSamples, + delayBetweenSamples, + maxStrackTraceDepth, + timeout); } else { return FutureUtils.completedExceptionally(new Exception("The execution has no slot assigned.")); } @@ -1011,7 +1009,7 @@ private void sendCancelRpcCall() { final TaskManagerGateway taskManagerGateway = slot.getTaskManagerGateway(); CompletableFuture cancelResultFuture = FutureUtils.retry( - () -> FutureUtils.toJava(taskManagerGateway.cancelTask(attemptId, timeout)), + () -> taskManagerGateway.cancelTask(attemptId, timeout), NUM_CANCEL_CALL_TRIES, executor); @@ -1050,8 +1048,7 @@ private void sendUpdatePartitionInfoRpcCall( final TaskManagerGateway taskManagerGateway = slot.getTaskManagerGateway(); final TaskManagerLocation taskManagerLocation = slot.getTaskManagerLocation(); - CompletableFuture updatePartitionsResultFuture = FutureUtils.toJava( - taskManagerGateway.updatePartitions(attemptId, partitionInfos, timeout)); + CompletableFuture updatePartitionsResultFuture = taskManagerGateway.updatePartitions(attemptId, partitionInfos, timeout); updatePartitionsResultFuture.whenCompleteAsync( (ack, failure) -> { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java index 9a26779732887..508e54f0f0da7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java @@ -25,8 +25,6 @@ import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; -import org.apache.flink.runtime.concurrent.Future; -import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.jobmanager.scheduler.Locality; import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; @@ -263,12 +261,12 @@ public void disconnectResourceManager() { // ------------------------------------------------------------------------ @RpcMethod - public Future allocateSlot( + public CompletableFuture allocateSlot( ScheduledUnit task, ResourceProfile resources, Iterable locationPreferences) { - return FutureUtils.toFlinkFuture(internalAllocateSlot(task, resources, locationPreferences)); + return internalAllocateSlot(task, resources, locationPreferences); } @RpcMethod @@ -316,11 +314,10 @@ private void requestSlotFromResourceManager( pendingRequests.put(allocationID, new PendingRequest(allocationID, future, resources)); - CompletableFuture rmResponse = FutureUtils.toJava( - resourceManagerGateway.requestSlot( - jobManagerLeaderId, resourceManagerLeaderId, - new SlotRequest(jobId, allocationID, resources, jobManagerAddress), - resourceManagerRequestsTimeout)); + CompletableFuture rmResponse = resourceManagerGateway.requestSlot( + jobManagerLeaderId, resourceManagerLeaderId, + new SlotRequest(jobId, allocationID, resources, jobManagerAddress), + resourceManagerRequestsTimeout); CompletableFuture slotRequestProcessingFuture = rmResponse.thenAcceptAsync( (Acknowledge value) -> { @@ -984,7 +981,7 @@ public CompletableFuture allocateSlot(ScheduledUnit task, boolean al Iterable locationPreferences = task.getTaskToExecute().getVertex().getPreferredLocations(); - return FutureUtils.toJava(gateway.allocateSlot(task, ResourceProfile.UNKNOWN, locationPreferences, timeout)); + return gateway.allocateSlot(task, ResourceProfile.UNKNOWN, locationPreferences, timeout); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPoolGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPoolGateway.java index 42942cabeb872..43f407aca0252 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPoolGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPoolGateway.java @@ -23,7 +23,6 @@ import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; @@ -33,6 +32,7 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import java.util.UUID; +import java.util.concurrent.CompletableFuture; /** * The gateway for calls on the {@link SlotPool}. @@ -75,9 +75,9 @@ public interface SlotPoolGateway extends RpcGateway { void releaseTaskManager(ResourceID resourceID); - Future offerSlot(AllocatedSlot slot); + CompletableFuture offerSlot(AllocatedSlot slot); - Future> offerSlots(Iterable> offers); + CompletableFuture> offerSlots(Iterable> offers); void failAllocation(AllocationID allocationID, Exception cause); @@ -85,7 +85,7 @@ public interface SlotPoolGateway extends RpcGateway { // allocating and disposing slots // ------------------------------------------------------------------------ - Future allocateSlot( + CompletableFuture allocateSlot( ScheduledUnit task, ResourceProfile resources, Iterable locationPreferences, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/ActorTaskManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/ActorTaskManagerGateway.java index 2876ebe2a3a00..a773ce9a5a57a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/ActorTaskManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/ActorTaskManagerGateway.java @@ -24,8 +24,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.messages.StopCluster; -import org.apache.flink.runtime.concurrent.Future; -import org.apache.flink.runtime.concurrent.impl.FlinkFuture; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.PartitionInfo; @@ -41,6 +40,9 @@ import org.apache.flink.runtime.messages.checkpoint.NotifyCheckpointComplete; import org.apache.flink.runtime.messages.checkpoint.TriggerCheckpoint; import org.apache.flink.util.Preconditions; + +import java.util.concurrent.CompletableFuture; + import scala.concurrent.duration.FiniteDuration; import scala.reflect.ClassTag$; @@ -78,7 +80,7 @@ public void stopCluster(final ApplicationStatus applicationStatus, final String } @Override - public Future requestStackTrace(final Time timeout) { + public CompletableFuture requestStackTrace(final Time timeout) { Preconditions.checkNotNull(timeout); scala.concurrent.Future stackTraceFuture = actorGateway.ask( @@ -86,11 +88,11 @@ public Future requestStackTrace(final Time timeout) { new FiniteDuration(timeout.getSize(), timeout.getUnit())) .mapTo(ClassTag$.MODULE$.apply(StackTrace.class)); - return new FlinkFuture<>(stackTraceFuture); + return FutureUtils.toJava(stackTraceFuture); } @Override - public Future requestStackTraceSample( + public CompletableFuture requestStackTraceSample( ExecutionAttemptID executionAttemptID, int sampleId, int numSamples, @@ -113,11 +115,11 @@ public Future requestStackTraceSample( new FiniteDuration(timeout.getSize(), timeout.getUnit())) .mapTo(ClassTag$.MODULE$.apply(StackTraceSampleResponse.class)); - return new FlinkFuture<>(stackTraceSampleResponseFuture); + return FutureUtils.toJava(stackTraceSampleResponseFuture); } @Override - public Future submitTask(TaskDeploymentDescriptor tdd, Time timeout) { + public CompletableFuture submitTask(TaskDeploymentDescriptor tdd, Time timeout) { Preconditions.checkNotNull(tdd); Preconditions.checkNotNull(timeout); @@ -126,11 +128,11 @@ public Future submitTask(TaskDeploymentDescriptor tdd, Time timeout new FiniteDuration(timeout.getSize(), timeout.getUnit())) .mapTo(ClassTag$.MODULE$.apply(Acknowledge.class)); - return new FlinkFuture<>(submitResult); + return FutureUtils.toJava(submitResult); } @Override - public Future stopTask(ExecutionAttemptID executionAttemptID, Time timeout) { + public CompletableFuture stopTask(ExecutionAttemptID executionAttemptID, Time timeout) { Preconditions.checkNotNull(executionAttemptID); Preconditions.checkNotNull(timeout); @@ -139,11 +141,11 @@ public Future stopTask(ExecutionAttemptID executionAttemptID, Time new FiniteDuration(timeout.getSize(), timeout.getUnit())) .mapTo(ClassTag$.MODULE$.apply(Acknowledge.class)); - return new FlinkFuture<>(stopResult); + return FutureUtils.toJava(stopResult); } @Override - public Future cancelTask(ExecutionAttemptID executionAttemptID, Time timeout) { + public CompletableFuture cancelTask(ExecutionAttemptID executionAttemptID, Time timeout) { Preconditions.checkNotNull(executionAttemptID); Preconditions.checkNotNull(timeout); @@ -152,11 +154,11 @@ public Future cancelTask(ExecutionAttemptID executionAttemptID, Tim new FiniteDuration(timeout.getSize(), timeout.getUnit())) .mapTo(ClassTag$.MODULE$.apply(Acknowledge.class)); - return new FlinkFuture<>(cancelResult); + return FutureUtils.toJava(cancelResult); } @Override - public Future updatePartitions(ExecutionAttemptID executionAttemptID, Iterable partitionInfos, Time timeout) { + public CompletableFuture updatePartitions(ExecutionAttemptID executionAttemptID, Iterable partitionInfos, Time timeout) { Preconditions.checkNotNull(executionAttemptID); Preconditions.checkNotNull(partitionInfos); @@ -169,7 +171,7 @@ public Future updatePartitions(ExecutionAttemptID executionAttemptI new FiniteDuration(timeout.getSize(), timeout.getUnit())) .mapTo(ClassTag$.MODULE$.apply(Acknowledge.class)); - return new FlinkFuture<>(updatePartitionsResult); + return FutureUtils.toJava(updatePartitionsResult); } @Override @@ -207,16 +209,16 @@ public void triggerCheckpoint( } @Override - public Future requestTaskManagerLog(Time timeout) { + public CompletableFuture requestTaskManagerLog(Time timeout) { return requestTaskManagerLog((TaskManagerMessages.RequestTaskManagerLog) TaskManagerMessages.getRequestTaskManagerLog(), timeout); } @Override - public Future requestTaskManagerStdout(Time timeout) { + public CompletableFuture requestTaskManagerStdout(Time timeout) { return requestTaskManagerLog((TaskManagerMessages.RequestTaskManagerLog) TaskManagerMessages.getRequestTaskManagerStdout(), timeout); } - private Future requestTaskManagerLog(TaskManagerMessages.RequestTaskManagerLog request, Time timeout) { + private CompletableFuture requestTaskManagerLog(TaskManagerMessages.RequestTaskManagerLog request, Time timeout) { Preconditions.checkNotNull(request); Preconditions.checkNotNull(timeout); @@ -226,6 +228,6 @@ private Future requestTaskManagerLog(TaskManagerMessages.RequestTaskMan new FiniteDuration(timeout.getSize(), timeout.getUnit())) .mapTo(ClassTag$.MODULE$.apply(BlobKey.class)); - return new FlinkFuture<>(blobKeyFuture); + return FutureUtils.toJava(blobKeyFuture); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/TaskManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/TaskManagerGateway.java index 09f104f4e7c71..36cea86f72dfe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/TaskManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/TaskManagerGateway.java @@ -23,7 +23,6 @@ import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.clusterframework.ApplicationStatus; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.PartitionInfo; @@ -32,6 +31,8 @@ import org.apache.flink.runtime.messages.StackTrace; import org.apache.flink.runtime.messages.StackTraceSampleResponse; +import java.util.concurrent.CompletableFuture; + /** * Task manager gateway interface to communicate with the task manager. */ @@ -66,7 +67,7 @@ public interface TaskManagerGateway { * @param timeout for the stack trace request * @return Future for a stack trace */ - Future requestStackTrace(final Time timeout); + CompletableFuture requestStackTrace(final Time timeout); /** * Request a stack trace sample from the given task. @@ -79,7 +80,7 @@ public interface TaskManagerGateway { * @param timeout of the request * @return Future of stack trace sample response */ - Future requestStackTraceSample( + CompletableFuture requestStackTraceSample( final ExecutionAttemptID executionAttemptID, final int sampleId, final int numSamples, @@ -94,7 +95,7 @@ Future requestStackTraceSample( * @param timeout of the submit operation * @return Future acknowledge of the successful operation */ - Future submitTask( + CompletableFuture submitTask( TaskDeploymentDescriptor tdd, Time timeout); @@ -105,7 +106,7 @@ Future submitTask( * @param timeout of the submit operation * @return Future acknowledge if the task is successfully stopped */ - Future stopTask( + CompletableFuture stopTask( ExecutionAttemptID executionAttemptID, Time timeout); @@ -116,7 +117,7 @@ Future stopTask( * @param timeout of the submit operation * @return Future acknowledge if the task is successfully canceled */ - Future cancelTask( + CompletableFuture cancelTask( ExecutionAttemptID executionAttemptID, Time timeout); @@ -128,7 +129,7 @@ Future cancelTask( * @param timeout of the submit operation * @return Future acknowledge if the partitions have been successfully updated */ - Future updatePartitions( + CompletableFuture updatePartitions( ExecutionAttemptID executionAttemptID, Iterable partitionInfos, Time timeout); @@ -176,7 +177,7 @@ void triggerCheckpoint( * @param timeout for the request * @return Future blob key under which the task manager log has been stored */ - Future requestTaskManagerLog(final Time timeout); + CompletableFuture requestTaskManagerLog(final Time timeout); /** * Request the task manager stdout from the task manager. @@ -184,5 +185,5 @@ void triggerCheckpoint( * @param timeout for the request * @return Future blob key under which the task manager stdout file has been stored */ - Future requestTaskManagerStdout(final Time timeout); + CompletableFuture requestTaskManagerStdout(final Time timeout); } 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 7922baae77600..e2e117accee88 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 @@ -36,10 +36,6 @@ import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.concurrent.BiFunction; -import org.apache.flink.runtime.concurrent.Future; -import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; import org.apache.flink.runtime.executiongraph.Execution; @@ -105,8 +101,8 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.UUID; -import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; @@ -684,7 +680,7 @@ public ClassloadingProps requestClassloadingProps() throws Exception { } @RpcMethod - public Future> offerSlots( + public CompletableFuture> offerSlots( final ResourceID taskManagerId, final Iterable slots, final UUID leaderId) throws Exception { @@ -736,7 +732,7 @@ public void failSlot(final ResourceID taskManagerId, } @RpcMethod - public Future registerTaskManager( + public CompletableFuture registerTaskManager( final String taskManagerRpcAddress, final TaskManagerLocation taskManagerLocation, final UUID leaderId) throws Exception @@ -755,48 +751,42 @@ public Future registerTaskManager( if (registeredTaskManagers.containsKey(taskManagerId)) { final RegistrationResponse response = new JMTMRegistrationSuccess( resourceId, libraryCacheManager.getBlobServerPort()); - return FlinkCompletableFuture.completed(response); + return CompletableFuture.completedFuture(response); } else { - return getRpcService().execute(new Callable() { - @Override - public TaskExecutorGateway call() throws Exception { - return getRpcService().connect(taskManagerRpcAddress, TaskExecutorGateway.class) - .get(rpcTimeout.getSize(), rpcTimeout.getUnit()); - } - }).handleAsync(new BiFunction() { - @Override - public RegistrationResponse apply(final TaskExecutorGateway taskExecutorGateway, Throwable throwable) { - if (throwable != null) { - return new RegistrationResponse.Decline(throwable.getMessage()); - } - - if (!JobMaster.this.leaderSessionID.equals(leaderId)) { - log.warn("Discard registration from TaskExecutor {} at ({}) because the expected " + - "leader session ID {} did not equal the received leader session ID {}.", - taskManagerId, taskManagerRpcAddress, - JobMaster.this.leaderSessionID, leaderId); - return new RegistrationResponse.Decline("Invalid leader session id"); - } - - slotPoolGateway.registerTaskManager(taskManagerId); - registeredTaskManagers.put(taskManagerId, Tuple2.of(taskManagerLocation, taskExecutorGateway)); - - // monitor the task manager as heartbeat target - taskManagerHeartbeatManager.monitorTarget(taskManagerId, new HeartbeatTarget() { - @Override - public void receiveHeartbeat(ResourceID resourceID, Void payload) { - // the task manager will not request heartbeat, so this method will never be called currently + return getRpcService() + .connect(taskManagerRpcAddress, TaskExecutorGateway.class) + .handleAsync( + (TaskExecutorGateway taskExecutorGateway, Throwable throwable) -> { + if (throwable != null) { + return new RegistrationResponse.Decline(throwable.getMessage()); } - @Override - public void requestHeartbeat(ResourceID resourceID, Void payload) { - taskExecutorGateway.heartbeatFromJobManager(resourceID); + if (!Objects.equals(leaderSessionID, leaderId)) { + log.warn("Discard registration from TaskExecutor {} at ({}) because the expected " + + "leader session ID {} did not equal the received leader session ID {}.", + taskManagerId, taskManagerRpcAddress, leaderSessionID, leaderId); + return new RegistrationResponse.Decline("Invalid leader session id"); } - }); - return new JMTMRegistrationSuccess(resourceId, libraryCacheManager.getBlobServerPort()); - } - }, getMainThreadExecutor()); + slotPoolGateway.registerTaskManager(taskManagerId); + registeredTaskManagers.put(taskManagerId, Tuple2.of(taskManagerLocation, taskExecutorGateway)); + + // monitor the task manager as heartbeat target + taskManagerHeartbeatManager.monitorTarget(taskManagerId, new HeartbeatTarget() { + @Override + public void receiveHeartbeat(ResourceID resourceID, Void payload) { + // the task manager will not request heartbeat, so this method will never be called currently + } + + @Override + public void requestHeartbeat(ResourceID resourceID, Void payload) { + taskExecutorGateway.heartbeatFromJobManager(resourceID); + } + }); + + return new JMTMRegistrationSuccess(resourceId, libraryCacheManager.getBlobServerPort()); + }, + getMainThreadExecutor()); } } @@ -1051,14 +1041,13 @@ protected CompletableFuture invokeRegistration( { Time timeout = Time.milliseconds(timeoutMillis); - return FutureUtils.toJava( - gateway.registerJobManager( - leaderId, - jobManagerLeaderID, - jobManagerResourceID, - jobManagerRpcAddress, - jobID, - timeout)); + return gateway.registerJobManager( + leaderId, + jobManagerLeaderID, + jobManagerResourceID, + jobManagerRpcAddress, + jobID, + timeout); } }; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java index 5a271f90c383e..e3611a3c13c40 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java @@ -22,7 +22,6 @@ import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorGateway; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; @@ -45,6 +44,7 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import java.util.UUID; +import java.util.concurrent.CompletableFuture; /** * {@link JobMaster} rpc gateway interface @@ -68,7 +68,7 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway { * @param taskExecutionState New task execution state for a given task * @return Future flag of the task execution state update result */ - Future updateTaskExecutionState( + CompletableFuture updateTaskExecutionState( final UUID leaderSessionID, final TaskExecutionState taskExecutionState); @@ -81,7 +81,7 @@ Future updateTaskExecutionState( * @param executionAttempt The execution attempt id * @return The future of the input split. If there is no further input split, will return an empty object. */ - Future requestNextInputSplit( + CompletableFuture requestNextInputSplit( final UUID leaderSessionID, final JobVertexID vertexID, final ExecutionAttemptID executionAttempt); @@ -95,7 +95,7 @@ Future requestNextInputSplit( * @param partitionId The partition ID of the partition to request the state of. * @return The future of the partition state */ - Future requestPartitionState( + CompletableFuture requestPartitionState( final UUID leaderSessionID, final IntermediateDataSetID intermediateResultId, final ResultPartitionID partitionId); @@ -114,7 +114,7 @@ Future requestPartitionState( * @param timeout before the rpc call fails * @return Future acknowledge of the schedule or update operation */ - Future scheduleOrUpdateConsumers( + CompletableFuture scheduleOrUpdateConsumers( final UUID leaderSessionID, final ResultPartitionID partitionID, @RpcTimeout final Time timeout); @@ -146,7 +146,7 @@ void disconnectResourceManager( * @param registrationName Name under which the KvState has been registered. * @return Future of the requested {@link InternalKvState} location */ - Future lookupKvStateLocation(final String registrationName); + CompletableFuture lookupKvStateLocation(final String registrationName); /** * @param jobVertexId JobVertexID the KvState instance belongs to. @@ -175,7 +175,7 @@ void notifyKvStateUnregistered( /** * Request the classloading props of this job. */ - Future requestClassloadingProps(); + CompletableFuture requestClassloadingProps(); /** * Offer the given slots to the job manager. The response contains the set of accepted slots. @@ -186,7 +186,7 @@ void notifyKvStateUnregistered( * @param timeout for the rpc call * @return Future set of accepted slots. */ - Future> offerSlots( + CompletableFuture> offerSlots( final ResourceID taskManagerId, final Iterable slots, final UUID leaderId, @@ -214,7 +214,7 @@ void failSlot(final ResourceID taskManagerId, * @param timeout for the rpc call * @return Future registration response indicating whether the registration was successful or not */ - Future registerTaskManager( + CompletableFuture registerTaskManager( final String taskManagerRpcAddress, final TaskManagerLocation taskManagerLocation, final UUID leaderId, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java index 28fef27c7f64b..e93c907d36b6b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java @@ -22,7 +22,6 @@ import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.clusterframework.ApplicationStatus; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.PartitionInfo; @@ -35,6 +34,7 @@ import org.apache.flink.util.Preconditions; import java.util.UUID; +import java.util.concurrent.CompletableFuture; /** * Implementation of the {@link TaskManagerGateway} for Flink's RPC system @@ -68,47 +68,40 @@ public void stopCluster(ApplicationStatus applicationStatus, String message) { } @Override - public Future requestStackTrace(Time timeout) { + public CompletableFuture requestStackTrace(Time timeout) { // return taskExecutorGateway.requestStackTrace(timeout); throw new UnsupportedOperationException("Operation is not yet supported."); } @Override - public Future requestStackTraceSample( + public CompletableFuture requestStackTraceSample( ExecutionAttemptID executionAttemptID, int sampleId, int numSamples, Time delayBetweenSamples, int maxStackTraceDepth, Time timeout) { -// return taskExecutorGateway.requestStackTraceSample( -// executionAttemptID, -// sampleId, -// numSamples, -// delayBetweenSamples, -// maxStackTraceDepth, -// timeout); throw new UnsupportedOperationException("Operation is not yet supported."); } @Override - public Future submitTask(TaskDeploymentDescriptor tdd, Time timeout) { + public CompletableFuture submitTask(TaskDeploymentDescriptor tdd, Time timeout) { return taskExecutorGateway.submitTask(tdd, leaderId, timeout); } @Override - public Future stopTask(ExecutionAttemptID executionAttemptID, Time timeout) { + public CompletableFuture stopTask(ExecutionAttemptID executionAttemptID, Time timeout) { return taskExecutorGateway.stopTask(executionAttemptID, timeout); } @Override - public Future cancelTask(ExecutionAttemptID executionAttemptID, Time timeout) { + public CompletableFuture cancelTask(ExecutionAttemptID executionAttemptID, Time timeout) { return taskExecutorGateway.cancelTask(executionAttemptID, timeout); } @Override - public Future updatePartitions(ExecutionAttemptID executionAttemptID, Iterable partitionInfos, Time timeout) { + public CompletableFuture updatePartitions(ExecutionAttemptID executionAttemptID, Iterable partitionInfos, Time timeout) { return taskExecutorGateway.updatePartitions(executionAttemptID, partitionInfos, timeout); } @@ -130,13 +123,13 @@ public void triggerCheckpoint(ExecutionAttemptID executionAttemptID, JobID jobId } @Override - public Future requestTaskManagerLog(Time timeout) { + public CompletableFuture requestTaskManagerLog(Time timeout) { // return taskExecutorGateway.requestTaskManagerLog(timeout); throw new UnsupportedOperationException("Operation is not yet supported."); } @Override - public Future requestTaskManagerStdout(Time timeout) { + public CompletableFuture requestTaskManagerStdout(Time timeout) { // return taskExecutorGateway.requestTaskManagerStdout(timeout); throw new UnsupportedOperationException("Operation is not yet supported."); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java index 1034a8936a6cc..6a18ffd78b5ed 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.registration; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcService; @@ -176,8 +175,7 @@ protected abstract CompletableFuture invokeRegistration( public void startRegistration() { try { // trigger resolution of the resource manager address to a callable gateway - CompletableFuture resourceManagerFuture = FutureUtils.toJava( - rpcService.connect(targetAddress, targetType)); + CompletableFuture resourceManagerFuture = rpcService.connect(targetAddress, targetType); // upon success, start the registration attempts CompletableFuture resourceManagerAcceptFuture = resourceManagerFuture.thenAcceptAsync( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java index 8bffcd094386e..aaa72d9fc78ad 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java @@ -20,10 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; -import org.apache.flink.runtime.concurrent.CompletableFuture; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.concurrent.ScheduledExecutor; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -37,7 +34,8 @@ import java.util.Map; import java.util.Objects; import java.util.UUID; -import java.util.concurrent.ExecutionException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; @@ -180,7 +178,7 @@ public boolean containsJob(JobID jobId) { return jobLeaderIdListeners.containsKey(jobId); } - public Future getLeaderId(JobID jobId) throws Exception { + public CompletableFuture getLeaderId(JobID jobId) throws Exception { if (!jobLeaderIdListeners.containsKey(jobId)) { addJob(jobId); } @@ -235,7 +233,7 @@ private JobLeaderIdListener( this.listenerJobLeaderIdActions = Preconditions.checkNotNull(listenerJobLeaderIdActions); this.leaderRetrievalService = Preconditions.checkNotNull(leaderRetrievalService); - leaderIdFuture = new FlinkCompletableFuture<>(); + leaderIdFuture = new CompletableFuture<>(); activateTimeout(); @@ -243,7 +241,7 @@ private JobLeaderIdListener( leaderRetrievalService.start(this); } - public Future getLeaderIdFuture() { + public CompletableFuture getLeaderIdFuture() { return leaderIdFuture; } @@ -269,12 +267,12 @@ public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionId) { if (leaderIdFuture.isDone()) { try { previousJobLeaderId = leaderIdFuture.getNow(null); - } catch (ExecutionException e) { + } catch (CompletionException e) { // this should never happen since we complete this future always properly handleError(e); } - leaderIdFuture = FlinkCompletableFuture.completed(leaderSessionId); + leaderIdFuture = CompletableFuture.completedFuture(leaderSessionId); } else { leaderIdFuture.complete(leaderSessionId); } 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 5e33c0e8aa46d..8318c09f7119f 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 @@ -26,7 +26,6 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.SlotID; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.heartbeat.HeartbeatListener; import org.apache.flink.runtime.heartbeat.HeartbeatManager; @@ -246,7 +245,7 @@ public void postStop() throws Exception { // ------------------------------------------------------------------------ @RpcMethod - public Future registerJobManager( + public CompletableFuture registerJobManager( final UUID resourceManagerLeaderId, final UUID jobManagerLeaderId, final ResourceID jobManagerResourceId, @@ -270,13 +269,13 @@ public Future registerJobManager( onFatalErrorAsync(exception); log.error("Could not add job {} to job leader id service.", jobId, e); - return FutureUtils.toFlinkFuture(FutureUtils.completedExceptionally(exception)); + return FutureUtils.completedExceptionally(exception); } } log.info("Registering job manager {}@{} for job {}.", jobManagerLeaderId, jobManagerAddress, jobId); - Future jobLeaderIdFuture; + CompletableFuture jobLeaderIdFuture; try { jobLeaderIdFuture = jobLeaderIdService.getLeaderId(jobId); @@ -289,12 +288,12 @@ public Future registerJobManager( onFatalErrorAsync(exception); log.debug("Could not obtain the job leader id future to verify the correct job leader."); - return FutureUtils.toFlinkFuture(FutureUtils.completedExceptionally(exception)); + return FutureUtils.completedExceptionally(exception); } - Future jobMasterGatewayFuture = getRpcService().connect(jobManagerAddress, JobMasterGateway.class); + CompletableFuture jobMasterGatewayFuture = getRpcService().connect(jobManagerAddress, JobMasterGateway.class); - Future registrationResponseFuture = jobMasterGatewayFuture.thenCombineAsync( + CompletableFuture registrationResponseFuture = jobMasterGatewayFuture.thenCombineAsync( jobLeaderIdFuture, (JobMasterGateway jobMasterGateway, UUID jobLeaderId) -> { if (isValid(resourceManagerLeaderId)) { @@ -339,8 +338,8 @@ public Future registerJobManager( "{} did not match the expected leader id {}.", jobManagerAddress, resourceManagerLeaderId, leaderSessionId); - return FutureUtils.toFlinkFuture(CompletableFuture.completedFuture( - new RegistrationResponse.Decline("Resource manager leader id did not match."))); + return CompletableFuture.completedFuture( + new RegistrationResponse.Decline("Resource manager leader id did not match.")); } } @@ -354,14 +353,14 @@ public Future registerJobManager( * @return The response by the ResourceManager. */ @RpcMethod - public Future registerTaskExecutor( + public CompletableFuture registerTaskExecutor( final UUID resourceManagerLeaderId, final String taskExecutorAddress, final ResourceID taskExecutorResourceId, final SlotReport slotReport) { if (Objects.equals(leaderSessionId, resourceManagerLeaderId)) { - Future taskExecutorGatewayFuture = getRpcService().connect(taskExecutorAddress, TaskExecutorGateway.class); + CompletableFuture taskExecutorGatewayFuture = getRpcService().connect(taskExecutorAddress, TaskExecutorGateway.class); return taskExecutorGatewayFuture.handleAsync( (TaskExecutorGateway taskExecutorGateway, Throwable throwable) -> { @@ -381,10 +380,10 @@ public Future registerTaskExecutor( "not equal the received leader session ID {}", taskExecutorResourceId, taskExecutorAddress, leaderSessionId, resourceManagerLeaderId); - return FutureUtils.toFlinkFuture(CompletableFuture.completedFuture( + return CompletableFuture.completedFuture( new RegistrationResponse.Decline("Discard registration because the leader id " + resourceManagerLeaderId + " does not match the expected leader id " + - leaderSessionId + '.'))); + leaderSessionId + '.')); } } @@ -493,8 +492,8 @@ public void registerInfoMessageListener(final String address) { if(infoMessageListeners.containsKey(address)) { log.warn("Receive a duplicate registration from info message listener on ({})", address); } else { - CompletableFuture infoMessageListenerRpcGatewayFuture = FutureUtils.toJava( - getRpcService().connect(address, InfoMessageListenerRpcGateway.class)); + CompletableFuture infoMessageListenerRpcGatewayFuture = getRpcService() + .connect(address, InfoMessageListenerRpcGateway.class); infoMessageListenerRpcGatewayFuture.whenCompleteAsync( (InfoMessageListenerRpcGateway gateway, Throwable failure) -> { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index eb091c4c5b481..1ba68932b45a5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -24,7 +24,6 @@ import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.SlotID; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rpc.RpcGateway; @@ -34,6 +33,7 @@ import org.apache.flink.runtime.taskexecutor.SlotReport; import java.util.UUID; +import java.util.concurrent.CompletableFuture; /** * The {@link ResourceManager}'s RPC gateway interface. @@ -51,7 +51,7 @@ public interface ResourceManagerGateway extends RpcGateway { * @param timeout Timeout for the future to complete * @return Future registration response */ - Future registerJobManager( + CompletableFuture registerJobManager( UUID resourceManagerLeaderId, UUID jobMasterLeaderId, ResourceID jobMasterResourceId, @@ -67,7 +67,7 @@ Future registerJobManager( * @param slotRequest The slot to request * @return The confirmation that the slot gets allocated */ - Future requestSlot( + CompletableFuture requestSlot( UUID resourceManagerLeaderID, UUID jobMasterLeaderID, SlotRequest slotRequest, @@ -84,7 +84,7 @@ Future requestSlot( * * @return The future to the response by the ResourceManager. */ - Future registerTaskExecutor( + CompletableFuture registerTaskExecutor( UUID resourceManagerLeaderId, String taskExecutorAddress, ResourceID resourceID, @@ -133,7 +133,7 @@ void notifySlotAvailable( * @param leaderSessionId The leader session ID with which to address the ResourceManager. * @return The future to the number of registered TaskManagers. */ - Future getNumberOfRegisteredTaskManagers(UUID leaderSessionId); + CompletableFuture getNumberOfRegisteredTaskManagers(UUID leaderSessionId); /** * Sends the heartbeat to resource manager from task manager diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java index 8354525658be3..3bda409f1a40a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java @@ -24,7 +24,6 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot; import org.apache.flink.runtime.clusterframework.types.SlotID; -import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.messages.Acknowledge; @@ -639,14 +638,13 @@ private void allocateSlot(TaskManagerSlot taskManagerSlot, PendingSlotRequest pe } // RPC call to the task manager - CompletableFuture requestFuture = FutureUtils.toJava( - gateway.requestSlot( - slotId, - pendingSlotRequest.getJobId(), - allocationId, - pendingSlotRequest.getTargetAddress(), - leaderId, - taskManagerRequestTimeout)); + CompletableFuture requestFuture = gateway.requestSlot( + slotId, + pendingSlotRequest.getJobId(), + allocationId, + pendingSlotRequest.getTargetAddress(), + leaderId, + taskManagerRequestTimeout); requestFuture.whenComplete( (Acknowledge acknowledge, Throwable throwable) -> { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutable.java index ec1c9842f5fc2..6e36bd3652e3b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutable.java @@ -19,9 +19,9 @@ package org.apache.flink.runtime.rpc; import org.apache.flink.api.common.time.Time; -import org.apache.flink.runtime.concurrent.Future; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeoutException; /** @@ -51,7 +51,7 @@ public interface MainThreadExecutable { * @param Return value of the callable * @return Future of the callable result */ - Future callAsync(Callable callable, Time callTimeout); + CompletableFuture callAsync(Callable callable, Time callTimeout); /** * Execute the runnable in the main thread of the underlying RPC endpoint, with 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 331f3a38e0e9f..b5bbc2b621f47 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 @@ -19,7 +19,6 @@ package org.apache.flink.runtime.rpc; import org.apache.flink.api.common.time.Time; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.util.Preconditions; import org.apache.flink.util.ReflectionUtil; import org.slf4j.Logger; @@ -28,6 +27,7 @@ import javax.annotation.Nonnull; import java.util.UUID; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -214,7 +214,7 @@ public RpcService getRpcService() { * * @return Future which is completed when the rpc endpoint has been terminated. */ - public Future getTerminationFuture() { + public CompletableFuture getTerminationFuture() { return ((SelfGateway)self).getTerminationFuture(); } @@ -264,7 +264,7 @@ protected void scheduleRunAsync(Runnable runnable, long delay, TimeUnit unit) { * @param Return type of the callable * @return Future for the result of the callable. */ - protected Future callAsync(Callable callable, Time timeout) { + protected CompletableFuture callAsync(Callable callable, Time timeout) { return ((MainThreadExecutable) self).callAsync(callable, timeout); } 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 51b7ca24e7fd9..a92f3e226981f 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 @@ -18,11 +18,11 @@ package org.apache.flink.runtime.rpc; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; @@ -61,7 +61,7 @@ public interface RpcService { * @return Future containing the rpc gateway or an {@link RpcConnectionException} if the * connection attempt failed */ - Future connect(String address, Class clazz); + CompletableFuture connect(String address, Class clazz); /** * Start a rpc server which forwards the remote procedure calls to the provided rpc endpoint. @@ -91,7 +91,7 @@ public interface RpcService { * * @return Termination future */ - Future getTerminationFuture(); + CompletableFuture getTerminationFuture(); /** * Gets the executor, provided by this RPC service. This executor can be used for example for @@ -145,7 +145,7 @@ public interface RpcService { void execute(Runnable runnable); /** - * Execute the given callable and return its result as a {@link Future}. This method can be used + * Execute the given callable and return its result as a {@link CompletableFuture}. This method can be used * to run code outside of the main thread of a {@link RpcEndpoint}. * *

IMPORTANT: This executor does not isolate the method invocations against @@ -158,5 +158,5 @@ public interface RpcService { * @param is the return value type * @return Future containing the callable's future result */ - Future execute(Callable callable); + CompletableFuture execute(Callable callable); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/SelfGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/SelfGateway.java index ed8ef9d1ad641..d39b1efb05143 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/SelfGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/SelfGateway.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.rpc; -import org.apache.flink.runtime.concurrent.Future; +import java.util.concurrent.CompletableFuture; /** * Interface for self gateways @@ -30,5 +30,5 @@ public interface SelfGateway { * * @return Future indicating when the rpc endpoint has been terminated */ - Future 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 c21accf2e5223..ae6b8322601d2 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 @@ -22,8 +22,7 @@ import akka.pattern.Patterns; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.runtime.concurrent.Future; -import org.apache.flink.runtime.concurrent.impl.FlinkFuture; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.rpc.MainThreadExecutable; import org.apache.flink.runtime.rpc.SelfGateway; import org.apache.flink.runtime.rpc.RpcGateway; @@ -44,7 +43,9 @@ import java.lang.reflect.InvocationHandler; import java.lang.reflect.Method; import java.util.BitSet; +import java.util.Objects; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkArgument; @@ -79,7 +80,7 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea private final long maximumFramesize; // null if gateway; otherwise non-null - private final Future terminationFuture; + private final CompletableFuture terminationFuture; AkkaInvocationHandler( String address, @@ -87,7 +88,7 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea ActorRef rpcEndpoint, Time timeout, long maximumFramesize, - Future terminationFuture) { + CompletableFuture terminationFuture) { this.address = Preconditions.checkNotNull(address); this.hostname = Preconditions.checkNotNull(hostname); @@ -146,20 +147,19 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl Class returnType = method.getReturnType(); - if (returnType.equals(Void.TYPE)) { + if (Objects.equals(returnType, Void.TYPE)) { rpcEndpoint.tell(rpcInvocation, ActorRef.noSender()); result = null; - } else if (returnType.equals(Future.class)) { + } else if (Objects.equals(returnType,CompletableFuture.class)) { // execute an asynchronous call - result = new FlinkFuture<>(Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout.toMilliseconds())); + result = FutureUtils.toJava(Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout.toMilliseconds())); } else { // execute a synchronous call - scala.concurrent.Future scalaFuture = Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout.toMilliseconds()); + CompletableFuture futureResult = FutureUtils.toJava( + Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout.toMilliseconds())); - Future futureResult = new FlinkFuture<>(scalaFuture); - - return futureResult.get(futureTimeout.getSize(), futureTimeout.getUnit()); + result = futureResult.get(futureTimeout.getSize(), futureTimeout.getUnit()); } } @@ -191,12 +191,12 @@ public void scheduleRunAsync(Runnable runnable, long delayMillis) { } @Override - public Future callAsync(Callable callable, Time callTimeout) { + public CompletableFuture callAsync(Callable callable, Time callTimeout) { if(isLocal) { @SuppressWarnings("unchecked") - scala.concurrent.Future result = (scala.concurrent.Future) Patterns.ask(rpcEndpoint, new CallAsync(callable), callTimeout.toMilliseconds()); + scala.concurrent.Future resultFuture = (scala.concurrent.Future) Patterns.ask(rpcEndpoint, new CallAsync(callable), callTimeout.toMilliseconds()); - return new FlinkFuture<>(result); + return FutureUtils.toJava(resultFuture); } else { throw new RuntimeException("Trying to send a Callable to a remote actor at " + rpcEndpoint.path() + ". This is not supported."); @@ -331,7 +331,7 @@ public String getHostname() { } @Override - public Future getTerminationFuture() { + public CompletableFuture getTerminationFuture() { return terminationFuture; } } 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 fe3fcc92a2df9..5c5351d590fb9 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 @@ -21,12 +21,8 @@ import akka.actor.ActorRef; import akka.actor.Status; import akka.actor.UntypedActor; -import akka.dispatch.Futures; import akka.japi.Procedure; import akka.pattern.Patterns; -import org.apache.flink.runtime.concurrent.CompletableFuture; -import org.apache.flink.runtime.concurrent.Future; -import org.apache.flink.runtime.concurrent.impl.FlinkFuture; import org.apache.flink.runtime.rpc.MainThreadValidatorUtil; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; @@ -44,11 +40,13 @@ import org.slf4j.LoggerFactory; import scala.concurrent.duration.FiniteDuration; +import scala.concurrent.impl.Promise; import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -209,24 +207,20 @@ private void handleRpcInvocation(RpcInvocation rpcInvocation) { return; } - if (result instanceof Future) { - final Future future = (Future) result; - - // pipe result to sender - if (future instanceof FlinkFuture) { - // FlinkFutures are currently backed by Scala's futures - FlinkFuture flinkFuture = (FlinkFuture) future; - - Patterns.pipe(flinkFuture.getScalaFuture(), getContext().dispatcher()).to(getSender()); - } else { - // We have to unpack the Flink future and pack it into a Scala future - Patterns.pipe(Futures.future(new Callable() { - @Override - public Object call() throws Exception { - return future.get(); + if (result instanceof CompletableFuture) { + final CompletableFuture future = (CompletableFuture) result; + Promise.DefaultPromise promise = new Promise.DefaultPromise<>(); + + future.whenComplete( + (value, throwable) -> { + if (throwable != null) { + promise.failure(throwable); + } else { + promise.success(value); } - }, getContext().dispatcher()), getContext().dispatcher()); - } + }); + + Patterns.pipe(promise.future(), getContext().dispatcher()); } else { // tell the sender the result of the computation getSender().tell(new Status.Success(result), getSelf()); 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 2f02e8c34ce74..80267f905f1fe 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 @@ -31,11 +31,8 @@ import akka.pattern.Patterns; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.concurrent.CompletableFuture; -import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.ScheduledExecutor; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; -import org.apache.flink.runtime.concurrent.impl.FlinkFuture; import org.apache.flink.runtime.rpc.MainThreadExecutable; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; @@ -57,6 +54,7 @@ import java.util.HashSet; import java.util.Set; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Delayed; import java.util.concurrent.Executor; import java.util.concurrent.FutureTask; @@ -133,7 +131,7 @@ public int getPort() { // this method does not mutate state and is thus thread-safe @Override - public Future connect(final String address, final Class clazz) { + public CompletableFuture connect(final String address, final Class clazz) { checkState(!stopped, "RpcService is stopped"); LOG.debug("Try to connect to remote RPC endpoint with address {}. Returning a {} gateway.", @@ -186,14 +184,14 @@ public C checkedApply(Object obj) throws Exception { } }, actorSystem.dispatcher()); - return new FlinkFuture<>(resultFuture); + return FutureUtils.toJava(resultFuture); } @Override public > C startServer(S rpcEndpoint) { checkNotNull(rpcEndpoint, "rpc endpoint"); - CompletableFuture terminationFuture = new FlinkCompletableFuture<>(); + CompletableFuture terminationFuture = new CompletableFuture<>(); Props akkaRpcActorProps = Props.create(AkkaRpcActor.class, rpcEndpoint, terminationFuture); ActorRef actorRef; @@ -283,14 +281,10 @@ public void stopService() { } @Override - public Future getTerminationFuture() { - return FlinkFuture.supplyAsync(new Callable(){ - @Override - public Void call() throws Exception { - actorSystem.awaitTermination(); - return null; - } - }, getExecutor()); + public CompletableFuture getTerminationFuture() { + return CompletableFuture.runAsync( + actorSystem::awaitTermination, + getExecutor()); } @Override @@ -317,10 +311,10 @@ public void execute(Runnable runnable) { } @Override - public Future execute(Callable callable) { + public CompletableFuture execute(Callable callable) { scala.concurrent.Future scalaFuture = Futures.future(callable, actorSystem.dispatcher()); - return new FlinkFuture<>(scalaFuture); + return FutureUtils.toJava(scalaFuture); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java index 71933feecd7e7..2ebf3c1f2dc24 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobmaster.JMTMRegistrationSuccess; import org.apache.flink.runtime.jobmaster.JobMasterGateway; @@ -379,8 +378,8 @@ private static final class JobManagerRetryingRegistration protected CompletableFuture invokeRegistration( JobMasterGateway gateway, UUID leaderId, long timeoutMillis) throws Exception { - return FutureUtils.toJava(gateway.registerTaskManager(taskManagerRpcAddress, taskManagerLocation, - leaderId, Time.milliseconds(timeoutMillis))); + return gateway.registerTaskManager(taskManagerRpcAddress, taskManagerLocation, + leaderId, Time.milliseconds(timeoutMillis)); } } 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 aa4d6d27ed06b..effa498b61972 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 @@ -27,7 +27,6 @@ import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.SlotID; -import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; @@ -768,12 +767,11 @@ private void offerSlotsToJobManager(final JobID jobId) { reservedSlots.add(offer); } - CompletableFuture> acceptedSlotsFuture = FutureUtils.toJava( - jobMasterGateway.offerSlots( - getResourceID(), - reservedSlots, - leaderId, - taskManagerConfiguration.getTimeout())); + CompletableFuture> acceptedSlotsFuture = jobMasterGateway.offerSlots( + getResourceID(), + reservedSlots, + leaderId, + taskManagerConfiguration.getTimeout()); acceptedSlotsFuture.whenCompleteAsync( (Iterable acceptedSlots, Throwable throwable) -> { @@ -985,8 +983,7 @@ private void updateTaskExecutionState( { final ExecutionAttemptID executionAttemptID = taskExecutionState.getID(); - CompletableFuture futureAcknowledge = FutureUtils.toJava( - jobMasterGateway.updateTaskExecutionState(jobMasterLeaderId, taskExecutionState)); + CompletableFuture futureAcknowledge = jobMasterGateway.updateTaskExecutionState(jobMasterLeaderId, taskExecutionState); futureAcknowledge.whenCompleteAsync( (ack, throwable) -> { @@ -1348,10 +1345,9 @@ public void reportPayload(ResourceID resourceID, Void payload) { @Override public CompletableFuture retrievePayload() { - return FutureUtils.toJava( - callAsync( + return callAsync( () -> taskSlotTable.createSlotReport(getResourceID()), - taskManagerConfiguration.getTimeout())); + taskManagerConfiguration.getTimeout()); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java index d4afdbd56f1f0..80841545f1059 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java @@ -24,7 +24,6 @@ import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.SlotID; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.PartitionInfo; @@ -34,6 +33,7 @@ import org.apache.flink.runtime.taskmanager.Task; import java.util.UUID; +import java.util.concurrent.CompletableFuture; /** * {@link TaskExecutor} RPC gateway interface @@ -48,7 +48,7 @@ public interface TaskExecutorGateway extends RpcGateway { * @param resourceManagerLeaderId current leader id of the ResourceManager * @return answer to the slot request */ - Future requestSlot( + CompletableFuture requestSlot( SlotID slotId, JobID jobId, AllocationID allocationId, @@ -64,7 +64,7 @@ Future requestSlot( * @param timeout of the submit operation * @return Future acknowledge of the successful operation */ - Future submitTask( + CompletableFuture submitTask( TaskDeploymentDescriptor tdd, UUID leaderId, @RpcTimeout Time timeout); @@ -77,7 +77,7 @@ Future submitTask( * @param timeout for the update partitions operation * @return Future acknowledge if the partitions have been successfully updated */ - Future updatePartitions( + CompletableFuture updatePartitions( ExecutionAttemptID executionAttemptID, Iterable partitionInfos, @RpcTimeout Time timeout); @@ -99,7 +99,7 @@ Future updatePartitions( * @param checkpointOptions for performing the checkpoint * @return Future acknowledge if the checkpoint has been successfully triggered */ - Future triggerCheckpoint(ExecutionAttemptID executionAttemptID, long checkpointID, long checkpointTimestamp, CheckpointOptions checkpointOptions); + CompletableFuture triggerCheckpoint(ExecutionAttemptID executionAttemptID, long checkpointID, long checkpointTimestamp, CheckpointOptions checkpointOptions); /** * Confirm a checkpoint for the given task. The checkpoint is identified by the checkpoint ID @@ -110,7 +110,7 @@ Future updatePartitions( * @param checkpointTimestamp is the timestamp when the checkpoint has been initiated * @return Future acknowledge if the checkpoint has been successfully confirmed */ - Future confirmCheckpoint(ExecutionAttemptID executionAttemptID, long checkpointId, long checkpointTimestamp); + CompletableFuture confirmCheckpoint(ExecutionAttemptID executionAttemptID, long checkpointId, long checkpointTimestamp); /** * Stop the given task. @@ -119,7 +119,7 @@ Future updatePartitions( * @param timeout for the stop operation * @return Future acknowledge if the task is successfully stopped */ - Future stopTask(ExecutionAttemptID executionAttemptID, @RpcTimeout Time timeout); + CompletableFuture stopTask(ExecutionAttemptID executionAttemptID, @RpcTimeout Time timeout); /** * Cancel the given task. @@ -128,7 +128,7 @@ Future updatePartitions( * @param timeout for the cancel operation * @return Future acknowledge if the task is successfully canceled */ - Future cancelTask(ExecutionAttemptID executionAttemptID, @RpcTimeout Time timeout); + CompletableFuture cancelTask(ExecutionAttemptID executionAttemptID, @RpcTimeout Time timeout); /** * Heartbeat request from the job manager diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java index 4f91166b4b792..4084d67a60055 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.registration.RegisteredRpcConnection; import org.apache.flink.runtime.registration.RegistrationConnectionListener; @@ -156,7 +155,7 @@ protected CompletableFuture invokeRegistration( ResourceManagerGateway resourceManager, UUID leaderId, long timeoutMillis) throws Exception { Time timeout = Time.milliseconds(timeoutMillis); - return FutureUtils.toJava(resourceManager.registerTaskExecutor(leaderId, taskExecutorAddress, resourceID, slotReport, timeout)); + return resourceManager.registerTaskExecutor(leaderId, taskExecutorAddress, resourceID, slotReport, timeout); } } } 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 78b49ef68cec9..b077b76055d65 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 @@ -26,7 +26,6 @@ import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.Executors; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; @@ -52,6 +51,7 @@ import java.net.InetAddress; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; @@ -162,7 +162,7 @@ protected void shutDownInternally() throws Exception { } // export the termination future for caller to know it is terminated - public Future getTerminationFuture() { + public CompletableFuture getTerminationFuture() { return taskManager.getTerminationFuture(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java index 3b9da48a78f28..a919c7878bbab 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.core.io.InputSplit; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; @@ -32,6 +31,7 @@ import org.apache.flink.util.Preconditions; import java.util.UUID; +import java.util.concurrent.CompletableFuture; public class RpcInputSplitProvider implements InputSplitProvider { private final UUID jobMasterLeaderId; @@ -61,7 +61,7 @@ public RpcInputSplitProvider( public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) throws InputSplitProviderException { Preconditions.checkNotNull(userCodeClassLoader); - Future futureInputSplit = jobMasterGateway.requestNextInputSplit( + CompletableFuture futureInputSplit = jobMasterGateway.requestNextInputSplit( jobMasterLeaderId, jobVertexID, executionAttemptID); try { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java index 07d04e6384b56..26e1b0efa781c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.taskexecutor.rpc; import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.io.network.netty.PartitionProducerStateChecker; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; @@ -46,6 +45,6 @@ public CompletableFuture requestPartitionProducerState( IntermediateDataSetID resultId, ResultPartitionID partitionId) { - return FutureUtils.toJava(jobMasterGateway.requestPartitionState(jobMasterLeaderId, resultId, partitionId)); + return jobMasterGateway.requestPartitionState(jobMasterLeaderId, resultId, partitionId); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java index cf01d5a45aef8..d8985620bb828 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java @@ -20,8 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; -import org.apache.flink.runtime.concurrent.ApplyFunction; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobmaster.JobMasterGateway; @@ -32,6 +30,7 @@ import org.slf4j.LoggerFactory; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; public class RpcResultPartitionConsumableNotifier implements ResultPartitionConsumableNotifier { @@ -55,18 +54,17 @@ public RpcResultPartitionConsumableNotifier( } @Override public void notifyPartitionConsumable(JobID jobId, ResultPartitionID partitionId, final TaskActions taskActions) { - Future acknowledgeFuture = jobMasterGateway.scheduleOrUpdateConsumers( + CompletableFuture acknowledgeFuture = jobMasterGateway.scheduleOrUpdateConsumers( jobMasterLeaderId, partitionId, timeout); - acknowledgeFuture.exceptionallyAsync(new ApplyFunction() { - @Override - public Void apply(Throwable value) { - LOG.error("Could not schedule or update consumers at the JobManager.", value); + acknowledgeFuture.whenCompleteAsync( + (Acknowledge ack, Throwable throwable) -> { + if (throwable != null) { + LOG.error("Could not schedule or update consumers at the JobManager.", throwable); - taskActions.failExternally(new RuntimeException("Could not notify JobManager to schedule or update consumers.", value)); - - return null; - } - }, executor); + taskActions.failExternally(new RuntimeException("Could not notify JobManager to schedule or update consumers.", throwable)); + } + }, + executor); } } 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 a6712ad6793d5..ef4fa86152309 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 @@ -22,7 +22,7 @@ import java.io.IOException import java.net._ import java.util.UUID import java.util.concurrent.{Future => JavaFuture, _} -import java.util.function.BiFunction +import java.util.function.{BiFunction, Consumer} import akka.actor.Status.{Failure, Success} import akka.actor._ @@ -1105,17 +1105,18 @@ class JobManager( val originalSender = new AkkaActorGateway(sender(), leaderSessionID.orNull) - val sendingFuture = stackTraceFuture.thenAccept(new AcceptFunction[StackTrace] { - override def accept(value: StackTrace): Unit = { - originalSender.tell(value) - } - }) + val sendingFuture = stackTraceFuture.thenAccept( + new Consumer[StackTrace]() { + override def accept(value: StackTrace): Unit = { + originalSender.tell(value) + } + }) - sendingFuture.exceptionally(new ApplyFunction[Throwable, Void] { + sendingFuture.exceptionally(new java.util.function.Function[Throwable, Void] { override def apply(value: Throwable): Void = { log.info("Could not send requested stack trace.", value) - return null + null } }) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/akka/QuarantineMonitorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/akka/QuarantineMonitorTest.java index 09e829e412b11..37a45477cc35c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/akka/QuarantineMonitorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/akka/QuarantineMonitorTest.java @@ -18,9 +18,6 @@ package org.apache.flink.runtime.akka; -import org.apache.flink.runtime.concurrent.CompletableFuture; -import org.apache.flink.runtime.concurrent.Future; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; @@ -43,6 +40,7 @@ import org.slf4j.LoggerFactory; import java.util.Properties; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -125,7 +123,7 @@ public void testWatcheeQuarantined() throws ExecutionException, InterruptedExcep // start watching the watchee watcher.tell(new Watch(watcheeAddress), ActorRef.noSender()); - Future quarantineFuture = quarantineHandler.getWasQuarantinedByFuture(); + CompletableFuture quarantineFuture = quarantineHandler.getWasQuarantinedByFuture(); Assert.assertEquals(actorSystem1Address.toString(), quarantineFuture.get()); } finally { @@ -166,7 +164,7 @@ public void testWatcherQuarantining() throws ExecutionException, InterruptedExce // start watching the watchee watcher.tell(new Watch(watcheeAddress), ActorRef.noSender()); - Future quarantineFuture = quarantineHandler.getHasQuarantinedFuture(); + CompletableFuture quarantineFuture = quarantineHandler.getHasQuarantinedFuture(); Assert.assertEquals(actorSystem1Address.toString(), quarantineFuture.get()); } finally { @@ -182,8 +180,8 @@ private static class TestingQuarantineHandler implements QuarantineHandler, Erro private final CompletableFuture hasQuarantinedFuture; public TestingQuarantineHandler() { - this.wasQuarantinedByFuture = new FlinkCompletableFuture<>(); - this.hasQuarantinedFuture = new FlinkCompletableFuture<>(); + this.wasQuarantinedByFuture = new CompletableFuture<>(); + this.hasQuarantinedFuture = new CompletableFuture<>(); } @Override @@ -196,11 +194,11 @@ public void hasQuarantined(String remoteSystem, ActorSystem actorSystem) { hasQuarantinedFuture.complete(remoteSystem); } - public Future getWasQuarantinedByFuture() { + public CompletableFuture getWasQuarantinedByFuture() { return wasQuarantinedByFuture; } - public Future getHasQuarantinedFuture() { + public CompletableFuture getHasQuarantinedFuture() { return hasQuarantinedFuture; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java index 5df5c58e8ce8e..e23f6a2f7d8f7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.runtime.concurrent.Executors; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; @@ -339,10 +338,10 @@ public void ensureRegisteredAtHookTime() throws Exception { final MasterTriggerRestoreHook hook = mockGeneric(MasterTriggerRestoreHook.class); when(hook.getIdentifier()).thenReturn(id); when(hook.triggerCheckpoint(anyLong(), anyLong(), any(Executor.class))).thenAnswer( - new Answer>() { + new Answer>() { @Override - public Future answer(InvocationOnMock invocation) throws Throwable { + public CompletableFuture answer(InvocationOnMock invocation) throws Throwable { assertEquals(1, cc.getNumberOfPendingCheckpoints()); long checkpointId = (Long) invocation.getArguments()[0]; 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 f1bc43b9ad5fb..e1144c99ebf26 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 @@ -32,7 +32,6 @@ import org.apache.flink.runtime.clusterframework.messages.ResourceRemoved; import org.apache.flink.runtime.clusterframework.messages.TriggerRegistrationAtJobManager; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices; @@ -71,6 +70,7 @@ import java.util.Collections; import java.util.List; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -524,7 +524,7 @@ public void testHeartbeatTimeoutWithTaskExecutor() throws Exception { final SlotReport slotReport = new SlotReport(); // test registration response successful and it will trigger monitor heartbeat target, schedule heartbeat request at interval time - Future successfulFuture = resourceManager.registerTaskExecutor( + CompletableFuture successfulFuture = resourceManager.registerTaskExecutor( rmLeaderSessionId, taskManagerAddress, taskManagerResourceID, @@ -622,7 +622,7 @@ public void testHeartbeatTimeoutWithJobManager() throws Exception { rmLeaderElectionService.isLeader(rmLeaderId); // test registration response successful and it will trigger monitor heartbeat target, schedule heartbeat request at interval time - Future successfulFuture = resourceManager.registerJobManager( + CompletableFuture successfulFuture = resourceManager.registerJobManager( rmLeaderId, jmLeaderId, jmResourceId, 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 a7a86c3a45654..267f10b45969d 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 @@ -24,7 +24,6 @@ import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.blob.BlobService; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneHaServices; @@ -42,6 +41,8 @@ import org.junit.Test; import org.mockito.Mockito; +import java.util.concurrent.CompletableFuture; + import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; @@ -86,7 +87,7 @@ public void testJobSubmission() throws Exception { DispatcherGateway dispatcherGateway = dispatcher.getSelf(); - Future acknowledgeFuture = dispatcherGateway.submitJob(jobGraph, timeout); + CompletableFuture acknowledgeFuture = dispatcherGateway.submitJob(jobGraph, timeout); acknowledgeFuture.get(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java index bfcab8779d202..f4e8b30ca7f3b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java @@ -25,7 +25,6 @@ import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.ScheduledExecutor; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.execution.SuppressRestartsException; @@ -120,7 +119,7 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti when(rootSlot.getSlotNumber()).thenReturn(0); - when(taskManagerGateway.submitTask(any(TaskDeploymentDescriptor.class), any(Time.class))).thenReturn(FlinkCompletableFuture.completed(Acknowledge.get())); + when(taskManagerGateway.submitTask(any(TaskDeploymentDescriptor.class), any(Time.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); TestingRestartStrategy testingRestartStrategy = new TestingRestartStrategy(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java index d3086a8e10935..b88a9280cee14 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java @@ -26,7 +26,6 @@ import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.instance.SimpleSlot; @@ -576,7 +575,7 @@ private SimpleSlot createSlot(TaskManagerGateway taskManager, JobID jobId, SlotO private static TaskManagerGateway createTaskManager() { TaskManagerGateway tm = mock(TaskManagerGateway.class); when(tm.submitTask(any(TaskDeploymentDescriptor.class), any(Time.class))) - .thenReturn(FlinkCompletableFuture.completed(Acknowledge.get())); + .thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); return tm; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphStopTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphStopTest.java index effe417764776..de9081b222e44 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphStopTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphStopTest.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.StoppingException; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; @@ -37,6 +36,8 @@ import org.junit.Test; +import java.util.concurrent.CompletableFuture; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -155,9 +156,9 @@ public void testStopRpc() throws Exception { final TaskManagerGateway gateway = mock(TaskManagerGateway.class); when(gateway.submitTask(any(TaskDeploymentDescriptor.class), any(Time.class))) - .thenReturn(FlinkCompletableFuture.completed(Acknowledge.get())); + .thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); when(gateway.stopTask(any(ExecutionAttemptID.class), any(Time.class))) - .thenReturn(FlinkCompletableFuture.completed(Acknowledge.get())); + .thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); final SimpleSlot slot = ExecutionGraphTestUtils.createMockSimpleSlot(jid, gateway); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/NotCancelAckingTaskGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/NotCancelAckingTaskGateway.java index f453d20912e12..be6853290ccbd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/NotCancelAckingTaskGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/NotCancelAckingTaskGateway.java @@ -19,14 +19,15 @@ package org.apache.flink.runtime.executiongraph.utils; import org.apache.flink.api.common.time.Time; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.messages.Acknowledge; +import java.util.concurrent.CompletableFuture; + public class NotCancelAckingTaskGateway extends SimpleAckingTaskManagerGateway { @Override - public org.apache.flink.runtime.concurrent.Future cancelTask(ExecutionAttemptID executionAttemptID, Time timeout) { - return new FlinkCompletableFuture<>(); + public CompletableFuture cancelTask(ExecutionAttemptID executionAttemptID, Time timeout) { + return new CompletableFuture<>(); } -} \ No newline at end of file +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java index 6e67c1a53dc81..b968d3930a2a7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java @@ -23,8 +23,7 @@ import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.clusterframework.ApplicationStatus; -import org.apache.flink.runtime.concurrent.Future; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.PartitionInfo; @@ -35,6 +34,7 @@ import org.apache.flink.runtime.messages.StackTraceSampleResponse; import java.util.UUID; +import java.util.concurrent.CompletableFuture; /** * A TaskManagerGateway that simply acks the basic operations (deploy, cancel, update) and does not @@ -56,39 +56,39 @@ public void disconnectFromJobManager(InstanceID instanceId, Exception cause) {} public void stopCluster(ApplicationStatus applicationStatus, String message) {} @Override - public Future requestStackTrace(Time timeout) { - return FlinkCompletableFuture.completedExceptionally(new UnsupportedOperationException()); + public CompletableFuture requestStackTrace(Time timeout) { + return FutureUtils.completedExceptionally(new UnsupportedOperationException()); } @Override - public Future requestStackTraceSample( + public CompletableFuture requestStackTraceSample( ExecutionAttemptID executionAttemptID, int sampleId, int numSamples, Time delayBetweenSamples, int maxStackTraceDepth, Time timeout) { - return FlinkCompletableFuture.completedExceptionally(new UnsupportedOperationException()); + return FutureUtils.completedExceptionally(new UnsupportedOperationException()); } @Override - public Future submitTask(TaskDeploymentDescriptor tdd, Time timeout) { - return FlinkCompletableFuture.completed(Acknowledge.get()); + public CompletableFuture submitTask(TaskDeploymentDescriptor tdd, Time timeout) { + return CompletableFuture.completedFuture(Acknowledge.get()); } @Override - public Future stopTask(ExecutionAttemptID executionAttemptID, Time timeout) { - return FlinkCompletableFuture.completed(Acknowledge.get()); + public CompletableFuture stopTask(ExecutionAttemptID executionAttemptID, Time timeout) { + return CompletableFuture.completedFuture(Acknowledge.get()); } @Override - public Future cancelTask(ExecutionAttemptID executionAttemptID, Time timeout) { - return FlinkCompletableFuture.completed(Acknowledge.get()); + public CompletableFuture cancelTask(ExecutionAttemptID executionAttemptID, Time timeout) { + return CompletableFuture.completedFuture(Acknowledge.get()); } @Override - public Future updatePartitions(ExecutionAttemptID executionAttemptID, Iterable partitionInfos, Time timeout) { - return FlinkCompletableFuture.completed(Acknowledge.get()); + public CompletableFuture updatePartitions(ExecutionAttemptID executionAttemptID, Iterable partitionInfos, Time timeout) { + return CompletableFuture.completedFuture(Acknowledge.get()); } @Override @@ -110,12 +110,12 @@ public void triggerCheckpoint( CheckpointOptions checkpointOptions) {} @Override - public Future requestTaskManagerLog(Time timeout) { - return FlinkCompletableFuture.completedExceptionally(new UnsupportedOperationException()); + public CompletableFuture requestTaskManagerLog(Time timeout) { + return FutureUtils.completedExceptionally(new UnsupportedOperationException()); } @Override - public Future requestTaskManagerStdout(Time timeout) { - return FlinkCompletableFuture.completedExceptionally(new UnsupportedOperationException()); + public CompletableFuture requestTaskManagerStdout(Time timeout) { + return FutureUtils.completedExceptionally(new UnsupportedOperationException()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolRpcTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolRpcTest.java index b4446400ed71d..4cc4f11d754bf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolRpcTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolRpcTest.java @@ -23,7 +23,6 @@ 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.Future; import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; import org.apache.flink.runtime.rpc.RpcService; @@ -35,6 +34,7 @@ import org.junit.Test; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -82,7 +82,7 @@ public void testSlotAllocationNoResourceManager() throws Exception { ); pool.start(UUID.randomUUID(), "foobar"); - Future future = pool.allocateSlot(mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null); + CompletableFuture future = pool.allocateSlot(mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null); try { future.get(4, TimeUnit.SECONDS); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java index cf95461a12bd9..3e2293b56c303 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java @@ -23,7 +23,6 @@ import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; @@ -41,6 +40,7 @@ import java.util.List; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import static org.apache.flink.runtime.instance.AvailableSlotsTest.DEFAULT_TESTING_PROFILE; @@ -85,7 +85,7 @@ public void setUp() throws Exception { this.resourceManagerGateway = mock(ResourceManagerGateway.class); when(resourceManagerGateway .requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class))) - .thenReturn(mock(Future.class, RETURNS_MOCKS)); + .thenReturn(mock(CompletableFuture.class, RETURNS_MOCKS)); slotPool.connectToResourceManager(UUID.randomUUID(), resourceManagerGateway); } @@ -101,7 +101,7 @@ public void testAllocateSimpleSlot() throws Exception { slotPool.registerTaskManager(resourceID); ScheduledUnit task = mock(ScheduledUnit.class); - Future future = slotPool.allocateSlot(task, DEFAULT_TESTING_PROFILE, null); + CompletableFuture future = slotPool.allocateSlot(task, DEFAULT_TESTING_PROFILE, null); assertFalse(future.isDone()); ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); @@ -126,8 +126,8 @@ public void testAllocationFulfilledByReturnedSlot() throws Exception { ResourceID resourceID = new ResourceID("resource"); slotPool.registerTaskManager(resourceID); - Future future1 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null); - Future future2 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null); + CompletableFuture future1 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null); + CompletableFuture future2 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null); assertFalse(future1.isDone()); assertFalse(future2.isDone()); @@ -165,7 +165,7 @@ public void testAllocateWithFreeSlot() throws Exception { ResourceID resourceID = new ResourceID("resource"); slotPool.registerTaskManager(resourceID); - Future future1 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null); + CompletableFuture future1 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null); assertFalse(future1.isDone()); ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); @@ -182,7 +182,7 @@ public void testAllocateWithFreeSlot() throws Exception { // return this slot to pool slot1.releaseSlot(); - Future future2 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null); + CompletableFuture future2 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null); // second allocation fulfilled by previous slot returning SimpleSlot slot2 = future2.get(1, TimeUnit.SECONDS); @@ -200,7 +200,7 @@ public void testOfferSlot() throws Exception { ResourceID resourceID = new ResourceID("resource"); slotPool.registerTaskManager(resourceID); - Future future = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null); + CompletableFuture future = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null); assertFalse(future.isDone()); ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); @@ -241,14 +241,14 @@ public void testReleaseResource() throws Exception { ResourceID resourceID = new ResourceID("resource"); slotPool.registerTaskManager(resourceID); - Future future1 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null); + CompletableFuture future1 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null); ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); - Future future2 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null); + CompletableFuture future2 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null); AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE); assertTrue(slotPool.offerSlot(allocatedSlot)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index 0b25e6cdb3854..48a1d45986237 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -24,7 +24,6 @@ import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.ScheduledExecutor; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoader; import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory; @@ -34,7 +33,6 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.OnCompletionActions; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; -import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.TestingSerialRpcService; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; @@ -50,6 +48,7 @@ import java.net.InetAddress; import java.net.URL; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -177,7 +176,7 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { anyString(), any(JobID.class), any(Time.class) - )).thenReturn(FlinkCompletableFuture.completed(new JobMasterRegistrationSuccess( + )).thenReturn(CompletableFuture.completedFuture(new JobMasterRegistrationSuccess( heartbeatInterval, rmLeaderId, rmResourceId))); final TestingSerialRpcService rpc = new TestingSerialRpcService(); 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 9a4917a2ba213..da992bb103b57 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 @@ -20,7 +20,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.util.TestLogger; @@ -123,8 +122,8 @@ public void testRetryConnectOnFailure() throws Exception { // RPC service that fails upon the first connection, but succeeds on the second RpcService rpc = mock(RpcService.class); when(rpc.connect(anyString(), any(Class.class))).thenReturn( - FlinkCompletableFuture.completedExceptionally(new Exception("test connect failure")), // first connection attempt fails - FlinkCompletableFuture.completed(testGateway) // second connection attempt succeeds + FutureUtils.completedExceptionally(new Exception("test connect failure")), // first connection attempt fails + CompletableFuture.completedFuture(testGateway) // second connection attempt succeeds ); when(rpc.getExecutor()).thenReturn(executor); @@ -245,8 +244,8 @@ public void testRetryOnError() throws Exception { TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class); when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn( - FlinkCompletableFuture.completedExceptionally(new Exception("test exception")), - FlinkCompletableFuture.completed(new TestRegistrationSuccess(testId))); + FutureUtils.completedExceptionally(new Exception("test exception")), + CompletableFuture.completedFuture(new TestRegistrationSuccess(testId))); rpc.registerGateway(testEndpointAddress, testGateway); @@ -281,7 +280,7 @@ public void testCancellation() throws Exception { TestingRpcService rpc = new TestingRpcService(); try { - FlinkCompletableFuture result = new FlinkCompletableFuture<>(); + CompletableFuture result = new CompletableFuture<>(); TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class); when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(result); @@ -340,7 +339,7 @@ public TestRetryingRegistration(RpcService rpc, String targetAddress, UUID leade @Override protected CompletableFuture invokeRegistration( TestRegistrationGateway gateway, UUID leaderId, long timeoutMillis) { - return FutureUtils.toJava(gateway.registrationCall(leaderId, timeoutMillis)); + return gateway.registrationCall(leaderId, timeoutMillis); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java index 1b23fa36f90cb..4cfbc128f7797 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java @@ -18,13 +18,12 @@ package org.apache.flink.runtime.registration; -import org.apache.flink.runtime.concurrent.Future; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.rpc.TestingGatewayBase; import org.apache.flink.util.Preconditions; import java.util.UUID; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.LinkedBlockingQueue; /** @@ -47,7 +46,7 @@ public TestRegistrationGateway(RegistrationResponse... responses) { // ------------------------------------------------------------------------ - public Future registrationCall(UUID leaderId, long timeout) { + public CompletableFuture registrationCall(UUID leaderId, long timeout) { invocations.add(new RegistrationCall(leaderId, timeout)); RegistrationResponse response = responses[pos]; @@ -56,7 +55,7 @@ public Future registrationCall(UUID leaderId, long timeout } // return a completed future (for a proper value), or one that never completes and will time out (for null) - return response != null ? FlinkCompletableFuture.completed(response) : this.futureWithTimeout(timeout); + return response != null ? CompletableFuture.completedFuture(response) : futureWithTimeout(timeout); } public BlockingQueue getInvocations() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdServiceTest.java index 4d5964ee14f2c..7b8703e768be6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdServiceTest.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; @@ -34,6 +33,7 @@ import java.util.Arrays; import java.util.Queue; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; @@ -83,7 +83,7 @@ public void testAddingJob() throws Exception { jobLeaderIdService.addJob(jobId); - Future leaderIdFuture = jobLeaderIdService.getLeaderId(jobId); + CompletableFuture leaderIdFuture = jobLeaderIdService.getLeaderId(jobId); // notify the leader id service about the new leader leaderRetrievalService.notifyListener(address, leaderId); @@ -117,7 +117,7 @@ public void testRemovingJob() throws Exception { jobLeaderIdService.addJob(jobId); - Future leaderIdFuture = jobLeaderIdService.getLeaderId(jobId); + CompletableFuture leaderIdFuture = jobLeaderIdService.getLeaderId(jobId); // remove the job before we could find a leader jobLeaderIdService.removeJob(jobId); @@ -228,7 +228,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { jobLeaderIdService.addJob(jobId); - Future leaderIdFuture = jobLeaderIdService.getLeaderId(jobId); + CompletableFuture leaderIdFuture = jobLeaderIdService.getLeaderId(jobId); // notify the leader id service about the new leader leaderRetrievalService.notifyListener(address, leaderId); 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 4836f74ac128d..6480d75466f99 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 @@ -22,7 +22,6 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.FlinkResourceManager; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; @@ -43,6 +42,7 @@ import org.junit.Test; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertTrue; @@ -74,11 +74,11 @@ public void testRegisterJobMaster() throws Exception { final ResourceID jmResourceId = new ResourceID(jobMasterAddress); TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jmLeaderID); TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); - final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); + final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); // test response successful - Future successfulFuture = resourceManager.registerJobManager( + CompletableFuture successfulFuture = resourceManager.registerJobManager( rmLeaderSessionId, jmLeaderID, jmResourceId, @@ -104,12 +104,12 @@ public void testRegisterJobMasterWithUnmatchedLeaderSessionId1() throws Exceptio final ResourceID jmResourceId = new ResourceID(jobMasterAddress); TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jmLeaderID); TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); - final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); + final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); // test throw exception when receive a registration from job master which takes unmatched leaderSessionId UUID differentLeaderSessionID = UUID.randomUUID(); - Future unMatchedLeaderFuture = resourceManager.registerJobManager( + CompletableFuture unMatchedLeaderFuture = resourceManager.registerJobManager( differentLeaderSessionID, jmLeaderID, jmResourceId, @@ -134,14 +134,14 @@ public void testRegisterJobMasterWithUnmatchedLeaderSessionId2() throws Exceptio "localhost", HighAvailabilityServices.DEFAULT_LEADER_ID); TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); - final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); + final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); final UUID jmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); final ResourceID jmResourceId = new ResourceID(jobMasterAddress); // test throw exception when receive a registration from job master which takes unmatched leaderSessionId UUID differentLeaderSessionID = UUID.randomUUID(); - Future unMatchedLeaderFuture = resourceManager.registerJobManager( + CompletableFuture unMatchedLeaderFuture = resourceManager.registerJobManager( rmLeaderSessionId, differentLeaderSessionID, jmResourceId, @@ -166,14 +166,14 @@ public void testRegisterJobMasterFromInvalidAddress() throws Exception { "localhost", HighAvailabilityServices.DEFAULT_LEADER_ID); TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); - final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); + final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); final UUID jmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); final ResourceID jmResourceId = new ResourceID(jobMasterAddress); // test throw exception when receive a registration from job master which takes invalid address String invalidAddress = "/jobMasterAddress2"; - Future invalidAddressFuture = resourceManager.registerJobManager( + CompletableFuture invalidAddressFuture = resourceManager.registerJobManager( rmLeaderSessionId, jmLeaderSessionId, jmResourceId, @@ -198,14 +198,14 @@ public void testRegisterJobMasterWithFailureLeaderListener() throws Exception { "localhost", HighAvailabilityServices.DEFAULT_LEADER_ID); TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); - final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); + final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); final UUID jmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); final ResourceID jmResourceId = new ResourceID(jobMasterAddress); JobID unknownJobIDToHAServices = new JobID(); // verify return RegistrationResponse.Decline when failed to start a job master Leader retrieval listener - Future declineFuture = resourceManager.registerJobManager( + CompletableFuture declineFuture = resourceManager.registerJobManager( rmLeaderSessionId, jmLeaderSessionId, jmResourceId, 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 85b7eb47e18c7..4127cead06ca0 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 @@ -21,7 +21,6 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.FlinkResourceManager; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; @@ -41,6 +40,7 @@ import org.junit.Test; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertNotEquals; @@ -89,13 +89,13 @@ public void teardown() throws Exception { public void testRegisterTaskExecutor() throws Exception { try { // test response successful - Future successfulFuture = + CompletableFuture successfulFuture = resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID, slotReport); RegistrationResponse response = successfulFuture.get(5, TimeUnit.SECONDS); assertTrue(response instanceof TaskExecutorRegistrationSuccess); // test response successful with instanceID not equal to previous when receive duplicate registration from taskExecutor - Future duplicateFuture = + CompletableFuture duplicateFuture = resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID, slotReport); RegistrationResponse duplicateResponse = duplicateFuture.get(); assertTrue(duplicateResponse instanceof TaskExecutorRegistrationSuccess); @@ -115,7 +115,7 @@ public void testRegisterTaskExecutorWithUnmatchedLeaderSessionId() throws Except try { // test throw exception when receive a registration from taskExecutor which takes unmatched leaderSessionId UUID differentLeaderSessionID = UUID.randomUUID(); - Future unMatchedLeaderFuture = + CompletableFuture unMatchedLeaderFuture = resourceManager.registerTaskExecutor(differentLeaderSessionID, taskExecutorAddress, taskExecutorResourceID, slotReport); assertTrue(unMatchedLeaderFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline); } finally { @@ -133,7 +133,7 @@ public void testRegisterTaskExecutorFromInvalidAddress() throws Exception { try { // test throw exception when receive a registration from taskExecutor which takes invalid address String invalidAddress = "/taskExecutor2"; - Future invalidAddressFuture = + CompletableFuture invalidAddressFuture = resourceManager.registerTaskExecutor(leaderSessionId, invalidAddress, taskExecutorResourceID, slotReport); assertTrue(invalidAddressFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline); } finally { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java index 39c5f25735156..93e96a7b0f026 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java @@ -25,9 +25,9 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.SlotID; import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot; -import org.apache.flink.runtime.concurrent.*; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; -import org.apache.flink.runtime.concurrent.impl.FlinkFuture; +import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.concurrent.FlinkFutureException; +import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.resourcemanager.SlotRequest; @@ -44,7 +44,7 @@ import java.util.Arrays; import java.util.UUID; -import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -114,7 +114,7 @@ public void testTaskManagerUnregistration() throws Exception { any(AllocationID.class), anyString(), eq(leaderId), - any(Time.class))).thenReturn(new FlinkCompletableFuture()); + any(Time.class))).thenReturn(new CompletableFuture<>()); final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(taskExecutorGateway); @@ -241,7 +241,7 @@ public void testSlotRequestWithFreeSlot() throws Exception { eq(allocationId), anyString(), eq(leaderId), - any(Time.class))).thenReturn(FlinkCompletableFuture.completed(Acknowledge.get())); + any(Time.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(taskExecutorGateway); @@ -280,7 +280,7 @@ public void testUnregisterPendingSlotRequest() throws Exception { any(AllocationID.class), anyString(), eq(leaderId), - any(Time.class))).thenReturn(new FlinkCompletableFuture()); + any(Time.class))).thenReturn(new CompletableFuture<>()); final ResourceProfile resourceProfile = new ResourceProfile(1.0, 1); final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile); @@ -338,7 +338,7 @@ public void testFulfillingPendingSlotRequest() throws Exception { eq(allocationId), anyString(), eq(leaderId), - any(Time.class))).thenReturn(FlinkCompletableFuture.completed(Acknowledge.get())); + any(Time.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(taskExecutorGateway); @@ -482,7 +482,7 @@ public void testDuplicatePendingSlotRequestAfterSuccessfulAllocation() throws Ex any(AllocationID.class), anyString(), eq(leaderId), - any(Time.class))).thenReturn(FlinkCompletableFuture.completed(Acknowledge.get())); + any(Time.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(taskExecutorGateway); @@ -527,7 +527,7 @@ public void testAcceptingDuplicateSlotRequestAfterAllocationRelease() throws Exc any(AllocationID.class), anyString(), eq(leaderId), - any(Time.class))).thenReturn(FlinkCompletableFuture.completed(Acknowledge.get())); + any(Time.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(taskExecutorGateway); @@ -747,8 +747,8 @@ public void testTaskManagerSlotRequestTimeoutHandling() throws Exception { final AllocationID allocationId = new AllocationID(); final ResourceProfile resourceProfile = new ResourceProfile(42.0, 1337); final SlotRequest slotRequest = new SlotRequest(jobId, allocationId, resourceProfile, "foobar"); - final FlinkCompletableFuture slotRequestFuture1 = new FlinkCompletableFuture<>(); - final FlinkCompletableFuture slotRequestFuture2 = new FlinkCompletableFuture<>(); + final CompletableFuture slotRequestFuture1 = new CompletableFuture<>(); + final CompletableFuture slotRequestFuture2 = new CompletableFuture<>(); final TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class); when(taskExecutorGateway.requestSlot( @@ -826,7 +826,7 @@ public void testSlotReportWhileActiveSlotRequest() throws Exception { final AllocationID allocationId = new AllocationID(); final ResourceProfile resourceProfile = new ResourceProfile(42.0, 1337); final SlotRequest slotRequest = new SlotRequest(jobId, allocationId, resourceProfile, "foobar"); - final FlinkCompletableFuture slotRequestFuture1 = new FlinkCompletableFuture<>(); + final CompletableFuture slotRequestFuture1 = new CompletableFuture<>(); final TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class); when(taskExecutorGateway.requestSlot( @@ -835,7 +835,7 @@ public void testSlotReportWhileActiveSlotRequest() throws Exception { eq(allocationId), anyString(), any(UUID.class), - any(Time.class))).thenReturn(slotRequestFuture1, FlinkCompletableFuture.completed(Acknowledge.get())); + any(Time.class))).thenReturn(slotRequestFuture1, CompletableFuture.completedFuture(Acknowledge.get())); final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(taskExecutorGateway); @@ -856,24 +856,21 @@ public void testSlotReportWhileActiveSlotRequest() throws Exception { slotManager.start(leaderId, mainThreadExecutor, resourceManagerActions); - Future registrationFuture = FlinkFuture.supplyAsync(new Callable() { - @Override - public Void call() throws Exception { + CompletableFuture registrationFuture = CompletableFuture.supplyAsync( + () -> { slotManager.registerTaskManager(taskManagerConnection, slotReport); return null; - } - }, mainThreadExecutor) - .thenAccept(new AcceptFunction() { - @Override - public void accept(Void value) { + }, + mainThreadExecutor) + .thenAccept( + (Object value) -> { try { slotManager.registerSlotRequest(slotRequest); } catch (SlotManagerException e) { throw new RuntimeException("Could not register slots.", e); } - } - }); + }); // check that no exception has been thrown registrationFuture.get(); @@ -891,12 +888,9 @@ public void accept(Void value) { final SlotID requestedSlotId = slotIdCaptor.getValue(); final SlotID freeSlotId = requestedSlotId.equals(slotId1) ? slotId2 : slotId1; - Future freeSlotFuture = FlinkFuture.supplyAsync(new Callable() { - @Override - public Boolean call() throws Exception { - return slotManager.getSlot(freeSlotId).isFree(); - } - }, mainThreadExecutor); + CompletableFuture freeSlotFuture = CompletableFuture.supplyAsync( + () -> slotManager.getSlot(freeSlotId).isFree(), + mainThreadExecutor); assertTrue(freeSlotFuture.get()); @@ -904,15 +898,10 @@ public Boolean call() throws Exception { final SlotStatus newSlotStatus2 = new SlotStatus(freeSlotId, resourceProfile); final SlotReport newSlotReport = new SlotReport(Arrays.asList(newSlotStatus1, newSlotStatus2)); - FlinkFuture.supplyAsync(new Callable() { - @Override - public Void call() throws Exception { - // this should update the slot with the pending slot request triggering the reassignment of it - slotManager.reportSlotStatus(taskManagerConnection.getInstanceID(), newSlotReport); - - return null; - } - }, mainThreadExecutor); + CompletableFuture.supplyAsync( + // this should update the slot with the pending slot request triggering the reassignment of it + () -> slotManager.reportSlotStatus(taskManagerConnection.getInstanceID(), newSlotReport), + mainThreadExecutor); verify(taskExecutorGateway, timeout(verifyTimeout).times(2)).requestSlot( slotIdCaptor.capture(), @@ -926,12 +915,9 @@ public Void call() throws Exception { assertEquals(slotId2, requestedSlotId2); - Future requestedSlotFuture = FlinkFuture.supplyAsync(new Callable() { - @Override - public TaskManagerSlot call() throws Exception { - return slotManager.getSlot(requestedSlotId2); - } - }, mainThreadExecutor); + CompletableFuture requestedSlotFuture = CompletableFuture.supplyAsync( + () -> slotManager.getSlot(requestedSlotId2), + mainThreadExecutor); TaskManagerSlot slot = requestedSlotFuture.get(); @@ -967,7 +953,7 @@ public void testTimeoutForUnusedTaskManager() throws Exception { eq(allocationId), anyString(), eq(leaderId), - any(Time.class))).thenReturn(FlinkCompletableFuture.completed(Acknowledge.get())); + any(Time.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(taskExecutorGateway); @@ -987,20 +973,16 @@ public void testTimeoutForUnusedTaskManager() throws Exception { slotManager.start(leaderId, mainThreadExecutor, resourceManagerActions); - FlinkFuture.supplyAsync(new Callable() { - @Override - public Void call() throws Exception { - slotManager.registerSlotRequest(slotRequest); - - return null; - } - }, mainThreadExecutor) - .thenAccept(new AcceptFunction() { - @Override - public void accept(Void value) { - slotManager.registerTaskManager(taskManagerConnection, initialSlotReport); - } - }); + CompletableFuture.supplyAsync( + () -> { + try { + return slotManager.registerSlotRequest(slotRequest); + } catch (SlotManagerException e) { + throw new FlinkFutureException(e); + } + }, + mainThreadExecutor) + .thenAccept((Object value) -> slotManager.registerTaskManager(taskManagerConnection, initialSlotReport)); ArgumentCaptor slotIdArgumentCaptor = ArgumentCaptor.forClass(SlotID.class); @@ -1012,44 +994,28 @@ public void accept(Void value) { eq(leaderId), any(Time.class)); - Future idleFuture = FlinkFuture.supplyAsync(new Callable() { - @Override - public Boolean call() throws Exception { - return slotManager.isTaskManagerIdle(taskManagerConnection.getInstanceID()); - } - }, mainThreadExecutor); + CompletableFuture idleFuture = CompletableFuture.supplyAsync( + () -> slotManager.isTaskManagerIdle(taskManagerConnection.getInstanceID()), + mainThreadExecutor); // check that the TaskManaer is not idle assertFalse(idleFuture.get()); final SlotID slotId = slotIdArgumentCaptor.getValue(); - Future slotFuture = FlinkFuture.supplyAsync(new Callable() { - @Override - public TaskManagerSlot call() throws Exception { - return slotManager.getSlot(slotId); - } - }, mainThreadExecutor); + CompletableFuture slotFuture = CompletableFuture.supplyAsync( + () -> slotManager.getSlot(slotId), + mainThreadExecutor); TaskManagerSlot slot = slotFuture.get(); assertTrue(slot.isAllocated()); assertEquals(allocationId, slot.getAllocationId()); - Future idleFuture2 = FlinkFuture.supplyAsync(new Callable() { - @Override - public Void call() throws Exception { - slotManager.freeSlot(slotId, allocationId); - - return null; - } - }, mainThreadExecutor) - .thenApply(new ApplyFunction() { - @Override - public Boolean apply(Void value) { - return slotManager.isTaskManagerIdle(taskManagerConnection.getInstanceID()); - } - }); + CompletableFuture idleFuture2 = CompletableFuture.runAsync( + () -> slotManager.freeSlot(slotId, allocationId), + mainThreadExecutor) + .thenApply((Object value) -> slotManager.isTaskManagerIdle(taskManagerConnection.getInstanceID())); assertTrue(idleFuture2.get()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java index a1ab1ab0dabd1..844e1597f7c94 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java @@ -26,7 +26,6 @@ import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; -import org.apache.flink.runtime.concurrent.impl.FlinkFuture; import org.apache.flink.runtime.resourcemanager.SlotRequest; import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection; import org.apache.flink.runtime.taskexecutor.SlotReport; @@ -41,6 +40,7 @@ import java.util.Collections; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -104,7 +104,7 @@ public void testSlotsUnavailableRequest() throws Exception { Mockito.when( taskExecutorGateway .requestSlot(any(SlotID.class), any(JobID.class), any(AllocationID.class), any(String.class), any(UUID.class), any(Time.class))) - .thenReturn(mock(FlinkFuture.class)); + .thenReturn(mock(CompletableFuture.class)); final ResourceID resourceID = ResourceID.generate(); final SlotID slotID = new SlotID(resourceID, 0); @@ -139,7 +139,7 @@ public void testSlotAvailableRequest() throws Exception { Mockito.when( taskExecutorGateway .requestSlot(any(SlotID.class), any(JobID.class), any(AllocationID.class), any(String.class), any(UUID.class), any(Time.class))) - .thenReturn(mock(FlinkFuture.class)); + .thenReturn(mock(CompletableFuture.class)); try (SlotManager slotManager = new SlotManager( scheduledExecutor, 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 e636d6c7af448..4be5257171ab7 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,14 +23,13 @@ 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.Future; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.Test; -import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantLock; @@ -90,9 +89,8 @@ public void run() { }); } - Future result = testEndpoint.callAsync(new Callable() { - @Override - public String call() throws Exception { + CompletableFuture result = testEndpoint.callAsync( + () -> { boolean holdsLock = lock.tryLock(); if (holdsLock) { lock.unlock(); @@ -100,8 +98,8 @@ public String call() throws Exception { concurrentAccess.set(true); } return "test"; - } - }, Time.seconds(30L)); + }, + Time.seconds(30L)); String str = result.get(30, TimeUnit.SECONDS); assertEquals("test", str); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java index bbde33156b933..07dadae9ded4a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.util.ReflectionUtil; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -42,6 +41,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -74,7 +74,7 @@ public class RpcCompletenessTest extends TestLogger { private static Logger LOG = LoggerFactory.getLogger(RpcCompletenessTest.class); - private static final Class futureClass = Future.class; + private static final Class futureClass = CompletableFuture.class; private static final Class timeoutClass = Time.class; @Test @@ -195,7 +195,7 @@ private void checkCompleteness(Class rpcEndpoint, Class - *
  • It checks whether the return type is void or a {@link Future} wrapping the actual result.
  • + *
  • It checks whether the return type is void or a {@link CompletableFuture} wrapping the actual result.
  • *
  • It checks that the method's parameter list contains at most one parameter annotated with {@link RpcTimeout}.
  • * * 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 e05c8d800b935..4220fff935aa9 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,7 +23,6 @@ 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.Future; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; @@ -33,6 +32,7 @@ import scala.Option; import scala.Tuple2; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -57,7 +57,7 @@ public void testConnectFailure() { // can only pass if the connection problem is not recognized merely via a timeout rpcService = new AkkaRpcService(actorSystem, Time.of(10000000, TimeUnit.SECONDS)); - Future future = rpcService.connect("foo.bar.com.test.invalid", TaskExecutorGateway.class); + CompletableFuture future = rpcService.connect("foo.bar.com.test.invalid", TaskExecutorGateway.class); future.get(10000000, TimeUnit.SECONDS); fail("should never complete normally"); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java index 03fe84b6ddbc7..ccf0acde05f2d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java @@ -18,10 +18,7 @@ package org.apache.flink.runtime.rpc; -import org.apache.flink.runtime.concurrent.CompletableFuture; -import org.apache.flink.runtime.concurrent.Future; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; - +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -77,8 +74,8 @@ public String getHostname() { // utilities // ------------------------------------------------------------------------ - public Future futureWithTimeout(long timeoutMillis) { - FlinkCompletableFuture future = new FlinkCompletableFuture<>(); + public CompletableFuture futureWithTimeout(long timeoutMillis) { + CompletableFuture future = new CompletableFuture<>(); executor.schedule(new FutureTimeout(future), timeoutMillis, TimeUnit.MILLISECONDS); return future; } 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 b56bf6b88de12..68cc9c7b0a45f 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 @@ -21,10 +21,10 @@ 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.Future; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -92,20 +92,19 @@ public void registerGateway(String address, RpcGateway gateway) { } @Override - public Future connect(String address, Class clazz) { + public CompletableFuture connect(String address, Class clazz) { RpcGateway gateway = registeredConnections.get(address); if (gateway != null) { if (clazz.isAssignableFrom(gateway.getClass())) { @SuppressWarnings("unchecked") C typedGateway = (C) gateway; - return FlinkCompletableFuture.completed(typedGateway); + return CompletableFuture.completedFuture(typedGateway); } else { - return FlinkCompletableFuture.completedExceptionally( - new Exception("Gateway registered under " + address + " is not of type " + clazz)); + return FutureUtils.completedExceptionally(new Exception("Gateway registered under " + address + " is not of type " + clazz)); } } else { - return FlinkCompletableFuture.completedExceptionally(new Exception("No gateway registered under that name")); + return FutureUtils.completedExceptionally(new Exception("No gateway registered under that name")); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java index ac3f40bbbcce4..d01fa48265db0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java @@ -20,11 +20,9 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.runtime.concurrent.CompletableFuture; -import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.util.DirectExecutorService; import org.apache.flink.util.Preconditions; @@ -36,10 +34,12 @@ import java.util.List; import java.util.UUID; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Delayed; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; +import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; @@ -65,7 +65,7 @@ public TestingSerialRpcService() { executorService = new DirectExecutorService(); scheduledExecutorService = new ScheduledThreadPoolExecutor(1); this.registeredConnections = new ConcurrentHashMap<>(16); - this.terminationFuture = new FlinkCompletableFuture<>(); + this.terminationFuture = new CompletableFuture<>(); this.scheduledExecutorServiceAdapter = new ScheduledExecutorServiceAdapter(scheduledExecutorService); } @@ -88,13 +88,13 @@ public void execute(Runnable runnable) { } @Override - public Future execute(Callable callable) { + public CompletableFuture execute(Callable callable) { try { T result = callable.call(); - return FlinkCompletableFuture.completed(result); + return CompletableFuture.completedFuture(result); } catch (Exception e) { - return FlinkCompletableFuture.completedExceptionally(e); + return FutureUtils.completedExceptionally(e); } } @@ -134,7 +134,7 @@ public void stopService() { } @Override - public Future getTerminationFuture() { + public CompletableFuture getTerminationFuture() { return terminationFuture; } @@ -178,20 +178,19 @@ public int getPort() { } @Override - public Future connect(String address, Class clazz) { + public CompletableFuture connect(String address, Class clazz) { RpcGateway gateway = registeredConnections.get(address); if (gateway != null) { if (clazz.isAssignableFrom(gateway.getClass())) { @SuppressWarnings("unchecked") C typedGateway = (C) gateway; - return FlinkCompletableFuture.completed(typedGateway); + return CompletableFuture.completedFuture(typedGateway); } else { - return FlinkCompletableFuture.completedExceptionally( - new Exception("Gateway registered under " + address + " is not of type " + clazz)); + return FutureUtils.completedExceptionally(new Exception("Gateway registered under " + address + " is not of type " + clazz)); } } else { - return FlinkCompletableFuture.completedExceptionally(new Exception("No gateway registered under that name")); + return FutureUtils.completedExceptionally(new Exception("No gateway registered under that name")); } } @@ -251,12 +250,12 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl Class returnType = method.getReturnType(); - if (returnType.equals(Future.class)) { + if (returnType.equals(CompletableFuture.class)) { try { Object result = handleRpcInvocationSync(methodName, filteredArguments.f0, filteredArguments.f1, futureTimeout); - return FlinkCompletableFuture.completed(result); + return CompletableFuture.completedFuture(result); } catch (Throwable e) { - return FlinkCompletableFuture.completedExceptionally(e); + return FutureUtils.completedExceptionally(e); } } else { return handleRpcInvocationSync(methodName, filteredArguments.f0, filteredArguments.f1, futureTimeout); @@ -290,11 +289,11 @@ public void runAsync(Runnable runnable) { } @Override - public Future callAsync(Callable callable, Time callTimeout) { + public CompletableFuture callAsync(Callable callable, Time callTimeout) { try { - return FlinkCompletableFuture.completed(callable.call()); + return CompletableFuture.completedFuture(callable.call()); } catch (Throwable e) { - return FlinkCompletableFuture.completedExceptionally(e); + return FutureUtils.completedExceptionally(e); } } 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 0b0626731bef1..793d292b56437 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 @@ -21,9 +21,6 @@ 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.Future; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; -import org.apache.flink.runtime.concurrent.impl.FlinkFuture; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcMethod; @@ -37,7 +34,7 @@ import org.junit.AfterClass; import org.junit.Test; -import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import static org.junit.Assert.assertEquals; @@ -74,7 +71,7 @@ public static void shutdown() { public void testAddressResolution() throws Exception { DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(akkaRpcService); - Future futureRpcGateway = akkaRpcService.connect(rpcEndpoint.getAddress(), DummyRpcGateway.class); + CompletableFuture futureRpcGateway = akkaRpcService.connect(rpcEndpoint.getAddress(), DummyRpcGateway.class); DummyRpcGateway rpcGateway = futureRpcGateway.get(timeout.getSize(), timeout.getUnit()); @@ -86,7 +83,7 @@ public void testAddressResolution() throws Exception { */ @Test public void testFailingAddressResolution() throws Exception { - Future futureRpcGateway = akkaRpcService.connect("foobar", DummyRpcGateway.class); + CompletableFuture futureRpcGateway = akkaRpcService.connect("foobar", DummyRpcGateway.class); try { futureRpcGateway.get(timeout.getSize(), timeout.getUnit()); @@ -111,7 +108,7 @@ public void testMessageDiscarding() throws Exception { DummyRpcGateway rpcGateway = rpcEndpoint.getSelf(); // this message should be discarded and completed with an AkkaRpcException - Future result = rpcGateway.foobar(); + CompletableFuture result = rpcGateway.foobar(); try { result.get(timeout.getSize(), timeout.getUnit()); @@ -150,14 +147,14 @@ public void testWrongGatewayEndpointConnection() throws Exception { rpcEndpoint.start(); - Future futureGateway = akkaRpcService.connect(rpcEndpoint.getAddress(), WrongRpcGateway.class); + CompletableFuture futureGateway = akkaRpcService.connect(rpcEndpoint.getAddress(), WrongRpcGateway.class); WrongRpcGateway gateway = futureGateway.get(timeout.getSize(), timeout.getUnit()); // since it is a tell operation we won't receive a RpcConnectionException, it's only logged gateway.tell("foobar"); - Future result = gateway.barfoo(); + CompletableFuture result = gateway.barfoo(); try { result.get(timeout.getSize(), timeout.getUnit()); @@ -178,18 +175,13 @@ public void testRpcEndpointTerminationFuture() throws Exception { final DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(akkaRpcService); rpcEndpoint.start(); - Future terminationFuture = rpcEndpoint.getTerminationFuture(); + CompletableFuture terminationFuture = rpcEndpoint.getTerminationFuture(); assertFalse(terminationFuture.isDone()); - FlinkFuture.supplyAsync(new Callable() { - @Override - public Void call() throws Exception { - rpcEndpoint.shutDown(); - - return null; - } - }, actorSystem.dispatcher()); + CompletableFuture.runAsync( + () -> rpcEndpoint.shutDown(), + actorSystem.dispatcher()); // wait until the rpc endpoint has terminated terminationFuture.get(); @@ -201,7 +193,7 @@ public void testExceptionPropagation() throws Exception { rpcEndpoint.start(); ExceptionalGateway rpcGateway = rpcEndpoint.getSelf(); - Future result = rpcGateway.doStuff(); + CompletableFuture result = rpcGateway.doStuff(); try { result.get(timeout.getSize(), timeout.getUnit()); @@ -220,7 +212,7 @@ public void testExceptionPropagationFuturePiping() throws Exception { rpcEndpoint.start(); ExceptionalGateway rpcGateway = rpcEndpoint.getSelf(); - Future result = rpcGateway.doStuff(); + CompletableFuture result = rpcGateway.doStuff(); try { result.get(timeout.getSize(), timeout.getUnit()); @@ -244,7 +236,7 @@ public void testPostStopExceptionPropagation() throws Exception { rpcEndpoint.shutDown(); - Future terminationFuture = rpcEndpoint.getTerminationFuture(); + CompletableFuture terminationFuture = rpcEndpoint.getTerminationFuture(); try { terminationFuture.get(); @@ -263,7 +255,7 @@ public void testPostStopExecutedByMainThread() throws Exception { simpleRpcEndpoint.shutDown(); - Future 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. @@ -275,11 +267,11 @@ public void testPostStopExecutedByMainThread() throws Exception { // ------------------------------------------------------------------------ private interface DummyRpcGateway extends RpcGateway { - Future foobar(); + CompletableFuture foobar(); } private interface WrongRpcGateway extends RpcGateway { - Future barfoo(); + CompletableFuture barfoo(); void tell(String message); } @@ -304,7 +296,7 @@ public void setFoobar(int value) { // ------------------------------------------------------------------------ private interface ExceptionalGateway extends RpcGateway { - Future doStuff(); + CompletableFuture doStuff(); } private static class ExceptionalEndpoint extends RpcEndpoint { @@ -326,8 +318,8 @@ protected ExceptionalFutureEndpoint(RpcService rpcService) { } @RpcMethod - public Future doStuff() { - final FlinkCompletableFuture future = new FlinkCompletableFuture<>(); + public CompletableFuture doStuff() { + final CompletableFuture future = new CompletableFuture<>(); // complete the future slightly in the, well, future... new Thread() { 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 42f63ef0eebc4..e0d11104a13ce 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 @@ -22,15 +22,14 @@ 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.Future; import org.apache.flink.runtime.concurrent.ScheduledExecutor; -import org.apache.flink.runtime.concurrent.impl.FlinkFuture; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.Test; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ScheduledFuture; @@ -91,26 +90,21 @@ public void run() { public void testExecuteRunnable() throws Exception { final OneShotLatch latch = new OneShotLatch(); - akkaRpcService.execute(new Runnable() { - @Override - public void run() { - latch.trigger(); - } - }); + akkaRpcService.execute(() -> latch.trigger()); latch.await(30L, TimeUnit.SECONDS); } /** * Tests that the {@link AkkaRpcService} can execute callables and returns their result as - * a {@link Future}. + * a {@link CompletableFuture}. */ @Test public void testExecuteCallable() throws InterruptedException, ExecutionException, TimeoutException { final OneShotLatch latch = new OneShotLatch(); final int expected = 42; - Future result = akkaRpcService.execute(new Callable() { + CompletableFuture result = akkaRpcService.execute(new Callable() { @Override public Integer call() throws Exception { latch.trigger(); @@ -145,18 +139,11 @@ public void testTerminationFuture() throws ExecutionException, InterruptedExcept final ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); final AkkaRpcService rpcService = new AkkaRpcService(actorSystem, Time.milliseconds(1000)); - Future terminationFuture = rpcService.getTerminationFuture(); + CompletableFuture terminationFuture = rpcService.getTerminationFuture(); assertFalse(terminationFuture.isDone()); - FlinkFuture.supplyAsync(new Callable() { - @Override - public Void call() throws Exception { - rpcService.stopService(); - - return null; - } - }, actorSystem.dispatcher()); + CompletableFuture.runAsync(() -> rpcService.stopService(), actorSystem.dispatcher()); terminationFuture.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 d640a970f5612..34cf41267a4cc 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 @@ -23,7 +23,6 @@ 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.Future; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcMethod; @@ -35,6 +34,7 @@ import org.junit.Test; import java.io.IOException; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.LinkedBlockingQueue; import static org.junit.Assert.assertThat; @@ -108,7 +108,7 @@ public void testNonSerializableRemoteMessageTransfer() throws Exception { String address = testEndpoint.getAddress(); - Future remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class); + CompletableFuture remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class); TestGateway remoteGateway = remoteGatewayFuture.get(timeout.getSize(), timeout.getUnit()); @@ -129,7 +129,7 @@ public void testSerializableRemoteMessageTransfer() throws Exception { String address = testEndpoint.getAddress(); - Future remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class); + CompletableFuture remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class); TestGateway remoteGateway = remoteGatewayFuture.get(timeout.getSize(), timeout.getUnit()); @@ -153,7 +153,7 @@ public void testMaximumFramesizeRemoteMessageTransfer() throws Exception { String address = testEndpoint.getAddress(); - Future remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class); + CompletableFuture remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class); TestGateway remoteGateway = remoteGatewayFuture.get(timeout.getSize(), timeout.getUnit()); 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 6a0bd8712d258..53c435e274750 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 @@ -26,8 +26,6 @@ import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; -import org.apache.flink.runtime.concurrent.Future; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.filecache.FileCache; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; @@ -62,6 +60,7 @@ import java.net.InetAddress; import java.util.Arrays; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; @@ -159,13 +158,13 @@ public void testSlotAllocation() throws Exception { JobMasterGateway jmGateway = mock(JobMasterGateway.class); when(jmGateway.registerTaskManager(any(String.class), any(TaskManagerLocation.class), eq(jmLeaderId), any(Time.class))) - .thenReturn(FlinkCompletableFuture.completed(new JMTMRegistrationSuccess(taskManagerResourceId, 1234))); + .thenReturn(CompletableFuture.completedFuture(new JMTMRegistrationSuccess(taskManagerResourceId, 1234))); when(jmGateway.getHostname()).thenReturn(jmAddress); when(jmGateway.offerSlots( eq(taskManagerResourceId), any(Iterable.class), eq(jmLeaderId), - any(Time.class))).thenReturn(mock(Future.class, RETURNS_MOCKS)); + any(Time.class))).thenReturn(mock(CompletableFuture.class, RETURNS_MOCKS)); rpcService.registerGateway(rmAddress, resourceManager.getSelf()); @@ -185,7 +184,7 @@ public void testSlotAllocation() throws Exception { // notify the TM about the new RM leader rmLeaderRetrievalService.notifyListener(rmAddress, rmLeaderId); - Future registrationResponseFuture = resourceManager.registerJobManager( + CompletableFuture registrationResponseFuture = resourceManager.registerJobManager( rmLeaderId, jmLeaderId, jmResourceId, 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 b596f75f11f53..43587343a04c8 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 @@ -28,10 +28,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.SlotID; -import org.apache.flink.runtime.concurrent.CompletableFuture; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.concurrent.ScheduledExecutor; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; @@ -65,7 +62,6 @@ import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.query.TaskKvStateRegistry; -import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.rpc.TestingSerialRpcService; @@ -95,6 +91,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import java.util.concurrent.TimeoutException; @@ -168,7 +165,7 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation) thro eq(taskManagerLocation), eq(jmLeaderId), any(Time.class) - )).thenReturn(FlinkCompletableFuture.completed(new JMTMRegistrationSuccess(jmResourceId, blobPort))); + )).thenReturn(java.util.concurrent.CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId, blobPort))); when(jobMasterGateway.getAddress()).thenReturn(jobMasterAddress); when(jobMasterGateway.getHostname()).thenReturn("localhost"); @@ -230,7 +227,7 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { when(rmGateway.registerTaskExecutor( any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) .thenReturn( - FlinkCompletableFuture.completed( + java.util.concurrent.CompletableFuture.completedFuture( new TaskExecutorRegistrationSuccess( new InstanceID(), rmResourceId, @@ -334,7 +331,7 @@ public void testHeartbeatSlotReporting() throws Exception { when(rmGateway.registerTaskExecutor( any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) .thenReturn( - FlinkCompletableFuture.completed( + java.util.concurrent.CompletableFuture.completedFuture( new TaskExecutorRegistrationSuccess( new InstanceID(), rmResourceId, @@ -467,7 +464,7 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class); when(rmGateway.registerTaskExecutor( any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) - .thenReturn(FlinkCompletableFuture.completed(new TaskExecutorRegistrationSuccess( + .thenReturn(java.util.concurrent.CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess( new InstanceID(), resourceManagerResourceId, 10L))); TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class); @@ -539,11 +536,11 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { when(rmGateway1.registerTaskExecutor( any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) - .thenReturn(FlinkCompletableFuture.completed( + .thenReturn(java.util.concurrent.CompletableFuture.completedFuture( new TaskExecutorRegistrationSuccess(new InstanceID(), rmResourceId1, 10L))); when(rmGateway2.registerTaskExecutor( any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) - .thenReturn(FlinkCompletableFuture.completed( + .thenReturn(java.util.concurrent.CompletableFuture.completedFuture( new TaskExecutorRegistrationSuccess(new InstanceID(), rmResourceId2, 10L))); rpc.registerGateway(address1, rmGateway1); @@ -728,7 +725,7 @@ public void testTaskSubmission() throws Exception { taskManager.submitTask(tdd, jobManagerLeaderId); - Future completionFuture = TestInvokable.completableFuture; + CompletableFuture completionFuture = TestInvokable.completableFuture; completionFuture.get(); @@ -744,7 +741,7 @@ public void testTaskSubmission() throws Exception { */ public static class TestInvokable extends AbstractInvokable { - static final CompletableFuture completableFuture = new FlinkCompletableFuture<>(); + static final CompletableFuture completableFuture = new CompletableFuture<>(); @Override public void invoke() throws Exception { @@ -793,7 +790,7 @@ public void testJobLeaderDetection() throws Exception { any(String.class), eq(resourceId), any(SlotReport.class), - any(Time.class))).thenReturn(FlinkCompletableFuture.completed(new TaskExecutorRegistrationSuccess(registrationId, resourceManagerResourceId, 1000L))); + any(Time.class))).thenReturn(java.util.concurrent.CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess(registrationId, resourceManagerResourceId, 1000L))); final String jobManagerAddress = "jm"; final UUID jobManagerLeaderId = UUID.randomUUID(); @@ -807,13 +804,13 @@ public void testJobLeaderDetection() throws Exception { eq(taskManagerLocation), eq(jobManagerLeaderId), any(Time.class) - )).thenReturn(FlinkCompletableFuture.completed(new JMTMRegistrationSuccess(jmResourceId, blobPort))); + )).thenReturn(java.util.concurrent.CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId, blobPort))); when(jobMasterGateway.getHostname()).thenReturn(jobManagerAddress); when(jobMasterGateway.offerSlots( any(ResourceID.class), any(Iterable.class), any(UUID.class), - any(Time.class))).thenReturn(mock(Future.class, RETURNS_MOCKS)); + any(Time.class))).thenReturn(mock(java.util.concurrent.CompletableFuture.class, RETURNS_MOCKS)); rpc.registerGateway(resourceManagerAddress, resourceManagerGateway); rpc.registerGateway(jobManagerAddress, jobMasterGateway); @@ -906,7 +903,7 @@ public void testSlotAcceptance() throws Exception { any(String.class), eq(resourceId), any(SlotReport.class), - any(Time.class))).thenReturn(FlinkCompletableFuture.completed(new TaskExecutorRegistrationSuccess(registrationId, resourceManagerResourceId, 1000L))); + any(Time.class))).thenReturn(java.util.concurrent.CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess(registrationId, resourceManagerResourceId, 1000L))); final ResourceID jmResourceId = new ResourceID(jobManagerAddress); final int blobPort = 42; @@ -923,12 +920,12 @@ public void testSlotAcceptance() throws Exception { eq(taskManagerLocation), eq(jobManagerLeaderId), any(Time.class) - )).thenReturn(FlinkCompletableFuture.completed(new JMTMRegistrationSuccess(jmResourceId, blobPort))); + )).thenReturn(java.util.concurrent.CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId, blobPort))); when(jobMasterGateway.getHostname()).thenReturn(jobManagerAddress); when(jobMasterGateway.offerSlots( any(ResourceID.class), any(Iterable.class), eq(jobManagerLeaderId), any(Time.class))) - .thenReturn(FlinkCompletableFuture.completed((Iterable)Collections.singleton(offer1))); + .thenReturn(java.util.concurrent.CompletableFuture.completedFuture((Iterable)Collections.singleton(offer1))); rpc.registerGateway(resourceManagerAddress, resourceManagerGateway); rpc.registerGateway(jobManagerAddress, jobMasterGateway); @@ -1120,7 +1117,7 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { eq(resourceId), any(SlotReport.class), any(Time.class))).thenReturn( - FlinkCompletableFuture.completed(new TaskExecutorRegistrationSuccess(registrationId, resourceManagerResourceId, 1000L))); + java.util.concurrent.CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess(registrationId, resourceManagerResourceId, 1000L))); final ResourceID jmResourceId = new ResourceID(jobManagerAddress); final int blobPort = 42; @@ -1137,7 +1134,7 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { eq(taskManagerLocation), eq(jobManagerLeaderId), any(Time.class) - )).thenReturn(FlinkCompletableFuture.completed(new JMTMRegistrationSuccess(jmResourceId, blobPort))); + )).thenReturn(java.util.concurrent.CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId, blobPort))); when(jobMasterGateway.getHostname()).thenReturn(jobManagerAddress); @@ -1224,7 +1221,7 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { Collections.emptyList(), Collections.emptyList()); - CompletableFuture> offerResultFuture = new FlinkCompletableFuture<>(); + CompletableFuture> offerResultFuture = new CompletableFuture<>(); // submit task first and then return acceptance response when( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java index e790ea814e63a..fdae2516e5ea6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java @@ -35,8 +35,6 @@ import org.apache.flink.runtime.akka.FlinkUntypedActor; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.clusterframework.types.AllocationID; -import org.apache.flink.runtime.concurrent.CompletableFuture; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; @@ -110,6 +108,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import static org.apache.flink.runtime.messages.JobManagerMessages.RequestPartitionProducerState; @@ -1612,7 +1611,7 @@ public void testFailingScheduleOrUpdateConsumersMessage() throws Exception { Await.result(result, timeout); - org.apache.flink.runtime.concurrent.Future cancelFuture = TestInvokableRecordCancel.gotCanceled(); + CompletableFuture cancelFuture = TestInvokableRecordCancel.gotCanceled(); assertEquals(true, cancelFuture.get()); } finally { @@ -2070,7 +2069,7 @@ public void invoke() throws Exception { public static final class TestInvokableRecordCancel extends AbstractInvokable { private static final Object lock = new Object(); - private static CompletableFuture gotCanceledFuture = new FlinkCompletableFuture<>(); + private static CompletableFuture gotCanceledFuture = new CompletableFuture<>(); @Override public void invoke() throws Exception { @@ -2099,11 +2098,11 @@ public void cancel() { public static void resetGotCanceledFuture() { synchronized (lock) { - gotCanceledFuture = new FlinkCompletableFuture<>(); + gotCanceledFuture = new CompletableFuture<>(); } } - public static org.apache.flink.runtime.concurrent.Future gotCanceled() { + public static CompletableFuture gotCanceled() { synchronized (lock) { return gotCanceledFuture; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/OrderedStreamElementQueueTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/OrderedStreamElementQueueTest.java index 5832b89606bf0..f3b68c4b36155 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/OrderedStreamElementQueueTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/OrderedStreamElementQueueTest.java @@ -18,8 +18,7 @@ package org.apache.flink.streaming.api.operators.async.queue; -import org.apache.flink.runtime.concurrent.Future; -import org.apache.flink.runtime.concurrent.impl.FlinkFuture; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.streaming.api.operators.async.OperatorActions; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -34,7 +33,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -92,17 +91,20 @@ public void testCompletionOrder() throws Exception { queue.put(entry); } - Future> pollOperation = FlinkFuture.supplyAsync(new Callable>() { - @Override - public List call() throws Exception { + CompletableFuture> pollOperation = CompletableFuture.supplyAsync( + () -> { List result = new ArrayList<>(4); while (!queue.isEmpty()) { - result.add(queue.poll()); + try { + result.add(queue.poll()); + } catch (InterruptedException e) { + throw new FlinkFutureException(e); + } } return result; - } - }, executor); + }, + executor); Thread.sleep(10L); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueTest.java index fe9db9597b62f..d3967567b7efd 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueTest.java @@ -18,8 +18,7 @@ package org.apache.flink.streaming.api.operators.async.queue; -import org.apache.flink.runtime.concurrent.Future; -import org.apache.flink.runtime.concurrent.impl.FlinkFuture; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.streaming.api.operators.async.OperatorActions; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -37,7 +36,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -176,14 +175,15 @@ public void testBlockingPut() throws Exception { Assert.assertEquals(1, queue.size()); - Future putOperation = FlinkFuture.supplyAsync(new Callable() { - @Override - public Void call() throws Exception { - queue.put(streamRecordQueueEntry2); - - return null; - } - }, executor); + CompletableFuture putOperation = CompletableFuture.runAsync( + () -> { + try { + queue.put(streamRecordQueueEntry2); + } catch (InterruptedException e) { + throw new FlinkFutureException(e); + } + }, + executor); // give the future a chance to complete Thread.sleep(10L); @@ -215,12 +215,15 @@ public void testBlockingPoll() throws Exception { Assert.assertTrue(queue.isEmpty()); - Future peekOperation = FlinkFuture.supplyAsync(new Callable() { - @Override - public AsyncResult call() throws Exception { - return queue.peekBlockingly(); - } - }, executor); + CompletableFuture peekOperation = CompletableFuture.supplyAsync( + () -> { + try { + return queue.peekBlockingly(); + } catch (InterruptedException e) { + throw new FlinkFutureException(e); + } + }, + executor); Thread.sleep(10L); @@ -236,12 +239,15 @@ public AsyncResult call() throws Exception { Assert.assertEquals(watermarkQueueEntry, queue.poll()); Assert.assertTrue(queue.isEmpty()); - Future pollOperation = FlinkFuture.supplyAsync(new Callable() { - @Override - public AsyncResult call() throws Exception { - return queue.poll(); - } - }, executor); + CompletableFuture pollOperation = CompletableFuture.supplyAsync( + () -> { + try { + return queue.poll(); + } catch (InterruptedException e) { + throw new FlinkFutureException(e); + } + }, + executor); Thread.sleep(10L); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueueTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueueTest.java index ba6ce42259a24..cc0bc309a35df 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueueTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueueTest.java @@ -18,8 +18,7 @@ package org.apache.flink.streaming.api.operators.async.queue; -import org.apache.flink.runtime.concurrent.Future; -import org.apache.flink.runtime.concurrent.impl.FlinkFuture; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.streaming.api.operators.async.OperatorActions; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -35,7 +34,7 @@ import java.util.HashSet; import java.util.List; import java.util.Set; -import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -100,12 +99,15 @@ public void testCompletionOrder() throws Exception { Assert.assertTrue(8 == queue.size()); - Future firstPoll = FlinkFuture.supplyAsync(new Callable() { - @Override - public AsyncResult call() throws Exception { - return queue.poll(); - } - }, executor); + CompletableFuture firstPoll = CompletableFuture.supplyAsync( + () -> { + try { + return queue.poll(); + } catch (InterruptedException e) { + throw new FlinkFutureException(e); + } + }, + executor); // this should not fulfill the poll, because R3 is behind W1 record3.collect(Collections.emptyList()); @@ -118,12 +120,15 @@ public AsyncResult call() throws Exception { Assert.assertEquals(record2, firstPoll.get()); - Future secondPoll = FlinkFuture.supplyAsync(new Callable() { - @Override - public AsyncResult call() throws Exception { - return queue.poll(); - } - }, executor); + CompletableFuture secondPoll = CompletableFuture.supplyAsync( + () -> { + try { + return queue.poll(); + } catch (InterruptedException e) { + throw new FlinkFutureException(e); + } + }, + executor); record6.collect(Collections.emptyList()); record4.collect(Collections.emptyList()); @@ -161,12 +166,15 @@ public AsyncResult call() throws Exception { // only R5 left in the queue Assert.assertTrue(1 == queue.size()); - Future thirdPoll = FlinkFuture.supplyAsync(new Callable() { - @Override - public AsyncResult call() throws Exception { - return queue.poll(); - } - }, executor); + CompletableFuture thirdPoll = CompletableFuture.supplyAsync( + () -> { + try { + return queue.poll(); + } catch (InterruptedException e) { + throw new FlinkFutureException(e); + } + }, + executor); Thread.sleep(10L); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java index f021b389c811e..4f2135d96bc0f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java @@ -28,8 +28,6 @@ import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.concurrent.Executors; -import org.apache.flink.runtime.concurrent.Future; -import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.execution.Environment; @@ -75,6 +73,7 @@ import java.net.URL; import java.util.Collections; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.FutureTask; import static org.junit.Assert.assertEquals; @@ -160,14 +159,9 @@ public void testConcurrentAsyncCheckpointCannotFailFinishedStreamTask() throws E mock(PartitionProducerStateChecker.class), Executors.directExecutor()); - Future taskRun = FlinkCompletableFuture.supplyAsync(new Callable() { - @Override - public Void call() throws Exception { - task.run(); - - return null; - } - }, TestingUtils.defaultExecutor()); + CompletableFuture taskRun = CompletableFuture.runAsync( + () -> task.run(), + TestingUtils.defaultExecutor()); // wait until the stream task started running RUN_LATCH.await(); From 9c45a386bbbd1dffc6106b4f0e73ae0d90551081 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 2 Aug 2017 16:04:16 +0200 Subject: [PATCH 2/3] Fix failing AkkaRpcActorTest --- .../java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java | 2 +- flink-runtime/src/test/resources/log4j-test.properties | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 5c5351d590fb9..584547380888c 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 @@ -220,7 +220,7 @@ private void handleRpcInvocation(RpcInvocation rpcInvocation) { } }); - Patterns.pipe(promise.future(), getContext().dispatcher()); + Patterns.pipe(promise.future(), getContext().dispatcher()).to(getSender()); } else { // tell the sender the result of the computation getSender().tell(new Status.Success(result), getSelf()); diff --git a/flink-runtime/src/test/resources/log4j-test.properties b/flink-runtime/src/test/resources/log4j-test.properties index 7ba163366a631..98f136aca541f 100644 --- a/flink-runtime/src/test/resources/log4j-test.properties +++ b/flink-runtime/src/test/resources/log4j-test.properties @@ -16,7 +16,7 @@ # limitations under the License. ################################################################################ -log4j.rootLogger=OFF, console +log4j.rootLogger=INFO, console # ----------------------------------------------------------------------------- # Console (use 'console') From 279833bedbc2882c9558af23340a1615fc3a274f Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 2 Aug 2017 18:46:05 +0200 Subject: [PATCH 3/3] Address PR comments --- .../flink/runtime/rpc/TestingRpcService.java | 2 +- .../runtime/rpc/TestingSerialRpcService.java | 2 +- .../taskexecutor/TaskExecutorTest.java | 28 +++++++++---------- 3 files changed, 16 insertions(+), 16 deletions(-) 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 68cc9c7b0a45f..14cf35a4d2a4e 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 @@ -104,7 +104,7 @@ public CompletableFuture connect(String address, Class return FutureUtils.completedExceptionally(new Exception("Gateway registered under " + address + " is not of type " + clazz)); } } else { - return FutureUtils.completedExceptionally(new Exception("No gateway registered under that name")); + return FutureUtils.completedExceptionally(new Exception("No gateway registered under " + address + '.')); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java index d01fa48265db0..37349a1f46427 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java @@ -190,7 +190,7 @@ public CompletableFuture connect(String address, Class return FutureUtils.completedExceptionally(new Exception("Gateway registered under " + address + " is not of type " + clazz)); } } else { - return FutureUtils.completedExceptionally(new Exception("No gateway registered under that name")); + return FutureUtils.completedExceptionally(new Exception("No gateway registered under " + address + '.')); } } 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 43587343a04c8..a4f0e03663761 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 @@ -165,7 +165,7 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation) thro eq(taskManagerLocation), eq(jmLeaderId), any(Time.class) - )).thenReturn(java.util.concurrent.CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId, blobPort))); + )).thenReturn(CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId, blobPort))); when(jobMasterGateway.getAddress()).thenReturn(jobMasterAddress); when(jobMasterGateway.getHostname()).thenReturn("localhost"); @@ -227,7 +227,7 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { when(rmGateway.registerTaskExecutor( any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) .thenReturn( - java.util.concurrent.CompletableFuture.completedFuture( + CompletableFuture.completedFuture( new TaskExecutorRegistrationSuccess( new InstanceID(), rmResourceId, @@ -331,7 +331,7 @@ public void testHeartbeatSlotReporting() throws Exception { when(rmGateway.registerTaskExecutor( any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) .thenReturn( - java.util.concurrent.CompletableFuture.completedFuture( + CompletableFuture.completedFuture( new TaskExecutorRegistrationSuccess( new InstanceID(), rmResourceId, @@ -464,7 +464,7 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class); when(rmGateway.registerTaskExecutor( any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) - .thenReturn(java.util.concurrent.CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess( + .thenReturn(CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess( new InstanceID(), resourceManagerResourceId, 10L))); TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class); @@ -536,11 +536,11 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { when(rmGateway1.registerTaskExecutor( any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) - .thenReturn(java.util.concurrent.CompletableFuture.completedFuture( + .thenReturn(CompletableFuture.completedFuture( new TaskExecutorRegistrationSuccess(new InstanceID(), rmResourceId1, 10L))); when(rmGateway2.registerTaskExecutor( any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) - .thenReturn(java.util.concurrent.CompletableFuture.completedFuture( + .thenReturn(CompletableFuture.completedFuture( new TaskExecutorRegistrationSuccess(new InstanceID(), rmResourceId2, 10L))); rpc.registerGateway(address1, rmGateway1); @@ -790,7 +790,7 @@ public void testJobLeaderDetection() throws Exception { any(String.class), eq(resourceId), any(SlotReport.class), - any(Time.class))).thenReturn(java.util.concurrent.CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess(registrationId, resourceManagerResourceId, 1000L))); + any(Time.class))).thenReturn(CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess(registrationId, resourceManagerResourceId, 1000L))); final String jobManagerAddress = "jm"; final UUID jobManagerLeaderId = UUID.randomUUID(); @@ -804,13 +804,13 @@ public void testJobLeaderDetection() throws Exception { eq(taskManagerLocation), eq(jobManagerLeaderId), any(Time.class) - )).thenReturn(java.util.concurrent.CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId, blobPort))); + )).thenReturn(CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId, blobPort))); when(jobMasterGateway.getHostname()).thenReturn(jobManagerAddress); when(jobMasterGateway.offerSlots( any(ResourceID.class), any(Iterable.class), any(UUID.class), - any(Time.class))).thenReturn(mock(java.util.concurrent.CompletableFuture.class, RETURNS_MOCKS)); + any(Time.class))).thenReturn(mock(CompletableFuture.class, RETURNS_MOCKS)); rpc.registerGateway(resourceManagerAddress, resourceManagerGateway); rpc.registerGateway(jobManagerAddress, jobMasterGateway); @@ -903,7 +903,7 @@ public void testSlotAcceptance() throws Exception { any(String.class), eq(resourceId), any(SlotReport.class), - any(Time.class))).thenReturn(java.util.concurrent.CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess(registrationId, resourceManagerResourceId, 1000L))); + any(Time.class))).thenReturn(CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess(registrationId, resourceManagerResourceId, 1000L))); final ResourceID jmResourceId = new ResourceID(jobManagerAddress); final int blobPort = 42; @@ -920,12 +920,12 @@ public void testSlotAcceptance() throws Exception { eq(taskManagerLocation), eq(jobManagerLeaderId), any(Time.class) - )).thenReturn(java.util.concurrent.CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId, blobPort))); + )).thenReturn(CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId, blobPort))); when(jobMasterGateway.getHostname()).thenReturn(jobManagerAddress); when(jobMasterGateway.offerSlots( any(ResourceID.class), any(Iterable.class), eq(jobManagerLeaderId), any(Time.class))) - .thenReturn(java.util.concurrent.CompletableFuture.completedFuture((Iterable)Collections.singleton(offer1))); + .thenReturn(CompletableFuture.completedFuture((Iterable)Collections.singleton(offer1))); rpc.registerGateway(resourceManagerAddress, resourceManagerGateway); rpc.registerGateway(jobManagerAddress, jobMasterGateway); @@ -1117,7 +1117,7 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { eq(resourceId), any(SlotReport.class), any(Time.class))).thenReturn( - java.util.concurrent.CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess(registrationId, resourceManagerResourceId, 1000L))); + CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess(registrationId, resourceManagerResourceId, 1000L))); final ResourceID jmResourceId = new ResourceID(jobManagerAddress); final int blobPort = 42; @@ -1134,7 +1134,7 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { eq(taskManagerLocation), eq(jobManagerLeaderId), any(Time.class) - )).thenReturn(java.util.concurrent.CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId, blobPort))); + )).thenReturn(CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId, blobPort))); when(jobMasterGateway.getHostname()).thenReturn(jobManagerAddress);