mridulm commented on code in PR #2373: URL: https://github.com/apache/celeborn/pull/2373#discussion_r1912557005
########## assets/spark-patch/Celeborn-Optimize-Skew-Partitions-spark3_2.patch: ########## @@ -0,0 +1,257 @@ +# 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/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +index e469c9989f2..245d9b3b9de 100644 +--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala ++++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +@@ -661,6 +661,8 @@ private[spark] class MapOutputTrackerMaster( + pool + } + ++ val skewShuffleIds = ConcurrentHashMap.newKeySet[Int]() ++ + // Make sure that we aren't going to exceed the max RPC message size by making sure + // we use broadcast to send large map output statuses. + if (minSizeForBroadcast > maxRpcMessageSize) { +@@ -839,6 +841,7 @@ private[spark] class MapOutputTrackerMaster( + shuffleStatus.invalidateSerializedMergeOutputStatusCache() + } + } ++ skewShuffleIds.remove(shuffleId) Review Comment: I would suggest introducing a new object which manages this stage - and delegate to it, instead of exposing global state and Celeborn related business logic through out spark. It will also make it explicitly clear to deployers what changes within Spark. A rough skeleton would be something like: ``` // or some such package org.apache.spark.celeborn object CelebornShuffleState { private val adaptivelyOptimizeSkewedPartitionReadEnabled = new AtomicBoolean() private val skewShuffleIds = ConcurrentHashMap.newKeySet[Int]() // call this from SparkEnv.create def init(env: SparkEnv): Unit = { // cleanup existing state (if required) - and initialize skewShuffleIds.clear() // use env.conf for all initialization, and not SQLConf adaptivelyOptimizeSkewedPartitionReadEnabled.set( env.conf.get("spark.shuffle.manager", "sort").contains("celeborn") && <etc> ) } def isAdaptivelyOptimizeSkewedPartitionReadEnabled = adaptivelyOptimizeSkewedPartitionReadEnabled.get() && SQLConf.get.celebornClientAdaptiveOptimizeSkewedPartitionReadEnabled def unregisterAdaptivelyOptimizedSkewedShuffle(shuffleId: Int): Unit = { skewShuffleIds.remove(shuffleId) } def isAdaptivelyOptimizedSkewedShuffle(shuffleId: Int): Boolean = { skewShuffleIds.contains(shuffleId) } } ``` We can evolve to using `CelebornShuffleState` not specifically for this PR, but any other state we need to maintain within Spark code (which might be used, without Celeborn jars, in some cases). Thoughts @waitinfuture, @RexXiong as well ? ########## client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/CelebornPartitionUtil.java: ########## @@ -0,0 +1,68 @@ +/* + * 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.spark.shuffle.celeborn; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; + +import org.apache.commons.lang3.tuple.Pair; + +import org.apache.celeborn.common.protocol.PartitionLocation; + +public class CelebornPartitionUtil { + public static Map<String, Pair<Integer, Integer>> splitSkewedPartitionLocations( + ArrayList<PartitionLocation> locations, int subPartitionSize, int subPartitionIndex) { + locations.sort(Comparator.comparing((PartitionLocation p) -> p.getUniqueId())); + long totalPartitionSize = + locations.stream().mapToLong((PartitionLocation p) -> p.getStorageInfo().fileSize).sum(); + long step = totalPartitionSize / subPartitionSize; + long startOffset = step * subPartitionIndex; + long endOffset = 0; + if (subPartitionIndex == subPartitionSize - 1) { + // last subPartition should include all remaining data + endOffset = totalPartitionSize + 1; + } else { + endOffset = step * (subPartitionIndex + 1); + } + + long partitionLocationOffset = 0; + Map<String, Pair<Integer, Integer>> chunkRange = new HashMap<>(); + for (int i = 0; i < locations.size(); i++) { + PartitionLocation p = locations.get(i); Review Comment: nit: ```suggestion for (PartitionLocation p : locations) { ``` ########## assets/spark-patch/Celeborn-Optimize-Skew-Partitions-spark3_2.patch: ########## @@ -0,0 +1,257 @@ +# 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/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +index e469c9989f2..245d9b3b9de 100644 +--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala ++++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +@@ -661,6 +661,8 @@ private[spark] class MapOutputTrackerMaster( + pool + } + ++ val skewShuffleIds = ConcurrentHashMap.newKeySet[Int]() ++ + // Make sure that we aren't going to exceed the max RPC message size by making sure + // we use broadcast to send large map output statuses. + if (minSizeForBroadcast > maxRpcMessageSize) { +@@ -839,6 +841,7 @@ private[spark] class MapOutputTrackerMaster( + shuffleStatus.invalidateSerializedMergeOutputStatusCache() + } + } ++ skewShuffleIds.remove(shuffleId) + } + + /** +diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +index b950c07f3d8..d081b4642c9 100644 +--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ++++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +@@ -1369,7 +1369,10 @@ private[spark] class DAGScheduler( + // The operation here can make sure for the partially completed intermediate stage, + // `findMissingPartitions()` returns all partitions every time. + stage match { +- case sms: ShuffleMapStage if stage.isIndeterminate && !sms.isAvailable => ++ case sms: ShuffleMapStage if (stage.isIndeterminate && !sms.isAvailable) || ++ mapOutputTracker.skewShuffleIds.contains(sms.shuffleDep.shuffleId) => ++ logInfo(s"Unregistering shuffle output for stage ${stage.id}" + ++ s" shuffle ${sms.shuffleDep.shuffleId}") + mapOutputTracker.unregisterAllMapAndMergeOutput(sms.shuffleDep.shuffleId) Review Comment: Thinking more, cant we not use something similar to `handleReportBarrierStageAttemptFailure` to handle this ? That is, when a stage attempt failure is reported for the current state attempt, and it is tracked in `skewShuffleIds`, invoke the callback to unregister its output as part of the stage attempt failure handling ? ########## client/src/main/java/org/apache/celeborn/client/ShuffleClient.java: ########## @@ -252,6 +257,8 @@ public abstract CelebornInputStream readPartition( ExceptionMaker exceptionMaker, ArrayList<PartitionLocation> locations, ArrayList<PbStreamHandler> streamHandlers, + Map<String, Set<PushFailedBatch>> failedBatchSetMap, Review Comment: I think this was discussed and I am missing some reference. Given the design is abandon the current stage (and the parent stage(s) shuffle output when relevant) - why are we tracking `PushFailedBatch` ? It will entirely get abandoned anyway, right ? ########## assets/spark-patch/Celeborn-Optimize-Skew-Partitions-spark3_2.patch: ########## @@ -0,0 +1,257 @@ +# 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/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +index e469c9989f2..245d9b3b9de 100644 +--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala ++++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +@@ -661,6 +661,8 @@ private[spark] class MapOutputTrackerMaster( + pool + } + ++ val skewShuffleIds = ConcurrentHashMap.newKeySet[Int]() ++ + // Make sure that we aren't going to exceed the max RPC message size by making sure + // we use broadcast to send large map output statuses. + if (minSizeForBroadcast > maxRpcMessageSize) { +@@ -839,6 +841,7 @@ private[spark] class MapOutputTrackerMaster( + shuffleStatus.invalidateSerializedMergeOutputStatusCache() + } + } ++ skewShuffleIds.remove(shuffleId) + } + + /** +diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +index b950c07f3d8..d081b4642c9 100644 +--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ++++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +@@ -1369,7 +1369,10 @@ private[spark] class DAGScheduler( + // The operation here can make sure for the partially completed intermediate stage, + // `findMissingPartitions()` returns all partitions every time. + stage match { +- case sms: ShuffleMapStage if stage.isIndeterminate && !sms.isAvailable => ++ case sms: ShuffleMapStage if (stage.isIndeterminate && !sms.isAvailable) || ++ mapOutputTracker.skewShuffleIds.contains(sms.shuffleDep.shuffleId) => ++ logInfo(s"Unregistering shuffle output for stage ${stage.id}" + ++ s" shuffle ${sms.shuffleDep.shuffleId}") + mapOutputTracker.unregisterAllMapAndMergeOutput(sms.shuffleDep.shuffleId) + sms.shuffleDep.newShuffleMergeState() + case _ => +@@ -1780,7 +1783,7 @@ private[spark] class DAGScheduler( + failedStage.failedAttemptIds.add(task.stageAttemptId) + val shouldAbortStage = + failedStage.failedAttemptIds.size >= maxConsecutiveStageAttempts || +- disallowStageRetryForTest ++ disallowStageRetryForTest || mapOutputTracker.skewShuffleIds.contains(shuffleId) + + // It is likely that we receive multiple FetchFailed for a single stage (because we have + // multiple tasks running concurrently on different executors). In that case, it is +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 6bc8ba4eebb..44db30dbaec 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 +@@ -3431,6 +3431,19 @@ object SQLConf { + .booleanConf + .createWithDefault(false) + ++ val CELEBORN_CLIENT_ADAPTIVE_OPTIMIZE_SKEWED_PARTITION_READ = ++ buildConf("spark.celeborn.client.adaptive.optimizeSkewedPartitionRead.enabled") ++ .version("3.0.0") ++ .booleanConf ++ .createWithDefault(false) ++ ++ val CELEBORN_STAGE_RERUN_ENABLED = ++ buildConf("spark.celeborn.client.spark.stageRerun.enabled") ++ .withAlternative("spark.celeborn.client.spark.fetch.throwsFetchFailure") ++ .version("3.0.0") ++ .booleanConf ++ .createWithDefault(false) Review Comment: Drop `CELEBORN_STAGE_RERUN_ENABLED` ? There is an existing config which applies, right ? (If this is discussed elsewhere, please do let me know) ########## assets/spark-patch/Celeborn-Optimize-Skew-Partitions-spark3_2.patch: ########## @@ -0,0 +1,257 @@ +# 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/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +index e469c9989f2..245d9b3b9de 100644 +--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala ++++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +@@ -661,6 +661,8 @@ private[spark] class MapOutputTrackerMaster( + pool + } + ++ val skewShuffleIds = ConcurrentHashMap.newKeySet[Int]() ++ + // Make sure that we aren't going to exceed the max RPC message size by making sure + // we use broadcast to send large map output statuses. + if (minSizeForBroadcast > maxRpcMessageSize) { +@@ -839,6 +841,7 @@ private[spark] class MapOutputTrackerMaster( + shuffleStatus.invalidateSerializedMergeOutputStatusCache() + } + } ++ skewShuffleIds.remove(shuffleId) + } + + /** +diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +index b950c07f3d8..d081b4642c9 100644 +--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ++++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +@@ -1369,7 +1369,10 @@ private[spark] class DAGScheduler( + // The operation here can make sure for the partially completed intermediate stage, + // `findMissingPartitions()` returns all partitions every time. + stage match { +- case sms: ShuffleMapStage if stage.isIndeterminate && !sms.isAvailable => ++ case sms: ShuffleMapStage if (stage.isIndeterminate && !sms.isAvailable) || ++ mapOutputTracker.skewShuffleIds.contains(sms.shuffleDep.shuffleId) => ++ logInfo(s"Unregistering shuffle output for stage ${stage.id}" + ++ s" shuffle ${sms.shuffleDep.shuffleId}") + mapOutputTracker.unregisterAllMapAndMergeOutput(sms.shuffleDep.shuffleId) + sms.shuffleDep.newShuffleMergeState() + case _ => +@@ -1780,7 +1783,7 @@ private[spark] class DAGScheduler( + failedStage.failedAttemptIds.add(task.stageAttemptId) + val shouldAbortStage = + failedStage.failedAttemptIds.size >= maxConsecutiveStageAttempts || +- disallowStageRetryForTest ++ disallowStageRetryForTest || mapOutputTracker.skewShuffleIds.contains(shuffleId) Review Comment: Why is the stage being aborted ? This will cause the job to fail. Do we have a test for this flow ? I would have expected it to fail, unless I am missing something ########## assets/spark-patch/Celeborn-Optimize-Skew-Partitions-spark3_2.patch: ########## @@ -0,0 +1,257 @@ +# 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/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +index e469c9989f2..245d9b3b9de 100644 +--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala ++++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +@@ -661,6 +661,8 @@ private[spark] class MapOutputTrackerMaster( + pool + } + ++ val skewShuffleIds = ConcurrentHashMap.newKeySet[Int]() ++ + // Make sure that we aren't going to exceed the max RPC message size by making sure + // we use broadcast to send large map output statuses. + if (minSizeForBroadcast > maxRpcMessageSize) { +@@ -839,6 +841,7 @@ private[spark] class MapOutputTrackerMaster( + shuffleStatus.invalidateSerializedMergeOutputStatusCache() + } + } ++ skewShuffleIds.remove(shuffleId) + } + + /** +diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +index b950c07f3d8..d081b4642c9 100644 +--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ++++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +@@ -1369,7 +1369,10 @@ private[spark] class DAGScheduler( + // The operation here can make sure for the partially completed intermediate stage, + // `findMissingPartitions()` returns all partitions every time. + stage match { +- case sms: ShuffleMapStage if stage.isIndeterminate && !sms.isAvailable => ++ case sms: ShuffleMapStage if (stage.isIndeterminate && !sms.isAvailable) || ++ mapOutputTracker.skewShuffleIds.contains(sms.shuffleDep.shuffleId) => ++ logInfo(s"Unregistering shuffle output for stage ${stage.id}" + ++ s" shuffle ${sms.shuffleDep.shuffleId}") + mapOutputTracker.unregisterAllMapAndMergeOutput(sms.shuffleDep.shuffleId) + sms.shuffleDep.newShuffleMergeState() + case _ => +@@ -1780,7 +1783,7 @@ private[spark] class DAGScheduler( + failedStage.failedAttemptIds.add(task.stageAttemptId) + val shouldAbortStage = + failedStage.failedAttemptIds.size >= maxConsecutiveStageAttempts || +- disallowStageRetryForTest ++ disallowStageRetryForTest || mapOutputTracker.skewShuffleIds.contains(shuffleId) + + // It is likely that we receive multiple FetchFailed for a single stage (because we have + // multiple tasks running concurrently on different executors). In that case, it is +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 6bc8ba4eebb..44db30dbaec 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 +@@ -3431,6 +3431,19 @@ object SQLConf { + .booleanConf + .createWithDefault(false) + ++ val CELEBORN_CLIENT_ADAPTIVE_OPTIMIZE_SKEWED_PARTITION_READ = ++ buildConf("spark.celeborn.client.adaptive.optimizeSkewedPartitionRead.enabled") Review Comment: I am wondering if this simply can be a `SparkConf` config instead. ########## client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/CelebornPartitionUtil.java: ########## @@ -0,0 +1,68 @@ +/* + * 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.spark.shuffle.celeborn; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; + +import org.apache.commons.lang3.tuple.Pair; + +import org.apache.celeborn.common.protocol.PartitionLocation; + +public class CelebornPartitionUtil { + public static Map<String, Pair<Integer, Integer>> splitSkewedPartitionLocations( + ArrayList<PartitionLocation> locations, int subPartitionSize, int subPartitionIndex) { + locations.sort(Comparator.comparing((PartitionLocation p) -> p.getUniqueId())); + long totalPartitionSize = + locations.stream().mapToLong((PartitionLocation p) -> p.getStorageInfo().fileSize).sum(); + long step = totalPartitionSize / subPartitionSize; + long startOffset = step * subPartitionIndex; + long endOffset = 0; + if (subPartitionIndex == subPartitionSize - 1) { + // last subPartition should include all remaining data + endOffset = totalPartitionSize + 1; + } else { + endOffset = step * (subPartitionIndex + 1); + } + + long partitionLocationOffset = 0; + Map<String, Pair<Integer, Integer>> chunkRange = new HashMap<>(); + for (int i = 0; i < locations.size(); i++) { + PartitionLocation p = locations.get(i); + int left = -1; + int right = -1; + // Start from index 1 since the first chunk offset is always 0. + for (int j = 1; j < p.getStorageInfo().getChunkOffsets().size(); j++) { + long currentOffset = partitionLocationOffset + p.getStorageInfo().getChunkOffsets().get(j); Review Comment: From a quick look, it appears that the current implementations in use might all be backed by an array - so the `get(j)` is cheap - but if that changes, this loop becomes O(n^2). One option, to be defensive, is to do something like: ``` int right = -1; int j = 0; Iterator<Long> chunkOffsets = p.getStorageInfo().getChunkOffsets().iterator(); // Skip first and start from index 1 since the first chunk offset is always 0. chunkOffsets.next(); j ++; while (chunkOffsets.hasNext()) { long currentOffset = partitionLocationOffset + chunkOffsets.next(); <snip> j ++; } ``` ########## client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/CelebornPartitionUtil.java: ########## @@ -0,0 +1,68 @@ +/* + * 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.spark.shuffle.celeborn; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; + +import org.apache.commons.lang3.tuple.Pair; + +import org.apache.celeborn.common.protocol.PartitionLocation; + +public class CelebornPartitionUtil { + public static Map<String, Pair<Integer, Integer>> splitSkewedPartitionLocations( + ArrayList<PartitionLocation> locations, int subPartitionSize, int subPartitionIndex) { + locations.sort(Comparator.comparing((PartitionLocation p) -> p.getUniqueId())); + long totalPartitionSize = + locations.stream().mapToLong((PartitionLocation p) -> p.getStorageInfo().fileSize).sum(); + long step = totalPartitionSize / subPartitionSize; + long startOffset = step * subPartitionIndex; + long endOffset = 0; + if (subPartitionIndex == subPartitionSize - 1) { + // last subPartition should include all remaining data + endOffset = totalPartitionSize + 1; + } else { + endOffset = step * (subPartitionIndex + 1); + } Review Comment: nit: ```suggestion final long endOffset = subPartitionIndex < subPartitionSize - 1 ? startOffset + step : // last subPartition should include all remaining data totalPartitionSize + 1; ``` ########## client/src/main/scala/org/apache/celeborn/client/ClientUtils.scala: ########## @@ -37,4 +39,20 @@ object ClientUtils { } true } + + /** + * If startMapIndex > endMapIndex, means partition is skew partition. + * locations will split to sub-partitions with startMapIndex size. + * + * @param conf cleborn conf + * @param startMapIndex shuffle start map index + * @param endMapIndex shuffle end map index + * @return true if read skew partition without map range + */ + def readSkewPartitionWithoutMapRange( + conf: CelebornConf, + startMapIndex: Int, + endMapIndex: Int): Boolean = { + conf.clientAdaptiveOptimizeSkewedPartitionReadEnabled && startMapIndex > endMapIndex Review Comment: QQ: Can `startMapIndex > endMapIndex` when `!clientAdaptiveOptimizeSkewedPartitionReadEnabled` ? -- 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]
