From f7014b7330531180a0b66a5c2ee91bd662ef9c42 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Fri, 19 Apr 2019 15:45:38 +0200 Subject: [PATCH] [FLINK-11391] Introduce shuffle master interface Introduce PartitionLocation in NettyShuffleDescriptor and NettyShuffleDescriptorBuilder for tests Add ShuffleDescriptor.getResultPartitionID and isUnknown Use NettyShuffleDescriptorBuilder in StreamNetworkBenchmarkEnvironment Introduce ShuffleUtils.applyWithShuffleTypeCheck to isolate inout channel shuffle descriptor 'known' check and cast This closes #8362. --- .../InputChannelDeploymentDescriptor.java | 186 ------------- .../InputGateDeploymentDescriptor.java | 18 +- .../ResultPartitionDeploymentDescriptor.java | 82 ++---- .../deployment/ResultPartitionLocation.java | 102 ------- .../TaskDeploymentDescriptorFactory.java | 258 ++++++++++++++++++ .../runtime/executiongraph/Execution.java | 146 +++++++++- .../executiongraph/ExecutionGraph.java | 9 + .../executiongraph/ExecutionJobVertex.java | 2 +- .../executiongraph/ExecutionVertex.java | 119 +------- .../runtime/executiongraph/PartitionInfo.java | 36 +-- .../io/network/NetworkEnvironment.java | 18 +- .../consumer/RemoteInputChannel.java | 6 +- .../partition/consumer/SingleInputGate.java | 34 +-- .../consumer/SingleInputGateFactory.java | 208 ++++++++------ .../shuffle/NettyShuffleDescriptor.java | 114 ++++++++ .../runtime/shuffle/NettyShuffleMaster.java | 57 ++++ .../runtime/shuffle/PartitionDescriptor.java | 127 +++++++++ .../runtime/shuffle/ProducerDescriptor.java | 89 ++++++ .../runtime/shuffle/ShuffleDescriptor.java | 53 ++++ .../flink/runtime/shuffle/ShuffleMaster.java | 44 +++ .../flink/runtime/shuffle/ShuffleUtils.java | 61 +++++ .../shuffle/UnknownShuffleDescriptor.java | 51 ++++ .../runtime/taskexecutor/TaskExecutor.java | 2 +- .../taskexecutor/TaskManagerServices.java | 6 +- .../InputChannelDeploymentDescriptorTest.java | 206 -------------- ...sultPartitionDeploymentDescriptorTest.java | 111 ++++++-- .../deployment/ShuffleDescriptorTest.java | 185 +++++++++++++ .../ExecutionGraphDeploymentTest.java | 3 +- .../runtime/executiongraph/ExecutionTest.java | 10 +- .../ExecutionVertexDeploymentTest.java | 23 +- .../executiongraph/FinalizeOnMasterTest.java | 1 + .../io/network/NetworkEnvironmentBuilder.java | 9 + .../partition/ResultPartitionTest.java | 13 +- .../consumer/InputChannelBuilder.java | 2 +- .../consumer/SingleInputGateTest.java | 81 +++--- .../TaskExecutorSubmissionTest.java | 212 ++++++-------- .../TaskSubmissionTestEnvironment.java | 15 +- .../flink/runtime/taskmanager/TaskTest.java | 40 ++- .../util/NettyShuffleDescriptorBuilder.java | 103 +++++++ .../StreamNetworkBenchmarkEnvironment.java | 36 ++- 40 files changed, 1818 insertions(+), 1060 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionLocation.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleDescriptor.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleMaster.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/PartitionDescriptor.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ProducerDescriptor.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleDescriptor.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleMaster.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleUtils.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/UnknownShuffleDescriptor.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptorTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ShuffleDescriptorTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/util/NettyShuffleDescriptorBuilder.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java deleted file mode 100644 index d45ca51f92826..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java +++ /dev/null @@ -1,186 +0,0 @@ -/* - * 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.deployment; - -import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.executiongraph.Execution; -import org.apache.flink.runtime.executiongraph.ExecutionEdge; -import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; -import org.apache.flink.runtime.io.network.ConnectionID; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; -import org.apache.flink.runtime.io.network.partition.consumer.InputChannel; -import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; -import org.apache.flink.runtime.jobmaster.LogicalSlot; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; - -import javax.annotation.Nonnull; - -import java.io.Serializable; -import java.util.List; -import java.util.stream.Collectors; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Deployment descriptor for a single input channel instance. - * - *

