azagrebin commented on a change in pull request #8362: [FLINK-11391] Introduce 
shuffle master interface
URL: https://github.com/apache/flink/pull/8362#discussion_r289631850
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorFactory.java
 ##########
 @@ -0,0 +1,296 @@
+/*
+ * 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.clusterframework.types.ResourceID;
+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.ExecutionGraphException;
+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.jobmaster.LogicalSlot;
+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.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * 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<JobInformation> serializedJobInformation;
+       private final MaybeOffloaded<TaskInformation> taskInfo;
+       private final JobID jobID;
+       private final boolean lazyScheduling;
+       private final int subtaskIndex;
+       private final ExecutionEdge[][] inputEdges;
+
+       private TaskDeploymentDescriptorFactory(
+                       ExecutionAttemptID executionId,
+                       int attemptNumber,
+                       MaybeOffloaded<JobInformation> serializedJobInformation,
+                       MaybeOffloaded<TaskInformation> taskInfo,
+                       JobID jobID,
+                       boolean lazyScheduling,
+                       int subtaskIndex,
+                       ExecutionEdge[][] inputEdges) {
+               this.executionId = executionId;
+               this.attemptNumber = attemptNumber;
+               this.serializedJobInformation = serializedJobInformation;
+               this.taskInfo = taskInfo;
+               this.jobID = jobID;
+               this.lazyScheduling = lazyScheduling;
+               this.subtaskIndex = subtaskIndex;
+               this.inputEdges = inputEdges;
+       }
+
+       public TaskDeploymentDescriptor createDeploymentDescriptor(
+                       LogicalSlot targetSlot,
+                       @Nullable JobManagerTaskRestore taskRestore,
+                       Collection<ResultPartitionDeploymentDescriptor> 
producedPartitions) throws Exception {
+               return createDeploymentDescriptor(
+                       targetSlot.getTaskManagerLocation().getResourceID(),
+                       targetSlot.getAllocationId(),
+                       targetSlot.getPhysicalSlotNumber(),
+                       taskRestore,
+                       producedPartitions);
+       }
+
+       @VisibleForTesting
+       public TaskDeploymentDescriptor createDeploymentDescriptor(
+                       ResourceID location,
+                       AllocationID allocationID,
+                       int targetSlotNumber,
+                       @Nullable JobManagerTaskRestore taskRestore,
+                       Collection<ResultPartitionDeploymentDescriptor> 
producedPartitions) throws Exception {
+               return new TaskDeploymentDescriptor(
+                       jobID,
+                       serializedJobInformation,
+                       taskInfo,
+                       executionId,
+                       allocationID,
+                       subtaskIndex,
+                       attemptNumber,
+                       targetSlotNumber,
+                       taskRestore,
+                       new ArrayList<>(producedPartitions),
+                       createInputGateDeploymentDescriptors(location, 
inputEdges, subtaskIndex, lazyScheduling));
+       }
+
+       public static TaskDeploymentDescriptorFactory fromExecutionVertex(
+                       ExecutionVertex executionVertex,
+                       ExecutionAttemptID executionId,
+                       int attemptNumber) throws ExecutionGraphException {
+               ExecutionGraph executionGraph = 
executionVertex.getExecutionGraph();
+               return new TaskDeploymentDescriptorFactory(
+                       executionId,
+                       attemptNumber,
+                       getSerializedJobInformation(executionGraph),
+                       
getSerializedTaskInformation(executionVertex.getJobVertex().getTaskInformationOrBlobKey()),
+                       executionGraph.getJobID(),
+                       executionGraph.getScheduleMode().allowLazyDeployment(),
+                       executionVertex.getParallelSubtaskIndex(),
+                       executionVertex.getAllInputEdges());
+       }
+
+       private static MaybeOffloaded<JobInformation> 
getSerializedJobInformation(
+               ExecutionGraph executionGraph) {
+               Either<SerializedValue<JobInformation>, PermanentBlobKey> 
jobInformationOrBlobKey =
+                       executionGraph.getJobInformationOrBlobKey();
+               if (jobInformationOrBlobKey.isLeft()) {
+                       return new 
TaskDeploymentDescriptor.NonOffloaded<>(jobInformationOrBlobKey.left());
+               } else {
+                       return new 
TaskDeploymentDescriptor.Offloaded<>(jobInformationOrBlobKey.right());
+               }
+       }
+
+       private static MaybeOffloaded<TaskInformation> 
getSerializedTaskInformation(
+                       Either<SerializedValue<TaskInformation>,
+                       PermanentBlobKey> taskInfo) {
+               return taskInfo.isLeft() ?
+                       new 
TaskDeploymentDescriptor.NonOffloaded<>(taskInfo.left()) :
+                       new 
TaskDeploymentDescriptor.Offloaded<>(taskInfo.right());
+       }
+
+       private static List<InputGateDeploymentDescriptor> 
createInputGateDeploymentDescriptors(
+                       ResourceID location,
+                       ExecutionEdge[][] inputEdges,
+                       int subtaskIndex,
+                       boolean allowLazyDeployment) throws 
ExecutionGraphException {
+               List<InputGateDeploymentDescriptor> consumedPartitions = new 
ArrayList<>(inputEdges.length);
+
+               for (ExecutionEdge[] edges : inputEdges) {
+                       ShuffleDescriptor[] consumedPartitionShuffleDescriptors 
=
 
 Review comment:
   I think we should preserve `consumedPartition` in the name to highlight the 
purpose of these shuffle descriptors but it can be shortened a bit, true.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to