zentol commented on a change in pull request #11443: [FLINK-14791][coordination] ResourceManager tracks ClusterPartitions URL: https://github.com/apache/flink/pull/11443#discussion_r400034100
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImpl.java ########## @@ -0,0 +1,232 @@ +/* + * 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.io.network.partition; + +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.taskexecutor.partition.ClusterPartitionReport; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +/** + * Default {@link ResourceManagerPartitionTracker} implementation. + * + * <p>Internal tracking info must only be updated upon reception of a {@link ClusterPartitionReport}, as the task + * executor state is the source of truth. + */ +public class ResourceManagerPartitionTrackerImpl implements ResourceManagerPartitionTracker { + + private static final Logger LOG = LoggerFactory.getLogger(ResourceManagerPartitionTrackerImpl.class); + + private final Map<ResourceID, Set<IntermediateDataSetID>> taskExecutorToDataSets = new HashMap<>(); + private final Map<IntermediateDataSetID, Map<ResourceID, Set<ResultPartitionID>>> dataSetToTaskExecutors = new HashMap<>(); + private final Map<IntermediateDataSetID, DataSetMetaInfo> dataSetMetaInfo = new HashMap<>(); + private final Map<IntermediateDataSetID, CompletableFuture<Void>> partitionReleaseCompletionFutures = new HashMap<>(); + + private final ClusterPartitionReleaser clusterPartitionReleaser; + + public ResourceManagerPartitionTrackerImpl(ClusterPartitionReleaser clusterPartitionReleaser) { + this.clusterPartitionReleaser = clusterPartitionReleaser; + } + + @Override + public void processTaskExecutorClusterPartitionReport(ResourceID taskExecutorId, ClusterPartitionReport clusterPartitionReport) { + Preconditions.checkNotNull(taskExecutorId); + Preconditions.checkNotNull(clusterPartitionReport); + LOG.debug("Processing cluster partition report from task executor {}: {}.", taskExecutorId, clusterPartitionReport); + + internalProcessClusterPartitionReport(taskExecutorId, clusterPartitionReport); + } + + @Override + public void processTaskExecutorShutdown(ResourceID taskExecutorId) { + Preconditions.checkNotNull(taskExecutorId); + LOG.debug("Processing shutdown of task executor {}.", taskExecutorId); + + internalProcessClusterPartitionReport(taskExecutorId, new ClusterPartitionReport(Collections.emptyList())); + } + + @Override + public CompletableFuture<Void> releaseClusterPartitions(IntermediateDataSetID dataSetId) { + Preconditions.checkNotNull(dataSetId); + LOG.debug("Releasing cluster partitions for data set {}.", dataSetId); + + CompletableFuture<Void> partitionReleaseCompletionFuture = partitionReleaseCompletionFutures.computeIfAbsent(dataSetId, ignored -> new CompletableFuture<>()); + internalReleasePartitions(Collections.singleton(dataSetId)); + return partitionReleaseCompletionFuture; + } + + private void internalProcessClusterPartitionReport(ResourceID taskExecutorId, ClusterPartitionReport clusterPartitionReport) { + final Set<IntermediateDataSetID> dataSetsWithLostPartitions = clusterPartitionReport.getEntries().isEmpty() + ? processEmptyReport(taskExecutorId) + : setHostedDataSetsAndCheckCorruption(taskExecutorId, clusterPartitionReport.getEntries()); + + updateDataSetMetaData(clusterPartitionReport); + + checkForFullyLostDatasets(dataSetsWithLostPartitions); + + internalReleasePartitions(dataSetsWithLostPartitions); + } + + private void internalReleasePartitions(Set<IntermediateDataSetID> dataSetsToRelease) { + Map<ResourceID, Set<IntermediateDataSetID>> releaseCalls = prepareReleaseCalls(dataSetsToRelease); + releaseCalls.forEach(clusterPartitionReleaser::releaseClusterPartitions); + } + + private Set<IntermediateDataSetID> processEmptyReport(ResourceID taskExecutorId) { + Set<IntermediateDataSetID> previouslyHostedDatasets = taskExecutorToDataSets.remove(taskExecutorId); + if (previouslyHostedDatasets == null) { + // default path for task executors that never have any cluster partitions + previouslyHostedDatasets = Collections.emptySet(); + } else { + previouslyHostedDatasets.forEach(dataSetId -> removeInnerKey(dataSetId, taskExecutorId, dataSetToTaskExecutors)); + } + return previouslyHostedDatasets; + } + + /** + * Updates the data sets for which the given task executor is hosting partitions and returns data sets that were + * corrupted due to a loss of partitions. + * + * @param taskExecutorId ID of the hosting TaskExecutor + * @param reportEntries IDs of data sets for which partitions are hosted + * @return corrupted data sets + */ + private Set<IntermediateDataSetID> setHostedDataSetsAndCheckCorruption(ResourceID taskExecutorId, Collection<ClusterPartitionReport.ClusterPartitionReportEntry> reportEntries) { + final Set<IntermediateDataSetID> currentlyHostedDatasets = reportEntries + .stream() + .map(ClusterPartitionReport.ClusterPartitionReportEntry::getDataSetId) + .collect(Collectors.toSet()); + + final Set<IntermediateDataSetID> previouslyHostedDataSets = taskExecutorToDataSets.put( + taskExecutorId, + currentlyHostedDatasets); + + // previously tracked data sets may be corrupted since we may be tracking less partitions than before + final Set<IntermediateDataSetID> potentiallyCorruptedDataSets = Optional + .ofNullable(previouslyHostedDataSets) + .orElse(new HashSet<>(0)); + + // update data set -> task executor mapping and find datasets for which lost a partition + reportEntries.forEach(hostedPartition -> { + final Map<ResourceID, Set<ResultPartitionID>> taskExecutorHosts = dataSetToTaskExecutors.computeIfAbsent(hostedPartition.getDataSetId(), ignored -> new HashMap<>()); + final Set<ResultPartitionID> previouslyHostedPartitions = taskExecutorHosts.put(taskExecutorId, hostedPartition.getHostedPartitions()); + + final boolean noPartitionLost = previouslyHostedPartitions == null || hostedPartition.getHostedPartitions().containsAll(previouslyHostedPartitions); + if (noPartitionLost) { + potentiallyCorruptedDataSets.remove(hostedPartition.getDataSetId()); + } + }); + + // now only contains data sets for which a partition is no longer tracked + return potentiallyCorruptedDataSets; + } + + private void updateDataSetMetaData(ClusterPartitionReport clusterPartitionReport) { + // add meta info for new data sets + clusterPartitionReport.getEntries().forEach(entry -> + dataSetMetaInfo.compute(entry.getDataSetId(), (dataSetID, dataSetMetaInfo) -> { + if (dataSetMetaInfo == null) { + return new DataSetMetaInfo(entry.getNumTotalPartitions()); + } else { + // double check that the meta data is consistent + Preconditions.checkState(dataSetMetaInfo.getNumTotalPartitions() == entry.getNumTotalPartitions()); + return dataSetMetaInfo; + } + })); + } + + private void checkForFullyLostDatasets(Set<IntermediateDataSetID> dataSetsWithLostPartitions) { + dataSetsWithLostPartitions.forEach(dataSetId -> { + if (getHostingTaskExecutors(dataSetId).isEmpty()) { + LOG.debug("There are no longer partitions being tracked for dataset {}.", dataSetId); + dataSetMetaInfo.remove(dataSetId); + Optional.ofNullable(partitionReleaseCompletionFutures.get(dataSetId)).map(future -> future.complete(null)); Review comment: yes, will change it to remove() the future instead of get(). ---------------------------------------------------------------- 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
