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_r292829820
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/partition/JobAwareShuffleEnvironmentImpl.java
 ##########
 @@ -0,0 +1,194 @@
+/*
+ * 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.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<NotifyingResultPartitionWriter, G> {
+
+       private static final Consumer<ResultPartitionID> NO_OP_NOTIFIER = 
partitionId -> {};
+
+       private final ShuffleEnvironment<?, G> backingShuffleEnvironment;
+       private final PartitionTable inProgressPartitionTable = new 
PartitionTable();
+       private final PartitionTable finishedPartitionTable = new 
PartitionTable();
+
+       /**     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 
map since it is modified by both the
+        * Task (via {@link #notifyPartitionFinished(JobID, 
ResultPartitionID)}} and
+        * TaskExecutor (via {@link 
#releaseAllFinishedPartitionsForJobAndMarkJobInactive(JobID)}) thread. */
+       private final Set<JobID> activeJobs = ConcurrentHashMap.newKeySet();
+
+       public JobAwareShuffleEnvironmentImpl(ShuffleEnvironment<?, G> 
backingShuffleEnvironment) {
+               this.backingShuffleEnvironment = 
Preconditions.checkNotNull(backingShuffleEnvironment);
+       }
+
+       @Override
+       public boolean hasPartitionsOccupyingLocalResources(JobID jobId) {
 
 Review comment:
   You'd still need separate methods. This method here is about having 
partitions for a specific job, while `getPartitionsOccupyingLocalResources()` 
checks across jobs.
   
   Additionally, we don't track all partitions in the 
`JobAwareShuffleEnvironment`; only those that are externally managed, so the 
semantics are a bit different.

----------------------------------------------------------------
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