You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@celeborn.apache.org by "waitinfuture (via GitHub)" <gi...@apache.org> on 2023/04/14 03:59:51 UTC

[GitHub] [incubator-celeborn] waitinfuture commented on a diff in pull request #1405: [CELEBORN-470][FLINK] support stopTrackingAndReleasePartitions

waitinfuture commented on code in PR #1405:
URL: https://github.com/apache/incubator-celeborn/pull/1405#discussion_r1166227716


##########
client-flink/flink-1.14/src/main/java/org/apache/celeborn/plugin/flink/ShuffleResourceTracker.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.celeborn.plugin.flink;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.shuffle.JobShuffleContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.celeborn.client.LifecycleManager;
+
+public class ShuffleResourceTracker {
+  private static final Logger LOG = LoggerFactory.getLogger(ShuffleResourceTracker.class);
+  private final ExecutorService executorService;
+  private final Map<JobID, JobShuffleContext> jobShuffleContextMap = new ConcurrentHashMap<>();
+  // shuffleId -> ShuffleResourceListener
+  private final Map<Integer, ShuffleResourceListener> shuffleResourceListeners =
+      new ConcurrentHashMap<>();
+  private static final int MAX_RETRY_TIMES = 3;
+
+  public ShuffleResourceTracker(
+      ExecutorService executorService, LifecycleManager lifecycleManager) {
+    this.executorService = executorService;
+    ShuffleWorkerStatusListener shuffleWorkerStatusListener =
+        new ShuffleWorkerStatusListener(lifecycleManager, shuffleResourceListeners);
+    lifecycleManager.registerWorkerStatusListener(shuffleWorkerStatusListener);
+  }
+
+  public void registerJobContext(JobShuffleContext jobShuffleContext) {
+    jobShuffleContextMap.put(jobShuffleContext.getJobId(), jobShuffleContext);
+  }
+
+  public void addPartitionResource(
+      JobID jobId, int shuffleId, int partitionId, ResultPartitionID partitionID) {
+    JobShuffleContext jobShuffleContext = jobShuffleContextMap.get(jobId);
+    ShuffleResourceListener shuffleResourceListener =
+        shuffleResourceListeners.computeIfAbsent(
+            shuffleId,
+            (s) -> new ShuffleResourceListener(jobShuffleContext, shuffleId, executorService));
+    shuffleResourceListener.addPartitionResource(partitionId, partitionID);
+  }
+
+  public void removePartitionResource(int shuffleId, int partitionId) {
+    ShuffleResourceListener shuffleResourceListener = shuffleResourceListeners.get(shuffleId);
+    if (shuffleResourceListener != null) {
+      shuffleResourceListener.removePartitionResource(partitionId);
+    }
+  }
+
+  public void removeShuffleListener(int shuffleId) {
+    shuffleResourceListeners.remove(shuffleId);
+  }
+
+  public void unRegisterJob(JobID jobID) {
+    jobShuffleContextMap.remove(jobID);
+  }
+
+  public static class ShuffleResourceListener {
+
+    private final JobShuffleContext context;
+    private final int shuffleId;
+    private final ExecutorService executorService;
+
+    // celeborn partitionId -> Flink ResultPartitionID
+    private Map<Integer, ResultPartitionID> resultPartitionMap = new ConcurrentHashMap<>();
+
+    public ShuffleResourceListener(
+        JobShuffleContext jobShuffleContext, int shuffleId, ExecutorService executorService) {
+      this.context = jobShuffleContext;
+      this.shuffleId = shuffleId;
+      this.executorService = executorService;
+    }
+
+    public void addPartitionResource(int partitionId, ResultPartitionID partitionID) {
+      resultPartitionMap.put(partitionId, partitionID);
+    }
+
+    public void notifyStopTrackingPartitions(Set<Integer> partitionIds) {
+      Set<ResultPartitionID> resultPartitionIDS = new HashSet<>();
+
+      for (Integer partitionId : partitionIds) {
+        ResultPartitionID resultPartitionID = resultPartitionMap.remove(partitionId);
+        if (resultPartitionID != null) {
+          resultPartitionIDS.add(resultPartitionID);
+        }
+      }
+
+      notifyStopTrackingPartitions(resultPartitionIDS, new AtomicInteger(MAX_RETRY_TIMES));

Review Comment:
   nit: IMO it's better to check null/empty here instead of in notifyStopTrackingPartitions to reduce function calls.



##########
client-flink/flink-1.14/src/main/java/org/apache/celeborn/plugin/flink/ShuffleResourceTracker.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.celeborn.plugin.flink;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.shuffle.JobShuffleContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.celeborn.client.LifecycleManager;
+
+public class ShuffleResourceTracker {
+  private static final Logger LOG = LoggerFactory.getLogger(ShuffleResourceTracker.class);
+  private final ExecutorService executorService;
+  private final Map<JobID, JobShuffleContext> jobShuffleContextMap = new ConcurrentHashMap<>();
+  // shuffleId -> ShuffleResourceListener
+  private final Map<Integer, ShuffleResourceListener> shuffleResourceListeners =
+      new ConcurrentHashMap<>();
+  private static final int MAX_RETRY_TIMES = 3;
+
+  public ShuffleResourceTracker(
+      ExecutorService executorService, LifecycleManager lifecycleManager) {
+    this.executorService = executorService;
+    ShuffleWorkerStatusListener shuffleWorkerStatusListener =
+        new ShuffleWorkerStatusListener(lifecycleManager, shuffleResourceListeners);
+    lifecycleManager.registerWorkerStatusListener(shuffleWorkerStatusListener);
+  }
+
+  public void registerJobContext(JobShuffleContext jobShuffleContext) {
+    jobShuffleContextMap.put(jobShuffleContext.getJobId(), jobShuffleContext);
+  }
+
+  public void addPartitionResource(
+      JobID jobId, int shuffleId, int partitionId, ResultPartitionID partitionID) {
+    JobShuffleContext jobShuffleContext = jobShuffleContextMap.get(jobId);
+    ShuffleResourceListener shuffleResourceListener =
+        shuffleResourceListeners.computeIfAbsent(
+            shuffleId,
+            (s) -> new ShuffleResourceListener(jobShuffleContext, shuffleId, executorService));
+    shuffleResourceListener.addPartitionResource(partitionId, partitionID);
+  }
+
+  public void removePartitionResource(int shuffleId, int partitionId) {
+    ShuffleResourceListener shuffleResourceListener = shuffleResourceListeners.get(shuffleId);
+    if (shuffleResourceListener != null) {
+      shuffleResourceListener.removePartitionResource(partitionId);
+    }
+  }
+
+  public void removeShuffleListener(int shuffleId) {
+    shuffleResourceListeners.remove(shuffleId);
+  }
+
+  public void unRegisterJob(JobID jobID) {
+    jobShuffleContextMap.remove(jobID);
+  }
+
+  public static class ShuffleResourceListener {
+
+    private final JobShuffleContext context;
+    private final int shuffleId;
+    private final ExecutorService executorService;
+
+    // celeborn partitionId -> Flink ResultPartitionID
+    private Map<Integer, ResultPartitionID> resultPartitionMap = new ConcurrentHashMap<>();
+
+    public ShuffleResourceListener(
+        JobShuffleContext jobShuffleContext, int shuffleId, ExecutorService executorService) {
+      this.context = jobShuffleContext;
+      this.shuffleId = shuffleId;
+      this.executorService = executorService;
+    }
+
+    public void addPartitionResource(int partitionId, ResultPartitionID partitionID) {
+      resultPartitionMap.put(partitionId, partitionID);
+    }
+
+    public void notifyStopTrackingPartitions(Set<Integer> partitionIds) {
+      Set<ResultPartitionID> resultPartitionIDS = new HashSet<>();
+
+      for (Integer partitionId : partitionIds) {
+        ResultPartitionID resultPartitionID = resultPartitionMap.remove(partitionId);
+        if (resultPartitionID != null) {
+          resultPartitionIDS.add(resultPartitionID);
+        }
+      }
+
+      notifyStopTrackingPartitions(resultPartitionIDS, new AtomicInteger(MAX_RETRY_TIMES));
+    }
+
+    private void notifyStopTrackingPartitions(
+        Set<ResultPartitionID> partitionIDS, AtomicInteger remainingRetries) {
+      if (partitionIDS == null || partitionIDS.isEmpty()) {
+        return;
+      }
+
+      LOG.debug(
+          "shuffleId: {}, stop tracking partitions {}.",
+          shuffleId,
+          Arrays.toString(partitionIDS.toArray()));
+
+      int count = remainingRetries.decrementAndGet();
+      try {
+        CompletableFuture<?> future = context.stopTrackingAndReleasePartitions(partitionIDS);
+        future.whenCompleteAsync(
+            (ignored, throwable) -> {
+              if (throwable == null) {
+                return;
+              }
+
+              if (count == 0) {
+                LOG.error(
+                    "shuffleId: {}, Failed to stop tracking partitions {}.",
+                    shuffleId,
+                    Arrays.toString(partitionIDS.toArray()));
+                return;
+              }
+              notifyStopTrackingPartitions(partitionIDS, remainingRetries);
+            },
+            executorService);
+      } catch (Throwable throwable) {

Review Comment:
   Seems the try-catch is unnecessary



##########
client-flink/flink-1.14/src/main/java/org/apache/celeborn/plugin/flink/ShuffleResourceTracker.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.celeborn.plugin.flink;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.shuffle.JobShuffleContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.celeborn.client.LifecycleManager;
+
+public class ShuffleResourceTracker {
+  private static final Logger LOG = LoggerFactory.getLogger(ShuffleResourceTracker.class);
+  private final ExecutorService executorService;
+  private final Map<JobID, JobShuffleContext> jobShuffleContextMap = new ConcurrentHashMap<>();
+  // shuffleId -> ShuffleResourceListener
+  private final Map<Integer, ShuffleResourceListener> shuffleResourceListeners =
+      new ConcurrentHashMap<>();
+  private static final int MAX_RETRY_TIMES = 3;
+
+  public ShuffleResourceTracker(
+      ExecutorService executorService, LifecycleManager lifecycleManager) {
+    this.executorService = executorService;
+    ShuffleWorkerStatusListener shuffleWorkerStatusListener =
+        new ShuffleWorkerStatusListener(lifecycleManager, shuffleResourceListeners);
+    lifecycleManager.registerWorkerStatusListener(shuffleWorkerStatusListener);
+  }
+
+  public void registerJobContext(JobShuffleContext jobShuffleContext) {
+    jobShuffleContextMap.put(jobShuffleContext.getJobId(), jobShuffleContext);
+  }
+
+  public void addPartitionResource(
+      JobID jobId, int shuffleId, int partitionId, ResultPartitionID partitionID) {
+    JobShuffleContext jobShuffleContext = jobShuffleContextMap.get(jobId);
+    ShuffleResourceListener shuffleResourceListener =
+        shuffleResourceListeners.computeIfAbsent(
+            shuffleId,
+            (s) -> new ShuffleResourceListener(jobShuffleContext, shuffleId, executorService));
+    shuffleResourceListener.addPartitionResource(partitionId, partitionID);
+  }
+
+  public void removePartitionResource(int shuffleId, int partitionId) {
+    ShuffleResourceListener shuffleResourceListener = shuffleResourceListeners.get(shuffleId);
+    if (shuffleResourceListener != null) {
+      shuffleResourceListener.removePartitionResource(partitionId);
+    }
+  }
+
+  public void removeShuffleListener(int shuffleId) {
+    shuffleResourceListeners.remove(shuffleId);
+  }
+
+  public void unRegisterJob(JobID jobID) {
+    jobShuffleContextMap.remove(jobID);
+  }
+
+  public static class ShuffleResourceListener {
+
+    private final JobShuffleContext context;
+    private final int shuffleId;
+    private final ExecutorService executorService;
+
+    // celeborn partitionId -> Flink ResultPartitionID
+    private Map<Integer, ResultPartitionID> resultPartitionMap = new ConcurrentHashMap<>();
+
+    public ShuffleResourceListener(
+        JobShuffleContext jobShuffleContext, int shuffleId, ExecutorService executorService) {
+      this.context = jobShuffleContext;
+      this.shuffleId = shuffleId;
+      this.executorService = executorService;
+    }
+
+    public void addPartitionResource(int partitionId, ResultPartitionID partitionID) {
+      resultPartitionMap.put(partitionId, partitionID);
+    }
+
+    public void notifyStopTrackingPartitions(Set<Integer> partitionIds) {
+      Set<ResultPartitionID> resultPartitionIDS = new HashSet<>();
+
+      for (Integer partitionId : partitionIds) {
+        ResultPartitionID resultPartitionID = resultPartitionMap.remove(partitionId);
+        if (resultPartitionID != null) {
+          resultPartitionIDS.add(resultPartitionID);
+        }
+      }
+
+      notifyStopTrackingPartitions(resultPartitionIDS, new AtomicInteger(MAX_RETRY_TIMES));
+    }
+
+    private void notifyStopTrackingPartitions(
+        Set<ResultPartitionID> partitionIDS, AtomicInteger remainingRetries) {

Review Comment:
   Seems it's unnecessary to use AtomicInteger because another try will only be triggered after the previous fails.



##########
client/src/main/scala/org/apache/celeborn/client/WorkerStatusTracker.scala:
##########
@@ -31,14 +33,26 @@ import org.apache.celeborn.common.protocol.message.StatusCode
 
 class WorkerStatusTracker(
     conf: CelebornConf,
-    lifecycleManager: LifecycleManager,
-    commitManager: CommitManager) extends Logging {
+    lifecycleManager: LifecycleManager) extends Logging {
   private val workerExcludedExpireTimeout = conf.workerExcludedExpireTimeout
+  private val workerStatusListeners = ConcurrentHashMap.newKeySet[WorkerStatusListener]()
 
   // blacklist
   val blacklist = new ShuffleFailedWorkers()
   private val shuttingWorkers: JSet[WorkerInfo] = new JHashSet[WorkerInfo]()
 
+  def registerWorkerStatusListener(workerStatusListener: WorkerStatusListener): Unit = {
+    workerStatusListeners.add(workerStatusListener)
+  }
+
+  def getCandidateBlackListWorkers(): Set[WorkerInfo] = {
+    if (conf.workerExcludedUseAllocatedWorkers) {

Review Comment:
   I don't quite understand why return all allocated workers here as blacklist



##########
client-flink/flink-1.14/src/main/java/org/apache/celeborn/plugin/flink/ShuffleWorkerStatusListener.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.celeborn.plugin.flink;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.celeborn.client.LifecycleManager;
+import org.apache.celeborn.client.listener.WorkerStatusListener;
+import org.apache.celeborn.client.listener.WorkersStatus;
+import org.apache.celeborn.common.meta.ShufflePartitionLocationInfo;
+import org.apache.celeborn.common.meta.WorkerInfo;
+import org.apache.celeborn.plugin.flink.ShuffleResourceTracker.ShuffleResourceListener;
+
+public class ShuffleWorkerStatusListener implements WorkerStatusListener {

Review Comment:
   IMO we should merge ShuffleWorkerStatusListener with ShuffleResourceTracker and make ShuffleResourceTracker a WorkerStatusListener. ShuffleWorkerStatusListener only has a reference to ShuffleResourceTracker's shuffleResourceListeners.



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

To unsubscribe, e-mail: issues-unsubscribe@celeborn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org