From 0e3027d4b87cb5aff5c640ec809f2968b64be732 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 29 Nov 2017 18:12:21 +0100 Subject: [PATCH 1/4] [FLINK-8029] Create WebMonitorEndpoint The WebMonitorEndpoint is the common rest endpoint used for serving the web frontend REST calls. It will be used by the Dispatcher and the JobMaster to fuel the web frontend. This closes #4987. --- .../dispatcher/DispatcherRestEndpoint.java | 348 +-------------- .../flink/runtime/jobmaster/JobMaster.java | 75 ++-- .../jobmaster/JobMasterRestEndpoint.java | 46 ++ .../webmonitor/WebMonitorEndpoint.java | 402 ++++++++++++++++++ 4 files changed, 494 insertions(+), 377 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRestEndpoint.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java index 4a99b9d70dde9..2ab97e2d9cf58 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java @@ -22,97 +22,28 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; -import org.apache.flink.runtime.rest.RestServerEndpoint; import org.apache.flink.runtime.rest.RestServerEndpointConfiguration; import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration; import org.apache.flink.runtime.rest.handler.RestHandlerSpecification; -import org.apache.flink.runtime.rest.handler.cluster.ClusterConfigHandler; -import org.apache.flink.runtime.rest.handler.cluster.ClusterOverviewHandler; -import org.apache.flink.runtime.rest.handler.cluster.DashboardConfigHandler; import org.apache.flink.runtime.rest.handler.job.BlobServerPortHandler; -import org.apache.flink.runtime.rest.handler.job.JobAccumulatorsHandler; -import org.apache.flink.runtime.rest.handler.job.JobConfigHandler; -import org.apache.flink.runtime.rest.handler.job.JobDetailsHandler; -import org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler; -import org.apache.flink.runtime.rest.handler.job.JobIdsHandler; -import org.apache.flink.runtime.rest.handler.job.JobPlanHandler; import org.apache.flink.runtime.rest.handler.job.JobSubmitHandler; import org.apache.flink.runtime.rest.handler.job.JobTerminationHandler; -import org.apache.flink.runtime.rest.handler.job.JobVertexAccumulatorsHandler; -import org.apache.flink.runtime.rest.handler.job.JobsOverviewHandler; -import org.apache.flink.runtime.rest.handler.job.SubtasksTimesHandler; -import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointConfigHandler; -import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointStatisticDetailsHandler; -import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointStatsCache; -import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointingStatisticsHandler; -import org.apache.flink.runtime.rest.handler.job.checkpoints.TaskCheckpointStatisticDetailsHandler; -import org.apache.flink.runtime.rest.handler.job.metrics.JobManagerMetricsHandler; -import org.apache.flink.runtime.rest.handler.job.metrics.JobMetricsHandler; -import org.apache.flink.runtime.rest.handler.job.metrics.JobVertexMetricsHandler; -import org.apache.flink.runtime.rest.handler.job.metrics.SubtaskMetricsHandler; -import org.apache.flink.runtime.rest.handler.job.metrics.TaskManagerMetricsHandler; -import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache; -import org.apache.flink.runtime.rest.handler.legacy.files.StaticFileServerHandler; -import org.apache.flink.runtime.rest.handler.legacy.files.WebContentHandlerSpecification; -import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher; -import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerDetailsHandler; -import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagersHandler; -import org.apache.flink.runtime.rest.messages.ClusterConfigurationInfoHeaders; -import org.apache.flink.runtime.rest.messages.ClusterOverviewHeaders; -import org.apache.flink.runtime.rest.messages.DashboardConfigurationHeaders; -import org.apache.flink.runtime.rest.messages.JobAccumulatorsHeaders; -import org.apache.flink.runtime.rest.messages.JobConfigHeaders; -import org.apache.flink.runtime.rest.messages.JobExceptionsHeaders; -import org.apache.flink.runtime.rest.messages.JobIdsWithStatusesOverviewHeaders; -import org.apache.flink.runtime.rest.messages.JobPlanHeaders; import org.apache.flink.runtime.rest.messages.JobTerminationHeaders; -import org.apache.flink.runtime.rest.messages.JobVertexAccumulatorsHeaders; -import org.apache.flink.runtime.rest.messages.JobsOverviewHeaders; -import org.apache.flink.runtime.rest.messages.SubtasksTimesHeaders; -import org.apache.flink.runtime.rest.messages.checkpoints.CheckpointConfigHeaders; -import org.apache.flink.runtime.rest.messages.checkpoints.CheckpointStatisticDetailsHeaders; -import org.apache.flink.runtime.rest.messages.checkpoints.CheckpointingStatisticsHeaders; -import org.apache.flink.runtime.rest.messages.checkpoints.TaskCheckpointStatisticsHeaders; -import org.apache.flink.runtime.rest.messages.job.JobDetailsHeaders; -import org.apache.flink.runtime.rest.messages.job.metrics.JobManagerMetricsHeaders; -import org.apache.flink.runtime.rest.messages.job.metrics.JobMetricsHeaders; -import org.apache.flink.runtime.rest.messages.job.metrics.JobVertexMetricsHeaders; -import org.apache.flink.runtime.rest.messages.job.metrics.SubtaskMetricsHeaders; -import org.apache.flink.runtime.rest.messages.job.metrics.TaskManagerMetricsHeaders; -import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerDetailsHeaders; -import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersHeaders; -import org.apache.flink.runtime.webmonitor.WebMonitorUtils; +import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever; -import org.apache.flink.util.FileUtils; -import org.apache.flink.util.Preconditions; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandler; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; /** * REST endpoint for the {@link Dispatcher} component. */ -public class DispatcherRestEndpoint extends RestServerEndpoint { - - private final GatewayRetriever leaderRetriever; - private final Configuration clusterConfiguration; - private final RestHandlerConfiguration restConfiguration; - private final GatewayRetriever resourceManagerRetriever; - private final Executor executor; - - private final ExecutionGraphCache executionGraphCache; - private final CheckpointStatsCache checkpointStatsCache; - - private final MetricFetcher metricFetcher; +public class DispatcherRestEndpoint extends WebMonitorEndpoint { public DispatcherRestEndpoint( RestServerEndpointConfiguration endpointConfiguration, @@ -122,71 +53,25 @@ public DispatcherRestEndpoint( GatewayRetriever resourceManagerRetriever, Executor executor, MetricQueryServiceRetriever metricQueryServiceRetriever) { - super(endpointConfiguration); - this.leaderRetriever = Preconditions.checkNotNull(leaderRetriever); - this.clusterConfiguration = Preconditions.checkNotNull(clusterConfiguration); - this.restConfiguration = Preconditions.checkNotNull(restConfiguration); - this.resourceManagerRetriever = Preconditions.checkNotNull(resourceManagerRetriever); - this.executor = Preconditions.checkNotNull(executor); - - this.executionGraphCache = new ExecutionGraphCache( - restConfiguration.getTimeout(), - Time.milliseconds(restConfiguration.getRefreshInterval())); - - this.checkpointStatsCache = new CheckpointStatsCache( - restConfiguration.getMaxCheckpointStatisticCacheEntries()); - - this.metricFetcher = new MetricFetcher<>( + super( + endpointConfiguration, leaderRetriever, - metricQueryServiceRetriever, + clusterConfiguration, + restConfiguration, + resourceManagerRetriever, executor, - restConfiguration.getTimeout()); + metricQueryServiceRetriever); } @Override protected List> initializeHandlers(CompletableFuture restAddressFuture) { - ArrayList> handlers = new ArrayList<>(3); + List> handlers = super.initializeHandlers(restAddressFuture); + + // Add the Dispatcher specific handlers final Time timeout = restConfiguration.getTimeout(); final Map responseHeaders = restConfiguration.getResponseHeaders(); - ClusterOverviewHandler clusterOverviewHandler = new ClusterOverviewHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - ClusterOverviewHeaders.getInstance()); - - DashboardConfigHandler dashboardConfigHandler = new DashboardConfigHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - DashboardConfigurationHeaders.getInstance(), - restConfiguration.getRefreshInterval()); - - JobIdsHandler jobIdsHandler = new JobIdsHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - JobIdsWithStatusesOverviewHeaders.getInstance()); - - JobsOverviewHandler jobsOverviewHandler = new JobsOverviewHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - JobsOverviewHeaders.getInstance()); - - ClusterConfigHandler clusterConfigurationHandler = new ClusterConfigHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - ClusterConfigurationInfoHeaders.getInstance(), - clusterConfiguration); - JobTerminationHandler jobTerminationHandler = new JobTerminationHandler( restAddressFuture, leaderRetriever, @@ -194,80 +79,6 @@ protected List> initiali responseHeaders, JobTerminationHeaders.getInstance()); - JobConfigHandler jobConfigHandler = new JobConfigHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - JobConfigHeaders.getInstance(), - executionGraphCache, - executor); - - CheckpointConfigHandler checkpointConfigHandler = new CheckpointConfigHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - CheckpointConfigHeaders.getInstance(), - executionGraphCache, - executor); - - CheckpointingStatisticsHandler checkpointStatisticsHandler = new CheckpointingStatisticsHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - CheckpointingStatisticsHeaders.getInstance(), - executionGraphCache, - executor); - - CheckpointStatisticDetailsHandler checkpointStatisticDetailsHandler = new CheckpointStatisticDetailsHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - CheckpointStatisticDetailsHeaders.getInstance(), - executionGraphCache, - executor, - checkpointStatsCache); - - JobPlanHandler jobPlanHandler = new JobPlanHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - JobPlanHeaders.getInstance(), - executionGraphCache, - executor); - - TaskCheckpointStatisticDetailsHandler taskCheckpointStatisticDetailsHandler = new TaskCheckpointStatisticDetailsHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - TaskCheckpointStatisticsHeaders.getInstance(), - executionGraphCache, - executor, - checkpointStatsCache); - - JobExceptionsHandler jobExceptionsHandler = new JobExceptionsHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - JobExceptionsHeaders.getInstance(), - executionGraphCache, - executor); - - JobVertexAccumulatorsHandler jobVertexAccumulatorsHandler = new JobVertexAccumulatorsHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - JobVertexAccumulatorsHeaders.getInstance(), - executionGraphCache, - executor); - BlobServerPortHandler blobServerPortHandler = new BlobServerPortHandler( restAddressFuture, leaderRetriever, @@ -280,147 +91,10 @@ protected List> initiali timeout, responseHeaders); - TaskManagersHandler taskManagersHandler = new TaskManagersHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - TaskManagersHeaders.getInstance(), - resourceManagerRetriever); - - TaskManagerDetailsHandler taskManagerDetailsHandler = new TaskManagerDetailsHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - TaskManagerDetailsHeaders.getInstance(), - resourceManagerRetriever, - metricFetcher); - - final JobDetailsHandler jobDetailsHandler = new JobDetailsHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - JobDetailsHeaders.getInstance(), - executionGraphCache, - executor, - metricFetcher); - - JobAccumulatorsHandler jobAccumulatorsHandler = new JobAccumulatorsHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - JobAccumulatorsHeaders.getInstance(), - executionGraphCache, - executor); - - SubtasksTimesHandler subtasksTimesHandler = new SubtasksTimesHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - SubtasksTimesHeaders.getInstance(), - executionGraphCache, - executor); - - final JobVertexMetricsHandler jobVertexMetricsHandler = new JobVertexMetricsHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - metricFetcher); - - final JobMetricsHandler jobMetricsHandler = new JobMetricsHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - metricFetcher); - - final SubtaskMetricsHandler subtaskMetricsHandler = new SubtaskMetricsHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - metricFetcher); - - final TaskManagerMetricsHandler taskManagerMetricsHandler = new TaskManagerMetricsHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - metricFetcher); - - final JobManagerMetricsHandler jobManagerMetricsHandler = new JobManagerMetricsHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - metricFetcher); - - final File tmpDir = restConfiguration.getTmpDir(); - - Optional> optWebContent; - - try { - optWebContent = WebMonitorUtils.tryLoadWebContent( - leaderRetriever, - restAddressFuture, - timeout, - tmpDir); - } catch (IOException e) { - log.warn("Could not load web content handler.", e); - optWebContent = Optional.empty(); - } - - handlers.add(Tuple2.of(ClusterOverviewHeaders.getInstance(), clusterOverviewHandler)); - handlers.add(Tuple2.of(ClusterConfigurationInfoHeaders.getInstance(), clusterConfigurationHandler)); - handlers.add(Tuple2.of(DashboardConfigurationHeaders.getInstance(), dashboardConfigHandler)); - handlers.add(Tuple2.of(JobIdsWithStatusesOverviewHeaders.getInstance(), jobIdsHandler)); - handlers.add(Tuple2.of(JobsOverviewHeaders.getInstance(), jobsOverviewHandler)); handlers.add(Tuple2.of(JobTerminationHeaders.getInstance(), jobTerminationHandler)); - handlers.add(Tuple2.of(JobConfigHeaders.getInstance(), jobConfigHandler)); - handlers.add(Tuple2.of(CheckpointConfigHeaders.getInstance(), checkpointConfigHandler)); - handlers.add(Tuple2.of(CheckpointingStatisticsHeaders.getInstance(), checkpointStatisticsHandler)); - handlers.add(Tuple2.of(CheckpointStatisticDetailsHeaders.getInstance(), checkpointStatisticDetailsHandler)); - handlers.add(Tuple2.of(JobPlanHeaders.getInstance(), jobPlanHandler)); - handlers.add(Tuple2.of(TaskCheckpointStatisticsHeaders.getInstance(), taskCheckpointStatisticDetailsHandler)); - handlers.add(Tuple2.of(JobExceptionsHeaders.getInstance(), jobExceptionsHandler)); - handlers.add(Tuple2.of(JobVertexAccumulatorsHeaders.getInstance(), jobVertexAccumulatorsHandler)); - handlers.add(Tuple2.of(JobDetailsHeaders.getInstance(), jobDetailsHandler)); - handlers.add(Tuple2.of(JobAccumulatorsHeaders.getInstance(), jobAccumulatorsHandler)); handlers.add(Tuple2.of(blobServerPortHandler.getMessageHeaders(), blobServerPortHandler)); handlers.add(Tuple2.of(jobSubmitHandler.getMessageHeaders(), jobSubmitHandler)); - handlers.add(Tuple2.of(TaskManagersHeaders.getInstance(), taskManagersHandler)); - handlers.add(Tuple2.of(TaskManagerDetailsHeaders.getInstance(), taskManagerDetailsHandler)); - handlers.add(Tuple2.of(SubtasksTimesHeaders.getInstance(), subtasksTimesHandler)); - handlers.add(Tuple2.of(JobVertexMetricsHeaders.getInstance(), jobVertexMetricsHandler)); - handlers.add(Tuple2.of(JobMetricsHeaders.getInstance(), jobMetricsHandler)); - handlers.add(Tuple2.of(SubtaskMetricsHeaders.getInstance(), subtaskMetricsHandler)); - handlers.add(Tuple2.of(TaskManagerMetricsHeaders.getInstance(), taskManagerMetricsHandler)); - handlers.add(Tuple2.of(JobManagerMetricsHeaders.getInstance(), jobManagerMetricsHandler)); - - optWebContent.ifPresent( - webContent -> handlers.add(Tuple2.of(WebContentHandlerSpecification.getInstance(), webContent))); return handlers; } - - @Override - public void shutdown(Time timeout) { - super.shutdown(timeout); - - executionGraphCache.close(); - - final File tmpDir = restConfiguration.getTmpDir(); - - try { - log.info("Removing cache directory {}", tmpDir); - FileUtils.deleteDirectory(tmpDir); - } catch (Throwable t) { - log.warn("Error while deleting cache directory {}", tmpDir, t); - } - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index 6d0de74ba1ff4..1d4eb6abc7b4a 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 @@ -127,8 +127,8 @@ /** * JobMaster implementation. The job master is responsible for the execution of a single * {@link JobGraph}. - *

- * It offers the following methods as part of its rpc interface to interact with the JobMaster + * + *

