You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2019/02/12 06:08:05 UTC

[GitHub] zhijiangW commented on a change in pull request #7631: [FLINK-11391][shuffle] Introduce PartitionShuffleDescriptor and ShuffleDeploymentDescriptor

zhijiangW commented on a change in pull request #7631: [FLINK-11391][shuffle] Introduce PartitionShuffleDescriptor and ShuffleDeploymentDescriptor
URL: https://github.com/apache/flink/pull/7631#discussion_r255814019
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
 ##########
 @@ -813,24 +815,27 @@ TaskDeploymentDescriptor createDeploymentDescriptor(
 		boolean lazyScheduling = getExecutionGraph().getScheduleMode().allowLazyDeployment();
 
 		for (IntermediateResultPartition partition : resultPartitions.values()) {
-
 			List<List<ExecutionEdge>> consumers = partition.getConsumers();
-
+			int maxParallelism;
 			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));
+				maxParallelism = KeyGroupRangeAssignment.UPPER_BOUND_MAX_PARALLELISM;
 			} else {
 				Preconditions.checkState(1 == consumers.size(),
-						"Only one consumer supported in the current implementation! Found: " + consumers.size());
+					"Only one consumer supported in the current implementation! Found: " + consumers.size());
 
 				List<ExecutionEdge> consumer = consumers.get(0);
 				ExecutionJobVertex vertex = consumer.get(0).getTarget().getJobVertex();
-				int maxParallelism = vertex.getMaxParallelism();
-				producedPartitions.add(ResultPartitionDeploymentDescriptor.from(partition, maxParallelism, lazyScheduling));
+				maxParallelism = vertex.getMaxParallelism();
 			}
+
+			PartitionShuffleDescriptor psd = PartitionShuffleDescriptor.from(targetSlot, executionId, partition, maxParallelism);
+
+			producedPartitions.add(ResultPartitionDeploymentDescriptor.fromShuffleDescriptor(psd));
+			getCurrentExecutionAttempt().cachePartitionShuffleDescriptor(partition.getIntermediateResult().getId(), psd);
 
 Review comment:
   From functional aspect, caching the `TaskDeploymentDescriptor` might also make sense. But I have other concerns:
   1. The structure of TDD is complicated and would take more memory if caching completely, such as unnecessary fields `serializedJobInformation`, `serializedTaskInformation`,etc.
   
   2. We might need adjust the current collection structure of `producedPartitions`, `inputGates` in TDD to map structure in order to find required PSD, SDD directly for other usages.
   
   3. If replacing the current three descriptors caches, we might not need the class of `PartialInputChannelDeploymentDescriptor` any more if I understand correctly. But I wonder there exists such scenarios that during deploying consumer execution, only some input channel descriptors are unknown. During sending partition infos we only want to send these unknown infos when consumer deployment, so how can we distinguish them from all the cached producer's TDD?  In other words, the current cached `partialInputChannelDeploymentDescriptors` might be only a sub collection of cached TDDs on producer side.

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


With regards,
Apache Git Services