mridulm commented on code in PR #2373:
URL: https://github.com/apache/celeborn/pull/2373#discussion_r1562071909


##########
assets/spark-patch/Celeborn-Optimize-Skew-Partitions-spark3_3.patch:
##########
@@ -0,0 +1,79 @@
+# 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.
+
+diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+index af03ad9a4cb..3b5c7ce4fce 100644
+--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
++++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+@@ -3784,6 +3784,12 @@ object SQLConf {
+     .booleanConf
+     .createWithDefault(false)
+ 
++  val CELEBORN_CLIENT_ADAPTIVE_OPTIMIZE_SKEWED_PARTITION_READ =
++    
buildConf("spark.celeborn.client.adaptive.optimizeSkewedPartitionRead.enabled")
++      .version("3.3.0")
++      .booleanConf
++      .createWithDefault(false)
++
+   /**
+    * Holds information about keys that have been deprecated.
+    *
+@@ -4549,6 +4555,9 @@ class SQLConf extends Serializable with Logging {
+   def histogramNumericPropagateInputType: Boolean =
+     getConf(SQLConf.HISTOGRAM_NUMERIC_PROPAGATE_INPUT_TYPE)
+ 
++  def celebornClientAdaptiveOptimizeSkewedPartitionReadEnabled: Boolean =
++    getConf(SQLConf.CELEBORN_CLIENT_ADAPTIVE_OPTIMIZE_SKEWED_PARTITION_READ)
++
+   /** ********************** SQLConf functionality methods ************ */
+ 
+   /** Set Spark SQL configuration properties. */
+diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala
+index af689db3379..38e54b3ed0a 100644
+--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala
++++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala
+@@ -22,6 +22,8 @@ import scala.collection.mutable.ArrayBuffer
+ import org.apache.spark.{MapOutputStatistics, MapOutputTrackerMaster, 
SparkEnv}
+ import org.apache.spark.internal.Logging
+ import org.apache.spark.sql.execution.{CoalescedPartitionSpec, 
PartialReducerPartitionSpec, ShufflePartitionSpec}
++import org.apache.spark.sql.internal.SQLConf
++import org.apache.spark.util.Utils
+ 
+ object ShufflePartitionsUtil extends Logging {
+   final val SMALL_PARTITION_FACTOR = 0.2
+@@ -387,6 +389,10 @@ object ShufflePartitionsUtil extends Logging {
+     val mapStartIndices = splitSizeListByTargetSize(
+       mapPartitionSizes, targetSize, smallPartitionFactor)
+     if (mapStartIndices.length > 1) {
++      // If Celeborn is enabled, split skew partitions without shuffle 
mapper-range reading
++      val isCelebornClientAdaptiveOptimizeSkewedPartitionReadEnabled = 
Utils.isCelebornEnabled(SparkEnv.get.conf) &&
++        SQLConf.get.celebornClientAdaptiveOptimizeSkewedPartitionReadEnabled
++
+       Some(mapStartIndices.indices.map { i =>
+         val startMapIndex = mapStartIndices(i)
+         val endMapIndex = if (i == mapStartIndices.length - 1) {
+@@ -400,7 +406,11 @@ object ShufflePartitionsUtil extends Logging {
+           dataSize += mapPartitionSizes(mapIndex)
+           mapIndex += 1
+         }
+-        PartialReducerPartitionSpec(reducerId, startMapIndex, endMapIndex, 
dataSize)
++        if (isCelebornClientAdaptiveOptimizeSkewedPartitionReadEnabled) {
++          PartialReducerPartitionSpec(reducerId, mapStartIndices.length, i, 
-1)

Review Comment:
   When `isCelebornClientAdaptiveOptimizeSkewedPartitionReadEnabled`, avoid the 
prefix computation here entirely ?
   That is
    ```
           var dataSize = 0L
           var mapIndex = startMapIndex
           while (mapIndex < endMapIndex) {
             dataSize += mapPartitionSizes(mapIndex)
             mapIndex += 1
           }
   ```



##########
client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java:
##########
@@ -1025,6 +1034,10 @@ public void onSuccess(ByteBuffer response) {
                       attemptId,
                       partitionId,
                       nextBatchId);
+                  if (dataPushFailureTrackingEnabled) {
+                    pushState.addFailedBatch(
+                        latest.getUniqueId(), new PushFailedBatch(mapId, 
attemptId, nextBatchId));

Review Comment:
   QQ: Do we want to track this here (and other `onFailure` below) ?
   This can simply be a retriable error right ?



##########
client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java:
##########
@@ -135,30 +136,37 @@ protected Compressor initialValue() {
 
   private final ReviveManager reviveManager;
 
+  private final boolean dataPushFailureTrackingEnabled;
+
   protected static class ReduceFileGroups {
     public Map<Integer, Set<PartitionLocation>> partitionGroups;
+    public Map<String, Set<PushFailedBatch>> pushFailedBatches;

Review Comment:
   QQ: This is essentially relevant only when hard_split's happen, right ?
   For all other cases, either we retry and it succeeds or it fails ?
   I want to make sure I am not missing some other cases.
   



##########
common/src/main/java/org/apache/celeborn/common/write/PushFailedBatch.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.Serializable;
+
+import com.google.common.base.Objects;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+
+public class PushFailedBatch implements Serializable {
+
+  private int mapId;
+  private int attemptId;
+  private int batchId;
+
+  public int getMapId() {
+    return mapId;
+  }
+
+  public void setMapId(int mapId) {
+    this.mapId = mapId;
+  }
+
+  public int getAttemptId() {
+    return attemptId;
+  }
+
+  public void setAttemptId(int attemptId) {
+    this.attemptId = attemptId;
+  }
+
+  public int getBatchId() {
+    return batchId;
+  }
+
+  public void setBatchId(int batchId) {
+    this.batchId = batchId;
+  }
+
+  public PushFailedBatch(int mapId, int attemptId, int batchId) {
+    this.mapId = mapId;
+    this.attemptId = attemptId;
+    this.batchId = batchId;
+  }

Review Comment:
   nit: move constructor to above get/set methods (after field definitions).



##########
common/src/main/scala/org/apache/celeborn/common/protocol/message/ControlMessages.scala:
##########
@@ -279,7 +281,9 @@ object ControlMessages extends Logging {
       status: StatusCode,
       fileGroup: util.Map[Integer, util.Set[PartitionLocation]],
       attempts: Array[Int],
-      partitionIds: util.Set[Integer] = new util.HashSet[Integer]())
+      partitionIds: util.Set[Integer] = new util.HashSet[Integer](),
+      pushFailedBatches: util.Map[String, util.Set[PushFailedBatch]] =
+        new util.HashMap[String, util.Set[PushFailedBatch]]())

Review Comment:
   ```suggestion
           Collections.emptyMap())
   ```



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to