zentol commented on a change in pull request #8687: [FLINK-12612][coordination] Track stored partition on the TaskExecutor URL: https://github.com/apache/flink/pull/8687#discussion_r296135387
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/partition/JobAwareShuffleEnvironmentImpl.java ########## @@ -0,0 +1,221 @@ +/* + * 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.taskexecutor.partition; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; +import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.PartitionInfo; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.runtime.io.network.partition.PartitionProducerStateProvider; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +/** + * Wraps a {@link ShuffleEnvironment} to allow tracking of partitions per job. + */ +public class JobAwareShuffleEnvironmentImpl<G extends InputGate> implements JobAwareShuffleEnvironment<ResultPartitionWriter, G> { + + private final ShuffleEnvironment<?, G> backingShuffleEnvironment; + + /** + * Releasing in-progress partitions while tasks are still running may lead to odd behaviors, hence we distinguish + * between in-progress and finished partitions so that we can exclusively release finished partitions. + */ + private final PartitionTable inProgressPartitionTable = new PartitionTable(); + private final PartitionTable finishedPartitionTable = new PartitionTable(); + + private Consumer<JobID> listener = jobId -> {}; + + /** Tracks which jobs are still being monitored, to ensure cleanup in cases where tasks are finishing while + * the jobmanager connection is being terminated. This is a concurrent set since it is modified by both the + * Task (via {@link #notifyPartitionFinished(JobID, ResultPartitionID)}} and + * TaskExecutor (via {@link #releaseAllFinishedPartitionsAndMarkJobInactive(JobID)}) threads. */ + private final Set<JobID> activeJobs = ConcurrentHashMap.newKeySet(); + + public JobAwareShuffleEnvironmentImpl(ShuffleEnvironment<?, G> backingShuffleEnvironment) { + this.backingShuffleEnvironment = Preconditions.checkNotNull(backingShuffleEnvironment); + } + + @Override + public void setPartitionFailedOrFinishedListener(Consumer<JobID> listener) { + Preconditions.checkNotNull(listener); + this.listener = listener; + } + + @Override + public boolean hasPartitionsOccupyingLocalResources(JobID jobId) { + return inProgressPartitionTable.hasTrackedPartitions(jobId) || finishedPartitionTable.hasTrackedPartitions(jobId); + } + + @Override + public void markJobActive(JobID jobId) { + activeJobs.add(jobId); + } + + @Override + public void releaseFinishedPartitions(JobID jobId, Collection<ResultPartitionID> resultPartitionIds) { + // maybe double-check that all partitions are finished? + finishedPartitionTable.stopTrackingPartitions(jobId, resultPartitionIds); + backingShuffleEnvironment.releasePartitions(resultPartitionIds); + } + + @Override + public void releaseAllFinishedPartitionsAndMarkJobInactive(JobID jobId) { + activeJobs.remove(jobId); + Collection<ResultPartitionID> finishedPartitionsForJob = finishedPartitionTable.stopTrackingPartitions(jobId); + backingShuffleEnvironment.releasePartitions(finishedPartitionsForJob); + } + + /** + * This method wraps partition writers if the partition is not relased and consumption + * and introduces callbacks into the lifecycle methods of the {@link ResultPartitionWriter}. + */ + @Override + public Collection<ResultPartitionWriter> createResultPartitionWriters( + JobID jobId, + String taskName, + ExecutionAttemptID executionAttemptID, + Collection<ResultPartitionDeploymentDescriptor> resultPartitionDeploymentDescriptors, + MetricGroup outputGroup, + MetricGroup buffersGroup) { + + final Collection<? extends ResultPartitionWriter> resultPartitionWriters = backingShuffleEnvironment + .createResultPartitionWriters(taskName, executionAttemptID, resultPartitionDeploymentDescriptors, outputGroup, buffersGroup); + + // determine which partitions are not released on consumption, and whose writers need to be wrapped + final Set<ResultPartitionID> externallyManagedPartitions = resultPartitionDeploymentDescriptors.stream() + .filter(descriptor -> !descriptor.isReleasedOnConsumption()) + .map(ResultPartitionDeploymentDescriptor::getShuffleDescriptor) + .map(ShuffleDescriptor::getResultPartitionID) + .collect(Collectors.toSet()); + + // wrap partition writers to introduce callbacks for lifecycle events + final Collection<ResultPartitionWriter> wrappedResultPartitionWriters = new ArrayList<>(resultPartitionWriters.size()); + for (ResultPartitionWriter resultPartitionWriter : resultPartitionWriters) { + final ResultPartitionWriter maybeWrappedWriter; + if (externallyManagedPartitions.contains(resultPartitionWriter.getPartitionId())) { + maybeWrappedWriter = new NotifyingResultPartitionWriter( + resultPartitionWriter, + partitionId -> notifyPartitionSetup(jobId, partitionId), + partitionId -> notifyPartitionFailed(jobId, partitionId), + partitionId -> notifyPartitionFinished(jobId, partitionId)); + } else { + maybeWrappedWriter = resultPartitionWriter; + } + wrappedResultPartitionWriters.add(maybeWrappedWriter); + } + + return wrappedResultPartitionWriters; + } + + private void notifyPartitionSetup(JobID jobId, ResultPartitionID resultPartitionId) { Review comment: From the perspective of the task there is nothing to clean up. The task itself only allocates something within the try-catch block; that's why it doesn't need to interact with the shuffle environment in any way if it fails there. The task constructor specifically says that nothing requiring cleanup should be done in the constructor, which we would do if we were to hook into `createResultPartitionWriters`. ---------------------------------------------------------------- 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