tillrohrmann commented on a change in pull request #10362: 
[FLINK-14792][coordination] Implement TE cluster partition release
URL: https://github.com/apache/flink/pull/10362#discussion_r397874560
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java
 ##########
 @@ -66,41 +72,67 @@ public void stopTrackingAndReleaseJobPartitionsFor(JobID 
producingJobId) {
 
        @Override
        public void promoteJobPartitions(Collection<ResultPartitionID> 
partitionsToPromote) {
+               if (partitionsToPromote.isEmpty()) {
+                       return;
+               }
+
                final Collection<PartitionTrackerEntry<JobID, 
TaskExecutorPartitionInfo>> partitionTrackerEntries = 
stopTrackingPartitions(partitionsToPromote);
 
-               final Map<TaskExecutorPartitionInfo, Set<ResultPartitionID>> 
newClusterPartitions = partitionTrackerEntries.stream()
-                       .collect(Collectors.groupingBy(
-                               PartitionTrackerEntry::getMetaInfo,
-                               
Collectors.mapping(PartitionTrackerEntry::getResultPartitionId, 
Collectors.toSet())));
-
-               newClusterPartitions.forEach(
-                       (dataSetMetaInfo, newPartitionEntries) -> 
clusterPartitions.compute(dataSetMetaInfo, (ignored, existingPartitions) -> {
-                               if (existingPartitions == null) {
-                                       return newPartitionEntries;
-                               } else {
-                                       
existingPartitions.addAll(newPartitionEntries);
-                                       return existingPartitions;
-                               }
-                       }));
+               for (PartitionTrackerEntry<JobID, TaskExecutorPartitionInfo> 
partitionTrackerEntry : partitionTrackerEntries) {
+                       final TaskExecutorPartitionInfo dataSetMetaInfo = 
partitionTrackerEntry.getMetaInfo();
+                       final DataSetEntry dataSetEntry = 
clusterPartitions.computeIfAbsent(
+                               dataSetMetaInfo.getIntermediateDataSetId(),
+                               ignored -> new 
DataSetEntry(dataSetMetaInfo.getNumberOfPartitions()));
+                       
dataSetEntry.addPartition(partitionTrackerEntry.getResultPartitionId());
+               }
+       }
+
+       @Override
+       public void 
stopTrackingAndReleaseClusterPartitions(Collection<IntermediateDataSetID> 
dataSetsToRelease) {
+               for (IntermediateDataSetID dataSetID : dataSetsToRelease) {
+                       final DataSetEntry dataSetEntry = 
clusterPartitions.remove(dataSetID);
+                       final Set<ResultPartitionID> partitionIds = 
dataSetEntry.getPartitionIds();
+                       
shuffleEnvironment.releasePartitionsLocally(partitionIds);
+               }
        }
 
        @Override
        public void stopTrackingAndReleaseAllClusterPartitions() {
-               
clusterPartitions.values().forEach(shuffleEnvironment::releasePartitionsLocally);
+               
clusterPartitions.values().stream().map(DataSetEntry::getPartitionIds).forEach(shuffleEnvironment::releasePartitionsLocally);
                clusterPartitions.clear();
        }
 
        @Override
        public ClusterPartitionReport createClusterPartitionReport() {
-               List<ClusterPartitionReport.ClusterPartitionReportEntry> 
collect = clusterPartitions.entrySet().stream().map(entry -> {
-                       TaskExecutorPartitionInfo dataSetMetaInfo = 
entry.getKey();
-                       Set<ResultPartitionID> partitionsIds = entry.getValue();
-                       return new 
ClusterPartitionReport.ClusterPartitionReportEntry(
-                               dataSetMetaInfo.getIntermediateDataSetId(),
-                               partitionsIds,
-                               dataSetMetaInfo.getNumberOfPartitions());
-               }).collect(Collectors.toList());
+               List<ClusterPartitionReport.ClusterPartitionReportEntry> 
reportEntries = clusterPartitions.entrySet().stream().map(entry ->
+                       new ClusterPartitionReport.ClusterPartitionReportEntry(
+                               entry.getKey(),
+                               entry.getValue().getPartitionIds(),
+                               entry.getValue().getTotalNumberOfPartitions()))
+                       .collect(Collectors.toList());
+
+               return new ClusterPartitionReport(reportEntries);
+       }
+
+       private static class DataSetEntry {
+
+               private final Set<ResultPartitionID> partitionIds = new 
HashSet<>();
+               private final int totalNumberOfPartitions;
+
+               private DataSetEntry(int totalNumberOfPartitions) {
+                       this.totalNumberOfPartitions = totalNumberOfPartitions;
+               }
+
+               void addPartition(ResultPartitionID resultPartitionId) {
+                       partitionIds.add(resultPartitionId);
+               }
+
+               public Set<ResultPartitionID> getPartitionIds() {
 
 Review comment:
   Nit: I guess we could limit the accessibility of the two getters in this 
class.

----------------------------------------------------------------
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:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to