It offers the following methods as part of its rpc interface to interact with the JobMaster * remotely: *

    *
  • {@link #updateTaskExecutionState} updates the task execution state for @@ -137,7 +137,7 @@ */ public class JobMaster extends FencedRpcEndpoint implements JobMasterGateway { - /** Default names for Flink's distributed components */ + /** Default names for Flink's distributed components. */ public static final String JOB_MANAGER_NAME = "jobmanager"; public static final String ARCHIVE_NAME = "archive"; @@ -147,36 +147,36 @@ public class JobMaster extends FencedRpcEndpoint implements JobMast private final ResourceID resourceId; - /** Logical representation of the job */ + /** Logical representation of the job. */ private final JobGraph jobGraph; - /** Configuration of the JobManager */ + /** Configuration of the JobManager. */ private final Configuration configuration; private final Time rpcTimeout; - /** Service to contend for and retrieve the leadership of JM and RM */ + /** Service to contend for and retrieve the leadership of JM and RM. */ private final HighAvailabilityServices highAvailabilityServices; - /** Blob server used across jobs */ + /** Blob server used across jobs. */ private final BlobServer blobServer; - /** Blob library cache manager used across jobs */ + /** Blob library cache manager used across jobs. */ private final BlobLibraryCacheManager libraryCacheManager; - /** The metrics for the JobManager itself */ + /** The metrics for the JobManager itself. */ private final MetricGroup jobManagerMetricGroup; - /** The metrics for the job */ + /** The metrics for the job. */ private final MetricGroup jobMetricGroup; - /** The heartbeat manager with task managers */ + /** The heartbeat manager with task managers. */ private final HeartbeatManager taskManagerHeartbeatManager; - /** The heartbeat manager with resource manager */ + /** The heartbeat manager with resource manager. */ private final HeartbeatManager resourceManagerHeartbeatManager; - /** The execution context which is used to execute futures */ + /** The execution context which is used to execute futures. */ private final Executor executor; private final OnCompletionActions jobCompletionActions; @@ -185,7 +185,7 @@ public class JobMaster extends FencedRpcEndpoint implements JobMast private final ClassLoader userCodeLoader; - /** The execution graph of this job */ + /** The execution graph of this job. */ private final ExecutionGraph executionGraph; private final SlotPool slotPool; @@ -198,10 +198,10 @@ public class JobMaster extends FencedRpcEndpoint implements JobMast // --------- ResourceManager -------- - /** Leader retriever service used to locate ResourceManager's address */ + /** Leader retriever service used to locate ResourceManager's address. */ private LeaderRetrievalService resourceManagerLeaderRetriever; - /** Connection with ResourceManager, null if not located address yet or we close it initiative */ + /** Connection with ResourceManager, null if not located address yet or we close it initiative. */ private ResourceManagerConnection resourceManagerConnection; // --------- TaskManagers -------- @@ -320,7 +320,6 @@ public JobMaster( // Lifecycle management //---------------------------------------------------------------------------------------------- - @Override public void start() { throw new UnsupportedOperationException("Should never call start() without leader ID"); @@ -405,8 +404,7 @@ public CompletableFuture stop(Time timeout) { */ @Override public CompletableFuture updateTaskExecutionState( - final TaskExecutionState taskExecutionState) - { + final TaskExecutionState taskExecutionState) { checkNotNull(taskExecutionState, "taskExecutionState"); if (executionGraph.updateState(taskExecutionState)) { @@ -534,8 +532,12 @@ public void acknowledgeCheckpoint( final TaskStateSnapshot checkpointState) { final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator(); - final AcknowledgeCheckpoint ackMessage = - new AcknowledgeCheckpoint(jobID, executionAttemptID, checkpointId, checkpointMetrics, checkpointState); + final AcknowledgeCheckpoint ackMessage = new AcknowledgeCheckpoint( + jobID, + executionAttemptID, + checkpointId, + checkpointMetrics, + checkpointState); if (checkpointCoordinator != null) { getRpcService().execute(new Runnable() { @@ -560,8 +562,7 @@ public void declineCheckpoint( final JobID jobID, final ExecutionAttemptID executionAttemptID, final long checkpointID, - final Throwable reason) - { + final Throwable reason) { final DeclineCheckpoint decline = new DeclineCheckpoint( jobID, executionAttemptID, checkpointID, reason); final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator(); @@ -605,8 +606,7 @@ public void notifyKvStateRegistered( final KeyGroupRange keyGroupRange, final String registrationName, final KvStateID kvStateId, - final InetSocketAddress kvStateServerAddress) - { + final InetSocketAddress kvStateServerAddress) { if (log.isDebugEnabled()) { log.debug("Key value state registered for job {} under name {}.", jobGraph.getJobID(), registrationName); @@ -624,8 +624,7 @@ public void notifyKvStateRegistered( public void notifyKvStateUnregistered( JobVertexID jobVertexId, KeyGroupRange keyGroupRange, - String registrationName) - { + String registrationName) { if (log.isDebugEnabled()) { log.debug("Key value state unregistered for job {} under name {}.", jobGraph.getJobID(), registrationName); @@ -786,7 +785,7 @@ public CompletableFuture requestRestAddress(Time timeout) { @Override public CompletableFuture requestJob(JobID jobId, Time timeout) { - if (Objects.equals(jobGraph.getJobID(), jobId)) { + if (jobGraph.getJobID().equals(jobId)) { return requestArchivedExecutionGraph(timeout); } else { return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId)); @@ -952,7 +951,7 @@ private void jobStatusChanged( final JobID jobID = executionGraph.getJobID(); final String jobName = executionGraph.getJobName(); - + if (newJobStatus.isGloballyTerminalState()) { switch (newJobStatus) { case FINISHED: @@ -960,14 +959,14 @@ private void jobStatusChanged( // TODO get correct job duration // job done, let's get the accumulators Map accumulatorResults = executionGraph.getAccumulators(); - JobExecutionResult result = new JobExecutionResult(jobID, 0L, accumulatorResults); + JobExecutionResult result = new JobExecutionResult(jobID, 0L, accumulatorResults); executor.execute(() -> jobCompletionActions.jobFinished(result)); } catch (Exception e) { log.error("Cannot fetch final accumulators for job {} ({})", jobName, jobID, e); - final JobExecutionException exception = new JobExecutionException(jobID, + final JobExecutionException exception = new JobExecutionException(jobID, "Failed to retrieve accumulator results. " + "The job is registered as 'FINISHED (successful), but this notification describes " + "a failure, since the resulting accumulators could not be fetched.", e); @@ -1038,7 +1037,7 @@ private void notifyOfNewResourceManagerLeader(final String resourceManagerAddres private void establishResourceManagerConnection(final JobMasterRegistrationSuccess success) { final ResourceManagerId resourceManagerId = success.getResourceManagerId(); - + // verify the response with current connection if (resourceManagerConnection != null && Objects.equals(resourceManagerConnection.getTargetLeaderId(), resourceManagerId)) { @@ -1105,8 +1104,7 @@ public void handleError(final Exception exception) { //---------------------------------------------------------------------------------------------- private class ResourceManagerConnection - extends RegisteredRpcConnection - { + extends RegisteredRpcConnection { private final JobID jobID; private final ResourceID jobManagerResourceID; @@ -1125,8 +1123,7 @@ private class ResourceManagerConnection final JobMasterId jobMasterId, final String resourceManagerAddress, final ResourceManagerId resourceManagerId, - final Executor executor) - { + final Executor executor) { super(log, resourceManagerAddress, resourceManagerId, executor); this.jobID = checkNotNull(jobID); this.jobManagerResourceID = checkNotNull(jobManagerResourceID); @@ -1138,12 +1135,10 @@ private class ResourceManagerConnection protected RetryingRegistration generateRegistration() { return new RetryingRegistration( log, getRpcService(), "ResourceManager", ResourceManagerGateway.class, - getTargetAddress(), getTargetLeaderId()) - { + getTargetAddress(), getTargetLeaderId()) { @Override protected CompletableFuture invokeRegistration( - ResourceManagerGateway gateway, ResourceManagerId fencingToken, long timeoutMillis) throws Exception - { + ResourceManagerGateway gateway, ResourceManagerId fencingToken, long timeoutMillis) throws Exception { Time timeout = Time.milliseconds(timeoutMillis); return gateway.registerJobManager( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRestEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRestEndpoint.java new file mode 100644 index 0000000000000..4baac95b0c8ee --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRestEndpoint.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rest.RestServerEndpointConfiguration; +import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration; +import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever; + +import java.util.concurrent.Executor; + +/** + * REST endpoint for the {@link JobMaster}. + */ +public class JobMasterRestEndpoint extends WebMonitorEndpoint { + + public JobMasterRestEndpoint( + RestServerEndpointConfiguration endpointConfiguration, + GatewayRetriever leaderRetriever, + Configuration clusterConfiguration, + RestHandlerConfiguration restConfiguration, + GatewayRetriever resourceManagerRetriever, + Executor executor, + MetricQueryServiceRetriever metricQueryServiceRetriever) { + super(endpointConfiguration, leaderRetriever, clusterConfiguration, restConfiguration, resourceManagerRetriever, executor, metricQueryServiceRetriever); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java new file mode 100644 index 0000000000000..5835c8742e765 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java @@ -0,0 +1,402 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.webmonitor; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rest.RestServerEndpoint; +import org.apache.flink.runtime.rest.RestServerEndpointConfiguration; +import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration; +import org.apache.flink.runtime.rest.handler.RestHandlerSpecification; +import org.apache.flink.runtime.rest.handler.cluster.ClusterConfigHandler; +import org.apache.flink.runtime.rest.handler.cluster.ClusterOverviewHandler; +import org.apache.flink.runtime.rest.handler.cluster.DashboardConfigHandler; +import org.apache.flink.runtime.rest.handler.job.JobAccumulatorsHandler; +import org.apache.flink.runtime.rest.handler.job.JobConfigHandler; +import org.apache.flink.runtime.rest.handler.job.JobDetailsHandler; +import org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler; +import org.apache.flink.runtime.rest.handler.job.JobIdsHandler; +import org.apache.flink.runtime.rest.handler.job.JobPlanHandler; +import org.apache.flink.runtime.rest.handler.job.JobVertexAccumulatorsHandler; +import org.apache.flink.runtime.rest.handler.job.JobsOverviewHandler; +import org.apache.flink.runtime.rest.handler.job.SubtasksTimesHandler; +import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointConfigHandler; +import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointStatisticDetailsHandler; +import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointStatsCache; +import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointingStatisticsHandler; +import org.apache.flink.runtime.rest.handler.job.checkpoints.TaskCheckpointStatisticDetailsHandler; +import org.apache.flink.runtime.rest.handler.job.metrics.JobManagerMetricsHandler; +import org.apache.flink.runtime.rest.handler.job.metrics.JobMetricsHandler; +import org.apache.flink.runtime.rest.handler.job.metrics.JobVertexMetricsHandler; +import org.apache.flink.runtime.rest.handler.job.metrics.SubtaskMetricsHandler; +import org.apache.flink.runtime.rest.handler.job.metrics.TaskManagerMetricsHandler; +import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache; +import org.apache.flink.runtime.rest.handler.legacy.files.StaticFileServerHandler; +import org.apache.flink.runtime.rest.handler.legacy.files.WebContentHandlerSpecification; +import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher; +import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerDetailsHandler; +import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagersHandler; +import org.apache.flink.runtime.rest.messages.ClusterConfigurationInfoHeaders; +import org.apache.flink.runtime.rest.messages.ClusterOverviewHeaders; +import org.apache.flink.runtime.rest.messages.DashboardConfigurationHeaders; +import org.apache.flink.runtime.rest.messages.JobAccumulatorsHeaders; +import org.apache.flink.runtime.rest.messages.JobConfigHeaders; +import org.apache.flink.runtime.rest.messages.JobExceptionsHeaders; +import org.apache.flink.runtime.rest.messages.JobIdsWithStatusesOverviewHeaders; +import org.apache.flink.runtime.rest.messages.JobPlanHeaders; +import org.apache.flink.runtime.rest.messages.JobVertexAccumulatorsHeaders; +import org.apache.flink.runtime.rest.messages.JobsOverviewHeaders; +import org.apache.flink.runtime.rest.messages.SubtasksTimesHeaders; +import org.apache.flink.runtime.rest.messages.checkpoints.CheckpointConfigHeaders; +import org.apache.flink.runtime.rest.messages.checkpoints.CheckpointStatisticDetailsHeaders; +import org.apache.flink.runtime.rest.messages.checkpoints.CheckpointingStatisticsHeaders; +import org.apache.flink.runtime.rest.messages.checkpoints.TaskCheckpointStatisticsHeaders; +import org.apache.flink.runtime.rest.messages.job.JobDetailsHeaders; +import org.apache.flink.runtime.rest.messages.job.metrics.JobManagerMetricsHeaders; +import org.apache.flink.runtime.rest.messages.job.metrics.JobMetricsHeaders; +import org.apache.flink.runtime.rest.messages.job.metrics.JobVertexMetricsHeaders; +import org.apache.flink.runtime.rest.messages.job.metrics.SubtaskMetricsHeaders; +import org.apache.flink.runtime.rest.messages.job.metrics.TaskManagerMetricsHeaders; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerDetailsHeaders; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersHeaders; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever; +import org.apache.flink.util.FileUtils; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandler; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; + +/** + * Rest endpoint which serves the web frontend REST calls. + * + * @param type of the leader gateway + */ +public class WebMonitorEndpoint extends RestServerEndpoint { + + protected final GatewayRetriever leaderRetriever; + private final Configuration clusterConfiguration; + protected final RestHandlerConfiguration restConfiguration; + private final GatewayRetriever resourceManagerRetriever; + private final Executor executor; + + private final ExecutionGraphCache executionGraphCache; + private final CheckpointStatsCache checkpointStatsCache; + + private final MetricFetcher metricFetcher; + + public WebMonitorEndpoint( + RestServerEndpointConfiguration endpointConfiguration, + GatewayRetriever leaderRetriever, + Configuration clusterConfiguration, + RestHandlerConfiguration restConfiguration, + GatewayRetriever resourceManagerRetriever, + Executor executor, + MetricQueryServiceRetriever metricQueryServiceRetriever) { + super(endpointConfiguration); + this.leaderRetriever = Preconditions.checkNotNull(leaderRetriever); + this.clusterConfiguration = Preconditions.checkNotNull(clusterConfiguration); + this.restConfiguration = Preconditions.checkNotNull(restConfiguration); + this.resourceManagerRetriever = Preconditions.checkNotNull(resourceManagerRetriever); + this.executor = Preconditions.checkNotNull(executor); + + this.executionGraphCache = new ExecutionGraphCache( + restConfiguration.getTimeout(), + Time.milliseconds(restConfiguration.getRefreshInterval())); + + this.checkpointStatsCache = new CheckpointStatsCache( + restConfiguration.getMaxCheckpointStatisticCacheEntries()); + + this.metricFetcher = new MetricFetcher<>( + leaderRetriever, + metricQueryServiceRetriever, + executor, + restConfiguration.getTimeout()); + } + + @Override + protected List> initializeHandlers(CompletableFuture restAddressFuture) { + ArrayList> handlers = new ArrayList<>(3); + + final Time timeout = restConfiguration.getTimeout(); + final Map responseHeaders = restConfiguration.getResponseHeaders(); + + ClusterOverviewHandler clusterOverviewHandler = new ClusterOverviewHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + ClusterOverviewHeaders.getInstance()); + + DashboardConfigHandler dashboardConfigHandler = new DashboardConfigHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + DashboardConfigurationHeaders.getInstance(), + restConfiguration.getRefreshInterval()); + + JobIdsHandler jobIdsHandler = new JobIdsHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + JobIdsWithStatusesOverviewHeaders.getInstance()); + + JobsOverviewHandler jobsOverviewHandler = new JobsOverviewHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + JobsOverviewHeaders.getInstance()); + + ClusterConfigHandler clusterConfigurationHandler = new ClusterConfigHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + ClusterConfigurationInfoHeaders.getInstance(), + clusterConfiguration); + + JobConfigHandler jobConfigHandler = new JobConfigHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + JobConfigHeaders.getInstance(), + executionGraphCache, + executor); + + CheckpointConfigHandler checkpointConfigHandler = new CheckpointConfigHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + CheckpointConfigHeaders.getInstance(), + executionGraphCache, + executor); + + CheckpointingStatisticsHandler checkpointStatisticsHandler = new CheckpointingStatisticsHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + CheckpointingStatisticsHeaders.getInstance(), + executionGraphCache, + executor); + + CheckpointStatisticDetailsHandler checkpointStatisticDetailsHandler = new CheckpointStatisticDetailsHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + CheckpointStatisticDetailsHeaders.getInstance(), + executionGraphCache, + executor, + checkpointStatsCache); + + JobPlanHandler jobPlanHandler = new JobPlanHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + JobPlanHeaders.getInstance(), + executionGraphCache, + executor); + + TaskCheckpointStatisticDetailsHandler taskCheckpointStatisticDetailsHandler = new TaskCheckpointStatisticDetailsHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + TaskCheckpointStatisticsHeaders.getInstance(), + executionGraphCache, + executor, + checkpointStatsCache); + + JobExceptionsHandler jobExceptionsHandler = new JobExceptionsHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + JobExceptionsHeaders.getInstance(), + executionGraphCache, + executor); + + JobVertexAccumulatorsHandler jobVertexAccumulatorsHandler = new JobVertexAccumulatorsHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + JobVertexAccumulatorsHeaders.getInstance(), + executionGraphCache, + executor); + + TaskManagersHandler taskManagersHandler = new TaskManagersHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + TaskManagersHeaders.getInstance(), + resourceManagerRetriever); + + TaskManagerDetailsHandler taskManagerDetailsHandler = new TaskManagerDetailsHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + TaskManagerDetailsHeaders.getInstance(), + resourceManagerRetriever, + metricFetcher); + + final JobDetailsHandler jobDetailsHandler = new JobDetailsHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + JobDetailsHeaders.getInstance(), + executionGraphCache, + executor, + metricFetcher); + + JobAccumulatorsHandler jobAccumulatorsHandler = new JobAccumulatorsHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + JobAccumulatorsHeaders.getInstance(), + executionGraphCache, + executor); + + SubtasksTimesHandler subtasksTimesHandler = new SubtasksTimesHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + SubtasksTimesHeaders.getInstance(), + executionGraphCache, + executor); + + final JobVertexMetricsHandler jobVertexMetricsHandler = new JobVertexMetricsHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + metricFetcher); + + final JobMetricsHandler jobMetricsHandler = new JobMetricsHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + metricFetcher); + + final SubtaskMetricsHandler subtaskMetricsHandler = new SubtaskMetricsHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + metricFetcher); + + final TaskManagerMetricsHandler taskManagerMetricsHandler = new TaskManagerMetricsHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + metricFetcher); + + final JobManagerMetricsHandler jobManagerMetricsHandler = new JobManagerMetricsHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + metricFetcher); + + final File tmpDir = restConfiguration.getTmpDir(); + + Optional> optWebContent; + + try { + optWebContent = WebMonitorUtils.tryLoadWebContent( + leaderRetriever, + restAddressFuture, + timeout, + tmpDir); + } catch (IOException e) { + log.warn("Could not load web content handler.", e); + optWebContent = Optional.empty(); + } + + handlers.add(Tuple2.of(ClusterOverviewHeaders.getInstance(), clusterOverviewHandler)); + handlers.add(Tuple2.of(ClusterConfigurationInfoHeaders.getInstance(), clusterConfigurationHandler)); + handlers.add(Tuple2.of(DashboardConfigurationHeaders.getInstance(), dashboardConfigHandler)); + handlers.add(Tuple2.of(JobIdsWithStatusesOverviewHeaders.getInstance(), jobIdsHandler)); + handlers.add(Tuple2.of(JobsOverviewHeaders.getInstance(), jobsOverviewHandler)); + handlers.add(Tuple2.of(JobConfigHeaders.getInstance(), jobConfigHandler)); + handlers.add(Tuple2.of(CheckpointConfigHeaders.getInstance(), checkpointConfigHandler)); + handlers.add(Tuple2.of(CheckpointingStatisticsHeaders.getInstance(), checkpointStatisticsHandler)); + handlers.add(Tuple2.of(CheckpointStatisticDetailsHeaders.getInstance(), checkpointStatisticDetailsHandler)); + handlers.add(Tuple2.of(JobPlanHeaders.getInstance(), jobPlanHandler)); + handlers.add(Tuple2.of(TaskCheckpointStatisticsHeaders.getInstance(), taskCheckpointStatisticDetailsHandler)); + handlers.add(Tuple2.of(JobExceptionsHeaders.getInstance(), jobExceptionsHandler)); + handlers.add(Tuple2.of(JobVertexAccumulatorsHeaders.getInstance(), jobVertexAccumulatorsHandler)); + handlers.add(Tuple2.of(JobDetailsHeaders.getInstance(), jobDetailsHandler)); + handlers.add(Tuple2.of(JobAccumulatorsHeaders.getInstance(), jobAccumulatorsHandler)); + handlers.add(Tuple2.of(TaskManagersHeaders.getInstance(), taskManagersHandler)); + handlers.add(Tuple2.of(TaskManagerDetailsHeaders.getInstance(), taskManagerDetailsHandler)); + handlers.add(Tuple2.of(SubtasksTimesHeaders.getInstance(), subtasksTimesHandler)); + handlers.add(Tuple2.of(JobVertexMetricsHeaders.getInstance(), jobVertexMetricsHandler)); + handlers.add(Tuple2.of(JobMetricsHeaders.getInstance(), jobMetricsHandler)); + handlers.add(Tuple2.of(SubtaskMetricsHeaders.getInstance(), subtaskMetricsHandler)); + handlers.add(Tuple2.of(TaskManagerMetricsHeaders.getInstance(), taskManagerMetricsHandler)); + handlers.add(Tuple2.of(JobManagerMetricsHeaders.getInstance(), jobManagerMetricsHandler)); + + // This handler MUST be added last, as it otherwise masks all subsequent GET handlers + optWebContent.ifPresent( + webContent -> handlers.add(Tuple2.of(WebContentHandlerSpecification.getInstance(), webContent))); + + return handlers; + } + + @Override + public void shutdown(Time timeout) { + super.shutdown(timeout); + + executionGraphCache.close(); + + final File tmpDir = restConfiguration.getTmpDir(); + + try { + log.info("Removing cache directory {}", tmpDir); + FileUtils.deleteDirectory(tmpDir); + } catch (Throwable t) { + log.warn("Error while deleting cache directory {}", tmpDir, t); + } + } +} From 0d551640e65073afd8755e04f7817a6379149251 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 7 Nov 2017 17:27:38 +0100 Subject: [PATCH 2/4] [FLINK-8030] Instantiate JobMasterRestEndpoint in JobClusterEntrypoint This closes #4988. --- .../entrypoint/JobClusterEntrypoint.java | 107 +++++++++++++++++- .../runtime/jobmaster/JobManagerRunner.java | 4 + .../webmonitor/WebMonitorEndpoint.java | 2 +- 3 files changed, 110 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java index 1c8fb21acf5e9..bd1f573e53c95 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java @@ -20,7 +20,9 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.WebOptions; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.heartbeat.HeartbeatServices; @@ -29,16 +31,35 @@ import org.apache.flink.runtime.jobmanager.OnCompletionActions; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerServices; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.jobmaster.JobMasterId; +import org.apache.flink.runtime.jobmaster.JobMasterRestEndpoint; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.resourcemanager.ResourceManager; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.resourcemanager.ResourceManagerId; +import org.apache.flink.runtime.rest.RestServerEndpointConfiguration; +import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.akka.AkkaRpcService; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.runtime.webmonitor.retriever.LeaderGatewayRetriever; +import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever; +import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaQueryServiceRetriever; +import org.apache.flink.runtime.webmonitor.retriever.impl.RpcGatewayRetriever; +import org.apache.flink.util.ConfigurationException; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; +import akka.actor.ActorSystem; + import javax.annotation.Nullable; +import java.util.concurrent.Executor; + /** * Base class for per-job cluster entry points. */ @@ -48,6 +69,12 @@ public abstract class JobClusterEntrypoint extends ClusterEntrypoint { private JobManagerServices jobManagerServices; + private JobMasterRestEndpoint jobMasterRestEndpoint; + + private LeaderRetrievalService jobMasterRetrievalService; + + private LeaderRetrievalService resourceManagerRetrievalService; + private JobManagerRunner jobManagerRunner; public JobClusterEntrypoint(Configuration configuration) { @@ -74,6 +101,36 @@ protected void startClusterComponents( jobManagerServices = JobManagerServices.fromConfiguration(configuration, blobServer); + resourceManagerRetrievalService = highAvailabilityServices.getResourceManagerLeaderRetriever(); + + final LeaderGatewayRetriever jobMasterGatewayRetriever = new RpcGatewayRetriever<>( + rpcService, + JobMasterGateway.class, + JobMasterId::new, + 10, + Time.milliseconds(50L)); + + final LeaderGatewayRetriever resourceManagerGatewayRetriever = new RpcGatewayRetriever<>( + rpcService, + ResourceManagerGateway.class, + ResourceManagerId::new, + 10, + Time.milliseconds(50L)); + + // TODO: Remove once we have ported the MetricFetcher to the RpcEndpoint + final ActorSystem actorSystem = ((AkkaRpcService) rpcService).getActorSystem(); + final Time timeout = Time.milliseconds(configuration.getLong(WebOptions.TIMEOUT)); + + jobMasterRestEndpoint = createJobMasterRestEndpoint( + configuration, + jobMasterGatewayRetriever, + resourceManagerGatewayRetriever, + rpcService.getExecutor(), + new AkkaQueryServiceRetriever(actorSystem, timeout)); + + LOG.debug("Starting JobMaster REST endpoint."); + jobMasterRestEndpoint.start(); + jobManagerRunner = createJobManagerRunner( configuration, ResourceID.generate(), @@ -83,13 +140,39 @@ protected void startClusterComponents( heartbeatServices, metricRegistry, this, - null); + jobMasterRestEndpoint.getRestAddress()); LOG.debug("Starting ResourceManager."); resourceManager.start(); + resourceManagerRetrievalService.start(resourceManagerGatewayRetriever); LOG.debug("Starting JobManager."); jobManagerRunner.start(); + + jobMasterRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever( + jobManagerRunner.getJobGraph().getJobID(), + jobManagerRunner.getAddress()); + jobMasterRetrievalService.start(jobMasterGatewayRetriever); + } + + protected JobMasterRestEndpoint createJobMasterRestEndpoint( + Configuration configuration, + GatewayRetriever jobMasterGatewayRetriever, + GatewayRetriever resourceManagerGatewayRetriever, + Executor executor, + MetricQueryServiceRetriever metricQueryServiceRetriever) throws ConfigurationException { + + final RestHandlerConfiguration restHandlerConfiguration = RestHandlerConfiguration.fromConfiguration(configuration); + + return new JobMasterRestEndpoint( + RestServerEndpointConfiguration.fromConfiguration(configuration), + jobMasterGatewayRetriever, + configuration, + restHandlerConfiguration, + resourceManagerGatewayRetriever, + executor, + metricQueryServiceRetriever); + } protected JobManagerRunner createJobManagerRunner( @@ -103,7 +186,7 @@ protected JobManagerRunner createJobManagerRunner( FatalErrorHandler fatalErrorHandler, @Nullable String restAddress) throws Exception { - JobGraph jobGraph = retrieveJobGraph(configuration); + final JobGraph jobGraph = retrieveJobGraph(configuration); return new JobManagerRunner( resourceId, @@ -123,6 +206,18 @@ protected JobManagerRunner createJobManagerRunner( protected void stopClusterComponents(boolean cleanupHaData) throws Exception { Throwable exception = null; + if (jobMasterRestEndpoint != null) { + jobMasterRestEndpoint.shutdown(Time.seconds(10L)); + } + + if (jobMasterRetrievalService != null) { + try { + jobMasterRetrievalService.stop(); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + } + if (jobManagerRunner != null) { try { jobManagerRunner.shutdown(); @@ -139,6 +234,14 @@ protected void stopClusterComponents(boolean cleanupHaData) throws Exception { } } + if (resourceManagerRetrievalService != null) { + try { + resourceManagerRetrievalService.stop(); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + } + if (resourceManager != null) { try { resourceManager.shutDown(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java index ed3d43d5bfed9..e699d6daa9c50 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java @@ -192,6 +192,10 @@ public JobMasterGateway getJobManagerGateway() { return jobManager.getSelfGateway(JobMasterGateway.class); } + public JobGraph getJobGraph() { + return jobGraph; + } + //---------------------------------------------------------------------------------------------- // Lifecycle management //---------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java index 5835c8742e765..703a754bf5c8f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java @@ -142,7 +142,7 @@ public WebMonitorEndpoint( @Override protected List> initializeHandlers(CompletableFuture restAddressFuture) { - ArrayList> handlers = new ArrayList<>(3); + ArrayList> handlers = new ArrayList<>(30); final Time timeout = restConfiguration.getTimeout(); final Map responseHeaders = restConfiguration.getResponseHeaders(); From bb9c64b1222a5e9568cf93186a6420bebcb306f9 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 14 Nov 2017 23:50:52 +0100 Subject: [PATCH 3/4] [FLINK-8078] Introduce LogicalSlot interface The LogicalSlot interface decouples the task deployment from the actual slot implementation which at the moment is Slot, SimpleSlot and SharedSlot. This is a helpful step to introduce a different slot implementation for Flip-6. This closes #5086. --- .../InputChannelDeploymentDescriptor.java | 9 +- .../runtime/executiongraph/Execution.java | 62 ++--- .../executiongraph/ExecutionVertex.java | 15 +- .../flink/runtime/instance/LogicalSlot.java | 93 +++++++ .../flink/runtime/instance/SimpleSlot.java | 16 +- .../flink/runtime/instance/SlotPool.java | 20 +- .../runtime/instance/SlotPoolGateway.java | 2 +- .../flink/runtime/instance/SlotProvider.java | 2 +- .../jobmanager/scheduler/Scheduler.java | 13 +- .../flink/runtime/jobmaster/JobMaster.java | 4 +- .../InputChannelDeploymentDescriptorTest.java | 19 +- .../ExecutionGraphDeploymentTest.java | 5 +- .../ExecutionGraphMetricsTest.java | 53 +--- .../ExecutionGraphSchedulingTest.java | 17 +- .../runtime/executiongraph/ExecutionTest.java | 3 +- .../ExecutionVertexDeploymentTest.java | 10 +- .../ProgrammedSlotProvider.java | 18 +- .../utils/SimpleSlotProvider.java | 3 +- .../runtime/instance/SharedSlotsTest.java | 3 +- .../runtime/instance/SimpleSlotTest.java | 16 +- .../runtime/instance/SlotPoolRpcTest.java | 10 +- .../flink/runtime/instance/SlotPoolTest.java | 48 ++-- .../runtime/instance/TestingLogicalSlot.java | 114 +++++++++ .../PartialConsumePipelinedResultTest.java | 4 +- .../ScheduleOrUpdateConsumersTest.java | 3 +- .../ScheduleWithCoLocationHintTest.java | 178 ++++++-------- .../scheduler/SchedulerIsolatedTasksTest.java | 85 ++++--- .../scheduler/SchedulerSlotSharingTest.java | 226 +++++++++--------- 28 files changed, 614 insertions(+), 437 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/instance/LogicalSlot.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/instance/TestingLogicalSlot.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java index fe1c599127647..8d7620711ac6a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionEdge; import org.apache.flink.runtime.executiongraph.ExecutionGraphException; import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; -import org.apache.flink.runtime.instance.SimpleSlot; +import org.apache.flink.runtime.instance.LogicalSlot; import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.consumer.InputChannel; @@ -85,10 +85,9 @@ public String toString() { */ public static InputChannelDeploymentDescriptor[] fromEdges( ExecutionEdge[] edges, - SimpleSlot consumerSlot, + ResourceID consumerResourceId, boolean allowLazyDeployment) throws ExecutionGraphException { - final ResourceID consumerTaskManager = consumerSlot.getTaskManagerID(); final InputChannelDeploymentDescriptor[] icdd = new InputChannelDeploymentDescriptor[edges.length]; // Each edge is connected to a different result partition @@ -97,7 +96,7 @@ public static InputChannelDeploymentDescriptor[] fromEdges( final Execution producer = consumedPartition.getProducer().getCurrentExecutionAttempt(); final ExecutionState producerState = producer.getState(); - final SimpleSlot producerSlot = producer.getAssignedResource(); + final LogicalSlot producerSlot = producer.getAssignedResource(); final ResultPartitionLocation partitionLocation; @@ -111,7 +110,7 @@ public static InputChannelDeploymentDescriptor[] fromEdges( final TaskManagerLocation partitionTaskManagerLocation = producerSlot.getTaskManagerLocation(); final ResourceID partitionTaskManager = partitionTaskManagerLocation.getResourceID(); - if (partitionTaskManager.equals(consumerTaskManager)) { + if (partitionTaskManager.equals(consumerResourceId)) { // Consuming task is deployed to the same TaskManager as the partition => local partitionLocation = ResultPartitionLocation.createLocal(); } 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 38c382108befe..00a452d6887ee 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 @@ -33,7 +33,7 @@ import org.apache.flink.runtime.deployment.ResultPartitionLocation; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.instance.SimpleSlot; +import org.apache.flink.runtime.instance.LogicalSlot; import org.apache.flink.runtime.instance.SlotProvider; import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; @@ -98,9 +98,9 @@ public class Execution implements AccessExecution, Archiveable STATE_UPDATER = AtomicReferenceFieldUpdater.newUpdater(Execution.class, ExecutionState.class, "state"); - private static final AtomicReferenceFieldUpdater ASSIGNED_SLOT_UPDATER = AtomicReferenceFieldUpdater.newUpdater( + private static final AtomicReferenceFieldUpdater ASSIGNED_SLOT_UPDATER = AtomicReferenceFieldUpdater.newUpdater( Execution.class, - SimpleSlot.class, + LogicalSlot.class, "assignedResource"); private static final Logger LOG = ExecutionGraph.LOG; @@ -141,7 +141,7 @@ public class Execution implements AccessExecution, Archiveable getTaskManagerLocationFuture() { return taskManagerLocationFuture; } - public SimpleSlot getAssignedResource() { + public LogicalSlot getAssignedResource() { return assignedResource; } @@ -248,21 +248,21 @@ public SimpleSlot getAssignedResource() { * Tries to assign the given slot to the execution. The assignment works only if the * Execution is in state SCHEDULED. Returns true, if the resource could be assigned. * - * @param slot to assign to this execution + * @param logicalSlot to assign to this execution * @return true if the slot could be assigned to the execution, otherwise false */ @VisibleForTesting - boolean tryAssignResource(final SimpleSlot slot) { - checkNotNull(slot); + boolean tryAssignResource(final LogicalSlot logicalSlot) { + checkNotNull(logicalSlot); // only allow to set the assigned resource in state SCHEDULED or CREATED // note: we also accept resource assignment when being in state CREATED for testing purposes if (state == SCHEDULED || state == CREATED) { - if (ASSIGNED_SLOT_UPDATER.compareAndSet(this, null, slot)) { + if (ASSIGNED_SLOT_UPDATER.compareAndSet(this, null, logicalSlot)) { // check for concurrent modification (e.g. cancelling call) if (state == SCHEDULED || state == CREATED) { checkState(!taskManagerLocationFuture.isDone(), "The TaskManagerLocationFuture should not be set if we haven't assigned a resource yet."); - taskManagerLocationFuture.complete(slot.getTaskManagerLocation()); + taskManagerLocationFuture.complete(logicalSlot.getTaskManagerLocation()); return true; } else { @@ -283,7 +283,7 @@ boolean tryAssignResource(final SimpleSlot slot) { @Override public TaskManagerLocation getAssignedResourceLocation() { // returns non-null only when a location is already assigned - final SimpleSlot currentAssignedResource = assignedResource; + final LogicalSlot currentAssignedResource = assignedResource; return currentAssignedResource != null ? currentAssignedResource.getTaskManagerLocation() : null; } @@ -442,14 +442,14 @@ public CompletableFuture allocateAndAssignSlotForExecution( queued, preferredLocations)) .thenApply( - (SimpleSlot slot) -> { - if (tryAssignResource(slot)) { + (LogicalSlot logicalSlot) -> { + if (tryAssignResource(logicalSlot)) { return this; } else { // release the slot - slot.releaseSlot(); + logicalSlot.releaseSlot(); - throw new CompletionException(new FlinkException("Could not assign slot " + slot + " to execution " + this + " because it has already been assigned ")); + throw new CompletionException(new FlinkException("Could not assign slot " + logicalSlot + " to execution " + this + " because it has already been assigned ")); } }); } @@ -465,7 +465,7 @@ public CompletableFuture allocateAndAssignSlotForExecution( * @throws JobException if the execution cannot be deployed to the assigned resource */ public void deploy() throws JobException { - final SimpleSlot slot = assignedResource; + final LogicalSlot slot = assignedResource; checkNotNull(slot, "In order to deploy the execution we first have to assign a resource via tryAssignResource."); @@ -493,7 +493,7 @@ public void deploy() throws JobException { try { // good, we are allowed to deploy - if (!slot.setExecutedVertex(this)) { + if (!slot.setExecution(this)) { throw new JobException("Could not assign the ExecutionVertex to the slot " + slot); } @@ -545,7 +545,7 @@ public void deploy() throws JobException { * Sends stop RPC call. */ public void stop() { - final SimpleSlot slot = assignedResource; + final LogicalSlot slot = assignedResource; if (slot != null) { final TaskManagerGateway taskManagerGateway = slot.getTaskManagerGateway(); @@ -608,7 +608,7 @@ else if (current == CREATED || current == SCHEDULED) { try { vertex.getExecutionGraph().deregisterExecution(this); - final SimpleSlot slot = assignedResource; + final LogicalSlot slot = assignedResource; if (slot != null) { slot.releaseSlot(); @@ -691,7 +691,7 @@ else if (numConsumers == 0) { // ---------------------------------------------------------------- else { if (consumerState == RUNNING) { - final SimpleSlot consumerSlot = consumer.getAssignedResource(); + final LogicalSlot consumerSlot = consumer.getAssignedResource(); if (consumerSlot == null) { // The consumer has been reset concurrently @@ -702,7 +702,7 @@ else if (numConsumers == 0) { .getCurrentAssignedResource().getTaskManagerLocation(); final ResourceID partitionTaskManager = partitionTaskManagerLocation.getResourceID(); - final ResourceID consumerTaskManager = consumerSlot.getTaskManagerID(); + final ResourceID consumerTaskManager = consumerSlot.getTaskManagerLocation().getResourceID(); final ResultPartitionID partitionId = new ResultPartitionID(partition.getPartitionId(), attemptId); @@ -778,7 +778,7 @@ public CompletableFuture requestStackTraceSample( int maxStrackTraceDepth, Time timeout) { - final SimpleSlot slot = assignedResource; + final LogicalSlot slot = assignedResource; if (slot != null) { final TaskManagerGateway taskManagerGateway = slot.getTaskManagerGateway(); @@ -802,7 +802,7 @@ public CompletableFuture requestStackTraceSample( * @param timestamp of the completed checkpoint */ public void notifyCheckpointComplete(long checkpointId, long timestamp) { - final SimpleSlot slot = assignedResource; + final LogicalSlot slot = assignedResource; if (slot != null) { final TaskManagerGateway taskManagerGateway = slot.getTaskManagerGateway(); @@ -822,7 +822,7 @@ public void notifyCheckpointComplete(long checkpointId, long timestamp) { * @param checkpointOptions of the checkpoint to trigger */ public void triggerCheckpoint(long checkpointId, long timestamp, CheckpointOptions checkpointOptions) { - final SimpleSlot slot = assignedResource; + final LogicalSlot slot = assignedResource; if (slot != null) { final TaskManagerGateway taskManagerGateway = slot.getTaskManagerGateway(); @@ -880,7 +880,7 @@ void markFinished(Map> userAccumulators, IOMetrics met updateAccumulatorsAndMetrics(userAccumulators, metrics); - final SimpleSlot slot = assignedResource; + final LogicalSlot slot = assignedResource; if (slot != null) { slot.releaseSlot(); @@ -938,7 +938,7 @@ else if (current == CANCELING || current == RUNNING || current == DEPLOYING) { if (transitionState(current, CANCELED)) { try { - final SimpleSlot slot = assignedResource; + final LogicalSlot slot = assignedResource; if (slot != null) { slot.releaseSlot(); @@ -1035,7 +1035,7 @@ private boolean processFail(Throwable t, boolean isCallback, Map partitionInfos) { - final SimpleSlot slot = assignedResource; + final LogicalSlot slot = assignedResource; if (slot != null) { final TaskManagerGateway taskManagerGateway = slot.getTaskManagerGateway(); @@ -1318,7 +1318,7 @@ private void updateAccumulatorsAndMetrics(Map> userAcc @Override public String toString() { - final SimpleSlot slot = assignedResource; + final LogicalSlot slot = assignedResource; return String.format("Attempt #%d (%s) @ %s - [%s]", attemptNumber, vertex.getTaskNameWithSubtaskIndex(), (slot == null ? "(unassigned)" : slot), state); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index 6d45d06d8f952..c2c986f485e09 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.instance.LogicalSlot; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.instance.SlotProvider; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; @@ -272,7 +273,7 @@ public CompletableFuture getCurrentTaskManagerLocationFutur return currentExecution.getTaskManagerLocationFuture(); } - public SimpleSlot getCurrentAssignedResource() { + public LogicalSlot getCurrentAssignedResource() { return currentExecution.getAssignedResource(); } @@ -744,7 +745,7 @@ void notifyStateTransition(Execution execution, ExecutionState newState, Throwab */ TaskDeploymentDescriptor createDeploymentDescriptor( ExecutionAttemptID executionId, - SimpleSlot targetSlot, + LogicalSlot targetSlot, TaskStateSnapshot taskStateHandles, int attemptNumber) throws ExecutionGraphException { @@ -779,8 +780,10 @@ TaskDeploymentDescriptor createDeploymentDescriptor( for (ExecutionEdge[] edges : inputEdges) { - InputChannelDeploymentDescriptor[] partitions = InputChannelDeploymentDescriptor - .fromEdges(edges, targetSlot, lazyScheduling); + InputChannelDeploymentDescriptor[] partitions = InputChannelDeploymentDescriptor.fromEdges( + edges, + targetSlot.getTaskManagerLocation().getResourceID(), + lazyScheduling); // If the produced partition has multiple consumers registered, we // need to request the one matching our sub task index. @@ -829,10 +832,10 @@ TaskDeploymentDescriptor createDeploymentDescriptor( serializedJobInformation, serializedTaskInformation, executionId, - targetSlot.getAllocatedSlot().getSlotAllocationId(), + targetSlot.getAllocationId(), subTaskIndex, attemptNumber, - targetSlot.getRoot().getSlotNumber(), + targetSlot.getPhysicalSlotNumber(), taskStateHandles, producedPartitions, consumedPartitions); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LogicalSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LogicalSlot.java new file mode 100644 index 0000000000000..3ebe107683f89 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LogicalSlot.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.instance; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + +/** + * A logical slot represents a resource on a TaskManager into + * which a single task can be deployed. + */ +public interface LogicalSlot { + + /** + * Return the TaskManager location of this slot + * + * @return TaskManager location of this slot + */ + TaskManagerLocation getTaskManagerLocation(); + + /** + * Return the TaskManager gateway to talk to the TaskManager. + * + * @return TaskManager gateway to talk to the TaskManager + */ + TaskManagerGateway getTaskManagerGateway(); + + /** + * True if the slot is still alive. + * + * @return True if the slot is still alive, otherwise false + */ + boolean isAlive(); + + /** + * True if the slot is canceled. + * + * @return True if the slot is canceled, otherwise false + */ + boolean isCanceled(); + + /** + * True if the slot is released. + * + * @return True if the slot is released, otherwise false + */ + boolean isReleased(); + + /** + * Sets the execution for this slot. + * + * @param execution to set for this slot + * @return true if the slot could be set, otherwise false + */ + boolean setExecution(Execution execution); + + /** + * Releases this slot. + */ + void releaseSlot(); + + /** + * Gets the slot number on the TaskManager. + * + * @return slot number + */ + int getPhysicalSlotNumber(); + + /** + * Gets the allocation id of this slot. + * + * @return allocation id of this slot + */ + AllocationID getAllocationId(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java index 8c7ec01c2f7c5..95910281f2edb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.instance; +import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.api.common.JobID; @@ -37,7 +38,7 @@ *

    If this slot is part of a {@link SharedSlot}, then the parent attribute will point to that shared slot. * If not, then the parent attribute is null. */ -public class SimpleSlot extends Slot { +public class SimpleSlot extends Slot implements LogicalSlot { /** The updater used to atomically swap in the execution */ private static final AtomicReferenceFieldUpdater VERTEX_UPDATER = @@ -163,7 +164,8 @@ public Execution getExecutedVertex() { * @param executedVertex The vertex to assign to this slot. * @return True, if the vertex was assigned, false, otherwise. */ - public boolean setExecutedVertex(Execution executedVertex) { + @Override + public boolean setExecution(Execution executedVertex) { if (executedVertex == null) { throw new NullPointerException(); } @@ -231,6 +233,16 @@ public void releaseSlot() { } } + @Override + public int getPhysicalSlotNumber() { + return getRootSlotNumber(); + } + + @Override + public AllocationID getAllocationId() { + return getAllocatedSlot().getSlotAllocationId(); + } + // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ 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 7e98e11b7feec..66af865e52ec2 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 @@ -266,7 +266,7 @@ public void disconnectResourceManager() { // ------------------------------------------------------------------------ @Override - public CompletableFuture allocateSlot( + public CompletableFuture allocateSlot( SlotRequestID requestId, ScheduledUnit task, ResourceProfile resources, @@ -303,7 +303,7 @@ public CompletableFuture cancelSlotAllocation(SlotRequestID request return CompletableFuture.completedFuture(Acknowledge.get()); } - CompletableFuture internalAllocateSlot( + CompletableFuture internalAllocateSlot( SlotRequestID requestId, ScheduledUnit task, ResourceProfile resources, @@ -318,7 +318,7 @@ CompletableFuture internalAllocateSlot( } // the request will be completed by a future - final CompletableFuture future = new CompletableFuture<>(); + final CompletableFuture future = new CompletableFuture<>(); // (2) need to request a slot if (resourceManagerGateway == null) { @@ -433,7 +433,7 @@ private void failPendingRequest(PendingRequest pendingRequest, Exception e) { private void stashRequestWaitingForResourceManager( final SlotRequestID requestId, final ResourceProfile resources, - final CompletableFuture future) { + final CompletableFuture future) { LOG.info("Cannot serve slot request, no ResourceManager connected. " + "Adding as pending request {}", requestId); @@ -1087,15 +1087,15 @@ public boolean returnAllocatedSlot(Slot slot) { } @Override - public CompletableFuture allocateSlot( + public CompletableFuture allocateSlot( ScheduledUnit task, boolean allowQueued, Collection preferredLocations) { final SlotRequestID requestId = new SlotRequestID(); - CompletableFuture slotFuture = gateway.allocateSlot(requestId, task, ResourceProfile.UNKNOWN, preferredLocations, timeout); + CompletableFuture slotFuture = gateway.allocateSlot(requestId, task, ResourceProfile.UNKNOWN, preferredLocations, timeout); slotFuture.whenComplete( - (SimpleSlot slot, Throwable failure) -> { + (LogicalSlot slot, Throwable failure) -> { if (failure != null) { gateway.cancelSlotAllocation(requestId); } @@ -1113,13 +1113,13 @@ private static class PendingRequest { private final SlotRequestID slotRequestId; - private final CompletableFuture future; + private final CompletableFuture future; private final ResourceProfile resourceProfile; PendingRequest( SlotRequestID slotRequestId, - CompletableFuture future, + CompletableFuture future, ResourceProfile resourceProfile) { this.slotRequestId = Preconditions.checkNotNull(slotRequestId); this.future = Preconditions.checkNotNull(future); @@ -1130,7 +1130,7 @@ public SlotRequestID getSlotRequestId() { return slotRequestId; } - public CompletableFuture getFuture() { + public CompletableFuture getFuture() { return future; } 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 bf520f55a65ef..ad2a6a6369d71 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 @@ -86,7 +86,7 @@ public interface SlotPoolGateway extends RpcGateway { // allocating and disposing slots // ------------------------------------------------------------------------ - CompletableFuture allocateSlot( + CompletableFuture allocateSlot( SlotRequestID requestId, ScheduledUnit task, ResourceProfile resources, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotProvider.java index ef988b4886a75..98427c2f855d8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotProvider.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotProvider.java @@ -45,7 +45,7 @@ public interface SlotProvider { * @param preferredLocations preferred locations for the slot allocation * @return The future of the allocation */ - CompletableFuture allocateSlot( + CompletableFuture allocateSlot( ScheduledUnit task, boolean allowQueued, Collection preferredLocations); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java index 1995c125ca169..271514605814a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.instance.InstanceDiedException; import org.apache.flink.runtime.instance.InstanceListener; +import org.apache.flink.runtime.instance.LogicalSlot; import org.apache.flink.runtime.instance.SharedSlot; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.instance.SlotProvider; @@ -133,7 +134,7 @@ public void shutdown() { @Override - public CompletableFuture allocateSlot( + public CompletableFuture allocateSlot( ScheduledUnit task, boolean allowQueued, Collection preferredLocations) { @@ -146,7 +147,7 @@ public CompletableFuture allocateSlot( } else if (ret instanceof CompletableFuture) { @SuppressWarnings("unchecked") - CompletableFuture typed = (CompletableFuture) ret; + CompletableFuture typed = (CompletableFuture) ret; return typed; } else { @@ -321,7 +322,7 @@ else if (slotFromGroup == null || !slotFromGroup.isAlive() || newSlot.getLocalit else { // no resource available now, so queue the request if (queueIfNoResource) { - CompletableFuture future = new CompletableFuture<>(); + CompletableFuture future = new CompletableFuture<>(); this.taskQueue.add(new QueuedTask(task, future)); return future; } @@ -837,10 +838,10 @@ private static final class QueuedTask { private final ScheduledUnit task; - private final CompletableFuture future; + private final CompletableFuture future; - public QueuedTask(ScheduledUnit task, CompletableFuture future) { + public QueuedTask(ScheduledUnit task, CompletableFuture future) { this.task = task; this.future = future; } @@ -849,7 +850,7 @@ public ScheduledUnit getTask() { return task; } - public CompletableFuture getFuture() { + public CompletableFuture getFuture() { return future; } } 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 1d4eb6abc7b4a..687b6d12a64c3 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 @@ -55,7 +55,7 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.heartbeat.HeartbeatTarget; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.instance.Slot; +import org.apache.flink.runtime.instance.LogicalSlot; import org.apache.flink.runtime.instance.SlotPool; import org.apache.flink.runtime.instance.SlotPoolGateway; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; @@ -444,7 +444,7 @@ public CompletableFuture requestNextInputSplit( return FutureUtils.completedExceptionally(new Exception("No InputSplitAssigner for vertex ID " + vertexID)); } - final Slot slot = execution.getAssignedResource(); + final LogicalSlot slot = execution.getAssignedResource(); final int taskId = execution.getVertex().getParallelSubtaskIndex(); final String host = slot != null ? slot.getTaskManagerLocation().getHostname() : null; final InputSplit nextInputSplit = splitAssigner.getNextInputSplit(host, taskId); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptorTest.java index 27e8af345a6ae..fc2c06f0127cd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptorTest.java @@ -27,7 +27,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.executiongraph.IntermediateResult; import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; -import org.apache.flink.runtime.instance.SimpleSlot; +import org.apache.flink.runtime.instance.LogicalSlot; import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; @@ -60,7 +60,7 @@ public void testMixedLocalRemoteUnknownDeployment() throws Exception { ResourceID consumerResourceId = ResourceID.generate(); ExecutionVertex consumer = mock(ExecutionVertex.class); - SimpleSlot consumerSlot = mockSlot(consumerResourceId); + LogicalSlot consumerSlot = mockSlot(consumerResourceId); // Local and remote channel are only allowed for certain execution // states. @@ -86,7 +86,7 @@ public void testMixedLocalRemoteUnknownDeployment() throws Exception { InputChannelDeploymentDescriptor[] desc = InputChannelDeploymentDescriptor.fromEdges( new ExecutionEdge[]{localEdge, remoteEdge, unknownEdge}, - consumerSlot, + consumerSlot.getTaskManagerLocation().getResourceID(), allowLazyDeployment); assertEquals(3, desc.length); @@ -124,7 +124,7 @@ public void testMixedLocalRemoteUnknownDeployment() throws Exception { public void testUnknownChannelWithoutLazyDeploymentThrows() throws Exception { ResourceID consumerResourceId = ResourceID.generate(); ExecutionVertex consumer = mock(ExecutionVertex.class); - SimpleSlot consumerSlot = mockSlot(consumerResourceId); + LogicalSlot consumerSlot = mockSlot(consumerResourceId); // Unknown partition ExecutionVertex unknownProducer = mockExecutionVertex(ExecutionState.CREATED, null); // no assigned resource @@ -137,7 +137,7 @@ public void testUnknownChannelWithoutLazyDeploymentThrows() throws Exception { InputChannelDeploymentDescriptor[] desc = InputChannelDeploymentDescriptor.fromEdges( new ExecutionEdge[]{unknownEdge}, - consumerSlot, + consumerSlot.getTaskManagerLocation().getResourceID(), allowLazyDeployment); assertEquals(1, desc.length); @@ -152,7 +152,7 @@ public void testUnknownChannelWithoutLazyDeploymentThrows() throws Exception { InputChannelDeploymentDescriptor.fromEdges( new ExecutionEdge[]{unknownEdge}, - consumerSlot, + consumerSlot.getTaskManagerLocation().getResourceID(), allowLazyDeployment); fail("Did not throw expected ExecutionGraphException"); @@ -162,10 +162,9 @@ public void testUnknownChannelWithoutLazyDeploymentThrows() throws Exception { // ------------------------------------------------------------------------ - private static SimpleSlot mockSlot(ResourceID resourceId) { - SimpleSlot slot = mock(SimpleSlot.class); + private static LogicalSlot mockSlot(ResourceID resourceId) { + LogicalSlot slot = mock(LogicalSlot.class); when(slot.getTaskManagerLocation()).thenReturn(new TaskManagerLocation(resourceId, InetAddress.getLoopbackAddress(), 5000)); - when(slot.getTaskManagerID()).thenReturn(resourceId); return slot; } @@ -178,7 +177,7 @@ private static ExecutionVertex mockExecutionVertex(ExecutionState state, Resourc when(exec.getAttemptId()).thenReturn(new ExecutionAttemptID()); if (resourceId != null) { - SimpleSlot slot = mockSlot(resourceId); + LogicalSlot slot = mockSlot(resourceId); when(exec.getAssignedResource()).thenReturn(slot); when(vertex.getCurrentAssignedResource()).thenReturn(slot); } else { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java index de91d788d445f..b4894788816c2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java @@ -40,6 +40,7 @@ import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; import org.apache.flink.runtime.instance.Instance; +import org.apache.flink.runtime.instance.LogicalSlot; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.DistributionPattern; @@ -587,10 +588,10 @@ public void testEagerSchedulingWaitsOnAllInputPreferredLocations() throws Except sinkVertex.connectNewDataSetAsInput(sourceVertex, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - final Map[]> slotFutures = new HashMap<>(2); + final Map[]> slotFutures = new HashMap<>(2); for (JobVertexID jobVertexID : Arrays.asList(sourceVertexId, sinkVertexId)) { - CompletableFuture[] slotFutureArray = new CompletableFuture[parallelism]; + CompletableFuture[] slotFutureArray = new CompletableFuture[parallelism]; for (int i = 0; i < parallelism; i++) { slotFutureArray[i] = new CompletableFuture<>(); 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 d3cec30b7baf9..92c7c61d7477f 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 @@ -18,38 +18,28 @@ package org.apache.flink.runtime.executiongraph; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.JobException; -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.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.execution.SuppressRestartsException; import org.apache.flink.runtime.executiongraph.metrics.RestartTimeGauge; import org.apache.flink.runtime.executiongraph.restart.RestartCallback; import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; -import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.SimpleSlot; -import org.apache.flink.runtime.instance.Slot; +import org.apache.flink.runtime.instance.LogicalSlot; +import org.apache.flink.runtime.instance.TestingLogicalSlot; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; -import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot; -import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; -import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.taskmanager.TaskExecutionState; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; import org.junit.Test; -import org.mockito.Matchers; import java.io.IOException; import java.util.ArrayList; @@ -87,40 +77,9 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti Time timeout = Time.seconds(10L); Scheduler scheduler = mock(Scheduler.class); - ResourceID taskManagerId = ResourceID.generate(); - - TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); - when(taskManagerLocation.getResourceID()).thenReturn(taskManagerId); - when(taskManagerLocation.getHostname()).thenReturn("localhost"); - - TaskManagerGateway taskManagerGateway = mock(TaskManagerGateway.class); - - Instance instance = mock(Instance.class); - when(instance.getTaskManagerLocation()).thenReturn(taskManagerLocation); - when(instance.getTaskManagerID()).thenReturn(taskManagerId); - when(instance.getTaskManagerGateway()).thenReturn(taskManagerGateway); - - Slot rootSlot = mock(Slot.class); - - AllocatedSlot mockAllocatedSlot = mock(AllocatedSlot.class); - when(mockAllocatedSlot.getSlotAllocationId()).thenReturn(new AllocationID()); - - SimpleSlot simpleSlot = mock(SimpleSlot.class); - when(simpleSlot.isAlive()).thenReturn(true); - when(simpleSlot.getTaskManagerLocation()).thenReturn(taskManagerLocation); - when(simpleSlot.getTaskManagerID()).thenReturn(taskManagerId); - when(simpleSlot.getTaskManagerGateway()).thenReturn(taskManagerGateway); - when(simpleSlot.setExecutedVertex(Matchers.any(Execution.class))).thenReturn(true); - when(simpleSlot.getRoot()).thenReturn(rootSlot); - when(simpleSlot.getAllocatedSlot()).thenReturn(mockAllocatedSlot); - - CompletableFuture future = new CompletableFuture<>(); - future.complete(simpleSlot); - when(scheduler.allocateSlot(any(ScheduledUnit.class), anyBoolean(), any(Collection.class))).thenReturn(future); - - when(rootSlot.getSlotNumber()).thenReturn(0); - - when(taskManagerGateway.submitTask(any(TaskDeploymentDescriptor.class), any(Time.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); + CompletableFuture slotFuture1 = CompletableFuture.completedFuture(new TestingLogicalSlot()); + CompletableFuture slotFuture2 = CompletableFuture.completedFuture(new TestingLogicalSlot()); + when(scheduler.allocateSlot(any(ScheduledUnit.class), anyBoolean(), any(Collection.class))).thenReturn(slotFuture1, slotFuture2); TestingRestartStrategy testingRestartStrategy = new TestingRestartStrategy(); @@ -130,7 +89,7 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti jobGraph.getJobID(), jobGraph.getName(), jobConfig, - new SerializedValue(null), + new SerializedValue<>(null), timeout, testingRestartStrategy, scheduler); 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 90136a69fe3fa..71ca3a5ace9d0 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 @@ -29,6 +29,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; +import org.apache.flink.runtime.instance.LogicalSlot; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.instance.Slot; import org.apache.flink.runtime.instance.SlotProvider; @@ -109,8 +110,8 @@ public void testScheduleSourceBeforeTarget() throws Exception { final JobID jobId = new JobID(); final JobGraph jobGraph = new JobGraph(jobId, "test", sourceVertex, targetVertex); - final CompletableFuture sourceFuture = new CompletableFuture<>(); - final CompletableFuture targetFuture = new CompletableFuture<>(); + final CompletableFuture sourceFuture = new CompletableFuture<>(); + final CompletableFuture targetFuture = new CompletableFuture<>(); ProgrammedSlotProvider slotProvider = new ProgrammedSlotProvider(parallelism); slotProvider.addSlot(sourceVertex.getID(), 0, sourceFuture); @@ -177,9 +178,9 @@ public void testDeployPipelinedConnectedComponentsTogether() throws Exception { final JobGraph jobGraph = new JobGraph(jobId, "test", sourceVertex, targetVertex); @SuppressWarnings({"unchecked", "rawtypes"}) - final CompletableFuture[] sourceFutures = new CompletableFuture[parallelism]; + final CompletableFuture[] sourceFutures = new CompletableFuture[parallelism]; @SuppressWarnings({"unchecked", "rawtypes"}) - final CompletableFuture[] targetFutures = new CompletableFuture[parallelism]; + final CompletableFuture[] targetFutures = new CompletableFuture[parallelism]; // // Create the slots, futures, and the slot provider @@ -283,9 +284,9 @@ public void testOneSlotFailureAbortsDeploy() throws Exception { final SimpleSlot[] targetSlots = new SimpleSlot[parallelism]; @SuppressWarnings({"unchecked", "rawtypes"}) - final CompletableFuture[] sourceFutures = new CompletableFuture[parallelism]; + final CompletableFuture[] sourceFutures = new CompletableFuture[parallelism]; @SuppressWarnings({"unchecked", "rawtypes"}) - final CompletableFuture[] targetFutures = new CompletableFuture[parallelism]; + final CompletableFuture[] targetFutures = new CompletableFuture[parallelism]; for (int i = 0; i < parallelism; i++) { sourceSlots[i] = createSlot(taskManager, jobId, slotOwner); @@ -358,7 +359,7 @@ public void testTimeoutForSlotAllocation() throws Exception { final TaskManagerGateway taskManager = mock(TaskManagerGateway.class); final SimpleSlot[] slots = new SimpleSlot[parallelism]; @SuppressWarnings({"unchecked", "rawtypes"}) - final CompletableFuture[] slotFutures = new CompletableFuture[parallelism]; + final CompletableFuture[] slotFutures = new CompletableFuture[parallelism]; for (int i = 0; i < parallelism; i++) { slots[i] = createSlot(taskManager, jobId, slotOwner); @@ -392,7 +393,7 @@ public void testTimeoutForSlotAllocation() throws Exception { // verify that no deployments have happened verify(taskManager, times(0)).submitTask(any(TaskDeploymentDescriptor.class), any(Time.class)); - for (CompletableFuture future : slotFutures) { + for (CompletableFuture future : slotFutures) { if (future.isDone()) { assertTrue(future.get().isCanceled()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java index fa845cff1d0ef..43a6432634dde 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java @@ -22,6 +22,7 @@ import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; +import org.apache.flink.runtime.instance.LogicalSlot; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.instance.Slot; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -62,7 +63,7 @@ public void testSlotReleaseOnFailedResourceAssignment() throws Exception { final JobVertex jobVertex = new JobVertex("Test vertex", jobVertexId); jobVertex.setInvokableClass(NoOpInvokable.class); - final CompletableFuture slotFuture = new CompletableFuture<>(); + final CompletableFuture slotFuture = new CompletableFuture<>(); final ProgrammedSlotProvider slotProvider = new ProgrammedSlotProvider(1); slotProvider.addSlot(jobVertexId, 0, slotFuture); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java index cf0868763282b..973c7d44f9678 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java @@ -25,8 +25,8 @@ import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.instance.Instance; +import org.apache.flink.runtime.instance.LogicalSlot; import org.apache.flink.runtime.instance.SimpleSlot; -import org.apache.flink.runtime.instance.Slot; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -374,12 +374,8 @@ public void testTddProducedPartitionsLazyScheduling() throws Exception { AllocatedSlot allocatedSlot = mock(AllocatedSlot.class); when(allocatedSlot.getSlotAllocationId()).thenReturn(new AllocationID()); - Slot root = mock(Slot.class); - when(root.getSlotNumber()).thenReturn(1); - SimpleSlot slot = mock(SimpleSlot.class); - when(slot.getRoot()).thenReturn(root); - when(slot.getAllocatedSlot()).thenReturn(allocatedSlot); - when(root.getAllocatedSlot()).thenReturn(allocatedSlot); + LogicalSlot slot = mock(LogicalSlot.class); + when(slot.getAllocationId()).thenReturn(new AllocationID()); for (ScheduleMode mode : ScheduleMode.values()) { vertex.getExecutionGraph().setScheduleMode(mode); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ProgrammedSlotProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ProgrammedSlotProvider.java index 5d7fa1f32dea5..24affadf8a254 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ProgrammedSlotProvider.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ProgrammedSlotProvider.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.executiongraph; -import org.apache.flink.runtime.instance.SimpleSlot; +import org.apache.flink.runtime.instance.LogicalSlot; import org.apache.flink.runtime.instance.SlotProvider; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; @@ -38,7 +38,7 @@ */ class ProgrammedSlotProvider implements SlotProvider { - private final Map[]> slotFutures = new HashMap<>(); + private final Map[]> slotFutures = new HashMap<>(); private final Map[]> slotFutureRequested = new HashMap<>(); @@ -49,17 +49,17 @@ public ProgrammedSlotProvider(int parallelism) { this.parallelism = parallelism; } - public void addSlot(JobVertexID vertex, int subtaskIndex, CompletableFuture future) { + public void addSlot(JobVertexID vertex, int subtaskIndex, CompletableFuture future) { checkNotNull(vertex); checkNotNull(future); checkArgument(subtaskIndex >= 0 && subtaskIndex < parallelism); - CompletableFuture[] futures = slotFutures.get(vertex); + CompletableFuture[] futures = slotFutures.get(vertex); CompletableFuture[] requestedFutures = slotFutureRequested.get(vertex); if (futures == null) { @SuppressWarnings("unchecked") - CompletableFuture[] newArray = (CompletableFuture[]) new CompletableFuture[parallelism]; + CompletableFuture[] newArray = (CompletableFuture[]) new CompletableFuture[parallelism]; futures = newArray; slotFutures.put(vertex, futures); @@ -71,7 +71,7 @@ public void addSlot(JobVertexID vertex, int subtaskIndex, CompletableFuture(); } - public void addSlots(JobVertexID vertex, CompletableFuture[] futures) { + public void addSlots(JobVertexID vertex, CompletableFuture[] futures) { checkNotNull(vertex); checkNotNull(futures); checkArgument(futures.length == parallelism); @@ -92,16 +92,16 @@ public CompletableFuture getSlotRequestedFuture(JobVertexID jobVertexId } @Override - public CompletableFuture allocateSlot( + public CompletableFuture allocateSlot( ScheduledUnit task, boolean allowQueued, Collection preferredLocations) { JobVertexID vertexId = task.getTaskToExecute().getVertex().getJobvertexId(); int subtask = task.getTaskToExecute().getParallelSubtaskIndex(); - CompletableFuture[] forTask = slotFutures.get(vertexId); + CompletableFuture[] forTask = slotFutures.get(vertexId); if (forTask != null) { - CompletableFuture future = forTask[subtask]; + CompletableFuture future = forTask[subtask]; if (future != null) { slotFutureRequested.get(vertexId)[subtask].complete(true); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java index a2323bfbc6c33..3d289832f8352 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.instance.LogicalSlot; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.instance.Slot; import org.apache.flink.runtime.instance.SlotProvider; @@ -71,7 +72,7 @@ public SimpleSlotProvider(JobID jobId, int numSlots, TaskManagerGateway taskMana } @Override - public CompletableFuture allocateSlot( + public CompletableFuture allocateSlot( ScheduledUnit task, boolean allowQueued, Collection preferredLocations) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java index 0edef5e2dc051..4a6bf75e5e9f7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.AbstractID; +import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -38,7 +39,7 @@ /** * Tests for the allocation, properties, and release of shared slots. */ -public class SharedSlotsTest { +public class SharedSlotsTest extends TestLogger { private static final Iterable NO_LOCATION = Collections.emptySet(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java index fd1c17bc53e5d..db712107df385 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java @@ -29,11 +29,13 @@ import org.apache.flink.runtime.jobmanager.slots.ActorTaskManagerGateway; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.util.TestLogger; + import org.junit.Test; import org.mockito.Matchers; -public class SimpleSlotTest { +public class SimpleSlotTest extends TestLogger { @Test public void testStateTransitions() { @@ -81,11 +83,11 @@ public void testSetExecutionVertex() { { SimpleSlot slot = getSlot(); - assertTrue(slot.setExecutedVertex(ev)); + assertTrue(slot.setExecution(ev)); assertEquals(ev, slot.getExecutedVertex()); // try to add another one - assertFalse(slot.setExecutedVertex(ev_2)); + assertFalse(slot.setExecution(ev_2)); assertEquals(ev, slot.getExecutedVertex()); } @@ -94,7 +96,7 @@ public void testSetExecutionVertex() { SimpleSlot slot = getSlot(); assertTrue(slot.markCancelled()); - assertFalse(slot.setExecutedVertex(ev)); + assertFalse(slot.setExecution(ev)); assertNull(slot.getExecutedVertex()); } @@ -104,7 +106,7 @@ public void testSetExecutionVertex() { assertTrue(slot.markCancelled()); assertTrue(slot.markReleased()); - assertFalse(slot.setExecutedVertex(ev)); + assertFalse(slot.setExecution(ev)); assertNull(slot.getExecutedVertex()); } @@ -113,7 +115,7 @@ public void testSetExecutionVertex() { SimpleSlot slot = getSlot(); slot.releaseSlot(); - assertFalse(slot.setExecutedVertex(ev)); + assertFalse(slot.setExecution(ev)); assertNull(slot.getExecutedVertex()); } } @@ -129,7 +131,7 @@ public void testReleaseCancelsVertex() { Execution ev = mock(Execution.class); SimpleSlot slot = getSlot(); - assertTrue(slot.setExecutedVertex(ev)); + assertTrue(slot.setExecution(ev)); assertEquals(ev, slot.getExecutedVertex()); slot.releaseSlot(); 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 ca5d826c33340..8875e000c7def 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 @@ -107,7 +107,7 @@ public void testSlotAllocationNoResourceManager() throws Exception { try { pool.start(JobMasterId.generate(), "foobar"); - CompletableFuture future = pool.allocateSlot( + CompletableFuture future = pool.allocateSlot( new SlotPoolGateway.SlotRequestID(), new ScheduledUnit(SchedulerTestUtils.getDummyTask()), DEFAULT_TESTING_PROFILE, @@ -142,7 +142,7 @@ public void testCancelSlotAllocationWithoutResourceManager() throws Exception { SlotPoolGateway slotPoolGateway = pool.getSelfGateway(SlotPoolGateway.class); SlotPoolGateway.SlotRequestID requestId = new SlotPoolGateway.SlotRequestID(); - CompletableFuture future = slotPoolGateway.allocateSlot( + CompletableFuture future = slotPoolGateway.allocateSlot( requestId, new ScheduledUnit(SchedulerTestUtils.getDummyTask()), DEFAULT_TESTING_PROFILE, @@ -186,7 +186,7 @@ public void testCancelSlotAllocationWithResourceManager() throws Exception { pool.connectToResourceManager(resourceManagerGateway); SlotPoolGateway.SlotRequestID requestId = new SlotPoolGateway.SlotRequestID(); - CompletableFuture future = slotPoolGateway.allocateSlot( + CompletableFuture future = slotPoolGateway.allocateSlot( requestId, new ScheduledUnit(SchedulerTestUtils.getDummyTask()), DEFAULT_TESTING_PROFILE, @@ -237,7 +237,7 @@ public void testCancelSlotAllocationWhileSlotFulfilled() throws Exception { pool.connectToResourceManager(resourceManagerGateway); SlotPoolGateway.SlotRequestID requestId = new SlotPoolGateway.SlotRequestID(); - CompletableFuture future = slotPoolGateway.allocateSlot( + CompletableFuture future = slotPoolGateway.allocateSlot( requestId, new ScheduledUnit(SchedulerTestUtils.getDummyTask()), DEFAULT_TESTING_PROFILE, @@ -300,7 +300,7 @@ public void testProviderAndOwner() throws Exception { ScheduledUnit mockScheduledUnit = new ScheduledUnit(SchedulerTestUtils.getDummyTask()); // test the pending request is clear when timed out - CompletableFuture future = pool.getSlotProvider().allocateSlot( + CompletableFuture future = pool.getSlotProvider().allocateSlot( mockScheduledUnit, true, Collections.emptyList()); 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 c3073732e9a69..271bc2a85941e 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 @@ -96,7 +96,7 @@ public void testAllocateSimpleSlot() throws Exception { slotPoolGateway.registerTaskManager(resourceID); SlotPoolGateway.SlotRequestID requestId = new SlotPoolGateway.SlotRequestID(); - CompletableFuture future = slotPoolGateway.allocateSlot(requestId, mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); + CompletableFuture future = slotPoolGateway.allocateSlot(requestId, mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); assertFalse(future.isDone()); ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); @@ -107,13 +107,11 @@ public void testAllocateSimpleSlot() throws Exception { AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE); assertTrue(slotPoolGateway.offerSlot(allocatedSlot).get()); - SimpleSlot slot = future.get(1, TimeUnit.SECONDS); + LogicalSlot slot = future.get(1, TimeUnit.SECONDS); assertTrue(future.isDone()); assertTrue(slot.isAlive()); - assertEquals(resourceID, slot.getTaskManagerID()); - assertEquals(jobId, slot.getJobID()); - assertEquals(slotPool.getSlotOwner(), slot.getOwner()); - assertEquals(slotPool.getAllocatedSlots().get(slot.getAllocatedSlot().getSlotAllocationId()), slot); + assertEquals(resourceID, slot.getTaskManagerLocation().getResourceID()); + assertEquals(slotPool.getAllocatedSlots().get(slot.getAllocationId()), slot); } finally { slotPool.shutDown(); } @@ -129,8 +127,8 @@ public void testAllocationFulfilledByReturnedSlot() throws Exception { ResourceID resourceID = new ResourceID("resource"); slotPool.registerTaskManager(resourceID); - CompletableFuture future1 = slotPoolGateway.allocateSlot(new SlotPoolGateway.SlotRequestID(), mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); - CompletableFuture future2 = slotPoolGateway.allocateSlot(new SlotPoolGateway.SlotRequestID(), mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); + CompletableFuture future1 = slotPoolGateway.allocateSlot(new SlotPoolGateway.SlotRequestID(), mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); + CompletableFuture future2 = slotPoolGateway.allocateSlot(new SlotPoolGateway.SlotRequestID(), mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); assertFalse(future1.isDone()); assertFalse(future2.isDone()); @@ -144,7 +142,7 @@ public void testAllocationFulfilledByReturnedSlot() throws Exception { AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequests.get(0).getAllocationId(), jobId, DEFAULT_TESTING_PROFILE); assertTrue(slotPoolGateway.offerSlot(allocatedSlot).get()); - SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS); + LogicalSlot slot1 = future1.get(1, TimeUnit.SECONDS); assertTrue(future1.isDone()); assertFalse(future2.isDone()); @@ -152,15 +150,15 @@ public void testAllocationFulfilledByReturnedSlot() throws Exception { slot1.releaseSlot(); // second allocation fulfilled by previous slot returning - SimpleSlot slot2 = future2.get(1, TimeUnit.SECONDS); + LogicalSlot slot2 = future2.get(1, TimeUnit.SECONDS); assertTrue(future2.isDone()); assertNotEquals(slot1, slot2); assertTrue(slot1.isReleased()); assertTrue(slot2.isAlive()); - assertEquals(slot1.getTaskManagerID(), slot2.getTaskManagerID()); - assertEquals(slot1.getSlotNumber(), slot2.getSlotNumber()); - assertEquals(slotPool.getAllocatedSlots().get(slot1.getAllocatedSlot().getSlotAllocationId()), slot2); + assertEquals(slot1.getTaskManagerLocation(), slot2.getTaskManagerLocation()); + assertEquals(slot1.getPhysicalSlotNumber(), slot2.getPhysicalSlotNumber()); + assertEquals(slotPool.getAllocatedSlots().get(slot1.getAllocationId()), slot2); } finally { slotPool.shutDown(); } @@ -176,7 +174,7 @@ public void testAllocateWithFreeSlot() throws Exception { ResourceID resourceID = new ResourceID("resource"); slotPoolGateway.registerTaskManager(resourceID); - CompletableFuture future1 = slotPoolGateway.allocateSlot(new SlotPoolGateway.SlotRequestID(), mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); + CompletableFuture future1 = slotPoolGateway.allocateSlot(new SlotPoolGateway.SlotRequestID(), mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); assertFalse(future1.isDone()); ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); @@ -187,23 +185,23 @@ public void testAllocateWithFreeSlot() throws Exception { AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE); assertTrue(slotPoolGateway.offerSlot(allocatedSlot).get()); - SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS); + LogicalSlot slot1 = future1.get(1, TimeUnit.SECONDS); assertTrue(future1.isDone()); // return this slot to pool slot1.releaseSlot(); - CompletableFuture future2 = slotPoolGateway.allocateSlot(new SlotPoolGateway.SlotRequestID(), mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); + CompletableFuture future2 = slotPoolGateway.allocateSlot(new SlotPoolGateway.SlotRequestID(), mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); // second allocation fulfilled by previous slot returning - SimpleSlot slot2 = future2.get(1, TimeUnit.SECONDS); + LogicalSlot slot2 = future2.get(1, TimeUnit.SECONDS); assertTrue(future2.isDone()); assertNotEquals(slot1, slot2); assertTrue(slot1.isReleased()); assertTrue(slot2.isAlive()); - assertEquals(slot1.getTaskManagerID(), slot2.getTaskManagerID()); - assertEquals(slot1.getSlotNumber(), slot2.getSlotNumber()); + assertEquals(slot1.getTaskManagerLocation(), slot2.getTaskManagerLocation()); + assertEquals(slot1.getPhysicalSlotNumber(), slot2.getPhysicalSlotNumber()); } finally { slotPool.shutDown(); } @@ -219,7 +217,7 @@ public void testOfferSlot() throws Exception { ResourceID resourceID = new ResourceID("resource"); slotPoolGateway.registerTaskManager(resourceID); - CompletableFuture future = slotPoolGateway.allocateSlot(new SlotPoolGateway.SlotRequestID(), mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); + CompletableFuture future = slotPoolGateway.allocateSlot(new SlotPoolGateway.SlotRequestID(), mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); assertFalse(future.isDone()); ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); @@ -240,7 +238,7 @@ public void testOfferSlot() throws Exception { // accepted slot assertTrue(slotPoolGateway.offerSlot(allocatedSlot).get()); - SimpleSlot slot = future.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + LogicalSlot slot = future.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); assertTrue(slot.isAlive()); // duplicated offer with using slot @@ -275,19 +273,19 @@ public void returnAllocatedSlot(Slot slot) { ResourceID resourceID = new ResourceID("resource"); slotPoolGateway.registerTaskManager(resourceID); - CompletableFuture future1 = slotPoolGateway.allocateSlot(new SlotPoolGateway.SlotRequestID(), mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); + CompletableFuture future1 = slotPoolGateway.allocateSlot(new SlotPoolGateway.SlotRequestID(), mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(JobMasterId.class), slotRequestArgumentCaptor.capture(), any(Time.class)); final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); - CompletableFuture future2 = slotPoolGateway.allocateSlot(new SlotPoolGateway.SlotRequestID(), mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); + CompletableFuture future2 = slotPoolGateway.allocateSlot(new SlotPoolGateway.SlotRequestID(), mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE); assertTrue(slotPoolGateway.offerSlot(allocatedSlot).get()); - SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS); + LogicalSlot slot1 = future1.get(1, TimeUnit.SECONDS); assertTrue(future1.isDone()); assertFalse(future2.isDone()); @@ -332,7 +330,7 @@ public void testSlotRequestCancellationUponFailingRequest() throws Exception { slotPoolGateway.connectToResourceManager(resourceManagerGateway); - CompletableFuture slotFuture = slotPoolGateway.allocateSlot( + CompletableFuture slotFuture = slotPoolGateway.allocateSlot( new SlotPoolGateway.SlotRequestID(), scheduledUnit, ResourceProfile.UNKNOWN, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/TestingLogicalSlot.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/TestingLogicalSlot.java new file mode 100644 index 0000000000000..36a47b7d7c5eb --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/TestingLogicalSlot.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.instance; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; +import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; +import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.util.Preconditions; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Simple testing logical slot for testing purposes. + */ +public class TestingLogicalSlot implements LogicalSlot { + + private final TaskManagerLocation taskManagerLocation; + + private final TaskManagerGateway taskManagerGateway; + + private final CompletableFuture releaseFuture; + + private final AtomicReference executionReference; + + private final int slotNumber; + + private final AllocationID allocationId; + + public TestingLogicalSlot() { + this( + new LocalTaskManagerLocation(), + new SimpleAckingTaskManagerGateway(), + 0, + new AllocationID()); + } + + public TestingLogicalSlot( + TaskManagerLocation taskManagerLocation, + TaskManagerGateway taskManagerGateway, + int slotNumber, + AllocationID allocationId) { + this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation); + this.taskManagerGateway = Preconditions.checkNotNull(taskManagerGateway); + this.releaseFuture = new CompletableFuture<>(); + this.executionReference = new AtomicReference<>(); + this.slotNumber = slotNumber; + this.allocationId = Preconditions.checkNotNull(allocationId); + } + + @Override + public TaskManagerLocation getTaskManagerLocation() { + return taskManagerLocation; + } + + @Override + public TaskManagerGateway getTaskManagerGateway() { + return taskManagerGateway; + } + + @Override + public boolean isAlive() { + return !releaseFuture.isDone(); + } + + @Override + public boolean isCanceled() { + return releaseFuture.isDone(); + } + + @Override + public boolean isReleased() { + return releaseFuture.isDone(); + } + + @Override + public boolean setExecution(Execution execution) { + return executionReference.compareAndSet(null, execution); + } + + @Override + public void releaseSlot() { + releaseFuture.complete(null); + } + + @Override + public int getPhysicalSlotNumber() { + return slotNumber; + } + + @Override + public AllocationID getAllocationId() { + return allocationId; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java index 0346e483d4d38..f57726c199f69 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java @@ -32,11 +32,13 @@ import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.runtime.testingUtils.TestingCluster; import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.util.TestLogger; + import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; -public class PartialConsumePipelinedResultTest { +public class PartialConsumePipelinedResultTest extends TestLogger { // Test configuration private final static int NUMBER_OF_TMS = 1; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java index 9c781ec101c18..d861455c53118 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.testingUtils.TestingCluster; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.types.IntValue; +import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; @@ -39,7 +40,7 @@ import static org.apache.flink.runtime.jobmanager.SlotCountExceedingParallelismTest.SubtaskIndexReceiver.CONFIG_KEY; -public class ScheduleOrUpdateConsumersTest { +public class ScheduleOrUpdateConsumersTest extends TestLogger { private final static int NUMBER_OF_TMS = 2; private final static int NUMBER_OF_SLOTS_PER_TM = 2; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java index 2d35ce29926a2..08db591885f49 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java @@ -18,18 +18,9 @@ package org.apache.flink.runtime.jobmanager.scheduler; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getRandomInstance; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertexWithLocation; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.instance.Instance; +import org.apache.flink.runtime.instance.LogicalSlot; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testingUtils.TestingUtils; @@ -40,6 +31,14 @@ import java.util.Collections; import java.util.concurrent.ExecutionException; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getRandomInstance; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertexWithLocation; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + public class ScheduleWithCoLocationHintTest extends TestLogger { @Test @@ -67,18 +66,18 @@ public void scheduleAllSharedAndCoLocated() { CoLocationConstraint c6 = new CoLocationConstraint(ccg); // schedule 4 tasks from the first vertex group - SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 6), sharingGroup, c1), false, Collections.emptyList()).get(); - SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 6), sharingGroup, c2), false, Collections.emptyList()).get(); - SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 6), sharingGroup, c3), false, Collections.emptyList()).get(); - SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 6), sharingGroup, c4), false, Collections.emptyList()).get(); - SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 6), sharingGroup, c1), false, Collections.emptyList()).get(); - SimpleSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 6), sharingGroup, c2), false, Collections.emptyList()).get(); - SimpleSlot s7 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 6), sharingGroup, c3), false, Collections.emptyList()).get(); - SimpleSlot s8 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 6), sharingGroup, c5), false, Collections.emptyList()).get(); - SimpleSlot s9 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 5, 6), sharingGroup, c6), false, Collections.emptyList()).get(); - SimpleSlot s10 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 6), sharingGroup, c4), false, Collections.emptyList()).get(); - SimpleSlot s11 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 4, 6), sharingGroup, c5), false, Collections.emptyList()).get(); - SimpleSlot s12 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 5, 6), sharingGroup, c6), false, Collections.emptyList()).get(); + LogicalSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 6), sharingGroup, c1), false, Collections.emptyList()).get(); + LogicalSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 6), sharingGroup, c2), false, Collections.emptyList()).get(); + LogicalSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 6), sharingGroup, c3), false, Collections.emptyList()).get(); + LogicalSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 6), sharingGroup, c4), false, Collections.emptyList()).get(); + LogicalSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 6), sharingGroup, c1), false, Collections.emptyList()).get(); + LogicalSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 6), sharingGroup, c2), false, Collections.emptyList()).get(); + LogicalSlot s7 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 6), sharingGroup, c3), false, Collections.emptyList()).get(); + LogicalSlot s8 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 6), sharingGroup, c5), false, Collections.emptyList()).get(); + LogicalSlot s9 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 5, 6), sharingGroup, c6), false, Collections.emptyList()).get(); + LogicalSlot s10 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 6), sharingGroup, c4), false, Collections.emptyList()).get(); + LogicalSlot s11 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 4, 6), sharingGroup, c5), false, Collections.emptyList()).get(); + LogicalSlot s12 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 5, 6), sharingGroup, c6), false, Collections.emptyList()).get(); assertNotNull(s1); assertNotNull(s2); @@ -94,25 +93,12 @@ public void scheduleAllSharedAndCoLocated() { assertNotNull(s12); // check that each slot got exactly two tasks - assertEquals(2, s1.getRoot().getNumberLeaves()); - assertEquals(2, s2.getRoot().getNumberLeaves()); - assertEquals(2, s3.getRoot().getNumberLeaves()); - assertEquals(2, s4.getRoot().getNumberLeaves()); - assertEquals(2, s5.getRoot().getNumberLeaves()); - assertEquals(2, s6.getRoot().getNumberLeaves()); - assertEquals(2, s7.getRoot().getNumberLeaves()); - assertEquals(2, s8.getRoot().getNumberLeaves()); - assertEquals(2, s9.getRoot().getNumberLeaves()); - assertEquals(2, s10.getRoot().getNumberLeaves()); - assertEquals(2, s11.getRoot().getNumberLeaves()); - assertEquals(2, s12.getRoot().getNumberLeaves()); - - assertEquals(s1.getTaskManagerID(), s5.getTaskManagerID()); - assertEquals(s2.getTaskManagerID(), s6.getTaskManagerID()); - assertEquals(s3.getTaskManagerID(), s7.getTaskManagerID()); - assertEquals(s4.getTaskManagerID(), s10.getTaskManagerID()); - assertEquals(s8.getTaskManagerID(), s11.getTaskManagerID()); - assertEquals(s9.getTaskManagerID(), s12.getTaskManagerID()); + assertEquals(s1.getTaskManagerLocation(), s5.getTaskManagerLocation()); + assertEquals(s2.getTaskManagerLocation(), s6.getTaskManagerLocation()); + assertEquals(s3.getTaskManagerLocation(), s7.getTaskManagerLocation()); + assertEquals(s4.getTaskManagerLocation(), s10.getTaskManagerLocation()); + assertEquals(s8.getTaskManagerLocation(), s11.getTaskManagerLocation()); + assertEquals(s9.getTaskManagerLocation(), s12.getTaskManagerLocation()); assertEquals(c1.getLocation(), s1.getTaskManagerLocation()); assertEquals(c2.getLocation(), s2.getTaskManagerLocation()); @@ -140,7 +126,7 @@ public void scheduleAllSharedAndCoLocated() { s12.releaseSlot(); assertTrue(scheduler.getNumberOfAvailableSlots() >= 1); - SimpleSlot single = scheduler.allocateSlot( + LogicalSlot single = scheduler.allocateSlot( new ScheduledUnit(getTestVertex(new JobVertexID(), 0, 1)), false, Collections.emptyList()).get(); assertNotNull(single); @@ -188,22 +174,22 @@ public void scheduleWithIntermediateRelease() { SlotSharingGroup sharingGroup = new SlotSharingGroup(); CoLocationConstraint c1 = new CoLocationConstraint(new CoLocationGroup()); - SimpleSlot s1 = scheduler.allocateSlot( + LogicalSlot s1 = scheduler.allocateSlot( new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup, c1), false, Collections.emptyList()).get(); - SimpleSlot s2 = scheduler.allocateSlot( + LogicalSlot s2 = scheduler.allocateSlot( new ScheduledUnit(getTestVertex(jid2, 0, 1), sharingGroup, c1), false, Collections.emptyList()).get(); - SimpleSlot sSolo = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 1)), false, Collections.emptyList()).get(); + LogicalSlot sSolo = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 1)), false, Collections.emptyList()).get(); - ResourceID taskManager = s1.getTaskManagerID(); + ResourceID taskManager = s1.getTaskManagerLocation().getResourceID(); s1.releaseSlot(); s2.releaseSlot(); sSolo.releaseSlot(); - SimpleSlot sNew = scheduler.allocateSlot( + LogicalSlot sNew = scheduler.allocateSlot( new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup, c1), false, Collections.emptyList()).get(); - assertEquals(taskManager, sNew.getTaskManagerID()); + assertEquals(taskManager, sNew.getTaskManagerLocation().getResourceID()); assertEquals(2, scheduler.getNumberOfLocalizedAssignments()); assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments()); @@ -235,7 +221,7 @@ public void scheduleWithReleaseNoResource() { SlotSharingGroup sharingGroup = new SlotSharingGroup(); CoLocationConstraint c1 = new CoLocationConstraint(new CoLocationGroup()); - SimpleSlot s1 = scheduler.allocateSlot( + LogicalSlot s1 = scheduler.allocateSlot( new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup, c1), false, Collections.emptyList()).get(); s1.releaseSlot(); @@ -290,23 +276,23 @@ public void scheduleMixedCoLocationSlotSharing() { scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), shareGroup), false, Collections.emptyList()); // second wave - SimpleSlot s21 = scheduler.allocateSlot( + LogicalSlot s21 = scheduler.allocateSlot( new ScheduledUnit(getTestVertex(jid2, 0, 4), shareGroup, clc1), false, Collections.emptyList()).get(); - SimpleSlot s22 = scheduler.allocateSlot( + LogicalSlot s22 = scheduler.allocateSlot( new ScheduledUnit(getTestVertex(jid2, 2, 4), shareGroup, clc2), false, Collections.emptyList()).get(); - SimpleSlot s23 = scheduler.allocateSlot( + LogicalSlot s23 = scheduler.allocateSlot( new ScheduledUnit(getTestVertex(jid2, 1, 4), shareGroup, clc3), false, Collections.emptyList()).get(); - SimpleSlot s24 = scheduler.allocateSlot( + LogicalSlot s24 = scheduler.allocateSlot( new ScheduledUnit(getTestVertex(jid2, 3, 4), shareGroup, clc4), false, Collections.emptyList()).get(); // third wave - SimpleSlot s31 = scheduler.allocateSlot( + LogicalSlot s31 = scheduler.allocateSlot( new ScheduledUnit(getTestVertex(jid3, 1, 4), shareGroup, clc2), false, Collections.emptyList()).get(); - SimpleSlot s32 = scheduler.allocateSlot( + LogicalSlot s32 = scheduler.allocateSlot( new ScheduledUnit(getTestVertex(jid3, 2, 4), shareGroup, clc3), false, Collections.emptyList()).get(); - SimpleSlot s33 = scheduler.allocateSlot( + LogicalSlot s33 = scheduler.allocateSlot( new ScheduledUnit(getTestVertex(jid3, 3, 4), shareGroup, clc4), false, Collections.emptyList()).get(); - SimpleSlot s34 = scheduler.allocateSlot( + LogicalSlot s34 = scheduler.allocateSlot( new ScheduledUnit(getTestVertex(jid3, 0, 4), shareGroup, clc1), false, Collections.emptyList()).get(); scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 4), shareGroup), false, Collections.emptyList()); @@ -314,10 +300,10 @@ public void scheduleMixedCoLocationSlotSharing() { scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 2, 4), shareGroup), false, Collections.emptyList()); scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 3, 4), shareGroup), false, Collections.emptyList()); - assertEquals(s21.getTaskManagerID(), s34.getTaskManagerID()); - assertEquals(s22.getTaskManagerID(), s31.getTaskManagerID()); - assertEquals(s23.getTaskManagerID(), s32.getTaskManagerID()); - assertEquals(s24.getTaskManagerID(), s33.getTaskManagerID()); + assertEquals(s21.getTaskManagerLocation(), s34.getTaskManagerLocation()); + assertEquals(s22.getTaskManagerLocation(), s31.getTaskManagerLocation()); + assertEquals(s23.getTaskManagerLocation(), s32.getTaskManagerLocation()); + assertEquals(s24.getTaskManagerLocation(), s33.getTaskManagerLocation()); assertEquals(4, scheduler.getNumberOfLocalizedAssignments()); assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments()); @@ -357,35 +343,32 @@ public void testGetsNonLocalFromSharingGroupFirst() { CoLocationConstraint cc2 = new CoLocationConstraint(ccg); // schedule something into the shared group so that both instances are in the sharing group - SimpleSlot s1 = scheduler.allocateSlot( + LogicalSlot s1 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); - SimpleSlot s2 = scheduler.allocateSlot( + LogicalSlot s2 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup), false, Collections.singleton(loc2)).get(); // schedule one locally to instance 1 - SimpleSlot s3 = scheduler.allocateSlot( + LogicalSlot s3 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup, cc1), false, Collections.singleton(loc1)).get(); // schedule with co location constraint (yet unassigned) and a preference for // instance 1, but it can only get instance 2 - SimpleSlot s4 = scheduler.allocateSlot( + LogicalSlot s4 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc2), false, Collections.singleton(loc1)).get(); // schedule something into the assigned co-location constraints and check that they override the // other preferences - SimpleSlot s5 = scheduler.allocateSlot( + LogicalSlot s5 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid3, 0, 2, loc2), sharingGroup, cc1), false, Collections.singleton(loc2)).get(); - SimpleSlot s6 = scheduler.allocateSlot( + LogicalSlot s6 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid3, 1, 2, loc1), sharingGroup, cc2), false, Collections.singleton(loc1)).get(); // check that each slot got three - assertEquals(3, s1.getRoot().getNumberLeaves()); - assertEquals(3, s2.getRoot().getNumberLeaves()); - - assertEquals(s1.getTaskManagerID(), s3.getTaskManagerID()); - assertEquals(s2.getTaskManagerID(), s4.getTaskManagerID()); - assertEquals(s1.getTaskManagerID(), s5.getTaskManagerID()); - assertEquals(s2.getTaskManagerID(), s6.getTaskManagerID()); + assertEquals(s1.getTaskManagerLocation(), s3.getTaskManagerLocation()); + assertEquals(s2.getTaskManagerLocation(), s4.getTaskManagerLocation()); + assertEquals(s1.getTaskManagerLocation(), s5.getTaskManagerLocation()); + assertEquals(s2.getTaskManagerLocation(), s6.getTaskManagerLocation()); // check the scheduler's bookkeeping assertEquals(0, scheduler.getNumberOfAvailableSlots()); @@ -434,9 +417,9 @@ public void testSlotReleasedInBetween() { CoLocationConstraint cc1 = new CoLocationConstraint(ccg); CoLocationConstraint cc2 = new CoLocationConstraint(ccg); - SimpleSlot s1 = scheduler.allocateSlot( + LogicalSlot s1 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1), false, Collections.singleton(loc1)).get(); - SimpleSlot s2 = scheduler.allocateSlot( + LogicalSlot s2 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2), false, Collections.singleton(loc2)).get(); s1.releaseSlot(); @@ -445,14 +428,14 @@ public void testSlotReleasedInBetween() { assertEquals(2, scheduler.getNumberOfAvailableSlots()); assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots()); - SimpleSlot s3 = scheduler.allocateSlot( + LogicalSlot s3 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup, cc1), false, Collections.singleton(loc2)).get(); - SimpleSlot s4 = scheduler.allocateSlot( + LogicalSlot s4 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc2), false, Collections.singleton(loc1)).get(); // still preserves the previous instance mapping) - assertEquals(i1.getTaskManagerID(), s3.getTaskManagerID()); - assertEquals(i2.getTaskManagerID(), s4.getTaskManagerID()); + assertEquals(i1.getTaskManagerLocation(), s3.getTaskManagerLocation()); + assertEquals(i2.getTaskManagerLocation(), s4.getTaskManagerLocation()); s3.releaseSlot(); s4.releaseSlot(); @@ -495,9 +478,9 @@ public void testSlotReleasedInBetweenAndNoNewLocal() { CoLocationConstraint cc1 = new CoLocationConstraint(ccg); CoLocationConstraint cc2 = new CoLocationConstraint(ccg); - SimpleSlot s1 = scheduler.allocateSlot( + LogicalSlot s1 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1), false, Collections.singleton(loc1)).get(); - SimpleSlot s2 = scheduler.allocateSlot( + LogicalSlot s2 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2), false, Collections.singleton(loc2)).get(); s1.releaseSlot(); @@ -506,9 +489,9 @@ public void testSlotReleasedInBetweenAndNoNewLocal() { assertEquals(2, scheduler.getNumberOfAvailableSlots()); assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots()); - SimpleSlot sa = scheduler.allocateSlot( + LogicalSlot sa = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jidx, 0, 2)), false, Collections.emptyList()).get(); - SimpleSlot sb = scheduler.allocateSlot( + LogicalSlot sb = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jidx, 1, 2)), false, Collections.emptyList()).get(); try { @@ -565,22 +548,19 @@ public void testScheduleOutOfOrder() { // schedule something from the second job vertex id before the first is filled, // and give locality preferences that hint at using the same shared slot for both // co location constraints (which we seek to prevent) - SimpleSlot s1 = scheduler.allocateSlot( + LogicalSlot s1 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1), false, Collections.singleton(loc1)).get(); - SimpleSlot s2 = scheduler.allocateSlot( + LogicalSlot s2 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup, cc2), false, Collections.singleton(loc1)).get(); - SimpleSlot s3 = scheduler.allocateSlot( + LogicalSlot s3 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc1), false, Collections.singleton(loc1)).get(); - SimpleSlot s4 = scheduler.allocateSlot( + LogicalSlot s4 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup, cc2), false, Collections.singleton(loc1)).get(); // check that each slot got three - assertEquals(2, s1.getRoot().getNumberLeaves()); - assertEquals(2, s2.getRoot().getNumberLeaves()); - - assertEquals(s1.getTaskManagerID(), s3.getTaskManagerID()); - assertEquals(s2.getTaskManagerID(), s4.getTaskManagerID()); + assertEquals(s1.getTaskManagerLocation(), s3.getTaskManagerLocation()); + assertEquals(s2.getTaskManagerLocation(), s4.getTaskManagerLocation()); // check the scheduler's bookkeeping assertEquals(0, scheduler.getNumberOfAvailableSlots()); @@ -631,19 +611,19 @@ public void nonColocationFollowsCoLocation() { CoLocationConstraint cc1 = new CoLocationConstraint(ccg); CoLocationConstraint cc2 = new CoLocationConstraint(ccg); - SimpleSlot s1 = scheduler.allocateSlot( + LogicalSlot s1 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1), false, Collections.emptyList()).get(); - SimpleSlot s2 = scheduler.allocateSlot( + LogicalSlot s2 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2), false, Collections.emptyList()).get(); - SimpleSlot s3 = scheduler.allocateSlot( + LogicalSlot s3 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s4 = scheduler.allocateSlot( + LogicalSlot s4 = scheduler.allocateSlot( new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup), false, Collections.emptyList()).get(); // check that each slot got two - assertEquals(2, s1.getRoot().getNumberLeaves()); - assertEquals(2, s2.getRoot().getNumberLeaves()); + assertEquals(s1.getTaskManagerLocation(), s3.getTaskManagerLocation()); + assertEquals(s2.getTaskManagerLocation(), s4.getTaskManagerLocation()); s1.releaseSlot(); s2.releaseSlot(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java index 7882f4a63b7fd..371cca7d267a6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java @@ -18,8 +18,9 @@ package org.apache.flink.runtime.jobmanager.scheduler; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.SimpleSlot; +import org.apache.flink.runtime.instance.LogicalSlot; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.junit.Test; @@ -30,6 +31,7 @@ import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Objects; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -125,11 +127,11 @@ public void testScheduleImmediately() { assertEquals(5, scheduler.getNumberOfAvailableSlots()); // schedule something into all slots - SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false, Collections.emptyList()).get(); - SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false, Collections.emptyList()).get(); - SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false, Collections.emptyList()).get(); - SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false, Collections.emptyList()).get(); - SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false, Collections.emptyList()).get(); + LogicalSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false, Collections.emptyList()).get(); + LogicalSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false, Collections.emptyList()).get(); + LogicalSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false, Collections.emptyList()).get(); + LogicalSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false, Collections.emptyList()).get(); + LogicalSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false, Collections.emptyList()).get(); // the slots should all be different assertTrue(areAllDistinct(s1, s2, s3, s4, s5)); @@ -148,8 +150,8 @@ public void testScheduleImmediately() { assertEquals(2, scheduler.getNumberOfAvailableSlots()); // now we can schedule some more slots - SimpleSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false, Collections.emptyList()).get(); - SimpleSlot s7 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false, Collections.emptyList()).get(); + LogicalSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false, Collections.emptyList()).get(); + LogicalSlot s7 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false, Collections.emptyList()).get(); assertTrue(areAllDistinct(s1, s2, s3, s4, s5, s6, s7)); @@ -198,10 +200,10 @@ public void testScheduleQueueing() { final int totalSlots = scheduler.getNumberOfAvailableSlots(); // all slots we ever got. - List> allAllocatedSlots = new ArrayList<>(); + List> allAllocatedSlots = new ArrayList<>(); // slots that need to be released - final Set toRelease = new HashSet(); + final Set toRelease = new HashSet<>(); // flag to track errors in the concurrent thread final AtomicBoolean errored = new AtomicBoolean(false); @@ -219,8 +221,8 @@ public void run() { toRelease.wait(); } - Iterator iter = toRelease.iterator(); - SimpleSlot next = iter.next(); + Iterator iter = toRelease.iterator(); + LogicalSlot next = iter.next(); iter.remove(); next.releaseSlot(); @@ -237,9 +239,9 @@ public void run() { disposeThread.start(); for (int i = 0; i < NUM_TASKS_TO_SCHEDULE; i++) { - CompletableFuture future = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), true, Collections.emptyList()); + CompletableFuture future = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), true, Collections.emptyList()); future.thenAcceptAsync( - (SimpleSlot slot) -> { + (LogicalSlot slot) -> { synchronized (toRelease) { toRelease.add(slot); toRelease.notifyAll(); @@ -253,8 +255,8 @@ public void run() { assertFalse("The slot releasing thread caused an error.", errored.get()); - List slotsAfter = new ArrayList(); - for (CompletableFuture future : allAllocatedSlots) { + List slotsAfter = new ArrayList<>(); + for (CompletableFuture future : allAllocatedSlots) { slotsAfter.add(future.get()); } @@ -285,7 +287,7 @@ public void testScheduleWithDyingInstances() { scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i3); - List slots = new ArrayList(); + List slots = new ArrayList<>(); slots.add(scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false, Collections.emptyList()).get()); slots.add(scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false, Collections.emptyList()).get()); slots.add(scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false, Collections.emptyList()).get()); @@ -294,8 +296,8 @@ public void testScheduleWithDyingInstances() { i2.markDead(); - for (SimpleSlot slot : slots) { - if (slot.getOwner() == i2) { + for (LogicalSlot slot : slots) { + if (Objects.equals(slot.getTaskManagerLocation().getResourceID(), i2.getTaskManagerID())) { assertTrue(slot.isCanceled()); } else { assertFalse(slot.isCanceled()); @@ -346,37 +348,48 @@ public void testSchedulingLocation() { scheduler.newInstanceAvailable(i3); // schedule something on an arbitrary instance - SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(new Instance[0])), false, Collections.emptyList()).get(); + LogicalSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(new Instance[0])), false, Collections.emptyList()).get(); // figure out how we use the location hints - Instance first = (Instance) s1.getOwner(); - Instance second = first != i1 ? i1 : i2; - Instance third = first == i3 ? i2 : i3; - + ResourceID firstResourceId = s1.getTaskManagerLocation().getResourceID(); + + List instances = Arrays.asList(i1, i2, i3); + + int index = 0; + for (; index < instances.size(); index++) { + if (Objects.equals(instances.get(index).getTaskManagerID(), firstResourceId)) { + break; + } + } + + Instance first = instances.get(index); + Instance second = instances.get((index + 1) % instances.size()); + Instance third = instances.get((index + 2) % instances.size()); + // something that needs to go to the first instance again - SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(s1.getTaskManagerLocation())), false, Collections.singleton(s1.getTaskManagerLocation())).get(); - assertEquals(first, s2.getOwner()); + LogicalSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(s1.getTaskManagerLocation())), false, Collections.singleton(s1.getTaskManagerLocation())).get(); + assertEquals(first.getTaskManagerID(), s2.getTaskManagerLocation().getResourceID()); // first or second --> second, because first is full - SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, second)), false, Arrays.asList(first.getTaskManagerLocation(), second.getTaskManagerLocation())).get(); - assertEquals(second, s3.getOwner()); + LogicalSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, second)), false, Arrays.asList(first.getTaskManagerLocation(), second.getTaskManagerLocation())).get(); + assertEquals(second.getTaskManagerID(), s3.getTaskManagerLocation().getResourceID()); // first or third --> third (because first is full) - SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false, Arrays.asList(first.getTaskManagerLocation(), third.getTaskManagerLocation())).get(); - SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false, Arrays.asList(first.getTaskManagerLocation(), third.getTaskManagerLocation())).get(); - assertEquals(third, s4.getOwner()); - assertEquals(third, s5.getOwner()); + LogicalSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false, Arrays.asList(first.getTaskManagerLocation(), third.getTaskManagerLocation())).get(); + LogicalSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false, Arrays.asList(first.getTaskManagerLocation(), third.getTaskManagerLocation())).get(); + assertEquals(third.getTaskManagerID(), s4.getTaskManagerLocation().getResourceID()); + assertEquals(third.getTaskManagerID(), s5.getTaskManagerLocation().getResourceID()); // first or third --> second, because all others are full - SimpleSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false, Arrays.asList(first.getTaskManagerLocation(), third.getTaskManagerLocation())).get(); - assertEquals(second, s6.getOwner()); + LogicalSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false, Arrays.asList(first.getTaskManagerLocation(), third.getTaskManagerLocation())).get(); + assertEquals(second.getTaskManagerID(), s6.getTaskManagerLocation().getResourceID()); // release something on the first and second instance s2.releaseSlot(); s6.releaseSlot(); - SimpleSlot s7 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false, Arrays.asList(first.getTaskManagerLocation(), third.getTaskManagerLocation())).get(); - assertEquals(first, s7.getOwner()); + LogicalSlot s7 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false, Arrays.asList(first.getTaskManagerLocation(), third.getTaskManagerLocation())).get(); + assertEquals(first.getTaskManagerID(), s7.getTaskManagerLocation().getResourceID()); assertEquals(1, scheduler.getNumberOfUnconstrainedAssignments()); assertEquals(1, scheduler.getNumberOfNonLocalizedAssignments()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java index a478eb92a3552..41a7f026dd61f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.jobmanager.scheduler; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.SimpleSlot; +import org.apache.flink.runtime.instance.LogicalSlot; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testingUtils.TestingUtils; @@ -65,10 +65,10 @@ public void scheduleSingleVertexType() { scheduler.newInstanceAvailable(i2); // schedule 4 tasks from the first vertex group - SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 8), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 8), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 8), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 8), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 8), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 8), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 8), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 8), sharingGroup), false, Collections.emptyList()).get(); assertNotNull(s1); assertNotNull(s2); @@ -93,7 +93,7 @@ public void scheduleSingleVertexType() { s3.releaseSlot(); // allocate another slot from that group - SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 8), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 8), sharingGroup), false, Collections.emptyList()).get(); assertNotNull(s5); // release all old slots @@ -101,9 +101,9 @@ public void scheduleSingleVertexType() { s2.releaseSlot(); s4.releaseSlot(); - SimpleSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 5, 8), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s7 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 6, 8), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s8 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 7, 8), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 5, 8), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s7 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 6, 8), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s8 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 7, 8), sharingGroup), false, Collections.emptyList()).get(); assertNotNull(s6); assertNotNull(s7); @@ -111,10 +111,10 @@ public void scheduleSingleVertexType() { // make sure we have two slots on the first instance, and two on the second int c = 0; - c += (s5.getTaskManagerID().equals(i1.getTaskManagerID())) ? 1 : -1; - c += (s6.getTaskManagerID().equals(i1.getTaskManagerID())) ? 1 : -1; - c += (s7.getTaskManagerID().equals(i1.getTaskManagerID())) ? 1 : -1; - c += (s8.getTaskManagerID().equals(i1.getTaskManagerID())) ? 1 : -1; + c += (s5.getTaskManagerLocation().equals(i1.getTaskManagerLocation())) ? 1 : -1; + c += (s6.getTaskManagerLocation().equals(i1.getTaskManagerLocation())) ? 1 : -1; + c += (s7.getTaskManagerLocation().equals(i1.getTaskManagerLocation())) ? 1 : -1; + c += (s8.getTaskManagerLocation().equals(i1.getTaskManagerLocation())) ? 1 : -1; assertEquals(0, c); // release all @@ -150,10 +150,10 @@ public void allocateSlotWithSharing() { scheduler.newInstanceAvailable(getRandomInstance(2)); // schedule 4 tasks from the first vertex group - SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 5), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 5), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 5), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 5), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 5), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 5), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 5), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 5), sharingGroup), false, Collections.emptyList()).get(); assertNotNull(s1); assertNotNull(s2); @@ -175,10 +175,10 @@ public void allocateSlotWithSharing() { } // schedule some tasks from the second ID group - SimpleSlot s1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 5), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 5), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s3_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 5), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 5), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 5), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 5), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s3_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 5), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 5), sharingGroup), false, Collections.emptyList()).get(); assertNotNull(s1_2); assertNotNull(s2_2); @@ -219,7 +219,7 @@ public void allocateSlotWithSharing() { } // we can schedule something from the first vertex group - SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 5), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 5), sharingGroup), false, Collections.emptyList()).get(); assertNotNull(s5); assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfSlots()); @@ -229,7 +229,7 @@ public void allocateSlotWithSharing() { // now we release a slot from the second vertex group and schedule another task from that group s2_2.releaseSlot(); - SimpleSlot s5_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 4, 5), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s5_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 4, 5), sharingGroup), false, Collections.emptyList()).get(); assertNotNull(s5_2); // release all slots @@ -270,10 +270,10 @@ public void allocateSlotWithIntermediateTotallyEmptySharingGroup() { scheduler.newInstanceAvailable(getRandomInstance(2)); // schedule 4 tasks from the first vertex group - SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup), false, Collections.emptyList()).get(); assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfSlots()); assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForGroup(jid1)); @@ -289,10 +289,10 @@ public void allocateSlotWithIntermediateTotallyEmptySharingGroup() { assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForGroup(jid2)); // schedule some tasks from the second ID group - SimpleSlot s1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s3_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s3_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup), false, Collections.emptyList()).get(); assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfSlots()); assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForGroup(jid1)); @@ -335,10 +335,10 @@ public void allocateSlotWithTemporarilyEmptyVertexGroup() { scheduler.newInstanceAvailable(getRandomInstance(2)); // schedule 4 tasks from the first vertex group - SimpleSlot s1_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s2_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s3_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s4_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s1_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s2_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s3_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s4_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup), false, Collections.emptyList()).get(); assertNotNull(s1_1); assertNotNull(s2_1); @@ -348,10 +348,10 @@ public void allocateSlotWithTemporarilyEmptyVertexGroup() { assertTrue(areAllDistinct(s1_1, s2_1, s3_1, s4_1)); // schedule 4 tasks from the second vertex group - SimpleSlot s1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 7), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 7), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s3_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 7), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 7), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 7), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 7), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s3_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 7), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 7), sharingGroup), false, Collections.emptyList()).get(); assertNotNull(s1_2); assertNotNull(s2_2); @@ -361,10 +361,10 @@ public void allocateSlotWithTemporarilyEmptyVertexGroup() { assertTrue(areAllDistinct(s1_2, s2_2, s3_2, s4_2)); // schedule 4 tasks from the third vertex group - SimpleSlot s1_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s2_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 1, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s3_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 2, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s4_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 3, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s1_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s2_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 1, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s3_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 2, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s4_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 3, 4), sharingGroup), false, Collections.emptyList()).get(); assertNotNull(s1_3); assertNotNull(s2_3); @@ -392,9 +392,9 @@ public void allocateSlotWithTemporarilyEmptyVertexGroup() { s3_2.releaseSlot(); s4_2.releaseSlot(); - SimpleSlot s5_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 5, 7), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s6_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 6, 7), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s7_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 7, 7), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s5_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 5, 7), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s6_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 6, 7), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s7_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 7, 7), sharingGroup), false, Collections.emptyList()).get(); assertNotNull(s5_2); assertNotNull(s6_2); @@ -445,9 +445,9 @@ public void allocateSlotWithTemporarilyEmptyVertexGroup2() { scheduler.newInstanceAvailable(getRandomInstance(2)); // schedule 1 tasks from the first vertex group and 2 from the second - SimpleSlot s1_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 2), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s2_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 2), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 2), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s1_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 2), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s2_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 2), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 2), sharingGroup), false, Collections.emptyList()).get(); assertNotNull(s1_1); assertNotNull(s2_1); @@ -463,7 +463,7 @@ public void allocateSlotWithTemporarilyEmptyVertexGroup2() { // this should free one slot so we can allocate one non-shared - SimpleSlot sx = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 1)), false, Collections.emptyList()).get(); + LogicalSlot sx = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 1)), false, Collections.emptyList()).get(); assertNotNull(sx); assertEquals(1, sharingGroup.getTaskAssignment().getNumberOfSlots()); @@ -498,23 +498,23 @@ public void scheduleMixedSharingAndNonSharing() { scheduler.newInstanceAvailable(getRandomInstance(2)); // schedule some individual vertices - SimpleSlot sA1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidA, 0, 2)), false, Collections.emptyList()).get(); - SimpleSlot sA2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidA, 1, 2)), false, Collections.emptyList()).get(); + LogicalSlot sA1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidA, 0, 2)), false, Collections.emptyList()).get(); + LogicalSlot sA2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidA, 1, 2)), false, Collections.emptyList()).get(); assertNotNull(sA1); assertNotNull(sA2); // schedule some vertices in the sharing group - SimpleSlot s1_0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s1_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s2_0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s2_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s1_0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s1_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s2_0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s2_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup), false, Collections.emptyList()).get(); assertNotNull(s1_0); assertNotNull(s1_1); assertNotNull(s2_0); assertNotNull(s2_1); // schedule another isolated vertex - SimpleSlot sB1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidB, 1, 3)), false, Collections.emptyList()).get(); + LogicalSlot sB1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidB, 1, 3)), false, Collections.emptyList()).get(); assertNotNull(sB1); // should not be able to schedule more vertices @@ -565,8 +565,8 @@ public void scheduleMixedSharingAndNonSharing() { // release some isolated task and check that the sharing group may grow sA1.releaseSlot(); - SimpleSlot s1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup), false, Collections.emptyList()).get(); assertNotNull(s1_2); assertNotNull(s2_2); @@ -578,19 +578,19 @@ public void scheduleMixedSharingAndNonSharing() { assertEquals(1, scheduler.getNumberOfAvailableSlots()); // schedule one more no-shared task - SimpleSlot sB0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidB, 0, 3)), false, Collections.emptyList()).get(); + LogicalSlot sB0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidB, 0, 3)), false, Collections.emptyList()).get(); assertNotNull(sB0); // release the last of the original shared slots and allocate one more non-shared slot s2_1.releaseSlot(); - SimpleSlot sB2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidB, 2, 3)), false, Collections.emptyList()).get(); + LogicalSlot sB2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidB, 2, 3)), false, Collections.emptyList()).get(); assertNotNull(sB2); // release on non-shared and add some shared slots sA2.releaseSlot(); - SimpleSlot s1_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s2_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s1_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s2_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup), false, Collections.emptyList()).get(); assertNotNull(s1_3); assertNotNull(s2_3); @@ -600,8 +600,8 @@ public void scheduleMixedSharingAndNonSharing() { s1_3.releaseSlot(); s2_3.releaseSlot(); - SimpleSlot sC0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidC, 1, 2)), false, Collections.emptyList()).get(); - SimpleSlot sC1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidC, 0, 2)), false, Collections.emptyList()).get(); + LogicalSlot sC0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidC, 1, 2)), false, Collections.emptyList()).get(); + LogicalSlot sC1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidC, 0, 2)), false, Collections.emptyList()).get(); assertNotNull(sC0); assertNotNull(sC1); @@ -649,8 +649,8 @@ public void testLocalizedAssignment1() { // schedule one to each instance - SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); - SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup), false, Collections.singleton(loc2)).get(); + LogicalSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); + LogicalSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup), false, Collections.singleton(loc2)).get(); assertNotNull(s1); assertNotNull(s2); @@ -659,8 +659,8 @@ public void testLocalizedAssignment1() { assertEquals(1, i2.getNumberOfAvailableSlots()); // schedule one from the other group to each instance - SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); - SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc2), sharingGroup), false, Collections.singleton(loc2)).get(); + LogicalSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); + LogicalSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc2), sharingGroup), false, Collections.singleton(loc2)).get(); assertNotNull(s3); assertNotNull(s4); @@ -702,8 +702,8 @@ public void testLocalizedAssignment2() { // schedule one to each instance - SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); - SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); + LogicalSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); + LogicalSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); assertNotNull(s1); assertNotNull(s2); @@ -712,8 +712,8 @@ public void testLocalizedAssignment2() { assertEquals(2, i2.getNumberOfAvailableSlots()); // schedule one from the other group to each instance - SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup), false, Collections.singleton(loc2)).get(); - SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc2), sharingGroup), false, Collections.singleton(loc2)).get(); + LogicalSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup), false, Collections.singleton(loc2)).get(); + LogicalSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc2), sharingGroup), false, Collections.singleton(loc2)).get(); assertNotNull(s3); assertNotNull(s4); @@ -753,14 +753,14 @@ public void testLocalizedAssignment3() { scheduler.newInstanceAvailable(i2); // schedule until the one instance is full - SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); - SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); - SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 4, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); - SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 4, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); + LogicalSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); + LogicalSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); + LogicalSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 4, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); + LogicalSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 4, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); // schedule two more with preference of same instance --> need to go to other instance - SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 3, 4, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); - SimpleSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 4, 4, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); + LogicalSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 3, 4, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); + LogicalSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 4, 4, loc1), sharingGroup), false, Collections.singleton(loc1)).get(); assertNotNull(s1); assertNotNull(s2); @@ -774,12 +774,12 @@ public void testLocalizedAssignment3() { assertEquals(0, i1.getNumberOfAvailableSlots()); assertEquals(0, i2.getNumberOfAvailableSlots()); - assertEquals(i1.getTaskManagerID(), s1.getTaskManagerID()); - assertEquals(i1.getTaskManagerID(), s2.getTaskManagerID()); - assertEquals(i1.getTaskManagerID(), s3.getTaskManagerID()); - assertEquals(i1.getTaskManagerID(), s4.getTaskManagerID()); - assertEquals(i2.getTaskManagerID(), s5.getTaskManagerID()); - assertEquals(i2.getTaskManagerID(), s6.getTaskManagerID()); + assertEquals(i1.getTaskManagerLocation(), s1.getTaskManagerLocation()); + assertEquals(i1.getTaskManagerLocation(), s2.getTaskManagerLocation()); + assertEquals(i1.getTaskManagerLocation(), s3.getTaskManagerLocation()); + assertEquals(i1.getTaskManagerLocation(), s4.getTaskManagerLocation()); + assertEquals(i2.getTaskManagerLocation(), s5.getTaskManagerLocation()); + assertEquals(i2.getTaskManagerLocation(), s6.getTaskManagerLocation()); // check the scheduler's bookkeeping assertEquals(4, scheduler.getNumberOfLocalizedAssignments()); @@ -806,19 +806,19 @@ public void testSequentialAllocateAndRelease() { scheduler.newInstanceAvailable(getRandomInstance(4)); // allocate something from group 1 and 2 interleaved with schedule for group 3 - SimpleSlot slot_1_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot slot_1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot slot_1_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot slot_1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot slot_2_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot slot_2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot slot_2_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot slot_2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot slot_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot slot_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot slot_1_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot slot_1_4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot slot_1_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot slot_1_4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot slot_2_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot slot_2_4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot slot_2_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot slot_2_4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup), false, Collections.emptyList()).get(); // release groups 1 and 2 @@ -834,10 +834,10 @@ public void testSequentialAllocateAndRelease() { // allocate group 4 - SimpleSlot slot_4_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot slot_4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 1, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot slot_4_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 2, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot slot_4_4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 3, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot slot_4_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot slot_4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 1, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot slot_4_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 2, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot slot_4_4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 3, 4), sharingGroup), false, Collections.emptyList()).get(); // release groups 3 and 4 @@ -888,7 +888,7 @@ public void testConcurrentAllocateAndRelease() { @Override public void run() { try { - SimpleSlot slot = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, enumerator4.getAndIncrement(), 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot slot = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, enumerator4.getAndIncrement(), 4), sharingGroup), false, Collections.emptyList()).get(); sleepUninterruptibly(rnd.nextInt(5)); slot.releaseSlot(); @@ -911,7 +911,7 @@ public void run() { public void run() { try { if (flag3.compareAndSet(false, true)) { - SimpleSlot slot = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot slot = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup), false, Collections.emptyList()).get(); sleepUninterruptibly(5); @@ -940,7 +940,7 @@ public void run() { @Override public void run() { try { - SimpleSlot slot = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, enumerator2.getAndIncrement(), 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot slot = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, enumerator2.getAndIncrement(), 4), sharingGroup), false, Collections.emptyList()).get(); // wait a bit till scheduling the successor sleepUninterruptibly(rnd.nextInt(5)); @@ -967,7 +967,7 @@ public void run() { @Override public void run() { try { - SimpleSlot slot = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, enumerator1.getAndIncrement(), 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot slot = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, enumerator1.getAndIncrement(), 4), sharingGroup), false, Collections.emptyList()).get(); // wait a bit till scheduling the successor sleepUninterruptibly(rnd.nextInt(5)); @@ -1042,24 +1042,24 @@ public void testDopIncreases() { scheduler.newInstanceAvailable(getRandomInstance(4)); // schedule one task for the first and second vertex - SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 1), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 1), sharingGroup), false, Collections.emptyList()).get(); - assertTrue( s1.getParent() == s2.getParent() ); + assertEquals( s1.getTaskManagerLocation(), s2.getTaskManagerLocation() ); assertEquals(3, scheduler.getNumberOfAvailableSlots()); - SimpleSlot s3_0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 5), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s3_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 1, 5), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s4_0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s4_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 1, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s3_0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 5), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s3_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 1, 5), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s4_0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s4_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 1, 4), sharingGroup), false, Collections.emptyList()).get(); s1.releaseSlot(); s2.releaseSlot(); - SimpleSlot s3_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 2, 5), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s3_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 3, 5), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 2, 4), sharingGroup), false, Collections.emptyList()).get(); - SimpleSlot s4_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 3, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s3_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 2, 5), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s3_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 3, 5), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 2, 4), sharingGroup), false, Collections.emptyList()).get(); + LogicalSlot s4_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 3, 4), sharingGroup), false, Collections.emptyList()).get(); try { scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 4, 5), sharingGroup), false, Collections.emptyList()).get(); From 7bca9e4613ff30ab6a9c11e673a785f3f5c86e69 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 15 Nov 2017 14:20:27 +0100 Subject: [PATCH 4/4] [FLINK-8085] Thin out LogicalSlot interface Remove isCanceled, isReleased method and decouple logical slot from Execution by introducing a Payload interface which is set for a LogicalSlot. The Payload interface is implemented by the Execution and allows to fail an implementation and obtaining a termination future. Introduce proper Execution#releaseFuture which is completed once the Execution's assigned resource has been released. This closes #5087. --- .../runtime/executiongraph/Execution.java | 83 ++++++++------ .../executiongraph/ExecutionGraph.java | 31 ++++-- .../executiongraph/ExecutionVertex.java | 4 +- .../failover/RestartIndividualStrategy.java | 2 +- .../flink/runtime/instance/Instance.java | 33 +++--- .../flink/runtime/instance/LogicalSlot.java | 54 +++++++--- .../flink/runtime/instance/SharedSlot.java | 2 +- .../flink/runtime/instance/SimpleSlot.java | 101 +++++++++++------- .../apache/flink/runtime/instance/Slot.java | 5 +- .../flink/runtime/instance/SlotPool.java | 8 +- .../instance/SlotSharingGroupAssignment.java | 8 +- .../scheduler/CoLocationConstraint.java | 2 +- .../jobmanager/scheduler/Scheduler.java | 10 +- .../runtime/jobmanager/slots/SlotOwner.java | 10 +- .../ExecutionGraphRestartTest.java | 12 +-- .../ExecutionGraphSchedulingTest.java | 24 ++++- .../ExecutionGraphSuspendTest.java | 2 +- .../runtime/executiongraph/ExecutionTest.java | 75 +++++++++++-- .../ExecutionVertexCancelTest.java | 2 +- .../ExecutionVertexSchedulingTest.java | 4 +- .../executiongraph/GlobalModVersionTest.java | 2 +- .../IndividualRestartsConcurrencyTest.java | 2 +- ...ipelinedRegionFailoverConcurrencyTest.java | 2 +- .../utils/SimpleSlotProvider.java | 4 +- .../flink/runtime/instance/InstanceTest.java | 16 +-- .../runtime/instance/SharedSlotsTest.java | 50 ++++----- .../runtime/instance/SimpleSlotTest.java | 66 ++++-------- .../flink/runtime/instance/SlotPoolTest.java | 6 +- .../runtime/instance/TestingLogicalSlot.java | 30 +++--- .../runtime/instance/TestingPayload.java | 44 ++++++++ .../scheduler/CoLocationConstraintTest.java | 2 +- .../scheduler/SchedulerIsolatedTasksTest.java | 9 +- .../jobmanager/slots/TestingSlotOwner.java | 47 ++++++++ .../runtime/testingUtils/TestingUtils.scala | 5 +- 34 files changed, 497 insertions(+), 260 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/instance/TestingPayload.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/slots/TestingSlotOwner.java 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 00a452d6887ee..12a6749486c5e 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 @@ -93,7 +93,7 @@ * occasional double-checking to ensure that the state after a completed call is as expected, and trigger correcting * actions if it is not. Many actions are also idempotent (like canceling). */ -public class Execution implements AccessExecution, Archiveable { +public class Execution implements AccessExecution, Archiveable, LogicalSlot.Payload { private static final AtomicReferenceFieldUpdater STATE_UPDATER = AtomicReferenceFieldUpdater.newUpdater(Execution.class, ExecutionState.class, "state"); @@ -135,7 +135,9 @@ public class Execution implements AccessExecution, Archiveable partialInputChannelDeploymentDescriptors; /** A future that completes once the Execution reaches a terminal ExecutionState */ - private final CompletableFuture terminationFuture; + private final CompletableFuture terminalStateFuture; + + private final CompletableFuture releaseFuture; private final CompletableFuture taskManagerLocationFuture; @@ -197,7 +199,8 @@ public Execution( markTimestamp(ExecutionState.CREATED, startTimestamp); this.partialInputChannelDeploymentDescriptors = new ConcurrentLinkedQueue<>(); - this.terminationFuture = new CompletableFuture<>(); + this.terminalStateFuture = new CompletableFuture<>(); + this.releaseFuture = new CompletableFuture<>(); this.taskManagerLocationFuture = new CompletableFuture<>(); this.assignedResource = null; @@ -329,10 +332,21 @@ public void setInitialState(TaskStateSnapshot checkpointStateHandles) { * Gets a future that completes once the task execution reaches a terminal state. * The future will be completed with specific state that the execution reached. * - * @return A future for the execution's termination + * @return A future which is completed once the execution reaches a terminal state + */ + @Override + public CompletableFuture getTerminalStateFuture() { + return terminalStateFuture; + } + + /** + * Gets the release future which is completed once the execution reaches a terminal + * state and the assigned resource has been released. + * + * @return A future which is completed once the assigned resource has been released */ - public CompletableFuture getTerminationFuture() { - return terminationFuture; + public CompletableFuture getReleaseFuture() { + return releaseFuture; } // -------------------------------------------------------------------------------------------- @@ -493,7 +507,7 @@ public void deploy() throws JobException { try { // good, we are allowed to deploy - if (!slot.setExecution(this)) { + if (!slot.tryAssignPayload(this)) { throw new JobException("Could not assign the ExecutionVertex to the slot " + slot); } @@ -608,15 +622,10 @@ else if (current == CREATED || current == SCHEDULED) { try { vertex.getExecutionGraph().deregisterExecution(this); - final LogicalSlot slot = assignedResource; - - if (slot != null) { - slot.releaseSlot(); - } + releaseAssignedResource(); } finally { vertex.executionCanceled(this); - terminationFuture.complete(CANCELED); } return; } @@ -757,6 +766,7 @@ else if (consumerState == SCHEDULED || consumerState == DEPLOYING) { * * @param t The exception that caused the task to fail. */ + @Override public void fail(Throwable t) { processFail(t, false); } @@ -880,17 +890,12 @@ void markFinished(Map> userAccumulators, IOMetrics met updateAccumulatorsAndMetrics(userAccumulators, metrics); - final LogicalSlot slot = assignedResource; - - if (slot != null) { - slot.releaseSlot(); - } + releaseAssignedResource(); vertex.getExecutionGraph().deregisterExecution(this); } finally { vertex.executionFinished(this); - terminationFuture.complete(FINISHED); } return; } @@ -938,17 +943,12 @@ else if (current == CANCELING || current == RUNNING || current == DEPLOYING) { if (transitionState(current, CANCELED)) { try { - final LogicalSlot slot = assignedResource; - - if (slot != null) { - slot.releaseSlot(); - } + releaseAssignedResource(); vertex.getExecutionGraph().deregisterExecution(this); } finally { vertex.executionCanceled(this); - terminationFuture.complete(CANCELED); } return; } @@ -1035,15 +1035,11 @@ private boolean processFail(Throwable t, boolean isCallback, Map { + if (throwable != null) { + releaseFuture.completeExceptionally(throwable); + } else { + releaseFuture.complete(null); + } + }); + } else { + // no assigned resource --> we can directly complete the release future + releaseFuture.complete(null); + } + } + // -------------------------------------------------------------------------------------------- // Miscellaneous // -------------------------------------------------------------------------------------------- @@ -1240,6 +1258,11 @@ private boolean transitionState(ExecutionState currentState, ExecutionState targ LOG.info("{} ({}) switched from {} to {}.", getVertex().getTaskNameWithSubtaskIndex(), getAttemptId(), currentState, targetState, error); } + if (targetState.isTerminal()) { + // complete the terminal state future + terminalStateFuture.complete(targetState); + } + // make sure that the state transition completes normally. // potential errors (in listeners may not affect the main logic) try { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 002f9a0647336..c4ff6fb8673a7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -965,11 +965,20 @@ public void cancel() { // we build a future that is complete once all vertices have reached a terminal state final ConjunctFuture allTerminal = FutureUtils.waitForAll(futures); - allTerminal.thenAccept( - (Void value) -> { - // cancellations may currently be overridden by failures which trigger - // restarts, so we need to pass a proper restart global version here - allVerticesInTerminalState(globalVersionForRestart); + allTerminal.whenComplete( + (Void value, Throwable throwable) -> { + if (throwable != null) { + transitionState( + JobStatus.CANCELLING, + JobStatus.FAILED, + new FlinkException( + "Could not cancel job " + getJobName() + " because not all execution job vertices could be cancelled.", + throwable)); + } else { + // cancellations may currently be overridden by failures which trigger + // restarts, so we need to pass a proper restart global version here + allVerticesInTerminalState(globalVersionForRestart); + } } ); @@ -1125,7 +1134,17 @@ else if (transitionState(current, JobStatus.FAILING, t)) { } final ConjunctFuture allTerminal = FutureUtils.waitForAll(futures); - allTerminal.thenAccept((Void value) -> allVerticesInTerminalState(globalVersionForRestart)); + allTerminal.whenComplete( + (Void ignored, Throwable throwable) -> { + if (throwable != null) { + transitionState( + JobStatus.FAILING, + JobStatus.FAILED, + new FlinkException("Could not cancel all execution job vertices properly.", throwable)); + } else { + allVerticesInTerminalState(globalVersionForRestart); + } + }); return; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index c2c986f485e09..27f2d5d95466d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -631,12 +631,12 @@ public void deployToSlot(SimpleSlot slot) throws JobException { * * @return A future that completes once the execution has reached its final state. */ - public CompletableFuture cancel() { + public CompletableFuture cancel() { // to avoid any case of mixup in the presence of concurrent calls, // we copy a reference to the stack to make sure both calls go to the same Execution final Execution exec = this.currentExecution; exec.cancel(); - return exec.getTerminationFuture(); + return exec.getReleaseFuture(); } public void stop() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/RestartIndividualStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/RestartIndividualStrategy.java index 80f1d2f9e5c6d..f133d34614bb0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/RestartIndividualStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/RestartIndividualStrategy.java @@ -106,7 +106,7 @@ public void onTaskFailure(Execution taskExecution, Throwable cause) { // Note: currently all tasks passed here are already in their terminal state, // so we could actually avoid the future. We use it anyways because it is cheap and // it helps to support better testing - final CompletableFuture terminationFuture = taskExecution.getTerminationFuture(); + final CompletableFuture terminationFuture = taskExecution.getTerminalStateFuture(); final ExecutionVertex vertexToRecover = taskExecution.getVertex(); final long globalModVersion = taskExecution.getGlobalModVersion(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java index b5c6f238a2a33..d099f6a857e85 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java @@ -18,26 +18,27 @@ package org.apache.flink.runtime.instance; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Queue; -import java.util.Set; - import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.jobmanager.scheduler.SlotAvailabilityListener; import org.apache.flink.runtime.jobmanager.slots.SlotOwner; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; - import org.apache.flink.util.Preconditions; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.flink.util.Preconditions.checkNotNull; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; /** * An instance represents a {@link org.apache.flink.runtime.taskmanager.TaskManager} @@ -164,7 +165,7 @@ public void markDead() { * the instance lock */ for (Slot slot : slots) { - slot.releaseSlot(); + slot.releaseInstanceSlot(); } } @@ -285,10 +286,10 @@ public SharedSlot allocateSharedSlot(JobID jobID, SlotSharingGroupAssignment sha * "released", this method will do nothing.

    * * @param slot The slot to return. - * @return True, if the slot was returned, false if not. + * @return Future which is completed with true, if the slot was returned, false if not. */ @Override - public boolean returnAllocatedSlot(Slot slot) { + public CompletableFuture returnAllocatedSlot(Slot slot) { checkNotNull(slot); checkArgument(!slot.isAlive(), "slot is still alive"); checkArgument(slot.getOwner() == this, "slot belongs to the wrong TaskManager."); @@ -297,7 +298,7 @@ public boolean returnAllocatedSlot(Slot slot) { LOG.debug("Return allocated slot {}.", slot); synchronized (instanceLock) { if (isDead) { - return false; + return CompletableFuture.completedFuture(false); } if (this.allocatedSlots.remove(slot)) { @@ -307,7 +308,7 @@ public boolean returnAllocatedSlot(Slot slot) { this.slotAvailabilityListener.newSlotAvailable(this); } - return true; + return CompletableFuture.completedFuture(true); } else { throw new IllegalArgumentException("Slot was not allocated from this TaskManager."); @@ -315,7 +316,7 @@ public boolean returnAllocatedSlot(Slot slot) { } } else { - return false; + return CompletableFuture.completedFuture(false); } } @@ -327,7 +328,7 @@ public void cancelAndReleaseAllSlots() { } for (Slot slot : copy) { - slot.releaseSlot(); + slot.releaseInstanceSlot(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LogicalSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LogicalSlot.java index 3ebe107683f89..e6632657e9ed2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LogicalSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LogicalSlot.java @@ -19,10 +19,13 @@ package org.apache.flink.runtime.instance; import org.apache.flink.runtime.clusterframework.types.AllocationID; -import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import javax.annotation.Nullable; + +import java.util.concurrent.CompletableFuture; + /** * A logical slot represents a resource on a TaskManager into * which a single task can be deployed. @@ -51,31 +54,29 @@ public interface LogicalSlot { boolean isAlive(); /** - * True if the slot is canceled. - * - * @return True if the slot is canceled, otherwise false - */ - boolean isCanceled(); - - /** - * True if the slot is released. + * Tries to assign a payload to this slot. This can only happens + * exactly once. * - * @return True if the slot is released, otherwise false + * @param payload to be assigned to this slot. + * @return true if the payload could be set, otherwise false */ - boolean isReleased(); + boolean tryAssignPayload(Payload payload); /** - * Sets the execution for this slot. + * Returns the set payload or null if none. * - * @param execution to set for this slot - * @return true if the slot could be set, otherwise false + * @return Payload of this slot of null if none */ - boolean setExecution(Execution execution); + @Nullable + Payload getPayload(); /** * Releases this slot. + * + * @return Future which is completed once the slot has been released, + * in case of a failure it is completed exceptionally */ - void releaseSlot(); + CompletableFuture releaseSlot(); /** * Gets the slot number on the TaskManager. @@ -90,4 +91,25 @@ public interface LogicalSlot { * @return allocation id of this slot */ AllocationID getAllocationId(); + + /** + * Payload for a logical slot. + */ + interface Payload { + + /** + * Fail the payload with the given cause. + * + * @param cause of the failure + */ + void fail(Throwable cause); + + /** + * Gets the terminal state future which is completed once the payload + * has reached a terminal state. + * + * @return Terminal state future + */ + CompletableFuture getTerminalStateFuture(); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java index 106a8efdae6cb..2ce4fc340b908 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java @@ -188,7 +188,7 @@ public boolean hasChildren() { } @Override - public void releaseSlot() { + public void releaseInstanceSlot() { assignmentGroup.releaseSharedSlot(this); if (!(isReleased() && subSlots.isEmpty())) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java index 95910281f2edb..0c9e11c9f994a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java @@ -18,18 +18,21 @@ package org.apache.flink.runtime.instance; +import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; -import org.apache.flink.runtime.executiongraph.Execution; -import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.jobmanager.scheduler.Locality; import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot; import org.apache.flink.runtime.jobmanager.slots.SlotOwner; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.AbstractID; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; import javax.annotation.Nullable; + +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; /** @@ -40,14 +43,16 @@ */ public class SimpleSlot extends Slot implements LogicalSlot { - /** The updater used to atomically swap in the execution */ - private static final AtomicReferenceFieldUpdater VERTEX_UPDATER = - AtomicReferenceFieldUpdater.newUpdater(SimpleSlot.class, Execution.class, "executedTask"); + /** The updater used to atomically swap in the payload */ + private static final AtomicReferenceFieldUpdater PAYLOAD_UPDATER = + AtomicReferenceFieldUpdater.newUpdater(SimpleSlot.class, Payload.class, "payload"); // ------------------------------------------------------------------------ - /** Task being executed in the slot. Volatile to force a memory barrier and allow for correct double-checking */ - private volatile Execution executedTask; + private final CompletableFuture releaseFuture = new CompletableFuture<>(); + + /** Id of the task being executed in the slot. Volatile to force a memory barrier and allow for correct double-checking */ + private volatile Payload payload; /** The locality attached to the slot, defining whether the slot was allocated at the desired location. */ private volatile Locality locality = Locality.UNCONSTRAINED; @@ -148,27 +153,15 @@ public int getNumberLeaves() { return 1; } - /** - * Gets the task execution attempt currently executed in this slot. This may return null, if no - * task execution attempt has been placed into this slot. - * - * @return The slot's task execution attempt, or null, if no task is executed in this slot, yet. - */ - public Execution getExecutedVertex() { - return executedTask; - } - /** * Atomically sets the executed vertex, if no vertex has been assigned to this slot so far. * - * @param executedVertex The vertex to assign to this slot. + * @param payload The vertex to assign to this slot. * @return True, if the vertex was assigned, false, otherwise. */ @Override - public boolean setExecution(Execution executedVertex) { - if (executedVertex == null) { - throw new NullPointerException(); - } + public boolean tryAssignPayload(Payload payload) { + Preconditions.checkNotNull(payload); // check that we can actually run in this slot if (isCanceled()) { @@ -176,19 +169,25 @@ public boolean setExecution(Execution executedVertex) { } // atomically assign the vertex - if (!VERTEX_UPDATER.compareAndSet(this, null, executedVertex)) { + if (!PAYLOAD_UPDATER.compareAndSet(this, null, payload)) { return false; } // we need to do a double check that we were not cancelled in the meantime if (isCanceled()) { - this.executedTask = null; + this.payload = null; return false; } return true; } + @Nullable + @Override + public Payload getPayload() { + return payload; + } + /** * Gets the locality information attached to this slot. * @return The locality attached to the slot. @@ -210,27 +209,51 @@ public void setLocality(Locality locality) { // ------------------------------------------------------------------------ @Override - public void releaseSlot() { + public void releaseInstanceSlot() { + releaseSlot(); + } + + @Override + public CompletableFuture releaseSlot() { if (!isCanceled()) { + final CompletableFuture terminationFuture; - // kill all tasks currently running in this slot - Execution exec = this.executedTask; - if (exec != null && !exec.isFinished()) { - exec.fail(new Exception("TaskManager was lost/killed: " + getTaskManagerLocation())); - } + if (payload != null) { + // trigger the failure of the slot payload + payload.fail(new FlinkException("TaskManager was lost/killed: " + getTaskManagerLocation())); - // release directly (if we are directly allocated), - // otherwise release through the parent shared slot - if (getParent() == null) { - // we have to give back the slot to the owning instance - if (markCancelled()) { - getOwner().returnAllocatedSlot(this); - } + // wait for the termination of the payload before releasing the slot + terminationFuture = payload.getTerminalStateFuture(); } else { - // we have to ask our parent to dispose us - getParent().releaseChild(this); + terminationFuture = CompletableFuture.completedFuture(null); } + + terminationFuture.whenComplete( + (Object ignored, Throwable throwable) -> { + // release directly (if we are directly allocated), + // otherwise release through the parent shared slot + if (getParent() == null) { + // we have to give back the slot to the owning instance + if (markCancelled()) { + getOwner().returnAllocatedSlot(this).whenComplete( + (value, t) -> { + if (t != null) { + releaseFuture.completeExceptionally(t); + } else { + releaseFuture.complete(null); + } + }); + } + } else { + // we have to ask our parent to dispose us + getParent().releaseChild(this); + + releaseFuture.complete(null); + } + }); } + + return releaseFuture; } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java index d6d8f12b3c289..804682bb9e396 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.instance; +import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; @@ -26,9 +27,9 @@ import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.AbstractID; -import org.apache.flink.api.common.JobID; import javax.annotation.Nullable; + import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import static org.apache.flink.util.Preconditions.checkArgument; @@ -343,7 +344,7 @@ final boolean markReleased() { * If this slot is a simple slot, it will be returned to its instance. If it is a shared slot, * it will release all of its sub-slots and release itself. */ - public abstract void releaseSlot(); + public abstract void releaseInstanceSlot(); // -------------------------------------------------------------------------------------------- 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 66af865e52ec2..771d690275232 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 @@ -643,7 +643,7 @@ else if (availableSlots.tryRemove(allocationID)) { if (slot != null) { // release the slot. // since it is not in 'allocatedSlots' any more, it will be dropped o return' - slot.releaseSlot(); + slot.releaseInstanceSlot(); } else { LOG.debug("Outdated request to fail slot [{}] with ", allocationID, cause); @@ -683,7 +683,7 @@ public CompletableFuture releaseTaskManager(final ResourceID resour final Set allocatedSlotsForResource = allocatedSlots.removeSlotsForTaskManager(resourceID); for (Slot slot : allocatedSlotsForResource) { - slot.releaseSlot(); + slot.releaseInstanceSlot(); } } @@ -1081,9 +1081,9 @@ private static class ProviderAndOwner implements SlotOwner, SlotProvider { } @Override - public boolean returnAllocatedSlot(Slot slot) { + public CompletableFuture returnAllocatedSlot(Slot slot) { gateway.returnAllocatedSlot(slot); - return true; + return CompletableFuture.completedFuture(true); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java index 4371290a8e659..45b4a969a6233 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java @@ -215,7 +215,7 @@ private SimpleSlot addSharedSlotAndAllocateSubSlot( // note that this does implicitly release the slot we have just added // as well, because we release its last child slot. That is expected // and desired. - constraintGroupSlot.releaseSlot(); + constraintGroupSlot.releaseInstanceSlot(); } } else { @@ -507,7 +507,7 @@ void releaseSimpleSlot(SimpleSlot simpleSlot) { } /** - * Called from {@link org.apache.flink.runtime.instance.SharedSlot#releaseSlot()}. + * Called from {@link org.apache.flink.runtime.instance.SharedSlot#releaseInstanceSlot()}. * * @param sharedSlot The slot to be released. */ @@ -520,7 +520,7 @@ void releaseSharedSlot(SharedSlot sharedSlot) { // by simply releasing all children, we should eventually release this slot. Set children = sharedSlot.getSubSlots(); while (children.size() > 0) { - children.iterator().next().releaseSlot(); + children.iterator().next().releaseInstanceSlot(); } } else { @@ -551,7 +551,7 @@ private void internalDisposeEmptySharedSlot(SharedSlot sharedSlot) { if (parent == null) { // root slot, return to the instance. sharedSlot.getOwner().returnAllocatedSlot(sharedSlot); - + // also, make sure we remove this slot from everywhere allSlots.remove(sharedSlot); removeSlotFromAllEntries(availableSlotsPerJid, sharedSlot); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java index c41f7bfc3c666..ffc1a7c82c8ef 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java @@ -141,7 +141,7 @@ else if (newSlot != this.sharedSlot){ "Cannot assign different location to a constraint whose location is locked."); } if (this.sharedSlot.isAlive()) { - this.sharedSlot.releaseSlot(); + this.sharedSlot.releaseInstanceSlot(); } this.sharedSlot = newSlot; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java index 271514605814a..8857be77a41ad 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java @@ -274,7 +274,7 @@ else if (slotFromGroup == null || !slotFromGroup.isAlive() || newSlot.getLocalit // if there is no slot from the group, or the new slot is local, // then we use the new slot if (slotFromGroup != null) { - slotFromGroup.releaseSlot(); + slotFromGroup.releaseInstanceSlot(); } toUse = newSlot; } @@ -282,7 +282,7 @@ else if (slotFromGroup == null || !slotFromGroup.isAlive() || newSlot.getLocalit // both are available and usable. neither is local. in that case, we may // as well use the slot from the sharing group, to minimize the number of // instances that the job occupies - newSlot.releaseSlot(); + newSlot.releaseInstanceSlot(); toUse = slotFromGroup; } @@ -299,10 +299,10 @@ else if (slotFromGroup == null || !slotFromGroup.isAlive() || newSlot.getLocalit } catch (Throwable t) { if (slotFromGroup != null) { - slotFromGroup.releaseSlot(); + slotFromGroup.releaseInstanceSlot(); } if (newSlot != null) { - newSlot.releaseSlot(); + newSlot.releaseInstanceSlot(); } ExceptionUtils.rethrow(t, "An error occurred while allocating a slot in a sharing group"); @@ -444,7 +444,7 @@ protected SimpleSlot getNewSlotForSharingGroup(ExecutionVertex vertex, } else { // could not add and allocate the sub-slot, so release shared slot - sharedSlot.releaseSlot(); + sharedSlot.releaseInstanceSlot(); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/SlotOwner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/SlotOwner.java index ad9c784c3407e..cb4488d783994 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/SlotOwner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/SlotOwner.java @@ -20,10 +20,18 @@ import org.apache.flink.runtime.instance.Slot; +import java.util.concurrent.CompletableFuture; + /** * Interface for components that hold slots and to which slots get released / recycled. */ public interface SlotOwner { - boolean returnAllocatedSlot(Slot slot); + /** + * Return the given slot to the slot owner. + * + * @param slot to return + * @return Future which is completed with true if the slot could be returned, otherwise with false + */ + CompletableFuture returnAllocatedSlot(Slot slot); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java index 8770b064f7c5f..80df8526313e5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java @@ -31,10 +31,10 @@ import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.execution.SuppressRestartsException; import org.apache.flink.runtime.executiongraph.restart.FixedDelayRestartStrategy; +import org.apache.flink.runtime.executiongraph.restart.InfiniteDelayRestartStrategy; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.executiongraph.restart.RestartCallback; import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; -import org.apache.flink.runtime.executiongraph.restart.InfiniteDelayRestartStrategy; import org.apache.flink.runtime.executiongraph.utils.NotCancelAckingTaskGateway; import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; import org.apache.flink.runtime.executiongraph.utils.SimpleSlotProvider; @@ -64,9 +64,6 @@ import org.junit.After; import org.junit.Test; -import scala.concurrent.duration.Deadline; -import scala.concurrent.duration.FiniteDuration; - import java.io.IOException; import java.net.InetAddress; import java.util.Iterator; @@ -79,6 +76,9 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; +import scala.concurrent.duration.Deadline; +import scala.concurrent.duration.FiniteDuration; + import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.SimpleActorGateway; import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.completeCancellingForAllVertices; import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createNoOpVertex; @@ -90,10 +90,8 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; - import static org.mockito.Mockito.spy; - public class ExecutionGraphRestartTest extends TestLogger { private final static int NUM_TASKS = 31; @@ -473,7 +471,7 @@ public void testFailExecutionAfterCancel() throws Exception { v.getCurrentExecutionAttempt().fail(new Exception("Test Exception")); } - assertEquals(JobStatus.CANCELED, eg.getState()); + assertEquals(JobStatus.CANCELED, eg.getTerminationFuture().get()); Execution execution = eg.getAllExecutionVertices().iterator().next().getCurrentExecutionAttempt(); 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 71ca3a5ace9d0..2e6558a57e456 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 @@ -42,6 +42,7 @@ import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot; import org.apache.flink.runtime.jobmanager.slots.SlotOwner; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; +import org.apache.flink.runtime.jobmanager.slots.TestingSlotOwner; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testtasks.NoOpInvokable; @@ -52,12 +53,15 @@ import org.mockito.verification.Timeout; import java.net.InetAddress; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.any; import static org.mockito.Mockito.mock; @@ -278,7 +282,10 @@ public void testOneSlotFailureAbortsDeploy() throws Exception { // Create the slots, futures, and the slot provider final TaskManagerGateway taskManager = mock(TaskManagerGateway.class); - final SlotOwner slotOwner = mock(SlotOwner.class); + final BlockingQueue returnedSlots = new ArrayBlockingQueue<>(parallelism); + final TestingSlotOwner slotOwner = new TestingSlotOwner(); + slotOwner.setReturnAllocatedSlotConsumer( + (Slot slot) -> returnedSlots.offer(slot.getAllocatedSlot().getSlotAllocationId())); final SimpleSlot[] sourceSlots = new SimpleSlot[parallelism]; final SimpleSlot[] targetSlots = new SimpleSlot[parallelism]; @@ -324,7 +331,9 @@ public void testOneSlotFailureAbortsDeploy() throws Exception { eg.getTerminationFuture().get(2000, TimeUnit.MILLISECONDS); // wait until all slots are back - verify(slotOwner, new Timeout(2000, times(6))).returnAllocatedSlot(any(Slot.class)); + for (int i = 0; i < parallelism; i++) { + returnedSlots.poll(2000L, TimeUnit.MILLISECONDS); + } // no deployment calls must have happened verify(taskManager, times(0)).submitTask(any(TaskDeploymentDescriptor.class), any(Time.class)); @@ -354,7 +363,10 @@ public void testTimeoutForSlotAllocation() throws Exception { final JobID jobId = new JobID(); final JobGraph jobGraph = new JobGraph(jobId, "test", vertex); - final SlotOwner slotOwner = mock(SlotOwner.class); + final BlockingQueue returnedSlots = new ArrayBlockingQueue<>(2); + final TestingSlotOwner slotOwner = new TestingSlotOwner(); + slotOwner.setReturnAllocatedSlotConsumer( + (Slot slot) -> returnedSlots.offer(slot.getAllocatedSlot().getSlotAllocationId())); final TaskManagerGateway taskManager = mock(TaskManagerGateway.class); final SimpleSlot[] slots = new SimpleSlot[parallelism]; @@ -388,14 +400,16 @@ public void testTimeoutForSlotAllocation() throws Exception { eg.getTerminationFuture().get(2000, TimeUnit.MILLISECONDS); // wait until all slots are back - verify(slotOwner, new Timeout(2000, times(2))).returnAllocatedSlot(any(Slot.class)); + for (int i = 0; i < parallelism - 1; i++) { + returnedSlots.poll(2000, TimeUnit.MILLISECONDS); + } // verify that no deployments have happened verify(taskManager, times(0)).submitTask(any(TaskDeploymentDescriptor.class), any(Time.class)); for (CompletableFuture future : slotFutures) { if (future.isDone()) { - assertTrue(future.get().isCanceled()); + assertFalse(future.get().isAlive()); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSuspendTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSuspendTest.java index f0adc327179cc..65a52bce5f189 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSuspendTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSuspendTest.java @@ -219,7 +219,7 @@ public void testSuspendedOutOfCanceled() throws Exception { validateCancelRpcCalls(gateway, parallelism); ExecutionGraphTestUtils.completeCancellingForAllVertices(eg); - assertEquals(JobStatus.CANCELED, eg.getState()); + assertEquals(JobStatus.CANCELED, eg.getTerminationFuture().get()); // suspend eg.suspend(new Exception("suspend")); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java index 43a6432634dde..c6fb83605b314 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.jobmanager.slots.SlotOwner; import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.util.TestLogger; @@ -77,7 +78,7 @@ public void testSlotReleaseOnFailedResourceAssignment() throws Exception { final Execution execution = executionJobVertex.getTaskVertices()[0].getCurrentExecutionAttempt(); - final TestingSlotOwner slotOwner = new TestingSlotOwner(); + final SingleSlotTestingSlotOwner slotOwner = new SingleSlotTestingSlotOwner(); final SimpleSlot slot = new SimpleSlot( new JobID(), @@ -117,7 +118,7 @@ public void testSlotReleaseOnExecutionCancellationInScheduled() throws Exception final JobVertex jobVertex = new JobVertex("Test vertex", jobVertexId); jobVertex.setInvokableClass(NoOpInvokable.class); - final TestingSlotOwner slotOwner = new TestingSlotOwner(); + final SingleSlotTestingSlotOwner slotOwner = new SingleSlotTestingSlotOwner(); final SimpleSlot slot = new SimpleSlot( new JobID(), @@ -167,7 +168,7 @@ public void testSlotReleaseOnExecutionCancellationInRunning() throws Exception { final JobVertex jobVertex = new JobVertex("Test vertex", jobVertexId); jobVertex.setInvokableClass(NoOpInvokable.class); - final TestingSlotOwner slotOwner = new TestingSlotOwner(); + final SingleSlotTestingSlotOwner slotOwner = new SingleSlotTestingSlotOwner(); final SimpleSlot slot = new SimpleSlot( new JobID(), @@ -268,10 +269,72 @@ public void testAnyPreferredLocationCalculation() throws ExecutionException, Int assertThat(preferredLocations, containsInAnyOrder(taskManagerLocation1, taskManagerLocation3)); } + /** + * Checks that the {@link Execution} termination future is only completed after the + * assigned slot has been released. + * + *

    NOTE: This test only fails spuriously without the fix of this commit. Thus, one has + * to execute this test multiple times to see the failure. + */ + @Test + public void testTerminationFutureIsCompletedAfterSlotRelease() throws Exception { + final JobVertexID jobVertexId = new JobVertexID(); + final JobVertex jobVertex = new JobVertex("Test vertex", jobVertexId); + jobVertex.setInvokableClass(NoOpInvokable.class); + + final SingleSlotTestingSlotOwner slotOwner = new SingleSlotTestingSlotOwner(); + + final SimpleSlot slot = new SimpleSlot( + new JobID(), + slotOwner, + new LocalTaskManagerLocation(), + 0, + new SimpleAckingTaskManagerGateway()); + + final ProgrammedSlotProvider slotProvider = new ProgrammedSlotProvider(1); + slotProvider.addSlot(jobVertexId, 0, CompletableFuture.completedFuture(slot)); + + ExecutionGraph executionGraph = ExecutionGraphTestUtils.createSimpleTestGraph( + new JobID(), + slotProvider, + new NoRestartStrategy(), + jobVertex); + + ExecutionJobVertex executionJobVertex = executionGraph.getJobVertex(jobVertexId); + + ExecutionVertex executionVertex = executionJobVertex.getTaskVertices()[0]; + + assertTrue(executionVertex.scheduleForExecution(slotProvider, false, LocationPreferenceConstraint.ANY)); + + Execution currentExecutionAttempt = executionVertex.getCurrentExecutionAttempt(); + + CompletableFuture returnedSlotFuture = slotOwner.getReturnedSlotFuture(); + CompletableFuture terminationFuture = executionVertex.cancel(); + + // run canceling in a separate thread to allow an interleaving between termination + // future callback registrations + CompletableFuture.runAsync( + () -> currentExecutionAttempt.cancelingComplete(), + TestingUtils.defaultExecutor()); + + // to increase probability for problematic interleaving, let the current thread yield the processor + Thread.yield(); + + CompletableFuture restartFuture = terminationFuture.thenApply( + ignored -> { + assertTrue(returnedSlotFuture.isDone()); + return true; + }); + + + // check if the returned slot future was completed first + restartFuture.get(); + } + /** * Slot owner which records the first returned slot. */ - public static final class TestingSlotOwner implements SlotOwner { + private static final class SingleSlotTestingSlotOwner implements SlotOwner { final CompletableFuture returnedSlot = new CompletableFuture<>(); @@ -280,8 +343,8 @@ public CompletableFuture getReturnedSlotFuture() { } @Override - public boolean returnAllocatedSlot(Slot slot) { - return returnedSlot.complete(slot); + public CompletableFuture returnAllocatedSlot(Slot slot) { + return CompletableFuture.completedFuture(returnedSlot.complete(slot)); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java index cb31d1509dd53..1b8dacad69fab 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java @@ -423,7 +423,7 @@ public void testSendCancelAndReceiveFail() throws Exception { exec.markFailed(new Exception("test")); assertTrue(exec.getState() == ExecutionState.FAILED || exec.getState() == ExecutionState.CANCELED); - assertTrue(exec.getAssignedResource().isCanceled()); + assertFalse(exec.getAssignedResource().isAlive()); assertEquals(vertices.length - 1, exec.getVertex().getExecutionGraph().getRegisteredExecutions().size()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java index 27f7f51032d87..2941739a2d18e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java @@ -59,7 +59,7 @@ public void testSlotReleasedWhenScheduledImmediately() { final Instance instance = getInstance(new ActorTaskManagerGateway(DummyActorGateway.INSTANCE)); final SimpleSlot slot = instance.allocateSimpleSlot(ejv.getJobId()); - slot.releaseSlot(); + slot.releaseInstanceSlot(); assertTrue(slot.isReleased()); Scheduler scheduler = mock(Scheduler.class); @@ -91,7 +91,7 @@ public void testSlotReleasedWhenScheduledQueued() { final Instance instance = getInstance(new ActorTaskManagerGateway(DummyActorGateway.INSTANCE)); final SimpleSlot slot = instance.allocateSimpleSlot(ejv.getJobId()); - slot.releaseSlot(); + slot.releaseInstanceSlot(); assertTrue(slot.isReleased()); final CompletableFuture future = new CompletableFuture<>(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/GlobalModVersionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/GlobalModVersionTest.java index 534c33ddce4e5..bfad3273b1686 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/GlobalModVersionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/GlobalModVersionTest.java @@ -88,7 +88,7 @@ public void testNoLocalFailoverWhileCancelling() throws Exception { exec.cancelingComplete(); } - assertEquals(JobStatus.CANCELED, graph.getState()); + assertEquals(JobStatus.CANCELED, graph.getTerminationFuture().get()); // no failure notification at all verify(mockStrategy, times(0)).onTaskFailure(any(Execution.class), any(Throwable.class)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IndividualRestartsConcurrencyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IndividualRestartsConcurrencyTest.java index c9775036e2ae4..85a6c2c4a6b6d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IndividualRestartsConcurrencyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IndividualRestartsConcurrencyTest.java @@ -139,7 +139,7 @@ public void testCancelWhileInLocalFailover() throws Exception { // now report that cancelling is complete for the other vertex vertex2.getCurrentExecutionAttempt().cancelingComplete(); - assertEquals(JobStatus.CANCELED, graph.getState()); + assertEquals(JobStatus.CANCELED, graph.getTerminationFuture().get()); assertTrue(vertex1.getCurrentExecutionAttempt().getState().isTerminal()); assertTrue(vertex2.getCurrentExecutionAttempt().getState().isTerminal()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PipelinedRegionFailoverConcurrencyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PipelinedRegionFailoverConcurrencyTest.java index 124a5b7d48e76..656c372786a3e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PipelinedRegionFailoverConcurrencyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PipelinedRegionFailoverConcurrencyTest.java @@ -113,7 +113,7 @@ public void testCancelWhileInLocalFailover() throws Exception { // now report that cancelling is complete for the other vertex vertex2.getCurrentExecutionAttempt().cancelingComplete(); - assertEquals(JobStatus.CANCELED, graph.getState()); + assertEquals(JobStatus.CANCELED, graph.getTerminationFuture().get()); assertTrue(vertex1.getCurrentExecutionAttempt().getState().isTerminal()); assertTrue(vertex2.getCurrentExecutionAttempt().getState().isTerminal()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java index 3d289832f8352..14e0e660fba4e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java @@ -96,11 +96,11 @@ public CompletableFuture allocateSlot( } @Override - public boolean returnAllocatedSlot(Slot slot) { + public CompletableFuture returnAllocatedSlot(Slot slot) { synchronized (slots) { slots.add(slot.getAllocatedSlot()); } - return true; + return CompletableFuture.completedFuture(true); } public int getNumberOfAvailableSlots() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java index 2bea039cbf72d..5b85f722bb5f6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java @@ -79,18 +79,18 @@ public void testAllocatingAndCancellingSlots() { } // release the slots. this returns them to the instance - slot1.releaseSlot(); - slot2.releaseSlot(); - slot3.releaseSlot(); - slot4.releaseSlot(); + slot1.releaseInstanceSlot(); + slot2.releaseInstanceSlot(); + slot3.releaseInstanceSlot(); + slot4.releaseInstanceSlot(); assertEquals(4, instance.getNumberOfAvailableSlots()); assertEquals(0, instance.getNumberOfAllocatedSlots()); - assertFalse(instance.returnAllocatedSlot(slot1)); - assertFalse(instance.returnAllocatedSlot(slot2)); - assertFalse(instance.returnAllocatedSlot(slot3)); - assertFalse(instance.returnAllocatedSlot(slot4)); + assertFalse(instance.returnAllocatedSlot(slot1).get()); + assertFalse(instance.returnAllocatedSlot(slot2).get()); + assertFalse(instance.returnAllocatedSlot(slot3).get()); + assertFalse(instance.returnAllocatedSlot(slot4).get()); assertEquals(4, instance.getNumberOfAvailableSlots()); assertEquals(0, instance.getNumberOfAllocatedSlots()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java index 4a6bf75e5e9f7..1e2b6af0aa229 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java @@ -80,7 +80,7 @@ public void allocateAndReleaseEmptySlot() { assertEquals(0, slot.getRootSlotNumber()); // release the slot immediately. - slot.releaseSlot(); + slot.releaseInstanceSlot(); assertTrue(slot.isCanceled()); assertTrue(slot.isReleased()); @@ -129,7 +129,7 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { SimpleSlot sub1 = assignment.addSharedSlotAndAllocateSubSlot(sharedSlot, Locality.LOCAL, vid1); assertNotNull(sub1); - assertNull(sub1.getExecutedVertex()); + assertNull(sub1.getPayload()); assertEquals(Locality.LOCAL, sub1.getLocality()); assertEquals(1, sub1.getNumberLeaves()); assertEquals(vid1, sub1.getGroupID()); @@ -148,7 +148,7 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { SimpleSlot sub2 = assignment.getSlotForTask(vid2, NO_LOCATION); assertNotNull(sub2); - assertNull(sub2.getExecutedVertex()); + assertNull(sub2.getPayload()); assertEquals(Locality.UNCONSTRAINED, sub2.getLocality()); assertEquals(1, sub2.getNumberLeaves()); assertEquals(vid2, sub2.getGroupID()); @@ -167,7 +167,7 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { SimpleSlot sub3 = assignment.getSlotForTask(vid3, Collections.singleton(instance.getTaskManagerLocation())); assertNotNull(sub3); - assertNull(sub3.getExecutedVertex()); + assertNull(sub3.getPayload()); assertEquals(Locality.LOCAL, sub3.getLocality()); assertEquals(1, sub3.getNumberLeaves()); assertEquals(vid3, sub3.getGroupID()); @@ -187,7 +187,7 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { Collections.singleton(SchedulerTestUtils.getRandomInstance(1).getTaskManagerLocation())); assertNotNull(sub4); - assertNull(sub4.getExecutedVertex()); + assertNull(sub4.getPayload()); assertEquals(Locality.NON_LOCAL, sub4.getLocality()); assertEquals(1, sub4.getNumberLeaves()); assertEquals(vid4, sub4.getGroupID()); @@ -204,7 +204,7 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { assertEquals(0, assignment.getNumberOfAvailableSlotsForGroup(vid4)); // release from the root. - sharedSlot.releaseSlot(); + sharedSlot.releaseInstanceSlot(); assertTrue(sharedSlot.isReleased()); assertTrue(sub1.isReleased()); @@ -264,7 +264,7 @@ public void allocateSimpleSlotsAndReleaseFromLeaves() { // release from the leaves. - sub2.releaseSlot(); + sub2.releaseInstanceSlot(); assertTrue(sharedSlot.isAlive()); assertTrue(sub1.isAlive()); @@ -279,7 +279,7 @@ public void allocateSimpleSlotsAndReleaseFromLeaves() { assertEquals(2, sharedSlot.getNumberLeaves()); - sub1.releaseSlot(); + sub1.releaseInstanceSlot(); assertTrue(sharedSlot.isAlive()); assertTrue(sub1.isReleased()); @@ -293,7 +293,7 @@ public void allocateSimpleSlotsAndReleaseFromLeaves() { assertEquals(1, sharedSlot.getNumberLeaves()); - sub3.releaseSlot(); + sub3.releaseInstanceSlot(); assertTrue(sharedSlot.isReleased()); assertTrue(sub1.isReleased()); @@ -348,7 +348,7 @@ public void allocateAndReleaseInMixedOrder() { assertEquals(1, assignment.getNumberOfSlots()); - sub2.releaseSlot(); + sub2.releaseInstanceSlot(); assertEquals(1, sharedSlot.getNumberLeaves()); assertEquals(0, assignment.getNumberOfAvailableSlotsForGroup(vid1)); @@ -366,8 +366,8 @@ public void allocateAndReleaseInMixedOrder() { assertEquals(0, assignment.getNumberOfAvailableSlotsForGroup(vid3)); assertEquals(1, assignment.getNumberOfSlots()); - sub3.releaseSlot(); - sub1.releaseSlot(); + sub3.releaseInstanceSlot(); + sub1.releaseInstanceSlot(); assertTrue(sharedSlot.isReleased()); assertEquals(0, sharedSlot.getNumberLeaves()); @@ -443,7 +443,7 @@ public void testAllocateAndReleaseTwoLevels() { assertFalse(constraint.isAssigned()); // we do not immediately lock the location - headSlot.releaseSlot(); + headSlot.releaseInstanceSlot(); assertEquals(1, sharedSlot.getNumberLeaves()); assertNotNull(constraint.getSharedSlot()); @@ -468,8 +468,8 @@ public void testAllocateAndReleaseTwoLevels() { assertEquals(4, sharedSlot.getNumberLeaves()); // we release our co-location constraint tasks - headSlot.releaseSlot(); - tailSlot.releaseSlot(); + headSlot.releaseInstanceSlot(); + tailSlot.releaseInstanceSlot(); assertEquals(2, sharedSlot.getNumberLeaves()); assertTrue(headSlot.isReleased()); @@ -501,10 +501,10 @@ public void testAllocateAndReleaseTwoLevels() { assertEquals(constraint.getGroupId(), constraint.getSharedSlot().getGroupID()); // release all - sourceSlot.releaseSlot(); - headSlot.releaseSlot(); - tailSlot.releaseSlot(); - sinkSlot.releaseSlot(); + sourceSlot.releaseInstanceSlot(); + headSlot.releaseInstanceSlot(); + tailSlot.releaseInstanceSlot(); + sinkSlot.releaseInstanceSlot(); assertTrue(sharedSlot.isReleased()); assertTrue(sourceSlot.isReleased()); @@ -577,10 +577,10 @@ public void testReleaseTwoLevelsFromRoot() { assertEquals(4, sharedSlot.getNumberLeaves()); // release all - sourceSlot.releaseSlot(); - headSlot.releaseSlot(); - tailSlot.releaseSlot(); - sinkSlot.releaseSlot(); + sourceSlot.releaseInstanceSlot(); + headSlot.releaseInstanceSlot(); + tailSlot.releaseInstanceSlot(); + sinkSlot.releaseInstanceSlot(); assertTrue(sharedSlot.isReleased()); assertTrue(sourceSlot.isReleased()); @@ -618,7 +618,7 @@ public void testImmediateReleaseOneLevel() { SharedSlot sharedSlot = instance.allocateSharedSlot(jobId, assignment); SimpleSlot sub = assignment.addSharedSlotAndAllocateSubSlot(sharedSlot, Locality.UNCONSTRAINED, vid); - sub.releaseSlot(); + sub.releaseInstanceSlot(); assertTrue(sub.isReleased()); assertTrue(sharedSlot.isReleased()); @@ -654,7 +654,7 @@ public void testImmediateReleaseTwoLevel() { assertNull(sub.getGroupID()); assertEquals(constraint.getSharedSlot(), sub.getParent()); - sub.releaseSlot(); + sub.releaseInstanceSlot(); assertTrue(sub.isReleased()); assertTrue(sharedSlot.isReleased()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java index db712107df385..42cbbbf03cbd4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java @@ -18,22 +18,21 @@ package org.apache.flink.runtime.instance; -import static org.mockito.Mockito.*; -import static org.junit.Assert.*; - -import java.net.InetAddress; - -import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.api.common.JobID; - +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.jobmanager.slots.ActorTaskManagerGateway; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.TestLogger; import org.junit.Test; -import org.mockito.Matchers; +import java.net.InetAddress; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; public class SimpleSlotTest extends TestLogger { @@ -45,7 +44,7 @@ public void testStateTransitions() { SimpleSlot slot = getSlot(); assertTrue(slot.isAlive()); - slot.releaseSlot(); + slot.releaseInstanceSlot(); assertFalse(slot.isAlive()); assertTrue(slot.isCanceled()); assertTrue(slot.isReleased()); @@ -76,19 +75,19 @@ public void testStateTransitions() { @Test public void testSetExecutionVertex() { try { - Execution ev = mock(Execution.class); - Execution ev_2 = mock(Execution.class); + TestingPayload payload1 = new TestingPayload(); + TestingPayload payload2 = new TestingPayload(); // assign to alive slot { SimpleSlot slot = getSlot(); - assertTrue(slot.setExecution(ev)); - assertEquals(ev, slot.getExecutedVertex()); + assertTrue(slot.tryAssignPayload(payload1)); + assertEquals(payload1, slot.getPayload()); // try to add another one - assertFalse(slot.setExecution(ev_2)); - assertEquals(ev, slot.getExecutedVertex()); + assertFalse(slot.tryAssignPayload(payload2)); + assertEquals(payload1, slot.getPayload()); } // assign to canceled slot @@ -96,8 +95,8 @@ public void testSetExecutionVertex() { SimpleSlot slot = getSlot(); assertTrue(slot.markCancelled()); - assertFalse(slot.setExecution(ev)); - assertNull(slot.getExecutedVertex()); + assertFalse(slot.tryAssignPayload(payload1)); + assertNull(slot.getPayload()); } // assign to released marked slot @@ -106,17 +105,17 @@ public void testSetExecutionVertex() { assertTrue(slot.markCancelled()); assertTrue(slot.markReleased()); - assertFalse(slot.setExecution(ev)); - assertNull(slot.getExecutedVertex()); + assertFalse(slot.tryAssignPayload(payload1)); + assertNull(slot.getPayload()); } // assign to released { SimpleSlot slot = getSlot(); - slot.releaseSlot(); + slot.releaseInstanceSlot(); - assertFalse(slot.setExecution(ev)); - assertNull(slot.getExecutedVertex()); + assertFalse(slot.tryAssignPayload(payload1)); + assertNull(slot.getPayload()); } } catch (Exception e) { @@ -125,27 +124,6 @@ public void testSetExecutionVertex() { } } - @Test - public void testReleaseCancelsVertex() { - try { - Execution ev = mock(Execution.class); - - SimpleSlot slot = getSlot(); - assertTrue(slot.setExecution(ev)); - assertEquals(ev, slot.getExecutedVertex()); - - slot.releaseSlot(); - slot.releaseSlot(); - slot.releaseSlot(); - - verify(ev, times(1)).fail(Matchers.any(Throwable.class)); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - public static SimpleSlot getSlot() throws Exception { ResourceID resourceID = ResourceID.generate(); HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024); 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 271bc2a85941e..450d3771621f5 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 @@ -154,7 +154,7 @@ public void testAllocationFulfilledByReturnedSlot() throws Exception { assertTrue(future2.isDone()); assertNotEquals(slot1, slot2); - assertTrue(slot1.isReleased()); + assertFalse(slot1.isAlive()); assertTrue(slot2.isAlive()); assertEquals(slot1.getTaskManagerLocation(), slot2.getTaskManagerLocation()); assertEquals(slot1.getPhysicalSlotNumber(), slot2.getPhysicalSlotNumber()); @@ -198,7 +198,7 @@ public void testAllocateWithFreeSlot() throws Exception { assertTrue(future2.isDone()); assertNotEquals(slot1, slot2); - assertTrue(slot1.isReleased()); + assertFalse(slot1.isAlive()); assertTrue(slot2.isAlive()); assertEquals(slot1.getTaskManagerLocation(), slot2.getTaskManagerLocation()); assertEquals(slot1.getPhysicalSlotNumber(), slot2.getPhysicalSlotNumber()); @@ -294,7 +294,7 @@ public void returnAllocatedSlot(Slot slot) { // wait until the slot has been returned slotReturnFuture.get(); - assertTrue(slot1.isReleased()); + assertFalse(slot1.isAlive()); // slot released and not usable, second allocation still not fulfilled Thread.sleep(10); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/TestingLogicalSlot.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/TestingLogicalSlot.java index 36a47b7d7c5eb..925933d28805e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/TestingLogicalSlot.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/TestingLogicalSlot.java @@ -19,13 +19,14 @@ package org.apache.flink.runtime.instance; import org.apache.flink.runtime.clusterframework.types.AllocationID; -import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.Preconditions; +import javax.annotation.Nullable; + import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; @@ -38,12 +39,12 @@ public class TestingLogicalSlot implements LogicalSlot { private final TaskManagerGateway taskManagerGateway; - private final CompletableFuture releaseFuture; - - private final AtomicReference executionReference; + private final AtomicReference payloadReference; private final int slotNumber; + private final CompletableFuture releaseFuture = new CompletableFuture<>(); + private final AllocationID allocationId; public TestingLogicalSlot() { @@ -61,8 +62,7 @@ public TestingLogicalSlot( AllocationID allocationId) { this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation); this.taskManagerGateway = Preconditions.checkNotNull(taskManagerGateway); - this.releaseFuture = new CompletableFuture<>(); - this.executionReference = new AtomicReference<>(); + this.payloadReference = new AtomicReference<>(); this.slotNumber = slotNumber; this.allocationId = Preconditions.checkNotNull(allocationId); } @@ -83,23 +83,21 @@ public boolean isAlive() { } @Override - public boolean isCanceled() { - return releaseFuture.isDone(); + public boolean tryAssignPayload(Payload payload) { + return payloadReference.compareAndSet(null, payload); } + @Nullable @Override - public boolean isReleased() { - return releaseFuture.isDone(); + public Payload getPayload() { + return payloadReference.get(); } @Override - public boolean setExecution(Execution execution) { - return executionReference.compareAndSet(null, execution); - } - - @Override - public void releaseSlot() { + public CompletableFuture releaseSlot() { releaseFuture.complete(null); + + return releaseFuture; } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/TestingPayload.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/TestingPayload.java new file mode 100644 index 0000000000000..3369882669f1d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/TestingPayload.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.instance; + +import java.util.concurrent.CompletableFuture; + +/** + * Simple payload implementation for testing purposes. + */ +public class TestingPayload implements LogicalSlot.Payload { + + private final CompletableFuture terminationFuture; + + public TestingPayload() { + this.terminationFuture = new CompletableFuture<>(); + } + + + @Override + public void fail(Throwable cause) { + terminationFuture.complete(null); + } + + @Override + public CompletableFuture getTerminalStateFuture() { + return terminationFuture; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraintTest.java index 1344aef289054..3f267ac80299f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraintTest.java @@ -143,7 +143,7 @@ public void testAssignSlotAndLockLocation() { assertEquals(instance2.getTaskManagerLocation(), constraint.getLocation()); // release the slot - slot2_1.releaseSlot(); + slot2_1.releaseInstanceSlot(); // we should still have a location assertTrue(constraint.isAssigned()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java index 371cca7d267a6..2ece70f6f8ef1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java @@ -22,6 +22,7 @@ import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.instance.LogicalSlot; import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -49,7 +50,7 @@ /** * Tests for the {@link Scheduler} when scheduling individual tasks. */ -public class SchedulerIsolatedTasksTest { +public class SchedulerIsolatedTasksTest extends TestLogger { @Test public void testAddAndRemoveInstance() { @@ -297,10 +298,10 @@ public void testScheduleWithDyingInstances() { i2.markDead(); for (LogicalSlot slot : slots) { - if (Objects.equals(slot.getTaskManagerLocation().getResourceID(), i2.getTaskManagerID())) { - assertTrue(slot.isCanceled()); + if (slot.getTaskManagerLocation().getResourceID().equals(i2.getTaskManagerID())) { + assertFalse(slot.isAlive()); } else { - assertFalse(slot.isCanceled()); + assertTrue(slot.isAlive()); } slot.releaseSlot(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/slots/TestingSlotOwner.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/slots/TestingSlotOwner.java new file mode 100644 index 0000000000000..7c124ef3c0585 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/slots/TestingSlotOwner.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmanager.slots; + +import org.apache.flink.runtime.instance.Slot; + +import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; + +/** + * {@link SlotOwner} implementation for testing purposes. + */ +public class TestingSlotOwner implements SlotOwner { + + private volatile Consumer returnAllocatedSlotConsumer; + + public void setReturnAllocatedSlotConsumer(Consumer returnAllocatedSlotConsumer) { + this.returnAllocatedSlotConsumer = returnAllocatedSlotConsumer; + } + + @Override + public CompletableFuture returnAllocatedSlot(Slot slot) { + final Consumer currentReturnAllocatedSlotConsumer = this.returnAllocatedSlotConsumer; + + if (currentReturnAllocatedSlotConsumer != null) { + currentReturnAllocatedSlotConsumer.accept(slot); + } + + return CompletableFuture.completedFuture(true); + } +} diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala index 2de6f9e59b771..2d8d02d008403 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala @@ -18,7 +18,6 @@ package org.apache.flink.runtime.testingUtils -import java.net.InetAddress import java.util import java.util.concurrent._ import java.util.{Collections, UUID} @@ -39,9 +38,7 @@ import org.apache.flink.runtime.jobmanager.{JobManager, MemoryArchivist} import org.apache.flink.runtime.jobmaster.JobMaster import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService import org.apache.flink.runtime.messages.TaskManagerMessages.{NotifyWhenRegisteredAtJobManager, RegisteredAtJobManager} -import org.apache.flink.runtime.metrics.{MetricRegistryImpl, MetricRegistryConfiguration} -import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup -import org.apache.flink.runtime.taskexecutor.{TaskManagerServices, TaskManagerServicesConfiguration} +import org.apache.flink.runtime.metrics.{MetricRegistryConfiguration, MetricRegistryImpl} import org.apache.flink.runtime.taskmanager.TaskManager import org.apache.flink.runtime.testutils.TestingResourceManager import org.apache.flink.runtime.util.LeaderRetrievalUtils