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

[GitHub] [incubator-celeborn] boneanxs commented on a diff in pull request #1167: [CELEBORN-227] Support different push strategies to control the push speed

boneanxs commented on code in PR #1167:
URL: https://github.com/apache/incubator-celeborn/pull/1167#discussion_r1098132784


##########
common/src/main/java/org/apache/celeborn/common/write/SlowStartPushStrategy.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.common.write;
+
+import java.io.IOException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.celeborn.common.CelebornConf;
+
+public class SlowStartPushStrategy extends PushStrategy {
+
+  protected static class CongestControlContext {
+    private final AtomicInteger currentMaxReqsInFlight;
+
+    // Indicate the number of congested times even after the in flight requests reduced to 1
+    private final AtomicInteger continueCongestedNumber;
+    private int congestionAvoidanceFlag;
+    private int reqsInFlightBlockThreshold;
+
+    public CongestControlContext(int reqsInFlightBlockThreshold) {
+      this.currentMaxReqsInFlight = new AtomicInteger(1);
+      this.continueCongestedNumber = new AtomicInteger(0);
+      this.congestionAvoidanceFlag = 0;
+      this.reqsInFlightBlockThreshold = reqsInFlightBlockThreshold;
+    }
+
+    public synchronized void increaseCurrentMaxReqs() {
+      continueCongestedNumber.set(0);
+      if (currentMaxReqsInFlight.get() >= reqsInFlightBlockThreshold) {
+        // Congestion avoidance
+        congestionAvoidanceFlag++;
+        if (congestionAvoidanceFlag >= currentMaxReqsInFlight.get()) {
+          currentMaxReqsInFlight.incrementAndGet();
+          congestionAvoidanceFlag = 0;
+        }
+      } else {
+        // Slow start
+        currentMaxReqsInFlight.incrementAndGet();
+      }
+    }
+
+    public synchronized void decreaseCurrentMaxReqs() {
+      if (currentMaxReqsInFlight.get() <= 1) {
+        currentMaxReqsInFlight.set(1);
+        continueCongestedNumber.incrementAndGet();
+      } else {
+        currentMaxReqsInFlight.updateAndGet(pre -> pre / 2);
+      }
+      reqsInFlightBlockThreshold = currentMaxReqsInFlight.get();
+      congestionAvoidanceFlag = 0;
+    }
+
+    public int getCurrentMaxReqsInFlight() {
+      return currentMaxReqsInFlight.get();
+    }
+
+    public int getContinueCongestedNumber() {
+      return continueCongestedNumber.get();
+    }
+  }
+
+  private static final Logger logger = LoggerFactory.getLogger(SlowStartPushStrategy.class);
+
+  private final int maxInFlight;
+  private final long initialSleepMills;
+  private final long maxSleepMills;
+  private final ConcurrentHashMap<String, CongestControlContext> congestControlInfoPerAddress;
+
+  public SlowStartPushStrategy(CelebornConf conf) {
+    super(conf);
+    this.maxInFlight = conf.pushMaxReqsInFlight();
+    this.initialSleepMills = conf.pushSlowStartInitialSleepTime();
+    this.maxSleepMills = conf.pushSlowStartMaxSleepMills();
+    this.congestControlInfoPerAddress = new ConcurrentHashMap<>();
+  }
+
+  @VisibleForTesting
+  protected CongestControlContext getCongestControlContextByAddress(String hostAndPushPort) {
+    return congestControlInfoPerAddress.computeIfAbsent(
+        hostAndPushPort, host -> new CongestControlContext(maxInFlight));
+  }
+
+  /**
+   * If `pushDataSlowStart` is enabled, will increase `currentMaxReqsInFlight` gradually to meet the
+   * max push speed.
+   *
+   * <p>1. slow start period: every RTT period, `currentMaxReqsInFlight` is doubled.

Review Comment:
   Yea, from @AngersZhuuuu's initial idea, here we define one RTT period means one batch of push data requests(batch size -> `currentMaxReqsInFlight`). So we plus 1 for every request, and for one batch request, the next batch will be increased to currentMaxReqsInFlight * currentMaxReqsInFlight, which is doubled.
   
   e.g.
   current currentMaxReqsInFlight is 1, the next maxReqs is 2
   current currentMaxReqsInFlight is 2, the next maxReqs is 4, etc...



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