Each input channel consumes a single subpartition. The index of the subpartition to consume - * is part of the {@link InputGateDeploymentDescriptor} as it is the same for each input channel of - * the respective input gate. - * - * @see InputChannel - * @see SingleInputGate - */ -public class InputChannelDeploymentDescriptor implements Serializable { - - private static final long serialVersionUID = 373711381640454080L; - - /** The ID of the partition the input channel is going to consume. */ - private final ResultPartitionID consumedPartitionId; - - /** The location of the partition the input channel is going to consume. */ - private final ResultPartitionLocation consumedPartitionLocation; - - public InputChannelDeploymentDescriptor( - ResultPartitionID consumedPartitionId, - ResultPartitionLocation consumedPartitionLocation) { - - this.consumedPartitionId = checkNotNull(consumedPartitionId); - this.consumedPartitionLocation = checkNotNull(consumedPartitionLocation); - } - - public ResultPartitionID getConsumedPartitionId() { - return consumedPartitionId; - } - - public ResultPartitionLocation getConsumedPartitionLocation() { - return consumedPartitionLocation; - } - - @Override - public String toString() { - return String.format("InputChannelDeploymentDescriptor [consumed partition id: %s, " + - "consumed partition location: %s]", - consumedPartitionId, consumedPartitionLocation); - } - - // ------------------------------------------------------------------------ - - /** - * Creates an input channel deployment descriptor for each partition. - */ - public static List fromEdges( - List edges, - boolean allowLazyDeployment) { - return edges.stream().map(edge -> fromEdgeAndValidate(allowLazyDeployment, edge)).collect(Collectors.toList()); - } - - @Nonnull - private static InputChannelDeploymentDescriptor fromEdgeAndValidate(boolean allowLazyDeployment, ExecutionEdge edge) { - final InputChannelDeploymentDescriptor inputChannelDeploymentDescriptor = fromEdge(edge); - - if (!allowLazyDeployment && inputChannelDeploymentDescriptor.getConsumedPartitionLocation().isUnknown()) { - final IntermediateResultPartition consumedPartition = edge.getSource(); - final Execution producer = consumedPartition.getProducer().getCurrentExecutionAttempt(); - final ExecutionState producerState = producer.getState(); - - if (producerState == ExecutionState.CANCELING - || producerState == ExecutionState.CANCELED - || producerState == ExecutionState.FAILED) { - String msg = "Trying to schedule a task whose inputs were canceled or failed. " + - "The producer is in state " + producerState + '.'; - throw new IllegalStateException(msg); - } else { - final LogicalSlot producerSlot = producer.getAssignedResource(); - - String msg = String.format("Trying to eagerly schedule a task whose inputs " + - "are not ready (result type: %s, partition consumable: %s, producer state: %s, producer slot: %s).", - consumedPartition.getResultType(), - consumedPartition.isConsumable(), - producerState, - producerSlot); - throw new IllegalStateException(msg); - } - } - - return inputChannelDeploymentDescriptor; - } - - @Nonnull - public static InputChannelDeploymentDescriptor fromEdge(ExecutionEdge edge) { - final IntermediateResultPartition consumedPartition = edge.getSource(); - final Execution producer = consumedPartition.getProducer().getCurrentExecutionAttempt(); - - final ExecutionState producerState = producer.getState(); - final LogicalSlot producerSlot = producer.getAssignedResource(); - - final ResultPartitionLocation partitionLocation; - - // The producing task needs to be RUNNING or already FINISHED - if ((consumedPartition.getResultType().isPipelined() || consumedPartition.isConsumable()) && - producerSlot != null && - (producerState == ExecutionState.RUNNING || - producerState == ExecutionState.FINISHED || - producerState == ExecutionState.SCHEDULED || - producerState == ExecutionState.DEPLOYING)) { - - final TaskManagerLocation partitionTaskManagerLocation = producerSlot.getTaskManagerLocation(); - final LogicalSlot consumerSlot = edge.getTarget().getCurrentAssignedResource(); - - if (consumerSlot != null) { - partitionLocation = createKnownResultPartitionLocation(consumerSlot.getTaskManagerLocation().getResourceID(), consumedPartition, partitionTaskManagerLocation); - } else { - throw new IllegalStateException("Cannot create an input channel descriptor for a consumer which has no slot assigned."); - } - } - else { - // The producing task might not have registered the partition yet - partitionLocation = ResultPartitionLocation.createUnknown(); - } - - final ResultPartitionID consumedPartitionId = new ResultPartitionID( - consumedPartition.getPartitionId(), producer.getAttemptId()); - - return new InputChannelDeploymentDescriptor(consumedPartitionId, partitionLocation); - } - - @Nonnull - private static ResultPartitionLocation createKnownResultPartitionLocation( - ResourceID consumerResourceId, - IntermediateResultPartition consumedPartition, - TaskManagerLocation partitionTaskManagerLocation) { - ResultPartitionLocation partitionLocation; - final ResourceID partitionTaskManager = partitionTaskManagerLocation.getResourceID(); - - if (partitionTaskManager.equals(consumerResourceId)) { - // Consuming task is deployed to the same TaskManager as the partition => local - partitionLocation = ResultPartitionLocation.createLocal(); - } - else { - // Different instances => remote - final ConnectionID connectionId = new ConnectionID( - partitionTaskManagerLocation, - consumedPartition.getIntermediateResult().getConnectionIndex()); - - partitionLocation = ResultPartitionLocation.createRemote(connectionId); - } - return partitionLocation; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java index 8b0dbad7fd4b9..345a99d3c9c17 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java @@ -22,11 +22,13 @@ import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; + +import javax.annotation.Nonnegative; import java.io.Serializable; import java.util.Arrays; -import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -54,23 +56,20 @@ public class InputGateDeploymentDescriptor implements Serializable { * The index of the consumed subpartition of each consumed partition. This index depends on the * {@link DistributionPattern} and the subtask indices of the producing and consuming task. */ + @Nonnegative private final int consumedSubpartitionIndex; /** An input channel for each consumed subpartition. */ - private final InputChannelDeploymentDescriptor[] inputChannels; + private final ShuffleDescriptor[] inputChannels; public InputGateDeploymentDescriptor( IntermediateDataSetID consumedResultId, ResultPartitionType consumedPartitionType, - int consumedSubpartitionIndex, - InputChannelDeploymentDescriptor[] inputChannels) { - + @Nonnegative int consumedSubpartitionIndex, + ShuffleDescriptor[] inputChannels) { this.consumedResultId = checkNotNull(consumedResultId); this.consumedPartitionType = checkNotNull(consumedPartitionType); - - checkArgument(consumedSubpartitionIndex >= 0); this.consumedSubpartitionIndex = consumedSubpartitionIndex; - this.inputChannels = checkNotNull(inputChannels); } @@ -87,11 +86,12 @@ public ResultPartitionType getConsumedPartitionType() { return consumedPartitionType; } + @Nonnegative public int getConsumedSubpartitionIndex() { return consumedSubpartitionIndex; } - public InputChannelDeploymentDescriptor[] getInputChannelDeploymentDescriptors() { + public ShuffleDescriptor[] getShuffleDescriptors() { return inputChannels; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java index 2a3feb960de2a..064c9bdff173f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java @@ -18,16 +18,16 @@ package org.apache.flink.runtime.deployment; -import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; import org.apache.flink.runtime.io.network.partition.ResultPartition; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.shuffle.PartitionDescriptor; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import java.io.Serializable; -import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -39,99 +39,59 @@ public class ResultPartitionDeploymentDescriptor implements Serializable { private static final long serialVersionUID = 6343547936086963705L; - /** The ID of the result this partition belongs to. */ - private final IntermediateDataSetID resultId; + private final PartitionDescriptor partitionDescriptor; - /** The ID of the partition. */ - private final IntermediateResultPartitionID partitionId; + private final ShuffleDescriptor shuffleDescriptor; - /** The type of the partition. */ - private final ResultPartitionType partitionType; - - /** The number of subpartitions. */ - private final int numberOfSubpartitions; - - /** The maximum parallelism. */ private final int maxParallelism; /** Flag whether the result partition should send scheduleOrUpdateConsumer messages. */ private final boolean sendScheduleOrUpdateConsumersMessage; public ResultPartitionDeploymentDescriptor( - IntermediateDataSetID resultId, - IntermediateResultPartitionID partitionId, - ResultPartitionType partitionType, - int numberOfSubpartitions, + PartitionDescriptor partitionDescriptor, + ShuffleDescriptor shuffleDescriptor, int maxParallelism, - boolean lazyScheduling) { - - this.resultId = checkNotNull(resultId); - this.partitionId = checkNotNull(partitionId); - this.partitionType = checkNotNull(partitionType); - + boolean sendScheduleOrUpdateConsumersMessage) { + this.partitionDescriptor = checkNotNull(partitionDescriptor); + this.shuffleDescriptor = checkNotNull(shuffleDescriptor); KeyGroupRangeAssignment.checkParallelismPreconditions(maxParallelism); - checkArgument(numberOfSubpartitions >= 1); - this.numberOfSubpartitions = numberOfSubpartitions; this.maxParallelism = maxParallelism; - this.sendScheduleOrUpdateConsumersMessage = lazyScheduling; + this.sendScheduleOrUpdateConsumersMessage = sendScheduleOrUpdateConsumersMessage; } public IntermediateDataSetID getResultId() { - return resultId; + return partitionDescriptor.getResultId(); } public IntermediateResultPartitionID getPartitionId() { - return partitionId; + return partitionDescriptor.getPartitionId(); } public ResultPartitionType getPartitionType() { - return partitionType; + return partitionDescriptor.getPartitionType(); } public int getNumberOfSubpartitions() { - return numberOfSubpartitions; + return partitionDescriptor.getNumberOfSubpartitions(); } public int getMaxParallelism() { return maxParallelism; } + public ShuffleDescriptor getShuffleDescriptor() { + return shuffleDescriptor; + } + public boolean sendScheduleOrUpdateConsumersMessage() { return sendScheduleOrUpdateConsumersMessage; } @Override public String toString() { - return String.format("ResultPartitionDeploymentDescriptor [result id: %s, " - + "partition id: %s, partition type: %s]", - resultId, partitionId, partitionType); - } - - // ------------------------------------------------------------------------ - - public static ResultPartitionDeploymentDescriptor from( - IntermediateResultPartition partition, int maxParallelism, boolean lazyScheduling) { - - final IntermediateDataSetID resultId = partition.getIntermediateResult().getId(); - final IntermediateResultPartitionID partitionId = partition.getPartitionId(); - final ResultPartitionType partitionType = partition.getIntermediateResult().getResultType(); - - // The produced data is partitioned among a number of subpartitions. - // - // If no consumers are known at this point, we use a single subpartition, otherwise we have - // one for each consuming sub task. - int numberOfSubpartitions = 1; - - if (!partition.getConsumers().isEmpty() && !partition.getConsumers().get(0).isEmpty()) { - - if (partition.getConsumers().size() > 1) { - throw new IllegalStateException("Currently, only a single consumer group per partition is supported."); - } - - numberOfSubpartitions = partition.getConsumers().get(0).size(); - } - - return new ResultPartitionDeploymentDescriptor( - resultId, partitionId, partitionType, numberOfSubpartitions, maxParallelism, lazyScheduling); + return String.format("ResultPartitionDeploymentDescriptor [PartitionDescriptor: %s, " + + "ShuffleDescriptor: %s]", + partitionDescriptor, shuffleDescriptor); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionLocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionLocation.java deleted file mode 100644 index 78b65273cf6ee..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionLocation.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * 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.deployment; - -import org.apache.flink.runtime.io.network.ConnectionID; - -import java.io.Serializable; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Location of a result partition from the perspective of the consuming task. - * - *

The location indicates both the instance, on which the partition is produced and the state of - * the producing task. There are three possibilities: - * - *

    - *
  1. Local: The partition is available at the same instance on which the - * consuming task is (being) deployed and the producing task has registered the result partition. - * - *
  2. Remote: The result partition is available at a different instance from the - * one, on which the consuming task is (being) deployed and the producing task has registered the - * result partition. - * - *
  3. Unknown: The producing task has not yet registered the result partition. - * When deploying the consuming task, the instance might be known or unknown. In any case, the - * consuming task cannot request it yet. Instead, it will be updated at runtime after the - * producing task is guaranteed to have registered the partition. A producing task is guaranteed - * to have registered the partition after its state has switched to running. - *
- */ -public class ResultPartitionLocation implements Serializable { - - private static final long serialVersionUID = -6354238166937194463L; - /** The type of location for the result partition. */ - private final LocationType locationType; - - /** The connection ID of a remote result partition. */ - private final ConnectionID connectionId; - - private enum LocationType { - LOCAL, - REMOTE, - UNKNOWN - } - - private ResultPartitionLocation(LocationType locationType, ConnectionID connectionId) { - this.locationType = checkNotNull(locationType); - this.connectionId = connectionId; - } - - public static ResultPartitionLocation createRemote(ConnectionID connectionId) { - return new ResultPartitionLocation(LocationType.REMOTE, checkNotNull(connectionId)); - } - - public static ResultPartitionLocation createLocal() { - return new ResultPartitionLocation(LocationType.LOCAL, null); - } - - public static ResultPartitionLocation createUnknown() { - return new ResultPartitionLocation(LocationType.UNKNOWN, null); - } - - // ------------------------------------------------------------------------ - - public boolean isLocal() { - return locationType == LocationType.LOCAL; - } - - public boolean isRemote() { - return locationType == LocationType.REMOTE; - } - - public boolean isUnknown() { - return locationType == LocationType.UNKNOWN; - } - - public ConnectionID getConnectionId() { - return connectionId; - } - - @Override - public String toString() { - return "ResultPartitionLocation [" + locationType + (isRemote() ? " [" + connectionId + "]]" : "]"); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorFactory.java new file mode 100644 index 0000000000000..9bef21ad9ed76 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorFactory.java @@ -0,0 +1,258 @@ +/* + * 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.deployment; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.blob.PermanentBlobKey; +import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor.MaybeOffloaded; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionEdge; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.IntermediateResult; +import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; +import org.apache.flink.runtime.executiongraph.JobInformation; +import org.apache.flink.runtime.executiongraph.TaskInformation; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; +import org.apache.flink.runtime.shuffle.UnknownShuffleDescriptor; +import org.apache.flink.types.Either; +import org.apache.flink.util.SerializedValue; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Optional; + +/** + * Factory of {@link TaskDeploymentDescriptor} to deploy {@link org.apache.flink.runtime.taskmanager.Task} from {@link Execution}. + */ +public class TaskDeploymentDescriptorFactory { + private final ExecutionAttemptID executionId; + private final int attemptNumber; + private final MaybeOffloaded serializedJobInformation; + private final MaybeOffloaded taskInfo; + private final JobID jobID; + private final boolean allowUnknownPartitions; + private final int subtaskIndex; + private final ExecutionEdge[][] inputEdges; + + private TaskDeploymentDescriptorFactory( + ExecutionAttemptID executionId, + int attemptNumber, + MaybeOffloaded serializedJobInformation, + MaybeOffloaded taskInfo, + JobID jobID, + boolean allowUnknownPartitions, + int subtaskIndex, + ExecutionEdge[][] inputEdges) { + this.executionId = executionId; + this.attemptNumber = attemptNumber; + this.serializedJobInformation = serializedJobInformation; + this.taskInfo = taskInfo; + this.jobID = jobID; + this.allowUnknownPartitions = allowUnknownPartitions; + this.subtaskIndex = subtaskIndex; + this.inputEdges = inputEdges; + } + + public TaskDeploymentDescriptor createDeploymentDescriptor( + AllocationID allocationID, + int targetSlotNumber, + @Nullable JobManagerTaskRestore taskRestore, + Collection producedPartitions) { + return new TaskDeploymentDescriptor( + jobID, + serializedJobInformation, + taskInfo, + executionId, + allocationID, + subtaskIndex, + attemptNumber, + targetSlotNumber, + taskRestore, + new ArrayList<>(producedPartitions), + createInputGateDeploymentDescriptors()); + } + + private List createInputGateDeploymentDescriptors() { + List inputGates = new ArrayList<>(inputEdges.length); + + for (ExecutionEdge[] edges : inputEdges) { + // If the produced partition has multiple consumers registered, we + // need to request the one matching our sub task index. + // TODO Refactor after removing the consumers from the intermediate result partitions + int numConsumerEdges = edges[0].getSource().getConsumers().get(0).size(); + + int queueToRequest = subtaskIndex % numConsumerEdges; + + IntermediateResult consumedIntermediateResult = edges[0].getSource().getIntermediateResult(); + IntermediateDataSetID resultId = consumedIntermediateResult.getId(); + ResultPartitionType partitionType = consumedIntermediateResult.getResultType(); + + inputGates.add(new InputGateDeploymentDescriptor( + resultId, + partitionType, + queueToRequest, + getConsumedPartitionShuffleDescriptors(edges))); + } + + return inputGates; + } + + private ShuffleDescriptor[] getConsumedPartitionShuffleDescriptors(ExecutionEdge[] edges) { + ShuffleDescriptor[] shuffleDescriptors = new ShuffleDescriptor[edges.length]; + // Each edge is connected to a different result partition + for (int i = 0; i < edges.length; i++) { + shuffleDescriptors[i] = + getConsumedPartitionShuffleDescriptor(edges[i], allowUnknownPartitions); + } + return shuffleDescriptors; + } + + public static TaskDeploymentDescriptorFactory fromExecutionVertex( + ExecutionVertex executionVertex, + int attemptNumber) throws IOException { + ExecutionGraph executionGraph = executionVertex.getExecutionGraph(); + return new TaskDeploymentDescriptorFactory( + executionVertex.getCurrentExecutionAttempt().getAttemptId(), + attemptNumber, + getSerializedJobInformation(executionGraph), + getSerializedTaskInformation(executionVertex.getJobVertex().getTaskInformationOrBlobKey()), + executionGraph.getJobID(), + executionGraph.getScheduleMode().allowLazyDeployment(), + executionVertex.getParallelSubtaskIndex(), + executionVertex.getAllInputEdges()); + } + + private static MaybeOffloaded getSerializedJobInformation(ExecutionGraph executionGraph) { + Either, PermanentBlobKey> jobInformationOrBlobKey = + executionGraph.getJobInformationOrBlobKey(); + if (jobInformationOrBlobKey.isLeft()) { + return new TaskDeploymentDescriptor.NonOffloaded<>(jobInformationOrBlobKey.left()); + } else { + return new TaskDeploymentDescriptor.Offloaded<>(jobInformationOrBlobKey.right()); + } + } + + private static MaybeOffloaded getSerializedTaskInformation( + Either, + PermanentBlobKey> taskInfo) { + return taskInfo.isLeft() ? + new TaskDeploymentDescriptor.NonOffloaded<>(taskInfo.left()) : + new TaskDeploymentDescriptor.Offloaded<>(taskInfo.right()); + } + + public static ShuffleDescriptor getConsumedPartitionShuffleDescriptor( + ExecutionEdge edge, + boolean allowUnknownPartitions) { + IntermediateResultPartition consumedPartition = edge.getSource(); + Execution producer = consumedPartition.getProducer().getCurrentExecutionAttempt(); + + ExecutionState producerState = producer.getState(); + Optional consumedPartitionDescriptor = + producer.getResultPartitionDeploymentDescriptor(consumedPartition.getPartitionId()); + + ResultPartitionID consumedPartitionId = new ResultPartitionID( + consumedPartition.getPartitionId(), + producer.getAttemptId()); + + return getConsumedPartitionShuffleDescriptor( + consumedPartitionId, + consumedPartition.getResultType(), + consumedPartition.isConsumable(), + producerState, + allowUnknownPartitions, + consumedPartitionDescriptor.orElse(null)); + } + + @VisibleForTesting + static ShuffleDescriptor getConsumedPartitionShuffleDescriptor( + ResultPartitionID consumedPartitionId, + ResultPartitionType resultPartitionType, + boolean isConsumable, + ExecutionState producerState, + boolean allowUnknownPartitions, + @Nullable ResultPartitionDeploymentDescriptor consumedPartitionDescriptor) { + // The producing task needs to be RUNNING or already FINISHED + if ((resultPartitionType.isPipelined() || isConsumable) && + consumedPartitionDescriptor != null && + isProducerAvailable(producerState)) { + // partition is already registered + return consumedPartitionDescriptor.getShuffleDescriptor(); + } + else if (allowUnknownPartitions) { + // The producing task might not have registered the partition yet + return new UnknownShuffleDescriptor(consumedPartitionId); + } + else { + // throw respective exceptions + handleConsumedPartitionShuffleDescriptorErrors( + consumedPartitionId, + resultPartitionType, + isConsumable, + producerState); + return null; // should never happen + } + } + + private static void handleConsumedPartitionShuffleDescriptorErrors( + ResultPartitionID consumedPartitionId, + ResultPartitionType resultPartitionType, + boolean isConsumable, + ExecutionState producerState) { + if (isProducerFailedOrCanceled(producerState)) { + String msg = "Trying to consume an input partition whose producer has been canceled or failed. " + + "The producer is in state " + producerState + "."; + throw new IllegalStateException(msg); + } + else { + String msg = String.format("Trying to consume an input partition whose producer " + + "is not ready (result type: %s, partition consumable: %s, producer state: %s, partition id: %s).", + resultPartitionType, + isConsumable, + producerState, + consumedPartitionId); + throw new IllegalStateException(msg); + } + } + + private static boolean isProducerAvailable(ExecutionState producerState) { + return producerState == ExecutionState.RUNNING || + producerState == ExecutionState.FINISHED || + producerState == ExecutionState.SCHEDULED || + producerState == ExecutionState.DEPLOYING; + } + + private static boolean isProducerFailedOrCanceled(ExecutionState producerState) { + return producerState == ExecutionState.CANCELING || + producerState == ExecutionState.CANCELED || + producerState == ExecutionState.FAILED; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index d744f60895089..48e2f87f1a949 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -34,10 +34,14 @@ import org.apache.flink.runtime.clusterframework.types.SlotProfile; import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptorFactory; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.instance.SlotSharingGroupId; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint; import org.apache.flink.runtime.jobmanager.scheduler.LocationPreferenceConstraint; import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; @@ -49,6 +53,10 @@ import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.StackTraceSampleResponse; +import org.apache.flink.runtime.shuffle.PartitionDescriptor; +import org.apache.flink.runtime.shuffle.ProducerDescriptor; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; @@ -64,8 +72,10 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; @@ -75,6 +85,7 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static org.apache.flink.runtime.deployment.TaskDeploymentDescriptorFactory.getConsumedPartitionShuffleDescriptor; import static org.apache.flink.runtime.execution.ExecutionState.CANCELED; import static org.apache.flink.runtime.execution.ExecutionState.CANCELING; import static org.apache.flink.runtime.execution.ExecutionState.CREATED; @@ -177,6 +188,8 @@ public class Execution implements AccessExecution, Archiveable producedPartitions; + // -------------------------------------------------------------------------------------------- /** @@ -215,6 +228,7 @@ public Execution( markTimestamp(CREATED, startTimestamp); this.partitionInfos = new ArrayList<>(16); + this.producedPartitions = Collections.emptyMap(); this.terminalStateFuture = new CompletableFuture<>(); this.releaseFuture = new CompletableFuture<>(); this.taskManagerLocationFuture = new CompletableFuture<>(); @@ -268,6 +282,11 @@ public LogicalSlot getAssignedResource() { return assignedResource; } + public Optional getResultPartitionDeploymentDescriptor( + IntermediateResultPartitionID id) { + return Optional.ofNullable(producedPartitions.get(id)); + } + /** * Tries to assign the given slot to the execution. The assignment works only if the * Execution is in state SCHEDULED. Returns true, if the resource could be assigned. @@ -423,7 +442,7 @@ public CompletableFuture scheduleForExecution( final ExecutionGraph executionGraph = vertex.getExecutionGraph(); final Time allocationTimeout = executionGraph.getAllocationTimeout(); try { - final CompletableFuture allocationFuture = allocateAndAssignSlotForExecution( + final CompletableFuture allocationFuture = allocateResourcesForExecution( slotProvider, queued, locationPreferenceConstraint, @@ -463,7 +482,13 @@ public CompletableFuture scheduleForExecution( } /** - * Allocates and assigns a slot obtained from the slot provider to the execution. + * Allocates resources for the execution. + * + *

Allocates following resources: + *

    + *
  1. slot obtained from the slot provider
  2. + *
  3. registers produced partitions with the {@link org.apache.flink.runtime.shuffle.ShuffleMaster}
  4. + *
* * @param slotProvider to obtain a new slot from * @param queued if the allocation can be queued @@ -473,14 +498,40 @@ public CompletableFuture scheduleForExecution( * @param allocationTimeout rpcTimeout for allocating a new slot * @return Future which is completed with this execution once the slot has been assigned * or with an exception if an error occurred. - * @throws IllegalExecutionStateException if this method has been called while not being in the CREATED state */ - public CompletableFuture allocateAndAssignSlotForExecution( + CompletableFuture allocateResourcesForExecution( + SlotProvider slotProvider, + boolean queued, + LocationPreferenceConstraint locationPreferenceConstraint, + @Nonnull Set allPreviousExecutionGraphAllocationIds, + Time allocationTimeout) { + return allocateAndAssignSlotForExecution( + slotProvider, + queued, + locationPreferenceConstraint, + allPreviousExecutionGraphAllocationIds, + allocationTimeout) + .thenCompose(slot -> registerProducedPartitions(slot.getTaskManagerLocation())); + } + + /** + * Allocates and assigns a slot obtained from the slot provider to the execution. + * + * @param slotProvider to obtain a new slot from + * @param queued if the allocation can be queued + * @param locationPreferenceConstraint constraint for the location preferences + * @param allPreviousExecutionGraphAllocationIds set with all previous allocation ids in the job graph. + * Can be empty if the allocation ids are not required for scheduling. + * @param allocationTimeout rpcTimeout for allocating a new slot + * @return Future which is completed with the allocated slot once it has been assigned + * or with an exception if an error occurred. + */ + private CompletableFuture allocateAndAssignSlotForExecution( SlotProvider slotProvider, boolean queued, LocationPreferenceConstraint locationPreferenceConstraint, @Nonnull Set allPreviousExecutionGraphAllocationIds, - Time allocationTimeout) throws IllegalExecutionStateException { + Time allocationTimeout) { checkNotNull(slotProvider); @@ -551,7 +602,7 @@ public CompletableFuture allocateAndAssignSlotForExecution( } if (tryAssignResource(logicalSlot)) { - return this; + return logicalSlot; } else { // release the slot logicalSlot.releaseSlot(new FlinkException("Could not assign logical slot to execution " + this + '.')); @@ -566,6 +617,68 @@ public CompletableFuture allocateAndAssignSlotForExecution( } } + @VisibleForTesting + CompletableFuture registerProducedPartitions(TaskManagerLocation location) { + assertRunningInJobMasterMainThread(); + + return FutureUtils.thenApplyAsyncIfNotDone( + registerProducedPartitions(vertex, location, attemptId), + vertex.getExecutionGraph().getJobMasterMainThreadExecutor(), + producedPartitionsCache -> { + producedPartitions = producedPartitionsCache; + return this; + }); + } + + @VisibleForTesting + static CompletableFuture> registerProducedPartitions( + ExecutionVertex vertex, + TaskManagerLocation location, + ExecutionAttemptID attemptId) { + ProducerDescriptor producerDescriptor = ProducerDescriptor.create(location, attemptId); + + boolean lazyScheduling = vertex.getExecutionGraph().getScheduleMode().allowLazyDeployment(); + + Collection partitions = vertex.getProducedPartitions().values(); + Collection> partitionRegistrations = + new ArrayList<>(partitions.size()); + + for (IntermediateResultPartition partition : partitions) { + PartitionDescriptor partitionDescriptor = PartitionDescriptor.from(partition); + int maxParallelism = getPartitionMaxParallelism(partition); + CompletableFuture shuffleDescriptorFuture = vertex + .getExecutionGraph() + .getShuffleMaster() + .registerPartitionWithProducer(partitionDescriptor, producerDescriptor); + CompletableFuture partitionRegistration = shuffleDescriptorFuture + .thenApply(shuffleDescriptor -> new ResultPartitionDeploymentDescriptor( + partitionDescriptor, + shuffleDescriptor, + maxParallelism, + lazyScheduling)); + partitionRegistrations.add(partitionRegistration); + } + + return FutureUtils.combineAll(partitionRegistrations).thenApply(rpdds -> { + Map producedPartitions = + new LinkedHashMap<>(partitions.size()); + rpdds.forEach(rpdd -> producedPartitions.put(rpdd.getPartitionId(), rpdd)); + return producedPartitions; + }); + } + + private static int getPartitionMaxParallelism(IntermediateResultPartition partition) { + // TODO consumers.isEmpty() only exists for test, currently there has to be exactly one consumer in real jobs! + final List> consumers = partition.getConsumers(); + int maxParallelism = KeyGroupRangeAssignment.UPPER_BOUND_MAX_PARALLELISM; + if (!consumers.isEmpty()) { + List consumer = consumers.get(0); + ExecutionJobVertex consumerVertex = consumer.get(0).getTarget().getJobVertex(); + maxParallelism = consumerVertex.getMaxParallelism(); + } + return maxParallelism; + } + /** * Deploys the execution to the previously assigned resource. * @@ -618,11 +731,13 @@ public void deploy() throws JobException { attemptNumber, getAssignedResourceLocation())); } - final TaskDeploymentDescriptor deployment = vertex.createDeploymentDescriptor( - attemptId, - slot, - taskRestore, - attemptNumber); + final TaskDeploymentDescriptor deployment = TaskDeploymentDescriptorFactory + .fromExecutionVertex(vertex, attemptNumber) + .createDeploymentDescriptor( + slot.getAllocationId(), + slot.getPhysicalSlotNumber(), + taskRestore, + producedPartitions.values()); // null taskRestore to let it be GC'ed taskRestore = null; @@ -632,7 +747,6 @@ public void deploy() throws JobException { final ComponentMainThreadExecutor jobMasterMainThreadExecutor = vertex.getExecutionGraph().getJobMasterMainThreadExecutor(); - // We run the submission in the future executor so that the serialization of large TDDs does not block // the main thread and sync back to the main thread once submission is completed. CompletableFuture.supplyAsync(() -> taskManagerGateway.submitTask(deployment, rpcTimeout), executor) @@ -783,7 +897,7 @@ void scheduleOrUpdateConsumers(List> allConsumers) { // sent after switching to running // ---------------------------------------------------------------- else if (consumerState == DEPLOYING || consumerState == RUNNING) { - final PartitionInfo partitionInfo = PartitionInfo.fromEdge(edge); + final PartitionInfo partitionInfo = createPartitionInfo(edge); if (consumerState == DEPLOYING) { consumerVertex.cachePartitionInfo(partitionInfo); @@ -794,6 +908,12 @@ else if (consumerState == DEPLOYING || consumerState == RUNNING) { } } + private static PartitionInfo createPartitionInfo(ExecutionEdge executionEdge) { + IntermediateDataSetID intermediateDataSetID = executionEdge.getSource().getIntermediateResult().getId(); + ShuffleDescriptor shuffleDescriptor = getConsumedPartitionShuffleDescriptor(executionEdge, false); + return new PartitionInfo(intermediateDataSetID, shuffleDescriptor); + } + /** * This method fails the vertex due to an external condition. The task will move to state FAILED. * If the task was in state RUNNING or DEPLOYING before, it will send a cancel call to the TaskManager. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index a82586061eb47..957f1ca7954f9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -64,6 +64,8 @@ import org.apache.flink.runtime.jobmaster.slotpool.Scheduler; import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; import org.apache.flink.runtime.query.KvStateLocationRegistry; +import org.apache.flink.runtime.shuffle.NettyShuffleMaster; +import org.apache.flink.runtime.shuffle.ShuffleMaster; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.taskmanager.TaskExecutionState; @@ -300,6 +302,9 @@ public class ExecutionGraph implements AccessExecutionGraph { // ------ Fields that are only relevant for archived execution graphs ------------ private String jsonPlan; + /** Shuffle master to register partitions for task deployment. */ + private final ShuffleMaster shuffleMaster = NettyShuffleMaster.INSTANCE; + // -------------------------------------------------------------------------------------------- // Constructors // -------------------------------------------------------------------------------------------- @@ -1803,4 +1808,8 @@ void assertRunningInJobMasterMainThread() { jobMasterMainThreadExecutor.assertRunningInMainThread(); } } + + ShuffleMaster getShuffleMaster() { + return shuffleMaster; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java index 9254ba270607d..0f7e7f6fc8800 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java @@ -529,7 +529,7 @@ public Collection> allocateResourcesForAll( for (int i = 0; i < vertices.length; i++) { // allocate the next slot (future) final Execution exec = vertices[i].getCurrentExecutionAttempt(); - final CompletableFuture allocationFuture = exec.allocateAndAssignSlotForExecution( + final CompletableFuture allocationFuture = exec.allocateResourcesForExecution( resourceProvider, queued, locationPreferenceConstraint, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index 757e2f4a73aa3..9b0cfa0c6d250 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -27,18 +27,10 @@ import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.io.InputSplitAssigner; import org.apache.flink.runtime.JobException; -import org.apache.flink.runtime.blob.PermanentBlobKey; -import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore; import org.apache.flink.runtime.clusterframework.types.AllocationID; -import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; -import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; -import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; -import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; -import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.DistributionPattern; -import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.jobgraph.JobEdge; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -47,20 +39,15 @@ import org.apache.flink.runtime.jobmanager.scheduler.LocationPreferenceConstraint; import org.apache.flink.runtime.jobmaster.LogicalSlot; import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; -import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.util.EvictingBoundedList; -import org.apache.flink.types.Either; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.SerializedValue; import org.slf4j.Logger; import javax.annotation.Nonnull; import javax.annotation.Nullable; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -252,6 +239,10 @@ public ExecutionEdge[] getInputEdges(int input) { return inputEdges[input]; } + public ExecutionEdge[][] getAllInputEdges() { + return inputEdges; + } + public CoLocationConstraint getLocationConstraint() { return locationConstraint; } @@ -810,108 +801,6 @@ void notifyStateTransition(Execution execution, ExecutionState newState, Throwab } } - /** - * Creates a task deployment descriptor to deploy a subtask to the given target slot. - * TODO: This should actually be in the EXECUTION - */ - TaskDeploymentDescriptor createDeploymentDescriptor( - ExecutionAttemptID executionId, - LogicalSlot targetSlot, - @Nullable JobManagerTaskRestore taskRestore, - int attemptNumber) throws ExecutionGraphException { - - // Produced intermediate results - List producedPartitions = new ArrayList<>(resultPartitions.size()); - - // Consumed intermediate results - List consumedPartitions = new ArrayList<>(inputEdges.length); - - boolean lazyScheduling = getExecutionGraph().getScheduleMode().allowLazyDeployment(); - - for (IntermediateResultPartition partition : resultPartitions.values()) { - - List> consumers = partition.getConsumers(); - - if (consumers.isEmpty()) { - //TODO this case only exists for test, currently there has to be exactly one consumer in real jobs! - producedPartitions.add(ResultPartitionDeploymentDescriptor.from( - partition, - KeyGroupRangeAssignment.UPPER_BOUND_MAX_PARALLELISM, - lazyScheduling)); - } else { - Preconditions.checkState(1 == consumers.size(), - "Only one consumer supported in the current implementation! Found: " + consumers.size()); - - List consumer = consumers.get(0); - ExecutionJobVertex vertex = consumer.get(0).getTarget().getJobVertex(); - int maxParallelism = vertex.getMaxParallelism(); - producedPartitions.add(ResultPartitionDeploymentDescriptor.from(partition, maxParallelism, lazyScheduling)); - } - } - - final InputChannelDeploymentDescriptor[] icddArray = new InputChannelDeploymentDescriptor[0]; - - for (ExecutionEdge[] edges : inputEdges) { - List partitions = InputChannelDeploymentDescriptor.fromEdges( - Arrays.asList(edges), - lazyScheduling); - - // If the produced partition has multiple consumers registered, we - // need to request the one matching our sub task index. - // TODO Refactor after removing the consumers from the intermediate result partitions - int numConsumerEdges = edges[0].getSource().getConsumers().get(0).size(); - - int queueToRequest = subTaskIndex % numConsumerEdges; - - IntermediateResult consumedIntermediateResult = edges[0].getSource().getIntermediateResult(); - final IntermediateDataSetID resultId = consumedIntermediateResult.getId(); - final ResultPartitionType partitionType = consumedIntermediateResult.getResultType(); - - consumedPartitions.add(new InputGateDeploymentDescriptor(resultId, partitionType, queueToRequest, partitions.toArray(icddArray))); - } - - final Either, PermanentBlobKey> jobInformationOrBlobKey = getExecutionGraph().getJobInformationOrBlobKey(); - - final TaskDeploymentDescriptor.MaybeOffloaded serializedJobInformation; - - if (jobInformationOrBlobKey.isLeft()) { - serializedJobInformation = new TaskDeploymentDescriptor.NonOffloaded<>(jobInformationOrBlobKey.left()); - } else { - serializedJobInformation = new TaskDeploymentDescriptor.Offloaded<>(jobInformationOrBlobKey.right()); - } - - final Either, PermanentBlobKey> taskInformationOrBlobKey; - - try { - taskInformationOrBlobKey = jobVertex.getTaskInformationOrBlobKey(); - } catch (IOException e) { - throw new ExecutionGraphException( - "Could not create a serialized JobVertexInformation for " + - jobVertex.getJobVertexId(), e); - } - - final TaskDeploymentDescriptor.MaybeOffloaded serializedTaskInformation; - - if (taskInformationOrBlobKey.isLeft()) { - serializedTaskInformation = new TaskDeploymentDescriptor.NonOffloaded<>(taskInformationOrBlobKey.left()); - } else { - serializedTaskInformation = new TaskDeploymentDescriptor.Offloaded<>(taskInformationOrBlobKey.right()); - } - - return new TaskDeploymentDescriptor( - getJobId(), - serializedJobInformation, - serializedTaskInformation, - executionId, - targetSlot.getAllocationId(), - subTaskIndex, - attemptNumber, - targetSlot.getPhysicalSlotNumber(), - taskRestore, - producedPartitions, - consumedPartitions); - } - // -------------------------------------------------------------------------------------------- // Utilities // -------------------------------------------------------------------------------------------- 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 index f2383748bea32..3714b520e150b 100644 --- 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 @@ -18,48 +18,36 @@ 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; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import java.io.Serializable; /** * Contains information where to find a partition. The partition is defined by the - * {@link IntermediateDataSetID} and the partition location is specified by - * {@link InputChannelDeploymentDescriptor}. + * {@link IntermediateDataSetID} and the partition is specified by + * {@link ShuffleDescriptor}. */ public class PartitionInfo implements Serializable { private static final long serialVersionUID = 1724490660830968430L; private final IntermediateDataSetID intermediateDataSetID; - private final InputChannelDeploymentDescriptor inputChannelDeploymentDescriptor; - public PartitionInfo(IntermediateDataSetID intermediateResultPartitionID, InputChannelDeploymentDescriptor inputChannelDeploymentDescriptor) { - this.intermediateDataSetID = Preconditions.checkNotNull(intermediateResultPartitionID); - this.inputChannelDeploymentDescriptor = Preconditions.checkNotNull(inputChannelDeploymentDescriptor); + private final ShuffleDescriptor shuffleDescriptor; + + public PartitionInfo( + IntermediateDataSetID intermediateResultPartitionID, + ShuffleDescriptor shuffleDescriptor) { + this.intermediateDataSetID = intermediateResultPartitionID; + this.shuffleDescriptor = shuffleDescriptor; } public IntermediateDataSetID getIntermediateDataSetID() { return intermediateDataSetID; } - public InputChannelDeploymentDescriptor getInputChannelDeploymentDescriptor() { - return inputChannelDeploymentDescriptor; - } - - // ------------------------------------------------------------------------ - - static PartitionInfo fromEdge(ExecutionEdge executionEdge) { - final InputChannelDeploymentDescriptor inputChannelDeploymentDescriptor = InputChannelDeploymentDescriptor.fromEdge(executionEdge); - - Preconditions.checkState( - !inputChannelDeploymentDescriptor.getConsumedPartitionLocation().isUnknown(), - "PartitionInfo contains an unknown partition location."); - - return new PartitionInfo( - executionEdge.getSource().getIntermediateResult().getId(), - inputChannelDeploymentDescriptor); + public ShuffleDescriptor getShuffleDescriptor() { + return shuffleDescriptor; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java index 7eac38140623a..995b260bc0dcf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -46,6 +47,8 @@ import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateFactory; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.taskexecutor.TaskExecutor; import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; import org.apache.flink.util.Preconditions; @@ -59,6 +62,7 @@ import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; +import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -80,6 +84,8 @@ public class NetworkEnvironment { private final Object lock = new Object(); + private final ResourceID taskExecutorLocation; + private final NetworkEnvironmentConfiguration config; private final NetworkBufferPool networkBufferPool; @@ -97,12 +103,14 @@ public class NetworkEnvironment { private boolean isShutdown; private NetworkEnvironment( + ResourceID taskExecutorLocation, NetworkEnvironmentConfiguration config, NetworkBufferPool networkBufferPool, ConnectionManager connectionManager, ResultPartitionManager resultPartitionManager, ResultPartitionFactory resultPartitionFactory, SingleInputGateFactory singleInputGateFactory) { + this.taskExecutorLocation = taskExecutorLocation; this.config = config; this.networkBufferPool = networkBufferPool; this.connectionManager = connectionManager; @@ -114,10 +122,12 @@ private NetworkEnvironment( } public static NetworkEnvironment create( + ResourceID taskExecutorLocation, NetworkEnvironmentConfiguration config, TaskEventPublisher taskEventPublisher, MetricGroup metricGroup, IOManager ioManager) { + checkNotNull(taskExecutorLocation); checkNotNull(ioManager); checkNotNull(taskEventPublisher); checkNotNull(config); @@ -145,6 +155,7 @@ public static NetworkEnvironment create( config.floatingNetworkBuffersPerGate()); SingleInputGateFactory singleInputGateFactory = new SingleInputGateFactory( + taskExecutorLocation, config, connectionManager, resultPartitionManager, @@ -152,6 +163,7 @@ public static NetworkEnvironment create( networkBufferPool); return new NetworkEnvironment( + taskExecutorLocation, config, networkBufferPool, connectionManager, @@ -299,7 +311,11 @@ public boolean updatePartitionInfo( if (inputGate == null) { return false; } - inputGate.updateInputChannel(partitionInfo.getInputChannelDeploymentDescriptor()); + ShuffleDescriptor shuffleDescriptor = partitionInfo.getShuffleDescriptor(); + checkArgument(shuffleDescriptor instanceof NettyShuffleDescriptor, + "Tried to update unknown channel with unknown ShuffleDescriptor %s.", + shuffleDescriptor.getClass().getName()); + inputGate.updateInputChannel(taskExecutorLocation, (NettyShuffleDescriptor) shuffleDescriptor); return true; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java index a42a93bee0f38..778aa1d3a5883 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java @@ -133,14 +133,14 @@ public RemoteInputChannel( * after this input channel is created. */ void assignExclusiveSegments() throws IOException { - checkState(this.initialCredit == 0, "Bug in input channel setup logic: exclusive buffers have " + + checkState(initialCredit == 0, "Bug in input channel setup logic: exclusive buffers have " + "already been set for this input channel."); Collection segments = checkNotNull(memorySegmentProvider.requestMemorySegments()); checkArgument(!segments.isEmpty(), "The number of exclusive buffers per channel should be larger than 0."); - this.initialCredit = segments.size(); - this.numRequiredBuffers = segments.size(); + initialCredit = segments.size(); + numRequiredBuffers = segments.size(); synchronized (bufferQueue) { for (MemorySegment segment : segments) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java index ac540e6e0c08e..7ba4298af1e8e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java @@ -19,8 +19,7 @@ package org.apache.flink.runtime.io.network.partition.consumer; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; -import org.apache.flink.runtime.deployment.ResultPartitionLocation; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; @@ -35,6 +34,7 @@ import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; import org.apache.flink.util.function.SupplierWithException; import org.slf4j.Logger; @@ -313,39 +313,33 @@ public void setInputChannel(IntermediateResultPartitionID partitionId, InputChan } } - public void updateInputChannel(InputChannelDeploymentDescriptor icdd) throws IOException, InterruptedException { + public void updateInputChannel( + ResourceID localLocation, + NettyShuffleDescriptor shuffleDescriptor) throws IOException, InterruptedException { synchronized (requestLock) { if (closeFuture.isDone()) { // There was a race with a task failure/cancel return; } - final IntermediateResultPartitionID partitionId = icdd.getConsumedPartitionId().getPartitionId(); + IntermediateResultPartitionID partitionId = shuffleDescriptor.getResultPartitionID().getPartitionId(); InputChannel current = inputChannels.get(partitionId); if (current instanceof UnknownInputChannel) { - UnknownInputChannel unknownChannel = (UnknownInputChannel) current; - + boolean isLocal = shuffleDescriptor.isLocalTo(localLocation); InputChannel newChannel; - - ResultPartitionLocation partitionLocation = icdd.getConsumedPartitionLocation(); - - if (partitionLocation.isLocal()) { + if (isLocal) { newChannel = unknownChannel.toLocalInputChannel(); - } - else if (partitionLocation.isRemote()) { - newChannel = unknownChannel.toRemoteInputChannel(partitionLocation.getConnectionId()); - - if (this.isCreditBased) { - ((RemoteInputChannel) newChannel).assignExclusiveSegments(); + } else { + RemoteInputChannel remoteInputChannel = + unknownChannel.toRemoteInputChannel(shuffleDescriptor.getConnectionId()); + if (isCreditBased) { + remoteInputChannel.assignExclusiveSegments(); } + newChannel = remoteInputChannel; } - else { - throw new IllegalStateException("Tried to update unknown channel with unknown channel."); - } - LOG.debug("{}: Updated unknown input channel to {}.", owningTaskName, newChannel); inputChannels.put(partitionId, newChannel); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java index 6caf0174fac70..554d723581f01 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java @@ -19,9 +19,8 @@ package org.apache.flink.runtime.io.network.partition.consumer; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; -import org.apache.flink.runtime.deployment.ResultPartitionLocation; import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.TaskEventPublisher; import org.apache.flink.runtime.io.network.buffer.BufferPool; @@ -32,7 +31,8 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; import org.apache.flink.util.function.SupplierWithException; @@ -43,8 +43,7 @@ import java.io.IOException; -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.runtime.shuffle.ShuffleUtils.applyWithShuffleTypeCheck; /** * Factory for {@link SingleInputGate} to use in {@link org.apache.flink.runtime.io.network.NetworkEnvironment}. @@ -52,6 +51,9 @@ public class SingleInputGateFactory { private static final Logger LOG = LoggerFactory.getLogger(SingleInputGate.class); + @Nonnull + private final ResourceID taskExecutorLocation; + private final boolean isCreditBased; private final int partitionRequestInitialBackoff; @@ -75,11 +77,13 @@ public class SingleInputGateFactory { private final int floatingNetworkBuffersPerGate; public SingleInputGateFactory( + @Nonnull ResourceID taskExecutorLocation, @Nonnull NetworkEnvironmentConfiguration networkConfig, @Nonnull ConnectionManager connectionManager, @Nonnull ResultPartitionManager partitionManager, @Nonnull TaskEventPublisher taskEventPublisher, @Nonnull NetworkBufferPool networkBufferPool) { + this.taskExecutorLocation = taskExecutorLocation; this.isCreditBased = networkConfig.isCreditBased(); this.partitionRequestInitialBackoff = networkConfig.partitionRequestInitialBackoff(); this.partitionRequestMaxBackoff = networkConfig.partitionRequestMaxBackoff(); @@ -99,58 +103,72 @@ public SingleInputGate create( @Nonnull InputGateDeploymentDescriptor igdd, @Nonnull PartitionProducerStateProvider partitionProducerStateProvider, @Nonnull InputChannelMetrics metrics) { - final IntermediateDataSetID consumedResultId = checkNotNull(igdd.getConsumedResultId()); - final ResultPartitionType consumedPartitionType = checkNotNull(igdd.getConsumedPartitionType()); - - final int consumedSubpartitionIndex = igdd.getConsumedSubpartitionIndex(); - checkArgument(consumedSubpartitionIndex >= 0); - - final InputChannelDeploymentDescriptor[] icdd = checkNotNull(igdd.getInputChannelDeploymentDescriptors()); + SupplierWithException bufferPoolFactory = createBufferPoolFactory( + networkBufferPool, + isCreditBased, + networkBuffersPerChannel, + floatingNetworkBuffersPerGate, + igdd.getShuffleDescriptors().length, + igdd.getConsumedPartitionType()); - final SingleInputGate inputGate = new SingleInputGate( + SingleInputGate inputGate = new SingleInputGate( owningTaskName, - consumedResultId, - consumedPartitionType, - consumedSubpartitionIndex, - icdd.length, + igdd.getConsumedResultId(), + igdd.getConsumedPartitionType(), + igdd.getConsumedSubpartitionIndex(), + igdd.getShuffleDescriptors().length, partitionProducerStateProvider, isCreditBased, - createBufferPoolFactory(icdd.length, consumedPartitionType)); + bufferPoolFactory); + + createInputChannels(owningTaskName, igdd, inputGate, metrics); + return inputGate; + } + + private void createInputChannels( + String owningTaskName, + InputGateDeploymentDescriptor inputGateDeploymentDescriptor, + SingleInputGate inputGate, + InputChannelMetrics metrics) { + ShuffleDescriptor[] shuffleDescriptors = inputGateDeploymentDescriptor.getShuffleDescriptors(); // Create the input channels. There is one input channel for each consumed partition. - final InputChannel[] inputChannels = new InputChannel[icdd.length]; + InputChannel[] inputChannels = new InputChannel[shuffleDescriptors.length]; - int numLocalChannels = 0; - int numRemoteChannels = 0; - int numUnknownChannels = 0; + ChannelStatistics channelStatistics = new ChannelStatistics(); for (int i = 0; i < inputChannels.length; i++) { - final ResultPartitionID partitionId = icdd[i].getConsumedPartitionId(); - final ResultPartitionLocation partitionLocation = icdd[i].getConsumedPartitionLocation(); - - if (partitionLocation.isLocal()) { - inputChannels[i] = new LocalInputChannel(inputGate, i, partitionId, - partitionManager, - taskEventPublisher, - partitionRequestInitialBackoff, - partitionRequestMaxBackoff, - metrics); + inputChannels[i] = createInputChannel( + inputGate, + i, + shuffleDescriptors[i], + channelStatistics, + metrics); + ResultPartitionID resultPartitionID = inputChannels[i].getPartitionId(); + inputGate.setInputChannel(resultPartitionID.getPartitionId(), inputChannels[i]); + } - numLocalChannels++; - } - else if (partitionLocation.isRemote()) { - inputChannels[i] = new RemoteInputChannel(inputGate, i, partitionId, - partitionLocation.getConnectionId(), - connectionManager, - partitionRequestInitialBackoff, - partitionRequestMaxBackoff, - metrics, - networkBufferPool); + LOG.debug("{}: Created {} input channels ({}).", + owningTaskName, + inputChannels.length, + channelStatistics); + } - numRemoteChannels++; - } - else if (partitionLocation.isUnknown()) { - inputChannels[i] = new UnknownInputChannel(inputGate, i, partitionId, + private InputChannel createInputChannel( + SingleInputGate inputGate, + int index, + ShuffleDescriptor shuffleDescriptor, + ChannelStatistics channelStatistics, + InputChannelMetrics metrics) { + return applyWithShuffleTypeCheck( + NettyShuffleDescriptor.class, + shuffleDescriptor, + unknownShuffleDescriptor -> { + channelStatistics.numUnknownChannels++; + return new UnknownInputChannel( + inputGate, + index, + unknownShuffleDescriptor.getResultPartitionID(), partitionManager, taskEventPublisher, connectionManager, @@ -158,51 +176,81 @@ else if (partitionLocation.isUnknown()) { partitionRequestMaxBackoff, metrics, networkBufferPool); - - numUnknownChannels++; - } - else { - throw new IllegalStateException("Unexpected partition location."); - } - - inputGate.setInputChannel(partitionId.getPartitionId(), inputChannels[i]); - } - - LOG.debug("{}: Created {} input channels (local: {}, remote: {}, unknown: {}).", - owningTaskName, - inputChannels.length, - numLocalChannels, - numRemoteChannels, - numUnknownChannels); - - return inputGate; + }, + nettyShuffleDescriptor -> + createKnownInputChannel( + inputGate, + index, + nettyShuffleDescriptor, + channelStatistics, + metrics)); } - private SupplierWithException createBufferPoolFactory(int size, ResultPartitionType type) { - return createBufferPoolFactory( - networkBufferPool, isCreditBased, networkBuffersPerChannel, floatingNetworkBuffersPerGate, size, type); + private InputChannel createKnownInputChannel( + SingleInputGate inputGate, + int index, + NettyShuffleDescriptor inputChannelDescriptor, + ChannelStatistics channelStatistics, + InputChannelMetrics metrics) { + ResultPartitionID partitionId = inputChannelDescriptor.getResultPartitionID(); + if (inputChannelDescriptor.isLocalTo(taskExecutorLocation)) { + // Consuming task is deployed to the same TaskManager as the partition => local + channelStatistics.numLocalChannels++; + return new LocalInputChannel( + inputGate, + index, + partitionId, + partitionManager, + taskEventPublisher, + partitionRequestInitialBackoff, + partitionRequestMaxBackoff, + metrics); + } else { + // Different instances => remote + channelStatistics.numRemoteChannels++; + return new RemoteInputChannel( + inputGate, + index, + partitionId, + inputChannelDescriptor.getConnectionId(), + connectionManager, + partitionRequestInitialBackoff, + partitionRequestMaxBackoff, + metrics, + networkBufferPool); + } } @VisibleForTesting static SupplierWithException createBufferPoolFactory( - BufferPoolFactory bufferPoolFactory, - boolean isCreditBased, - int networkBuffersPerChannel, - int floatingNetworkBuffersPerGate, - int size, - ResultPartitionType type) { - + BufferPoolFactory bufferPoolFactory, + boolean isCreditBased, + int networkBuffersPerChannel, + int floatingNetworkBuffersPerGate, + int size, + ResultPartitionType type) { if (isCreditBased) { - int maxNumberOfMemorySegments = type.isBounded() ? - floatingNetworkBuffersPerGate : Integer.MAX_VALUE; - + int maxNumberOfMemorySegments = type.isBounded() ? floatingNetworkBuffersPerGate : Integer.MAX_VALUE; return () -> bufferPoolFactory.createBufferPool(0, maxNumberOfMemorySegments); } else { int maxNumberOfMemorySegments = type.isBounded() ? - size * networkBuffersPerChannel + - floatingNetworkBuffersPerGate : Integer.MAX_VALUE; - + size * networkBuffersPerChannel + floatingNetworkBuffersPerGate : Integer.MAX_VALUE; return () -> bufferPoolFactory.createBufferPool(size, maxNumberOfMemorySegments); } } + + private static class ChannelStatistics { + int numLocalChannels = 0; + int numRemoteChannels = 0; + int numUnknownChannels = 0; + + @Override + public String toString() { + return String.format( + "local: %s, remote: %s, unknown: %s", + numLocalChannels, + numRemoteChannels, + numUnknownChannels); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleDescriptor.java new file mode 100644 index 0000000000000..34739199cc33d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleDescriptor.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.shuffle; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.io.network.ConnectionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; + +import java.io.Serializable; +import java.net.InetSocketAddress; + +/** + * Default implementation of {@link ShuffleDescriptor} for {@link NettyShuffleMaster}. + */ +public class NettyShuffleDescriptor implements ShuffleDescriptor { + + private static final long serialVersionUID = 852181945034989215L; + + private final ResourceID producerLocation; + + private final PartitionConnectionInfo partitionConnectionInfo; + + private final ResultPartitionID resultPartitionID; + + public NettyShuffleDescriptor( + ResourceID producerLocation, + PartitionConnectionInfo partitionConnectionInfo, + ResultPartitionID resultPartitionID) { + this.producerLocation = producerLocation; + this.partitionConnectionInfo = partitionConnectionInfo; + this.resultPartitionID = resultPartitionID; + } + + public ConnectionID getConnectionId() { + return partitionConnectionInfo.getConnectionId(); + } + + @Override + public ResultPartitionID getResultPartitionID() { + return resultPartitionID; + } + + public boolean isLocalTo(ResourceID consumerLocation) { + return producerLocation.equals(consumerLocation); + } + + /** + * Information for connection to partition producer for shuffle exchange. + */ + public interface PartitionConnectionInfo extends Serializable { + ConnectionID getConnectionId(); + } + + /** + * Remote partition connection information with index to query partition. + * + *

Normal connection information with network address and port for connection in case of distributed execution. + */ + public static class NetworkPartitionConnectionInfo implements PartitionConnectionInfo { + + private static final long serialVersionUID = 5992534320110743746L; + + private final ConnectionID connectionID; + + @VisibleForTesting + public NetworkPartitionConnectionInfo(ConnectionID connectionID) { + this.connectionID = connectionID; + } + + @Override + public ConnectionID getConnectionId() { + return connectionID; + } + + static NetworkPartitionConnectionInfo fromProducerDescriptor( + ProducerDescriptor producerDescriptor, + int connectionIndex) { + InetSocketAddress address = + new InetSocketAddress(producerDescriptor.getAddress(), producerDescriptor.getDataPort()); + return new NetworkPartitionConnectionInfo(new ConnectionID(address, connectionIndex)); + } + } + + /** + * Local partition connection information. + * + *

Does not have any network connection information in case of local execution. + */ + public enum LocalExecutionPartitionConnectionInfo implements PartitionConnectionInfo { + INSTANCE; + + @Override + public ConnectionID getConnectionId() { + throw new UnsupportedOperationException("Local execution does not support shuffle connection."); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleMaster.java new file mode 100644 index 0000000000000..d396ce3fabf28 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleMaster.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.shuffle; + +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor.LocalExecutionPartitionConnectionInfo; +import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor.NetworkPartitionConnectionInfo; + +import java.util.concurrent.CompletableFuture; + +/** + * Default {@link ShuffleMaster} for netty and local file based shuffle implementation. + */ +public enum NettyShuffleMaster implements ShuffleMaster { + INSTANCE; + + @Override + public CompletableFuture registerPartitionWithProducer( + PartitionDescriptor partitionDescriptor, + ProducerDescriptor producerDescriptor) { + + ResultPartitionID resultPartitionID = new ResultPartitionID( + partitionDescriptor.getPartitionId(), + producerDescriptor.getProducerExecutionId()); + + NettyShuffleDescriptor shuffleDeploymentDescriptor = new NettyShuffleDescriptor( + producerDescriptor.getProducerLocation(), + createConnectionInfo(producerDescriptor, partitionDescriptor.getConnectionIndex()), + resultPartitionID); + + return CompletableFuture.completedFuture(shuffleDeploymentDescriptor); + } + + private static NettyShuffleDescriptor.PartitionConnectionInfo createConnectionInfo( + ProducerDescriptor producerDescriptor, + int connectionIndex) { + return producerDescriptor.getDataPort() >= 0 ? + NetworkPartitionConnectionInfo.fromProducerDescriptor(producerDescriptor, connectionIndex) : + LocalExecutionPartitionConnectionInfo.INSTANCE; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/PartitionDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/PartitionDescriptor.java new file mode 100644 index 0000000000000..e63c975070883 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/PartitionDescriptor.java @@ -0,0 +1,127 @@ +/* + * 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.shuffle; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.executiongraph.ExecutionEdge; +import org.apache.flink.runtime.executiongraph.IntermediateResult; +import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; + +import java.io.Serializable; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Partition descriptor for {@link ShuffleMaster} to obtain {@link ShuffleDescriptor}. + */ +public class PartitionDescriptor implements Serializable { + + private static final long serialVersionUID = 6343547936086963705L; + + /** The ID of the result this partition belongs to. */ + private final IntermediateDataSetID resultId; + + /** The ID of the partition. */ + private final IntermediateResultPartitionID partitionId; + + /** The type of the partition. */ + private final ResultPartitionType partitionType; + + /** The number of subpartitions. */ + private final int numberOfSubpartitions; + + /** Connection index to identify this partition of intermediate result. */ + private final int connectionIndex; + + @VisibleForTesting + public PartitionDescriptor( + IntermediateDataSetID resultId, + IntermediateResultPartitionID partitionId, + ResultPartitionType partitionType, + int numberOfSubpartitions, + int connectionIndex) { + this.resultId = checkNotNull(resultId); + this.partitionId = checkNotNull(partitionId); + this.partitionType = checkNotNull(partitionType); + checkArgument(numberOfSubpartitions >= 1); + this.numberOfSubpartitions = numberOfSubpartitions; + this.connectionIndex = connectionIndex; + } + + public IntermediateDataSetID getResultId() { + return resultId; + } + + public IntermediateResultPartitionID getPartitionId() { + return partitionId; + } + + public ResultPartitionType getPartitionType() { + return partitionType; + } + + public int getNumberOfSubpartitions() { + return numberOfSubpartitions; + } + + int getConnectionIndex() { + return connectionIndex; + } + + @Override + public String toString() { + return String.format( + "PartitionDescriptor [result id: %s, partition id: %s, partition type: %s, " + + "subpartitions: %d, connection index: %d]", + resultId, + partitionId, + partitionType, + numberOfSubpartitions, + connectionIndex); + } + + public static PartitionDescriptor from(IntermediateResultPartition partition) { + checkNotNull(partition); + + // The produced data is partitioned among a number of subpartitions. + // + // If no consumers are known at this point, we use a single subpartition, otherwise we have + // one for each consuming sub task. + int numberOfSubpartitions = 1; + List> consumers = partition.getConsumers(); + if (!consumers.isEmpty() && !consumers.get(0).isEmpty()) { + if (consumers.size() > 1) { + throw new IllegalStateException("Currently, only a single consumer group per partition is supported."); + } + numberOfSubpartitions = consumers.get(0).size(); + } + IntermediateResult result = partition.getIntermediateResult(); + return new PartitionDescriptor( + result.getId(), + partition.getPartitionId(), + result.getResultType(), + numberOfSubpartitions, + result.getConnectionIndex()); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ProducerDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ProducerDescriptor.java new file mode 100644 index 0000000000000..ad3fc487a5386 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ProducerDescriptor.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.shuffle; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + +import java.net.InetAddress; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Partition producer descriptor for {@link ShuffleMaster} to obtain {@link ShuffleDescriptor}. + * + *

The producer descriptor contains general producer specific information relevant for the shuffle service: + * the producer location as {@link ResourceID}, {@link ExecutionAttemptID} and the network connection information + * for shuffle data exchange (address and port). + */ +public class ProducerDescriptor { + /** The resource ID to identify the container where the producer execution is deployed. */ + private final ResourceID producerLocation; + + /** The ID of the producer execution attempt. */ + private final ExecutionAttemptID producerExecutionId; + + /** The address to connect to the producer. */ + private final InetAddress address; + + /** + * The port to connect to the producer for shuffle exchange. + * + *

Negative value means local execution. + */ + private final int dataPort; + + @VisibleForTesting + public ProducerDescriptor( + ResourceID producerLocation, + ExecutionAttemptID producerExecutionId, + InetAddress address, + int dataPort) { + this.producerLocation = checkNotNull(producerLocation); + this.producerExecutionId = checkNotNull(producerExecutionId); + this.address = checkNotNull(address); + this.dataPort = dataPort; + } + + ResourceID getProducerLocation() { + return producerLocation; + } + + ExecutionAttemptID getProducerExecutionId() { + return producerExecutionId; + } + + public InetAddress getAddress() { + return address; + } + + public int getDataPort() { + return dataPort; + } + + public static ProducerDescriptor create(TaskManagerLocation producerLocation, ExecutionAttemptID attemptId) { + return new ProducerDescriptor( + producerLocation.getResourceID(), + attemptId, + producerLocation.address(), + producerLocation.dataPort()); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleDescriptor.java new file mode 100644 index 0000000000000..fcb1b327783e5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleDescriptor.java @@ -0,0 +1,53 @@ +/* + * 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.shuffle; + +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; + +import java.io.Serializable; + +/** + * Interface for shuffle deployment descriptor of result partition resource. + * + *

The descriptor is used for the deployment of the partition producer/consumer and their data exchange + */ +public interface ShuffleDescriptor extends Serializable { + + ResultPartitionID getResultPartitionID(); + + /** + * Returns whether the partition is known and registered with the {@link ShuffleMaster} implementation. + * + *

When a partition consumer is being scheduled, it can happen + * that the producer of the partition (consumer input channel) has not been scheduled + * and its location and other relevant data is yet to be defined. + * To proceed with the consumer deployment, currently unknown input channels have to be + * marked with placeholders which are special implementation of {@link ShuffleDescriptor}: + * {@link UnknownShuffleDescriptor}. + * + *

Note: this method is not supposed to be overridden in concrete shuffle implementation. + * The only class where it returns {@code true} is {@link UnknownShuffleDescriptor}. + * + * @return whether the partition producer has been ever deployed and + * the corresponding {@link ShuffleDescriptor} is obtained from the {@link ShuffleMaster} implementation. + */ + default boolean isUnknown() { + return false; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleMaster.java new file mode 100644 index 0000000000000..a4bc816b00b2f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleMaster.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.shuffle; + +import java.util.concurrent.CompletableFuture; + +/** + * Intermediate result partition registry to use in {@link org.apache.flink.runtime.jobmaster.JobMaster}. + * + * @param partition shuffle descriptor used for producer/consumer deployment and their data exchange. + */ +public interface ShuffleMaster { + + /** + * Asynchronously register a partition and its producer with the shuffle service. + * + *

The returned shuffle descriptor is an internal handle + * which identifies the partition internally within the shuffle service. + * The descriptor should provide enough information to read from or write data to the partition. + * + * @param partitionDescriptor general job graph information about the partition + * @param producerDescriptor general producer information (location, execution id, connection info) + * @return future with the partition shuffle descriptor used for producer/consumer deployment and their data exchange. + */ + CompletableFuture registerPartitionWithProducer( + PartitionDescriptor partitionDescriptor, + ProducerDescriptor producerDescriptor); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleUtils.java new file mode 100644 index 0000000000000..f6d5274e22897 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleUtils.java @@ -0,0 +1,61 @@ +/* + * 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.shuffle; + +import java.util.function.Function; + +/** + * Common utility methods for shuffle service. + */ +public class ShuffleUtils { + + private ShuffleUtils() { + } + + /** + * Apply different functions to known and unknown {@link ShuffleDescriptor}s. + * + *

Also casts known {@link ShuffleDescriptor}. + * + * @param shuffleDescriptorClass concrete class of {@code shuffleDescriptor} + * @param shuffleDescriptor concrete shuffle descriptor to check + * @param functionOfUnknownDescriptor supplier to call in case {@code shuffleDescriptor} is unknown + * @param functionOfKnownDescriptor function to call in case {@code shuffleDescriptor} is known + * @param return type of called functions + * @param concrete type of {@code shuffleDescriptor} to check + * @return result of either function call + */ + @SuppressWarnings("unchecked") + public static T applyWithShuffleTypeCheck( + Class shuffleDescriptorClass, + ShuffleDescriptor shuffleDescriptor, + Function functionOfUnknownDescriptor, + Function functionOfKnownDescriptor) { + if (shuffleDescriptor.isUnknown()) { + return functionOfUnknownDescriptor.apply((UnknownShuffleDescriptor) shuffleDescriptor); + } else if (shuffleDescriptorClass.equals(shuffleDescriptor.getClass())) { + return functionOfKnownDescriptor.apply((SD) shuffleDescriptor); + } else { + throw new IllegalArgumentException(String.format( + "Unsupported ShuffleDescriptor type <%s>, only <%s> is supported", + shuffleDescriptor.getClass().getName(), + shuffleDescriptorClass.getName())); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/UnknownShuffleDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/UnknownShuffleDescriptor.java new file mode 100644 index 0000000000000..8ce71c3a2c4f4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/UnknownShuffleDescriptor.java @@ -0,0 +1,51 @@ +/* + * 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.shuffle; + +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; + +/** + * Unknown {@link ShuffleDescriptor} for which the producer has not been deployed yet. + * +

When a partition consumer is being scheduled, it can happen + * that the producer of the partition (consumer input channel) has not been scheduled + * and its location and other relevant data is yet to be defined. + * To proceed with the consumer deployment, currently unknown input channels have to be + * marked with placeholders which are this special implementation of {@link ShuffleDescriptor}. + */ +public final class UnknownShuffleDescriptor implements ShuffleDescriptor { + + private static final long serialVersionUID = -4001330825983412431L; + + private final ResultPartitionID resultPartitionID; + + public UnknownShuffleDescriptor(ResultPartitionID resultPartitionID) { + this.resultPartitionID = resultPartitionID; + } + + @Override + public ResultPartitionID getResultPartitionID() { + return resultPartitionID; + } + + @Override + public boolean isUnknown() { + return true; + } +} 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 efc2b427c48a7..9d62f7179f862 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 @@ -619,7 +619,7 @@ public CompletableFuture updatePartitions( log.debug( "Discard update for input gate partition {} of result {} in task {}. " + "The partition is no longer available.", - partitionInfo.getInputChannelDeploymentDescriptor().getConsumedPartitionId(), + partitionInfo.getShuffleDescriptor().getResultPartitionID(), partitionInfo.getIntermediateDataSetID(), executionAttemptID); } 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 6e9b864affc8b..a9689ce9f4e00 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 @@ -247,7 +247,11 @@ public static TaskManagerServices fromConfiguration( final IOManager ioManager = new IOManagerAsync(taskManagerServicesConfiguration.getTmpDirPaths()); final NetworkEnvironment network = NetworkEnvironment.create( - taskManagerServicesConfiguration.getNetworkConfig(), taskEventDispatcher, taskManagerMetricGroup, ioManager); + resourceID, + taskManagerServicesConfiguration.getNetworkConfig(), + taskEventDispatcher, + taskManagerMetricGroup, + ioManager); int dataPort = network.start(); final KvStateService kvStateService = KvStateService.fromConfiguration(taskManagerServicesConfiguration); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptorTest.java deleted file mode 100644 index f582978c9122d..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptorTest.java +++ /dev/null @@ -1,206 +0,0 @@ -/* - * 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.deployment; - -import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.executiongraph.Execution; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.executiongraph.ExecutionEdge; -import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.executiongraph.IntermediateResult; -import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; -import org.apache.flink.runtime.io.network.ConnectionID; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; -import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; -import org.apache.flink.runtime.jobmaster.LogicalSlot; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; -import org.apache.flink.util.TestLogger; - -import org.junit.Test; - -import java.net.InetAddress; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -/** - * Tests for the {@link InputChannelDeploymentDescriptor}. - */ -public class InputChannelDeploymentDescriptorTest extends TestLogger { - - /** - * Tests the deployment descriptors for local, remote, and unknown partition - * locations (with lazy deployment allowed and all execution states for the - * producers). - */ - @Test - public void testMixedLocalRemoteUnknownDeployment() { - boolean allowLazyDeployment = true; - - ResourceID consumerResourceId = ResourceID.generate(); - ExecutionVertex consumer = mockExecutionVertex(ExecutionState.DEPLOYING, consumerResourceId); - - // Local and remote channel are only allowed for certain execution - // states. - for (ExecutionState state : ExecutionState.values()) { - // Local partition - ExecutionVertex localProducer = mockExecutionVertex(state, consumerResourceId); - IntermediateResultPartition localPartition = mockPartition(localProducer); - ResultPartitionID localPartitionId = new ResultPartitionID(localPartition.getPartitionId(), localProducer.getCurrentExecutionAttempt().getAttemptId()); - ExecutionEdge localEdge = new ExecutionEdge(localPartition, consumer, 0); - - // Remote partition - ExecutionVertex remoteProducer = mockExecutionVertex(state, ResourceID.generate()); // new resource ID - IntermediateResultPartition remotePartition = mockPartition(remoteProducer); - ResultPartitionID remotePartitionId = new ResultPartitionID(remotePartition.getPartitionId(), remoteProducer.getCurrentExecutionAttempt().getAttemptId()); - ConnectionID remoteConnectionId = new ConnectionID(remoteProducer.getCurrentAssignedResource().getTaskManagerLocation(), 0); - ExecutionEdge remoteEdge = new ExecutionEdge(remotePartition, consumer, 1); - - // Unknown partition - ExecutionVertex unknownProducer = mockExecutionVertex(state, null); // no assigned resource - IntermediateResultPartition unknownPartition = mockPartition(unknownProducer); - ResultPartitionID unknownPartitionId = new ResultPartitionID(unknownPartition.getPartitionId(), unknownProducer.getCurrentExecutionAttempt().getAttemptId()); - ExecutionEdge unknownEdge = new ExecutionEdge(unknownPartition, consumer, 2); - - List desc = InputChannelDeploymentDescriptor.fromEdges( - Arrays.asList(localEdge, remoteEdge, unknownEdge), - allowLazyDeployment); - - assertEquals(3, desc.size()); - - // These states are allowed - if (state == ExecutionState.RUNNING || state == ExecutionState.FINISHED || - state == ExecutionState.SCHEDULED || state == ExecutionState.DEPLOYING) { - - // Create local or remote channels - assertEquals(localPartitionId, desc.get(0).getConsumedPartitionId()); - assertTrue(desc.get(0).getConsumedPartitionLocation().isLocal()); - assertNull(desc.get(0).getConsumedPartitionLocation().getConnectionId()); - - assertEquals(remotePartitionId, desc.get(1).getConsumedPartitionId()); - assertTrue(desc.get(1).getConsumedPartitionLocation().isRemote()); - assertEquals(remoteConnectionId, desc.get(1).getConsumedPartitionLocation().getConnectionId()); - } else { - // Unknown (lazy deployment allowed) - assertEquals(localPartitionId, desc.get(0).getConsumedPartitionId()); - assertTrue(desc.get(0).getConsumedPartitionLocation().isUnknown()); - assertNull(desc.get(0).getConsumedPartitionLocation().getConnectionId()); - - assertEquals(remotePartitionId, desc.get(1).getConsumedPartitionId()); - assertTrue(desc.get(1).getConsumedPartitionLocation().isUnknown()); - assertNull(desc.get(1).getConsumedPartitionLocation().getConnectionId()); - } - - assertEquals(unknownPartitionId, desc.get(2).getConsumedPartitionId()); - assertTrue(desc.get(2).getConsumedPartitionLocation().isUnknown()); - assertNull(desc.get(2).getConsumedPartitionLocation().getConnectionId()); - } - } - - @Test - public void testUnknownChannelWithoutLazyDeploymentThrows() { - ResourceID consumerResourceId = ResourceID.generate(); - ExecutionVertex consumer = mockExecutionVertex(ExecutionState.DEPLOYING, consumerResourceId); - - // Unknown partition - ExecutionVertex unknownProducer = mockExecutionVertex(ExecutionState.CREATED, null); // no assigned resource - IntermediateResultPartition unknownPartition = mockPartition(unknownProducer); - ResultPartitionID unknownPartitionId = new ResultPartitionID(unknownPartition.getPartitionId(), unknownProducer.getCurrentExecutionAttempt().getAttemptId()); - ExecutionEdge unknownEdge = new ExecutionEdge(unknownPartition, consumer, 2); - - // This should work if lazy deployment is allowed - boolean allowLazyDeployment = true; - - List desc = InputChannelDeploymentDescriptor.fromEdges( - Collections.singletonList(unknownEdge), - allowLazyDeployment); - - assertEquals(1, desc.size()); - - assertEquals(unknownPartitionId, desc.get(0).getConsumedPartitionId()); - assertTrue(desc.get(0).getConsumedPartitionLocation().isUnknown()); - assertNull(desc.get(0).getConsumedPartitionLocation().getConnectionId()); - - try { - // Fail if lazy deployment is *not* allowed - allowLazyDeployment = false; - - InputChannelDeploymentDescriptor.fromEdges( - Collections.singletonList(unknownEdge), - allowLazyDeployment); - - fail("Did not throw expected IllegalStateException"); - } catch (IllegalStateException expected) {} - } - - // ------------------------------------------------------------------------ - - private static LogicalSlot mockSlot(ResourceID resourceId) { - LogicalSlot slot = mock(LogicalSlot.class); - when(slot.getTaskManagerLocation()).thenReturn(new TaskManagerLocation(resourceId, InetAddress.getLoopbackAddress(), 5000)); - - return slot; - } - - private static ExecutionVertex mockExecutionVertex(ExecutionState state, ResourceID resourceId) { - ExecutionVertex vertex = mock(ExecutionVertex.class); - - Execution exec = mock(Execution.class); - when(exec.getState()).thenReturn(state); - when(exec.getAttemptId()).thenReturn(new ExecutionAttemptID()); - - if (resourceId != null) { - LogicalSlot slot = mockSlot(resourceId); - when(exec.getAssignedResource()).thenReturn(slot); - when(vertex.getCurrentAssignedResource()).thenReturn(slot); - } else { - when(exec.getAssignedResource()).thenReturn(null); // no resource - when(vertex.getCurrentAssignedResource()).thenReturn(null); - } - - when(vertex.getCurrentExecutionAttempt()).thenReturn(exec); - - return vertex; - } - - private static IntermediateResultPartition mockPartition(ExecutionVertex producer) { - IntermediateResultPartition partition = mock(IntermediateResultPartition.class); - when(partition.getResultType()).thenReturn(ResultPartitionType.PIPELINED); - when(partition.isConsumable()).thenReturn(true); - - IntermediateResult result = mock(IntermediateResult.class); - when(result.getConnectionIndex()).thenReturn(0); - - when(partition.getIntermediateResult()).thenReturn(result); - when(partition.getPartitionId()).thenReturn(new IntermediateResultPartitionID()); - - when(partition.getProducer()).thenReturn(producer); - - return partition; - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptorTest.java index f78d143506030..37b4a7f7adbf6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptorTest.java @@ -19,47 +19,108 @@ package org.apache.flink.runtime.deployment; import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.io.network.ConnectionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; +import org.apache.flink.runtime.shuffle.PartitionDescriptor; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; +import org.apache.flink.runtime.shuffle.UnknownShuffleDescriptor; +import org.apache.flink.util.TestLogger; import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import java.io.IOException; +import java.net.InetSocketAddress; + +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; /** * Tests for the {@link ResultPartitionDeploymentDescriptor}. */ -public class ResultPartitionDeploymentDescriptorTest { +public class ResultPartitionDeploymentDescriptorTest extends TestLogger { + private static final IntermediateDataSetID resultId = new IntermediateDataSetID(); + + private static final IntermediateResultPartitionID partitionId = new IntermediateResultPartitionID(); + private static final ExecutionAttemptID producerExecutionId = new ExecutionAttemptID(); + + private static final ResultPartitionType partitionType = ResultPartitionType.PIPELINED; + private static final int numberOfSubpartitions = 24; + private static final int connectionIndex = 10; + + private static final PartitionDescriptor partitionDescriptor = new PartitionDescriptor( + resultId, + partitionId, + partitionType, + numberOfSubpartitions, + connectionIndex); + + private static final ResultPartitionID resultPartitionID = new ResultPartitionID(partitionId, producerExecutionId); + + private static final ResourceID producerLocation = new ResourceID("producerLocation"); + private static final InetSocketAddress address = new InetSocketAddress("localhost", 10000); + private static final ConnectionID connectionID = new ConnectionID(address, connectionIndex); + + /** + * Tests simple de/serialization with {@link UnknownShuffleDescriptor}. + */ + @Test + public void testSerializationWithUnknownShuffleDescriptor() throws Exception { + ShuffleDescriptor shuffleDescriptor = new UnknownShuffleDescriptor(resultPartitionID); + + ResultPartitionDeploymentDescriptor copy = + createCopyAndVerifyResultPartitionDeploymentDescriptor(shuffleDescriptor); + + ShuffleDescriptor shuffleDescriptorCopy = copy.getShuffleDescriptor(); + assertThat(shuffleDescriptorCopy, instanceOf(UnknownShuffleDescriptor.class)); + assertThat(shuffleDescriptorCopy.getResultPartitionID(), is(resultPartitionID)); + assertThat(shuffleDescriptorCopy.isUnknown(), is(true)); + } /** - * Tests simple de/serialization. + * Tests simple de/serialization with {@link NettyShuffleDescriptor}. */ @Test - public void testSerialization() throws Exception { - // Expected values - IntermediateDataSetID resultId = new IntermediateDataSetID(); - IntermediateResultPartitionID partitionId = new IntermediateResultPartitionID(); - ResultPartitionType partitionType = ResultPartitionType.PIPELINED; - int numberOfSubpartitions = 24; - - ResultPartitionDeploymentDescriptor orig = - new ResultPartitionDeploymentDescriptor( - resultId, - partitionId, - partitionType, - numberOfSubpartitions, - numberOfSubpartitions, - true); + public void testSerializationWithNettyShuffleDescriptor() throws Exception { + ShuffleDescriptor shuffleDescriptor = new NettyShuffleDescriptor( + producerLocation, + new NettyShuffleDescriptor.NetworkPartitionConnectionInfo(connectionID), + resultPartitionID); ResultPartitionDeploymentDescriptor copy = - CommonTestUtils.createCopySerializable(orig); + createCopyAndVerifyResultPartitionDeploymentDescriptor(shuffleDescriptor); + + assertThat(copy.getShuffleDescriptor(), instanceOf(NettyShuffleDescriptor.class)); + NettyShuffleDescriptor shuffleDescriptorCopy = (NettyShuffleDescriptor) copy.getShuffleDescriptor(); + assertThat(shuffleDescriptorCopy.getResultPartitionID(), is(resultPartitionID)); + assertThat(shuffleDescriptorCopy.isUnknown(), is(false)); + assertThat(shuffleDescriptorCopy.isLocalTo(producerLocation), is(true)); + assertThat(shuffleDescriptorCopy.getConnectionId(), is(connectionID)); + } + + private static ResultPartitionDeploymentDescriptor createCopyAndVerifyResultPartitionDeploymentDescriptor( + ShuffleDescriptor shuffleDescriptor) throws IOException { + ResultPartitionDeploymentDescriptor orig = new ResultPartitionDeploymentDescriptor( + partitionDescriptor, + shuffleDescriptor, + numberOfSubpartitions, + true); + ResultPartitionDeploymentDescriptor copy = CommonTestUtils.createCopySerializable(orig); + verifyResultPartitionDeploymentDescriptorCopy(copy); + return copy; + } - assertEquals(resultId, copy.getResultId()); - assertEquals(partitionId, copy.getPartitionId()); - assertEquals(partitionType, copy.getPartitionType()); - assertEquals(numberOfSubpartitions, copy.getNumberOfSubpartitions()); - assertTrue(copy.sendScheduleOrUpdateConsumersMessage()); + private static void verifyResultPartitionDeploymentDescriptorCopy(ResultPartitionDeploymentDescriptor copy) { + assertThat(copy.getResultId(), is(resultId)); + assertThat(copy.getPartitionId(), is(partitionId)); + assertThat(copy.getPartitionType(), is(partitionType)); + assertThat(copy.getNumberOfSubpartitions(), is(numberOfSubpartitions)); + assertThat(copy.sendScheduleOrUpdateConsumersMessage(), is(true)); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ShuffleDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ShuffleDescriptorTest.java new file mode 100644 index 0000000000000..f1c58cba3d46c --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ShuffleDescriptorTest.java @@ -0,0 +1,185 @@ +/* + * 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.deployment; + +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; +import org.apache.flink.runtime.shuffle.NettyShuffleMaster; +import org.apache.flink.runtime.shuffle.PartitionDescriptor; +import org.apache.flink.runtime.shuffle.ProducerDescriptor; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; +import org.apache.flink.runtime.shuffle.UnknownShuffleDescriptor; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import javax.annotation.Nullable; + +import java.util.concurrent.ExecutionException; + +import static org.apache.flink.runtime.io.network.partition.consumer.InputChannelBuilder.STUB_CONNECTION_ID; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +/** + * Tests for the {@link ShuffleDescriptor}. + */ +public class ShuffleDescriptorTest extends TestLogger { + + /** + * Tests the deployment descriptors for local, remote, and unknown partition + * locations (with lazy deployment allowed and all execution states for the + * producers). + */ + @Test + public void testMixedLocalRemoteUnknownDeployment() throws Exception { + ResourceID consumerResourceID = ResourceID.generate(); + + // Local and remote channel are only allowed for certain execution + // states. + for (ExecutionState state : ExecutionState.values()) { + ResultPartitionID localPartitionId = new ResultPartitionID(); + ResultPartitionDeploymentDescriptor localPartition = + createResultPartitionDeploymentDescriptor(localPartitionId, consumerResourceID); + + ResultPartitionID remotePartitionId = new ResultPartitionID(); + ResultPartitionDeploymentDescriptor remotePartition = + createResultPartitionDeploymentDescriptor(remotePartitionId, ResourceID.generate()); + + ResultPartitionID unknownPartitionId = new ResultPartitionID(); + + ShuffleDescriptor localShuffleDescriptor = + getConsumedPartitionShuffleDescriptor(localPartitionId, state, localPartition, true); + ShuffleDescriptor remoteShuffleDescriptor = + getConsumedPartitionShuffleDescriptor(remotePartitionId, state, remotePartition, true); + ShuffleDescriptor unknownShuffleDescriptor = + getConsumedPartitionShuffleDescriptor(unknownPartitionId, state, null, true); + + // These states are allowed + if (state == ExecutionState.RUNNING || + state == ExecutionState.FINISHED || + state == ExecutionState.SCHEDULED || + state == ExecutionState.DEPLOYING) { + NettyShuffleDescriptor nettyShuffleDescriptor; + + // Create local or remote channels + verifyShuffleDescriptor(localShuffleDescriptor, NettyShuffleDescriptor.class, false, localPartitionId); + nettyShuffleDescriptor = (NettyShuffleDescriptor) localShuffleDescriptor; + assertThat(nettyShuffleDescriptor.isLocalTo(consumerResourceID), is(true)); + + verifyShuffleDescriptor(remoteShuffleDescriptor, NettyShuffleDescriptor.class, false, remotePartitionId); + nettyShuffleDescriptor = (NettyShuffleDescriptor) remoteShuffleDescriptor; + assertThat(nettyShuffleDescriptor.isLocalTo(consumerResourceID), is(false)); + assertThat(nettyShuffleDescriptor.getConnectionId(), is(STUB_CONNECTION_ID)); + } else { + // Unknown (lazy deployment allowed) + verifyShuffleDescriptor(localShuffleDescriptor, UnknownShuffleDescriptor.class, true, localPartitionId); + verifyShuffleDescriptor(remoteShuffleDescriptor, UnknownShuffleDescriptor.class, true, remotePartitionId); + } + + verifyShuffleDescriptor(unknownShuffleDescriptor, UnknownShuffleDescriptor.class, true, unknownPartitionId); + } + } + + private static void verifyShuffleDescriptor( + ShuffleDescriptor descriptor, + Class cl, + boolean unknown, + ResultPartitionID partitionID) { + assertThat(descriptor, instanceOf(cl)); + assertThat(descriptor.isUnknown(), is(unknown)); + assertThat(descriptor.getResultPartitionID(), is(partitionID)); + } + + @Test + public void testUnknownDescriptorWithOrWithoutLazyDeployment() { + ResultPartitionID unknownPartitionId = new ResultPartitionID(); + + // This should work if lazy deployment is allowed + ShuffleDescriptor unknownSdd = getConsumedPartitionShuffleDescriptor( + unknownPartitionId, + ExecutionState.CREATED, + null, + true); + + assertThat(unknownSdd, instanceOf(UnknownShuffleDescriptor.class)); + assertThat(unknownSdd.isUnknown(), is(true)); + assertThat(unknownSdd.getResultPartitionID(), is(unknownPartitionId)); + + try { + // Fail if lazy deployment is *not* allowed + getConsumedPartitionShuffleDescriptor( + unknownPartitionId, + ExecutionState.CREATED, + null, + false); + fail("Did not throw expected ExecutionGraphException"); + } catch (IllegalStateException ignored) { + } + } + + private static ShuffleDescriptor getConsumedPartitionShuffleDescriptor( + ResultPartitionID id, + ExecutionState state, + @Nullable ResultPartitionDeploymentDescriptor producedPartition, + boolean allowLazyDeployment) { + ShuffleDescriptor shuffleDescriptor = TaskDeploymentDescriptorFactory.getConsumedPartitionShuffleDescriptor( + id, + ResultPartitionType.PIPELINED, + true, + state, + allowLazyDeployment, + producedPartition); + assertThat(shuffleDescriptor, is(notNullValue())); + assertThat(shuffleDescriptor.getResultPartitionID(), is(id)); + return shuffleDescriptor; + } + + private static ResultPartitionDeploymentDescriptor createResultPartitionDeploymentDescriptor( + ResultPartitionID id, + ResourceID location) throws ExecutionException, InterruptedException { + ProducerDescriptor producerDescriptor = new ProducerDescriptor( + location, + id.getProducerId(), + STUB_CONNECTION_ID.getAddress().getAddress(), + STUB_CONNECTION_ID.getAddress().getPort()); + PartitionDescriptor partitionDescriptor = new PartitionDescriptor( + new IntermediateDataSetID(), + id.getPartitionId(), + ResultPartitionType.PIPELINED, + 1, + 0); + ShuffleDescriptor shuffleDescriptor = + NettyShuffleMaster.INSTANCE.registerPartitionWithProducer( + partitionDescriptor, + producerDescriptor).get(); + return new ResultPartitionDeploymentDescriptor( + partitionDescriptor, + shuffleDescriptor, + 1, + true); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java index df80d3e6b4f0f..fac040abacbdd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java @@ -207,6 +207,7 @@ public void testBuildDeploymentDescriptor() { assertEquals(ExecutionState.CREATED, vertex.getExecutionState()); + vertex.getCurrentExecutionAttempt().registerProducedPartitions(slot.getTaskManagerLocation()).get(); vertex.deployToSlot(slot); assertEquals(ExecutionState.DEPLOYING, vertex.getExecutionState()); @@ -239,7 +240,7 @@ public void testBuildDeploymentDescriptor() { assertEquals(10, iteratorProducedPartitions.next().getNumberOfSubpartitions()); assertEquals(10, iteratorProducedPartitions.next().getNumberOfSubpartitions()); - assertEquals(10, iteratorConsumedPartitions.next().getInputChannelDeploymentDescriptors().length); + assertEquals(10, iteratorConsumedPartitions.next().getShuffleDescriptors().length); } catch (Exception e) { e.printStackTrace(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java index 4ee79e4819ec3..892552c9708ea 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java @@ -114,7 +114,7 @@ public void testSlotReleaseOnFailedResourceAssignment() throws Exception { final LogicalSlot otherSlot = new TestingLogicalSlot(); - CompletableFuture allocationFuture = execution.allocateAndAssignSlotForExecution( + CompletableFuture allocationFuture = execution.allocateResourcesForExecution( slotProvider, false, LocationPreferenceConstraint.ALL, @@ -166,7 +166,7 @@ public void testSlotReleaseOnExecutionCancellationInScheduled() throws Exception final Execution execution = executionJobVertex.getTaskVertices()[0].getCurrentExecutionAttempt(); - CompletableFuture allocationFuture = execution.allocateAndAssignSlotForExecution( + CompletableFuture allocationFuture = execution.allocateResourcesForExecution( slotProvider, false, LocationPreferenceConstraint.ALL, @@ -216,7 +216,7 @@ public void testSlotReleaseOnExecutionCancellationInRunning() throws Exception { final Execution execution = executionJobVertex.getTaskVertices()[0].getCurrentExecutionAttempt(); - CompletableFuture allocationFuture = execution.allocateAndAssignSlotForExecution( + CompletableFuture allocationFuture = execution.allocateResourcesForExecution( slotProvider, false, LocationPreferenceConstraint.ALL, @@ -268,7 +268,7 @@ public void testSlotAllocationCancellationWhenExecutionCancelled() throws Except final Execution currentExecutionAttempt = executionJobVertex.getTaskVertices()[0].getCurrentExecutionAttempt(); - final CompletableFuture allocationFuture = currentExecutionAttempt.allocateAndAssignSlotForExecution( + final CompletableFuture allocationFuture = currentExecutionAttempt.allocateResourcesForExecution( slotProvider, false, LocationPreferenceConstraint.ALL, @@ -417,7 +417,7 @@ public void testTaskRestoreStateIsNulledAfterDeployment() throws Exception { slotProvider, new NoRestartStrategy(), jobVertex); - + ExecutionJobVertex executionJobVertex = executionGraph.getJobVertex(jobVertexId); ExecutionVertex executionVertex = executionJobVertex.getTaskVertices()[0]; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java index e5dd2a1036703..a0442651fc84a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java @@ -21,8 +21,10 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptorFactory; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; @@ -30,14 +32,15 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.ScheduleMode; import org.apache.flink.runtime.jobmaster.LogicalSlot; -import org.apache.flink.runtime.jobmaster.SlotContext; import org.apache.flink.runtime.jobmaster.TestingLogicalSlot; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testutils.DirectScheduledExecutorService; import org.apache.flink.util.TestLogger; import org.junit.Test; +import java.net.InetAddress; import java.util.Collection; import java.util.concurrent.CompletableFuture; @@ -288,24 +291,28 @@ public void testTddProducedPartitionsLazyScheduling() throws Exception { IntermediateResult result = new IntermediateResult(new IntermediateDataSetID(), jobVertex, 1, ResultPartitionType.PIPELINED); + ExecutionAttemptID attemptID = new ExecutionAttemptID(); ExecutionVertex vertex = new ExecutionVertex(jobVertex, 0, new IntermediateResult[]{result}, Time.minutes(1)); + TaskDeploymentDescriptorFactory tddFactory = + TaskDeploymentDescriptorFactory.fromExecutionVertex(vertex, 1); ExecutionEdge mockEdge = createMockExecutionEdge(1); result.getPartitions()[0].addConsumerGroup(); result.getPartitions()[0].addConsumer(mockEdge, 0); - SlotContext slotContext = mock(SlotContext.class); - when(slotContext.getAllocationId()).thenReturn(new AllocationID()); - - LogicalSlot slot = mock(LogicalSlot.class); - when(slot.getAllocationId()).thenReturn(new AllocationID()); + TaskManagerLocation location = + new TaskManagerLocation(ResourceID.generate(), InetAddress.getLoopbackAddress(), 1); for (ScheduleMode mode : ScheduleMode.values()) { vertex.getExecutionGraph().setScheduleMode(mode); - TaskDeploymentDescriptor tdd = vertex.createDeploymentDescriptor(new ExecutionAttemptID(), slot, null, 1); + TaskDeploymentDescriptor tdd = tddFactory.createDeploymentDescriptor( + new AllocationID(), + 0, + null, + Execution.registerProducedPartitions(vertex, location, attemptID).get().values()); Collection producedPartitions = tdd.getProducedPartitions(); @@ -315,8 +322,6 @@ public void testTddProducedPartitionsLazyScheduling() throws Exception { } } - - private ExecutionEdge createMockExecutionEdge(int maxParallelism) { ExecutionVertex targetVertex = mock(ExecutionVertex.class); ExecutionJobVertex targetJobVertex = mock(ExecutionJobVertex.class); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FinalizeOnMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FinalizeOnMasterTest.java index a2b1c58c6c52b..df882ffc8a755 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FinalizeOnMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FinalizeOnMasterTest.java @@ -54,6 +54,7 @@ public void testFinalizeIsCalledUponSuccess() throws Exception { vertex2.setParallelism(2); final ExecutionGraph eg = createSimpleTestGraph(jid, vertex1, vertex2); + eg.start(TestingComponentMainThreadExecutorServiceAdapter.forMainThread()); eg.scheduleForExecution(); assertEquals(JobStatus.RUNNING, eg.getState()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentBuilder.java index e510be4100acc..1950638d18096 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentBuilder.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.io.network; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.clusterframework.types.ResourceID; 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.netty.NettyConfig; @@ -46,6 +47,8 @@ public class NetworkEnvironmentBuilder { private boolean isNetworkDetailedMetrics = false; + private ResourceID taskManagerLocation = ResourceID.generate(); + private NettyConfig nettyConfig; private TaskEventDispatcher taskEventDispatcher = new TaskEventDispatcher(); @@ -54,6 +57,11 @@ public class NetworkEnvironmentBuilder { private IOManager ioManager = new IOManagerAsync(); + public NetworkEnvironmentBuilder setTaskManagerLocation(ResourceID taskManagerLocation) { + this.taskManagerLocation = taskManagerLocation; + return this; + } + public NetworkEnvironmentBuilder setNumNetworkBuffers(int numNetworkBuffers) { this.numNetworkBuffers = numNetworkBuffers; return this; @@ -111,6 +119,7 @@ public NetworkEnvironmentBuilder setIOManager(IOManager ioManager) { public NetworkEnvironment build() { return NetworkEnvironment.create( + taskManagerLocation, new NetworkEnvironmentConfiguration( numNetworkBuffers, networkBufferSize, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java index 6166e20a716a8..36a98fffea344 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java @@ -27,10 +27,12 @@ import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.shuffle.PartitionDescriptor; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.taskmanager.ConsumableNotifyingResultPartitionWriterDecorator; import org.apache.flink.runtime.taskmanager.NoOpTaskActions; import org.apache.flink.runtime.taskmanager.TaskActions; +import org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder; import org.junit.Assert; import org.junit.Test; @@ -274,11 +276,16 @@ private ResultPartitionWriter createConsumableNotifyingResultPartitionWriter( } private ResultPartitionDeploymentDescriptor createPartitionDeploymentDescriptor(ResultPartitionType partitionType) { - return new ResultPartitionDeploymentDescriptor( + ShuffleDescriptor shuffleDescriptor = NettyShuffleDescriptorBuilder.newBuilder().buildLocal(); + PartitionDescriptor partitionDescriptor = new PartitionDescriptor( new IntermediateDataSetID(), - new IntermediateResultPartitionID(), + shuffleDescriptor.getResultPartitionID().getPartitionId(), partitionType, 1, + 0); + return new ResultPartitionDeploymentDescriptor( + partitionDescriptor, + shuffleDescriptor, 1, true); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java index 4153e8c099ee1..2e6fcbc6d8f4f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java @@ -36,7 +36,7 @@ * Builder for various {@link InputChannel} types. */ public class InputChannelBuilder { - static final ConnectionID STUB_CONNECTION_ID = + public static final ConnectionID STUB_CONNECTION_ID = new ConnectionID(new InetSocketAddress("localhost", 5000), 0); private int channelIndex = 0; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java index 7f7a3115945ba..11c6a51bc3cae 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java @@ -20,9 +20,8 @@ import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; -import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; -import org.apache.flink.runtime.deployment.ResultPartitionLocation; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.network.NetworkEnvironment; @@ -43,6 +42,9 @@ import org.apache.flink.runtime.io.network.util.TestTaskEvent; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; +import org.apache.flink.runtime.shuffle.UnknownShuffleDescriptor; import org.junit.Test; @@ -55,6 +57,7 @@ import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createLocalInputChannel; import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createSingleInputGate; +import static org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder.createRemoteWithIdAndLocation; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; @@ -225,7 +228,9 @@ public void testBackwardsEventWithUninitializedChannel() throws Exception { verify(taskEventDispatcher, times(1)).publish(any(ResultPartitionID.class), any(TaskEvent.class)); // After the update, the pending event should be send to local channel - inputGate.updateInputChannel(new InputChannelDeploymentDescriptor(new ResultPartitionID(unknownPartitionId.getPartitionId(), unknownPartitionId.getProducerId()), ResultPartitionLocation.createLocal())); + + ResourceID location = ResourceID.generate(); + inputGate.updateInputChannel(location, createRemoteWithIdAndLocation(unknownPartitionId.getPartitionId(), location)); verify(partitionManager, times(2)).createSubpartitionView(any(ResultPartitionID.class), anyInt(), any(BufferAvailabilityListener.class)); verify(taskEventDispatcher, times(2)).publish(any(ResultPartitionID.class), any(TaskEvent.class)); @@ -248,9 +253,9 @@ public void testUpdateChannelBeforeRequest() throws Exception { .buildUnknownAndSetToGate(inputGate); // Update to a local channel and verify that no request is triggered - inputGate.updateInputChannel(new InputChannelDeploymentDescriptor( - unknown.partitionId, - ResultPartitionLocation.createLocal())); + ResultPartitionID resultPartitionID = unknown.getPartitionId(); + ResourceID location = ResourceID.generate(); + inputGate.updateInputChannel(location, createRemoteWithIdAndLocation(resultPartitionID.getPartitionId(), location)); verify(partitionManager, never()).createSubpartitionView( any(ResultPartitionID.class), anyInt(), any(BufferAvailabilityListener.class)); @@ -317,29 +322,26 @@ public void run() { */ @Test public void testRequestBackoffConfiguration() throws Exception { - ResultPartitionID[] partitionIds = new ResultPartitionID[] { - new ResultPartitionID(), - new ResultPartitionID(), - new ResultPartitionID() + IntermediateResultPartitionID[] partitionIds = new IntermediateResultPartitionID[] { + new IntermediateResultPartitionID(), + new IntermediateResultPartitionID(), + new IntermediateResultPartitionID() }; - InputChannelDeploymentDescriptor[] channelDescs = new InputChannelDeploymentDescriptor[]{ + ResourceID localLocation = ResourceID.generate(); + ShuffleDescriptor[] channelDescs = new ShuffleDescriptor[]{ // Local - new InputChannelDeploymentDescriptor( - partitionIds[0], - ResultPartitionLocation.createLocal()), + createRemoteWithIdAndLocation(partitionIds[0], localLocation), // Remote - new InputChannelDeploymentDescriptor( - partitionIds[1], - ResultPartitionLocation.createRemote(InputChannelBuilder.STUB_CONNECTION_ID)), + createRemoteWithIdAndLocation(partitionIds[1], ResourceID.generate()), // Unknown - new InputChannelDeploymentDescriptor( - partitionIds[2], - ResultPartitionLocation.createUnknown())}; + new UnknownShuffleDescriptor(new ResultPartitionID(partitionIds[2], new ExecutionAttemptID()))}; - InputGateDeploymentDescriptor gateDesc = - new InputGateDeploymentDescriptor(new IntermediateDataSetID(), - ResultPartitionType.PIPELINED, 0, channelDescs); + InputGateDeploymentDescriptor gateDesc = new InputGateDeploymentDescriptor( + new IntermediateDataSetID(), + ResultPartitionType.PIPELINED, + 0, + channelDescs); int initialBackoff = 137; int maxBackoff = 1001; @@ -351,6 +353,7 @@ public void testRequestBackoffConfiguration() throws Exception { .build(); SingleInputGate gate = new SingleInputGateFactory( + localLocation, netEnv.getConfiguration(), netEnv.getConnectionManager(), netEnv.getResultPartitionManager(), @@ -368,13 +371,13 @@ public void testRequestBackoffConfiguration() throws Exception { Map channelMap = gate.getInputChannels(); assertEquals(3, channelMap.size()); - InputChannel localChannel = channelMap.get(partitionIds[0].getPartitionId()); + InputChannel localChannel = channelMap.get(partitionIds[0]); assertEquals(LocalInputChannel.class, localChannel.getClass()); - InputChannel remoteChannel = channelMap.get(partitionIds[1].getPartitionId()); + InputChannel remoteChannel = channelMap.get(partitionIds[1]); assertEquals(RemoteInputChannel.class, remoteChannel.getClass()); - InputChannel unknownChannel = channelMap.get(partitionIds[2].getPartitionId()); + InputChannel unknownChannel = channelMap.get(partitionIds[2]); assertEquals(UnknownInputChannel.class, unknownChannel.getClass()); InputChannel[] channels = @@ -465,9 +468,9 @@ public void testRequestBuffersWithUnknownInputChannel() throws Exception { } // Trigger updates to remote input channel from unknown input channel - inputGate.updateInputChannel(new InputChannelDeploymentDescriptor( - resultPartitionId, - ResultPartitionLocation.createRemote(InputChannelBuilder.STUB_CONNECTION_ID))); + inputGate.updateInputChannel( + ResourceID.generate(), + createRemoteWithIdAndLocation(resultPartitionId.getPartitionId(), ResourceID.generate())); if (enableCreditBasedFlowControl) { RemoteInputChannel remote = (RemoteInputChannel) inputGate.getInputChannels() @@ -511,10 +514,12 @@ public void testUpdateUnknownInputChannel() throws Exception { assertThat(inputGate.getInputChannels().get(localResultPartitionId.getPartitionId()), is(instanceOf((UnknownInputChannel.class)))); + ResourceID localLocation = ResourceID.generate(); + // Trigger updates to remote input channel from unknown input channel - inputGate.updateInputChannel(new InputChannelDeploymentDescriptor( - remoteResultPartitionId, - ResultPartitionLocation.createRemote(InputChannelBuilder.STUB_CONNECTION_ID))); + inputGate.updateInputChannel( + localLocation, + createRemoteWithIdAndLocation(remoteResultPartitionId.getPartitionId(), ResourceID.generate())); assertThat(inputGate.getInputChannels().get(remoteResultPartitionId.getPartitionId()), is(instanceOf((RemoteInputChannel.class)))); @@ -522,9 +527,9 @@ public void testUpdateUnknownInputChannel() throws Exception { is(instanceOf((UnknownInputChannel.class)))); // Trigger updates to local input channel from unknown input channel - inputGate.updateInputChannel(new InputChannelDeploymentDescriptor( - localResultPartitionId, - ResultPartitionLocation.createLocal())); + inputGate.updateInputChannel( + localLocation, + createRemoteWithIdAndLocation(localResultPartitionId.getPartitionId(), localLocation)); assertThat(inputGate.getInputChannels().get(remoteResultPartitionId.getPartitionId()), is(instanceOf((RemoteInputChannel.class)))); @@ -585,11 +590,9 @@ private static Map createInputGateWithLocalChannel NetworkEnvironment network, int numberOfGates, @SuppressWarnings("SameParameterValue") int numberOfLocalChannels) { - InputChannelDeploymentDescriptor[] channelDescs = new InputChannelDeploymentDescriptor[numberOfLocalChannels]; + ShuffleDescriptor[] channelDescs = new NettyShuffleDescriptor[numberOfLocalChannels]; for (int i = 0; i < numberOfLocalChannels; i++) { - channelDescs[i] = new InputChannelDeploymentDescriptor( - new ResultPartitionID(), - ResultPartitionLocation.createLocal()); + channelDescs[i] = createRemoteWithIdAndLocation(new IntermediateResultPartitionID(), ResourceID.generate()); } InputGateDeploymentDescriptor[] gateDescs = new InputGateDeploymentDescriptor[numberOfGates]; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java index 8a6f784d87863..8a21da9eca806 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java @@ -26,11 +26,10 @@ import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.blob.PermanentBlobKey; import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; -import org.apache.flink.runtime.deployment.ResultPartitionLocation; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -38,25 +37,26 @@ import org.apache.flink.runtime.executiongraph.JobInformation; import org.apache.flink.runtime.executiongraph.PartitionInfo; import org.apache.flink.runtime.executiongraph.TaskInformation; -import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.NetworkEnvironment; import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; -import org.apache.flink.runtime.jobmanager.Tasks; import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.jobmaster.TestingAbstractInvokables; import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway; import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.StackTraceSampleResponse; +import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; +import org.apache.flink.runtime.shuffle.PartitionDescriptor; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable; +import org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.NetUtils; import org.apache.flink.util.Preconditions; @@ -69,7 +69,6 @@ import org.mockito.Mockito; import java.io.IOException; -import java.net.InetSocketAddress; import java.net.URL; import java.util.Arrays; import java.util.Collection; @@ -77,6 +76,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; +import static org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder.createRemoteWithIdAndLocation; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.startsWith; import static org.hamcrest.Matchers.instanceOf; @@ -244,37 +244,14 @@ public void testGateChannelEdgeMismatch() throws Exception { @Test(timeout = 10000L) public void testRunJobWithForwardChannel() throws Exception { - final ExecutionAttemptID eid1 = new ExecutionAttemptID(); - final ExecutionAttemptID eid2 = new ExecutionAttemptID(); - - IntermediateResultPartitionID partitionId = new IntermediateResultPartitionID(); - - ResultPartitionDeploymentDescriptor task1ResultPartitionDescriptor = - new ResultPartitionDeploymentDescriptor(new IntermediateDataSetID(), partitionId, ResultPartitionType.PIPELINED, 1, - 1, true); + ResourceID producerLocation = ResourceID.generate(); + NettyShuffleDescriptor sdd = + createRemoteWithIdAndLocation(new IntermediateResultPartitionID(), producerLocation); - InputGateDeploymentDescriptor task2InputGateDescriptor = - new InputGateDeploymentDescriptor(new IntermediateDataSetID(), ResultPartitionType.PIPELINED, 0, - new InputChannelDeploymentDescriptor[] { - new InputChannelDeploymentDescriptor(new ResultPartitionID(partitionId, eid1), - ResultPartitionLocation.createLocal()) }); - - final TaskDeploymentDescriptor tdd1 = - createTestTaskDeploymentDescriptor( - "Sender", - eid1, - TestingAbstractInvokables.Sender.class, - 1, - Collections.singletonList(task1ResultPartitionDescriptor), - Collections.emptyList()); - final TaskDeploymentDescriptor tdd2 = - createTestTaskDeploymentDescriptor( - "Receiver", - eid2, - TestingAbstractInvokables.Receiver.class, - 1, - Collections.emptyList(), - Collections.singletonList(task2InputGateDescriptor)); + TaskDeploymentDescriptor tdd1 = createSender(sdd); + TaskDeploymentDescriptor tdd2 = createReceiver(sdd); + ExecutionAttemptID eid1 = tdd1.getExecutionAttemptId(); + ExecutionAttemptID eid2 = tdd2.getExecutionAttemptId(); final CompletableFuture task1RunningFuture = new CompletableFuture<>(); final CompletableFuture task2RunningFuture = new CompletableFuture<>(); @@ -291,6 +268,7 @@ public void testRunJobWithForwardChannel() throws Exception { try (TaskSubmissionTestEnvironment env = new TaskSubmissionTestEnvironment.Builder(jobId) + .setResourceID(producerLocation) .setSlotSize(2) .addTaskManagerActionListener(eid1, ExecutionState.RUNNING, task1RunningFuture) .addTaskManagerActionListener(eid2, ExecutionState.RUNNING, task2RunningFuture) @@ -326,34 +304,14 @@ public void testRunJobWithForwardChannel() throws Exception { */ @Test(timeout = 10000L) public void testCancellingDependentAndStateUpdateFails() throws Exception { - final ExecutionAttemptID eid1 = new ExecutionAttemptID(); - final ExecutionAttemptID eid2 = new ExecutionAttemptID(); - - IntermediateResultPartitionID partitionId = new IntermediateResultPartitionID(); - - ResultPartitionDeploymentDescriptor task1ResultPartitionDescriptor = - new ResultPartitionDeploymentDescriptor(new IntermediateDataSetID(), partitionId, ResultPartitionType.PIPELINED, 1, - 1, true); - - InputGateDeploymentDescriptor task2InputGateDescriptor = - new InputGateDeploymentDescriptor(new IntermediateDataSetID(), ResultPartitionType.PIPELINED, 0, - new InputChannelDeploymentDescriptor[] { - new InputChannelDeploymentDescriptor(new ResultPartitionID(partitionId, eid1), - ResultPartitionLocation.createLocal()) }); + ResourceID producerLocation = ResourceID.generate(); + NettyShuffleDescriptor sdd = + createRemoteWithIdAndLocation(new IntermediateResultPartitionID(), producerLocation); - final TaskDeploymentDescriptor tdd1 = - createTestTaskDeploymentDescriptor("Sender", - eid1, - TestingAbstractInvokables.Sender.class, 1, - Collections.singletonList(task1ResultPartitionDescriptor), - Collections.emptyList()); - final TaskDeploymentDescriptor tdd2 = - createTestTaskDeploymentDescriptor("Receiver", - eid2, - TestingAbstractInvokables.Receiver.class, - 1, - Collections.emptyList(), - Collections.singletonList(task2InputGateDescriptor)); + TaskDeploymentDescriptor tdd1 = createSender(sdd); + TaskDeploymentDescriptor tdd2 = createReceiver(sdd); + ExecutionAttemptID eid1 = tdd1.getExecutionAttemptId(); + ExecutionAttemptID eid2 = tdd2.getExecutionAttemptId(); final CompletableFuture task1RunningFuture = new CompletableFuture<>(); final CompletableFuture task2RunningFuture = new CompletableFuture<>(); @@ -376,6 +334,7 @@ public void testCancellingDependentAndStateUpdateFails() throws Exception { try (TaskSubmissionTestEnvironment env = new TaskSubmissionTestEnvironment.Builder(jobId) + .setResourceID(producerLocation) .setSlotSize(2) .addTaskManagerActionListener(eid1, ExecutionState.RUNNING, task1RunningFuture) .addTaskManagerActionListener(eid2, ExecutionState.RUNNING, task2RunningFuture) @@ -411,11 +370,6 @@ public void testCancellingDependentAndStateUpdateFails() throws Exception { */ @Test(timeout = 10000L) public void testRemotePartitionNotFound() throws Exception { - final ExecutionAttemptID eid = new ExecutionAttemptID(); - - final IntermediateDataSetID resultId = new IntermediateDataSetID(); - final ResultPartitionID partitionId = new ResultPartitionID(); - final int dataPort = NetUtils.getAvailablePort(); Configuration config = new Configuration(); config.setInteger(NetworkEnvironmentOptions.DATA_PORT, dataPort); @@ -423,23 +377,10 @@ public void testRemotePartitionNotFound() throws Exception { config.setInteger(NetworkEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX, 200); // Remote location (on the same TM though) for the partition - final ResultPartitionLocation loc = ResultPartitionLocation - .createRemote(new ConnectionID( - new InetSocketAddress("localhost", dataPort), 0)); - - final InputChannelDeploymentDescriptor[] inputChannelDeploymentDescriptors = - new InputChannelDeploymentDescriptor[] { - new InputChannelDeploymentDescriptor(partitionId, loc)}; - - final InputGateDeploymentDescriptor inputGateDeploymentDescriptor = - new InputGateDeploymentDescriptor(resultId, ResultPartitionType.PIPELINED, 0, inputChannelDeploymentDescriptors); - - final TaskDeploymentDescriptor tdd = - createTestTaskDeploymentDescriptor("Receiver", - eid, - Tasks.AgnosticReceiver.class, 1, - Collections.emptyList(), - Collections.singletonList(inputGateDeploymentDescriptor)); + NettyShuffleDescriptor sdd = + NettyShuffleDescriptorBuilder.newBuilder().setDataPort(dataPort).buildRemote(); + TaskDeploymentDescriptor tdd = createReceiver(sdd); + ExecutionAttemptID eid = tdd.getExecutionAttemptId(); final CompletableFuture taskRunningFuture = new CompletableFuture<>(); final CompletableFuture taskFailedFuture = new CompletableFuture<>(); @@ -476,11 +417,7 @@ public void testUpdateTaskInputPartitionsFailure() throws Exception { final CompletableFuture taskRunningFuture = new CompletableFuture<>(); final CompletableFuture taskFailedFuture = new CompletableFuture<>(); - final PartitionInfo partitionUpdate = new PartitionInfo( - new IntermediateDataSetID(), - new InputChannelDeploymentDescriptor(new ResultPartitionID(), ResultPartitionLocation.createLocal())); final NetworkEnvironment networkEnvironment = mock(NetworkEnvironment.class, Mockito.RETURNS_MOCKS); - doThrow(new IOException()).when(networkEnvironment).updatePartitionInfo(eid, partitionUpdate); try (TaskSubmissionTestEnvironment env = new TaskSubmissionTestEnvironment.Builder(jobId) @@ -496,7 +433,13 @@ public void testUpdateTaskInputPartitionsFailure() throws Exception { tmGateway.submitTask(tdd, env.getJobMasterId(), timeout).get(); taskRunningFuture.get(); - CompletableFuture updateFuture = tmGateway.updatePartitions( + final ResourceID producerLocation = env.getTaskExecutor().getResourceID(); + NettyShuffleDescriptor shuffleDescriptor = + createRemoteWithIdAndLocation(new IntermediateResultPartitionID(), producerLocation); + final PartitionInfo partitionUpdate = new PartitionInfo(new IntermediateDataSetID(), shuffleDescriptor); + doThrow(new IOException()).when(networkEnvironment).updatePartitionInfo(eid, partitionUpdate); + + final CompletableFuture updateFuture = tmGateway.updatePartitions( eid, Collections.singletonList(partitionUpdate), timeout); @@ -514,26 +457,11 @@ public void testUpdateTaskInputPartitionsFailure() throws Exception { */ @Test(timeout = 10000L) public void testLocalPartitionNotFound() throws Exception { - final ExecutionAttemptID eid = new ExecutionAttemptID(); - - final IntermediateDataSetID resultId = new IntermediateDataSetID(); - final ResultPartitionID partitionId = new ResultPartitionID(); - - final ResultPartitionLocation loc = ResultPartitionLocation.createLocal(); - - final InputChannelDeploymentDescriptor[] inputChannelDeploymentDescriptors = - new InputChannelDeploymentDescriptor[] { - new InputChannelDeploymentDescriptor(partitionId, loc)}; - - final InputGateDeploymentDescriptor inputGateDeploymentDescriptor = - new InputGateDeploymentDescriptor(resultId, ResultPartitionType.PIPELINED, 0, inputChannelDeploymentDescriptors); - - final TaskDeploymentDescriptor tdd = - createTestTaskDeploymentDescriptor("Receiver", - eid, - Tasks.AgnosticReceiver.class, - 1, Collections.emptyList(), - Collections.singletonList(inputGateDeploymentDescriptor)); + ResourceID producerLocation = ResourceID.generate(); + NettyShuffleDescriptor shuffleDescriptor = + createRemoteWithIdAndLocation(new IntermediateResultPartitionID(), producerLocation); + TaskDeploymentDescriptor tdd = createReceiver(shuffleDescriptor); + ExecutionAttemptID eid = tdd.getExecutionAttemptId(); Configuration config = new Configuration(); config.setInteger(NetworkEnvironmentOptions.NETWORK_REQUEST_BACKOFF_INITIAL, 100); @@ -544,6 +472,7 @@ public void testLocalPartitionNotFound() throws Exception { try (TaskSubmissionTestEnvironment env = new TaskSubmissionTestEnvironment.Builder(jobId) + .setResourceID(producerLocation) .setSlotSize(1) .addTaskManagerActionListener(eid, ExecutionState.RUNNING, taskRunningFuture) .addTaskManagerActionListener(eid, ExecutionState.FAILED, taskFailedFuture) @@ -584,23 +513,10 @@ public void testFailingScheduleOrUpdateConsumers() throws Exception { // operators configuration.setString(TaskManagerOptions.MEMORY_SEGMENT_SIZE, "4096"); - final ExecutionAttemptID eid = new ExecutionAttemptID(); - - final ResultPartitionDeploymentDescriptor resultPartitionDeploymentDescriptor = new ResultPartitionDeploymentDescriptor( - new IntermediateDataSetID(), - new IntermediateResultPartitionID(), - ResultPartitionType.PIPELINED, - 1, - 1, - true); - - final TaskDeploymentDescriptor tdd = createTestTaskDeploymentDescriptor( - "test task", - eid, - TestingAbstractInvokables.TestInvokableRecordCancel.class, - 1, - Collections.singletonList(resultPartitionDeploymentDescriptor), - Collections.emptyList()); + NettyShuffleDescriptor sdd = + createRemoteWithIdAndLocation(new IntermediateResultPartitionID(), ResourceID.generate()); + TaskDeploymentDescriptor tdd = createSender(sdd, TestingAbstractInvokables.TestInvokableRecordCancel.class); + ExecutionAttemptID eid = tdd.getExecutionAttemptId(); final CompletableFuture taskRunningFuture = new CompletableFuture<>(); @@ -774,6 +690,48 @@ public void testRequestStackTraceSample() throws Exception { } } + private TaskDeploymentDescriptor createSender(NettyShuffleDescriptor shuffleDescriptor) throws IOException { + return createSender(shuffleDescriptor, TestingAbstractInvokables.Sender.class); + } + + private TaskDeploymentDescriptor createSender( + NettyShuffleDescriptor shuffleDescriptor, + Class abstractInvokable) throws IOException { + PartitionDescriptor partitionDescriptor = new PartitionDescriptor( + new IntermediateDataSetID(), + shuffleDescriptor.getResultPartitionID().getPartitionId(), + ResultPartitionType.PIPELINED, + 1, + 0); + ResultPartitionDeploymentDescriptor resultPartitionDeploymentDescriptor = new ResultPartitionDeploymentDescriptor( + partitionDescriptor, + shuffleDescriptor, + 1, + true); + return createTestTaskDeploymentDescriptor( + "Sender", + shuffleDescriptor.getResultPartitionID().getProducerId(), + abstractInvokable, + 1, + Collections.singletonList(resultPartitionDeploymentDescriptor), + Collections.emptyList()); + } + + private TaskDeploymentDescriptor createReceiver(NettyShuffleDescriptor shuffleDescriptor) throws IOException { + InputGateDeploymentDescriptor inputGateDeploymentDescriptor = new InputGateDeploymentDescriptor( + new IntermediateDataSetID(), + ResultPartitionType.PIPELINED, + 0, + new ShuffleDescriptor[] {shuffleDescriptor}); + return createTestTaskDeploymentDescriptor( + "Receiver", + new ExecutionAttemptID(), + TestingAbstractInvokables.Receiver.class, + 1, + Collections.emptyList(), + Collections.singletonList(inputGateDeploymentDescriptor)); + } + private TaskDeploymentDescriptor createTestTaskDeploymentDescriptor( String taskName, ExecutionAttemptID eid, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java index f0364a6700fc9..e6e60f0ff556e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java @@ -231,6 +231,7 @@ private static JobManagerConnection createJobManagerConnection(JobID jobId, JobM } private static NetworkEnvironment createNetworkEnvironment( + ResourceID taskManagerLocation, boolean localCommunication, Configuration configuration, RpcService testingRpcService, @@ -246,6 +247,7 @@ private static NetworkEnvironment createNetworkEnvironment( NetworkEnvironmentConfiguration.getPageSize(configuration), ConfigurationParserUtils.getSlot(configuration), configuration); networkEnvironment = new NetworkEnvironmentBuilder() + .setTaskManagerLocation(taskManagerLocation) .setPartitionRequestInitialBackoff(configuration.getInteger(NetworkEnvironmentOptions.NETWORK_REQUEST_BACKOFF_INITIAL)) .setPartitionRequestMaxBackoff(configuration.getInteger(NetworkEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX)) .setNettyConfig(localCommunication ? null : nettyConfig) @@ -281,6 +283,7 @@ public static final class Builder { private Configuration configuration = new Configuration(); @SuppressWarnings("OptionalUsedAsFieldOrParameterType") private Optional optionalNetworkEnvironment = Optional.empty(); + private ResourceID resourceID = ResourceID.generate(); private List>> taskManagerActionListeners = new ArrayList<>(); @@ -330,11 +333,21 @@ public Builder addTaskManagerActionListener(ExecutionAttemptID eid, ExecutionSta return this; } + public Builder setResourceID(ResourceID resourceID) { + this.resourceID = resourceID; + return this; + } + public TaskSubmissionTestEnvironment build() throws Exception { final TestingRpcService testingRpcService = new TestingRpcService(); final NetworkEnvironment network = optionalNetworkEnvironment.orElseGet(() -> { try { - return createNetworkEnvironment(localCommunication, configuration, testingRpcService, mockNetworkEnvironment); + return createNetworkEnvironment( + resourceID, + localCommunication, + configuration, + testingRpcService, + mockNetworkEnvironment); } catch (Exception e) { throw new FlinkRuntimeException("Failed to build TaskSubmissionTestEnvironment", e); } 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 8ed009e549c52..f23fc9d737376 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 @@ -32,11 +32,10 @@ import org.apache.flink.runtime.blob.VoidBlobStore; 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.concurrent.Executors; -import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; -import org.apache.flink.runtime.deployment.ResultPartitionLocation; import org.apache.flink.runtime.execution.CancelTaskException; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.execution.ExecutionState; @@ -67,11 +66,14 @@ import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; import org.apache.flink.runtime.query.KvStateRegistry; +import org.apache.flink.runtime.shuffle.PartitionDescriptor; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.state.TestTaskStateManager; import org.apache.flink.runtime.taskexecutor.KvStateService; import org.apache.flink.runtime.taskexecutor.PartitionProducerStateChecker; import org.apache.flink.runtime.taskexecutor.TestGlobalAggregateManager; import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder; import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; @@ -271,26 +273,35 @@ public void testExecutionFailsInBlobsMissing() throws Exception { @Test public void testExecutionFailsInNetworkRegistrationForPartitions() throws Exception { - ResultPartitionDeploymentDescriptor dummyPartition = new ResultPartitionDeploymentDescriptor( - new IntermediateDataSetID(), new IntermediateResultPartitionID(), - ResultPartitionType.PIPELINED, 1, 1, true); + final PartitionDescriptor partitionDescriptor = new PartitionDescriptor( + new IntermediateDataSetID(), + new IntermediateResultPartitionID(), + ResultPartitionType.PIPELINED, + 1, + 1); + final ShuffleDescriptor shuffleDescriptor = NettyShuffleDescriptorBuilder.newBuilder().buildLocal(); + final ResultPartitionDeploymentDescriptor dummyPartition = new ResultPartitionDeploymentDescriptor( + partitionDescriptor, + shuffleDescriptor, + 1, + false); testExecutionFailsInNetworkRegistration(Collections.singleton(dummyPartition), Collections.emptyList()); } @Test public void testExecutionFailsInNetworkRegistrationForGates() throws Exception { - InputChannelDeploymentDescriptor dummyChannel = - new InputChannelDeploymentDescriptor(new ResultPartitionID(), ResultPartitionLocation.createLocal()); - InputGateDeploymentDescriptor dummyGate = new InputGateDeploymentDescriptor( - new IntermediateDataSetID(), ResultPartitionType.PIPELINED, 0, - new InputChannelDeploymentDescriptor[] { dummyChannel }); + final ShuffleDescriptor dummyChannel = NettyShuffleDescriptorBuilder.newBuilder().buildRemote(); + final InputGateDeploymentDescriptor dummyGate = new InputGateDeploymentDescriptor( + new IntermediateDataSetID(), + ResultPartitionType.PIPELINED, + 0, + new ShuffleDescriptor[] { dummyChannel }); testExecutionFailsInNetworkRegistration(Collections.emptyList(), Collections.singleton(dummyGate)); } private void testExecutionFailsInNetworkRegistration( - Collection resultPartitions, - Collection inputGates) throws Exception { - + Collection resultPartitions, + Collection inputGates) throws Exception { final String errorMessage = "Network buffer pool has already been destroyed."; final ResultPartitionConsumableNotifier consumableNotifier = new NoOpResultPartitionConsumableNotifier(); @@ -316,8 +327,7 @@ private void testExecutionFailsInNetworkRegistration( assertTrue(task.isCanceledOrFailed()); assertTrue(task.getFailureCause().getMessage().contains(errorMessage)); - taskManagerActions.validateListenerMessage( - ExecutionState.FAILED, task, new IllegalStateException(errorMessage)); + taskManagerActions.validateListenerMessage(ExecutionState.FAILED, task, new IllegalStateException(errorMessage)); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/NettyShuffleDescriptorBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/NettyShuffleDescriptorBuilder.java new file mode 100644 index 0000000000000..0dcccaf24538c --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/NettyShuffleDescriptorBuilder.java @@ -0,0 +1,103 @@ +/* + * 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.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.io.network.ConnectionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; +import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor.LocalExecutionPartitionConnectionInfo; +import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor.NetworkPartitionConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + +import java.net.InetAddress; +import java.net.InetSocketAddress; + +/** + * Builder to mock {@link NettyShuffleDescriptor} in tests. + */ +public class NettyShuffleDescriptorBuilder { + private ResourceID producerLocation = ResourceID.generate(); + private ResultPartitionID id = new ResultPartitionID(); + private InetAddress address = InetAddress.getLoopbackAddress(); + private int dataPort = 0; + private int connectionIndex = 0; + + public NettyShuffleDescriptorBuilder setProducerLocation(ResourceID producerLocation) { + this.producerLocation = producerLocation; + return this; + } + + public NettyShuffleDescriptorBuilder setId(ResultPartitionID id) { + this.id = id; + return this; + } + + public NettyShuffleDescriptorBuilder setAddress(InetAddress address) { + this.address = address; + return this; + } + + public NettyShuffleDescriptorBuilder setDataPort(int dataPort) { + this.dataPort = dataPort; + return this; + } + + public NettyShuffleDescriptorBuilder setProducerInfoFromTaskManagerLocation( + TaskManagerLocation producerTaskManagerLocation) { + return setProducerLocation(producerTaskManagerLocation.getResourceID()) + .setAddress(producerTaskManagerLocation.address()) + .setDataPort(producerTaskManagerLocation.dataPort()); + } + + public NettyShuffleDescriptorBuilder setConnectionIndex(int connectionIndex) { + this.connectionIndex = connectionIndex; + return this; + } + + public NettyShuffleDescriptor buildRemote() { + ConnectionID connectionID = new ConnectionID(new InetSocketAddress(address, dataPort), connectionIndex); + return new NettyShuffleDescriptor( + producerLocation, + new NetworkPartitionConnectionInfo(connectionID), + id); + } + + public NettyShuffleDescriptor buildLocal() { + return new NettyShuffleDescriptor( + producerLocation, + LocalExecutionPartitionConnectionInfo.INSTANCE, + id); + } + + public static NettyShuffleDescriptorBuilder newBuilder() { + return new NettyShuffleDescriptorBuilder(); + } + + public static NettyShuffleDescriptor createRemoteWithIdAndLocation( + IntermediateResultPartitionID partitionId, + ResourceID producerLocation) { + return newBuilder() + .setId(new ResultPartitionID(partitionId, new ExecutionAttemptID())) + .setProducerLocation(producerLocation) + .buildRemote(); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java index 86e1d6f436efa..770af88d9eb58 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java @@ -23,12 +23,9 @@ import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; -import org.apache.flink.runtime.deployment.ResultPartitionLocation; 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.ConnectionID; import org.apache.flink.runtime.io.network.NetworkEnvironment; import org.apache.flink.runtime.io.network.NetworkEnvironmentBuilder; import org.apache.flink.runtime.io.network.TaskEventDispatcher; @@ -47,12 +44,14 @@ import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateFactory; import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.taskmanager.ConsumableNotifyingResultPartitionWriterDecorator; import org.apache.flink.runtime.taskmanager.InputGateWithMetrics; import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; import org.apache.flink.runtime.taskmanager.NoOpTaskActions; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.util.ConfigurationParserUtils; +import org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder; import java.io.IOException; import java.net.InetAddress; @@ -77,6 +76,7 @@ public class StreamNetworkBenchmarkEnvironment { } } + private final ResourceID location = ResourceID.generate(); protected final JobID jobId = new JobID(); protected final IntermediateDataSetID dataSetID = new IntermediateDataSetID(); @@ -156,6 +156,7 @@ public void setUp( } gateFactory = new SingleInputGateFactory( + location, receiverEnv.getConfiguration(), receiverEnv.getConnectionManager(), receiverEnv.getResultPartitionManager(), @@ -243,7 +244,8 @@ private InputGate createInputGate(TaskManagerLocation senderLocation) throws IOE for (int channel = 0; channel < channels; ++channel) { final InputGateDeploymentDescriptor gateDescriptor = createInputGateDeploymentDescriptor( senderLocation, - channel); + channel, + location); final InputGate gate = createInputGateWithMetrics(gateFactory, gateDescriptor, channel); @@ -260,14 +262,13 @@ private InputGate createInputGate(TaskManagerLocation senderLocation) throws IOE private InputGateDeploymentDescriptor createInputGateDeploymentDescriptor( TaskManagerLocation senderLocation, - int consumedSubpartitionIndex) { + int consumedSubpartitionIndex, + ResourceID localLocation) { - final InputChannelDeploymentDescriptor[] channelDescriptors = Arrays.stream(partitionIds) - .map(partitionId -> new InputChannelDeploymentDescriptor( - partitionId, - localMode ? ResultPartitionLocation.createLocal() : ResultPartitionLocation.createRemote( - new ConnectionID(senderLocation, consumedSubpartitionIndex)))) - .toArray(InputChannelDeploymentDescriptor[]::new); + final ShuffleDescriptor[] channelDescriptors = Arrays.stream(partitionIds) + .map(partitionId -> + createShuffleDescriptor(localMode, partitionId, localLocation, senderLocation, consumedSubpartitionIndex)) + .toArray(ShuffleDescriptor[]::new); return new InputGateDeploymentDescriptor( dataSetID, @@ -289,4 +290,17 @@ private InputGate createInputGateWithMetrics( return new InputGateWithMetrics(singleGate, new SimpleCounter()); } + + private static ShuffleDescriptor createShuffleDescriptor( + boolean localMode, + ResultPartitionID resultPartitionID, + ResourceID location, + TaskManagerLocation senderLocation, + int channel) { + final NettyShuffleDescriptorBuilder builder = NettyShuffleDescriptorBuilder.newBuilder() + .setId(resultPartitionID) + .setProducerInfoFromTaskManagerLocation(senderLocation) + .setConnectionIndex(channel); + return localMode ? builder.setProducerLocation(location).buildLocal() : builder.buildRemote(); + } }