From d2f0d724c7ad28de6a070f625447aa4296c11592 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 28 Sep 2016 14:39:51 +0200 Subject: [PATCH 1/2] [FLINK-4738] [TaskManager] Port TaskManager logic to new Flip-6 TaskManager The ported logic contains the task lifecycle management methods, JobManager association and setup of TaskManager components. Introduce Rpc implementations for TaskManager components Implement metrics setup Move more TaskManager components out of the constructor to make TaskExecutor more testable Add RpcMethod annotation to TaskExecutor#confirmCheckpoint This closes #2594. --- .../CheckpointCoordinatorGateway.java | 43 ++ .../deployment/TaskDeploymentDescriptor.java | 9 + .../runtime/executiongraph/PartitionInfo.java | 47 ++ .../flink/runtime/filecache/FileCache.java | 17 +- .../jobgraph/tasks/InputSplitProvider.java | 3 +- .../tasks/InputSplitProviderException.java} | 23 +- .../jobmaster/ExecutionGraphException.java | 41 ++ .../jobmaster/JobManagerException.java | 39 ++ .../flink/runtime/jobmaster/JobMaster.java | 50 +- .../runtime/jobmaster/JobMasterGateway.java | 30 +- .../jobmaster/MiniClusterJobDispatcher.java | 2 +- .../jobmaster/SerializedInputSplit.java | 39 ++ .../runtime/operators/DataSourceTask.java | 12 +- .../runtime/query/KvStateRegistryGateway.java | 57 ++ .../taskexecutor/JobManagerConnection.java | 91 +++ .../runtime/taskexecutor/TaskExecutor.java | 627 ++++++++++++++++-- .../taskexecutor/TaskExecutorGateway.java | 80 ++- .../TaskManagerConfiguration.java | 3 +- .../taskexecutor/TaskManagerRunner.java | 3 + .../taskexecutor/TaskManagerServices.java | 51 +- .../flink/runtime/taskexecutor/TaskSlot.java | 73 ++ .../runtime/taskexecutor/TaskSlotMapping.java | 44 ++ .../exceptions/CheckpointException.java | 41 ++ .../exceptions/PartitionException.java | 41 ++ .../exceptions/TaskException.java | 41 ++ .../exceptions/TaskManagerException.java | 41 ++ .../exceptions/TaskSubmissionException.java | 41 ++ .../rpc/RpcCheckpointResponder.java | 71 ++ .../rpc/RpcInputSplitProvider.java | 73 ++ .../rpc/RpcKvStateRegistryListener.java | 73 ++ .../rpc/RpcPartitionStateChecker.java | 48 ++ .../RpcResultPartitionConsumableNotifier.java | 67 ++ .../utils/TaskExecutorMetricsInitializer.java | 257 +++++++ ...va => ActorGatewayTaskManagerActions.java} | 4 +- .../flink/runtime/taskmanager/Task.java | 23 +- .../taskmanager/TaskExecutionState.java | 4 +- .../taskmanager/TaskInputSplitProvider.java | 49 +- ...onnection.java => TaskManagerActions.java} | 2 +- .../runtime/taskmanager/TaskManager.scala | 169 +---- .../FileCacheDeleteValidationTest.java | 4 +- .../jobmaster/JobManagerRunnerMockTest.java | 1 - .../taskexecutor/TaskExecutorTest.java | 24 +- .../taskmanager/TaskAsyncCallTest.java | 2 +- .../TaskInputSplitProviderTest.java | 3 +- .../runtime/taskmanager/TaskStopTest.java | 4 +- .../flink/runtime/taskmanager/TaskTest.java | 4 +- .../source/InputFormatSourceFunction.java | 8 +- .../tasks/InterruptSensitiveRestoreTest.java | 11 +- .../runtime/tasks/StreamTaskTest.java | 4 +- 49 files changed, 2201 insertions(+), 293 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/PartitionInfo.java rename flink-runtime/src/main/java/org/apache/flink/runtime/{jobmaster/message/NextInputSplit.java => jobgraph/tasks/InputSplitProviderException.java} (66%) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/ExecutionGraphException.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerException.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/SerializedInputSplit.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryGateway.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskSlot.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskSlotMapping.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/CheckpointException.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/PartitionException.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskException.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskManagerException.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskSubmissionException.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcKvStateRegistryListener.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/utils/TaskExecutorMetricsInitializer.java rename flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/{ActorGatewayTaskManagerConnection.java => ActorGatewayTaskManagerActions.java} (93%) rename flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/{TaskManagerConnection.java => TaskManagerActions.java} (97%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java new file mode 100644 index 0000000000000..e448ebc35cf00 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java @@ -0,0 +1,43 @@ +/* + * 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.checkpoint; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.state.CheckpointStateHandles; + +public interface CheckpointCoordinatorGateway extends RpcGateway { + + void acknowledgeCheckpoint( + JobID jobID, + ExecutionAttemptID executionAttemptID, + long checkpointID, + CheckpointStateHandles checkpointStateHandles, + long synchronousDurationMillis, + long asynchronousDurationMillis, + long bytesBufferedInAlignment, + long alignmentDurationNanos); + + void declineCheckpoint( + JobID jobID, + ExecutionAttemptID executionAttemptID, + long checkpointID, + long checkpointTimestamp); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java index 7bbdb2acde9ac..b1ac665f59da5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.state.ChainedStateHandle; @@ -57,6 +58,9 @@ public final class TaskDeploymentDescriptor implements Serializable { /** The ID referencing the attempt to execute the task. */ private final ExecutionAttemptID executionId; + /** The allocation ID of the slot in which the task shall be run */ + private final AllocationID allocationID; + /** The task's name. */ private final String taskName; @@ -158,6 +162,7 @@ public TaskDeploymentDescriptor( this.operatorState = operatorState; this.keyGroupState = keyGroupState; this.partitionableOperatorState = partitionableOperatorStateHandles; + this.allocationID = new AllocationID(); } public TaskDeploymentDescriptor( @@ -322,6 +327,10 @@ public List getRequiredClasspaths() { return requiredClasspaths; } + public AllocationID getAllocationID() { + return allocationID; + } + @Override public String toString() { return String.format("TaskDeploymentDescriptor [job id: %s, job vertex id: %s, " + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/PartitionInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/PartitionInfo.java new file mode 100644 index 0000000000000..1a79a990e57ee --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/PartitionInfo.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.executiongraph; + +import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.util.Preconditions; + +/** + * Contains information where to find a partition. The partition is defined by the + * {@link IntermediateDataSetID} and the partition location is specified by + * {@link InputChannelDeploymentDescriptor}. + */ +public class PartitionInfo { + + private final IntermediateDataSetID intermediateDataSetID; + private final InputChannelDeploymentDescriptor inputChannelDeploymentDescriptor; + + public PartitionInfo(IntermediateDataSetID intermediateResultPartitionID, InputChannelDeploymentDescriptor inputChannelDeploymentDescriptor) { + this.intermediateDataSetID = Preconditions.checkNotNull(intermediateResultPartitionID); + this.inputChannelDeploymentDescriptor = Preconditions.checkNotNull(inputChannelDeploymentDescriptor); + } + + public IntermediateDataSetID getIntermediateDataSetID() { + return intermediateDataSetID; + } + + public InputChannelDeploymentDescriptor getInputChannelDeploymentDescriptor() { + return inputChannelDeploymentDescriptor; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java index b5bdcaf59e23b..a07f1a082cdfc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java @@ -33,8 +33,6 @@ import org.apache.commons.io.FileUtils; import org.apache.flink.api.common.cache.DistributedCache.DistributedCacheEntry; import org.apache.flink.api.java.tuple.Tuple4; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.fs.FSDataOutputStream; import org.apache.flink.core.fs.FileStatus; @@ -44,6 +42,7 @@ import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.flink.util.IOUtils; +import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -71,17 +70,15 @@ public class FileCache { // ------------------------------------------------------------------------ - public FileCache(Configuration config) throws IOException { - - String tempDirs = config.getString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH); + public FileCache(String[] tempDirectories) throws IOException { - String[] directories = tempDirs.split(",|" + File.pathSeparator); - storageDirectories = new File[directories.length]; + Preconditions.checkNotNull(tempDirectories); - for (int i = 0; i < directories.length; i++) { + storageDirectories = new File[tempDirectories.length]; + + for (int i = 0; i < tempDirectories.length; i++) { String cacheDirName = "flink-dist-cache-" + UUID.randomUUID().toString(); - storageDirectories[i] = new File(directories[i], cacheDirName); + storageDirectories[i] = new File(tempDirectories[i], cacheDirName); String path = storageDirectories[i].getAbsolutePath(); if (storageDirectories[i].mkdirs()) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java index e0cde1778f193..464b13fe61be3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java @@ -34,6 +34,7 @@ public interface InputSplitProvider { * @param userCodeClassLoader used to deserialize input splits * @return the next input split to be consumed by the calling task or null if the * task shall not consume any further input splits. + * @throws InputSplitProviderException if fetching the next input split fails */ - InputSplit getNextInputSplit(ClassLoader userCodeClassLoader); + InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) throws InputSplitProviderException; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/NextInputSplit.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProviderException.java similarity index 66% rename from flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/NextInputSplit.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProviderException.java index fe511eda2edcc..ac73c6f84381e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/NextInputSplit.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProviderException.java @@ -16,24 +16,21 @@ * limitations under the License. */ -package org.apache.flink.runtime.jobmaster.message; +package org.apache.flink.runtime.jobgraph.tasks; -import java.io.Serializable; +public class InputSplitProviderException extends Exception { -/** - * Contains the next input split for a task. - */ -public class NextInputSplit implements Serializable { - - private static final long serialVersionUID = -1355784074565856240L; + private static final long serialVersionUID = -8043190713983651548L; - private final byte[] splitData; + public InputSplitProviderException(String message) { + super(message); + } - public NextInputSplit(final byte[] splitData) { - this.splitData = splitData; + public InputSplitProviderException(String message, Throwable cause) { + super(message, cause); } - public byte[] getSplitData() { - return splitData; + public InputSplitProviderException(Throwable cause) { + super(cause); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/ExecutionGraphException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/ExecutionGraphException.java new file mode 100644 index 0000000000000..7c35f3dbbbe0b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/ExecutionGraphException.java @@ -0,0 +1,41 @@ +/* + * 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.runtime.executiongraph.ExecutionGraph; + +/** + * Exceptions thrown by operations on the {@link ExecutionGraph} by the {@link JobMaster}. + */ +public class ExecutionGraphException extends JobManagerException { + + private static final long serialVersionUID = -5439002256464886357L; + + public ExecutionGraphException(String message) { + super(message); + } + + public ExecutionGraphException(String message, Throwable cause) { + super(message, cause); + } + + public ExecutionGraphException(Throwable cause) { + super(cause); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerException.java new file mode 100644 index 0000000000000..bc2759d0ed0c1 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerException.java @@ -0,0 +1,39 @@ +/* + * 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; + +/** + * Base exception thrown by the {@link JobMaster}. + */ +public class JobManagerException extends Exception { + + private static final long serialVersionUID = -7290962952242188064L; + + public JobManagerException(final String message) { + super(message); + } + + public JobManagerException(final String message, Throwable cause) { + super(message, cause); + } + + public JobManagerException(Throwable cause) { + super(cause); + } +} 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 e67a167f07fb0..8f3a3428b24fd 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 @@ -38,6 +38,7 @@ import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.client.SerializedJobExecutionResult; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; @@ -60,9 +61,9 @@ import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings; import org.apache.flink.runtime.jobmanager.OnCompletionActions; import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; -import org.apache.flink.runtime.jobmaster.message.NextInputSplit; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.registration.RegisteredRpcConnection; import org.apache.flink.runtime.registration.RegistrationResponse; @@ -71,6 +72,7 @@ import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.state.CheckpointStateHandles; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.util.SerializedThrowable; import org.apache.flink.util.InstantiationUtil; @@ -507,12 +509,18 @@ public void suspendJob(final Throwable cause) { * @return Acknowledge the task execution state update */ @RpcMethod - public boolean updateTaskExecutionState(final TaskExecutionState taskExecutionState) { + public Acknowledge updateTaskExecutionState(final TaskExecutionState taskExecutionState) throws ExecutionGraphException { if (taskExecutionState == null) { - return false; + throw new NullPointerException("TaskExecutionState must not be null."); + } + + if (executionGraph.updateState(taskExecutionState)) { + return Acknowledge.get(); } else { - return executionGraph.updateState(taskExecutionState); + throw new ExecutionGraphException("The execution attempt " + + taskExecutionState.getID() + " was not found."); } + } //----------------------------------------------------------------------------------------------
 @@ -531,7 +539,7 @@ public void run() { } @RpcMethod - public NextInputSplit requestNextInputSplit( + public SerializedInputSplit requestNextInputSplit( final JobVertexID vertexID, final ExecutionAttemptID executionAttempt) throws Exception { @@ -569,7 +577,7 @@ public NextInputSplit requestNextInputSplit( try { final byte[] serializedInputSplit = InstantiationUtil.serializeObject(nextInputSplit); - return new NextInputSplit(serializedInputSplit); + return new SerializedInputSplit(serializedInputSplit); } catch (Exception ex) { log.error("Could not serialize the next input split of class {}.", nextInputSplit.getClass(), ex); IOException reason = new IOException("Could not serialize the next input split of class " + @@ -591,8 +599,36 @@ public PartitionState requestPartitionState( } @RpcMethod - public void scheduleOrUpdateConsumers(final ResultPartitionID partitionID) { + public Acknowledge scheduleOrUpdateConsumers(final ResultPartitionID partitionID) { executionGraph.scheduleOrUpdateConsumers(partitionID); + return Acknowledge.get(); + } + + @RpcMethod + public void disconnectTaskManager(final ResourceID resourceID) { + throw new UnsupportedOperationException(); + } + + @RpcMethod + public void acknowledgeCheckpoint( + JobID jobID, + ExecutionAttemptID executionAttemptID, + long checkpointID, + CheckpointStateHandles checkpointStateHandles, + long synchronousDurationMillis, + long asynchronousDurationMillis, + long bytesBufferedInAlignment, + long alignmentDurationNanos) { + throw new UnsupportedOperationException(); + } + + @RpcMethod + public void declineCheckpoint( + JobID jobID, + ExecutionAttemptID executionAttemptID, + long checkpointID, + long checkpointTimestamp) { + throw new UnsupportedOperationException(); } //---------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java index 686a3f3a02d0b..e3e57d4516a64 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java @@ -18,6 +18,9 @@ package org.apache.flink.runtime.jobmaster; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorGateway; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; @@ -27,8 +30,8 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.jobmaster.message.NextInputSplit; -import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import java.util.UUID; @@ -36,7 +39,7 @@ /** * {@link JobMaster} rpc gateway interface */ -public interface JobMasterGateway extends RpcGateway { +public interface JobMasterGateway extends CheckpointCoordinatorGateway { /** * Starting the job under the given leader session ID. @@ -57,20 +60,19 @@ public interface JobMasterGateway extends RpcGateway { * @param taskExecutionState New task execution state for a given task * @return Future flag of the task execution state update result */ - Future updateTaskExecutionState(TaskExecutionState taskExecutionState); + Future updateTaskExecutionState(TaskExecutionState taskExecutionState); /** * Requesting next input split for the {@link ExecutionJobVertex}. The next input split is sent back to the sender - * as a {@link NextInputSplit} message. + * as a {@link SerializedInputSplit} message. * * @param vertexID The job vertex id * @param executionAttempt The execution attempt id * @return The future of the input split. If there is no further input split, will return an empty object. - * @throws Exception if some error occurred or information mismatch. */ - Future requestNextInputSplit( + Future requestNextInputSplit( final JobVertexID vertexID, - final ExecutionAttemptID executionAttempt) throws Exception; + final ExecutionAttemptID executionAttempt); /** * Requests the current state of the partition. @@ -96,6 +98,16 @@ Future requestPartitionState( * The JobManager then can decide when to schedule the partition consumers of the given session. * * @param partitionID The partition which has already produced data + * @param timeout before the rpc call fails + * @return Future acknowledge of the schedule or update operation */ - void scheduleOrUpdateConsumers(final ResultPartitionID partitionID); + Future scheduleOrUpdateConsumers(final ResultPartitionID partitionID, @RpcTimeout final Time timeout); + + /** + * Disconnects the given {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} from the + * {@link JobMaster}. + * + * @param resourceID identifying the TaskManager to disconnect + */ + void disconnectTaskManager(ResourceID resourceID); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java index 792bfd564c6b6..e8fb5bb5b8f28 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java @@ -356,7 +356,7 @@ public JobExecutionResult getResult() throws JobExecutionException, InterruptedE final Throwable runnerException = this.runnerException; final JobExecutionResult result = this.result; - // (1) we check if teh job terminated with an exception + // (1) we check if the job terminated with an exception // (2) we check whether the job completed successfully // (3) we check if we have exceptions from the JobManagers. the job may still have // completed successfully in that case, if multiple JobMasters were running diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/SerializedInputSplit.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/SerializedInputSplit.java new file mode 100644 index 0000000000000..bfdc65acb1fec --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/SerializedInputSplit.java @@ -0,0 +1,39 @@ +/* + * 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 java.io.Serializable; + +public class SerializedInputSplit implements Serializable { + private static final long serialVersionUID = -2063021844254152064L; + + private final byte[] inputSplitData; + + public SerializedInputSplit(byte[] inputSplitData) { + this.inputSplitData = inputSplitData; + } + + public byte[] getInputSplitData() { + return inputSplitData; + } + + public boolean isEmpty() { + return inputSplitData == null || inputSplitData.length == 0; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java index c062bf8ba8854..1c751fd3a0068 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; +import org.apache.flink.runtime.jobgraph.tasks.InputSplitProviderException; import org.apache.flink.runtime.operators.chaining.ChainedDriver; import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException; import org.apache.flink.runtime.operators.util.DistributedRuntimeUDFContext; @@ -332,9 +333,14 @@ public boolean hasNext() { if (nextSplit != null) { return true; } - - InputSplit split = provider.getNextInputSplit(getUserCodeClassLoader()); - + + final InputSplit split; + try { + split = provider.getNextInputSplit(getUserCodeClassLoader()); + } catch (InputSplitProviderException e) { + throw new RuntimeException("Could not retrieve next input split.", e); + } + if (split != null) { this.nextSplit = split; return true; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryGateway.java new file mode 100644 index 0000000000000..d2850740f5c24 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryGateway.java @@ -0,0 +1,57 @@ +/* + * 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.query; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.state.KeyGroupRange; + +public interface KvStateRegistryGateway extends RpcGateway { + /** + * Notifies the listener about a registered KvState instance. + * + * @param jobId Job ID the KvState instance belongs to + * @param jobVertexId JobVertexID the KvState instance belongs to + * @param keyGroupRange Key group range the KvState instance belongs to + * @param registrationName Name under which the KvState is registered + * @param kvStateId ID of the KvState instance + */ + void notifyKvStateRegistered( + JobID jobId, + JobVertexID jobVertexId, + KeyGroupRange keyGroupRange, + String registrationName, + KvStateID kvStateId, + KvStateServerAddress kvStateServerAddress); + + /** + * Notifies the listener about an unregistered KvState instance. + * + * @param jobId Job ID the KvState instance belongs to + * @param jobVertexId JobVertexID the KvState instance belongs to + * @param keyGroupRange Key group range the KvState instance belongs to + * @param registrationName Name under which the KvState is registered + */ + void notifyKvStateUnregistered( + JobID jobId, + JobVertexID jobVertexId, + KeyGroupRange keyGroupRange, + String registrationName); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java new file mode 100644 index 0000000000000..ef62ef195b170 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java @@ -0,0 +1,91 @@ +/* + * 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.taskexecutor; + +import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; +import org.apache.flink.runtime.io.network.netty.PartitionStateChecker; +import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.taskmanager.CheckpointResponder; +import org.apache.flink.runtime.taskmanager.TaskManagerActions; +import org.apache.flink.util.Preconditions; + +/** + * Container class for JobManager specific communication utils used by the {@link TaskExecutor}. + */ +public class JobManagerConnection { + + // Gateway to the job master + private final JobMasterGateway jobMasterGateway; + + // Task manager actions with respect to the connected job manager + private final TaskManagerActions taskManagerActions; + + // Checkpoint responder for the specific job manager + private final CheckpointResponder checkpointResponder; + + // Library cache manager connected to the specific job manager + private final LibraryCacheManager libraryCacheManager; + + // Result partition consumable notifier for the specific job manager + private final ResultPartitionConsumableNotifier resultPartitionConsumableNotifier; + + // Partition state checker for the specific job manager + private final PartitionStateChecker partitionStateChecker; + + public JobManagerConnection( + JobMasterGateway jobMasterGateway, + TaskManagerActions taskManagerActions, + CheckpointResponder checkpointResponder, + LibraryCacheManager libraryCacheManager, + ResultPartitionConsumableNotifier resultPartitionConsumableNotifier, + PartitionStateChecker partitionStateChecker) { + + this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway); + this.taskManagerActions = Preconditions.checkNotNull(taskManagerActions); + this.checkpointResponder = Preconditions.checkNotNull(checkpointResponder); + this.libraryCacheManager = Preconditions.checkNotNull(libraryCacheManager); + this.resultPartitionConsumableNotifier = Preconditions.checkNotNull(resultPartitionConsumableNotifier); + this.partitionStateChecker = Preconditions.checkNotNull(partitionStateChecker); + } + + public JobMasterGateway getJobManagerGateway() { + return jobMasterGateway; + } + + public TaskManagerActions getTaskManagerActions() { + return taskManagerActions; + } + + public CheckpointResponder getCheckpointResponder() { + return checkpointResponder; + } + + public LibraryCacheManager getLibraryCacheManager() { + return libraryCacheManager; + } + + public ResultPartitionConsumableNotifier getResultPartitionConsumableNotifier() { + return resultPartitionConsumableNotifier; + } + + public PartitionStateChecker getPartitionStateChecker() { + return partitionStateChecker; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index c0041a3d75fcb..35b639b1ffd47 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -18,15 +18,48 @@ package org.apache.flink.runtime.taskexecutor; +import org.apache.flink.configuration.UnmodifiableConfiguration; +import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; +import org.apache.flink.runtime.blob.BlobCache; +import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.SlotID; import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.concurrent.ApplyFunction; +import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; +import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; +import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.PartitionInfo; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.io.network.netty.PartitionStateChecker; +import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; +import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestRegistered; import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestRejected; +import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; +import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestReply; import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.taskexecutor.exceptions.CheckpointException; +import org.apache.flink.runtime.taskexecutor.exceptions.PartitionException; +import org.apache.flink.runtime.taskexecutor.exceptions.TaskException; +import org.apache.flink.runtime.taskexecutor.exceptions.TaskSubmissionException; +import org.apache.flink.runtime.taskexecutor.rpc.RpcCheckpointResponder; +import org.apache.flink.runtime.taskexecutor.rpc.RpcInputSplitProvider; +import org.apache.flink.runtime.taskexecutor.rpc.RpcPartitionStateChecker; +import org.apache.flink.runtime.taskexecutor.rpc.RpcResultPartitionConsumableNotifier; +import org.apache.flink.runtime.taskmanager.CheckpointResponder; +import org.apache.flink.runtime.taskmanager.Task; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.runtime.taskmanager.TaskManagerActions; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -38,11 +71,17 @@ import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcService; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; +import org.apache.flink.util.Preconditions; import java.util.HashSet; import java.util.Set; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; import java.util.UUID; import static org.apache.flink.util.Preconditions.checkArgument; @@ -50,12 +89,10 @@ /** * TaskExecutor implementation. The task executor is responsible for the execution of multiple - * {@link org.apache.flink.runtime.taskmanager.Task}. + * {@link Task}. */ public class TaskExecutor extends RpcEndpoint { - private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class); - /** The connection information of this task manager */ private final TaskManagerLocation taskManagerLocation; @@ -77,19 +114,38 @@ public class TaskExecutor extends RpcEndpoint { /** The metric registry in the task manager */ private final MetricRegistry metricRegistry; - /** The number of slots in the task manager, should be 1 for YARN */ - private final int numberOfSlots; - /** The fatal error handler to use in case of a fatal error */ private final FatalErrorHandler fatalErrorHandler; + private final TaskManagerMetricGroup taskManagerMetricGroup; + + private final BroadcastVariableManager broadcastVariableManager; + /** Slots which have become available but haven't been confirmed by the RM */ private final Set unconfirmedFreeSlots; + + private final FileCache fileCache; + + // TODO: Try to get rid of it + private final TaskManagerRuntimeInfo taskManagerRuntimeInfo; + // --------- resource manager -------- private TaskExecutorToResourceManagerConnection resourceManagerConnection; + // --------- job manager connections ----------- + + private Map jobManagerConnections; + + // --------- Slot allocation table -------- + + private Map taskSlots; + + // --------- Slot allocation table -------- + + private Map taskSlotMappings; + // ------------------------------------------------------------------------ public TaskExecutor( @@ -101,6 +157,9 @@ public TaskExecutor( NetworkEnvironment networkEnvironment, HighAvailabilityServices haServices, MetricRegistry metricRegistry, + TaskManagerMetricGroup taskManagerMetricGroup, + BroadcastVariableManager broadcastVariableManager, + FileCache fileCache, FatalErrorHandler fatalErrorHandler) { super(rpcService); @@ -115,10 +174,19 @@ public TaskExecutor( this.haServices = checkNotNull(haServices); this.metricRegistry = checkNotNull(metricRegistry); this.fatalErrorHandler = checkNotNull(fatalErrorHandler); + this.taskManagerMetricGroup = checkNotNull(taskManagerMetricGroup); + this.broadcastVariableManager = checkNotNull(broadcastVariableManager); + this.fileCache = checkNotNull(fileCache); + this.taskManagerRuntimeInfo = new TaskManagerRuntimeInfo( + taskManagerLocation.getHostname(), + new UnmodifiableConfiguration(taskManagerConfiguration.getConfiguration()), + taskManagerConfiguration.getTmpDirPaths()); - this.numberOfSlots = taskManagerConfiguration.getNumberSlots(); + this.jobManagerConnections = new HashMap<>(4); this.unconfirmedFreeSlots = new HashSet<>(); + this.taskSlots = new HashMap<>(taskManagerConfiguration.getNumberSlots()); + this.taskSlotMappings = new HashMap<>(taskManagerConfiguration.getNumberSlots() * 2); } // ------------------------------------------------------------------------ @@ -137,12 +205,436 @@ public void start() { } } + /** + * Called to shut down the TaskManager. The method closes all TaskManager services. + */ + @Override + public void shutDown() { + log.info("Stopping TaskManager {}.", getAddress()); + + if (resourceManagerConnection.isConnected()) { + try { + resourceManagerConnection.close(); + } catch (Exception e) { + log.error("Could not cleanly close the ResourceManager connection.", e); + } + } + + try { + ioManager.shutdown(); + } catch (Exception e) { + log.error("IOManager did not shut down properly.", e); + } + + try { + memoryManager.shutdown(); + } catch (Exception e) { + log.error("MemoryManager did not shut down properly.", e); + } + + try { + networkEnvironment.shutdown(); + } catch (Exception e) { + log.error("Network environment did not shut down properly.", e); + } + + try { + fileCache.shutdown(); + } catch (Exception e) { + log.error("File cache did not shut down properly.", e); + } + + try { + metricRegistry.shutdown(); + } catch (Exception e) { + log.error("MetricRegistry did not shut down properly.", e); + } + + log.info("Stopped TaskManager {}.", getAddress()); + } + + // ======================================================================== + // RPC methods + // ======================================================================== + + // ---------------------------------------------------------------------- + // Task lifecycle RPCs + // ---------------------------------------------------------------------- + + @RpcMethod + public Acknowledge submitTask(TaskDeploymentDescriptor tdd, ResourceID jobManagerID) throws TaskSubmissionException { + + JobManagerConnection jobManagerConnection = getJobManagerConnection(jobManagerID); + + if (jobManagerConnection == null) { + final String message = "Could not submit task because JobManager " + jobManagerID + + " was not associated."; + + log.debug(message); + throw new TaskSubmissionException(message); + } + + TaskSlot taskSlot = taskSlots.get(tdd.getAllocationID()); + + if (taskSlot == null) { + final String message = "No task slot allocated for allocation ID " + tdd.getAllocationID() + '.'; + log.debug(message); + throw new TaskSubmissionException(message); + } + + TaskMetricGroup taskMetricGroup = taskManagerMetricGroup.addTaskForJob(tdd); + + InputSplitProvider inputSplitProvider = new RpcInputSplitProvider( + jobManagerConnection.getJobManagerGateway(), + tdd.getJobID(), + tdd.getVertexID(), + tdd.getExecutionId(), + taskManagerConfiguration.getTimeout()); + + TaskManagerActions taskManagerActions = jobManagerConnection.getTaskManagerActions(); + CheckpointResponder checkpointResponder = jobManagerConnection.getCheckpointResponder(); + LibraryCacheManager libraryCache = jobManagerConnection.getLibraryCacheManager(); + ResultPartitionConsumableNotifier resultPartitionConsumableNotifier = jobManagerConnection.getResultPartitionConsumableNotifier(); + PartitionStateChecker partitionStateChecker = jobManagerConnection.getPartitionStateChecker(); + + Task task = new Task( + tdd, + memoryManager, + ioManager, + networkEnvironment, + broadcastVariableManager, + taskManagerActions, + inputSplitProvider, + checkpointResponder, + libraryCache, + fileCache, + taskManagerRuntimeInfo, + taskMetricGroup, + resultPartitionConsumableNotifier, + partitionStateChecker, + getRpcService().getExecutor()); + + log.info("Received task {}.", task.getTaskInfo().getTaskNameWithSubtasks()); + + if(taskSlot.add(task)) { + TaskSlotMapping taskSlotMapping = new TaskSlotMapping(task, taskSlot); + + taskSlotMappings.put(task.getExecutionId(), taskSlotMapping); + task.startTaskThread(); + + return Acknowledge.get(); + } else { + final String message = "TaskManager already contains a task for id " + + task.getExecutionId() + '.'; + + log.debug(message); + throw new TaskSubmissionException(message); + } + } + + @RpcMethod + public Acknowledge cancelTask(ExecutionAttemptID executionAttemptID) throws TaskException { + final Task task = getTask(executionAttemptID); + + if (task != null) { + try { + task.cancelExecution(); + return Acknowledge.get(); + } catch (Throwable t) { + throw new TaskException("Cannot cancel task for execution " + executionAttemptID + '.', t); + } + } else { + final String message = "Cannot find task to stop for execution " + executionAttemptID + '.'; + + log.debug(message); + throw new TaskException(message); + } + } + + @RpcMethod + public Acknowledge stopTask(ExecutionAttemptID executionAttemptID) throws TaskException { + final Task task = getTask(executionAttemptID); + + if (task != null) { + try { + task.stopExecution(); + return Acknowledge.get(); + } catch (Throwable t) { + throw new TaskException("Cannot stop task for execution " + executionAttemptID + '.', t); + } + } else { + final String message = "Cannot find task to stop for execution " + executionAttemptID + '.'; + + log.debug(message); + throw new TaskException(message); + } + } + + // ---------------------------------------------------------------------- + // Partition lifecycle RPCs + // ---------------------------------------------------------------------- + + @RpcMethod + public Acknowledge updatePartitions(final ExecutionAttemptID executionAttemptID, Collection partitionInfos) throws PartitionException { + final Task task = getTask(executionAttemptID); + + if (task != null) { + for (final PartitionInfo partitionInfo: partitionInfos) { + IntermediateDataSetID intermediateResultPartitionID = partitionInfo.getIntermediateDataSetID(); + + final SingleInputGate singleInputGate = task.getInputGateById(intermediateResultPartitionID); + + if (singleInputGate != null) { + // Run asynchronously because it might be blocking + getRpcService().execute(new Runnable() { + @Override + public void run() { + try { + singleInputGate.updateInputChannel(partitionInfo.getInputChannelDeploymentDescriptor()); + } catch (IOException | InterruptedException e) { + log.error("Could not update input data location for task {}. Trying to fail task.", task.getTaskInfo().getTaskName(), e); + + try { + task.failExternally(e); + } catch (RuntimeException re) { + // TODO: Check whether we need this or make exception in failExtenally checked + log.error("Failed canceling task with execution ID {} after task update failure.", executionAttemptID, re); + } + } + } + }); + } else { + throw new PartitionException("No reader with ID " + + intermediateResultPartitionID + " for task " + executionAttemptID + + " was found."); + } + } + + return Acknowledge.get(); + } else { + log.debug("Discard update for input partitions of task {}. Task is no longer running.", executionAttemptID); + return Acknowledge.get(); + } + } + + @RpcMethod + public void failPartition(ExecutionAttemptID executionAttemptID) { + log.info("Discarding the results produced by task execution {}.", executionAttemptID); + + try { + networkEnvironment.getResultPartitionManager().releasePartitionsProducedBy(executionAttemptID); + } catch (Throwable t) { + // TODO: Do we still need this catch branch? + onFatalError(t); + } + + // TODO: Maybe it's better to return an Acknowledge here to notify the JM about the success/failure with an Exception + } + + // ---------------------------------------------------------------------- + // Checkpointing RPCs + // ---------------------------------------------------------------------- + + @RpcMethod + public Acknowledge triggerCheckpoint(ExecutionAttemptID executionAttemptID, long checkpointId, long checkpointTimestamp) throws CheckpointException { + log.debug("Trigger checkpoint {}@{} for {}.", checkpointId, checkpointTimestamp, executionAttemptID); + + final Task task = getTask(executionAttemptID); + + if (task != null) { + task.triggerCheckpointBarrier(checkpointId, checkpointTimestamp); + + return Acknowledge.get(); + } else { + final String message = "TaskManager received a checkpoint request for unknown task " + executionAttemptID + '.'; + + log.debug(message); + throw new CheckpointException(message); + } + } + + @RpcMethod + public Acknowledge confirmCheckpoint(ExecutionAttemptID executionAttemptID, long checkpointId, long checkpointTimestamp) throws CheckpointException { + log.debug("Confirm checkpoint {}@{} for {}.", checkpointId, checkpointTimestamp, executionAttemptID); + + final Task task = getTask(executionAttemptID); + + if (task != null) { + task.notifyCheckpointComplete(checkpointId); + + return Acknowledge.get(); + } else { + final String message = "TaskManager received a checkpoint confirmation for unknown task " + executionAttemptID + '.'; + + log.debug(message); + throw new CheckpointException(message); + } + } + + /** + * Requests a slot from the TaskManager + * + * @param slotID Slot id for the request + * @param allocationID id for the request + * @param resourceManagerLeaderID current leader id of the ResourceManager + * @return answer to the slot request + */ + @RpcMethod + public TMSlotRequestReply requestSlot(SlotID slotID, AllocationID allocationID, UUID resourceManagerLeaderID) { + if (!resourceManagerConnection.getTargetLeaderId().equals(resourceManagerLeaderID)) { + return new TMSlotRequestRejected( + resourceManagerConnection.getRegistrationId(), getResourceID(), allocationID); + } + if (unconfirmedFreeSlots.contains(slotID)) { + // check if request has not been blacklisted because the notification of a free slot + // has not been confirmed by the ResourceManager + return new TMSlotRequestRejected( + resourceManagerConnection.getRegistrationId(), getResourceID(), allocationID); + } + return new TMSlotRequestRegistered(new InstanceID(), ResourceID.generate(), allocationID); + + } + // ------------------------------------------------------------------------ - // RPC methods - ResourceManager related + // Internal methods // ------------------------------------------------------------------------ - @RpcMethod - public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLeaderId) { + private JobManagerConnection getJobManagerConnection(ResourceID jobManagerID) { + return jobManagerConnections.get(jobManagerID); + } + + private Task getTask(ExecutionAttemptID executionAttemptID) { + TaskSlotMapping taskSlotMapping = taskSlotMappings.get(executionAttemptID); + + if (taskSlotMapping != null) { + return taskSlotMapping.getTask(); + } else { + return null; + } + } + + private Task removeTask(ExecutionAttemptID executionAttemptID) { + TaskSlotMapping taskSlotMapping = taskSlotMappings.remove(executionAttemptID); + + if (taskSlotMapping != null) { + final Task task = taskSlotMapping.getTask(); + final TaskSlot taskSlot = taskSlotMapping.getTaskSlot(); + + taskSlot.remove(task); + + return task; + } else { + return null; + } + } + + private Iterable getAllTasks() { + final Iterator taskEntryIterator = taskSlotMappings.values().iterator(); + final Iterator iterator = new Iterator() { + @Override + public boolean hasNext() { + return taskEntryIterator.hasNext(); + } + + @Override + public Task next() { + return taskEntryIterator.next().getTask(); + } + + @Override + public void remove() { + taskEntryIterator.remove(); + } + }; + + return new Iterable() { + @Override + public Iterator iterator() { + return iterator; + } + }; + } + + private void clearTasks() { + taskSlotMappings.clear(); + + for (TaskSlot taskSlot: taskSlots.values()) { + taskSlot.clear(); + } + } + + private void failTask(final ExecutionAttemptID executionAttemptID, final Throwable cause) { + final Task task = getTask(executionAttemptID); + + if (task != null) { + try { + task.failExternally(cause); + } catch (Throwable t) { + log.error("Could not fail task {}.", executionAttemptID, t); + } + } else { + log.debug("Cannot find task to fail for execution {}.", executionAttemptID); + } + } + + private void cancelAndClearAllTasks(Throwable cause) { + log.info("Cancellaing all computations and discarding all cached data."); + + Iterable tasks = getAllTasks(); + + for (Task task: tasks) { + task.failExternally(cause); + } + + clearTasks(); + } + + private void updateTaskExecutionState(final JobMasterGateway jobMasterGateway, final TaskExecutionState taskExecutionState) { + final ExecutionAttemptID executionAttemptID = taskExecutionState.getID(); + + Future futureAcknowledge = jobMasterGateway.updateTaskExecutionState(taskExecutionState); + + futureAcknowledge.exceptionallyAsync(new ApplyFunction() { + @Override + public Void apply(Throwable value) { + failTask(executionAttemptID, value); + + return null; + } + }, getMainThreadExecutor()); + } + + private void unregisterTaskAndNotifyFinalState(final JobMasterGateway jobMasterGateway, ExecutionAttemptID executionAttemptID) { + Task task = removeTask(executionAttemptID); + + if (task != null) { + if (!task.getExecutionState().isTerminal()) { + try { + task.failExternally(new IllegalStateException("Task is being remove from TaskManager.")); + } catch (Exception e) { + log.error("Could not properly fail task.", e); + } + } + + log.info("Un-registering task and sending final execution state {} to JobManager for task {} {}.", + task.getExecutionState(), task.getTaskInfo().getTaskName(), task.getExecutionId()); + + AccumulatorSnapshot accumulatorSnapshot = task.getAccumulatorRegistry().getSnapshot(); + + updateTaskExecutionState( + jobMasterGateway, + new TaskExecutionState( + task.getJobID(), + task.getExecutionId(), + task.getExecutionState(), + task.getFailureCause(), + accumulatorSnapshot)); + } else { + log.error("Cannot find task with ID {} to unregister.", executionAttemptID); + } + } + + private void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLeaderId) { if (resourceManagerConnection != null) { if (newLeaderAddress != null) { // the resource manager switched to a new leader @@ -178,28 +670,46 @@ public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLe } } - /** - * Requests a slot from the TaskManager - * - * @param slotID Slot id for the request - * @param allocationID id for the request - * @param resourceManagerLeaderID current leader id of the ResourceManager - * @return answer to the slot request - */ - @RpcMethod - public TMSlotRequestReply requestSlot(SlotID slotID, AllocationID allocationID, UUID resourceManagerLeaderID) { - if (!resourceManagerConnection.getTargetLeaderId().equals(resourceManagerLeaderID)) { - return new TMSlotRequestRejected( - resourceManagerConnection.getRegistrationId(), getResourceID(), allocationID); - } - if (unconfirmedFreeSlots.contains(slotID)) { - // check if request has not been blacklisted because the notification of a free slot - // has not been confirmed by the ResourceManager - return new TMSlotRequestRejected( - resourceManagerConnection.getRegistrationId(), getResourceID(), allocationID); - } - return new TMSlotRequestRegistered(new InstanceID(), ResourceID.generate(), allocationID); + private JobManagerConnection associateWithJobManager(JobMasterGateway jobMasterGateway, int blobPort) { + Preconditions.checkNotNull(jobMasterGateway); + Preconditions.checkArgument(blobPort > 0 || blobPort <= 65535, "Blob port is out of range."); + TaskManagerActions taskManagerActions = new TaskManagerActionsImpl(jobMasterGateway); + + CheckpointResponder checkpointResponder = new RpcCheckpointResponder(jobMasterGateway); + + InetSocketAddress address = new InetSocketAddress(jobMasterGateway.getAddress(), blobPort); + + BlobCache blobCache = new BlobCache(address, taskManagerConfiguration.getConfiguration()); + + LibraryCacheManager libraryCacheManager = new BlobLibraryCacheManager( + blobCache, + taskManagerConfiguration.getCleanupInterval()); + + ResultPartitionConsumableNotifier resultPartitionConsumableNotifier = new RpcResultPartitionConsumableNotifier( + jobMasterGateway, + getRpcService().getExecutor(), + taskManagerConfiguration.getTimeout()); + + PartitionStateChecker partitionStateChecker = new RpcPartitionStateChecker(jobMasterGateway); + + return new JobManagerConnection( + jobMasterGateway, + taskManagerActions, + checkpointResponder, + libraryCacheManager, + resultPartitionConsumableNotifier, + partitionStateChecker); + } + + private void disassociateFromJobManager(JobManagerConnection jobManagerConnection) throws IOException { + if (jobManagerConnection != null) { + JobMasterGateway jobManagerGateway = jobManagerConnection.getJobManagerGateway(); + + jobManagerGateway.disconnectTaskManager(getResourceID()); + + jobManagerConnection.getLibraryCacheManager().shutdown(); + } } // ------------------------------------------------------------------------ @@ -237,8 +747,7 @@ public void run() { * @param t The exception describing the fatal error */ void onFatalError(Throwable t) { - // to be determined, probably delegate to a fatal error handler that - // would either log (mini cluster) ot kill the process (yarn, mesos, ...) + log.error("Fatal error occurred.", t); fatalErrorHandler.onFatalError(t); } @@ -266,8 +775,13 @@ public void addUnconfirmedFreeSlotNotification(SlotID slotID) { private class ResourceManagerLeaderListener implements LeaderRetrievalListener { @Override - public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) { - getSelf().notifyOfNewResourceManagerLeader(leaderAddress, leaderSessionID); + public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) { + runAsync(new Runnable() { + @Override + public void run() { + notifyOfNewResourceManagerLeader(leaderAddress, leaderSessionID); + } + }); } @Override @@ -276,4 +790,43 @@ public void handleError(Exception exception) { } } + private class TaskManagerActionsImpl implements TaskManagerActions { + private final JobMasterGateway jobMasterGateway; + + private TaskManagerActionsImpl(JobMasterGateway jobMasterGateway) { + this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway); + } + + @Override + public void notifyFinalState(final ExecutionAttemptID executionAttemptID) { + runAsync(new Runnable() { + @Override + public void run() { + unregisterTaskAndNotifyFinalState(jobMasterGateway, executionAttemptID); + } + }); + } + + @Override + public void notifyFatalError(String message, Throwable cause) { + log.error(message, cause); + fatalErrorHandler.onFatalError(cause); + } + + @Override + public void failTask(final ExecutionAttemptID executionAttemptID, final Throwable cause) { + runAsync(new Runnable() { + @Override + public void run() { + TaskExecutor.this.failTask(executionAttemptID, cause); + } + }); + } + + @Override + public void updateTaskExecutionState(final TaskExecutionState taskExecutionState) { + TaskExecutor.this.updateTaskExecutionState(jobMasterGateway, taskExecutionState); + } + } + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java index 2360b535d9878..f062b96c9f61f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java @@ -21,11 +21,18 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.SlotID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestReply; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.PartitionInfo; +import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; +import org.apache.flink.runtime.taskmanager.Task; +import java.util.Collection; import java.util.UUID; /** @@ -33,12 +40,6 @@ */ public interface TaskExecutorGateway extends RpcGateway { - // ------------------------------------------------------------------------ - // ResourceManager handlers - // ------------------------------------------------------------------------ - - void notifyOfNewResourceManagerLeader(String address, UUID resourceManagerLeaderId); - /** * Requests a slot from the TaskManager * @@ -52,4 +53,71 @@ Future requestSlot( AllocationID allocationID, UUID resourceManagerLeaderID, @RpcTimeout Time timeout); + + /** + * Submit a {@link Task} to the {@link TaskExecutor}. + * + * @param tdd describing the task to submit + * @param jobManagerID identifying the submitting JobManager + * @param timeout of the submit operation + * @return Future acknowledge of the successful operation + */ + Future submitTask( + TaskDeploymentDescriptor tdd, + ResourceID jobManagerID, + @RpcTimeout Time timeout); + + /** + * Update the task where the given partitions can be found. + * + * @param executionAttemptID identifying the task + * @param partitionInfos telling where the partition can be retrieved from + * @return Future acknowledge if the partitions have been successfully updated + */ + Future updatePartitions(ExecutionAttemptID executionAttemptID, Collection partitionInfos); + + /** + * Fail all intermediate result partitions of the given task. + * + * @param executionAttemptID identifying the task + */ + void failPartition(ExecutionAttemptID executionAttemptID); + + /** + * Trigger the checkpoint for the given task. The checkpoint is identified by the checkpoint ID + * and the checkpoint timestamp. + * + * @param executionAttemptID identifying the task + * @param checkpointID unique id for the checkpoint + * @param checkpointTimestamp is the timestamp when the checkpoint has been initiated + * @return Future acknowledge if the checkpoint has been successfully triggered + */ + Future triggerCheckpoint(ExecutionAttemptID executionAttemptID, long checkpointID, long checkpointTimestamp); + + /** + * Confirm a checkpoint for the given task. The checkpoint is identified by the checkpoint ID + * and the checkpoint timestamp. + * + * @param executionAttemptID identifying the task + * @param checkpointId unique id for the checkpoint + * @param checkpointTimestamp is the timestamp when the checkpoint has been initiated + * @return Future acknowledge if the checkpoint has been successfully confirmed + */ + Future confirmCheckpoint(ExecutionAttemptID executionAttemptID, long checkpointId, long checkpointTimestamp); + + /** + * Stop the given task. + * + * @param executionAttemptID identifying the task + * @return Future acknowledge if the task is successfully stopped + */ + Future stopTask(ExecutionAttemptID executionAttemptID); + + /** + * Cancel the given task. + * + * @param executionAttemptID identifying the task + * @return Future acknowledge if the task is successfully canceled + */ + Future cancelTask(ExecutionAttemptID executionAttemptID); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java index f58af77980ce5..bce3dc37eaa4c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; @@ -71,7 +72,7 @@ public TaskManagerConfiguration( this.maxRegistrationPause = Preconditions.checkNotNull(maxRegistrationPause); this.refusedRegistrationPause = Preconditions.checkNotNull(refusedRegistrationPause); this.cleanupInterval = Preconditions.checkNotNull(cleanupInterval); - this.configuration = Preconditions.checkNotNull(configuration); + this.configuration = new UnmodifiableConfiguration(Preconditions.checkNotNull(configuration)); } public int getNumberSlots() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java index 8ac0ddd5b4e06..bb66655039036 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java @@ -95,6 +95,9 @@ public TaskManagerRunner( taskManagerServices.getNetworkEnvironment(), highAvailabilityServices, taskManagerServices.getMetricRegistry(), + taskManagerServices.getTaskManagerMetricGroup(), + taskManagerServices.getBroadcastVariableManager(), + taskManagerServices.getFileCache(), this); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java index ff7f7d5a639b7..e264a1c50fbcd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java @@ -19,7 +19,9 @@ package org.apache.flink.runtime.taskexecutor; import org.apache.flink.core.memory.MemoryType; +import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.filecache.FileCache; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.ConnectionManager; @@ -32,9 +34,11 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats; import org.apache.flink.runtime.query.netty.KvStateServer; +import org.apache.flink.runtime.taskexecutor.utils.TaskExecutorMetricsInitializer; import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.util.EnvironmentInformation; @@ -58,19 +62,28 @@ public class TaskManagerServices { private final IOManager ioManager; private final NetworkEnvironment networkEnvironment; private final MetricRegistry metricRegistry; + private final TaskManagerMetricGroup taskManagerMetricGroup; + private final BroadcastVariableManager broadcastVariableManager; + private final FileCache fileCache; private TaskManagerServices( TaskManagerLocation taskManagerLocation, MemoryManager memoryManager, IOManager ioManager, NetworkEnvironment networkEnvironment, - MetricRegistry metricRegistry) { + MetricRegistry metricRegistry, + TaskManagerMetricGroup taskManagerMetricGroup, + BroadcastVariableManager broadcastVariableManager, + FileCache fileCache) { this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation); this.memoryManager = Preconditions.checkNotNull(memoryManager); this.ioManager = Preconditions.checkNotNull(ioManager); this.networkEnvironment = Preconditions.checkNotNull(networkEnvironment); this.metricRegistry = Preconditions.checkNotNull(metricRegistry); + this.taskManagerMetricGroup = Preconditions.checkNotNull(taskManagerMetricGroup); + this.broadcastVariableManager = Preconditions.checkNotNull(broadcastVariableManager); + this.fileCache = Preconditions.checkNotNull(fileCache); } // -------------------------------------------------------------------------------------------- @@ -97,6 +110,18 @@ public MetricRegistry getMetricRegistry() { return metricRegistry; } + public TaskManagerMetricGroup getTaskManagerMetricGroup() { + return taskManagerMetricGroup; + } + + public BroadcastVariableManager getBroadcastVariableManager() { + return broadcastVariableManager; + } + + public FileCache getFileCache() { + return fileCache; + } + // -------------------------------------------------------------------------------------------- // Static factory methods for task manager services // -------------------------------------------------------------------------------------------- @@ -128,9 +153,29 @@ public static TaskManagerServices fromConfiguration( // start the I/O manager, it will create some temp directories. final IOManager ioManager = new IOManagerAsync(taskManagerServicesConfiguration.getTmpDirPaths()); - MetricRegistry metricsRegistry = new MetricRegistry(taskManagerServicesConfiguration.getMetricRegistryConfiguration()); + final MetricRegistry metricRegistry = new MetricRegistry(taskManagerServicesConfiguration.getMetricRegistryConfiguration()); + + final TaskManagerMetricGroup taskManagerMetricGroup = new TaskManagerMetricGroup( + metricRegistry, + taskManagerLocation.getHostname(), + taskManagerLocation.getResourceID().toString()); + + // Initialize the TM metrics + TaskExecutorMetricsInitializer.instantiateStatusMetrics(taskManagerMetricGroup, network); + + final BroadcastVariableManager broadcastVariableManager = new BroadcastVariableManager(); + + final FileCache fileCache = new FileCache(taskManagerServicesConfiguration.getTmpDirPaths()); - return new TaskManagerServices(taskManagerLocation, memoryManager, ioManager, network, metricsRegistry); + return new TaskManagerServices( + taskManagerLocation, + memoryManager, + ioManager, + network, + metricRegistry, + taskManagerMetricGroup, + broadcastVariableManager, + fileCache); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskSlot.java new file mode 100644 index 0000000000000..4fc1d660428cc --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskSlot.java @@ -0,0 +1,73 @@ +/* + * 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.taskexecutor; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.taskmanager.Task; +import org.apache.flink.util.Preconditions; + +import java.util.HashMap; +import java.util.Map; + +/** + * Container for multiple {@link Task} belonging to the same slot. + */ +public class TaskSlot { + private final AllocationID allocationID; + private final ResourceID resourceID; + private final Map tasks; + + public TaskSlot(AllocationID allocationID, ResourceID resourceID) { + this.allocationID = Preconditions.checkNotNull(allocationID); + this.resourceID = Preconditions.checkNotNull(resourceID); + tasks = new HashMap<>(4); + } + + public AllocationID getAllocationID() { + return allocationID; + } + + public ResourceID getResourceID() { + return resourceID; + } + + public boolean add(Task task) { + // sanity check + Preconditions.checkArgument(allocationID.equals(task.getAllocationID())); + + Task oldTask = tasks.put(task.getExecutionId(), task); + + if (oldTask != null) { + tasks.put(task.getExecutionId(), oldTask); + return false; + } else { + return true; + } + } + + public Task remove(Task task) { + return tasks.remove(task.getExecutionId()); + } + + public void clear() { + tasks.clear(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskSlotMapping.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskSlotMapping.java new file mode 100644 index 0000000000000..e67fd5252af3c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskSlotMapping.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.taskexecutor; + +import org.apache.flink.runtime.taskmanager.Task; +import org.apache.flink.util.Preconditions; + +/** + * Mapping between a {@link Task} and its {@link TaskSlot}. + */ +public class TaskSlotMapping { + + private final Task task; + private final TaskSlot taskSlot; + + public TaskSlotMapping(Task task, TaskSlot taskSlot) { + this.task = Preconditions.checkNotNull(task); + this.taskSlot = Preconditions.checkNotNull(taskSlot); + } + + public Task getTask() { + return task; + } + + public TaskSlot getTaskSlot() { + return taskSlot; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/CheckpointException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/CheckpointException.java new file mode 100644 index 0000000000000..80f2aa04946b5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/CheckpointException.java @@ -0,0 +1,41 @@ +/* + * 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.taskexecutor.exceptions; + +import org.apache.flink.runtime.taskexecutor.TaskExecutor; + +/** + * Exception indicating a problem with checkpointing on the {@link TaskExecutor} side. + */ +public class CheckpointException extends TaskManagerException { + + private static final long serialVersionUID = 3366394086880327955L; + + public CheckpointException(String message) { + super(message); + } + + public CheckpointException(String message, Throwable cause) { + super(message, cause); + } + + public CheckpointException(Throwable cause) { + super(cause); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/PartitionException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/PartitionException.java new file mode 100644 index 0000000000000..eecd0ae12e4f5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/PartitionException.java @@ -0,0 +1,41 @@ +/* + * 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.taskexecutor.exceptions; + +import org.apache.flink.runtime.taskexecutor.TaskExecutor; + +/** + * Exception indicating a problem with the result partitions on the {@link TaskExecutor} side. + */ +public class PartitionException extends TaskManagerException { + + private static final long serialVersionUID = 6248696963418276618L; + + public PartitionException(String message) { + super(message); + } + + public PartitionException(String message, Throwable cause) { + super(message, cause); + } + + public PartitionException(Throwable cause) { + super(cause); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskException.java new file mode 100644 index 0000000000000..a4a89c2773fad --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskException.java @@ -0,0 +1,41 @@ +/* + * 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.taskexecutor.exceptions; + +import org.apache.flink.runtime.taskexecutor.TaskExecutor; + +/** + * Exception indicating a task related problem on the {@link TaskExecutor}. + */ +public class TaskException extends TaskManagerException { + + private static final long serialVersionUID = 968001398103156856L; + + public TaskException(String message) { + super(message); + } + + public TaskException(String message, Throwable cause) { + super(message, cause); + } + + public TaskException(Throwable cause) { + super(cause); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskManagerException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskManagerException.java new file mode 100644 index 0000000000000..62d186e6b053b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskManagerException.java @@ -0,0 +1,41 @@ +/* + * 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.taskexecutor.exceptions; + +import org.apache.flink.runtime.taskexecutor.TaskExecutor; + +/** + * Base exception thrown by the {@link TaskExecutor}. + */ +public class TaskManagerException extends Exception { + + private static final long serialVersionUID = -2997745772227694731L; + + public TaskManagerException(String message) { + super(message); + } + + public TaskManagerException(String message, Throwable cause) { + super(message, cause); + } + + public TaskManagerException(Throwable cause) { + super(cause); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskSubmissionException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskSubmissionException.java new file mode 100644 index 0000000000000..23f7812843db4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskSubmissionException.java @@ -0,0 +1,41 @@ +/* + * 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.taskexecutor.exceptions; + +import org.apache.flink.runtime.taskmanager.Task; + +/** + * Exception indicating a problem with the {@link Task} submission at the {@link TaskException}. + */ +public class TaskSubmissionException extends TaskManagerException { + + private static final long serialVersionUID = 4589813591317690486L; + + public TaskSubmissionException(String message) { + super(message); + } + + public TaskSubmissionException(String message, Throwable cause) { + super(message, cause); + } + + public TaskSubmissionException(Throwable cause) { + super(cause); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java new file mode 100644 index 0000000000000..b1c98245d180b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java @@ -0,0 +1,71 @@ +/* + * 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.taskexecutor.rpc; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorGateway; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.state.CheckpointStateHandles; +import org.apache.flink.runtime.taskmanager.CheckpointResponder; +import org.apache.flink.util.Preconditions; + +public class RpcCheckpointResponder implements CheckpointResponder { + private final CheckpointCoordinatorGateway checkpointCoordinatorGateway; + + public RpcCheckpointResponder(CheckpointCoordinatorGateway checkpointCoordinatorGateway) { + this.checkpointCoordinatorGateway = Preconditions.checkNotNull(checkpointCoordinatorGateway); + } + + @Override + public void acknowledgeCheckpoint( + JobID jobID, + ExecutionAttemptID executionAttemptID, + long checkpointID, + CheckpointStateHandles checkpointStateHandles, + long synchronousDurationMillis, + long asynchronousDurationMillis, + long bytesBufferedInAlignment, + long alignmentDurationNanos) { + + checkpointCoordinatorGateway.acknowledgeCheckpoint( + jobID, + executionAttemptID, + checkpointID, + checkpointStateHandles, + synchronousDurationMillis, + asynchronousDurationMillis, + bytesBufferedInAlignment, + alignmentDurationNanos); + + } + + @Override + public void declineCheckpoint( + JobID jobID, + ExecutionAttemptID executionAttemptID, + long checkpointID, + long checkpointTimestamp) { + + checkpointCoordinatorGateway.declineCheckpoint( + jobID, + executionAttemptID, + checkpointID, + checkpointTimestamp); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java new file mode 100644 index 0000000000000..4850d632d2b53 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java @@ -0,0 +1,73 @@ +/* + * 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.taskexecutor.rpc; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; +import org.apache.flink.runtime.jobgraph.tasks.InputSplitProviderException; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.jobmaster.SerializedInputSplit; +import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.Preconditions; + +public class RpcInputSplitProvider implements InputSplitProvider { + private final JobMasterGateway jobMasterGateway; + private final JobID jobID; + private final JobVertexID jobVertexID; + private final ExecutionAttemptID executionAttemptID; + private final Time timeout; + + public RpcInputSplitProvider( + JobMasterGateway jobMasterGateway, + JobID jobID, + JobVertexID jobVertexID, + ExecutionAttemptID executionAttemptID, + Time timeout) { + this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway); + this.jobID = Preconditions.checkNotNull(jobID); + this.jobVertexID = Preconditions.checkNotNull(jobVertexID); + this.executionAttemptID = Preconditions.checkNotNull(executionAttemptID); + this.timeout = Preconditions.checkNotNull(timeout); + } + + + @Override + public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) throws InputSplitProviderException { + Preconditions.checkNotNull(userCodeClassLoader); + + Future futureInputSplit = jobMasterGateway.requestNextInputSplit(jobVertexID, executionAttemptID); + + try { + SerializedInputSplit serializedInputSplit = futureInputSplit.get(timeout.getSize(), timeout.getUnit()); + + if (serializedInputSplit.isEmpty()) { + return null; + } else { + return InstantiationUtil.deserializeObject(serializedInputSplit.getInputSplitData(), userCodeClassLoader); + } + } catch (Exception e) { + throw new InputSplitProviderException("Requesting the next input split failed.", e); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcKvStateRegistryListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcKvStateRegistryListener.java new file mode 100644 index 0000000000000..3692a71c7dd74 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcKvStateRegistryListener.java @@ -0,0 +1,73 @@ +/* + * 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.taskexecutor.rpc; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.query.KvStateID; +import org.apache.flink.runtime.query.KvStateRegistryGateway; +import org.apache.flink.runtime.query.KvStateRegistryListener; +import org.apache.flink.runtime.query.KvStateServerAddress; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.util.Preconditions; + +public class RpcKvStateRegistryListener implements KvStateRegistryListener { + + private final KvStateRegistryGateway kvStateRegistryGateway; + private final KvStateServerAddress kvStateServerAddress; + + public RpcKvStateRegistryListener( + KvStateRegistryGateway kvStateRegistryGateway, + KvStateServerAddress kvStateServerAddress) { + this.kvStateRegistryGateway = Preconditions.checkNotNull(kvStateRegistryGateway); + this.kvStateServerAddress = Preconditions.checkNotNull(kvStateServerAddress); + } + + @Override + public void notifyKvStateRegistered( + JobID jobId, + JobVertexID jobVertexId, + KeyGroupRange keyGroupRange, + String registrationName, + KvStateID kvStateId) { + kvStateRegistryGateway.notifyKvStateRegistered( + jobId, + jobVertexId, + keyGroupRange, + registrationName, + kvStateId, + kvStateServerAddress); + + } + + @Override + public void notifyKvStateUnregistered( + JobID jobId, + JobVertexID jobVertexId, + KeyGroupRange keyGroupRange, + String registrationName) { + + kvStateRegistryGateway.notifyKvStateUnregistered( + jobId, + jobVertexId, + keyGroupRange, + registrationName); + + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java new file mode 100644 index 0000000000000..ab111ad446256 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java @@ -0,0 +1,48 @@ +/* + * 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.taskexecutor.rpc; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.io.network.PartitionState; +import org.apache.flink.runtime.io.network.netty.PartitionStateChecker; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.util.Preconditions; + +public class RpcPartitionStateChecker implements PartitionStateChecker { + + private final JobMasterGateway jobMasterGateway; + + public RpcPartitionStateChecker(JobMasterGateway jobMasterGateway) { + this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway); + } + + @Override + public Future requestPartitionState( + JobID jobId, + ExecutionAttemptID executionId, + IntermediateDataSetID resultId, + ResultPartitionID partitionId) { + + return jobMasterGateway.requestPartitionState(partitionId, executionId, resultId); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java new file mode 100644 index 0000000000000..29ad3b654e49e --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java @@ -0,0 +1,67 @@ +/* + * 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.taskexecutor.rpc; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.ApplyFunction; +import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.taskmanager.TaskActions; +import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.Executor; + +public class RpcResultPartitionConsumableNotifier implements ResultPartitionConsumableNotifier { + + private static final Logger LOG = LoggerFactory.getLogger(RpcResultPartitionConsumableNotifier.class); + + private final JobMasterGateway jobMasterGateway; + private final Executor executor; + private final Time timeout; + + public RpcResultPartitionConsumableNotifier( + JobMasterGateway jobMasterGateway, + Executor executor, + Time timeout) { + this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway); + this.executor = Preconditions.checkNotNull(executor); + this.timeout = Preconditions.checkNotNull(timeout); + } + @Override + public void notifyPartitionConsumable(JobID jobId, ResultPartitionID partitionId, final TaskActions taskActions) { + Future acknowledgeFuture = jobMasterGateway.scheduleOrUpdateConsumers(partitionId, timeout); + + acknowledgeFuture.exceptionallyAsync(new ApplyFunction() { + @Override + public Void apply(Throwable value) { + LOG.error("Could not schedule or update consumers at the JobManager.", value); + + taskActions.failExternally(new RuntimeException("Could not notify JobManager to schedule or update consumers.", value)); + + return null; + } + }, executor); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/utils/TaskExecutorMetricsInitializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/utils/TaskExecutorMetricsInitializer.java new file mode 100644 index 0000000000000..1f8d5ed62d87f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/utils/TaskExecutorMetricsInitializer.java @@ -0,0 +1,257 @@ +/* + * 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.taskexecutor.utils; + +import com.sun.management.OperatingSystemMXBean; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.taskexecutor.TaskExecutor; +import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.management.AttributeNotFoundException; +import javax.management.InstanceNotFoundException; +import javax.management.MBeanException; +import javax.management.MBeanServer; +import javax.management.MalformedObjectNameException; +import javax.management.ObjectName; +import javax.management.ReflectionException; +import java.lang.management.ClassLoadingMXBean; +import java.lang.management.GarbageCollectorMXBean; +import java.lang.management.ManagementFactory; +import java.lang.management.MemoryMXBean; +import java.lang.management.ThreadMXBean; +import java.util.List; + +/** + * Utility class ot initialize {@link TaskExecutor} specific metrics. + */ +public class TaskExecutorMetricsInitializer { + private static final Logger LOG = LoggerFactory.getLogger(TaskExecutorMetricsInitializer.class); + + public static void instantiateStatusMetrics( + MetricGroup taskManagerMetricGroup, + NetworkEnvironment network) { + MetricGroup status = taskManagerMetricGroup.addGroup("Status"); + + instantiateNetworkMetrics(status.addGroup("Network"), network); + + MetricGroup jvm = status.addGroup("JVM"); + + instantiateClassLoaderMetrics(jvm.addGroup("ClassLoader")); + instantiateGarbageCollectorMetrics(jvm.addGroup("GarbageCollector")); + instantiateMemoryMetrics(jvm.addGroup("Memory")); + instantiateThreadMetrics(jvm.addGroup("Threads")); + instantiateCPUMetrics(jvm.addGroup("CPU")); + } + + private static void instantiateNetworkMetrics( + MetricGroup metrics, + final NetworkEnvironment network) { + metrics.>gauge("TotalMemorySegments", new Gauge () { + @Override + public Long getValue() { + return (long) network.getNetworkBufferPool().getTotalNumberOfMemorySegments(); + } + }); + + metrics.>gauge("AvailableMemorySegments", new Gauge () { + @Override + public Long getValue() { + return (long) network.getNetworkBufferPool().getNumberOfAvailableMemorySegments(); + } + }); + } + + private static void instantiateClassLoaderMetrics(MetricGroup metrics) { + final ClassLoadingMXBean mxBean = ManagementFactory.getClassLoadingMXBean(); + + metrics.>gauge("ClassesLoaded", new Gauge () { + @Override + public Long getValue() { + return mxBean.getTotalLoadedClassCount(); + } + }); + + metrics.>gauge("ClassesUnloaded", new Gauge () { + @Override + public Long getValue() { + return mxBean.getUnloadedClassCount(); + } + }); + } + + private static void instantiateGarbageCollectorMetrics(MetricGroup metrics) { + List garbageCollectors = ManagementFactory.getGarbageCollectorMXBeans(); + + for (final GarbageCollectorMXBean garbageCollector: garbageCollectors) { + MetricGroup gcGroup = metrics.addGroup(garbageCollector.getName()); + + gcGroup.>gauge("Count", new Gauge () { + @Override + public Long getValue() { + return garbageCollector.getCollectionCount(); + } + }); + + gcGroup.>gauge("Time", new Gauge () { + @Override + public Long getValue() { + return garbageCollector.getCollectionTime(); + } + }); + } + } + + private static void instantiateMemoryMetrics(MetricGroup metrics) { + final MemoryMXBean mxBean = ManagementFactory.getMemoryMXBean(); + + MetricGroup heap = metrics.addGroup("Heap"); + + heap.>gauge("Used", new Gauge () { + @Override + public Long getValue() { + return mxBean.getHeapMemoryUsage().getUsed(); + } + }); + heap.>gauge("Committed", new Gauge () { + @Override + public Long getValue() { + return mxBean.getHeapMemoryUsage().getCommitted(); + } + }); + heap.>gauge("Max", new Gauge () { + @Override + public Long getValue() { + return mxBean.getHeapMemoryUsage().getMax(); + } + }); + + MetricGroup nonHeap = metrics.addGroup("NonHeap"); + + nonHeap.>gauge("Used", new Gauge () { + @Override + public Long getValue() { + return mxBean.getNonHeapMemoryUsage().getUsed(); + } + }); + nonHeap.>gauge("Committed", new Gauge () { + @Override + public Long getValue() { + return mxBean.getNonHeapMemoryUsage().getCommitted(); + } + }); + nonHeap.>gauge("Max", new Gauge () { + @Override + public Long getValue() { + return mxBean.getNonHeapMemoryUsage().getMax(); + } + }); + + final MBeanServer con = ManagementFactory.getPlatformMBeanServer(); + + final String directBufferPoolName = "java.nio:type=BufferPool,name=direct"; + + try { + final ObjectName directObjectName = new ObjectName(directBufferPoolName); + + MetricGroup direct = metrics.addGroup("Direct"); + + direct.>gauge("Count", new TaskExecutorMetricsInitializer.AttributeGauge<>(con, directObjectName, "Count", -1L)); + direct.>gauge("MemoryUsed", new TaskExecutorMetricsInitializer.AttributeGauge<>(con, directObjectName, "MemoryUsed", -1L)); + direct.>gauge("TotalCapacity", new TaskExecutorMetricsInitializer.AttributeGauge<>(con, directObjectName, "TotalCapacity", -1L)); + } catch (MalformedObjectNameException e) { + LOG.warn("Could not create object name {}.", directBufferPoolName, e); + } + + final String mappedBufferPoolName = "java.nio:type=BufferPool,name=mapped"; + + try { + final ObjectName mappedObjectName = new ObjectName(mappedBufferPoolName); + + MetricGroup mapped = metrics.addGroup("Mapped"); + + mapped.>gauge("Count", new TaskExecutorMetricsInitializer.AttributeGauge<>(con, mappedObjectName, "Count", -1L)); + mapped.>gauge("MemoryUsed", new TaskExecutorMetricsInitializer.AttributeGauge<>(con, mappedObjectName, "MemoryUsed", -1L)); + mapped.>gauge("TotalCapacity", new TaskExecutorMetricsInitializer.AttributeGauge<>(con, mappedObjectName, "TotalCapacity", -1L)); + } catch (MalformedObjectNameException e) { + LOG.warn("Could not create object name {}.", mappedBufferPoolName, e); + } + } + + private static void instantiateThreadMetrics(MetricGroup metrics) { + final ThreadMXBean mxBean = ManagementFactory.getThreadMXBean(); + + metrics.>gauge("Count", new Gauge () { + @Override + public Integer getValue() { + return mxBean.getThreadCount(); + } + }); + } + + private static void instantiateCPUMetrics(MetricGroup metrics) { + try { + final OperatingSystemMXBean mxBean = (OperatingSystemMXBean) ManagementFactory.getOperatingSystemMXBean(); + + metrics.>gauge("Load", new Gauge () { + @Override + public Double getValue() { + return mxBean.getProcessCpuLoad(); + } + }); + metrics.>gauge("Time", new Gauge () { + @Override + public Long getValue() { + return mxBean.getProcessCpuTime(); + } + }); + } catch (Exception e) { + LOG.warn("Cannot access com.sun.management.OperatingSystemMXBean.getProcessCpuLoad()" + + " - CPU load metrics will not be available.", e); + } + } + + private static final class AttributeGauge implements Gauge { + private final MBeanServer server; + private final ObjectName objectName; + private final String attributeName; + private final T errorValue; + + private AttributeGauge(MBeanServer server, ObjectName objectName, String attributeName, T errorValue) { + this.server = Preconditions.checkNotNull(server); + this.objectName = Preconditions.checkNotNull(objectName); + this.attributeName = Preconditions.checkNotNull(attributeName); + this.errorValue = errorValue; + } + + @SuppressWarnings("unchecked") + @Override + public T getValue() { + try { + return (T) server.getAttribute(objectName, attributeName); + } catch (MBeanException | AttributeNotFoundException | InstanceNotFoundException | ReflectionException e) { + LOG.warn("Could not read attribute {}.", attributeName, e); + return errorValue; + } + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerActions.java similarity index 93% rename from flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerConnection.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerActions.java index cddac55da7ec2..b3a0cbbac7a7e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerActions.java @@ -27,11 +27,11 @@ /** * Implementation using {@link ActorGateway} to forward the messages. */ -public class ActorGatewayTaskManagerConnection implements TaskManagerConnection { +public class ActorGatewayTaskManagerActions implements TaskManagerActions { private final ActorGateway actorGateway; - public ActorGatewayTaskManagerConnection(ActorGateway actorGateway) { + public ActorGatewayTaskManagerActions(ActorGateway actorGateway) { this.actorGateway = Preconditions.checkNotNull(actorGateway); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 8463fa06c26a3..ca4884b516fb7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -25,6 +25,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.concurrent.BiFunction; import org.apache.flink.runtime.io.network.PartitionState; import org.apache.flink.runtime.io.network.netty.PartitionStateChecker; @@ -130,6 +131,9 @@ public class Task implements Runnable, TaskActions { /** The execution attempt of the parallel subtask */ private final ExecutionAttemptID executionId; + /** ID which identifies the slot in which the task is supposed to run */ + private final AllocationID allocationID; + /** TaskInfo object for this task */ private final TaskInfo taskInfo; @@ -175,7 +179,7 @@ public class Task implements Runnable, TaskActions { private final Map inputGatesById; /** Connection to the task manager */ - private final TaskManagerConnection taskManagerConnection; + private final TaskManagerActions taskManagerActions; /** Input split provider for the task */ private final InputSplitProvider inputSplitProvider; @@ -258,7 +262,7 @@ public Task( IOManager ioManager, NetworkEnvironment networkEnvironment, BroadcastVariableManager bcVarManager, - TaskManagerConnection taskManagerConnection, + TaskManagerActions taskManagerActions, InputSplitProvider inputSplitProvider, CheckpointResponder checkpointResponder, LibraryCacheManager libraryCache, @@ -273,6 +277,7 @@ public Task( this.jobId = checkNotNull(tdd.getJobID()); this.vertexId = checkNotNull(tdd.getVertexID()); this.executionId = checkNotNull(tdd.getExecutionId()); + this.allocationID = checkNotNull(tdd.getAllocationID()); this.taskNameWithSubtask = taskInfo.getTaskNameWithSubtasks(); this.jobConfiguration = checkNotNull(tdd.getJobConfiguration()); this.taskConfiguration = checkNotNull(tdd.getTaskConfiguration()); @@ -295,7 +300,7 @@ public Task( this.inputSplitProvider = checkNotNull(inputSplitProvider); this.checkpointResponder = checkNotNull(checkpointResponder); - this.taskManagerConnection = checkNotNull(taskManagerConnection); + this.taskManagerActions = checkNotNull(taskManagerActions); this.libraryCache = checkNotNull(libraryCache); this.fileCache = checkNotNull(fileCache); @@ -379,6 +384,10 @@ public ExecutionAttemptID getExecutionId() { return executionId; } + public AllocationID getAllocationID() { + return allocationID; + } + public TaskInfo getTaskInfo() { return taskInfo; } @@ -599,7 +608,7 @@ else if (current == ExecutionState.CANCELING) { // notify everyone that we switched to running notifyObservers(ExecutionState.RUNNING, null); - taskManagerConnection.updateTaskExecutionState(new TaskExecutionState(jobId, executionId, ExecutionState.RUNNING)); + taskManagerActions.updateTaskExecutionState(new TaskExecutionState(jobId, executionId, ExecutionState.RUNNING)); // make sure the user code classloader is accessible thread-locally executingThread.setContextClassLoader(userCodeClassLoader); @@ -792,11 +801,11 @@ private void removeCachedFiles(Map> entries, FileCache file } private void notifyFinalState() { - taskManagerConnection.notifyFinalState(executionId); + taskManagerActions.notifyFinalState(executionId); } private void notifyFatalError(String message, Throwable cause) { - taskManagerConnection.notifyFatalError(message, cause); + taskManagerActions.notifyFatalError(message, cause); } // ---------------------------------------------------------------------------------------------------------------- @@ -822,7 +831,7 @@ public void run() { ((StoppableTask)Task.this.invokable).stop(); } catch(RuntimeException e) { LOG.error("Stopping task " + taskNameWithSubtask + " failed.", e); - taskManagerConnection.failTask(executionId, e); + taskManagerActions.failTask(executionId, e); } } }; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java index 60aadf5a9a474..877cc1eeb8244 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java @@ -24,6 +24,8 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.util.SerializedThrowable; +import java.io.Serializable; + /** * This class represents an update about a task's execution state. * @@ -34,7 +36,7 @@ * exception field transient and deserialized it lazily, with the * appropriate class loader. */ -public class TaskExecutionState implements java.io.Serializable { +public class TaskExecutionState implements Serializable { private static final long serialVersionUID = 1L; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java index 60beae097fe01..31c518a4bf031 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; +import org.apache.flink.runtime.jobgraph.tasks.InputSplitProviderException; import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.util.InstantiationUtil; @@ -63,35 +64,45 @@ public TaskInputSplitProvider( } @Override - public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) { + public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) throws InputSplitProviderException { Preconditions.checkNotNull(userCodeClassLoader); + final Future response = jobManager.ask( + new JobManagerMessages.RequestNextInputSplit(jobID, vertexID, executionID), + timeout); + + final Object result; + try { - final Future response = jobManager.ask( - new JobManagerMessages.RequestNextInputSplit(jobID, vertexID, executionID), - timeout); + result = Await.result(response, timeout); + } catch (Exception e) { + throw new InputSplitProviderException("Did not receive next input split from JobManager.", e); + } - final Object result = Await.result(response, timeout); + if(result instanceof JobManagerMessages.NextInputSplit){ + final JobManagerMessages.NextInputSplit nextInputSplit = + (JobManagerMessages.NextInputSplit) result; - if(result instanceof JobManagerMessages.NextInputSplit){ - final JobManagerMessages.NextInputSplit nextInputSplit = - (JobManagerMessages.NextInputSplit) result; + byte[] serializedData = nextInputSplit.splitData(); - byte[] serializedData = nextInputSplit.splitData(); + if(serializedData == null) { + return null; + } else { + final Object deserialized; - if(serializedData == null) { - return null; - } else { - Object deserialized = InstantiationUtil.deserializeObject(serializedData, + try { + deserialized = InstantiationUtil.deserializeObject(serializedData, userCodeClassLoader); - return (InputSplit) deserialized; + } catch (Exception e) { + throw new InputSplitProviderException("Could not deserialize the serialized input split.", e); } - } else { - throw new Exception("RequestNextInputSplit requires a response of type " + - "NextInputSplit. Instead response is of type " + result.getClass() + '.'); + + return (InputSplit) deserialized; } - } catch (Exception e) { - throw new RuntimeException("Requesting the next InputSplit failed.", e); + } else { + throw new InputSplitProviderException("RequestNextInputSplit requires a response of type " + + "NextInputSplit. Instead response is of type " + result.getClass() + '.'); } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerActions.java similarity index 97% rename from flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerConnection.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerActions.java index dc1b40f62d760..2f3a0cb558dae 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerActions.java @@ -23,7 +23,7 @@ /** * Interface for the communication of the {@link Task} with the {@link TaskManager}. */ -public interface TaskManagerConnection { +public interface TaskManagerActions { /** * Notifies the task manager that the given task is in a final state. diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 7a764ca79d1b9..da8c14e11e31c 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -25,7 +25,6 @@ import java.net.{InetAddress, InetSocketAddress} import java.util import java.util.UUID import java.util.concurrent.TimeUnit -import javax.management.ObjectName import _root_.akka.actor._ import _root_.akka.pattern.ask @@ -37,7 +36,6 @@ import com.fasterxml.jackson.databind.ObjectMapper import grizzled.slf4j.Logger import org.apache.flink.configuration._ import org.apache.flink.core.fs.FileSystem -import org.apache.flink.metrics.{MetricGroup, Gauge => FlinkGauge} import org.apache.flink.runtime.accumulators.AccumulatorSnapshot import org.apache.flink.runtime.clusterframework.messages.StopCluster import org.apache.flink.runtime.clusterframework.types.ResourceID @@ -59,7 +57,7 @@ import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalListener, Leader import org.apache.flink.runtime.memory.MemoryManager import org.apache.flink.runtime.messages.Messages._ import org.apache.flink.runtime.messages.RegistrationMessages._ -import org.apache.flink.runtime.messages.StackTraceSampleMessages.{ResponseStackTraceSampleFailure, ResponseStackTraceSampleSuccess, SampleTaskStackTrace, StackTraceSampleMessages, TriggerStackTraceSample} +import org.apache.flink.runtime.messages.StackTraceSampleMessages._ import org.apache.flink.runtime.messages.TaskManagerMessages._ import org.apache.flink.runtime.messages.TaskMessages._ import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, NotifyCheckpointComplete, TriggerCheckpoint} @@ -68,7 +66,8 @@ import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup import org.apache.flink.runtime.process.ProcessReaper import org.apache.flink.runtime.security.SecurityContext.{FlinkSecuredRunner, SecurityConfiguration} import org.apache.flink.runtime.security.SecurityContext -import org.apache.flink.runtime.taskexecutor.{TaskManagerConfiguration, TaskManagerServices, TaskManagerServicesConfiguration} +import org.apache.flink.runtime.taskexecutor._ +import org.apache.flink.runtime.taskexecutor.utils.TaskExecutorMetricsInitializer import org.apache.flink.runtime.util._ import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessageFilter, LogMessages} import org.apache.flink.util.NetUtils @@ -150,7 +149,7 @@ class TaskManager( protected val bcVarManager = new BroadcastVariableManager() /** Handler for distributed files cached by this TaskManager */ - protected val fileCache = new FileCache(config.getConfiguration()) + protected val fileCache = new FileCache(config.getTmpDirPaths()) /** Registry of metrics periodically transmitted to the JobManager */ private val metricRegistry = TaskManager.createMetricsRegistry() @@ -196,7 +195,7 @@ class TaskManager( CheckpointResponder, PartitionStateChecker, ResultPartitionConsumableNotifier, - TaskManagerConnection)] = None + TaskManagerActions)] = None // -------------------------------------------------------------------------- // Actor messages and life cycle @@ -940,9 +939,9 @@ class TaskManager( val jobManagerGateway = new AkkaActorGateway(jobManager, leaderSessionID.orNull) val taskManagerGateway = new AkkaActorGateway(self, leaderSessionID.orNull) - val checkpointResponder = new ActorGatewayCheckpointResponder(jobManagerGateway); + val checkpointResponder = new ActorGatewayCheckpointResponder(jobManagerGateway) - val taskManagerConnection = new ActorGatewayTaskManagerConnection(taskManagerGateway) + val taskManagerConnection = new ActorGatewayTaskManagerActions(taskManagerGateway) val partitionStateChecker = new ActorGatewayPartitionStateChecker( jobManagerGateway, @@ -998,7 +997,7 @@ class TaskManager( taskManagerMetricGroup = new TaskManagerMetricGroup(metricsRegistry, this.runtimeInfo.getHostname, id.toString) - TaskManager.instantiateStatusMetrics(taskManagerMetricGroup, network) + TaskExecutorMetricsInitializer.instantiateStatusMetrics(taskManagerMetricGroup, network) // watch job manager to detect when it dies context.watch(jobManager) @@ -2008,23 +2007,23 @@ object TaskManager { // Pre-processing steps for registering cpuLoad val osBean: OperatingSystemMXBean = ManagementFactory.getOperatingSystemMXBean() - - val fetchCPULoadMethod: Option[Method] = + + val fetchCPULoadMethod: Option[Method] = try { Class.forName("com.sun.management.OperatingSystemMXBean") .getMethods() - .find( _.getName() == "getProcessCpuLoad" ) + .find(_.getName() == "getProcessCpuLoad") } catch { case t: Throwable => LOG.warn("Cannot access com.sun.management.OperatingSystemMXBean.getProcessCpuLoad()" + - " - CPU load metrics will not be available.") + " - CPU load metrics will not be available.") None } metricRegistry.register("cpuLoad", new Gauge[Double] { override def getValue: Double = { - try{ + try { fetchCPULoadMethod.map(_.invoke(osBean).asInstanceOf[Double]).getOrElse(-1.0) } catch { @@ -2036,146 +2035,4 @@ object TaskManager { }) metricRegistry } - - private def instantiateStatusMetrics( - taskManagerMetricGroup: MetricGroup, - network: NetworkEnvironment) - : Unit = { - val status = taskManagerMetricGroup - .addGroup("Status") - - instantiateNetworkMetrics(status.addGroup("Network"), network) - - val jvm = status - .addGroup("JVM") - - instantiateClassLoaderMetrics(jvm.addGroup("ClassLoader")) - instantiateGarbageCollectorMetrics(jvm.addGroup("GarbageCollector")) - instantiateMemoryMetrics(jvm.addGroup("Memory")) - instantiateThreadMetrics(jvm.addGroup("Threads")) - instantiateCPUMetrics(jvm.addGroup("CPU")) - } - - private def instantiateNetworkMetrics( - metrics: MetricGroup, - network: NetworkEnvironment) - : Unit = { - metrics.gauge[Long, FlinkGauge[Long]]("TotalMemorySegments", new FlinkGauge[Long] { - override def getValue: Long = network.getNetworkBufferPool.getTotalNumberOfMemorySegments - }) - metrics.gauge[Long, FlinkGauge[Long]]("AvailableMemorySegments", new FlinkGauge[Long] { - override def getValue: Long = network.getNetworkBufferPool.getNumberOfAvailableMemorySegments - }) - } - - private def instantiateClassLoaderMetrics(metrics: MetricGroup) { - val mxBean = ManagementFactory.getClassLoadingMXBean - - metrics.gauge[Long, FlinkGauge[Long]]("ClassesLoaded", new FlinkGauge[Long] { - override def getValue: Long = mxBean.getTotalLoadedClassCount - }) - metrics.gauge[Long, FlinkGauge[Long]]("ClassesUnloaded", new FlinkGauge[Long] { - override def getValue: Long = mxBean.getUnloadedClassCount - }) - } - - private def instantiateGarbageCollectorMetrics(metrics: MetricGroup) { - val garbageCollectors = ManagementFactory.getGarbageCollectorMXBeans - - for (garbageCollector <- garbageCollectors.asScala) { - val gcGroup = metrics.addGroup(garbageCollector.getName) - gcGroup.gauge[Long, FlinkGauge[Long]]("Count", new FlinkGauge[Long] { - override def getValue: Long = garbageCollector.getCollectionCount - }) - gcGroup.gauge[Long, FlinkGauge[Long]]("Time", new FlinkGauge[Long] { - override def getValue: Long = garbageCollector.getCollectionTime - }) - } - } - - private def instantiateMemoryMetrics(metrics: MetricGroup) { - val mxBean = ManagementFactory.getMemoryMXBean - val heap = metrics.addGroup("Heap") - heap.gauge[Long, FlinkGauge[Long]]("Used", new FlinkGauge[Long] { - override def getValue: Long = mxBean.getHeapMemoryUsage.getUsed - }) - heap.gauge[Long, FlinkGauge[Long]]("Committed", new FlinkGauge[Long] { - override def getValue: Long = mxBean.getHeapMemoryUsage.getCommitted - }) - heap.gauge[Long, FlinkGauge[Long]]("Max", new FlinkGauge[Long] { - override def getValue: Long = mxBean.getHeapMemoryUsage.getMax - }) - - val nonHeap = metrics.addGroup("NonHeap") - nonHeap.gauge[Long, FlinkGauge[Long]]("Used", new FlinkGauge[Long] { - override def getValue: Long = mxBean.getNonHeapMemoryUsage.getUsed - }) - nonHeap.gauge[Long, FlinkGauge[Long]]("Committed", new FlinkGauge[Long] { - override def getValue: Long = mxBean.getNonHeapMemoryUsage.getCommitted - }) - nonHeap.gauge[Long, FlinkGauge[Long]]("Max", new FlinkGauge[Long] { - override def getValue: Long = mxBean.getNonHeapMemoryUsage.getMax - }) - - val con = ManagementFactory.getPlatformMBeanServer; - - val directObjectName = new ObjectName("java.nio:type=BufferPool,name=direct") - - val direct = metrics.addGroup("Direct") - direct.gauge[Long, FlinkGauge[Long]]("Count", new FlinkGauge[Long] { - override def getValue: Long = con - .getAttribute(directObjectName, "Count").asInstanceOf[Long] - }) - direct.gauge[Long, FlinkGauge[Long]]("MemoryUsed", new FlinkGauge[Long] { - override def getValue: Long = con - .getAttribute(directObjectName, "MemoryUsed").asInstanceOf[Long] - }) - direct.gauge[Long, FlinkGauge[Long]]("TotalCapacity", new FlinkGauge[Long] { - override def getValue: Long = con - .getAttribute(directObjectName, "TotalCapacity").asInstanceOf[Long] - }) - - val mappedObjectName = new ObjectName("java.nio:type=BufferPool,name=mapped") - - val mapped = metrics.addGroup("Mapped") - mapped.gauge[Long, FlinkGauge[Long]]("Count", new FlinkGauge[Long] { - override def getValue: Long = con - .getAttribute(mappedObjectName, "Count").asInstanceOf[Long] - }) - mapped.gauge[Long, FlinkGauge[Long]]("MemoryUsed", new FlinkGauge[Long] { - override def getValue: Long = con - .getAttribute(mappedObjectName, "MemoryUsed").asInstanceOf[Long] - }) - mapped.gauge[Long, FlinkGauge[Long]]("TotalCapacity", new FlinkGauge[Long] { - override def getValue: Long = con - .getAttribute(mappedObjectName, "TotalCapacity").asInstanceOf[Long] - }) - } - - private def instantiateThreadMetrics(metrics: MetricGroup): Unit = { - val mxBean = ManagementFactory.getThreadMXBean - - metrics.gauge[Int, FlinkGauge[Int]]("Count", new FlinkGauge[Int] { - override def getValue: Int = mxBean.getThreadCount - }) - } - - private def instantiateCPUMetrics(metrics: MetricGroup): Unit = { - try { - val mxBean = ManagementFactory.getOperatingSystemMXBean - .asInstanceOf[com.sun.management.OperatingSystemMXBean] - - metrics.gauge[Double, FlinkGauge[Double]]("Load", new FlinkGauge[Double] { - override def getValue: Double = mxBean.getProcessCpuLoad - }) - metrics.gauge[Long, FlinkGauge[Long]]("Time", new FlinkGauge[Long] { - override def getValue: Long = mxBean.getProcessCpuTime - }) - } - catch { - case t: Throwable => - LOG.warn("Cannot access com.sun.management.OperatingSystemMXBean.getProcessCpuLoad()" + - " - CPU load metrics will not be available.") - } - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java index c369674b46bed..4db0d93b45465 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java @@ -21,7 +21,6 @@ import java.io.File; import java.util.concurrent.Future; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.api.common.cache.DistributedCache.DistributedCacheEntry; import org.apache.flink.api.common.JobID; @@ -62,8 +61,9 @@ public class FileCacheDeleteValidationTest { @Before public void setup() { + String[] tmpDirectories = System.getProperty("java.io.tmpdir").split(",|" + File.pathSeparator); try { - fileCache = new FileCache(new Configuration()); + fileCache = new FileCache(tmpDirectories); } catch (Exception e) { e.printStackTrace(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java index f8a0b6a00b12f..30dfef5c3caef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java @@ -42,7 +42,6 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 7710fa9f4e537..f5fe52ce3f96b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -20,8 +20,11 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.SlotID; +import org.apache.flink.runtime.filecache.FileCache; import org.apache.flink.runtime.highavailability.NonHaServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.io.disk.iomanager.IOManager; @@ -29,6 +32,7 @@ import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestRegistered; import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestRejected; @@ -38,7 +42,6 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.TestLogger; -import org.hamcrest.Matchers; import org.junit.Test; import org.powermock.api.mockito.PowerMockito; @@ -60,10 +63,14 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class); TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class); PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1); + PowerMockito.when(taskManagerServicesConfiguration.getConfiguration()).thenReturn(new Configuration()); + PowerMockito.when(taskManagerServicesConfiguration.getTmpDirPaths()).thenReturn(new String[1]); + rpc.registerGateway(resourceManagerAddress, rmGateway); TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); when(taskManagerLocation.getResourceID()).thenReturn(resourceID); + when(taskManagerLocation.getHostname()).thenReturn("foobar"); NonHaServices haServices = new NonHaServices(resourceManagerAddress); @@ -76,6 +83,9 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { mock(NetworkEnvironment.class), haServices, mock(MetricRegistry.class), + mock(TaskManagerMetricGroup.class), + mock(BroadcastVariableManager.class), + mock(FileCache.class), mock(FatalErrorHandler.class)); taskManager.start(); @@ -113,9 +123,12 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class); PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1); + PowerMockito.when(taskManagerServicesConfiguration.getConfiguration()).thenReturn(new Configuration()); + PowerMockito.when(taskManagerServicesConfiguration.getTmpDirPaths()).thenReturn(new String[1]); TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); when(taskManagerLocation.getResourceID()).thenReturn(resourceID); + when(taskManagerLocation.getHostname()).thenReturn("foobar"); TaskExecutor taskManager = new TaskExecutor( taskManagerServicesConfiguration, @@ -126,6 +139,9 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { mock(NetworkEnvironment.class), haServices, mock(MetricRegistry.class), + mock(TaskManagerMetricGroup.class), + mock(BroadcastVariableManager.class), + mock(FileCache.class), mock(FatalErrorHandler.class)); taskManager.start(); @@ -182,9 +198,12 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() { TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class); PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1); + PowerMockito.when(taskManagerServicesConfiguration.getConfiguration()).thenReturn(new Configuration()); + PowerMockito.when(taskManagerServicesConfiguration.getTmpDirPaths()).thenReturn(new String[1]); TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); when(taskManagerLocation.getResourceID()).thenReturn(resourceID); + when(taskManagerLocation.getHostname()).thenReturn("foobar"); TaskExecutor taskManager = new TaskExecutor( taskManagerServicesConfiguration, @@ -195,6 +214,9 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() { mock(NetworkEnvironment.class), haServices, mock(MetricRegistry.class), + mock(TaskManagerMetricGroup.class), + mock(BroadcastVariableManager.class), + mock(FileCache.class), mock(FatalErrorHandler.class)); taskManager.start(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java index 7bc2c29a498dd..aea5294004bfc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java @@ -174,7 +174,7 @@ private static Task createTask() throws Exception { mock(IOManager.class), networkEnvironment, mock(BroadcastVariableManager.class), - mock(TaskManagerConnection.class), + mock(TaskManagerActions.class), mock(InputSplitProvider.class), mock(CheckpointResponder.class), libCache, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java index 642300d439506..777633d6ea37f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.instance.BaseTestingActorGateway; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.tasks.InputSplitProviderException; import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.junit.Test; @@ -36,7 +37,7 @@ public class TaskInputSplitProviderTest { @Test - public void testRequestNextInputSplitWithInvalidExecutionID() { + public void testRequestNextInputSplitWithInvalidExecutionID() throws InputSplitProviderException { final JobID jobID = new JobID(); final JobVertexID vertexID = new JobVertexID(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java index 9791ceef20fac..5d3eb3a8c038a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.io.network.netty.PartitionStateChecker; import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; @@ -68,6 +69,7 @@ public void doMocking(AbstractInvokable taskMock) throws Exception { when(tddMock.getTaskConfiguration()).thenReturn(mock(Configuration.class)); when(tddMock.getSerializedExecutionConfig()).thenReturn(mock(SerializedValue.class)); when(tddMock.getInvokableClassName()).thenReturn("className"); + when(tddMock.getAllocationID()).thenReturn(mock(AllocationID.class)); task = new Task( tddMock, @@ -75,7 +77,7 @@ public void doMocking(AbstractInvokable taskMock) throws Exception { mock(IOManager.class), mock(NetworkEnvironment.class), mock(BroadcastVariableManager.class), - mock(TaskManagerConnection.class), + mock(TaskManagerActions.class), mock(InputSplitProvider.class), mock(CheckpointResponder.class), mock(LibraryCacheManager.class), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index 9a13cdeab07fb..fe618ff01c644 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -97,7 +97,7 @@ public class TaskTest { private ActorGateway listenerGateway; private ActorGatewayTaskExecutionStateListener listener; - private ActorGatewayTaskManagerConnection taskManagerConnection; + private ActorGatewayTaskManagerActions taskManagerConnection; private BlockingQueue taskManagerMessages; private BlockingQueue jobManagerMessages; @@ -113,7 +113,7 @@ public void createQueuesAndActors() { listenerGateway = new ForwardingActorGateway(listenerMessages); listener = new ActorGatewayTaskExecutionStateListener(listenerGateway); - taskManagerConnection = new ActorGatewayTaskManagerConnection(taskManagerGateway); + taskManagerConnection = new ActorGatewayTaskManagerActions(taskManagerGateway); awaitLatch = new OneShotLatch(); triggerLatch = new OneShotLatch(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java index 343affe13f604..c067ca7785288 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java @@ -26,6 +26,7 @@ import org.apache.flink.core.io.InputSplit; import org.apache.flink.metrics.Counter; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; +import org.apache.flink.runtime.jobgraph.tasks.InputSplitProviderException; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import java.util.Iterator; @@ -146,7 +147,12 @@ public boolean hasNext() { return true; } - InputSplit split = provider.getNextInputSplit(getRuntimeContext().getUserCodeClassLoader()); + final InputSplit split; + try { + split = provider.getNextInputSplit(getRuntimeContext().getUserCodeClassLoader()); + } catch (InputSplitProviderException e) { + throw new RuntimeException("Could not retrieve next input split.", e); + } if (split != null) { this.nextSplit = split; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java index b5b6582939a1e..ffda1265cb6aa 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java @@ -47,7 +47,7 @@ import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.runtime.taskmanager.Task; -import org.apache.flink.runtime.taskmanager.TaskManagerConnection; +import org.apache.flink.runtime.taskmanager.TaskManagerActions; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.streaming.api.TimeCharacteristic; @@ -59,6 +59,7 @@ import org.junit.Test; import java.io.EOFException; +import java.io.File; import java.io.IOException; import java.io.Serializable; import java.net.URL; @@ -153,17 +154,19 @@ private static Task createTask(TaskDeploymentDescriptor tdd) throws IOException when(networkEnvironment.createKvStateTaskRegistry(any(JobID.class), any(JobVertexID.class))) .thenReturn(mock(TaskKvStateRegistry.class)); + String[] tmpDirectories = EnvironmentInformation.getTemporaryFileDirectory().split(",|" + File.pathSeparator); + return new Task( tdd, mock(MemoryManager.class), mock(IOManager.class), networkEnvironment, mock(BroadcastVariableManager.class), - mock(TaskManagerConnection.class), + mock(TaskManagerActions.class), mock(InputSplitProvider.class), mock(CheckpointResponder.class), new FallbackLibraryCacheManager(), - new FileCache(new Configuration()), + new FileCache(tmpDirectories), new TaskManagerRuntimeInfo( "localhost", new Configuration(), EnvironmentInformation.getTemporaryFileDirectory()), new UnregisteredTaskMetricsGroup(), @@ -266,4 +269,4 @@ public void restoreState(Serializable state) throws Exception { fail("should never be called"); } } -} \ No newline at end of file +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 47a40908488d3..5a8ca04646e13 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -50,7 +50,7 @@ import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskExecutionStateListener; -import org.apache.flink.runtime.taskmanager.TaskManagerConnection; +import org.apache.flink.runtime.taskmanager.TaskManagerActions; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.functions.source.SourceFunction; @@ -216,7 +216,7 @@ private Task createTask(Class invokable, StreamConf mock(IOManager.class), network, mock(BroadcastVariableManager.class), - mock(TaskManagerConnection.class), + mock(TaskManagerActions.class), mock(InputSplitProvider.class), mock(CheckpointResponder.class), libCache, From 0aeb98beca6fbc420f3d80ec3d7bf6bcaf069b70 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 5 Oct 2016 14:47:24 +0200 Subject: [PATCH 2/2] [FLINK-4746] Make TaskManagerRuntimeInfo an interface Let the TaskManagerConfiguration implement the TaskManagerRuntimeInformation to make some of the TaskManager's configuration values accessible from different components. This closes #2599. --- .../runtime/taskexecutor/TaskExecutor.java | 11 +--- .../TaskManagerConfiguration.java | 22 ++++--- .../taskmanager/TaskManagerRuntimeInfo.java | 61 ++----------------- .../runtime/taskmanager/TaskManager.scala | 11 +--- .../operators/drivers/TestTaskContext.java | 4 +- .../testutils/BinaryOperatorTestBase.java | 4 +- .../operators/testutils/DriverTestBase.java | 4 +- .../operators/testutils/MockEnvironment.java | 10 +-- .../testutils/UnaryOperatorTestBase.java | 4 +- .../taskexecutor/TaskExecutorTest.java | 8 +-- .../taskmanager/TaskAsyncCallTest.java | 3 +- .../flink/runtime/taskmanager/TaskTest.java | 3 +- .../util/TestingTaskManagerRuntimeInfo.java | 52 ++++++++++++++++ .../tasks/InterruptSensitiveRestoreTest.java | 5 +- .../runtime/tasks/StreamMockEnvironment.java | 6 +- .../runtime/tasks/StreamTaskTest.java | 4 +- 16 files changed, 95 insertions(+), 117 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingTaskManagerRuntimeInfo.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index 35b639b1ffd47..a2716e589b5d5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.taskexecutor; -import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; import org.apache.flink.runtime.blob.BlobCache; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; @@ -71,7 +70,6 @@ import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.util.Preconditions; import java.util.HashSet; @@ -127,9 +125,6 @@ public class TaskExecutor extends RpcEndpoint { private final FileCache fileCache; - // TODO: Try to get rid of it - private final TaskManagerRuntimeInfo taskManagerRuntimeInfo; - // --------- resource manager -------- private TaskExecutorToResourceManagerConnection resourceManagerConnection; @@ -177,10 +172,6 @@ public TaskExecutor( this.taskManagerMetricGroup = checkNotNull(taskManagerMetricGroup); this.broadcastVariableManager = checkNotNull(broadcastVariableManager); this.fileCache = checkNotNull(fileCache); - this.taskManagerRuntimeInfo = new TaskManagerRuntimeInfo( - taskManagerLocation.getHostname(), - new UnmodifiableConfiguration(taskManagerConfiguration.getConfiguration()), - taskManagerConfiguration.getTmpDirPaths()); this.jobManagerConnections = new HashMap<>(4); @@ -308,7 +299,7 @@ public Acknowledge submitTask(TaskDeploymentDescriptor tdd, ResourceID jobManage checkpointResponder, libraryCache, fileCache, - taskManagerRuntimeInfo, + taskManagerConfiguration, taskMetricGroup, resultPartitionConsumableNotifier, partitionStateChecker, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java index bce3dc37eaa4c..1d1e7328a244b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,13 +34,13 @@ /** * Configuration object for {@link TaskExecutor}. */ -public class TaskManagerConfiguration { +public class TaskManagerConfiguration implements TaskManagerRuntimeInfo { private static final Logger LOG = LoggerFactory.getLogger(TaskManagerConfiguration.class); private final int numberSlots; - private final String[] tmpDirPaths; + private final String[] tmpDirectories; private final Time timeout; // null indicates an infinite duration @@ -50,12 +51,11 @@ public class TaskManagerConfiguration { private final long cleanupInterval; - // TODO: remove necessity for complete configuration object - private final Configuration configuration; + private final UnmodifiableConfiguration configuration; public TaskManagerConfiguration( int numberSlots, - String[] tmpDirPaths, + String[] tmpDirectories, Time timeout, Time maxRegistrationDuration, Time initialRegistrationPause, @@ -65,7 +65,7 @@ public TaskManagerConfiguration( Configuration configuration) { this.numberSlots = numberSlots; - this.tmpDirPaths = Preconditions.checkNotNull(tmpDirPaths); + this.tmpDirectories = Preconditions.checkNotNull(tmpDirectories); this.timeout = Preconditions.checkNotNull(timeout); this.maxRegistrationDuration = maxRegistrationDuration; this.initialRegistrationPause = Preconditions.checkNotNull(initialRegistrationPause); @@ -79,10 +79,6 @@ public int getNumberSlots() { return numberSlots; } - public String[] getTmpDirPaths() { - return tmpDirPaths; - } - public Time getTimeout() { return timeout; } @@ -107,10 +103,16 @@ public long getCleanupInterval() { return cleanupInterval; } + @Override public Configuration getConfiguration() { return configuration; } + @Override + public String[] getTmpDirectories() { + return tmpDirectories; + } + // -------------------------------------------------------------------------------------------- // Static factory methods // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRuntimeInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRuntimeInfo.java index 9ac982eba3dd7..d1efe34e2c122 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRuntimeInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRuntimeInfo.java @@ -20,71 +20,22 @@ import org.apache.flink.configuration.Configuration; -import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.flink.util.Preconditions.checkArgument; - /** - * Encapsulation of TaskManager runtime information, like hostname and configuration. + * Interface to access {@link TaskManager} information. */ -public class TaskManagerRuntimeInfo implements java.io.Serializable { - - private static final long serialVersionUID = 5598219619760274072L; - - /** host name of the interface that the TaskManager uses to communicate */ - private final String hostname; - - /** configuration that the TaskManager was started with */ - private final Configuration configuration; - - /** list of temporary file directories */ - private final String[] tmpDirectories; - - /** - * Creates a runtime info. - * - * @param hostname The host name of the interface that the TaskManager uses to communicate. - * @param configuration The configuration that the TaskManager was started with. - * @param tmpDirectory The temporary file directory. - */ - public TaskManagerRuntimeInfo(String hostname, Configuration configuration, String tmpDirectory) { - this(hostname, configuration, new String[] { tmpDirectory }); - } - - /** - * Creates a runtime info. - * @param hostname The host name of the interface that the TaskManager uses to communicate. - * @param configuration The configuration that the TaskManager was started with. - * @param tmpDirectories The list of temporary file directories. - */ - public TaskManagerRuntimeInfo(String hostname, Configuration configuration, String[] tmpDirectories) { - checkArgument(tmpDirectories.length > 0); - this.hostname = checkNotNull(hostname); - this.configuration = checkNotNull(configuration); - this.tmpDirectories = tmpDirectories; - - } - - /** - * Gets host name of the interface that the TaskManager uses to communicate. - * @return The host name of the interface that the TaskManager uses to communicate. - */ - public String getHostname() { - return hostname; - } +public interface TaskManagerRuntimeInfo { /** * Gets the configuration that the TaskManager was started with. + * * @return The configuration that the TaskManager was started with. */ - public Configuration getConfiguration() { - return configuration; - } + Configuration getConfiguration(); /** * Gets the list of temporary file directories. + * * @return The list of temporary file directories. */ - public String[] getTmpDirectories() { - return tmpDirectories; - } + String[] getTmpDirectories(); } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index da8c14e11e31c..26e13ba95b983 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -149,7 +149,7 @@ class TaskManager( protected val bcVarManager = new BroadcastVariableManager() /** Handler for distributed files cached by this TaskManager */ - protected val fileCache = new FileCache(config.getTmpDirPaths()) + protected val fileCache = new FileCache(config.getTmpDirectories()) /** Registry of metrics periodically transmitted to the JobManager */ private val metricRegistry = TaskManager.createMetricsRegistry() @@ -183,11 +183,6 @@ class TaskManager( var leaderSessionID: Option[UUID] = None - private val runtimeInfo = new TaskManagerRuntimeInfo( - location.getHostname(), - new UnmodifiableConfiguration(config.getConfiguration()), - config.getTmpDirPaths()) - private var scheduledTaskManagerRegistration: Option[Cancellable] = None private var currentRegistrationRun: UUID = UUID.randomUUID() @@ -995,7 +990,7 @@ class TaskManager( } taskManagerMetricGroup = - new TaskManagerMetricGroup(metricsRegistry, this.runtimeInfo.getHostname, id.toString) + new TaskManagerMetricGroup(metricsRegistry, location.getHostname, id.toString) TaskExecutorMetricsInitializer.instantiateStatusMetrics(taskManagerMetricGroup, network) @@ -1179,7 +1174,7 @@ class TaskManager( checkpointResponder, libCache, fileCache, - runtimeInfo, + config, taskMetricGroup, resultPartitionConsumableNotifier, partitionStateChecker, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java index 62110a709e897..d34bb404dd2e8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java @@ -35,6 +35,7 @@ import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; +import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; @@ -74,8 +75,7 @@ public TestTaskContext() {} public TestTaskContext(long memoryInBytes) { this.memoryManager = new MemoryManager(memoryInBytes, 1, 32 * 1024, MemoryType.HEAP, true); - this.taskManageInfo = new TaskManagerRuntimeInfo( - "localhost", new Configuration(), System.getProperty("java.io.tmpdir")); + this.taskManageInfo = new TestingTaskManagerRuntimeInfo(); } // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java index 75f960e162f40..3d4c45f928511 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java @@ -38,6 +38,7 @@ import org.apache.flink.runtime.operators.sort.UnilateralSortMerger; import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; +import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; import org.apache.flink.util.TestLogger; @@ -110,8 +111,7 @@ protected BinaryOperatorTestBase(ExecutionConfig executionConfig, long memory, i this.owner = new DummyInvokable(); this.taskConfig = new TaskConfig(new Configuration()); this.executionConfig = executionConfig; - this.taskManageInfo = new TaskManagerRuntimeInfo( - "localhost", new Configuration(), System.getProperty("java.io.tmpdir")); + this.taskManageInfo = new TestingTaskManagerRuntimeInfo(); } @Parameterized.Parameters diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java index 088435ab74219..f43632cabcca6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java @@ -38,6 +38,7 @@ import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.runtime.testutils.recordutils.RecordComparator; import org.apache.flink.runtime.testutils.recordutils.RecordSerializerFactory; +import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; @@ -115,8 +116,7 @@ protected DriverTestBase(ExecutionConfig executionConfig, long memory, int maxNu this.owner = new DummyInvokable(); this.taskConfig = new TaskConfig(new Configuration()); this.executionConfig = executionConfig; - this.taskManageInfo = new TaskManagerRuntimeInfo( - "localhost", new Configuration(), System.getProperty("java.io.tmpdir")); + this.taskManageInfo = new TestingTaskManagerRuntimeInfo(); } @Parameterized.Parameters diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java index c3ed6c0720a39..d2d4094067e3a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.core.fs.Path; import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; @@ -45,12 +44,10 @@ import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.query.TaskKvStateRegistry; -import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.CheckpointStateHandles; -import org.apache.flink.runtime.state.KeyGroupsStateHandle; -import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; +import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; import org.apache.flink.types.Record; import org.apache.flink.util.MutableObjectIterator; import org.mockito.invocation.InvocationOnMock; @@ -236,10 +233,7 @@ public Configuration getJobConfiguration() { @Override public TaskManagerRuntimeInfo getTaskManagerInfo() { - return new TaskManagerRuntimeInfo( - "localhost", - new UnmodifiableConfiguration(new Configuration()), - System.getProperty("java.io.tmpdir")); + return new TestingTaskManagerRuntimeInfo(); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java index a94e694dd75dc..85137cfa6abf8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java @@ -37,6 +37,7 @@ import org.apache.flink.runtime.operators.sort.UnilateralSortMerger; import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; +import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; @@ -115,8 +116,7 @@ protected UnaryOperatorTestBase(ExecutionConfig executionConfig, long memory, in this.executionConfig = executionConfig; this.comparators = new ArrayList>(2); - this.taskManageInfo = new TaskManagerRuntimeInfo( - "localhost", new Configuration(), System.getProperty("java.io.tmpdir")); + this.taskManageInfo = new TestingTaskManagerRuntimeInfo(); } @Parameterized.Parameters diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index f5fe52ce3f96b..ecbd9b5eb629e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -63,14 +63,11 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class); TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class); PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1); - PowerMockito.when(taskManagerServicesConfiguration.getConfiguration()).thenReturn(new Configuration()); - PowerMockito.when(taskManagerServicesConfiguration.getTmpDirPaths()).thenReturn(new String[1]); rpc.registerGateway(resourceManagerAddress, rmGateway); TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); when(taskManagerLocation.getResourceID()).thenReturn(resourceID); - when(taskManagerLocation.getHostname()).thenReturn("foobar"); NonHaServices haServices = new NonHaServices(resourceManagerAddress); @@ -124,7 +121,7 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class); PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1); PowerMockito.when(taskManagerServicesConfiguration.getConfiguration()).thenReturn(new Configuration()); - PowerMockito.when(taskManagerServicesConfiguration.getTmpDirPaths()).thenReturn(new String[1]); + PowerMockito.when(taskManagerServicesConfiguration.getTmpDirectories()).thenReturn(new String[1]); TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); when(taskManagerLocation.getResourceID()).thenReturn(resourceID); @@ -198,12 +195,9 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() { TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class); PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1); - PowerMockito.when(taskManagerServicesConfiguration.getConfiguration()).thenReturn(new Configuration()); - PowerMockito.when(taskManagerServicesConfiguration.getTmpDirPaths()).thenReturn(new String[1]); TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); when(taskManagerLocation.getResourceID()).thenReturn(resourceID); - when(taskManagerLocation.getHostname()).thenReturn("foobar"); TaskExecutor taskManager = new TaskExecutor( taskManagerServicesConfiguration, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java index aea5294004bfc..090880fe3feee 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java @@ -49,6 +49,7 @@ import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; import org.apache.flink.util.SerializedValue; import org.junit.Before; import org.junit.Test; @@ -179,7 +180,7 @@ private static Task createTask() throws Exception { mock(CheckpointResponder.class), libCache, mock(FileCache.class), - new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")), + new TestingTaskManagerRuntimeInfo(), mock(TaskMetricGroup.class), consumableNotifier, partitionStateChecker, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index fe618ff01c644..50fc181146453 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -48,6 +48,7 @@ import org.apache.flink.runtime.messages.TaskMessages; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.query.TaskKvStateRegistry; +import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; import org.apache.flink.util.SerializedValue; import org.junit.After; import org.junit.Before; @@ -648,7 +649,7 @@ private Task createTask( checkpointResponder, libCache, mock(FileCache.class), - new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")), + new TestingTaskManagerRuntimeInfo(), mock(TaskMetricGroup.class), consumableNotifier, partitionStateChecker, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingTaskManagerRuntimeInfo.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingTaskManagerRuntimeInfo.java new file mode 100644 index 0000000000000..e56da97dc7ccd --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingTaskManagerRuntimeInfo.java @@ -0,0 +1,52 @@ +/* + * 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.util; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; + +import java.io.File; + +/** + * TaskManagerRuntimeInfo implementation for testing purposes + */ +public class TestingTaskManagerRuntimeInfo implements TaskManagerRuntimeInfo { + + private final Configuration configuration; + private final String[] tmpDirectories; + + public TestingTaskManagerRuntimeInfo() { + this(new Configuration(), System.getProperty("java.io.tmpdir").split(",|" + File.pathSeparator)); + } + + public TestingTaskManagerRuntimeInfo(Configuration configuration, String[] tmpDirectories) { + this.configuration = configuration; + this.tmpDirectories = tmpDirectories; + } + + @Override + public Configuration getConfiguration() { + return configuration; + } + + @Override + public String[] getTmpDirectories() { + return tmpDirectories; + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java index ffda1265cb6aa..fb1b3b3b30f52 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java @@ -48,8 +48,8 @@ import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TaskManagerActions; -import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.checkpoint.Checkpointed; import org.apache.flink.streaming.api.functions.source.SourceFunction; @@ -167,8 +167,7 @@ private static Task createTask(TaskDeploymentDescriptor tdd) throws IOException mock(CheckpointResponder.class), new FallbackLibraryCacheManager(), new FileCache(tmpDirectories), - new TaskManagerRuntimeInfo( - "localhost", new Configuration(), EnvironmentInformation.getTemporaryFileDirectory()), + new TestingTaskManagerRuntimeInfo(new Configuration(), tmpDirectories), new UnregisteredTaskMetricsGroup(), mock(ResultPartitionConsumableNotifier.class), mock(PartitionStateChecker.class), diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java index 9b773d895b0db..6f9d8ddb82519 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java @@ -49,12 +49,10 @@ import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate; import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.query.TaskKvStateRegistry; -import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.CheckpointStateHandles; -import org.apache.flink.runtime.state.KeyGroupsStateHandle; -import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; +import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -336,7 +334,7 @@ public boolean wasFailedExternally() { @Override public TaskManagerRuntimeInfo getTaskManagerInfo() { - return new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")); + return new TestingTaskManagerRuntimeInfo(); } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 5a8ca04646e13..205fba05fb7f9 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -51,7 +51,7 @@ import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskExecutionStateListener; import org.apache.flink.runtime.taskmanager.TaskManagerActions; -import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; +import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.graph.StreamConfig; @@ -221,7 +221,7 @@ private Task createTask(Class invokable, StreamConf mock(CheckpointResponder.class), libCache, mock(FileCache.class), - new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")), + new TestingTaskManagerRuntimeInfo(), mock(TaskMetricGroup.class), consumableNotifier, partitionStateChecker,