From eb1be3fbf8078cc71e2208095931d9510bf00095 Mon Sep 17 00:00:00 2001 From: SteNicholas Date: Fri, 10 Nov 2023 11:44:47 +0800 Subject: [PATCH 01/31] [CELEBORN-1120] ShuffleClientImpl should close batchReviveRequestScheduler of ReviveManager ### What changes were proposed in this pull request? `ShuffleClientImpl` closes `batchReviveRequestScheduler` of `ReviveManager`. ### Why are the changes needed? After shuffle client is closed, `ReviveManager` still schedules invoker to `ShuffleClientImpl#reviveBatch`, which causes the `NullPointerException`. Therefore, `ShuffleClientImpl` should close `batchReviveRequestScheduler` of `ReviveManager` to avoid `NullPointerException`. ``` 23/11/08 18:09:25,819 [batch-revive-scheduler] ERROR ShuffleClientImpl: Exception raised while reviving for shuffle 0 partitionIds 1988, epochs 0,. java.lang.NullPointerException at org.apache.celeborn.client.ShuffleClientImpl.reviveBatch(ShuffleClientImpl.java:705) at org.apache.celeborn.client.ReviveManager.lambda$new$1(ReviveManager.java:94) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) 23/11/08 18:09:25,844 [celeborn-retry-sender-6] ERROR ShuffleClientImpl: Push data to xx.xx.xx.xx:9092 failed for shuffle 0 map 216 attempt 0 partition 1988 batch 2623, remain revive times 4. org.apache.celeborn.common.exception.CelebornIOException: PUSH_DATA_CONNECTION_EXCEPTION_PRIMARY then revive but REVIVE_FAILED, revive status 12(REVIVE_FAILED), old location: PartitionLocation[ id-epoch:1988-0 host-rpcPort-pushPort-fetchPort-replicatePort:xx.xx.xx.xx-9091-9092-9093-9094 mode:PRIMARY peer:(empty) storage hint:StorageInfo{type=MEMORY, mountPoint='/tmp/storage', finalResult=false, filePath=} mapIdBitMap:null] at org.apache.celeborn.client.ShuffleClientImpl.submitRetryPushData(ShuffleClientImpl.java:261) at org.apache.celeborn.client.ShuffleClientImpl.access$600(ShuffleClientImpl.java:62) at org.apache.celeborn.client.ShuffleClientImpl$3.lambda$onFailure$1(ShuffleClientImpl.java:1045) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? No. Closes #2084 from SteNicholas/CELEBORN-1120. Authored-by: SteNicholas Signed-off-by: Fu Chen --- .../apache/celeborn/client/ReviveManager.java | 11 +++++-- .../celeborn/client/ShuffleClientImpl.java | 33 ++++++++++--------- 2 files changed, 25 insertions(+), 19 deletions(-) diff --git a/client/src/main/java/org/apache/celeborn/client/ReviveManager.java b/client/src/main/java/org/apache/celeborn/client/ReviveManager.java index 47d87878d46..3c52e6b9f62 100644 --- a/client/src/main/java/org/apache/celeborn/client/ReviveManager.java +++ b/client/src/main/java/org/apache/celeborn/client/ReviveManager.java @@ -22,6 +22,8 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import scala.concurrent.duration.Duration; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,17 +37,16 @@ class ReviveManager { private static final Logger logger = LoggerFactory.getLogger(ReviveManager.class); LinkedBlockingQueue requestQueue = new LinkedBlockingQueue<>(); - private final long interval; private final int batchSize; ShuffleClientImpl shuffleClient; - private ScheduledExecutorService batchReviveRequestScheduler = + private final ScheduledExecutorService batchReviveRequestScheduler = ThreadUtils.newDaemonSingleThreadScheduledExecutor("batch-revive-scheduler"); public ReviveManager(ShuffleClientImpl shuffleClient, CelebornConf conf) { this.shuffleClient = shuffleClient; - this.interval = conf.clientPushReviveInterval(); this.batchSize = conf.clientPushReviveBatchSize(); + long interval = conf.clientPushReviveInterval(); batchReviveRequestScheduler.scheduleWithFixedDelay( () -> { Map> shuffleMap = new HashMap<>(); @@ -124,4 +125,8 @@ public void addRequest(ReviveRequest request) { logger.error("Exception when put into requests!", e); } } + + public void close() { + ThreadUtils.shutdown(batchReviveRequestScheduler, Duration.apply("800ms")); + } } diff --git a/client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java b/client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java index 39d382d1064..a300ddf5f86 100644 --- a/client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java +++ b/client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java @@ -73,8 +73,8 @@ public class ShuffleClientImpl extends ShuffleClient { private final int registerShuffleMaxRetries; private final long registerShuffleRetryWaitMs; - private int maxReviveTimes; - private boolean testRetryRevive; + private final int maxReviveTimes; + private final boolean testRetryRevive; private final int pushBufferMaxSize; protected final long pushDataTimeout; @@ -113,7 +113,7 @@ public class ShuffleClientImpl extends ShuffleClient { protected final String appUniqueId; - private ThreadLocal compressorThreadLocal = + private final ThreadLocal compressorThreadLocal = new ThreadLocal() { @Override protected Compressor initialValue() { @@ -601,13 +601,13 @@ protected void limitZeroInFlight(String mapKey, PushState pushState) throws IOEx } /** - * check if a newer PartitionLocation(with larger epoch) exists in local cache + * Check if a newer PartitionLocation(with larger epoch) exists in local cache. * - * @param shuffleMap - * @param partitionId - * @param epoch - * @param wait whether to wait for some time for a newer PartitionLocation - * @return + * @param shuffleMap The mapping between shuffle id and partition location. + * @param partitionId The id of partition. + * @param epoch The epoch of revive. + * @param wait Whether to wait for some time for a newer partition location. + * @return whether newer partition location exists in local cache. */ boolean newerPartitionLocationExists( Map shuffleMap, int partitionId, int epoch, boolean wait) { @@ -675,12 +675,10 @@ private boolean revive( attemptId, partitionId); return true; - } else if (results == null - || !results.containsKey(partitionId) - || results.get(partitionId) != StatusCode.SUCCESS.getValue()) { - return false; } else { - return true; + return results != null + && results.containsKey(partitionId) + && results.get(partitionId) == StatusCode.SUCCESS.getValue(); } } @@ -1595,7 +1593,7 @@ public CelebornInputStream readPartition( throws IOException { ReduceFileGroups fileGroups = loadFileGroup(shuffleId, partitionId); - if (fileGroups.partitionGroups.size() == 0 + if (fileGroups.partitionGroups.isEmpty() || !fileGroups.partitionGroups.containsKey(partitionId)) { logger.warn("Shuffle data is empty for shuffle {} partition {}.", shuffleId, partitionId); return CelebornInputStream.empty(); @@ -1622,6 +1620,9 @@ public Map getReduceFileGroupsMap() { @Override public void shutdown() { + if (null != reviveManager) { + reviveManager.close(); + } if (null != rpcEnv) { rpcEnv.shutdown(); } @@ -1666,7 +1667,7 @@ private StatusCode getPushDataFailCause(String message) { logger.debug("Push data failed cause message: {}", message); StatusCode cause; if (message == null) { - logger.error("Push data throw unexpected exception: {}", message); + logger.error("Push data throw unexpected exception"); cause = StatusCode.PUSH_DATA_FAIL_NON_CRITICAL_CAUSE; } else if (message.startsWith(StatusCode.PUSH_DATA_WRITE_FAIL_REPLICA.name())) { cause = StatusCode.PUSH_DATA_WRITE_FAIL_REPLICA; From c7c2f6a35ae0bf5119df5e87ef7c389a09886e59 Mon Sep 17 00:00:00 2001 From: Luke Yan <108530647+lukeyan2023@users.noreply.github.com> Date: Fri, 10 Nov 2023 15:35:54 +0800 Subject: [PATCH 02/31] [CELEBORN-858] Generate patch to each Spark 3.x minor version MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Add the following patch files in directory `incubator-celeborn/tree/spark3-patch/assets/spark-patch` : 1. Celeborn_Dynamic_Allocation_spark3_0.patch 2. Celeborn_Dynamic_Allocation_spark3_1.patch 3. Celeborn_Dynamic_Allocation_spark3_2.patch 4. Celeborn_Dynamic_Allocation_spark3_3.patch Delete a patch at the same time: 1. Celeborn_Dynamic_Allocation_spark3.patch Modified `Support Spark Dynamic Allocation` in incubator-celeborn/README.md : ![image](https://github.com/apache/incubator-celeborn/assets/108530647/61e2e69b-d3f5-4d11-a20b-374622936443) ### Why are the changes needed? Convenient for customers to apply patches in Spark 3.X for `Support Spark Dynamic Allocation` ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? yes. All patch files can be applied to the corresponding version of spark source code through `git apply` without any code conflicts. Closes #2085 from lukeyan2023/spark3-patch. Authored-by: Luke Yan <108530647+lukeyan2023@users.noreply.github.com> Signed-off-by: zky.zhoukeyong --- README.md | 5 +- ...Celeborn_Dynamic_Allocation_spark3_0.patch | 87 ++++++++++++++++++ ...Celeborn_Dynamic_Allocation_spark3_1.patch | 91 +++++++++++++++++++ ...eleborn_Dynamic_Allocation_spark3_2.patch} | 0 ...Celeborn_Dynamic_Allocation_spark3_3.patch | 88 ++++++++++++++++++ 5 files changed, 270 insertions(+), 1 deletion(-) create mode 100644 assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_0.patch create mode 100644 assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_1.patch rename assets/spark-patch/{Celeborn_Dynamic_Allocation_spark3.patch => Celeborn_Dynamic_Allocation_spark3_2.patch} (100%) create mode 100644 assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_3.patch diff --git a/README.md b/README.md index e1d06d177d4..c88cb05c3fa 100644 --- a/README.md +++ b/README.md @@ -333,7 +333,10 @@ for more details. For Spark versions < 3.5.0, we provide a patch to enable users to use Spark with DRA and Celeborn. - For Spark 2.x check [Spark2 Patch](assets/spark-patch/Celeborn_Dynamic_Allocation_spark2.patch). -- For Spark 3.0-3.3 check [Spark3 Patch](assets/spark-patch/Celeborn_Dynamic_Allocation_spark3.patch). +- For Spark 3.0 check [Spark3.0 Patch](assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_0.patch). +- For Spark 3.1 check [Spark3.1 Patch](assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_1.patch). +- For Spark 3.2 check [Spark3.2 Patch](assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_2.patch). +- For Spark 3.3 check [Spark3.3 Patch](assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_3.patch). - For Spark 3.4 check [Spark3.4 Patch](assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_4.patch). ### Metrics diff --git a/assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_0.patch b/assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_0.patch new file mode 100644 index 00000000000..59081a2e8c4 --- /dev/null +++ b/assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_0.patch @@ -0,0 +1,87 @@ +# 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. + +Subject: [PATCH] [CORE][SHUFFLE] Support enabling DRA with Apache Celeborn +--- +Index: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +IDEA additional info: +Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP +<+>UTF-8 +=================================================================== +diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala (revision 65ac1e75dc468f53fc778cd2ce1ba3f21067aab8) ++++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala (revision 1f5cb2ec9e9652d03c9775954b69a708b5d05ab3) +@@ -198,7 +198,7 @@ + if (!conf.get(config.SHUFFLE_SERVICE_ENABLED)) { + if (conf.get(config.DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED)) { + logWarning("Dynamic allocation without a shuffle service is an experimental feature.") +- } else if (!testing) { ++ } else if (!testing && !Utils.isCelebornEnabled(conf)) { + throw new SparkException("Dynamic allocation of executors requires the external " + + "shuffle service. You may enable this through spark.shuffle.service.enabled.") + } +Index: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +IDEA additional info: +Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP +<+>UTF-8 +=================================================================== +diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala (revision 65ac1e75dc468f53fc778cd2ce1ba3f21067aab8) ++++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala (revision 1f5cb2ec9e9652d03c9775954b69a708b5d05ab3) +@@ -1851,7 +1851,8 @@ + // if the cluster manager explicitly tells us that the entire worker was lost, then + // we know to unregister shuffle output. (Note that "worker" specifically refers to the process + // from a Standalone cluster, where the shuffle service lives in the Worker.) +- val fileLost = workerLost || !env.blockManager.externalShuffleServiceEnabled ++ val fileLost = !Utils.isCelebornEnabled(sc.getConf) && ++ (workerLost || !env.blockManager.externalShuffleServiceEnabled) + removeExecutorAndUnregisterOutputs( + execId = execId, + fileLost = fileLost, +Index: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +IDEA additional info: +Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP +<+>UTF-8 +=================================================================== +diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala (revision 65ac1e75dc468f53fc778cd2ce1ba3f21067aab8) ++++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala (revision 1f5cb2ec9e9652d03c9775954b69a708b5d05ab3) +@@ -944,7 +944,7 @@ + // The reason is the next stage wouldn't be able to fetch the data from this dead executor + // so we would need to rerun these tasks on other executors. + if (tasks(0).isInstanceOf[ShuffleMapTask] && !env.blockManager.externalShuffleServiceEnabled +- && !isZombie) { ++ && !isZombie && !Utils.isCelebornEnabled(conf)) { + for ((tid, info) <- taskInfos if info.executorId == execId) { + val index = taskInfos(tid).index + // We may have a running task whose partition has been marked as successful, +Index: core/src/main/scala/org/apache/spark/util/Utils.scala +IDEA additional info: +Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP +<+>UTF-8 +=================================================================== +diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala +--- a/core/src/main/scala/org/apache/spark/util/Utils.scala (revision 65ac1e75dc468f53fc778cd2ce1ba3f21067aab8) ++++ b/core/src/main/scala/org/apache/spark/util/Utils.scala (revision 1f5cb2ec9e9652d03c9775954b69a708b5d05ab3) +@@ -2934,6 +2934,9 @@ + props.forEach((k, v) => resultProps.put(k, v)) + resultProps + } ++ ++ def isCelebornEnabled(conf: SparkConf): Boolean = ++ conf.get("spark.shuffle.manager", "sort").contains("celeborn") + } + + private[util] object CallerContext extends Logging { diff --git a/assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_1.patch b/assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_1.patch new file mode 100644 index 00000000000..8d9bfd6a04a --- /dev/null +++ b/assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_1.patch @@ -0,0 +1,91 @@ +# 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. + +Subject: [PATCH] [CORE][SHUFFLE] Support enabling DRA with Apache Celeborn +--- +Index: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +IDEA additional info: +Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP +<+>UTF-8 +=================================================================== +diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala (revision d1f8a503a26bcfb4e466d9accc5fa241a7933667) ++++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala (revision c8609fa08534e1cc6fd377b3573e5e65ddc79f15) +@@ -210,7 +210,7 @@ + (decommissionEnabled && + conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED))) { + logWarning("Dynamic allocation without a shuffle service is an experimental feature.") +- } else if (!testing) { ++ } else if (!testing && !Utils.isCelebornEnabled(conf)) { + throw new SparkException("Dynamic allocation of executors requires the external " + + "shuffle service. You may enable this through spark.shuffle.service.enabled.") + } +Index: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +IDEA additional info: +Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP +<+>UTF-8 +=================================================================== +diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala (revision d1f8a503a26bcfb4e466d9accc5fa241a7933667) ++++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala (revision c8609fa08534e1cc6fd377b3573e5e65ddc79f15) +@@ -2080,7 +2080,8 @@ + // if the cluster manager explicitly tells us that the entire worker was lost, then + // we know to unregister shuffle output. (Note that "worker" specifically refers to the process + // from a Standalone cluster, where the shuffle service lives in the Worker.) +- val fileLost = workerHost.isDefined || !env.blockManager.externalShuffleServiceEnabled ++ val fileLost = !Utils.isCelebornEnabled(sc.getConf) && ++ (workerHost.isDefined || !env.blockManager.externalShuffleServiceEnabled) + removeExecutorAndUnregisterOutputs( + execId = execId, + fileLost = fileLost, +Index: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +IDEA additional info: +Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP +<+>UTF-8 +=================================================================== +diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala (revision d1f8a503a26bcfb4e466d9accc5fa241a7933667) ++++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala (revision c8609fa08534e1cc6fd377b3573e5e65ddc79f15) +@@ -973,7 +973,8 @@ + // and we are not using an external shuffle server which could serve the shuffle outputs. + // The reason is the next stage wouldn't be able to fetch the data from this dead executor + // so we would need to rerun these tasks on other executors. +- if (isShuffleMapTasks && !env.blockManager.externalShuffleServiceEnabled && !isZombie) { ++ if (isShuffleMapTasks && !env.blockManager.externalShuffleServiceEnabled && !isZombie && ++ !Utils.isCelebornEnabled(conf)) { + for ((tid, info) <- taskInfos if info.executorId == execId) { + val index = taskInfos(tid).index + // We may have a running task whose partition has been marked as successful, +Index: core/src/main/scala/org/apache/spark/util/Utils.scala +IDEA additional info: +Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP +<+>UTF-8 +=================================================================== +diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala +--- a/core/src/main/scala/org/apache/spark/util/Utils.scala (revision d1f8a503a26bcfb4e466d9accc5fa241a7933667) ++++ b/core/src/main/scala/org/apache/spark/util/Utils.scala (revision c8609fa08534e1cc6fd377b3573e5e65ddc79f15) +@@ -3057,7 +3057,12 @@ + 0 + } + } ++ ++ def isCelebornEnabled(conf: SparkConf): Boolean = ++ conf.get("spark.shuffle.manager", "sort").contains("celeborn") + } ++ ++ + + private[util] object CallerContext extends Logging { + val callerContextSupported: Boolean = { diff --git a/assets/spark-patch/Celeborn_Dynamic_Allocation_spark3.patch b/assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_2.patch similarity index 100% rename from assets/spark-patch/Celeborn_Dynamic_Allocation_spark3.patch rename to assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_2.patch diff --git a/assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_3.patch b/assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_3.patch new file mode 100644 index 00000000000..e998e2e5ff8 --- /dev/null +++ b/assets/spark-patch/Celeborn_Dynamic_Allocation_spark3_3.patch @@ -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. + +Subject: [PATCH] [CORE][SHUFFLE] Support enabling DRA with Apache Celeborn +--- +Index: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +IDEA additional info: +Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP +<+>UTF-8 +=================================================================== +diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala (revision 8c2b3319c6734250ff9d72f3d7e5cab56b142195) ++++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala (revision 5e318f8fbeadec03986b3ad9cad4c1cda33c9ed2) +@@ -209,7 +209,7 @@ + } else if (decommissionEnabled && + conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED)) { + logInfo("Shuffle data decommission is enabled without a shuffle service.") +- } else if (!testing) { ++ } else if (!testing && !Utils.isCelebornEnabled(conf)) { + throw new SparkException("Dynamic allocation of executors requires the external " + + "shuffle service. You may enable this through spark.shuffle.service.enabled.") + } +Index: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +IDEA additional info: +Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP +<+>UTF-8 +=================================================================== +diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala (revision 8c2b3319c6734250ff9d72f3d7e5cab56b142195) ++++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala (revision 5e318f8fbeadec03986b3ad9cad4c1cda33c9ed2) +@@ -2414,7 +2414,8 @@ + // if the cluster manager explicitly tells us that the entire worker was lost, then + // we know to unregister shuffle output. (Note that "worker" specifically refers to the process + // from a Standalone cluster, where the shuffle service lives in the Worker.) +- val fileLost = workerHost.isDefined || !env.blockManager.externalShuffleServiceEnabled ++ val fileLost = !Utils.isCelebornEnabled(sc.getConf) && ++ (workerHost.isDefined || !env.blockManager.externalShuffleServiceEnabled) + removeExecutorAndUnregisterOutputs( + execId = execId, + fileLost = fileLost, +Index: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +IDEA additional info: +Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP +<+>UTF-8 +=================================================================== +diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala (revision 8c2b3319c6734250ff9d72f3d7e5cab56b142195) ++++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala (revision 5e318f8fbeadec03986b3ad9cad4c1cda33c9ed2) +@@ -1032,7 +1032,8 @@ + // and we are not using an external shuffle server which could serve the shuffle outputs. + // The reason is the next stage wouldn't be able to fetch the data from this dead executor + // so we would need to rerun these tasks on other executors. +- if (isShuffleMapTasks && !env.blockManager.externalShuffleServiceEnabled && !isZombie) { ++ if (isShuffleMapTasks && !env.blockManager.externalShuffleServiceEnabled && !isZombie && ++ !Utils.isCelebornEnabled(conf)) { + for ((tid, info) <- taskInfos if info.executorId == execId) { + val index = info.index + // We may have a running task whose partition has been marked as successful, +Index: core/src/main/scala/org/apache/spark/util/Utils.scala +IDEA additional info: +Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP +<+>UTF-8 +=================================================================== +diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala +--- a/core/src/main/scala/org/apache/spark/util/Utils.scala (revision 8c2b3319c6734250ff9d72f3d7e5cab56b142195) ++++ b/core/src/main/scala/org/apache/spark/util/Utils.scala (revision 5e318f8fbeadec03986b3ad9cad4c1cda33c9ed2) +@@ -3246,6 +3246,9 @@ + files.toSeq + } + ++ def isCelebornEnabled(conf: SparkConf): Boolean = ++ conf.get("spark.shuffle.manager", "sort").contains("celeborn") ++ + /** + * Return the median number of a long array + * From ae3bbc50f49f574912ff9594142b8b306a97285f Mon Sep 17 00:00:00 2001 From: onebox-li Date: Fri, 10 Nov 2023 16:32:48 +0800 Subject: [PATCH 03/31] [CELEBORN-1114][FOLLOWUP] Make SlotsAllocated metrics panel to follow previous behavior ### What changes were proposed in this pull request? As title. ### Why are the changes needed? To avoid users being confused after upgrading. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual test. Closes #2087 from onebox-li/slots_allocated_metric_panel. Authored-by: onebox-li Signed-off-by: zky.zhoukeyong --- assets/grafana/celeborn-dashboard.json | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/assets/grafana/celeborn-dashboard.json b/assets/grafana/celeborn-dashboard.json index 22fd1c19b7d..1c1e58def30 100644 --- a/assets/grafana/celeborn-dashboard.json +++ b/assets/grafana/celeborn-dashboard.json @@ -1184,12 +1184,12 @@ "type": "prometheus", "uid": "${DS_PROMETHEUS}" }, - "expr": "metrics_SlotsAllocated_Count", + "expr": "increase(metrics_SlotsAllocated_Count[1h])", "legendFormat": "${baseLegend}", "refId": "A" } ], - "title": "metrics_SlotsAllocated_Count", + "title": "metrics_SlotsAllocated_increase_1h", "type": "timeseries" }, { From 3c6e4697f3270298ea3dc2549f2979be0854fbc4 Mon Sep 17 00:00:00 2001 From: SteNicholas Date: Fri, 10 Nov 2023 18:11:39 +0800 Subject: [PATCH 04/31] [CELEBORN-1110][FOLLOWUP] Support celeborn.worker.storage.disk.reserve.ratio to configure worker reserved ratio for each disk ### What changes were proposed in this pull request? Followup support `celeborn.worker.storage.disk.reserve.ratio` with `minimumUsableSize` cache in a variable instead of calculate for every pushdata. ### Why are the changes needed? Cache `minimumUsableSize` in a variable instead of calculate for every pushdata because `DiskUtils.getMinimumUsableSize` is costly. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? `SlotsAllocatorSuiteJ` Closes #2083 from SteNicholas/CELEBORN-1110. Authored-by: SteNicholas Signed-off-by: mingji --- .../service/deploy/worker/PushDataHandler.scala | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/PushDataHandler.scala b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/PushDataHandler.scala index d8f039ac165..d7b111eb32a 100644 --- a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/PushDataHandler.scala +++ b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/PushDataHandler.scala @@ -21,6 +21,8 @@ import java.nio.ByteBuffer import java.util.concurrent.{ConcurrentHashMap, ThreadPoolExecutor} import java.util.concurrent.atomic.{AtomicBoolean, AtomicIntegerArray} +import scala.collection.JavaConverters._ + import com.google.common.base.Throwables import com.google.protobuf.GeneratedMessageV3 import io.netty.buffer.ByteBuf @@ -55,6 +57,7 @@ class PushDataHandler(val workerSource: WorkerSource) extends BaseMessageHandler private var workerInfo: WorkerInfo = _ private var diskReserveSize: Long = _ private var diskReserveRatio: Option[Double] = _ + private var diskUsableSizes: Map[String, Long] = _ private var partitionSplitMinimumSize: Long = _ private var partitionSplitMaximumSize: Long = _ private var shutdown: AtomicBoolean = _ @@ -77,6 +80,9 @@ class PushDataHandler(val workerSource: WorkerSource) extends BaseMessageHandler workerInfo = worker.workerInfo diskReserveSize = worker.conf.workerDiskReserveSize diskReserveRatio = worker.conf.workerDiskReserveRatio + diskUsableSizes = workerInfo.diskInfos.asScala.map { case (mountPoint, diskInfo) => + (mountPoint, DiskUtils.getMinimumUsableSize(diskInfo, diskReserveSize, diskReserveRatio)) + }.toMap partitionSplitMinimumSize = worker.conf.partitionSplitMinimumSize partitionSplitMaximumSize = worker.conf.partitionSplitMaximumSize storageManager = worker.storageManager @@ -1195,14 +1201,10 @@ class PushDataHandler(val workerSource: WorkerSource) extends BaseMessageHandler if (fileWriter.flusher.isInstanceOf[HdfsFlusher]) { return false } - val diskInfo = workerInfo.diskInfos - .get(fileWriter.flusher.asInstanceOf[LocalFlusher].mountPoint) - - val minimumUsableSize = - DiskUtils.getMinimumUsableSize(diskInfo, diskReserveSize, diskReserveRatio) + val mountPoint = fileWriter.flusher.asInstanceOf[LocalFlusher].mountPoint + val diskInfo = workerInfo.diskInfos.get(mountPoint) val diskFull = diskInfo.status.equals( - DiskStatus.HIGH_DISK_USAGE) || diskInfo.actualUsableSpace < minimumUsableSize - + DiskStatus.HIGH_DISK_USAGE) || diskInfo.actualUsableSpace < diskUsableSizes(mountPoint) diskFull } From 65fb07e694e81acf4eaaa89c48790209b27d4d01 Mon Sep 17 00:00:00 2001 From: SteNicholas Date: Mon, 13 Nov 2023 17:31:44 +0800 Subject: [PATCH 05/31] [CELEBORN-1124] Exclude workers of shuffle manager remove worker of connect exception primary or replica ### What changes were proposed in this pull request? Exclude workers of shuffle manager remove worker of connect exception primary or replica. ### Why are the changes needed? Exclude workers of shuffle manager should not always remove worker of connect exception replica. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? No. Closes #2091 from SteNicholas/CELEBORN-1124. Authored-by: SteNicholas Signed-off-by: Fu Chen --- .../scala/org/apache/celeborn/client/WorkerStatusTracker.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/src/main/scala/org/apache/celeborn/client/WorkerStatusTracker.scala b/client/src/main/scala/org/apache/celeborn/client/WorkerStatusTracker.scala index 1a83d2a8fe2..ef570e787e5 100644 --- a/client/src/main/scala/org/apache/celeborn/client/WorkerStatusTracker.scala +++ b/client/src/main/scala/org/apache/celeborn/client/WorkerStatusTracker.scala @@ -165,7 +165,7 @@ class WorkerStatusTracker( StatusCode.RESERVE_SLOTS_FAILED | StatusCode.PUSH_DATA_CREATE_CONNECTION_FAIL_PRIMARY | StatusCode.PUSH_DATA_CREATE_CONNECTION_FAIL_REPLICA | - StatusCode.PUSH_DATA_CONNECTION_EXCEPTION_REPLICA | + StatusCode.PUSH_DATA_CONNECTION_EXCEPTION_PRIMARY | StatusCode.PUSH_DATA_CONNECTION_EXCEPTION_REPLICA | StatusCode.PUSH_DATA_TIMEOUT_PRIMARY | StatusCode.PUSH_DATA_TIMEOUT_REPLICA From f0d4319c1851c5cf5cbca2b6dc2a71a349a5137d Mon Sep 17 00:00:00 2001 From: Aravind Patnam Date: Mon, 13 Nov 2023 21:17:29 +0800 Subject: [PATCH 06/31] [CELEBORN-1106] Ensure data is written into flush buffer before sending message to client ### What changes were proposed in this pull request? The changes are to ensure that the data is at least written into the flush buffer before sending a message back to the client. Earlier, the message would be sent before this happens. ### Why are the changes needed? Changes are needed because currently the primary will send a response back to client before it is even written into the flush buffer to persist locally. We do this persist async. Additionally, this will prevent data corruption issues when data may not be present properly in primary but only on replica, but client fetches only from primary. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Will let CI run, and also tested on our internal cluster Closes #2064 from akpatnam25/CELEBORN-1106. Lead-authored-by: Aravind Patnam Co-authored-by: Aravind Patnam Signed-off-by: zky.zhoukeyong --- .../deploy/worker/PushDataHandler.scala | 350 +++++++++--------- 1 file changed, 185 insertions(+), 165 deletions(-) diff --git a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/PushDataHandler.scala b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/PushDataHandler.scala index d7b111eb32a..208ff8bcac6 100644 --- a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/PushDataHandler.scala +++ b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/PushDataHandler.scala @@ -22,6 +22,9 @@ import java.util.concurrent.{ConcurrentHashMap, ThreadPoolExecutor} import java.util.concurrent.atomic.{AtomicBoolean, AtomicIntegerArray} import scala.collection.JavaConverters._ +import scala.concurrent.{Await, Promise} +import scala.concurrent.duration.Duration +import scala.util.{Failure, Success, Try} import com.google.common.base.Throwables import com.google.protobuf.GeneratedMessageV3 @@ -254,7 +257,7 @@ class PushDataHandler(val workerSource: WorkerSource) extends BaseMessageHandler fileWriter.decrementPendingWrites() return; } - + val writePromise = Promise[Unit]() // for primary, send data to replica if (doReplicate) { pushData.body().retain() @@ -280,34 +283,38 @@ class PushDataHandler(val workerSource: WorkerSource) extends BaseMessageHandler // Handle the response from replica val wrappedCallback = new RpcResponseCallback() { override def onSuccess(response: ByteBuffer): Unit = { - if (response.remaining() > 0) { - val resp = ByteBuffer.allocate(response.remaining()) - resp.put(response) - resp.flip() - callbackWithTimer.onSuccess(resp) - } else if (softSplit.get()) { - // TODO Currently if the worker is in soft split status, given the guess that the client - // will fast stop pushing data to the worker, we won't return congest status. But - // in the long term, especially if this issue could frequently happen, we may need to return - // congest&softSplit status together - callbackWithTimer.onSuccess( - ByteBuffer.wrap(Array[Byte](StatusCode.SOFT_SPLIT.getValue))) - } else { - Option(CongestionController.instance()) match { - case Some(congestionController) => - if (congestionController.isUserCongested( - fileWriter.getFileInfo.getUserIdentifier)) { - // Check whether primary congest the data though the replicas doesn't congest - // it(the response is empty) - callbackWithTimer.onSuccess( - ByteBuffer.wrap( - Array[Byte](StatusCode.PUSH_DATA_SUCCESS_PRIMARY_CONGESTED.getValue))) - } else { - callbackWithTimer.onSuccess(ByteBuffer.wrap(Array[Byte]())) + Try(Await.result(writePromise.future, Duration.Inf)) match { + case Success(_) => + if (response.remaining() > 0) { + val resp = ByteBuffer.allocate(response.remaining()) + resp.put(response) + resp.flip() + callbackWithTimer.onSuccess(resp) + } else if (softSplit.get()) { + // TODO Currently if the worker is in soft split status, given the guess that the client + // will fast stop pushing data to the worker, we won't return congest status. But + // in the long term, especially if this issue could frequently happen, we may need to return + // congest&softSplit status together + callbackWithTimer.onSuccess( + ByteBuffer.wrap(Array[Byte](StatusCode.SOFT_SPLIT.getValue))) + } else { + Option(CongestionController.instance()) match { + case Some(congestionController) => + if (congestionController.isUserCongested( + fileWriter.getFileInfo.getUserIdentifier)) { + // Check whether primary congest the data though the replicas doesn't congest + // it(the response is empty) + callbackWithTimer.onSuccess( + ByteBuffer.wrap( + Array[Byte](StatusCode.PUSH_DATA_SUCCESS_PRIMARY_CONGESTED.getValue))) + } else { + callbackWithTimer.onSuccess(ByteBuffer.wrap(Array[Byte]())) + } + case None => + callbackWithTimer.onSuccess(ByteBuffer.wrap(Array[Byte]())) } - case None => - callbackWithTimer.onSuccess(ByteBuffer.wrap(Array[Byte]())) - } + } + case Failure(e) => callbackWithTimer.onFailure(e) } } @@ -350,6 +357,7 @@ class PushDataHandler(val workerSource: WorkerSource) extends BaseMessageHandler } } }) + writeLocalData(Seq(fileWriter), body, shuffleKey, isPrimary, None, writePromise) } else { // The codes here could be executed if // 1. the client doesn't enable push data to the replica, the primary worker could hit here @@ -358,47 +366,36 @@ class PushDataHandler(val workerSource: WorkerSource) extends BaseMessageHandler // will fast stop pushing data to the worker, we won't return congest status. But // in the long term, especially if this issue could frequently happen, we may need to return // congest&softSplit status together - if (softSplit.get()) { - callbackWithTimer.onSuccess(ByteBuffer.wrap(Array[Byte](StatusCode.SOFT_SPLIT.getValue))) - } else { - Option(CongestionController.instance()) match { - case Some(congestionController) => - if (congestionController.isUserCongested(fileWriter.getFileInfo.getUserIdentifier)) { - if (isPrimary) { - callbackWithTimer.onSuccess( - ByteBuffer.wrap( - Array[Byte](StatusCode.PUSH_DATA_SUCCESS_PRIMARY_CONGESTED.getValue))) - } else { - callbackWithTimer.onSuccess( - ByteBuffer.wrap( - Array[Byte](StatusCode.PUSH_DATA_SUCCESS_REPLICA_CONGESTED.getValue))) - } - } else { - callbackWithTimer.onSuccess(ByteBuffer.wrap(Array[Byte]())) + writeLocalData(Seq(fileWriter), body, shuffleKey, isPrimary, None, writePromise) + Try(Await.result(writePromise.future, Duration.Inf)) match { + case Success(_) => + if (softSplit.get()) { + callbackWithTimer.onSuccess( + ByteBuffer.wrap(Array[Byte](StatusCode.SOFT_SPLIT.getValue))) + } else { + Option(CongestionController.instance()) match { + case Some(congestionController) => + if (congestionController.isUserCongested( + fileWriter.getFileInfo.getUserIdentifier)) { + if (isPrimary) { + callbackWithTimer.onSuccess( + ByteBuffer.wrap( + Array[Byte](StatusCode.PUSH_DATA_SUCCESS_PRIMARY_CONGESTED.getValue))) + } else { + callbackWithTimer.onSuccess( + ByteBuffer.wrap( + Array[Byte](StatusCode.PUSH_DATA_SUCCESS_REPLICA_CONGESTED.getValue))) + } + } else { + callbackWithTimer.onSuccess(ByteBuffer.wrap(Array[Byte]())) + } + case None => + callbackWithTimer.onSuccess(ByteBuffer.wrap(Array[Byte]())) } - case None => - callbackWithTimer.onSuccess(ByteBuffer.wrap(Array[Byte]())) - } + } + case Failure(e) => callbackWithTimer.onFailure(e) } } - - try { - fileWriter.write(body) - } catch { - case e: AlreadyClosedException => - fileWriter.decrementPendingWrites() - val (mapId, attemptId) = getMapAttempt(body) - val endedAttempt = - if (shuffleMapperAttempts.containsKey(shuffleKey)) { - shuffleMapperAttempts.get(shuffleKey).get(mapId) - } else -1 - // TODO just info log for ended attempt - logError( - s"[handlePushData] Append data failed for task(shuffle $shuffleKey, map $mapId, attempt" + - s" $attemptId), caused by AlreadyClosedException, endedAttempt $endedAttempt, error message: ${e.getMessage}") - case e: Exception => - logError("Exception encountered when write.", e) - } } def handlePushMergedData( @@ -525,7 +522,7 @@ class PushDataHandler(val workerSource: WorkerSource) extends BaseMessageHandler fileWriters.foreach(_.decrementPendingWrites()) return } - + val writePromise = Promise[Unit]() // for primary, send data to replica if (doReplicate) { pushMergedData.body().retain() @@ -552,28 +549,32 @@ class PushDataHandler(val workerSource: WorkerSource) extends BaseMessageHandler // Handle the response from replica val wrappedCallback = new RpcResponseCallback() { override def onSuccess(response: ByteBuffer): Unit = { - // Only primary data enable replication will push data to replica - if (response.remaining() > 0) { - val resp = ByteBuffer.allocate(response.remaining()) - resp.put(response) - resp.flip() - callbackWithTimer.onSuccess(resp) - } else { - Option(CongestionController.instance()) match { - case Some(congestionController) if fileWriters.nonEmpty => - if (congestionController.isUserCongested( - fileWriters.head.getFileInfo.getUserIdentifier)) { - // Check whether primary congest the data though the replicas doesn't congest - // it(the response is empty) - callbackWithTimer.onSuccess( - ByteBuffer.wrap( - Array[Byte](StatusCode.PUSH_DATA_SUCCESS_PRIMARY_CONGESTED.getValue))) - } else { - callbackWithTimer.onSuccess(ByteBuffer.wrap(Array[Byte]())) + Try(Await.result(writePromise.future, Duration.Inf)) match { + case Success(_) => + // Only primary data enable replication will push data to replica + if (response.remaining() > 0) { + val resp = ByteBuffer.allocate(response.remaining()) + resp.put(response) + resp.flip() + callbackWithTimer.onSuccess(resp) + } else { + Option(CongestionController.instance()) match { + case Some(congestionController) if fileWriters.nonEmpty => + if (congestionController.isUserCongested( + fileWriters.head.getFileInfo.getUserIdentifier)) { + // Check whether primary congest the data though the replicas doesn't congest + // it(the response is empty) + callbackWithTimer.onSuccess( + ByteBuffer.wrap( + Array[Byte](StatusCode.PUSH_DATA_SUCCESS_PRIMARY_CONGESTED.getValue))) + } else { + callbackWithTimer.onSuccess(ByteBuffer.wrap(Array[Byte]())) + } + case None => + callbackWithTimer.onSuccess(ByteBuffer.wrap(Array[Byte]())) } - case None => - callbackWithTimer.onSuccess(ByteBuffer.wrap(Array[Byte]())) - } + } + case Failure(e) => callbackWithTimer.onFailure(e) } } @@ -621,69 +622,36 @@ class PushDataHandler(val workerSource: WorkerSource) extends BaseMessageHandler } } }) + writeLocalData(fileWriters, body, shuffleKey, isPrimary, Some(batchOffsets), writePromise) } else { // The codes here could be executed if // 1. the client doesn't enable push data to the replica, the primary worker could hit here // 2. the client enables push data to the replica, and the replica worker could hit here - Option(CongestionController.instance()) match { - case Some(congestionController) if fileWriters.nonEmpty => - if (congestionController.isUserCongested( - fileWriters.head.getFileInfo.getUserIdentifier)) { - if (isPrimary) { - callbackWithTimer.onSuccess( - ByteBuffer.wrap( - Array[Byte](StatusCode.PUSH_DATA_SUCCESS_PRIMARY_CONGESTED.getValue))) - } else { - callbackWithTimer.onSuccess( - ByteBuffer.wrap( - Array[Byte](StatusCode.PUSH_DATA_SUCCESS_REPLICA_CONGESTED.getValue))) - } - } else { - callbackWithTimer.onSuccess(ByteBuffer.wrap(Array[Byte]())) + writeLocalData(fileWriters, body, shuffleKey, isPrimary, Some(batchOffsets), writePromise) + Try(Await.result(writePromise.future, Duration.Inf)) match { + case Success(_) => + Option(CongestionController.instance()) match { + case Some(congestionController) if fileWriters.nonEmpty => + if (congestionController.isUserCongested( + fileWriters.head.getFileInfo.getUserIdentifier)) { + if (isPrimary) { + callbackWithTimer.onSuccess( + ByteBuffer.wrap( + Array[Byte](StatusCode.PUSH_DATA_SUCCESS_PRIMARY_CONGESTED.getValue))) + } else { + callbackWithTimer.onSuccess( + ByteBuffer.wrap( + Array[Byte](StatusCode.PUSH_DATA_SUCCESS_REPLICA_CONGESTED.getValue))) + } + } else { + callbackWithTimer.onSuccess(ByteBuffer.wrap(Array[Byte]())) + } + case None => + callbackWithTimer.onSuccess(ByteBuffer.wrap(Array[Byte]())) } - case None => - callbackWithTimer.onSuccess(ByteBuffer.wrap(Array[Byte]())) + case Failure(e) => callbackWithTimer.onFailure(e) } } - - index = 0 - var fileWriter: FileWriter = null - var alreadyClosed = false - while (index < fileWriters.length) { - fileWriter = fileWriters(index) - val offset = body.readerIndex() + batchOffsets(index) - val length = - if (index == fileWriters.length - 1) { - body.readableBytes() - batchOffsets(index) - } else { - batchOffsets(index + 1) - batchOffsets(index) - } - val batchBody = body.slice(offset, length) - - try { - if (!alreadyClosed) { - fileWriter.write(batchBody) - } else { - fileWriter.decrementPendingWrites() - } - } catch { - case e: AlreadyClosedException => - fileWriter.decrementPendingWrites() - alreadyClosed = true - val (mapId, attemptId) = getMapAttempt(body) - val endedAttempt = - if (shuffleMapperAttempts.containsKey(shuffleKey)) { - shuffleMapperAttempts.get(shuffleKey).get(mapId) - } else -1 - // TODO just info log for ended attempt - logError( - s"[handlePushMergedData] Append data failed for task(shuffle $shuffleKey, map $mapId, attempt" + - s" $attemptId), caused by AlreadyClosedException, endedAttempt $endedAttempt, error message: ${e.getMessage}") - case e: Exception => - logError("Exception encountered when write.", e) - } - index += 1 - } } /** @@ -827,31 +795,20 @@ class PushDataHandler(val workerSource: WorkerSource) extends BaseMessageHandler fileWriter.decrementPendingWrites() return; } - + val writePromise = Promise[Unit]() + writeLocalData(Seq(fileWriter), body, shuffleKey, isPrimary, None, writePromise) // for primary, send data to replica if (location.hasPeer && isPrimary) { // to do - wrappedCallback.onSuccess(ByteBuffer.wrap(Array[Byte]())) + Try(Await.result(writePromise.future, Duration.Inf)) match { + case Success(_) => wrappedCallback.onSuccess(ByteBuffer.wrap(Array[Byte]())) + case Failure(e) => wrappedCallback.onFailure(e) + } } else { - wrappedCallback.onSuccess(ByteBuffer.wrap(Array[Byte]())) - } - - try { - fileWriter.write(body) - } catch { - case e: AlreadyClosedException => - fileWriter.decrementPendingWrites() - val (mapId, attemptId) = getMapAttempt(body) - val endedAttempt = - if (shuffleMapperAttempts.containsKey(shuffleKey)) { - shuffleMapperAttempts.get(shuffleKey).get(mapId) - } else -1 - // TODO just info log for ended attempt - logError( - s"[handleMapPartitionPushData] Append data failed for task(shuffle $shuffleKey, map $mapId, attempt" + - s" $attemptId), caused by AlreadyClosedException, endedAttempt $endedAttempt, error message: ${e.getMessage}") - case e: Exception => - logError("Exception encountered when write.", e) + Try(Await.result(writePromise.future, Duration.Inf)) match { + case Success(_) => wrappedCallback.onSuccess(ByteBuffer.wrap(Array[Byte]())) + case Failure(e) => wrappedCallback.onFailure(e) + } } } @@ -1253,6 +1210,69 @@ class PushDataHandler(val workerSource: WorkerSource) extends BaseMessageHandler } } + private def writeLocalData( + fileWriters: Seq[FileWriter], + body: ByteBuf, + shuffleKey: String, + isPrimary: Boolean, + batchOffsets: Option[Array[Int]], + writePromise: Promise[Unit]): Unit = { + def writeData(fileWriter: FileWriter, body: ByteBuf, shuffleKey: String): Unit = { + try { + fileWriter.write(body) + } catch { + case e: Exception => + if (e.isInstanceOf[AlreadyClosedException]) { + val (mapId, attemptId) = getMapAttempt(body) + val endedAttempt = + if (shuffleMapperAttempts.containsKey(shuffleKey)) { + shuffleMapperAttempts.get(shuffleKey).get(mapId) + } else -1 + // TODO just info log for ended attempt + logWarning(s"Append data failed for task(shuffle $shuffleKey, map $mapId, attempt" + + s" $attemptId), caused by AlreadyClosedException, endedAttempt $endedAttempt, error message: ${e.getMessage}") + } else { + logError("Exception encountered when write.", e) + } + val cause = + if (isPrimary) { + StatusCode.PUSH_DATA_WRITE_FAIL_PRIMARY + } else { + StatusCode.PUSH_DATA_WRITE_FAIL_REPLICA + } + writePromise.failure(new CelebornIOException(cause)) + fileWriter.decrementPendingWrites() + } + } + batchOffsets match { + case Some(batchOffsets) => + var index = 0 + var fileWriter: FileWriter = null + while (index < fileWriters.length) { + if (!writePromise.isCompleted) { + fileWriter = fileWriters(index) + val offset = body.readerIndex() + batchOffsets(index) + val length = + if (index == fileWriters.length - 1) { + body.readableBytes() - batchOffsets(index) + } else { + batchOffsets(index + 1) - batchOffsets(index) + } + val batchBody = body.slice(offset, length) + writeData(fileWriter, batchBody, shuffleKey) + } else { + fileWriter.decrementPendingWrites() + } + index += 1 + } + case _ => + writeData(fileWriters.head, body, shuffleKey) + } + if (!writePromise.isCompleted) { + writePromise.success() + } + } + /** * Invoked when the channel associated with the given client is active. */ From 7263f64f234627a50d911575cb2d7a089a62c14d Mon Sep 17 00:00:00 2001 From: Marwan Salem Date: Tue, 14 Nov 2023 11:48:43 +0800 Subject: [PATCH 07/31] [CELEBORN-1126] Set kubernetes resources field for master and worker init container for helm chart ### What changes were proposed in this pull request? For the `helm` chart. I specified the resources field for the `initContainers` for `worker` and `master` statefulsets. I used the same values which are specified for the "main" container ### Why are the changes needed? For users that have a `ResourceQuota` (such as myself), worker and master pods do not start since the initContainers for the statefulsets do not specify the resources (cpu/ memory requests and limits). ### Does this PR introduce _any_ user-facing change? ### [Issue](https://github.com/apache/incubator-celeborn/issues/2094) ### How was this patch tested? I installed the chart on my GKE cluster. Closes #2093 from marwansalem/set-k8s-resources-for-init-containers. Authored-by: Marwan Salem Signed-off-by: Shuang --- charts/celeborn/templates/master-statefulset.yaml | 2 ++ charts/celeborn/templates/worker-statefulset.yaml | 2 ++ 2 files changed, 4 insertions(+) diff --git a/charts/celeborn/templates/master-statefulset.yaml b/charts/celeborn/templates/master-statefulset.yaml index fbd10192988..e988783f57b 100644 --- a/charts/celeborn/templates/master-statefulset.yaml +++ b/charts/celeborn/templates/master-statefulset.yaml @@ -88,6 +88,8 @@ spec: - chown - {{ .Values.securityContext.runAsUser | default 10006 }}:{{ .Values.securityContext.runAsGroup | default 10006 }} - {{ (index $dirs 0).mountPath }} + resources: + {{- toYaml .Values.resources.master | nindent 12 }} volumeMounts: - name: {{ $.Release.Name }}-master-vol-0 mountPath: {{ (index $dirs 0).mountPath }} diff --git a/charts/celeborn/templates/worker-statefulset.yaml b/charts/celeborn/templates/worker-statefulset.yaml index 11efd50de14..a9a003adc89 100644 --- a/charts/celeborn/templates/worker-statefulset.yaml +++ b/charts/celeborn/templates/worker-statefulset.yaml @@ -90,6 +90,8 @@ spec: {{- range $dir := $dirs }} - {{ $dir.mountPath }} {{- end}} + resources: + {{- toYaml .Values.resources.worker | nindent 12 }} volumeMounts: {{- range $index, $dir := $dirs }} - name: {{ $.Release.Name }}-worker-vol-{{ $index }} From 69e14fd3419f2d8b61f1a9f09f7a9ce0c452d861 Mon Sep 17 00:00:00 2001 From: liangyongyuan Date: Wed, 15 Nov 2023 19:48:39 +0800 Subject: [PATCH 08/31] [CELEBORN-1128] Fix incorrect method reference in ConcurrentHashMap.contains ### What changes were proposed in this pull request? ConcurrentHashMap.contains main containsValue ,not containsKey. In the current codebase, there is a misuse of the contains method in the ConcurrentHashMap class. ### Why are the changes needed? ConcurrentHashMap.contains misuse ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? No Closes #2102 from lyy-pineapple/hashMap. Authored-by: liangyongyuan Signed-off-by: zky.zhoukeyong --- .../org/apache/celeborn/client/WorkerStatusTrackerSuite.scala | 4 ++-- .../apache/celeborn/service/deploy/worker/Controller.scala | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/client/src/test/scala/org/apache/celeborn/client/WorkerStatusTrackerSuite.scala b/client/src/test/scala/org/apache/celeborn/client/WorkerStatusTrackerSuite.scala index 5d3355a33bc..1f606dd3280 100644 --- a/client/src/test/scala/org/apache/celeborn/client/WorkerStatusTrackerSuite.scala +++ b/client/src/test/scala/org/apache/celeborn/client/WorkerStatusTrackerSuite.scala @@ -61,13 +61,13 @@ class WorkerStatusTrackerSuite extends CelebornFunSuite { // test new added workers Assert.assertTrue(statusTracker.excludedWorkers.containsKey(mock("host0"))) Assert.assertTrue(statusTracker.excludedWorkers.containsKey(mock("host3"))) - Assert.assertTrue(!statusTracker.excludedWorkers.contains(mock("host4"))) + Assert.assertTrue(!statusTracker.excludedWorkers.containsKey(mock("host4"))) Assert.assertTrue(statusTracker.shuttingWorkers.contains(mock("host4"))) // test re heartbeat with shutdown workers val response3 = buildResponse(Array.empty, Array.empty, Array("host4")) statusTracker.handleHeartbeatResponse(response3) - Assert.assertTrue(!statusTracker.excludedWorkers.contains(mock("host4"))) + Assert.assertTrue(!statusTracker.excludedWorkers.containsKey(mock("host4"))) Assert.assertTrue(statusTracker.shuttingWorkers.contains(mock("host4"))) // test remove diff --git a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/Controller.scala b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/Controller.scala index 8418d69d638..d55b7dadb8c 100644 --- a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/Controller.scala +++ b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/Controller.scala @@ -353,7 +353,8 @@ private[deploy] class Controller( epoch: Long): Unit = { def alreadyCommitted(shuffleKey: String, epoch: Long): Boolean = { - shuffleCommitInfos.contains(shuffleKey) && shuffleCommitInfos.get(shuffleKey).contains(epoch) + shuffleCommitInfos.containsKey(shuffleKey) && shuffleCommitInfos.get(shuffleKey).containsKey( + epoch) } // Reply SHUFFLE_NOT_REGISTERED if shuffleKey does not exist AND the shuffle is not committed. From 12d60522394607691ed066e2fd065db43a45a8e2 Mon Sep 17 00:00:00 2001 From: "zky.zhoukeyong" Date: Wed, 15 Nov 2023 22:12:38 +0800 Subject: [PATCH 09/31] [CELEBORN-1130] LifecycleManager#requestWorkerReserveSlots should check null for endpoint ### What changes were proposed in this pull request? When I kill -9 a Worker process, Master will not exclude the worker until heartbeat timeout. During this time, Master will still allocate slots on this Worker, causing NPE when register shuffle ``` Caused by: java.lang.NullPointerException at org.apache.celeborn.client.LifecycleManager.requestWorkerReserveSlots(LifecycleManager.scala:1246) ~[celeborn-client-spark-3-shaded_2.12-0.4.0-SNAPSHOT.jar:?] at org.apache.celeborn.client.LifecycleManager.$anonfun$reserveSlots$2(LifecycleManager.scala:864) ~[celeborn-client-spark-3-shaded_2.12-0.4.0-SNAPSHOT.jar:?] at org.apache.celeborn.common.util.ThreadUtils$.$anonfun$parmap$2(ThreadUtils.scala:301) ~[celeborn-client-spark-3-shaded_2.12-0.4.0-SNAPSHOT.jar:?] at scala.concurrent.Future$.$anonfun$apply$1(Future.scala:659) ~[scala-library-2.12.15.jar:?] at scala.util.Success.$anonfun$map$1(Try.scala:255) ~[scala-library-2.12.15.jar:?] at scala.util.Success.map(Try.scala:213) ~[scala-library-2.12.15.jar:?] at scala.concurrent.Future.$anonfun$map$1(Future.scala:292) ~[scala-library-2.12.15.jar:?] at scala.concurrent.impl.Promise.liftedTree1$1(Promise.scala:33) ~[scala-library-2.12.15.jar:?] at scala.concurrent.impl.Promise.$anonfun$transform$1(Promise.scala:33) ~[scala-library-2.12.15.jar:?] at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:64) ~[scala-library-2.12.15.jar:?] at java.util.concurrent.ForkJoinTask$RunnableExecuteAction.exec(ForkJoinTask.java:1402) ~[?:1.8.0_372] at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289) ~[?:1.8.0_372] at java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056) ~[?:1.8.0_372] at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692) ~[?:1.8.0_372] at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:175) ~[?:1.8.0_372] ``` ### Why are the changes needed? ditto ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manual test and passes GA Closes #2104 from waitinfuture/1130. Authored-by: zky.zhoukeyong Signed-off-by: zky.zhoukeyong --- .../celeborn/client/LifecycleManager.scala | 40 +++++++++++-------- 1 file changed, 24 insertions(+), 16 deletions(-) diff --git a/client/src/main/scala/org/apache/celeborn/client/LifecycleManager.scala b/client/src/main/scala/org/apache/celeborn/client/LifecycleManager.scala index 9f0ca9da47e..cf54dfcd804 100644 --- a/client/src/main/scala/org/apache/celeborn/client/LifecycleManager.scala +++ b/client/src/main/scala/org/apache/celeborn/client/LifecycleManager.scala @@ -21,6 +21,8 @@ import java.nio.ByteBuffer import java.util import java.util.{function, List => JList} import java.util.concurrent.{Callable, ConcurrentHashMap, ScheduledFuture, TimeUnit} +import java.util.concurrent.atomic.AtomicInteger +import java.util.function.Consumer import scala.collection.JavaConverters._ import scala.collection.mutable @@ -33,6 +35,7 @@ import org.apache.celeborn.client.LifecycleManager.{ShuffleAllocatedWorkers, Shu import org.apache.celeborn.client.listener.WorkerStatusListener import org.apache.celeborn.common.CelebornConf import org.apache.celeborn.common.client.MasterClient +import org.apache.celeborn.common.exception.CelebornIOException import org.apache.celeborn.common.identity.{IdentityProvider, UserIdentifier} import org.apache.celeborn.common.internal.Logging import org.apache.celeborn.common.meta.{ShufflePartitionLocationInfo, WorkerInfo} @@ -720,22 +723,27 @@ class LifecycleManager(val appUniqueId: String, val conf: CelebornConf) extends Math.min(Math.max(1, workerPartitionLocations.size), conf.clientRpcMaxParallelism) ThreadUtils.parmap(workerPartitionLocations, "ReserveSlot", parallelism) { case (workerInfo, (primaryLocations, replicaLocations)) => - val res = requestWorkerReserveSlots( - workerInfo.endpoint, - ReserveSlots( - appUniqueId, - shuffleId, - primaryLocations, - replicaLocations, - partitionSplitThreshold, - partitionSplitMode, - getPartitionType(shuffleId), - rangeReadFilter, - userIdentifier, - conf.pushDataTimeoutMs, - if (getPartitionType(shuffleId) == PartitionType.MAP) - conf.clientShuffleMapPartitionSplitEnabled - else true)) + val res = + if (workerInfo.endpoint == null) { + ReserveSlotsResponse(StatusCode.REQUEST_FAILED, s"$workerInfo endpoint is NULL!") + } else { + requestWorkerReserveSlots( + workerInfo.endpoint, + ReserveSlots( + appUniqueId, + shuffleId, + primaryLocations, + replicaLocations, + partitionSplitThreshold, + partitionSplitMode, + getPartitionType(shuffleId), + rangeReadFilter, + userIdentifier, + conf.pushDataTimeoutMs, + if (getPartitionType(shuffleId) == PartitionType.MAP) + conf.clientShuffleMapPartitionSplitEnabled + else true)) + } if (res.status.equals(StatusCode.SUCCESS)) { logDebug(s"Successfully allocated " + s"partitions buffer for shuffleId $shuffleId" + From 758018f5125187f467a4889f8aa7aa48e2977aa2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=90=B4=E7=A5=A5=E5=B9=B3?= Date: Wed, 15 Nov 2023 22:30:39 +0800 Subject: [PATCH 10/31] [CELEBORN-1129] More easy to dedicate createReaderWithRetry error MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Add lastException to CelebornIOException when createReaderWithRetry meet error ### Why are the changes needed? Now we should to find the detail executor to dedicate the detail error msg ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? Closes #2103 from wxplovecc/easy-to-dedicate-error. Authored-by: 吴祥平 Signed-off-by: zky.zhoukeyong --- .../org/apache/celeborn/client/read/CelebornInputStream.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java b/client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java index 22cc54a5ff7..29f63538071 100644 --- a/client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java +++ b/client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java @@ -312,6 +312,7 @@ private boolean isCriticalCause(Exception e) { } private PartitionReader createReaderWithRetry(PartitionLocation location) throws IOException { + Exception lastException = null; while (fetchChunkRetryCnt < fetchChunkMaxRetry) { try { if (isExcluded(location)) { @@ -319,6 +320,7 @@ private PartitionReader createReaderWithRetry(PartitionLocation location) throws } return createReader(location, fetchChunkRetryCnt, fetchChunkMaxRetry); } catch (Exception e) { + lastException = e; excludeFailedLocation(location, e); fetchChunkRetryCnt++; if (location.hasPeer()) { @@ -345,7 +347,7 @@ private PartitionReader createReaderWithRetry(PartitionLocation location) throws } } } - throw new CelebornIOException("createPartitionReader failed! " + location); + throw new CelebornIOException("createPartitionReader failed! " + location, lastException); } private ByteBuf getNextChunk() throws IOException { From b5c5aa6d9d97949bda790076d218fccbc8c9bb4d Mon Sep 17 00:00:00 2001 From: onebox-li Date: Fri, 17 Nov 2023 10:31:57 +0800 Subject: [PATCH 11/31] [CELEBORN-1121] Improve WorkerInfo#hashCode method ### What changes were proposed in this pull request? Change WorkerInfo#hashCode() from map+foldLeft to while and cache. Test the each way to calculate, code and result show as below: ``` val state = Seq(host, rpcPort, pushPort, fetchPort, replicatePort) // origin val originHash = state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b) // for var forHash = 0 for (i <- state) { forHash = 31 * forHash + i.hashCode() } // while var whileHash = 0 var i = 0 while (i < state.size) { whileHash = 31 * whileHash + state(i).hashCode() i = i + 1 } ``` Result: ``` java version "1.8.0_261" origin hash result = -831724440, costs 1103914 ns for hash result = -831724440, costs 444588 ns (2.5x) while hash result = -831724440, costs 46510 ns (23x) ``` ### Why are the changes needed? The current WorkerInfo's hashCode() is a little time-consuming. Since it is widely used in lots of hash maps, it needs to be improved. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added UT. Closes #2086 from onebox-li/improve-worker-hash. Authored-by: onebox-li Signed-off-by: Fu Chen --- .../celeborn/common/meta/WorkerInfo.scala | 8 +++-- .../common/ComputeIfAbsentBenchmark.scala | 4 +-- .../common/meta/WorkerInfoSuite.scala | 33 +++++++++++++++++-- 3 files changed, 38 insertions(+), 7 deletions(-) diff --git a/common/src/main/scala/org/apache/celeborn/common/meta/WorkerInfo.scala b/common/src/main/scala/org/apache/celeborn/common/meta/WorkerInfo.scala index 59778481643..65dc22918a2 100644 --- a/common/src/main/scala/org/apache/celeborn/common/meta/WorkerInfo.scala +++ b/common/src/main/scala/org/apache/celeborn/common/meta/WorkerInfo.scala @@ -232,8 +232,12 @@ class WorkerInfo( } override def hashCode(): Int = { - val state = Seq(host, rpcPort, pushPort, fetchPort, replicatePort) - state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b) + var result = host.hashCode() + result = 31 * result + rpcPort.hashCode() + result = 31 * result + pushPort.hashCode() + result = 31 * result + fetchPort.hashCode() + result = 31 * result + replicatePort.hashCode() + result } } diff --git a/common/src/test/scala/org/apache/celeborn/common/ComputeIfAbsentBenchmark.scala b/common/src/test/scala/org/apache/celeborn/common/ComputeIfAbsentBenchmark.scala index eeee4d78a11..eb347e64f8e 100644 --- a/common/src/test/scala/org/apache/celeborn/common/ComputeIfAbsentBenchmark.scala +++ b/common/src/test/scala/org/apache/celeborn/common/ComputeIfAbsentBenchmark.scala @@ -30,9 +30,9 @@ import org.apache.celeborn.benchmark.{Benchmark, BenchmarkBase} * ComputeIfAbsent benchmark. * To run this benchmark: * {{{ - * 1. build/sbt "common/test:runMain " + * 1. build/sbt "celeborn-common/test:runMain " * 2. generate result: - * CELEBORN_GENERATE_BENCHMARK_FILES=1 build/sbt "common/test:runMain " + * CELEBORN_GENERATE_BENCHMARK_FILES=1 build/sbt "celeborn-common/test:runMain " * Results will be written to "benchmarks/ComputeIfAbsentBenchmark-results.txt". * }}} */ diff --git a/common/src/test/scala/org/apache/celeborn/common/meta/WorkerInfoSuite.scala b/common/src/test/scala/org/apache/celeborn/common/meta/WorkerInfoSuite.scala index 93f210f1ffa..b4286292488 100644 --- a/common/src/test/scala/org/apache/celeborn/common/meta/WorkerInfoSuite.scala +++ b/common/src/test/scala/org/apache/celeborn/common/meta/WorkerInfoSuite.scala @@ -24,7 +24,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ -import scala.reflect.ClassTag +import scala.util.Random import org.junit.Assert.{assertEquals, assertNotEquals, assertNotNull} @@ -32,8 +32,7 @@ import org.apache.celeborn.CelebornFunSuite import org.apache.celeborn.common.CelebornConf import org.apache.celeborn.common.identity.UserIdentifier import org.apache.celeborn.common.quota.ResourceConsumption -import org.apache.celeborn.common.rpc.{RpcAddress, RpcEndpointAddress, RpcEndpointRef, RpcEnv, RpcTimeout} -import org.apache.celeborn.common.rpc.netty.{NettyRpcEndpointRef, NettyRpcEnv} +import org.apache.celeborn.common.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv} import org.apache.celeborn.common.util.{JavaUtils, ThreadUtils} class WorkerInfoSuite extends CelebornFunSuite { @@ -303,4 +302,32 @@ class WorkerInfoSuite extends CelebornFunSuite { } } } + + def generateRandomIPv4Address: String = { + val ipAddress = new StringBuilder + for (i <- 0 until 4) { + ipAddress.append(Random.nextInt(256)) + if (i < 3) ipAddress.append(".") + } + ipAddress.toString + } + + test("Test WorkerInfo hashcode") { + val host = generateRandomIPv4Address + val rpcPort = Random.nextInt(65536) + val pushPort = Random.nextInt(65536) + val fetchPort = Random.nextInt(65536) + val replicatePort = Random.nextInt(65536) + val workerInfo = new WorkerInfo(host, rpcPort, pushPort, fetchPort, replicatePort) + + // origin hashCode() logic + val state = Seq(host, rpcPort, pushPort, fetchPort, replicatePort) + val originHash = state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b) + + val hashCode1 = workerInfo.hashCode() + assert(originHash === hashCode1) + + val hashCode2 = workerInfo.hashCode() + assert(hashCode1 === hashCode2) + } } From 465b0938f7a44b694e14e05a17eaee0d093cecb8 Mon Sep 17 00:00:00 2001 From: SteNicholas Date: Fri, 17 Nov 2023 20:32:04 +0800 Subject: [PATCH 12/31] [CELEBORN-1134] Celeborn Flink client should validate whether execution.batch-shuffle-mode is ALL_EXCHANGES_BLOCKING ### What changes were proposed in this pull request? Celeborn Flink client validates whether `execution.batch-shuffle-mode` is `ALL_EXCHANGES_BLOCKING`. ### Why are the changes needed? The config option `execution.batch-shuffle-mode` of Flink is `ALL_EXCHANGES_BLOCKING` by default. Celeborn Flink client should validate whether `execution.batch-shuffle-mode` is `ALL_EXCHANGES_BLOCKING`. If `execution.batch-shuffle-mode` is set as `ALL_EXCHANGES_PIPELINED`, there is `NullPointerException` for `ReducePartitionCommitHandler#handleGetReducerFileGroup`, which exception is as follows: ``` 023-11-16 14:40:55,984 ERROR org.apache.celeborn.common.rpc.netty.Inbox - Ignoring error java.lang.NullPointerException: Cannot invoke "java.util.Set.add(Object)" because the return value of "java.util.concurrent.ConcurrentHashMap.get(Object)" is null at org.apache.celeborn.client.commit.ReducePartitionCommitHandler.handleGetReducerFileGroup(ReducePartitionCommitHandler.scala:307) at org.apache.celeborn.client.CommitManager.handleGetReducerFileGroup(CommitManager.scala:266) at org.apache.celeborn.client.LifecycleManager.org$apache$celeborn$client$LifecycleManager$$handleGetReducerFileGroup(LifecycleManager.scala:559) at org.apache.celeborn.client.LifecycleManager$$anonfun$receiveAndReply$1.applyOrElse(LifecycleManager.scala:297) at org.apache.celeborn.common.rpc.netty.Inbox.$anonfun$process$1(Inbox.scala:115) at org.apache.celeborn.common.rpc.netty.Inbox.safelyCall(Inbox.scala:222) at org.apache.celeborn.common.rpc.netty.Inbox.process(Inbox.scala:110) at org.apache.celeborn.common.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:227) at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) at java.base/java.lang.Thread.run(Thread.java:833) ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? `RemoteShuffleServiceFactorySuitJ#testInvalidShuffleServiceConfig`. Closes #2106 from SteNicholas/CELEBORN-1134. Authored-by: SteNicholas Signed-off-by: mingji --- README.md | 2 + .../plugin/flink/RemoteShuffleMaster.java | 102 +++++++++++------- .../plugin/flink/RemoteShuffleMasterTest.java | 18 ++++ .../plugin/flink/RemoteShuffleMasterTest.java | 18 ++++ .../plugin/flink/RemoteShuffleMasterTest.java | 18 ++++ .../plugin/flink/RemoteShuffleMasterTest.java | 18 ++++ docs/README.md | 3 + 7 files changed, 138 insertions(+), 41 deletions(-) diff --git a/README.md b/README.md index c88cb05c3fa..d424648ca61 100644 --- a/README.md +++ b/README.md @@ -289,6 +289,7 @@ Copy $CELEBORN_HOME/flink/*.jar to $FLINK_HOME/lib/ To use Celeborn, the following flink configurations should be added. ```properties shuffle-service-factory.class: org.apache.celeborn.plugin.flink.RemoteShuffleServiceFactory +execution.batch-shuffle-mode: ALL_EXCHANGES_BLOCKING celeborn.master.endpoints: clb-1:9097,clb-2:9097,clb-3:9097 celeborn.client.shuffle.batchHandleReleasePartition.enabled: true @@ -306,6 +307,7 @@ taskmanager.network.memory.floating-buffers-per-gate: 4096 taskmanager.network.memory.buffers-per-channel: 0 taskmanager.memory.task.off-heap.size: 512m ``` +**Note**: The config option `execution.batch-shuffle-mode` should configure as `ALL_EXCHANGES_BLOCKING`. ### Deploy mapreduce client Add $CELEBORN_HOME/mr/*.jar to to `mapreduce.application.classpath` and `yarn.application.classpath`. diff --git a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleMaster.java b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleMaster.java index 7bfa1d1bed2..8bcd2827b79 100644 --- a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleMaster.java +++ b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleMaster.java @@ -24,7 +24,10 @@ import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import org.apache.flink.api.common.BatchShuffleMode; import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.configuration.MemorySize; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.shuffle.JobShuffleContext; @@ -48,10 +51,10 @@ public class RemoteShuffleMaster implements ShuffleMaster Celeborn register shuffleIds - private Map> jobShuffleIds = JavaUtils.newConcurrentHashMap(); + private final Map> jobShuffleIds = JavaUtils.newConcurrentHashMap(); private String celebornAppId; private volatile LifecycleManager lifecycleManager; - private ShuffleTaskInfo shuffleTaskInfo = new ShuffleTaskInfo(); + private final ShuffleTaskInfo shuffleTaskInfo = new ShuffleTaskInfo(); private ShuffleResourceTracker shuffleResourceTracker; private final ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor( @@ -61,6 +64,7 @@ public class RemoteShuffleMaster implements ShuffleMaster registerPartitionWithProducer( JobID jobID, PartitionDescriptor partitionDescriptor, ProducerDescriptor producerDescriptor) { - CompletableFuture completableFuture = - CompletableFuture.supplyAsync( - () -> { - Set shuffleIds = jobShuffleIds.get(jobID); - if (shuffleIds == null) { - throw new RuntimeException("Can not find job in lifecycleManager, job: " + jobID); - } + return CompletableFuture.supplyAsync( + () -> { + Set shuffleIds = jobShuffleIds.get(jobID); + if (shuffleIds == null) { + throw new RuntimeException("Can not find job in lifecycleManager, job: " + jobID); + } - FlinkResultPartitionInfo resultPartitionInfo = - new FlinkResultPartitionInfo(jobID, partitionDescriptor, producerDescriptor); - ShuffleResourceDescriptor shuffleResourceDescriptor = - shuffleTaskInfo.genShuffleResourceDescriptor( - resultPartitionInfo.getShuffleId(), - resultPartitionInfo.getTaskId(), - resultPartitionInfo.getAttemptId()); + FlinkResultPartitionInfo resultPartitionInfo = + new FlinkResultPartitionInfo(jobID, partitionDescriptor, producerDescriptor); + ShuffleResourceDescriptor shuffleResourceDescriptor = + shuffleTaskInfo.genShuffleResourceDescriptor( + resultPartitionInfo.getShuffleId(), + resultPartitionInfo.getTaskId(), + resultPartitionInfo.getAttemptId()); - synchronized (shuffleIds) { - shuffleIds.add(shuffleResourceDescriptor.getShuffleId()); - } + synchronized (shuffleIds) { + shuffleIds.add(shuffleResourceDescriptor.getShuffleId()); + } - RemoteShuffleResource remoteShuffleResource = - new RemoteShuffleResource( - lifecycleManager.getHost(), - lifecycleManager.getPort(), - lifecycleManagerTimestamp, - shuffleResourceDescriptor); - - shuffleResourceTracker.addPartitionResource( - jobID, - shuffleResourceDescriptor.getShuffleId(), - shuffleResourceDescriptor.getPartitionId(), - resultPartitionInfo.getResultPartitionId()); - - return new RemoteShuffleDescriptor( - celebornAppId, - jobID, - resultPartitionInfo.getShuffleId(), - resultPartitionInfo.getResultPartitionId(), - remoteShuffleResource); - }, - executor); + RemoteShuffleResource remoteShuffleResource = + new RemoteShuffleResource( + lifecycleManager.getHost(), + lifecycleManager.getPort(), + lifecycleManagerTimestamp, + shuffleResourceDescriptor); + + shuffleResourceTracker.addPartitionResource( + jobID, + shuffleResourceDescriptor.getShuffleId(), + shuffleResourceDescriptor.getPartitionId(), + resultPartitionInfo.getResultPartitionId()); - return completableFuture; + return new RemoteShuffleDescriptor( + celebornAppId, + jobID, + resultPartitionInfo.getShuffleId(), + resultPartitionInfo.getResultPartitionId(), + remoteShuffleResource); + }, + executor); } @Override @@ -238,4 +239,23 @@ public void close() throws Exception { ExecutorUtils.gracefulShutdown(10, TimeUnit.SECONDS, executor); } + + /** + * Checks the shuffle config given the Flink configuration. + * + *

The config option {@link ExecutionOptions#BATCH_SHUFFLE_MODE} should configure as {@link + * BatchShuffleMode#ALL_EXCHANGES_BLOCKING}. + * + * @param configuration The Flink configuration with shuffle config. + */ + private void checkShuffleConfig(Configuration configuration) { + if (configuration.get(ExecutionOptions.BATCH_SHUFFLE_MODE) + != BatchShuffleMode.ALL_EXCHANGES_BLOCKING) { + throw new IllegalArgumentException( + String.format( + "The config option %s should configure as %s", + ExecutionOptions.BATCH_SHUFFLE_MODE.key(), + BatchShuffleMode.ALL_EXCHANGES_BLOCKING.name())); + } + } } diff --git a/client-flink/flink-1.14/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java b/client-flink/flink-1.14/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java index a8607a6d87d..7cb6139f1bd 100644 --- a/client-flink/flink-1.14/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java +++ b/client-flink/flink-1.14/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java @@ -26,8 +26,10 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import org.apache.flink.api.common.BatchShuffleMode; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.configuration.MemorySize; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -198,6 +200,22 @@ public void testShuffleMemoryAnnouncing() { Assert.assertEquals(expected, calculated); } + @Test + public void testInvalidShuffleConfig() { + Assert.assertThrows( + String.format( + "The config option %s should configure as %s", + ExecutionOptions.BATCH_SHUFFLE_MODE.key(), + BatchShuffleMode.ALL_EXCHANGES_BLOCKING.name()), + IllegalArgumentException.class, + () -> + createShuffleMaster( + new Configuration() + .set( + ExecutionOptions.BATCH_SHUFFLE_MODE, + BatchShuffleMode.ALL_EXCHANGES_PIPELINED))); + } + @After public void tearDown() { if (remoteShuffleMaster != null) { diff --git a/client-flink/flink-1.15/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java b/client-flink/flink-1.15/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java index a8607a6d87d..7cb6139f1bd 100644 --- a/client-flink/flink-1.15/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java +++ b/client-flink/flink-1.15/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java @@ -26,8 +26,10 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import org.apache.flink.api.common.BatchShuffleMode; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.configuration.MemorySize; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -198,6 +200,22 @@ public void testShuffleMemoryAnnouncing() { Assert.assertEquals(expected, calculated); } + @Test + public void testInvalidShuffleConfig() { + Assert.assertThrows( + String.format( + "The config option %s should configure as %s", + ExecutionOptions.BATCH_SHUFFLE_MODE.key(), + BatchShuffleMode.ALL_EXCHANGES_BLOCKING.name()), + IllegalArgumentException.class, + () -> + createShuffleMaster( + new Configuration() + .set( + ExecutionOptions.BATCH_SHUFFLE_MODE, + BatchShuffleMode.ALL_EXCHANGES_PIPELINED))); + } + @After public void tearDown() { if (remoteShuffleMaster != null) { diff --git a/client-flink/flink-1.17/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java b/client-flink/flink-1.17/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java index 8548ca8af3e..8cdf82c79c5 100644 --- a/client-flink/flink-1.17/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java +++ b/client-flink/flink-1.17/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java @@ -26,8 +26,10 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import org.apache.flink.api.common.BatchShuffleMode; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.configuration.MemorySize; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -220,6 +222,22 @@ public void testShuffleMemoryAnnouncing() { Assert.assertEquals(expected, calculated); } + @Test + public void testInvalidShuffleConfig() { + Assert.assertThrows( + String.format( + "The config option %s should configure as %s", + ExecutionOptions.BATCH_SHUFFLE_MODE.key(), + BatchShuffleMode.ALL_EXCHANGES_BLOCKING.name()), + IllegalArgumentException.class, + () -> + createShuffleMaster( + new Configuration() + .set( + ExecutionOptions.BATCH_SHUFFLE_MODE, + BatchShuffleMode.ALL_EXCHANGES_PIPELINED))); + } + @After public void tearDown() { if (remoteShuffleMaster != null) { diff --git a/client-flink/flink-1.18/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java b/client-flink/flink-1.18/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java index 8548ca8af3e..8cdf82c79c5 100644 --- a/client-flink/flink-1.18/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java +++ b/client-flink/flink-1.18/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java @@ -26,8 +26,10 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import org.apache.flink.api.common.BatchShuffleMode; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.configuration.MemorySize; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -220,6 +222,22 @@ public void testShuffleMemoryAnnouncing() { Assert.assertEquals(expected, calculated); } + @Test + public void testInvalidShuffleConfig() { + Assert.assertThrows( + String.format( + "The config option %s should configure as %s", + ExecutionOptions.BATCH_SHUFFLE_MODE.key(), + BatchShuffleMode.ALL_EXCHANGES_BLOCKING.name()), + IllegalArgumentException.class, + () -> + createShuffleMaster( + new Configuration() + .set( + ExecutionOptions.BATCH_SHUFFLE_MODE, + BatchShuffleMode.ALL_EXCHANGES_PIPELINED))); + } + @After public void tearDown() { if (remoteShuffleMaster != null) { diff --git a/docs/README.md b/docs/README.md index 7cd683bc98c..c857cc9337a 100644 --- a/docs/README.md +++ b/docs/README.md @@ -132,7 +132,10 @@ vi conf/flink-conf.yaml ``` ```properties shuffle-service-factory.class: org.apache.celeborn.plugin.flink.RemoteShuffleServiceFactory +execution.batch-shuffle-mode: ALL_EXCHANGES_BLOCKING ``` +**Note**: The config option `execution.batch-shuffle-mode` should configure as `ALL_EXCHANGES_BLOCKING`. + Then deploy the example word count job to the running cluster: ```shell cd $FLINK_HOME From ca47cce78e817b11b8bb0f8a1e2c2781089a9f06 Mon Sep 17 00:00:00 2001 From: gaochao0509 <1623735386@qq.com> Date: Mon, 20 Nov 2023 19:48:56 +0800 Subject: [PATCH 13/31] [CELEBORN-1123] Support fallback to non-columnar shuffle for schema that cannot be obtained from shuffle dependency ### What changes were proposed in this pull request? Support fallback to non-columnar shuffle for schema that cannot be obtained from shuffle dependency. ### Why are the changes needed? When columnar shuffle is enabled, it was found that the shuffle class operator of Spark RDD is not supported. It's recommended to support fallback to non-columnar shuffle for schema that cannot be obtained from shuffle dependency. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - `CelebornColumnarShuffleReaderSuite#columnarShuffleReaderNewSerializerInstance` - `ColumnarHashBasedShuffleWriterSuiteJ#createColumnarShuffleWriter` Closes #2101 from gaochao0509/CELEBORN-1123. Authored-by: gaochao0509 <1623735386@qq.com> Signed-off-by: xiyu.zk --- .../ColumnarHashBasedShuffleWriter.java | 38 ++++++--- .../CustomShuffleDependencyUtils.java | 19 +++-- .../CelebornColumnarShuffleReader.scala | 9 +- .../columnar/CelebornBatchBuilder.scala | 4 +- .../columnar/CelebornColumnAccessor.scala | 9 +- .../columnar/CelebornColumnBuilder.scala | 8 +- .../columnar/CelebornColumnStats.scala | 59 +++---------- .../columnar/CelebornColumnType.scala | 83 +++++-------------- .../CelebornColumnarBatchBuilder.scala | 26 +----- .../CelebornColumnarBatchCodeGenBuild.scala | 15 ---- .../CelebornColumnarBatchSerializer.scala | 13 +-- .../CelebornCompressibleColumnBuilder.scala | 2 +- .../columnar/CelebornCompressionScheme.scala | 9 +- .../columnar/CelebornCompressionSchemes.scala | 16 ++-- .../ColumnarHashBasedShuffleWriterSuiteJ.java | 63 ++++++++++++-- .../CelebornColumnarShuffleReaderSuite.scala | 59 ++++++++++++- .../celeborn/CelebornShuffleReader.scala | 2 +- .../CelebornShuffleWriterSuiteBase.java | 16 ++-- 18 files changed, 224 insertions(+), 226 deletions(-) diff --git a/client-spark/spark-3-columnar-shuffle/src/main/java/org/apache/spark/shuffle/celeborn/ColumnarHashBasedShuffleWriter.java b/client-spark/spark-3-columnar-shuffle/src/main/java/org/apache/spark/shuffle/celeborn/ColumnarHashBasedShuffleWriter.java index 2943d310df5..be5d78c5023 100644 --- a/client-spark/spark-3-columnar-shuffle/src/main/java/org/apache/spark/shuffle/celeborn/ColumnarHashBasedShuffleWriter.java +++ b/client-spark/spark-3-columnar-shuffle/src/main/java/org/apache/spark/shuffle/celeborn/ColumnarHashBasedShuffleWriter.java @@ -21,6 +21,8 @@ import scala.Product2; +import com.google.common.annotations.VisibleForTesting; +import org.apache.spark.ShuffleDependency; import org.apache.spark.TaskContext; import org.apache.spark.annotation.Private; import org.apache.spark.serializer.Serializer; @@ -32,6 +34,8 @@ import org.apache.spark.sql.execution.columnar.CelebornColumnarBatchCodeGenBuild; import org.apache.spark.sql.execution.metric.SQLMetric; import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.celeborn.client.ShuffleClient; import org.apache.celeborn.common.CelebornConf; @@ -39,14 +43,19 @@ @Private public class ColumnarHashBasedShuffleWriter extends HashBasedShuffleWriter { - private CelebornBatchBuilder[] celebornBatchBuilders; - private StructType schema; - private Serializer depSerializer; - private boolean isColumnarShuffle = false; - private int columnarShuffleBatchSize; - private boolean columnarShuffleCodeGenEnabled; - private boolean columnarShuffleDictionaryEnabled; - private double columnarShuffleDictionaryMaxFactor; + private static final Logger logger = + LoggerFactory.getLogger(ColumnarHashBasedShuffleWriter.class); + + private final int stageId; + private final int shuffleId; + private final CelebornBatchBuilder[] celebornBatchBuilders; + private final StructType schema; + private final Serializer depSerializer; + private final boolean isColumnarShuffle; + private final int columnarShuffleBatchSize; + private final boolean columnarShuffleCodeGenEnabled; + private final boolean columnarShuffleDictionaryEnabled; + private final double columnarShuffleDictionaryMaxFactor; public ColumnarHashBasedShuffleWriter( CelebornShuffleHandle handle, @@ -61,17 +70,21 @@ public ColumnarHashBasedShuffleWriter( columnarShuffleCodeGenEnabled = conf.columnarShuffleCodeGenEnabled(); columnarShuffleDictionaryEnabled = conf.columnarShuffleDictionaryEnabled(); columnarShuffleDictionaryMaxFactor = conf.columnarShuffleDictionaryMaxFactor(); - this.schema = CustomShuffleDependencyUtils.getSchema(handle.dependency()); + ShuffleDependency shuffleDependency = handle.dependency(); + this.stageId = taskContext.stageId(); + this.shuffleId = shuffleDependency.shuffleId(); + this.schema = CustomShuffleDependencyUtils.getSchema(shuffleDependency); this.depSerializer = handle.dependency().serializer(); this.celebornBatchBuilders = new CelebornBatchBuilder[handle.dependency().partitioner().numPartitions()]; - this.isColumnarShuffle = CelebornBatchBuilder.supportsColumnarType(schema); + this.isColumnarShuffle = schema != null && CelebornBatchBuilder.supportsColumnarType(schema); } @Override protected void fastWrite0(scala.collection.Iterator iterator) throws IOException, InterruptedException { if (isColumnarShuffle) { + logger.info("Fast columnar write of columnar shuffle {} for stage {}.", shuffleId, stageId); fastColumnarWrite0(iterator); } else { super.fastWrite0(iterator); @@ -141,4 +154,9 @@ private void closeColumnarWrite() throws IOException { } } } + + @VisibleForTesting + public boolean isColumnarShuffle() { + return isColumnarShuffle; + } } diff --git a/client-spark/spark-3-columnar-shuffle/src/main/java/org/apache/spark/shuffle/celeborn/CustomShuffleDependencyUtils.java b/client-spark/spark-3-columnar-shuffle/src/main/java/org/apache/spark/shuffle/celeborn/CustomShuffleDependencyUtils.java index ace98601b45..b5a646b062d 100644 --- a/client-spark/spark-3-columnar-shuffle/src/main/java/org/apache/spark/shuffle/celeborn/CustomShuffleDependencyUtils.java +++ b/client-spark/spark-3-columnar-shuffle/src/main/java/org/apache/spark/shuffle/celeborn/CustomShuffleDependencyUtils.java @@ -17,15 +17,17 @@ package org.apache.spark.shuffle.celeborn; -import java.io.IOException; - import org.apache.spark.ShuffleDependency; import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.celeborn.reflect.DynFields; public class CustomShuffleDependencyUtils { + private static final Logger logger = LoggerFactory.getLogger(CustomShuffleDependencyUtils.class); + /** * Columnar Shuffle requires a field, `ShuffleDependency#schema`, which does not exist in vanilla * Spark. @@ -33,10 +35,17 @@ public class CustomShuffleDependencyUtils { private static final DynFields.UnboundField SCHEMA_FIELD = DynFields.builder().hiddenImpl(ShuffleDependency.class, "schema").defaultAlwaysNull().build(); - public static StructType getSchema(ShuffleDependency dep) throws IOException { - StructType schema = SCHEMA_FIELD.bind(dep).get(); + public static StructType getSchema(ShuffleDependency dep) { + StructType schema = null; + try { + schema = SCHEMA_FIELD.bind(dep).get(); + } catch (Exception e) { + logger.error("Failed to bind shuffle dependency of shuffle {}.", dep.shuffleId(), e); + } if (schema == null) { - throw new IOException("Failed to get Schema, columnar shuffle won't work properly."); + logger.warn( + "Failed to get Schema of shuffle {}, columnar shuffle won't work properly.", + dep.shuffleId()); } return schema; } diff --git a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornColumnarShuffleReader.scala b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornColumnarShuffleReader.scala index e6f3cdccb2c..f47f9880ca8 100644 --- a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornColumnarShuffleReader.scala +++ b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornColumnarShuffleReader.scala @@ -46,14 +46,11 @@ class CelebornColumnarShuffleReader[K, C]( override def newSerializerInstance(dep: ShuffleDependency[K, _, C]): SerializerInstance = { val schema = CustomShuffleDependencyUtils.getSchema(dep) - if (CelebornBatchBuilder.supportsColumnarType( - schema)) { - val dataSize = SparkUtils.getDataSize( - dep.serializer.asInstanceOf[UnsafeRowSerializer]) + if (schema != null && CelebornBatchBuilder.supportsColumnarType(schema)) { + logInfo(s"Creating column batch serializer of columnar shuffle ${dep.shuffleId}.") + val dataSize = SparkUtils.getDataSize(dep.serializer.asInstanceOf[UnsafeRowSerializer]) new CelebornColumnarBatchSerializer( schema, - conf.columnarShuffleBatchSize, - conf.columnarShuffleDictionaryEnabled, conf.columnarShuffleOffHeapEnabled, dataSize).newInstance() } else { diff --git a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornBatchBuilder.scala b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornBatchBuilder.scala index 7ae77fec030..bc93c10b5f0 100644 --- a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornBatchBuilder.scala +++ b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornBatchBuilder.scala @@ -28,7 +28,7 @@ abstract class CelebornBatchBuilder { def writeRow(row: InternalRow): Unit - def getRowCnt(): Int + def getRowCnt: Int def int2ByteArray(i: Int): Array[Byte] = { val result = new Array[Byte](4) @@ -46,7 +46,7 @@ object CelebornBatchBuilder { f.dataType match { case BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType | StringType => true - case dt: DecimalType => true + case _: DecimalType => true case _ => false }) } diff --git a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnAccessor.scala b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnAccessor.scala index 064bbefc6af..a75c8d32a89 100644 --- a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnAccessor.scala +++ b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnAccessor.scala @@ -61,13 +61,9 @@ abstract class CelebornBasicColumnAccessor[JvmType]( columnType.extract(buffer, row, ordinal) } - protected def underlyingBuffer = buffer + protected def underlyingBuffer: ByteBuffer = buffer } -class CelebornNullColumnAccessor(buffer: ByteBuffer) - extends CelebornBasicColumnAccessor[Any](buffer, CELEBORN_NULL) - with CelebornNullableColumnAccessor - abstract class CelebornNativeColumnAccessor[T <: AtomicType]( override protected val buffer: ByteBuffer, override protected val columnType: NativeCelebornColumnType[T]) @@ -112,7 +108,6 @@ private[sql] object CelebornColumnAccessor { val buf = buffer.order(ByteOrder.nativeOrder) dataType match { - case NullType => new CelebornNullColumnAccessor(buf) case BooleanType => new CelebornBooleanColumnAccessor(buf) case ByteType => new CelebornByteColumnAccessor(buf) case ShortType => new CelebornShortColumnAccessor(buf) @@ -135,7 +130,7 @@ private[sql] object CelebornColumnAccessor { columnAccessor match { case nativeAccessor: CelebornNativeColumnAccessor[_] => nativeAccessor.decompress(columnVector, numRows) - case d: CelebornDecimalColumnAccessor => + case _: CelebornDecimalColumnAccessor => (0 until numRows).foreach(columnAccessor.extractToColumnVector(columnVector, _)) case _ => throw new RuntimeException("Not support non-primitive type now") diff --git a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnBuilder.scala b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnBuilder.scala index 0abfdd0cd4e..f65a5fd8653 100644 --- a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnBuilder.scala +++ b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnBuilder.scala @@ -88,10 +88,6 @@ class CelebornBasicColumnBuilder[JvmType]( } } -class CelebornNullColumnBuilder - extends CelebornBasicColumnBuilder[Any](new CelebornObjectColumnStats(NullType), CELEBORN_NULL) - with CelebornNullableColumnBuilder - abstract class CelebornComplexColumnBuilder[JvmType]( columnStats: CelebornColumnStats, columnType: CelebornColumnType[JvmType]) @@ -318,7 +314,6 @@ class CelebornDecimalCodeGenColumnBuilder(dataType: DecimalType) } object CelebornColumnBuilder { - val MAX_BATCH_SIZE_IN_BYTE: Long = 4 * 1024 * 1024L def ensureFreeSpace(orig: ByteBuffer, size: Int): ByteBuffer = { if (orig.remaining >= size) { @@ -343,7 +338,6 @@ object CelebornColumnBuilder { encodingEnabled: Boolean, encoder: Encoder[_ <: AtomicType]): CelebornColumnBuilder = { val builder: CelebornColumnBuilder = dataType match { - case NullType => new CelebornNullColumnBuilder case ByteType => new CelebornByteColumnBuilder case BooleanType => new CelebornBooleanColumnBuilder case ShortType => new CelebornShortColumnBuilder @@ -367,7 +361,7 @@ object CelebornColumnBuilder { new CelebornCompactDecimalColumnBuilder(dt) case dt: DecimalType => new CelebornDecimalColumnBuilder(dt) case other => - throw new Exception(s"not support type: $other") + throw new Exception(s"Unsupported type: $other") } builder.initialize(rowCnt, columnName, encodingEnabled) diff --git a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnStats.scala b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnStats.scala index 6c2aa0f7b83..b0b9f61db9a 100644 --- a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnStats.scala +++ b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnStats.scala @@ -63,7 +63,7 @@ final private[columnar] class CelebornBooleanColumnStats extends CelebornColumnS val value = row.getBoolean(ordinal) gatherValueStats(value) } else { - gatherNullStats + gatherNullStats() } } @@ -87,7 +87,7 @@ final private[columnar] class CelebornByteColumnStats extends CelebornColumnStat val value = row.getByte(ordinal) gatherValueStats(value) } else { - gatherNullStats + gatherNullStats() } } @@ -111,7 +111,7 @@ final private[columnar] class CelebornShortColumnStats extends CelebornColumnSta val value = row.getShort(ordinal) gatherValueStats(value) } else { - gatherNullStats + gatherNullStats() } } @@ -135,7 +135,7 @@ final private[columnar] class CelebornIntColumnStats extends CelebornColumnStats val value = row.getInt(ordinal) gatherValueStats(value) } else { - gatherNullStats + gatherNullStats() } } @@ -159,7 +159,7 @@ final private[columnar] class CelebornLongColumnStats extends CelebornColumnStat val value = row.getLong(ordinal) gatherValueStats(value) } else { - gatherNullStats + gatherNullStats() } } @@ -183,7 +183,7 @@ final private[columnar] class CelebornFloatColumnStats extends CelebornColumnSta val value = row.getFloat(ordinal) gatherValueStats(value) } else { - gatherNullStats + gatherNullStats() } } @@ -207,7 +207,7 @@ final private[columnar] class CelebornDoubleColumnStats extends CelebornColumnSt val value = row.getDouble(ordinal) gatherValueStats(value) } else { - gatherNullStats + gatherNullStats() } } @@ -223,8 +223,8 @@ final private[columnar] class CelebornDoubleColumnStats extends CelebornColumnSt } final private[columnar] class CelebornStringColumnStats extends CelebornColumnStats { - protected var upper: UTF8String = null - protected var lower: UTF8String = null + protected var upper: UTF8String = _ + protected var lower: UTF8String = _ override def gatherStats(row: InternalRow, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { @@ -232,7 +232,7 @@ final private[columnar] class CelebornStringColumnStats extends CelebornColumnSt val size = CELEBORN_STRING.actualSize(row, ordinal) gatherValueStats(value, size) } else { - gatherNullStats + gatherNullStats() } } @@ -247,34 +247,19 @@ final private[columnar] class CelebornStringColumnStats extends CelebornColumnSt Array[Any](lower, upper, nullCount, count, sizeInBytes) } -final private[columnar] class CelebornBinaryColumnStats extends CelebornColumnStats { - override def gatherStats(row: InternalRow, ordinal: Int): Unit = { - if (!row.isNullAt(ordinal)) { - val size = CELEBORN_BINARY.actualSize(row, ordinal) - sizeInBytes += size - count += 1 - } else { - gatherNullStats - } - } - - override def collectedStatistics: Array[Any] = - Array[Any](null, null, nullCount, count, sizeInBytes) -} - final private[columnar] class CelebornDecimalColumnStats(precision: Int, scale: Int) extends CelebornColumnStats { def this(dt: DecimalType) = this(dt.precision, dt.scale) - protected var upper: Decimal = null - protected var lower: Decimal = null + protected var upper: Decimal = _ + protected var lower: Decimal = _ override def gatherStats(row: InternalRow, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { val value = row.getDecimal(ordinal, precision, scale) gatherValueStats(value) } else { - gatherNullStats + gatherNullStats() } } @@ -294,21 +279,3 @@ final private[columnar] class CelebornDecimalColumnStats(precision: Int, scale: override def collectedStatistics: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) } - -final private[columnar] class CelebornObjectColumnStats(dataType: DataType) - extends CelebornColumnStats { - val columnType = CelebornColumnType(dataType) - - override def gatherStats(row: InternalRow, ordinal: Int): Unit = { - if (!row.isNullAt(ordinal)) { - val size = columnType.actualSize(row, ordinal) - sizeInBytes += size - count += 1 - } else { - gatherNullStats - } - } - - override def collectedStatistics: Array[Any] = - Array[Any](null, null, nullCount, count, sizeInBytes) -} diff --git a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnType.scala b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnType.scala index d1d5461a431..69cf10a2ea4 100644 --- a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnType.scala +++ b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnType.scala @@ -20,8 +20,6 @@ package org.apache.spark.sql.execution.columnar import java.math.{BigDecimal, BigInteger} import java.nio.ByteBuffer -import scala.reflect.runtime.universe.TypeTag - import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ @@ -177,26 +175,10 @@ sealed abstract private[columnar] class CelebornColumnType[JvmType] { override def toString: String = getClass.getSimpleName.stripSuffix("$") } -private[columnar] object CELEBORN_NULL extends CelebornColumnType[Any] { - - override def dataType: DataType = NullType - override def defaultSize: Int = 0 - override def append(v: Any, buffer: ByteBuffer): Unit = {} - override def extract(buffer: ByteBuffer): Any = null - override def setField(row: InternalRow, ordinal: Int, value: Any): Unit = row.setNullAt(ordinal) - override def getField(row: InternalRow, ordinal: Int): Any = null -} - abstract private[columnar] class NativeCelebornColumnType[T <: AtomicType]( val dataType: T, val defaultSize: Int) - extends CelebornColumnType[T#InternalType] { - - /** - * Scala TypeTag. Can be used to create primitive arrays and hash tables. - */ - def scalaTag: TypeTag[dataType.InternalType] = dataType.tag -} + extends CelebornColumnType[T#InternalType] {} private[columnar] object CELEBORN_INT extends NativeCelebornColumnType(IntegerType, 4) { override def append(v: Int, buffer: ByteBuffer): Unit = { @@ -428,26 +410,28 @@ private[columnar] trait DirectCopyCelebornColumnType[JvmType] extends CelebornCo // copy the bytes from ByteBuffer to UnsafeRow override def extract(buffer: ByteBuffer, row: InternalRow, ordinal: Int): Unit = { - if (row.isInstanceOf[MutableUnsafeRow]) { - val numBytes = buffer.getInt - val cursor = buffer.position() - buffer.position(cursor + numBytes) - row.asInstanceOf[MutableUnsafeRow].writer.write( - ordinal, - buffer.array(), - buffer.arrayOffset() + cursor, - numBytes) - } else { - setField(row, ordinal, extract(buffer)) + row match { + case r: MutableUnsafeRow => + val numBytes = buffer.getInt + val cursor = buffer.position() + buffer.position(cursor + numBytes) + r.writer.write( + ordinal, + buffer.array(), + buffer.arrayOffset() + cursor, + numBytes) + case _ => + setField(row, ordinal, extract(buffer)) } } // copy the bytes from UnsafeRow to ByteBuffer override def append(row: InternalRow, ordinal: Int, buffer: ByteBuffer): Unit = { - if (row.isInstanceOf[UnsafeRow]) { - row.asInstanceOf[UnsafeRow].writeFieldTo(ordinal, buffer) - } else { - super.append(row, ordinal, buffer) + row match { + case r: UnsafeRow => + r.writeFieldTo(ordinal, buffer) + case _ => + super.append(row, ordinal, buffer) } } } @@ -472,10 +456,11 @@ private[columnar] object CELEBORN_STRING } override def setField(row: InternalRow, ordinal: Int, value: UTF8String): Unit = { - if (row.isInstanceOf[MutableUnsafeRow]) { - row.asInstanceOf[MutableUnsafeRow].writer.write(ordinal, value) - } else { - row.update(ordinal, value.clone()) + row match { + case r: MutableUnsafeRow => + r.writer.write(ordinal, value) + case _ => + row.update(ordinal, value.clone()) } } @@ -617,26 +602,6 @@ sealed abstract private[columnar] class ByteArrayCelebornColumnType[JvmType](val } } -private[columnar] object CELEBORN_BINARY extends ByteArrayCelebornColumnType[Array[Byte]](16) { - - def dataType: DataType = BinaryType - - override def setField(row: InternalRow, ordinal: Int, value: Array[Byte]): Unit = { - row.update(ordinal, value) - } - - override def getField(row: InternalRow, ordinal: Int): Array[Byte] = { - row.getBinary(ordinal) - } - - override def actualSize(row: InternalRow, ordinal: Int): Int = { - row.getBinary(ordinal).length + 4 - } - - def serialize(value: Array[Byte]): Array[Byte] = value - def deserialize(bytes: Array[Byte]): Array[Byte] = bytes -} - private[columnar] case class CELEBORN_LARGE_DECIMAL(precision: Int, scale: Int) extends ByteArrayCelebornColumnType[Decimal](12) { @@ -673,7 +638,6 @@ private[columnar] object CELEBORN_LARGE_DECIMAL { private[columnar] object CelebornColumnType { def apply(dataType: DataType): CelebornColumnType[_] = { dataType match { - case NullType => CELEBORN_NULL case BooleanType => CELEBORN_BOOLEAN case ByteType => CELEBORN_BYTE case ShortType => CELEBORN_SHORT @@ -682,7 +646,6 @@ private[columnar] object CelebornColumnType { case FloatType => CELEBORN_FLOAT case DoubleType => CELEBORN_DOUBLE case StringType => CELEBORN_STRING - case BinaryType => CELEBORN_BINARY case dt: DecimalType if dt.precision <= Decimal.MAX_INT_DIGITS => CELEBORN_COMPACT_MINI_DECIMAL(dt) case dt: DecimalType if dt.precision <= Decimal.MAX_LONG_DIGITS => diff --git a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnarBatchBuilder.scala b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnarBatchBuilder.scala index 159b15e327a..23a81370d45 100644 --- a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnarBatchBuilder.scala +++ b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnarBatchBuilder.scala @@ -30,7 +30,8 @@ class CelebornColumnarBatchBuilder( encodingEnabled: Boolean = false) extends CelebornBatchBuilder { var rowCnt = 0 - val typeConversion: PartialFunction[DataType, NativeCelebornColumnType[_ <: AtomicType]] = { + private val typeConversion + : PartialFunction[DataType, NativeCelebornColumnType[_ <: AtomicType]] = { case IntegerType => CELEBORN_INT case LongType => CELEBORN_LONG case StringType => CELEBORN_STRING @@ -45,7 +46,7 @@ class CelebornColumnarBatchBuilder( case _ => null } - val encodersArr: Array[Encoder[_ <: AtomicType]] = schema.map { attribute => + private val encodersArr: Array[Encoder[_ <: AtomicType]] = schema.map { attribute => val nativeColumnType = typeConversion(attribute.dataType) if (nativeColumnType == null) { null @@ -63,7 +64,6 @@ class CelebornColumnarBatchBuilder( var columnBuilders: Array[CelebornColumnBuilder] = _ def newBuilders(): Unit = { - totalSize = 0 rowCnt = 0 var i = -1 columnBuilders = schema.map { attribute => @@ -100,8 +100,6 @@ class CelebornColumnarBatchBuilder( giantBuffer.toByteArray } - var totalSize = 0 - def writeRow(row: InternalRow): Unit = { var i = 0 while (i < row.numFields) { @@ -111,21 +109,5 @@ class CelebornColumnarBatchBuilder( rowCnt += 1 } - def getTotalSize(): Int = { - var i = 0 - var tempTotalSize = 0 - while (i < schema.length) { - columnBuilders(i) match { - case builder: CelebornCompressibleColumnBuilder[_] => - tempTotalSize += builder.getTotalSize.toInt - case builder: CelebornNullableColumnBuilder => tempTotalSize += builder.getTotalSize.toInt - case _ => - } - i += 1 - } - totalSize = tempTotalSize + 4 + 4 * schema.length - totalSize - } - - def getRowCnt(): Int = rowCnt + def getRowCnt: Int = rowCnt } diff --git a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnarBatchCodeGenBuild.scala b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnarBatchCodeGenBuild.scala index 1c15d163a2b..e510e645259 100644 --- a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnarBatchCodeGenBuild.scala +++ b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnarBatchCodeGenBuild.scala @@ -102,21 +102,6 @@ class CelebornColumnarBatchCodeGenBuild { val writeRowCode = new mutable.StringBuilder() for (index <- schema.indices) { schema.fields(index).dataType match { - case NullType => - initCode.append( - s""" - | ${classOf[CelebornNullColumnBuilder].getName} b$index; - """.stripMargin) - buildCode.append( - s""" - | b$index = new ${classOf[CelebornNullColumnBuilder].getName}(); - | builder.initialize($batchSize, "${schema.fields(index).name}", false); - """.stripMargin) - writeCode.append(genWriteCode(index)) - writeRowCode.append( - s""" - | b$index.appendFrom(row, $index); - """.stripMargin) case ByteType => initCode.append( s""" diff --git a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnarBatchSerializer.scala b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnarBatchSerializer.scala index c4be15c0e13..f9c08a0f6a1 100644 --- a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnarBatchSerializer.scala +++ b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornColumnarBatchSerializer.scala @@ -34,24 +34,18 @@ import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} class CelebornColumnarBatchSerializer( schema: StructType, - columnBatchSize: Int, - encodingEnabled: Boolean, offHeapColumnVectorEnabled: Boolean, dataSize: SQLMetric = null) extends Serializer with Serializable { override def newInstance(): SerializerInstance = new CelebornColumnarBatchSerializerInstance( schema, - columnBatchSize, - encodingEnabled, offHeapColumnVectorEnabled, dataSize) override def supportsRelocationOfSerializedObjects: Boolean = true } -private class CelebornColumnarBatchSerializerInstance( +class CelebornColumnarBatchSerializerInstance( schema: StructType, - columnBatchSize: Int, - encodingEnabled: Boolean, offHeapColumnVectorEnabled: Boolean, dataSize: SQLMetric) extends SerializerInstance { @@ -93,7 +87,8 @@ private class CelebornColumnarBatchSerializerInstance( } } - val toUnsafe: UnsafeProjection = UnsafeProjection.create(schema.fields.map(f => f.dataType)) + private val toUnsafe: UnsafeProjection = + UnsafeProjection.create(schema.fields.map(f => f.dataType)) override def deserializeStream(in: InputStream): DeserializationStream = { val numFields = schema.fields.length @@ -160,7 +155,7 @@ private class CelebornColumnarBatchSerializerInstance( try { dIn.readInt() } catch { - case e: EOFException => + case _: EOFException => dIn.close() EOF } diff --git a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornCompressibleColumnBuilder.scala b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornCompressibleColumnBuilder.scala index 2d87856c199..6b7d5b50564 100644 --- a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornCompressibleColumnBuilder.scala +++ b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornCompressibleColumnBuilder.scala @@ -29,7 +29,7 @@ trait CelebornCompressibleColumnBuilder[T <: AtomicType] this: CelebornNativeColumnBuilder[T] with WithCelebornCompressionSchemes => - var compressionEncoder: Encoder[T] = CelebornPassThrough.encoder(columnType) + private var compressionEncoder: Encoder[T] = CelebornPassThrough.encoder(columnType) def init(encoder: Encoder[T]): Unit = { compressionEncoder = encoder diff --git a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornCompressionScheme.scala b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornCompressionScheme.scala index a6ba31176c4..1e7ebae0e28 100644 --- a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornCompressionScheme.scala +++ b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornCompressionScheme.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.columnar -import java.nio.{ByteBuffer, ByteOrder} +import java.nio.ByteBuffer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.vectorized.WritableColumnVector @@ -76,11 +76,4 @@ object CelebornCompressionScheme { typeId, throw new UnsupportedOperationException(s"Unrecognized compression scheme type ID: $typeId")) } - - def columnHeaderSize(columnBuffer: ByteBuffer): Int = { - val header = columnBuffer.duplicate().order(ByteOrder.nativeOrder) - val nullCount = header.getInt() - // null count + null positions - 4 + 4 * nullCount - } } diff --git a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornCompressionSchemes.scala b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornCompressionSchemes.scala index 316e213c8ca..c2dfb53c211 100644 --- a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornCompressionSchemes.scala +++ b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/sql/execution/columnar/CelebornCompressionSchemes.scala @@ -33,7 +33,7 @@ case object CelebornPassThrough extends CelebornCompressionScheme { override def supports(columnType: CelebornColumnType[_]): Boolean = true override def encoder[T <: AtomicType](columnType: NativeCelebornColumnType[T]): Encoder[T] = { - new this.CelebornEncoder[T](columnType) + new this.CelebornEncoder[T]() } override def decoder[T <: AtomicType]( @@ -42,7 +42,7 @@ case object CelebornPassThrough extends CelebornCompressionScheme { new this.CelebornDecoder(buffer, columnType) } - class CelebornEncoder[T <: AtomicType](columnType: NativeCelebornColumnType[T]) + class CelebornEncoder[T <: AtomicType]() extends Encoder[T] { override def uncompressedSize: Int = 0 @@ -247,7 +247,7 @@ case object CelebornDictionaryEncoding extends CelebornCompressionScheme { override val typeId = 1 // 32K unique values allowed - var MAX_DICT_SIZE = Short.MaxValue + var MAX_DICT_SIZE: Short = Short.MaxValue override def decoder[T <: AtomicType]( buffer: ByteBuffer, @@ -277,7 +277,7 @@ case object CelebornDictionaryEncoding extends CelebornCompressionScheme { // Total number of elements. private var count = 0 - def cleanBatch: Unit = { + def cleanBatch(): Unit = { count = 0 _uncompressedSize = 0 } @@ -341,11 +341,11 @@ case object CelebornDictionaryEncoding extends CelebornCompressionScheme { buffer: ByteBuffer, columnType: NativeCelebornColumnType[T]) extends Decoder[T] { - val elementNum = ByteBufferHelper.getInt(buffer) + private val elementNum: Int = ByteBufferHelper.getInt(buffer) private val dictionary: Array[Any] = new Array[Any](elementNum) - private var intDictionary: Array[Int] = null - private var longDictionary: Array[Long] = null - private var stringDictionary: Array[String] = null + private var intDictionary: Array[Int] = _ + private var longDictionary: Array[Long] = _ + private var stringDictionary: Array[String] = _ columnType.dataType match { case _: IntegerType => diff --git a/client-spark/spark-3-columnar-shuffle/src/test/java/org/apache/spark/shuffle/celeborn/ColumnarHashBasedShuffleWriterSuiteJ.java b/client-spark/spark-3-columnar-shuffle/src/test/java/org/apache/spark/shuffle/celeborn/ColumnarHashBasedShuffleWriterSuiteJ.java index 2cb6d3548ef..e481b6181cc 100644 --- a/client-spark/spark-3-columnar-shuffle/src/test/java/org/apache/spark/shuffle/celeborn/ColumnarHashBasedShuffleWriterSuiteJ.java +++ b/client-spark/spark-3-columnar-shuffle/src/test/java/org/apache/spark/shuffle/celeborn/ColumnarHashBasedShuffleWriterSuiteJ.java @@ -17,7 +17,15 @@ package org.apache.spark.shuffle.celeborn; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.util.UUID; + +import org.apache.spark.HashPartitioner; import org.apache.spark.TaskContext; +import org.apache.spark.serializer.KryoSerializer; import org.apache.spark.serializer.Serializer; import org.apache.spark.serializer.SerializerInstance; import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; @@ -28,28 +36,59 @@ import org.apache.spark.sql.types.IntegerType$; import org.apache.spark.sql.types.StringType$; import org.apache.spark.sql.types.StructType; +import org.junit.Test; import org.mockito.MockedStatic; import org.mockito.Mockito; +import org.apache.celeborn.client.DummyShuffleClient; import org.apache.celeborn.client.ShuffleClient; import org.apache.celeborn.common.CelebornConf; public class ColumnarHashBasedShuffleWriterSuiteJ extends CelebornShuffleWriterSuiteBase { - private StructType schema = + private final StructType schema = new StructType().add("key", IntegerType$.MODULE$).add("value", StringType$.MODULE$); + @Test + public void createColumnarShuffleWriter() throws Exception { + Mockito.doReturn(new HashPartitioner(numPartitions)).when(dependency).partitioner(); + final CelebornConf conf = new CelebornConf(); + final File tempFile = new File(tempDir, UUID.randomUUID().toString()); + final DummyShuffleClient client = new DummyShuffleClient(conf, tempFile); + client.initReducePartitionMap(shuffleId, numPartitions, 1); + + // Create ColumnarHashBasedShuffleWriter with handle of which dependency has null schema. + Mockito.doReturn(new KryoSerializer(sparkConf)).when(dependency).serializer(); + ShuffleWriter writer = + createShuffleWriterWithoutSchema( + new CelebornShuffleHandle<>( + "appId", "host", 0, this.userIdentifier, 0, 10, this.dependency), + taskContext, + conf, + client, + metrics.shuffleWriteMetrics()); + assertTrue(writer instanceof ColumnarHashBasedShuffleWriter); + assertFalse(((ColumnarHashBasedShuffleWriter) writer).isColumnarShuffle()); + + // Create ColumnarHashBasedShuffleWriter with handle of which dependency has non-null schema. + Mockito.doReturn(new UnsafeRowSerializer(2, null)).when(dependency).serializer(); + writer = + createShuffleWriter( + new CelebornShuffleHandle<>( + "appId", "host", 0, this.userIdentifier, 0, 10, this.dependency), + taskContext, + conf, + client, + metrics.shuffleWriteMetrics()); + assertTrue(((ColumnarHashBasedShuffleWriter) writer).isColumnarShuffle()); + } + @Override protected SerializerInstance newSerializerInstance(Serializer serializer) { if (serializer instanceof UnsafeRowSerializer && CelebornBatchBuilder.supportsColumnarType(schema)) { CelebornConf conf = new CelebornConf(); - return new CelebornColumnarBatchSerializer( - schema, - conf.columnarShuffleBatchSize(), - conf.columnarShuffleDictionaryEnabled(), - conf.columnarShuffleOffHeapEnabled(), - null) + return new CelebornColumnarBatchSerializer(schema, conf.columnarShuffleOffHeapEnabled(), null) .newInstance(); } else { return serializer.newInstance(); @@ -72,4 +111,14 @@ protected ShuffleWriter createShuffleWriter( handle, context, conf, client, metrics, SendBufferPool.get(1, 30, 60)); } } + + private ShuffleWriter createShuffleWriterWithoutSchema( + CelebornShuffleHandle handle, + TaskContext context, + CelebornConf conf, + ShuffleClient client, + ShuffleWriteMetricsReporter metrics) { + return SparkUtils.createColumnarHashBasedShuffleWriter( + handle, context, conf, client, metrics, SendBufferPool.get(1, 30, 60)); + } } diff --git a/client-spark/spark-3-columnar-shuffle/src/test/scala/org/apache/spark/shuffle/celeborn/CelebornColumnarShuffleReaderSuite.scala b/client-spark/spark-3-columnar-shuffle/src/test/scala/org/apache/spark/shuffle/celeborn/CelebornColumnarShuffleReaderSuite.scala index ec57c192b16..5a14d021906 100644 --- a/client-spark/spark-3-columnar-shuffle/src/test/scala/org/apache/spark/shuffle/celeborn/CelebornColumnarShuffleReaderSuite.scala +++ b/client-spark/spark-3-columnar-shuffle/src/test/scala/org/apache/spark/shuffle/celeborn/CelebornColumnarShuffleReaderSuite.scala @@ -17,6 +17,11 @@ package org.apache.spark.shuffle.celeborn +import org.apache.spark.{ShuffleDependency, SparkConf} +import org.apache.spark.serializer.{KryoSerializer, KryoSerializerInstance} +import org.apache.spark.sql.execution.UnsafeRowSerializer +import org.apache.spark.sql.execution.columnar.CelebornColumnarBatchSerializerInstance +import org.apache.spark.sql.types.{IntegerType, StringType, StructType} import org.junit.Test import org.mockito.{MockedStatic, Mockito} @@ -37,9 +42,9 @@ class CelebornColumnarShuffleReaderSuite { 10, null) - var shuffleClientClass: MockedStatic[ShuffleClient] = null + var shuffleClient: MockedStatic[ShuffleClient] = null try { - shuffleClientClass = Mockito.mockStatic(classOf[ShuffleClient]) + shuffleClient = Mockito.mockStatic(classOf[ShuffleClient]) val shuffleReader = SparkUtils.createColumnarShuffleReader( handle, 0, @@ -51,8 +56,54 @@ class CelebornColumnarShuffleReaderSuite { null) assert(shuffleReader.getClass == classOf[CelebornColumnarShuffleReader[Int, String]]) } finally { - if (shuffleClientClass != null) { - shuffleClientClass.close() + if (shuffleClient != null) { + shuffleClient.close() + } + } + } + + @Test + def columnarShuffleReaderNewSerializerInstance(): Unit = { + var shuffleClient: MockedStatic[ShuffleClient] = null + try { + shuffleClient = Mockito.mockStatic(classOf[ShuffleClient]) + val shuffleReader = SparkUtils.createColumnarShuffleReader( + new CelebornShuffleHandle[Int, String, String]( + "appId", + "host", + 0, + new UserIdentifier("mock", "mock"), + 0, + 10, + null), + 0, + 10, + 0, + 10, + null, + new CelebornConf(), + null) + val shuffleDependency = Mockito.mock(classOf[ShuffleDependency[Int, String, String]]) + Mockito.when(shuffleDependency.shuffleId).thenReturn(0) + Mockito.when(shuffleDependency.serializer).thenReturn(new KryoSerializer( + new SparkConf(false))) + + // CelebornColumnarShuffleReader creates new serializer instance with dependency which has null schema. + var serializerInstance = shuffleReader.newSerializerInstance(shuffleDependency) + assert(serializerInstance.getClass == classOf[KryoSerializerInstance]) + + // CelebornColumnarShuffleReader creates new serializer instance with dependency which has non-null schema. + val dependencyUtils = Mockito.mockStatic(classOf[CustomShuffleDependencyUtils]) + dependencyUtils.when(() => + CustomShuffleDependencyUtils.getSchema(shuffleDependency)).thenReturn(new StructType().add( + "key", + IntegerType).add("value", StringType)) + Mockito.when(shuffleDependency.serializer).thenReturn(new UnsafeRowSerializer(2, null)) + serializerInstance = shuffleReader.newSerializerInstance(shuffleDependency) + assert(serializerInstance.getClass == classOf[CelebornColumnarBatchSerializerInstance]) + } finally { + if (shuffleClient != null) { + shuffleClient.close() } } } diff --git a/client-spark/spark-3/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleReader.scala b/client-spark/spark-3/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleReader.scala index 5ec0fed9b79..d83df1a5b76 100644 --- a/client-spark/spark-3/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleReader.scala +++ b/client-spark/spark-3/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleReader.scala @@ -192,7 +192,7 @@ class CelebornShuffleReader[K, C]( } } - protected def newSerializerInstance(dep: ShuffleDependency[K, _, C]): SerializerInstance = { + def newSerializerInstance(dep: ShuffleDependency[K, _, C]): SerializerInstance = { dep.serializer.newInstance() } diff --git a/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/CelebornShuffleWriterSuiteBase.java b/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/CelebornShuffleWriterSuiteBase.java index d8a3d4986e3..f4109753ea4 100644 --- a/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/CelebornShuffleWriterSuiteBase.java +++ b/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/CelebornShuffleWriterSuiteBase.java @@ -94,23 +94,23 @@ public abstract class CelebornShuffleWriterSuiteBase { private final String appId = "appId"; private final String host = "host"; private final int port = 0; - private final int shuffleId = 0; + protected final int shuffleId = 0; - private final UserIdentifier userIdentifier = new UserIdentifier("mock", "mock"); + protected final UserIdentifier userIdentifier = new UserIdentifier("mock", "mock"); private final int numMaps = 10; - private final int numPartitions = 10; - private final SparkConf sparkConf = new SparkConf(false); + protected final int numPartitions = 10; + protected final SparkConf sparkConf = new SparkConf(false); private final BlockManagerId bmId = BlockManagerId.apply("execId", "host", 1, None$.empty()); private final TaskMemoryManager taskMemoryManager = new TaskMemoryManager(UnifiedMemoryManager.apply(sparkConf, 1), 0); @Mock(answer = Answers.RETURNS_SMART_NULLS) - private TaskContext taskContext = null; + protected TaskContext taskContext = null; @Mock(answer = Answers.RETURNS_SMART_NULLS) - private ShuffleDependency dependency = null; + protected ShuffleDependency dependency = null; @Mock(answer = Answers.RETURNS_SMART_NULLS) private SparkEnv env = null; @@ -118,9 +118,9 @@ public abstract class CelebornShuffleWriterSuiteBase { @Mock(answer = Answers.RETURNS_SMART_NULLS) private BlockManager blockManager = null; - private TaskMetrics metrics = null; + protected TaskMetrics metrics = null; - private static File tempDir = null; + protected static File tempDir = null; @BeforeClass public static void beforeAll() { From aab073ab1613abd6382bd2e5f93906fec7aeb007 Mon Sep 17 00:00:00 2001 From: Fu Chen Date: Tue, 21 Nov 2023 16:18:14 +0800 Subject: [PATCH 14/31] [CELEBORN-1125] Bump guava from 14.0.1 to 32.1.3-jre ### What changes were proposed in this pull request? As title ### Why are the changes needed? - bump guava from 14.0.1 to 32.1.3-jre - refer to https://github.com/apache/spark/pull/26911, remove usages of Guava that no longer work in Guava 27/32, and replace with workalikes. After this PR, Celeborn no longer relies on a specific version of Guava, and is compatible with Guava 14/27/32. we have the ability to specify Guava to 27 when running MapReduce integration tests. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GA Closes #2090 from cfmcgrady/guava-27. Authored-by: Fu Chen Signed-off-by: mingji --- .../buffer/FileSegmentManagedBuffer.java | 11 +-- .../network/buffer/NettyManagedBuffer.java | 7 +- .../network/buffer/NioManagedBuffer.java | 7 +- .../network/client/TransportClient.java | 9 ++- .../network/protocol/ChunkFetchFailure.java | 13 ++-- .../network/protocol/ChunkFetchRequest.java | 7 +- .../network/protocol/ChunkFetchSuccess.java | 13 ++-- .../common/network/protocol/Message.java | 4 +- .../network/protocol/OneWayMessage.java | 11 ++- .../common/network/protocol/OpenStream.java | 16 ++-- .../common/network/protocol/PushData.java | 19 +++-- .../network/protocol/PushDataHandShake.java | 22 +++--- .../network/protocol/PushMergedData.java | 20 ++--- .../common/network/protocol/RegionFinish.java | 17 ++-- .../common/network/protocol/RegionStart.java | 21 ++--- .../common/network/protocol/RpcFailure.java | 13 ++-- .../common/network/protocol/RpcRequest.java | 12 ++- .../common/network/protocol/RpcResponse.java | 12 ++- .../network/protocol/StreamChunkSlice.java | 17 ++-- .../common/network/protocol/StreamHandle.java | 13 ++-- .../celeborn/common/util/ThreadUtils.scala | 78 ++++++++++++++++++- dev/deps/dependencies-client-flink-1.14 | 3 +- dev/deps/dependencies-client-flink-1.15 | 3 +- dev/deps/dependencies-client-flink-1.17 | 3 +- dev/deps/dependencies-client-flink-1.18 | 3 +- dev/deps/dependencies-client-mr | 3 +- dev/deps/dependencies-client-spark-2.4 | 3 +- dev/deps/dependencies-client-spark-3.0 | 3 +- dev/deps/dependencies-client-spark-3.1 | 3 +- dev/deps/dependencies-client-spark-3.2 | 3 +- dev/deps/dependencies-client-spark-3.3 | 3 +- dev/deps/dependencies-client-spark-3.4 | 3 +- dev/deps/dependencies-client-spark-3.5 | 3 +- dev/deps/dependencies-server | 3 +- pom.xml | 24 +++++- project/CelebornBuild.scala | 9 ++- 36 files changed, 294 insertions(+), 120 deletions(-) diff --git a/common/src/main/java/org/apache/celeborn/common/network/buffer/FileSegmentManagedBuffer.java b/common/src/main/java/org/apache/celeborn/common/network/buffer/FileSegmentManagedBuffer.java index dbd9d71ad82..6af9e43050d 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/buffer/FileSegmentManagedBuffer.java +++ b/common/src/main/java/org/apache/celeborn/common/network/buffer/FileSegmentManagedBuffer.java @@ -22,9 +22,10 @@ import java.nio.channels.FileChannel; import java.nio.file.StandardOpenOption; -import com.google.common.base.Objects; import com.google.common.io.ByteStreams; import io.netty.channel.DefaultFileRegion; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.celeborn.common.network.util.LimitedInputStream; import org.apache.celeborn.common.network.util.TransportConf; @@ -145,10 +146,10 @@ public long getLength() { @Override public String toString() { - return Objects.toStringHelper(this) - .add("file", file) - .add("offset", offset) - .add("length", length) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("file", file) + .append("offset", offset) + .append("length", length) .toString(); } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/buffer/NettyManagedBuffer.java b/common/src/main/java/org/apache/celeborn/common/network/buffer/NettyManagedBuffer.java index d3be0a6d960..60cf8625b1c 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/buffer/NettyManagedBuffer.java +++ b/common/src/main/java/org/apache/celeborn/common/network/buffer/NettyManagedBuffer.java @@ -21,10 +21,11 @@ import java.io.InputStream; import java.nio.ByteBuffer; -import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufInputStream; import io.netty.buffer.Unpooled; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; /** A {@link ManagedBuffer} backed by a Netty {@link ByteBuf}. */ public class NettyManagedBuffer extends ManagedBuffer { @@ -77,6 +78,8 @@ public Object convertToNetty() throws IOException { @Override public String toString() { - return Objects.toStringHelper(this).add("buf", buf).toString(); + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("buf", buf) + .toString(); } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/buffer/NioManagedBuffer.java b/common/src/main/java/org/apache/celeborn/common/network/buffer/NioManagedBuffer.java index 70a355a2efd..b14cb1f8ee0 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/buffer/NioManagedBuffer.java +++ b/common/src/main/java/org/apache/celeborn/common/network/buffer/NioManagedBuffer.java @@ -21,9 +21,10 @@ import java.io.InputStream; import java.nio.ByteBuffer; -import com.google.common.base.Objects; import io.netty.buffer.ByteBufInputStream; import io.netty.buffer.Unpooled; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; /** A {@link ManagedBuffer} backed by {@link ByteBuffer}. */ public class NioManagedBuffer extends ManagedBuffer { @@ -65,6 +66,8 @@ public Object convertToNetty() throws IOException { @Override public String toString() { - return Objects.toStringHelper(this).add("buf", buf).toString(); + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("buf", buf) + .toString(); } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/client/TransportClient.java b/common/src/main/java/org/apache/celeborn/common/network/client/TransportClient.java index 7865732c37d..482356b046b 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/client/TransportClient.java +++ b/common/src/main/java/org/apache/celeborn/common/network/client/TransportClient.java @@ -25,13 +25,14 @@ import java.util.concurrent.atomic.AtomicLong; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.util.concurrent.SettableFuture; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.GenericFutureListener; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -314,9 +315,9 @@ public void close() { @Override public String toString() { - return Objects.toStringHelper(this) - .add("remoteAddress", channel.remoteAddress()) - .add("isActive", isActive()) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("remoteAdress", channel.remoteAddress()) + .append("isActive", isActive()) .toString(); } diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/ChunkFetchFailure.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/ChunkFetchFailure.java index 532d0bab049..b746aa75399 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/ChunkFetchFailure.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/ChunkFetchFailure.java @@ -17,8 +17,11 @@ package org.apache.celeborn.common.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.celeborn.common.protocol.PbChunkFetchRequest; @@ -56,7 +59,7 @@ public static ChunkFetchFailure decode(ByteBuf buf) { @Override public int hashCode() { - return Objects.hashCode(streamChunkSlice, errorString); + return Objects.hash(streamChunkSlice, errorString); } @Override @@ -70,9 +73,9 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("streamChunkId", streamChunkSlice) - .add("errorString", errorString) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamChunkId", streamChunkSlice) + .append("errorString", errorString) .toString(); } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/ChunkFetchRequest.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/ChunkFetchRequest.java index 28672eac1d8..fe4a8a48fb1 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/ChunkFetchRequest.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/ChunkFetchRequest.java @@ -17,8 +17,9 @@ package org.apache.celeborn.common.network.protocol; -import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; /** * Request to fetch a sequence of a single chunk of a stream. This will correspond to a single @@ -67,6 +68,8 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this).add("streamChunkId", streamChunkSlice).toString(); + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamChunkId", streamChunkSlice) + .toString(); } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/ChunkFetchSuccess.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/ChunkFetchSuccess.java index 7d59920035d..d0297ec5a4a 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/ChunkFetchSuccess.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/ChunkFetchSuccess.java @@ -17,8 +17,11 @@ package org.apache.celeborn.common.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.celeborn.common.network.buffer.ManagedBuffer; import org.apache.celeborn.common.network.buffer.NettyManagedBuffer; @@ -76,7 +79,7 @@ public static ChunkFetchSuccess decode(ByteBuf buf, boolean decodeBody) { @Override public int hashCode() { - return Objects.hashCode(streamChunkSlice, body()); + return Objects.hash(streamChunkSlice, body()); } @Override @@ -90,9 +93,9 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("streamChunkId", streamChunkSlice) - .add("buffer", body()) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamChunkId", streamChunkSlice) + .append("buffer", body()) .toString(); } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/Message.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/Message.java index b2aaf735417..612dfb7bf9a 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/Message.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/Message.java @@ -18,8 +18,8 @@ package org.apache.celeborn.common.network.protocol; import java.nio.ByteBuffer; +import java.util.Objects; -import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; @@ -56,7 +56,7 @@ public boolean needCopyOut() { } protected boolean equals(Message other) { - return Objects.equal(body, other.body); + return Objects.equals(body, other.body); } public ByteBuffer toByteBuffer() { diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/OneWayMessage.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/OneWayMessage.java index 68c5a4263b8..8824e11a38e 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/OneWayMessage.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/OneWayMessage.java @@ -17,8 +17,11 @@ package org.apache.celeborn.common.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.celeborn.common.network.buffer.ManagedBuffer; import org.apache.celeborn.common.network.buffer.NettyManagedBuffer; @@ -66,7 +69,7 @@ public static OneWayMessage decode(ByteBuf buf, boolean decodeBody) { @Override public int hashCode() { - return Objects.hashCode(body()); + return Objects.hash(body()); } @Override @@ -80,6 +83,8 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this).add("body", body()).toString(); + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("body", body()) + .toString(); } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/OpenStream.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/OpenStream.java index 7919900e764..4a041e32758 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/OpenStream.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/OpenStream.java @@ -19,9 +19,11 @@ import java.nio.charset.StandardCharsets; import java.util.Arrays; +import java.util.Objects; -import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; /** * Request to read a set of blocks. Returns {@link StreamHandle}. Use PbOpenStream instead of this @@ -81,7 +83,7 @@ public static OpenStream decode(ByteBuf buf) { @Override public int hashCode() { - return Objects.hashCode( + return Objects.hash( Arrays.hashCode(shuffleKey), Arrays.hashCode(fileName), startMapIndex, endMapIndex); } @@ -99,11 +101,11 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("shuffleKey", new String(shuffleKey, StandardCharsets.UTF_8)) - .add("fileName", new String(fileName, StandardCharsets.UTF_8)) - .add("startMapIndex", startMapIndex) - .add("endMapIndex", endMapIndex) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("shuffleKey", new String(shuffleKey, StandardCharsets.UTF_8)) + .append("fileName", new String(fileName, StandardCharsets.UTF_8)) + .append("startMapIndex", startMapIndex) + .append("endMapIndex", endMapIndex) .toString(); } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/PushData.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/PushData.java index a3e5da3d2e3..f8765d5a200 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/PushData.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/PushData.java @@ -17,8 +17,11 @@ package org.apache.celeborn.common.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.celeborn.common.network.buffer.ManagedBuffer; import org.apache.celeborn.common.network.buffer.NettyManagedBuffer; @@ -86,7 +89,7 @@ public static PushData decode(ByteBuf buf, boolean decodeBody) { @Override public int hashCode() { - return Objects.hashCode(requestId, mode, shuffleKey, partitionUniqueId, body()); + return Objects.hash(requestId, mode, shuffleKey, partitionUniqueId, body()); } @Override @@ -104,12 +107,12 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("requestId", requestId) - .add("mode", mode) - .add("shuffleKey", shuffleKey) - .add("partitionUniqueId", partitionUniqueId) - .add("body size", body().size()) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("requestId", requestId) + .append("mode", mode) + .append("shuffleKey", shuffleKey) + .append("partitionUniqueId", partitionUniqueId) + .append("body size", body().size()) .toString(); } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/PushDataHandShake.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/PushDataHandShake.java index dc8c0481680..1f6745d729a 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/PushDataHandShake.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/PushDataHandShake.java @@ -17,8 +17,11 @@ package org.apache.celeborn.common.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; @Deprecated public final class PushDataHandShake extends RequestMessage { @@ -83,8 +86,7 @@ public static PushDataHandShake decode(ByteBuf buf) { @Override public int hashCode() { - return Objects.hashCode( - mode, shuffleKey, partitionUniqueId, attemptId, numPartitions, bufferSize); + return Objects.hash(mode, shuffleKey, partitionUniqueId, attemptId, numPartitions, bufferSize); } @Override @@ -104,13 +106,13 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("mode", mode) - .add("shuffleKey", shuffleKey) - .add("partitionUniqueId", partitionUniqueId) - .add("attemptId", attemptId) - .add("numSubPartitions", numPartitions) - .add("bufferSize", bufferSize) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("mode", mode) + .append("shuffleKey", shuffleKey) + .append("partitionUniqueId", partitionUniqueId) + .append("attemptId", attemptId) + .append("numSubPartitions", numPartitions) + .append("bufferSize", bufferSize) .toString(); } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/PushMergedData.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/PushMergedData.java index 332e66f6f1e..9152e5a5951 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/PushMergedData.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/PushMergedData.java @@ -18,9 +18,11 @@ package org.apache.celeborn.common.network.protocol; import java.util.Arrays; +import java.util.Objects; -import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.celeborn.common.network.buffer.ManagedBuffer; import org.apache.celeborn.common.network.buffer.NettyManagedBuffer; @@ -99,7 +101,7 @@ public static PushMergedData decode(ByteBuf buf, boolean decodeBody) { @Override public int hashCode() { - return Objects.hashCode(requestId, mode, shuffleKey); + return Objects.hash(requestId, mode, shuffleKey); } @Override @@ -118,13 +120,13 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("requestId", requestId) - .add("mode", mode) - .add("shuffleKey", shuffleKey) - .add("partitionIds", Arrays.toString(partitionUniqueIds)) - .add("batchOffsets", Arrays.toString(batchOffsets)) - .add("body size", body().size()) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("requestId", requestId) + .append("mode", mode) + .append("shuffleKey", shuffleKey) + .append("partitionIds", Arrays.toString(partitionUniqueIds)) + .append("batchOffsets", Arrays.toString(batchOffsets)) + .append("body size", body().size()) .toString(); } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/RegionFinish.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/RegionFinish.java index c7f804d84c7..0d64bdcdd26 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/RegionFinish.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/RegionFinish.java @@ -17,8 +17,11 @@ package org.apache.celeborn.common.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; @Deprecated public final class RegionFinish extends RequestMessage { @@ -68,7 +71,7 @@ public static RegionFinish decode(ByteBuf buf) { @Override public int hashCode() { - return Objects.hashCode(mode, shuffleKey, partitionUniqueId, attemptId); + return Objects.hash(mode, shuffleKey, partitionUniqueId, attemptId); } @Override @@ -86,11 +89,11 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("mode", mode) - .add("shuffleKey", shuffleKey) - .add("partitionUniqueId", partitionUniqueId) - .add("attemptId", attemptId) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("mode", mode) + .append("shuffleKey", shuffleKey) + .append("partitionUniqueId", partitionUniqueId) + .append("attemptId", attemptId) .toString(); } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/RegionStart.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/RegionStart.java index 2c327c880b7..f12c4a010f2 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/RegionStart.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/RegionStart.java @@ -17,8 +17,11 @@ package org.apache.celeborn.common.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; @Deprecated public final class RegionStart extends RequestMessage { @@ -85,7 +88,7 @@ public static RegionStart decode(ByteBuf buf) { @Override public int hashCode() { - return Objects.hashCode( + return Objects.hash( mode, shuffleKey, partitionUniqueId, attemptId, currentRegionIndex, isBroadcast); } @@ -106,13 +109,13 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("mode", mode) - .add("shuffleKey", shuffleKey) - .add("partitionUniqueId", partitionUniqueId) - .add("attemptId", attemptId) - .add("currentRegionIndex", currentRegionIndex) - .add("isBroadcast", isBroadcast) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("mode", mode) + .append("shuffleKey", shuffleKey) + .append("partitionUniqueId", partitionUniqueId) + .append("attemptId", attemptId) + .append("currentRegionIndex", currentRegionIndex) + .append("isBroadcast", isBroadcast) .toString(); } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/RpcFailure.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/RpcFailure.java index f1041959544..b49fa1b4dbb 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/RpcFailure.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/RpcFailure.java @@ -17,8 +17,11 @@ package org.apache.celeborn.common.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; /** Response to {@link RpcRequest} for a failed RPC. */ public final class RpcFailure extends ResponseMessage { @@ -54,7 +57,7 @@ public static RpcFailure decode(ByteBuf buf) { @Override public int hashCode() { - return Objects.hashCode(requestId, errorString); + return Objects.hash(requestId, errorString); } @Override @@ -68,9 +71,9 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("requestId", requestId) - .add("errorString", errorString) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("requestId", requestId) + .append("errorString", errorString) .toString(); } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/RpcRequest.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/RpcRequest.java index 93066f2245c..8d408ec81f2 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/RpcRequest.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/RpcRequest.java @@ -17,8 +17,11 @@ package org.apache.celeborn.common.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.celeborn.common.network.buffer.ManagedBuffer; import org.apache.celeborn.common.network.buffer.NettyManagedBuffer; @@ -74,7 +77,7 @@ public static RpcRequest decode(ByteBuf buf, boolean decodeBody) { @Override public int hashCode() { - return Objects.hashCode(requestId, body()); + return Objects.hash(requestId, body()); } @Override @@ -88,6 +91,9 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this).add("requestId", requestId).add("body", body()).toString(); + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("requestId", requestId) + .append("body", body()) + .toString(); } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/RpcResponse.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/RpcResponse.java index ea37627944f..4d31716a5c9 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/RpcResponse.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/RpcResponse.java @@ -17,8 +17,11 @@ package org.apache.celeborn.common.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.celeborn.common.network.buffer.ManagedBuffer; import org.apache.celeborn.common.network.buffer.NettyManagedBuffer; @@ -74,7 +77,7 @@ public static RpcResponse decode(ByteBuf buf, boolean decodeBody) { @Override public int hashCode() { - return Objects.hashCode(requestId, body()); + return Objects.hash(requestId, body()); } @Override @@ -88,6 +91,9 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this).add("requestId", requestId).add("body", body()).toString(); + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("requestId", requestId) + .append("body", body()) + .toString(); } } diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/StreamChunkSlice.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/StreamChunkSlice.java index 4771faf8726..b0129a42b9b 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/StreamChunkSlice.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/StreamChunkSlice.java @@ -17,8 +17,11 @@ package org.apache.celeborn.common.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.celeborn.common.protocol.PbStreamChunkSlice; @@ -68,7 +71,7 @@ public static StreamChunkSlice decode(ByteBuf buffer) { @Override public int hashCode() { - return Objects.hashCode(streamId, chunkIndex, offset, len); + return Objects.hash(streamId, chunkIndex, offset, len); } @Override @@ -85,11 +88,11 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("streamId", streamId) - .add("chunkIndex", chunkIndex) - .add("offset", offset) - .add("len", len) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamId", streamId) + .append("chunkIndex", chunkIndex) + .append("offset", offset) + .append("len", len) .toString(); } diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/StreamHandle.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/StreamHandle.java index c2428728ef7..859ec5fe417 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/StreamHandle.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/StreamHandle.java @@ -17,8 +17,11 @@ package org.apache.celeborn.common.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; /** * Identifier for a fixed number of chunks to read from a stream created by an "open blocks" @@ -56,7 +59,7 @@ public static StreamHandle decode(ByteBuf buf) { @Override public int hashCode() { - return Objects.hashCode(streamId, numChunks); + return Objects.hash(streamId, numChunks); } @Override @@ -70,9 +73,9 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("streamId", streamId) - .add("numChunks", numChunks) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamId", streamId) + .append("numChunks", numChunks) .toString(); } } diff --git a/common/src/main/scala/org/apache/celeborn/common/util/ThreadUtils.scala b/common/src/main/scala/org/apache/celeborn/common/util/ThreadUtils.scala index 822ee032776..233bacd0b14 100644 --- a/common/src/main/scala/org/apache/celeborn/common/util/ThreadUtils.scala +++ b/common/src/main/scala/org/apache/celeborn/common/util/ThreadUtils.scala @@ -19,6 +19,7 @@ package org.apache.celeborn.common.util import java.util.concurrent._ import java.util.concurrent.{ForkJoinPool => SForkJoinPool, ForkJoinWorkerThread => SForkJoinWorkerThread} +import java.util.concurrent.locks.ReentrantLock import scala.concurrent.{Awaitable, ExecutionContext, ExecutionContextExecutor, Future} import scala.concurrent.duration.{Duration, FiniteDuration} @@ -33,7 +34,82 @@ import org.apache.celeborn.common.internal.Logging object ThreadUtils { private val sameThreadExecutionContext = - ExecutionContext.fromExecutorService(MoreExecutors.sameThreadExecutor()) + ExecutionContext.fromExecutorService(sameThreadExecutorService()) + + // Inspired by Guava MoreExecutors.sameThreadExecutor; inlined and converted + // to Scala here to avoid Guava version issues + def sameThreadExecutorService(): ExecutorService = new AbstractExecutorService { + private val lock = new ReentrantLock() + private val termination = lock.newCondition() + private var runningTasks = 0 + private var serviceIsShutdown = false + + override def shutdown(): Unit = { + lock.lock() + try { + serviceIsShutdown = true + } finally { + lock.unlock() + } + } + + override def shutdownNow(): java.util.List[Runnable] = { + shutdown() + java.util.Collections.emptyList() + } + + override def isShutdown: Boolean = { + lock.lock() + try { + serviceIsShutdown + } finally { + lock.unlock() + } + } + + override def isTerminated: Boolean = synchronized { + lock.lock() + try { + serviceIsShutdown && runningTasks == 0 + } finally { + lock.unlock() + } + } + + override def awaitTermination(timeout: Long, unit: TimeUnit): Boolean = { + var nanos = unit.toNanos(timeout) + lock.lock() + try { + while (nanos > 0 && !isTerminated()) { + nanos = termination.awaitNanos(nanos) + } + isTerminated() + } finally { + lock.unlock() + } + } + + override def execute(command: Runnable): Unit = { + lock.lock() + try { + if (isShutdown()) throw new RejectedExecutionException("Executor already shutdown") + runningTasks += 1 + } finally { + lock.unlock() + } + try { + command.run() + } finally { + lock.lock() + try { + runningTasks -= 1 + if (isTerminated()) termination.signalAll() + } finally { + lock.unlock() + } + } + } + } /** * An `ExecutionContextExecutor` that runs each task in the thread that invokes `execute/submit`. diff --git a/dev/deps/dependencies-client-flink-1.14 b/dev/deps/dependencies-client-flink-1.14 index 7e8c870ff48..456250bb961 100644 --- a/dev/deps/dependencies-client-flink-1.14 +++ b/dev/deps/dependencies-client-flink-1.14 @@ -20,7 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -guava/14.0.1//guava-14.0.1.jar +failureaccess/1.0.1//failureaccess-1.0.1.jar +guava/32.1.3-jre//guava-32.1.3-jre.jar hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar diff --git a/dev/deps/dependencies-client-flink-1.15 b/dev/deps/dependencies-client-flink-1.15 index 7e8c870ff48..456250bb961 100644 --- a/dev/deps/dependencies-client-flink-1.15 +++ b/dev/deps/dependencies-client-flink-1.15 @@ -20,7 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -guava/14.0.1//guava-14.0.1.jar +failureaccess/1.0.1//failureaccess-1.0.1.jar +guava/32.1.3-jre//guava-32.1.3-jre.jar hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar diff --git a/dev/deps/dependencies-client-flink-1.17 b/dev/deps/dependencies-client-flink-1.17 index 7e8c870ff48..456250bb961 100644 --- a/dev/deps/dependencies-client-flink-1.17 +++ b/dev/deps/dependencies-client-flink-1.17 @@ -20,7 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -guava/14.0.1//guava-14.0.1.jar +failureaccess/1.0.1//failureaccess-1.0.1.jar +guava/32.1.3-jre//guava-32.1.3-jre.jar hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar diff --git a/dev/deps/dependencies-client-flink-1.18 b/dev/deps/dependencies-client-flink-1.18 index 7e8c870ff48..456250bb961 100644 --- a/dev/deps/dependencies-client-flink-1.18 +++ b/dev/deps/dependencies-client-flink-1.18 @@ -20,7 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -guava/14.0.1//guava-14.0.1.jar +failureaccess/1.0.1//failureaccess-1.0.1.jar +guava/32.1.3-jre//guava-32.1.3-jre.jar hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar diff --git a/dev/deps/dependencies-client-mr b/dev/deps/dependencies-client-mr index 8f6a629affa..cc8ef1ade7c 100644 --- a/dev/deps/dependencies-client-mr +++ b/dev/deps/dependencies-client-mr @@ -41,9 +41,10 @@ curator-framework/2.13.0//curator-framework-2.13.0.jar curator-recipes/2.13.0//curator-recipes-2.13.0.jar dnsjava/2.1.7//dnsjava-2.1.7.jar ehcache/3.3.1//ehcache-3.3.1.jar +failureaccess/1.0.1//failureaccess-1.0.1.jar geronimo-jcache_1.0_spec/1.0-alpha-1//geronimo-jcache_1.0_spec-1.0-alpha-1.jar gson/2.9.0//gson-2.9.0.jar -guava/14.0.1//guava-14.0.1.jar +guava/32.1.3-jre//guava-32.1.3-jre.jar guice-servlet/4.0//guice-servlet-4.0.jar guice/4.0//guice-4.0.jar hadoop-annotations/3.2.4//hadoop-annotations-3.2.4.jar diff --git a/dev/deps/dependencies-client-spark-2.4 b/dev/deps/dependencies-client-spark-2.4 index b759797500a..c439d7e08fe 100644 --- a/dev/deps/dependencies-client-spark-2.4 +++ b/dev/deps/dependencies-client-spark-2.4 @@ -20,7 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -guava/14.0.1//guava-14.0.1.jar +failureaccess/1.0.1//failureaccess-1.0.1.jar +guava/32.1.3-jre//guava-32.1.3-jre.jar hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar diff --git a/dev/deps/dependencies-client-spark-3.0 b/dev/deps/dependencies-client-spark-3.0 index 5005ba525cf..8ad78eccb8a 100644 --- a/dev/deps/dependencies-client-spark-3.0 +++ b/dev/deps/dependencies-client-spark-3.0 @@ -20,7 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -guava/14.0.1//guava-14.0.1.jar +failureaccess/1.0.1//failureaccess-1.0.1.jar +guava/32.1.3-jre//guava-32.1.3-jre.jar hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar diff --git a/dev/deps/dependencies-client-spark-3.1 b/dev/deps/dependencies-client-spark-3.1 index 6e8fd970315..1cbd01b35fc 100644 --- a/dev/deps/dependencies-client-spark-3.1 +++ b/dev/deps/dependencies-client-spark-3.1 @@ -20,7 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -guava/14.0.1//guava-14.0.1.jar +failureaccess/1.0.1//failureaccess-1.0.1.jar +guava/32.1.3-jre//guava-32.1.3-jre.jar hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar diff --git a/dev/deps/dependencies-client-spark-3.2 b/dev/deps/dependencies-client-spark-3.2 index 6fbebf79d10..e83df4a7192 100644 --- a/dev/deps/dependencies-client-spark-3.2 +++ b/dev/deps/dependencies-client-spark-3.2 @@ -20,7 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -guava/14.0.1//guava-14.0.1.jar +failureaccess/1.0.1//failureaccess-1.0.1.jar +guava/32.1.3-jre//guava-32.1.3-jre.jar hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar diff --git a/dev/deps/dependencies-client-spark-3.3 b/dev/deps/dependencies-client-spark-3.3 index 7e8c870ff48..456250bb961 100644 --- a/dev/deps/dependencies-client-spark-3.3 +++ b/dev/deps/dependencies-client-spark-3.3 @@ -20,7 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -guava/14.0.1//guava-14.0.1.jar +failureaccess/1.0.1//failureaccess-1.0.1.jar +guava/32.1.3-jre//guava-32.1.3-jre.jar hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar diff --git a/dev/deps/dependencies-client-spark-3.4 b/dev/deps/dependencies-client-spark-3.4 index c6a87cc07ac..35698f48756 100644 --- a/dev/deps/dependencies-client-spark-3.4 +++ b/dev/deps/dependencies-client-spark-3.4 @@ -20,7 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -guava/14.0.1//guava-14.0.1.jar +failureaccess/1.0.1//failureaccess-1.0.1.jar +guava/32.1.3-jre//guava-32.1.3-jre.jar hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar diff --git a/dev/deps/dependencies-client-spark-3.5 b/dev/deps/dependencies-client-spark-3.5 index 98abc8a5fb9..481caffcc68 100644 --- a/dev/deps/dependencies-client-spark-3.5 +++ b/dev/deps/dependencies-client-spark-3.5 @@ -20,7 +20,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -guava/14.0.1//guava-14.0.1.jar +failureaccess/1.0.1//failureaccess-1.0.1.jar +guava/32.1.3-jre//guava-32.1.3-jre.jar hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar diff --git a/dev/deps/dependencies-server b/dev/deps/dependencies-server index f93601cc352..89ac60462c3 100644 --- a/dev/deps/dependencies-server +++ b/dev/deps/dependencies-server @@ -21,7 +21,8 @@ commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar -guava/14.0.1//guava-14.0.1.jar +failureaccess/1.0.1//failureaccess-1.0.1.jar +guava/32.1.3-jre//guava-32.1.3-jre.jar hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar diff --git a/pom.xml b/pom.xml index 8ba97fbff35..1927ebffea6 100644 --- a/pom.xml +++ b/pom.xml @@ -78,7 +78,7 @@ 9+181-r4173-1 1.3.9 1.44.0 - 14.0.1 + 32.1.3-jre 3.1.0 4.13.2 1.8 @@ -311,6 +311,28 @@ com.google.guava guava ${guava.version} + + + org.checkerframework + checker-qual + + + org.codehaus.mojo + animal-sniffer-annotations + + + com.google.errorprone + error_prone_annotations + + + com.google.guava + listenablefuture + + + com.google.j2objc + j2objc-annotations + + com.google.code.findbugs diff --git a/project/CelebornBuild.scala b/project/CelebornBuild.scala index 168a6f3bd5f..3d7b748cff0 100644 --- a/project/CelebornBuild.scala +++ b/project/CelebornBuild.scala @@ -42,7 +42,7 @@ object Dependencies { val commonsLoggingVersion = "1.1.3" val commonsLang3Version = "3.12.0" val findbugsVersion = "1.3.9" - val guavaVersion = "14.0.1" + val guavaVersion = "32.1.3-jre" val hadoopVersion = "3.2.4" val javaxServletVersion = "3.1.0" val junitInterfaceVersion = "0.13.3" @@ -72,7 +72,12 @@ object Dependencies { val commonsLang3 = "org.apache.commons" % "commons-lang3" % commonsLang3Version val commonsLogging = "commons-logging" % "commons-logging" % commonsLoggingVersion val findbugsJsr305 = "com.google.code.findbugs" % "jsr305" % findbugsVersion - val guava = "com.google.guava" % "guava" % guavaVersion + val guava = "com.google.guava" % "guava" % guavaVersion excludeAll( + ExclusionRule("org.checkerframework", "checker-qual"), + ExclusionRule("org.codehaus.mojo", "animal-sniffer-annotations"), + ExclusionRule("com.google.errorprone", "error_prone_annotations"), + ExclusionRule("com.google.guava", "listenablefuture"), + ExclusionRule("com.google.j2objc", "j2objc-annotations")) val hadoopClientApi = "org.apache.hadoop" % "hadoop-client-api" % hadoopVersion val hadoopClientRuntime = "org.apache.hadoop" % "hadoop-client-runtime" % hadoopVersion val hadoopMapreduceClientApp = "org.apache.hadoop" % "hadoop-mapreduce-client-app" % hadoopVersion excludeAll( From a275b64b320852acfa5870f40ce21111e39fb6d5 Mon Sep 17 00:00:00 2001 From: SteNicholas Date: Wed, 22 Nov 2023 10:26:53 +0800 Subject: [PATCH 15/31] [CELEBORN-1137] Correct suggested leader of exception message for MasterNotLeaderException ### What changes were proposed in this pull request? `MasterNotLeaderException` corrects the suggested leader of exception message. ### Why are the changes needed? When current peer isn't the leader of master and the leader is switching which cache isn't expired, the suggested leader of exception message in MasterNotLeaderException is confusing that the suggested leader is current peer. It's recommened to correct suggested leader of exception message for MasterNotLeaderException if current peer is equal to the suggested leader. ``` Caused by: org.apache.celeborn.common.haclient.MasterNotLeaderException: Master:xx.xx.xx.xx:9099 is not the leader. Suggested leader is Master:xx.xx.xx.xx:9099. Exception:bound must be positive. at org.apache.celeborn.service.deploy.master.clustermeta.ha.HAHelper.sendFailure(HAHelper.java:58) at org.apache.celeborn.service.deploy.master.Master.executeWithLeaderChecker(Master.scala:236) at org.apache.celeborn.service.deploy.master.Master$$anonfun$receiveAndReply$1.applyOrElse(Master.scala:314) ... 7 more Caused by: java.lang.IllegalArgumentException: bound must be positive at java.util.Random.nextInt(Random.java:388) at org.apache.celeborn.service.deploy.master.SlotsAllocator.roundRobin(SlotsAllocator.java:202) at org.apache.celeborn.service.deploy.master.SlotsAllocator.offerSlotsLoadAware(SlotsAllocator.java:151) at org.apache.celeborn.service.deploy.master.Master.$anonfun$handleRequestSlots$1(Master.scala:598) at org.apache.celeborn.common.metrics.source.AbstractSource.sample(AbstractSource.scala:199) at org.apache.celeborn.common.metrics.source.AbstractSource.sample(AbstractSource.scala:189) at org.apache.celeborn.service.deploy.master.Master.handleRequestSlots(Master.scala:587) at org.apache.celeborn.service.deploy.master.Master$$anonfun$receiveAndReply$1.$anonfun$applyOrElse$12(Master.scala:314) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) at org.apache.celeborn.service.deploy.master.Master.executeWithLeaderChecker(Master.scala:233) ... 8 more ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? No. Closes #2109 from SteNicholas/CELEBORN-1137. Authored-by: SteNicholas Signed-off-by: mingji --- .../celeborn/common/client/MasterNotLeaderException.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/common/src/main/java/org/apache/celeborn/common/client/MasterNotLeaderException.java b/common/src/main/java/org/apache/celeborn/common/client/MasterNotLeaderException.java index 6502f44dbd5..38198cf786e 100644 --- a/common/src/main/java/org/apache/celeborn/common/client/MasterNotLeaderException.java +++ b/common/src/main/java/org/apache/celeborn/common/client/MasterNotLeaderException.java @@ -35,9 +35,11 @@ public MasterNotLeaderException( String currentPeer, String suggestedLeaderPeer, @Nullable Throwable cause) { super( String.format( - "Master:%s is not the leader. Suggested leader is Master:%s.%s", + "Master:%s is not the leader.%s%s", currentPeer, - suggestedLeaderPeer, + currentPeer.equals(suggestedLeaderPeer) + ? StringUtils.EMPTY + : String.format(" Suggested leader is Master:%s.", suggestedLeaderPeer), cause == null ? StringUtils.EMPTY : String.format(" Exception:%s.", cause.getMessage())), From 60871750e4c21360054584c4ba5f10809ee172f2 Mon Sep 17 00:00:00 2001 From: SteNicholas Date: Wed, 22 Nov 2023 14:08:06 +0800 Subject: [PATCH 16/31] [CELEBORN-1136] Support policy for master to assign slots fallback to roundrobin with no available slots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? `SlotsAllocator` supports policy for master to assign slots fallback to roundrobin with no available slots. ### Why are the changes needed? When the selected workers have no available slots, the loadaware policy could throw `MasterNotLeaderException`. It's recommended to support policy for master to assign slots fallback to roundrobin with no available slots. Meanwhile, the situation that there is no available slots would occur when the partition size has increased a lot in a short period of time. ``` Caused by: org.apache.celeborn.common.haclient.MasterNotLeaderException: Master:xx.xx.xx.xx:9099 is not the leader. Suggested leader is Master:xx.xx.xx.xx:9099. Exception:bound must be positive.     at org.apache.celeborn.service.deploy.master.clustermeta.ha.HAHelper.sendFailure(HAHelper.java:58)     at org.apache.celeborn.service.deploy.master.Master.executeWithLeaderChecker(Master.scala:236)     at org.apache.celeborn.service.deploy.master.Master$$anonfun$receiveAndReply$1.applyOrElse(Master.scala:314)     ... 7 more Caused by: java.lang.IllegalArgumentException: bound must be positive     at java.util.Random.nextInt(Random.java:388)     at org.apache.celeborn.service.deploy.master.SlotsAllocator.roundRobin(SlotsAllocator.java:202)     at org.apache.celeborn.service.deploy.master.SlotsAllocator.offerSlotsLoadAware(SlotsAllocator.java:151)     at org.apache.celeborn.service.deploy.master.Master.$anonfun$handleRequestSlots$1(Master.scala:598)     at org.apache.celeborn.common.metrics.source.AbstractSource.sample(AbstractSource.scala:199)     at org.apache.celeborn.common.metrics.source.AbstractSource.sample(AbstractSource.scala:189)     at org.apache.celeborn.service.deploy.master.Master.handleRequestSlots(Master.scala:587)     at org.apache.celeborn.service.deploy.master.Master$$anonfun$receiveAndReply$1.$anonfun$applyOrElse$12(Master.scala:314)     at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)     at org.apache.celeborn.service.deploy.master.Master.executeWithLeaderChecker(Master.scala:233)     ... 8 more ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? `SlotsAllocatorSuiteJ#testAllocateSlotsWithNoAvailableSlots` Closes #2108 from SteNicholas/CELEBORN-1136. Authored-by: SteNicholas Signed-off-by: mingji --- .../org/apache/celeborn/common/meta/DeviceInfo.scala | 11 ++++++++++- .../service/deploy/master/SlotsAllocator.java | 10 ++++++---- .../service/deploy/master/SlotsAllocatorSuiteJ.java | 12 ++++++++++++ 3 files changed, 28 insertions(+), 5 deletions(-) diff --git a/common/src/main/scala/org/apache/celeborn/common/meta/DeviceInfo.scala b/common/src/main/scala/org/apache/celeborn/common/meta/DeviceInfo.scala index 6bd49a1a0a2..56933a2cf04 100644 --- a/common/src/main/scala/org/apache/celeborn/common/meta/DeviceInfo.scala +++ b/common/src/main/scala/org/apache/celeborn/common/meta/DeviceInfo.scala @@ -92,8 +92,17 @@ class DiskInfo( avgFetchTime = fetchTimeMetrics.getAverage() } + /** + * Returns the available slots of the disk calculated by maxSlots minus activeSlots. + * Returns zero for the negative slots calculated. + * + * Note:`maxSlots` is calculated by actualUsableSpace divided estimatedPartitionSize. + * Meanwhile, `activeSlots` include slots reserved. + * + * @return the available slots of the disk. + */ def availableSlots(): Long = this.synchronized { - maxSlots - activeSlots + math.max(maxSlots - activeSlots, 0L) } def allocateSlots(shuffleKey: String, slots: Int): Unit = this.synchronized { diff --git a/master/src/main/java/org/apache/celeborn/service/deploy/master/SlotsAllocator.java b/master/src/main/java/org/apache/celeborn/service/deploy/master/SlotsAllocator.java index 2ee40de63c8..e964e57e556 100644 --- a/master/src/main/java/org/apache/celeborn/service/deploy/master/SlotsAllocator.java +++ b/master/src/main/java/org/apache/celeborn/service/deploy/master/SlotsAllocator.java @@ -131,16 +131,18 @@ static class UsableDiskInfo { } })); - boolean shouldFallback = + boolean noUsableDisks = usableDisks.isEmpty() || (shouldReplicate && (usableDisks.size() == 1 || usableDisks.stream().map(diskToWorkerMap::get).distinct().count() <= 1)); + boolean noAvailableSlots = usableDisks.stream().mapToLong(DiskInfo::availableSlots).sum() <= 0; - if (shouldFallback) { + if (noUsableDisks || noAvailableSlots) { logger.warn( - "offer slots for {} fallback to roundrobin because there is no usable disks", - StringUtils.join(partitionIds, ',')); + "offer slots for {} fallback to roundrobin because there is no {}", + StringUtils.join(partitionIds, ','), + noUsableDisks ? "usable disks" : "available slots"); return offerSlotsRoundRobin( workers, partitionIds, shouldReplicate, shouldRackAware, availableStorageTypes); } diff --git a/master/src/test/java/org/apache/celeborn/service/deploy/master/SlotsAllocatorSuiteJ.java b/master/src/test/java/org/apache/celeborn/service/deploy/master/SlotsAllocatorSuiteJ.java index 3300f6de2a2..92f2ecfbd14 100644 --- a/master/src/test/java/org/apache/celeborn/service/deploy/master/SlotsAllocatorSuiteJ.java +++ b/master/src/test/java/org/apache/celeborn/service/deploy/master/SlotsAllocatorSuiteJ.java @@ -334,4 +334,16 @@ public void testHDFSOnly() { final boolean shouldReplicate = true; checkSlotsOnHDFS(workers, partitionIds, shouldReplicate, true); } + + @Test + public void testAllocateSlotsWithNoAvailableSlots() { + final List workers = prepareWorkers(true); + // Simulates no available slots behavior with greatly changed estimatedPartitionSize for workers + // with usable disks. + workers.forEach(workerInfo -> workerInfo.updateDiskMaxSlots(Long.MAX_VALUE)); + final List partitionIds = Collections.singletonList(0); + final boolean shouldReplicate = false; + + check(workers, partitionIds, shouldReplicate, true); + } } From 3054813a0f1060359e0bd12f05b19eaef344c7a1 Mon Sep 17 00:00:00 2001 From: sychen Date: Wed, 22 Nov 2023 14:36:29 +0800 Subject: [PATCH 17/31] [CELEBORN-856] Add mapreduce integration test ### What changes were proposed in this pull request? ### Why are the changes needed? ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? Closes #2073 from cxzl25/CELEBORN-856. Authored-by: sychen Signed-off-by: mingji --- .github/workflows/maven.yml | 31 +++ .github/workflows/sbt.yml | 27 +++ pom.xml | 1 + project/CelebornBuild.scala | 53 +++++- tests/mr-it/pom.xml | 177 ++++++++++++++++++ .../test/resources/container-log4j.properties | 25 +++ .../mr-it/src/test/resources/log4j2-test.xml | 41 ++++ .../celeborn/tests/mr/WordCountTest.scala | 149 +++++++++++++++ 8 files changed, 502 insertions(+), 2 deletions(-) create mode 100644 tests/mr-it/pom.xml create mode 100644 tests/mr-it/src/test/resources/container-log4j.properties create mode 100644 tests/mr-it/src/test/resources/log4j2-test.xml create mode 100644 tests/mr-it/src/test/scala/org/apache/celeborn/tests/mr/WordCountTest.scala diff --git a/.github/workflows/maven.yml b/.github/workflows/maven.yml index 43e602d639d..52f7bb7a285 100644 --- a/.github/workflows/maven.yml +++ b/.github/workflows/maven.yml @@ -177,3 +177,34 @@ jobs: name: flink-${{ matrix.flink }}-unit-test-log path: | **/target/unit-tests.log + + mr: + runs-on: ubuntu-22.04 + strategy: + fail-fast: false + matrix: + java: + - 8 + - 11 + steps: + - uses: actions/checkout@v2 + - name: Setup JDK ${{ matrix.java }} + uses: actions/setup-java@v2 + with: + distribution: zulu + java-version: ${{ matrix.java }} + cache: maven + check-latest: false + - name: Test with Maven + run: | + PROFILES="-Pgoogle-mirror,mr" + TEST_MODULES="client-mr/mr,client-mr/mr-shaded,tests/mr-it" + build/mvn $PROFILES -pl $TEST_MODULES -am clean install -DskipTests + build/mvn $PROFILES -pl $TEST_MODULES test + - name: Upload test log + if: failure() + uses: actions/upload-artifact@v3 + with: + name: mr-unit-test-log + path: | + **/target/unit-tests.log diff --git a/.github/workflows/sbt.yml b/.github/workflows/sbt.yml index 740e6bdf794..71a58d36f21 100644 --- a/.github/workflows/sbt.yml +++ b/.github/workflows/sbt.yml @@ -222,3 +222,30 @@ jobs: name: flink-${{ matrix.flink }}-unit-test-log path: | **/target/test-reports/** + + mr: + runs-on: ubuntu-22.04 + strategy: + fail-fast: false + matrix: + java: + - 8 + - 11 + steps: + - uses: actions/checkout@v2 + - name: Setup JDK ${{ matrix.java }} + uses: actions/setup-java@v2 + with: + distribution: zulu + java-version: ${{ matrix.java }} + check-latest: false + - name: Test with SBT + run: | + build/sbt -Pmr "clean; celeborn-mr-group/test" + - name: Upload test log + if: failure() + uses: actions/upload-artifact@v3 + with: + name: mr-unit-test-log + path: | + **/target/test-reports/** \ No newline at end of file diff --git a/pom.xml b/pom.xml index 1927ebffea6..34dfaa680a2 100644 --- a/pom.xml +++ b/pom.xml @@ -1312,6 +1312,7 @@ client-mr/mr client-mr/mr-shaded + tests/mr-it diff --git a/project/CelebornBuild.scala b/project/CelebornBuild.scala index 3d7b748cff0..cf6b43e19e3 100644 --- a/project/CelebornBuild.scala +++ b/project/CelebornBuild.scala @@ -34,7 +34,7 @@ object Dependencies { val zstdJniVersion = sparkClientProjects.map(_.zstdJniVersion).getOrElse("1.5.2-1") val lz4JavaVersion = sparkClientProjects.map(_.lz4JavaVersion).getOrElse("1.8.0") - + // Dependent library versions val commonsCompressVersion = "1.4.1" val commonsCryptoVersion = "1.0.0" @@ -933,6 +933,24 @@ object MRClientProjects { ) } + def mrIt: Project = { + Project("celeborn-mr-it", file("tests/mr-it")) + // ref: https://www.scala-sbt.org/1.x/docs/Multi-Project.html#Classpath+dependencies + .dependsOn(CelebornCommon.common % "test->test;compile->compile") + .dependsOn(CelebornClient.client % "test->test;compile->compile") + .dependsOn(CelebornMaster.master % "test->test;compile->compile") + .dependsOn(CelebornWorker.worker % "test->test;compile->compile") + .dependsOn(mrClient % "test->test;compile->compile") + .settings( + commonSettings, + copyDepsSettings, + libraryDependencies ++= Seq( + "org.apache.hadoop" % "hadoop-client-minicluster" % Dependencies.hadoopVersion % "test", + "org.apache.hadoop" % "hadoop-mapreduce-examples" % Dependencies.hadoopVersion % "test" + ) ++ commonUnitTestDependencies + ) + } + def mrClientShade: Project = { Project("celeborn-client-mr-shaded", file("client-mr/mr-shaded")) .dependsOn(mrClient) @@ -996,6 +1014,37 @@ object MRClientProjects { } def modules: Seq[Project] = { - Seq(mrClient, mrClientShade) + Seq(mrClient, mrIt, mrGroup, mrClientShade) + } + + // for test only, don't use this group for any other projects + lazy val mrGroup = (project withId "celeborn-mr-group").aggregate(mrClient, mrIt) + + val copyDeps = TaskKey[Unit]("copyDeps", "Copies needed dependencies to the build directory.") + val destPath = (Compile / crossTarget) { + _ / "mapreduce_lib" } + + lazy val copyDepsSettings = Seq( + copyDeps := { + val dest = destPath.value + if (!dest.isDirectory() && !dest.mkdirs()) { + throw new java.io.IOException("Failed to create jars directory.") + } + + (Compile / dependencyClasspath).value.map(_.data) + .filter { jar => jar.isFile() } + .foreach { jar => + val destJar = new File(dest, jar.getName()) + if (destJar.isFile()) { + destJar.delete() + } + Files.copy(jar.toPath(), destJar.toPath()) + } + }, + (Test / compile) := { + copyDeps.value + (Test / compile).value + } + ) } diff --git a/tests/mr-it/pom.xml b/tests/mr-it/pom.xml new file mode 100644 index 00000000000..7c3f1c1e43c --- /dev/null +++ b/tests/mr-it/pom.xml @@ -0,0 +1,177 @@ + + + + 4.0.0 + + org.apache.celeborn + celeborn-parent_${scala.binary.version} + ${project.version} + ../../pom.xml + + + celeborn-mr-it_${scala.binary.version} + jar + Celeborn MapReduce Integration Test + + + + org.apache.celeborn + celeborn-common_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.celeborn + celeborn-client_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.celeborn + celeborn-master_${scala.binary.version} + ${project.version} + test + + + org.apache.celeborn + celeborn-worker_${scala.binary.version} + ${project.version} + test + + + org.apache.celeborn + celeborn-worker_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.celeborn + celeborn-client-mr-shaded_${scala.binary.version} + ${project.version} + test + + + org.apache.hadoop + hadoop-yarn-common + + + org.apache.hadoop + hadoop-yarn-server-common + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-auth + + + org.apache.hadoop + hadoop-yarn-server-web-proxy + + + org.apache.hadoop + hadoop-yarn-server-nodemanager + + + org.apache.hadoop + hadoop-mapreduce-client-common + + + + + org.apache.hadoop + hadoop-client-minicluster + ${hadoop.version} + test + + + org.xerial.snappy + snappy-java + + + junit + junit + + + + + org.apache.hadoop + hadoop-mapreduce-examples + ${hadoop.version} + test + + + org.apache.hadoop + hadoop-client-api + ${hadoop.version} + test + + + + + org.apache.hadoop + hadoop-yarn-common + ${hadoop.version} + provided + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + provided + + + org.apache.hadoop + hadoop-yarn-server-web-proxy + ${hadoop.version} + provided + + + org.apache.hadoop + hadoop-mapreduce-client-common + ${hadoop.version} + provided + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-dependencies + + copy-dependencies + + package + + ${project.build.directory}/mapreduce_lib + + + + + + + diff --git a/tests/mr-it/src/test/resources/container-log4j.properties b/tests/mr-it/src/test/resources/container-log4j.properties new file mode 100644 index 00000000000..c37e1d32f01 --- /dev/null +++ b/tests/mr-it/src/test/resources/container-log4j.properties @@ -0,0 +1,25 @@ +# +# 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. +# + +# STDOUT Appender +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.Target=System.out +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{1} - %m%n + +log4j.rootLogger=INFO, stdout \ No newline at end of file diff --git a/tests/mr-it/src/test/resources/log4j2-test.xml b/tests/mr-it/src/test/resources/log4j2-test.xml new file mode 100644 index 00000000000..9adcdccfd0e --- /dev/null +++ b/tests/mr-it/src/test/resources/log4j2-test.xml @@ -0,0 +1,41 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/tests/mr-it/src/test/scala/org/apache/celeborn/tests/mr/WordCountTest.scala b/tests/mr-it/src/test/scala/org/apache/celeborn/tests/mr/WordCountTest.scala new file mode 100644 index 00000000000..0f1bb91aaf2 --- /dev/null +++ b/tests/mr-it/src/test/scala/org/apache/celeborn/tests/mr/WordCountTest.scala @@ -0,0 +1,149 @@ +/* + * 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.tests.mr + +import java.io.File +import java.nio.charset.StandardCharsets +import java.nio.file.{Files, Paths} + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.examples.WordCount +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.{IntWritable, Text} +import org.apache.hadoop.mapred.ShuffleHandler +import org.apache.hadoop.mapreduce.{Job, MRJobConfig} +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat +import org.apache.hadoop.service.Service +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.server.MiniYARNCluster +import org.scalatest.BeforeAndAfterAll +import org.scalatest.funsuite.AnyFunSuite + +import org.apache.celeborn.common.internal.Logging +import org.apache.celeborn.common.util.Utils +import org.apache.celeborn.service.deploy.MiniClusterFeature +import org.apache.celeborn.service.deploy.worker.Worker + +class WordCountTest extends AnyFunSuite with Logging with MiniClusterFeature + with BeforeAndAfterAll { + var workers: collection.Set[Worker] = null + + var yarnCluster: MiniYARNCluster = null + var hadoopConf: Configuration = null + + override def beforeAll(): Unit = { + logInfo("test initialized , setup celeborn mini cluster") + val masterConf = Map( + "celeborn.master.host" -> "localhost", + "celeborn.master.port" -> "9097") + val workerConf = Map("celeborn.master.endpoints" -> "localhost:9097") + workers = setUpMiniCluster(masterConf, workerConf)._2 + + hadoopConf = new Configuration() + hadoopConf.set("yarn.scheduler.capacity.root.queues", "default,other_queue") + + hadoopConf.setInt("yarn.scheduler.capacity.root.default.capacity", 100) + hadoopConf.setInt("yarn.scheduler.capacity.root.default.maximum-capacity", 100) + hadoopConf.setInt("yarn.scheduler.capacity.root.other_queue.maximum-capacity", 100) + + hadoopConf.setStrings( + YarnConfiguration.NM_AUX_SERVICES, + ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID) + hadoopConf.setClass( + String.format( + YarnConfiguration.NM_AUX_SERVICE_FMT, + ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID), + classOf[ShuffleHandler], + classOf[Service]) + + yarnCluster = new MiniYARNCluster("MiniClusterWordCount", 1, 1, 1) + yarnCluster.init(hadoopConf) + yarnCluster.start() + } + + override def afterAll(): Unit = { + logInfo("all test complete , stop celeborn mini cluster") + shutdownMiniCluster() + if (yarnCluster != null) { + yarnCluster.stop() + } + } + + test("celeborn mr integration test - word count") { + val input = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "input") + Files.write( + Paths.get(input.getPath, "v1.txt"), + "hello world celeborn".getBytes(StandardCharsets.UTF_8)) + Files.write( + Paths.get(input.getPath, "v2.txt"), + "hello world mapreduce".getBytes(StandardCharsets.UTF_8)) + + val output = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "output") + val mrOutputPath = new Path(output.getPath + File.separator + "mr_output") + + val conf = new Configuration(yarnCluster.getConfig) + // YARN config + conf.set("yarn.app.mapreduce.am.job.recovery.enable", "false") + conf.set( + "yarn.app.mapreduce.am.command-opts", + "org.apache.celeborn.mapreduce.v2.app.MRAppMasterWithCeleborn") + + // MapReduce config + conf.set("mapreduce.framework.name", "yarn") + conf.set("mapreduce.job.user.classpath.first", "true") + + conf.set("mapreduce.job.reduce.slowstart.completedmaps", "1") + conf.set("mapreduce.celeborn.master.endpoints", "localhost:9097") + conf.set( + MRJobConfig.MAP_OUTPUT_COLLECTOR_CLASS_ATTR, + "org.apache.hadoop.mapred.CelebornMapOutputCollector") + conf.set( + "mapreduce.job.reduce.shuffle.consumer.plugin.class", + "org.apache.hadoop.mapreduce.task.reduce.CelebornShuffleConsumer") + + val job = Job.getInstance(conf, "word count") + job.setJarByClass(classOf[WordCount]) + job.setMapperClass(classOf[WordCount.TokenizerMapper]) + job.setCombinerClass(classOf[WordCount.IntSumReducer]) + job.setReducerClass(classOf[WordCount.IntSumReducer]) + job.setOutputKeyClass(classOf[Text]) + job.setOutputValueClass(classOf[IntWritable]) + FileInputFormat.addInputPath(job, new Path(input.getPath)) + FileOutputFormat.setOutputPath(job, mrOutputPath) + + val mapreduceLibPath = (Utils.getCodeSourceLocation(getClass).split("/").dropRight(1) ++ Array( + "mapreduce_lib")).mkString("/") + val excludeJarList = + Seq("hadoop-client-api", "hadoop-client-runtime", "hadoop-client-minicluster") + Files.list(Paths.get(mapreduceLibPath)).iterator().asScala.foreach(path => { + if (!excludeJarList.exists(path.toFile.getPath.contains(_))) { + job.addFileToClassPath(new Path(path.toString)) + } + }) + + val exitCode = job.waitForCompletion(true) + assert(exitCode, "Returned error code.") + + val outputFilePath = Paths.get(mrOutputPath.toString, "part-r-00000") + assert(outputFilePath.toFile.exists()) + assert(Files.readAllLines(outputFilePath).contains("celeborn\t1")) + } +} From 6f328382b344646edbb261f4c098af9c05dda4ab Mon Sep 17 00:00:00 2001 From: jiaoqingbo <1178404354@qq.com> Date: Thu, 23 Nov 2023 19:58:13 +0800 Subject: [PATCH 18/31] [CELEBORN-1138] Fix log error in createReaderWithRetry method As Title As Title NO PASS GA Closes #2111 from jiaoqingbo/1138. Authored-by: jiaoqingbo <1178404354@qq.com> Signed-off-by: Shuang --- .../org/apache/celeborn/client/read/CelebornInputStream.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java b/client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java index 29f63538071..8484406c6cb 100644 --- a/client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java +++ b/client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java @@ -329,13 +329,13 @@ private PartitionReader createReaderWithRetry(PartitionLocation location) throws if (fetchChunkRetryCnt % 2 == 0) { Uninterruptibles.sleepUninterruptibly(retryWaitMs, TimeUnit.MILLISECONDS); } - location = location.getPeer(); logger.warn( "CreatePartitionReader failed {}/{} times for location {}, change to peer", fetchChunkRetryCnt, fetchChunkMaxRetry, location, e); + location = location.getPeer(); } else { logger.warn( "CreatePartitionReader failed {}/{} times for location {}, retry the same location", From 788b0c340b6c5d31de76fc12983d60f71c3ef811 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Fri, 24 Nov 2023 09:57:04 +0800 Subject: [PATCH 19/31] [CELEBORN-1135] Added tests for the RpcEnv and related classes ### What changes were proposed in this pull request? Added test suites for `RpcEnv`, `NettyRpcEnv`, and other related classes. These are copied over from Apache Spark. Some of the UTs in Apache Spark required changes in the source code like [SPARK-39468](https://issues.apache.org/jira/browse/SPARK-39468) which I didn't copy over. ### Why are the changes needed? The change adds unit tests. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Just adds UTs. The source code changes are minimal. Closes #2107 from otterc/CELEBORN-1135. Authored-by: Chandni Singh Signed-off-by: Shuang --- .../celeborn/common/rpc/netty/Inbox.scala | 28 +- .../celeborn/common/rpc/RpcAddressSuite.scala | 57 ++ .../celeborn/common/rpc/RpcEnvSuite.scala | 855 ++++++++++++++++++ .../celeborn/common/rpc/TestRpcEndpoint.scala | 124 +++ .../common/rpc/netty/InboxSuite.scala | 158 ++++ .../rpc/netty/NettyRpcAddressSuite.scala | 34 + .../common/rpc/netty/NettyRpcEnvSuite.scala | 138 +++ .../rpc/netty/NettyRpcHandlerSuite.scala | 66 ++ 8 files changed, 1458 insertions(+), 2 deletions(-) create mode 100644 common/src/test/scala/org/apache/celeborn/common/rpc/RpcAddressSuite.scala create mode 100644 common/src/test/scala/org/apache/celeborn/common/rpc/RpcEnvSuite.scala create mode 100644 common/src/test/scala/org/apache/celeborn/common/rpc/TestRpcEndpoint.scala create mode 100644 common/src/test/scala/org/apache/celeborn/common/rpc/netty/InboxSuite.scala create mode 100644 common/src/test/scala/org/apache/celeborn/common/rpc/netty/NettyRpcAddressSuite.scala create mode 100644 common/src/test/scala/org/apache/celeborn/common/rpc/netty/NettyRpcEnvSuite.scala create mode 100644 common/src/test/scala/org/apache/celeborn/common/rpc/netty/NettyRpcHandlerSuite.scala diff --git a/common/src/main/scala/org/apache/celeborn/common/rpc/netty/Inbox.scala b/common/src/main/scala/org/apache/celeborn/common/rpc/netty/Inbox.scala index b24290fe7e6..09cdd08e256 100644 --- a/common/src/main/scala/org/apache/celeborn/common/rpc/netty/Inbox.scala +++ b/common/src/main/scala/org/apache/celeborn/common/rpc/netty/Inbox.scala @@ -106,7 +106,7 @@ private[celeborn] class Inbox( } } while (true) { - safelyCall(endpoint) { + safelyCall(endpoint, endpointRef.name) { message match { case RpcMessage(_sender, content, context) => try { @@ -218,7 +218,21 @@ private[celeborn] class Inbox( /** * Calls action closure, and calls the endpoint's onError function in the case of exceptions. */ - private def safelyCall(endpoint: RpcEndpoint)(action: => Unit): Unit = { + private def safelyCall( + endpoint: RpcEndpoint, + endpointRefName: String)(action: => Unit): Unit = { + def dealWithFatalError(fatal: Throwable): Unit = { + inbox.synchronized { + assert(numActiveThreads > 0, "The number of active threads should be positive.") + // Should reduce the number of active threads before throw the error. + numActiveThreads -= 1 + } + logError( + s"An error happened while processing message in the inbox for $endpointRefName", + fatal) + throw fatal + } + try action catch { case NonFatal(e) => @@ -230,8 +244,18 @@ private[celeborn] class Inbox( } else { logError("Ignoring error", ee) } + case fatal: Throwable => + dealWithFatalError(fatal) } + case fatal: Throwable => + dealWithFatalError(fatal) } } + // exposed only for testing + def getNumActiveThreads: Int = { + inbox.synchronized { + inbox.numActiveThreads + } + } } diff --git a/common/src/test/scala/org/apache/celeborn/common/rpc/RpcAddressSuite.scala b/common/src/test/scala/org/apache/celeborn/common/rpc/RpcAddressSuite.scala new file mode 100644 index 00000000000..9bc3220857f --- /dev/null +++ b/common/src/test/scala/org/apache/celeborn/common/rpc/RpcAddressSuite.scala @@ -0,0 +1,57 @@ +/* + * 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.rpc + +import org.apache.celeborn.CelebornFunSuite +import org.apache.celeborn.common.exception.CelebornException + +class RpcAddressSuite extends CelebornFunSuite { + + test("hostPort") { + val address = RpcAddress("1.2.3.4", 1234) + assert(address.host === "1.2.3.4") + assert(address.port === 1234) + assert(address.hostPort === "1.2.3.4:1234") + } + + test("fromCelebornURL") { + val address = RpcAddress.fromCelebornURL("celeborn://1.2.3.4:1234") + assert(address.host === "1.2.3.4") + assert(address.port === 1234) + } + + test("fromCelebornURL: a typo url") { + val e = intercept[CelebornException] { + RpcAddress.fromCelebornURL("celeborn://1.2. 3.4:1234") + } + assert("Invalid master URL: celeborn://1.2. 3.4:1234" === e.getMessage) + } + + test("fromCelebornURL: invalid scheme") { + val e = intercept[CelebornException] { + RpcAddress.fromCelebornURL("invalid://1.2.3.4:1234") + } + assert("Invalid master URL: invalid://1.2.3.4:1234" === e.getMessage) + } + + test("toCelebornURL") { + val address = RpcAddress("1.2.3.4", 1234) + assert(address.toCelebornURL === "celeborn://1.2.3.4:1234") + } + +} diff --git a/common/src/test/scala/org/apache/celeborn/common/rpc/RpcEnvSuite.scala b/common/src/test/scala/org/apache/celeborn/common/rpc/RpcEnvSuite.scala new file mode 100644 index 00000000000..4843cf744e3 --- /dev/null +++ b/common/src/test/scala/org/apache/celeborn/common/rpc/RpcEnvSuite.scala @@ -0,0 +1,855 @@ +/* + * 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.rpc + +import java.io.NotSerializableException +import java.util.concurrent.{ConcurrentLinkedQueue, CountDownLatch, TimeUnit} + +import scala.collection.JavaConverters.collectionAsScalaIterableConverter +import scala.collection.mutable +import scala.concurrent.Await +import scala.concurrent.duration._ + +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito.{mock, never, verify} +import org.scalatest.concurrent.Eventually._ + +import org.apache.celeborn.CelebornFunSuite +import org.apache.celeborn.common.CelebornConf +import org.apache.celeborn.common.exception.CelebornException +import org.apache.celeborn.common.util.ThreadUtils + +/** + * Common tests for an RpcEnv implementation. + */ +abstract class RpcEnvSuite extends CelebornFunSuite { + + var env: RpcEnv = _ + + def createCelebornConf(): CelebornConf = { + new CelebornConf() + } + + override def beforeAll(): Unit = { + super.beforeAll() + val conf = createCelebornConf() + env = createRpcEnv(conf, "local", 0) + + } + + override def afterAll(): Unit = { + try { + if (env != null) { + env.shutdown() + } + } finally { + super.afterAll() + } + } + + def createRpcEnv(conf: CelebornConf, name: String, port: Int, clientMode: Boolean = false): RpcEnv + + test("send a message locally") { + @volatile var message: String = null + val rpcEndpointRef = env.setupEndpoint( + "send-locally", + new RpcEndpoint { + override val rpcEnv = env + + override def receive = { + case msg: String => message = msg + } + }) + rpcEndpointRef.send("hello") + eventually(timeout(5.seconds), interval(10.milliseconds)) { + assert("hello" === message) + } + } + + test("send a message remotely") { + @volatile var message: String = null + // Set up a RpcEndpoint using env + env.setupEndpoint( + "send-remotely", + new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case msg: String => message = msg + } + }) + + val anotherEnv = createRpcEnv(createCelebornConf(), "remote", 0, clientMode = true) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "send-remotely") + try { + rpcEndpointRef.send("hello") + eventually(timeout(5.seconds), interval(10.milliseconds)) { + assert("hello" === message) + } + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("send a RpcEndpointRef") { + val endpoint = new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext) = { + case "Hello" => context.reply(self) + case "Echo" => context.reply("Echo") + } + } + val rpcEndpointRef = env.setupEndpoint("send-ref", endpoint) + val newRpcEndpointRef = rpcEndpointRef.askSync[RpcEndpointRef]("Hello") + val reply = newRpcEndpointRef.askSync[String]("Echo") + assert("Echo" === reply) + } + + test("ask a message locally") { + val rpcEndpointRef = env.setupEndpoint( + "ask-locally", + new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => + context.reply(msg) + } + }) + val reply = rpcEndpointRef.askSync[String]("hello") + assert("hello" === reply) + } + + test("ask a message remotely") { + env.setupEndpoint( + "ask-remotely", + new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => + context.reply(msg) + } + }) + + val anotherEnv = createRpcEnv(createCelebornConf(), "remote", 0, clientMode = true) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "ask-remotely") + try { + val reply = rpcEndpointRef.askSync[String]("hello") + assert("hello" === reply) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("ask a message timeout") { + env.setupEndpoint( + "ask-timeout", + new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => + Thread.sleep(100) + context.reply(msg) + } + }) + + val conf = createCelebornConf() + val shortProp = "celeborn.rpc.short.timeout" + val anotherEnv = createRpcEnv(conf, "remote", 0, clientMode = true) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "ask-timeout") + try { + val e = intercept[RpcTimeoutException] { + rpcEndpointRef.askSync[String]("hello", new RpcTimeout(1.millisecond, shortProp)) + } + // The Celeborn exception cause should be a RpcTimeoutException with message indicating the + // controlling timeout property + assert(e.isInstanceOf[RpcTimeoutException]) + assert(e.getMessage.contains(shortProp)) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("onStart and onStop") { + val stopLatch = new CountDownLatch(1) + val calledMethods = mutable.ArrayBuffer[String]() + + val endpoint = new RpcEndpoint { + override val rpcEnv = env + + override def onStart(): Unit = { + calledMethods += "start" + } + + override def receive: PartialFunction[Any, Unit] = { + case msg: String => + } + + override def onStop(): Unit = { + calledMethods += "stop" + stopLatch.countDown() + } + } + val rpcEndpointRef = env.setupEndpoint("start-stop-test", endpoint) + env.stop(rpcEndpointRef) + stopLatch.await(10, TimeUnit.SECONDS) + assert(List("start", "stop") === calledMethods) + } + + test("onError: error in onStart") { + @volatile var e: Throwable = null + env.setupEndpoint( + "onError-onStart", + new RpcEndpoint { + override val rpcEnv = env + + override def onStart(): Unit = { + throw new RuntimeException("Oops!") + } + + override def receive: PartialFunction[Any, Unit] = { + case m => + } + + override def onError(cause: Throwable): Unit = { + e = cause + } + }) + + eventually(timeout(5.seconds), interval(10.milliseconds)) { + assert(e.getMessage === "Oops!") + } + } + + test("onError: error in onStop") { + @volatile var e: Throwable = null + val endpointRef = env.setupEndpoint( + "onError-onStop", + new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => + } + + override def onError(cause: Throwable): Unit = { + e = cause + } + + override def onStop(): Unit = { + throw new RuntimeException("Oops!") + } + }) + + env.stop(endpointRef) + + eventually(timeout(5.seconds), interval(10.milliseconds)) { + assert(e.getMessage === "Oops!") + } + } + + test("onError: error in receive") { + @volatile var e: Throwable = null + val endpointRef = env.setupEndpoint( + "onError-receive", + new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => throw new RuntimeException("Oops!") + } + + override def onError(cause: Throwable): Unit = { + e = cause + } + }) + + endpointRef.send("Foo") + + eventually(timeout(5.seconds), interval(10.milliseconds)) { + assert(e.getMessage === "Oops!") + } + } + + test("self: call in onStart") { + @volatile var callSelfSuccessfully = false + + env.setupEndpoint( + "self-onStart", + new RpcEndpoint { + override val rpcEnv = env + + override def onStart(): Unit = { + self + callSelfSuccessfully = true + } + + override def receive: PartialFunction[Any, Unit] = { + case m => + } + }) + + eventually(timeout(5.seconds), interval(10.milliseconds)) { + // Calling `self` in `onStart` is fine + assert(callSelfSuccessfully) + } + } + + test("self: call in receive") { + @volatile var callSelfSuccessfully = false + + val endpointRef = env.setupEndpoint( + "self-receive", + new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => + self + callSelfSuccessfully = true + } + }) + + endpointRef.send("Foo") + + eventually(timeout(5.seconds), interval(10.milliseconds)) { + // Calling `self` in `receive` is fine + assert(callSelfSuccessfully) + } + } + + test("self: call in onStop") { + @volatile var selfOption: Option[RpcEndpointRef] = null + + val endpointRef = env.setupEndpoint( + "self-onStop", + new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => + } + + override def onStop(): Unit = { + selfOption = Option(self) + } + }) + + env.stop(endpointRef) + + eventually(timeout(5.seconds), interval(10.milliseconds)) { + // Calling `self` in `onStop` will return null, so selfOption will be None + assert(selfOption.isEmpty) + } + } + + test("call receive in sequence") { + // If a RpcEnv implementation breaks the `receive` contract, hope this test can expose it + for (i <- 0 until 100) { + @volatile var result = 0 + val endpointRef = env.setupEndpoint( + s"receive-in-sequence-$i", + new ThreadSafeRpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => result += 1 + } + + }) + + (0 until 10) foreach { _ => + new Thread { + override def run(): Unit = { + (0 until 100) foreach { _ => + endpointRef.send("Hello") + } + } + }.start() + } + + eventually(timeout(5.seconds), interval(5.milliseconds)) { + assert(result == 1000) + } + + env.stop(endpointRef) + } + } + + test("stop(RpcEndpointRef) reentrant") { + @volatile var onStopCount = 0 + val endpointRef = env.setupEndpoint( + "stop-reentrant", + new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => + } + + override def onStop(): Unit = { + onStopCount += 1 + } + }) + + env.stop(endpointRef) + env.stop(endpointRef) + + eventually(timeout(5.seconds), interval(5.milliseconds)) { + // Calling stop twice should only trigger onStop once. + assert(onStopCount == 1) + } + } + + test("sendWithReply") { + val endpointRef = env.setupEndpoint( + "sendWithReply", + new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case m => context.reply("ack") + } + }) + + val f = endpointRef.ask[String]("Hi") + val ack = ThreadUtils.awaitResult(f, 5.seconds) + assert("ack" === ack) + + env.stop(endpointRef) + } + + test("sendWithReply: remotely") { + env.setupEndpoint( + "sendWithReply-remotely", + new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case m => context.reply("ack") + } + }) + + val anotherEnv = createRpcEnv(createCelebornConf(), "remote", 0, clientMode = true) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "sendWithReply-remotely") + try { + val f = rpcEndpointRef.ask[String]("hello") + val ack = ThreadUtils.awaitResult(f, 5.seconds) + assert("ack" === ack) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("sendWithReply: error") { + val endpointRef = env.setupEndpoint( + "sendWithReply-error", + new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case m => context.sendFailure(new CelebornException("Oops")) + } + }) + + val f = endpointRef.ask[String]("Hi") + val e = intercept[CelebornException] { + ThreadUtils.awaitResult(f, 5.seconds) + } + assert("Oops" === e.getCause.getMessage) + + env.stop(endpointRef) + } + + test("sendWithReply: remotely error") { + env.setupEndpoint( + "sendWithReply-remotely-error", + new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => context.sendFailure(new CelebornException("Oops")) + } + }) + + val anotherEnv = createRpcEnv(createCelebornConf(), "remote", 0, clientMode = true) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "sendWithReply-remotely-error") + try { + val f = rpcEndpointRef.ask[String]("hello") + val e = intercept[CelebornException] { + ThreadUtils.awaitResult(f, 5.seconds) + } + assert("Oops" === e.getCause.getMessage) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + /** + * Setup an [[RpcEndpoint]] to collect all network events. + * + * @return the [[RpcEndpointRef]] and a `ConcurrentLinkedQueue` that contains network events. + */ + private def setupNetworkEndpoint( + _env: RpcEnv, + name: String): (RpcEndpointRef, ConcurrentLinkedQueue[(Any, Any)]) = { + val events = new ConcurrentLinkedQueue[(Any, Any)] + val ref = _env.setupEndpoint( + "network-events-non-client", + new ThreadSafeRpcEndpoint { + override val rpcEnv = _env + + override def receive: PartialFunction[Any, Unit] = { + case "hello" => + case m => events.add("receive" -> m) + } + + override def onConnected(remoteAddress: RpcAddress): Unit = { + events.add("onConnected" -> remoteAddress) + } + + override def onDisconnected(remoteAddress: RpcAddress): Unit = { + events.add("onDisconnected" -> remoteAddress) + } + + override def onNetworkError(cause: Throwable, remoteAddress: RpcAddress): Unit = { + events.add("onNetworkError" -> remoteAddress) + } + + }) + (ref, events) + } + + test("network events in sever RpcEnv when another RpcEnv is in server mode") { + val serverEnv1 = createRpcEnv(createCelebornConf(), "server1", 0, clientMode = false) + val serverEnv2 = createRpcEnv(createCelebornConf(), "server2", 0, clientMode = false) + val (_, events) = setupNetworkEndpoint(serverEnv1, "network-events") + val (serverRef2, _) = setupNetworkEndpoint(serverEnv2, "network-events") + try { + val serverRefInServer2 = serverEnv1.setupEndpointRef(serverRef2.address, serverRef2.name) + // Send a message to set up the connection + serverRefInServer2.send("hello") + + eventually(timeout(5.seconds), interval(5.milliseconds)) { + assert(events.contains(("onConnected", serverEnv2.address))) + } + + serverEnv2.shutdown() + serverEnv2.awaitTermination() + + eventually(timeout(5.seconds), interval(5.milliseconds)) { + assert(events.contains(("onConnected", serverEnv2.address))) + assert(events.contains(("onDisconnected", serverEnv2.address))) + } + } finally { + serverEnv1.shutdown() + serverEnv2.shutdown() + serverEnv1.awaitTermination() + serverEnv2.awaitTermination() + } + } + + test("network events in sever RpcEnv when another RpcEnv is in client mode") { + val serverEnv = createRpcEnv(createCelebornConf(), "server", 0, clientMode = false) + val (serverRef, events) = setupNetworkEndpoint(serverEnv, "network-events") + val clientEnv = createRpcEnv(createCelebornConf(), "client", 0, clientMode = true) + try { + val serverRefInClient = clientEnv.setupEndpointRef(serverRef.address, serverRef.name) + // Send a message to set up the connection + serverRefInClient.send("hello") + + eventually(timeout(5.seconds), interval(5.milliseconds)) { + // We don't know the exact client address but at least we can verify the message type + assert(events.asScala.map(_._1).exists(_ == "onConnected")) + } + + clientEnv.shutdown() + clientEnv.awaitTermination() + + eventually(timeout(5.seconds), interval(5.milliseconds)) { + // We don't know the exact client address but at least we can verify the message type + assert(events.asScala.map(_._1).exists(_ == "onConnected")) + assert(events.asScala.map(_._1).exists(_ == "onDisconnected")) + } + } finally { + clientEnv.shutdown() + serverEnv.shutdown() + clientEnv.awaitTermination() + serverEnv.awaitTermination() + } + } + + test("network events in client RpcEnv when another RpcEnv is in server mode") { + val clientEnv = createRpcEnv(createCelebornConf(), "client", 0, clientMode = true) + val serverEnv = createRpcEnv(createCelebornConf(), "server", 0, clientMode = false) + val (_, events) = setupNetworkEndpoint(clientEnv, "network-events") + val (serverRef, _) = setupNetworkEndpoint(serverEnv, "network-events") + try { + val serverRefInClient = clientEnv.setupEndpointRef(serverRef.address, serverRef.name) + // Send a message to set up the connection + serverRefInClient.send("hello") + + eventually(timeout(5.seconds), interval(5.milliseconds)) { + assert(events.contains(("onConnected", serverEnv.address))) + } + + serverEnv.shutdown() + serverEnv.awaitTermination() + + eventually(timeout(5.seconds), interval(5.milliseconds)) { + assert(events.contains(("onConnected", serverEnv.address))) + assert(events.contains(("onDisconnected", serverEnv.address))) + } + } finally { + clientEnv.shutdown() + serverEnv.shutdown() + clientEnv.awaitTermination() + serverEnv.awaitTermination() + } + } + + test("sendWithReply: unserializable error") { + env.setupEndpoint( + "sendWithReply-unserializable-error", + new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => context.sendFailure(new UnserializableException) + } + }) + + val anotherEnv = createRpcEnv(createCelebornConf(), "remote", 0, clientMode = true) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = + anotherEnv.setupEndpointRef(env.address, "sendWithReply-unserializable-error") + try { + val f = rpcEndpointRef.ask[String]("hello") + val e = intercept[CelebornException] { + ThreadUtils.awaitResult(f, 1.second) + } + assert(e.getCause.isInstanceOf[NotSerializableException]) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("port conflict") { + val anotherEnv = createRpcEnv(createCelebornConf(), "remote", env.address.port) + try { + assert(anotherEnv.address.port != env.address.port) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + private def testSend(conf: CelebornConf): Unit = { + val localEnv = createRpcEnv(conf, "authentication-local", 0) + val remoteEnv = createRpcEnv(conf, "authentication-remote", 0, clientMode = true) + + try { + @volatile var message: String = null + localEnv.setupEndpoint( + "send-authentication", + new RpcEndpoint { + override val rpcEnv = localEnv + + override def receive: PartialFunction[Any, Unit] = { + case msg: String => message = msg + } + }) + val rpcEndpointRef = remoteEnv.setupEndpointRef(localEnv.address, "send-authentication") + rpcEndpointRef.send("hello") + eventually(timeout(5.seconds), interval(10.milliseconds)) { + assert("hello" === message) + } + } finally { + localEnv.shutdown() + localEnv.awaitTermination() + remoteEnv.shutdown() + remoteEnv.awaitTermination() + } + } + + private def testAsk(conf: CelebornConf): Unit = { + val localEnv = createRpcEnv(conf, "authentication-local", 0) + val remoteEnv = createRpcEnv(conf, "authentication-remote", 0, clientMode = true) + + try { + localEnv.setupEndpoint( + "ask-authentication", + new RpcEndpoint { + override val rpcEnv = localEnv + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => + context.reply(msg) + } + }) + val rpcEndpointRef = remoteEnv.setupEndpointRef(localEnv.address, "ask-authentication") + val reply = rpcEndpointRef.askSync[String]("hello") + assert("hello" === reply) + } finally { + localEnv.shutdown() + localEnv.awaitTermination() + remoteEnv.shutdown() + remoteEnv.awaitTermination() + } + } + + test("construct RpcTimeout with conf property") { + val conf = new CelebornConf() + + val testProp = "celeborn.ask.test.timeout" + val testDurationSeconds = 30 + val secondaryProp = "celeborn.ask.secondary.timeout" + + conf.set(testProp, s"${testDurationSeconds}s") + conf.set(secondaryProp, "100s") + + // Construct RpcTimeout with a single property + val rt1 = RpcTimeout(conf, testProp) + assert(testDurationSeconds === rt1.duration.toSeconds) + + // Construct RpcTimeout with prioritized list of properties + val rt2 = RpcTimeout(conf, Seq("celeborn.ask.invalid.timeout", testProp, secondaryProp), "1s") + assert(testDurationSeconds === rt2.duration.toSeconds) + + // Construct RpcTimeout with default value, + val defaultProp = "celeborn.ask.default.timeout" + val defaultDurationSeconds = 1 + val rt3 = RpcTimeout(conf, Seq(defaultProp), defaultDurationSeconds.toString + "s") + assert(defaultDurationSeconds === rt3.duration.toSeconds) + assert(rt3.timeoutProp.contains(defaultProp)) + + // Try to construct RpcTimeout with an unconfigured property + intercept[NoSuchElementException] { + RpcTimeout(conf, "celeborn.ask.invalid.timeout") + } + } + + test("ask a message timeout on Future using RpcTimeout") { + case class NeverReply(msg: String) + + val rpcEndpointRef = env.setupEndpoint( + "ask-future", + new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => context.reply(msg) + case _: NeverReply => + } + }) + + val longTimeout = new RpcTimeout(1.second, "celeborn.rpc.long.timeout") + val shortTimeout = new RpcTimeout(10.milliseconds, "celeborn.rpc.short.timeout") + + // Ask with immediate response, should complete successfully + val fut1 = rpcEndpointRef.ask[String]("hello", longTimeout) + val reply1 = longTimeout.awaitResult(fut1) + assert("hello" === reply1) + + // Ask with a delayed response and wait for response immediately that should timeout + val fut2 = rpcEndpointRef.ask[String](NeverReply("doh"), shortTimeout) + val reply2 = + intercept[RpcTimeoutException] { + shortTimeout.awaitResult(fut2) + }.getMessage + + // RpcTimeout.awaitResult should have added the property to the TimeoutException message + assert(reply2.contains(shortTimeout.timeoutProp)) + + // Ask with delayed response and allow the Future to timeout before ThreadUtils.awaitResult + val fut3 = rpcEndpointRef.ask[String](NeverReply("goodbye"), shortTimeout) + + // scalastyle:off awaitresult + // Allow future to complete with failure using plain Await.result, this will return + // once the future is complete to verify addMessageIfTimeout was invoked + val reply3 = + intercept[RpcTimeoutException] { + Await.result(fut3, 2.seconds) + }.getMessage + // scalastyle:on awaitresult + + // When the future timed out, the recover callback should have used + // RpcTimeout.addMessageIfTimeout to add the property to the TimeoutException message + assert(reply3.contains(shortTimeout.timeoutProp)) + + // Use RpcTimeout.awaitResult to process Future, since it has already failed with + // RpcTimeoutException, the same RpcTimeoutException should be thrown + val reply4 = + intercept[RpcTimeoutException] { + shortTimeout.awaitResult(fut3) + }.getMessage + + // Ensure description is not in message twice after addMessageIfTimeout and awaitResult + assert(shortTimeout.timeoutProp.r.findAllIn(reply4).length === 1) + } + + test("RpcEnv.shutdown should not fire onDisconnected events") { + env.setupEndpoint( + "test_ep_11212023", + new RpcEndpoint { + override val rpcEnv: RpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case m => context.reply(m) + } + }) + + val anotherEnv = createRpcEnv(createCelebornConf(), "remote", 0) + val endpoint = mock(classOf[RpcEndpoint]) + anotherEnv.setupEndpoint("test_ep_11212023", endpoint) + + val ref = anotherEnv.setupEndpointRef(env.address, "test_ep_11212023") + // Make sure the connect is set up + assert(ref.askSync[String]("hello") === "hello") + anotherEnv.shutdown() + anotherEnv.awaitTermination() + + env.stop(ref) + + verify(endpoint).onStop() + verify(endpoint, never()).onDisconnected(any()) + verify(endpoint, never()).onNetworkError(any(), any()) + } +} + +case class Register(ref: RpcEndpointRef) + +class UnserializableClass + +class UnserializableException extends Exception { + private val unserializableField = new UnserializableClass +} diff --git a/common/src/test/scala/org/apache/celeborn/common/rpc/TestRpcEndpoint.scala b/common/src/test/scala/org/apache/celeborn/common/rpc/TestRpcEndpoint.scala new file mode 100644 index 00000000000..9d343b03738 --- /dev/null +++ b/common/src/test/scala/org/apache/celeborn/common/rpc/TestRpcEndpoint.scala @@ -0,0 +1,124 @@ +/* + * 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.rpc + +import scala.collection.mutable.ArrayBuffer + +import org.scalactic.TripleEquals +import org.scalatest.Assertions._ + +class TestRpcEndpoint extends ThreadSafeRpcEndpoint with TripleEquals { + + override val rpcEnv: RpcEnv = null + + @volatile private var receiveMessages = ArrayBuffer[Any]() + + @volatile private var receiveAndReplyMessages = ArrayBuffer[Any]() + + @volatile private var onConnectedMessages = ArrayBuffer[RpcAddress]() + + @volatile private var onDisconnectedMessages = ArrayBuffer[RpcAddress]() + + @volatile private var onNetworkErrorMessages = ArrayBuffer[(Throwable, RpcAddress)]() + + @volatile private var started = false + + @volatile private var stopped = false + + override def receive: PartialFunction[Any, Unit] = { + case message: Any => receiveMessages += message + } + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case message: Any => receiveAndReplyMessages += message + } + + override def onConnected(remoteAddress: RpcAddress): Unit = { + onConnectedMessages += remoteAddress + } + + /** + * Invoked when some network error happens in the connection between the current node and + * `remoteAddress`. + */ + override def onNetworkError(cause: Throwable, remoteAddress: RpcAddress): Unit = { + onNetworkErrorMessages += cause -> remoteAddress + } + + override def onDisconnected(remoteAddress: RpcAddress): Unit = { + onDisconnectedMessages += remoteAddress + } + + def numReceiveMessages: Int = receiveMessages.size + + override def onStart(): Unit = { + started = true + } + + override def onStop(): Unit = { + stopped = true + } + + def verifyStarted(): Unit = { + assert(started, "RpcEndpoint is not started") + } + + def verifyStopped(): Unit = { + assert(stopped, "RpcEndpoint is not stopped") + } + + def verifyReceiveMessages(expected: Seq[Any]): Unit = { + assert(receiveMessages === expected) + } + + def verifySingleReceiveMessage(message: Any): Unit = { + verifyReceiveMessages(List(message)) + } + + def verifyReceiveAndReplyMessages(expected: Seq[Any]): Unit = { + assert(receiveAndReplyMessages === expected) + } + + def verifySingleReceiveAndReplyMessage(message: Any): Unit = { + verifyReceiveAndReplyMessages(List(message)) + } + + def verifySingleOnConnectedMessage(remoteAddress: RpcAddress): Unit = { + verifyOnConnectedMessages(List(remoteAddress)) + } + + def verifyOnConnectedMessages(expected: Seq[RpcAddress]): Unit = { + assert(onConnectedMessages === expected) + } + + def verifySingleOnDisconnectedMessage(remoteAddress: RpcAddress): Unit = { + verifyOnDisconnectedMessages(List(remoteAddress)) + } + + def verifyOnDisconnectedMessages(expected: Seq[RpcAddress]): Unit = { + assert(onDisconnectedMessages === expected) + } + + def verifySingleOnNetworkErrorMessage(cause: Throwable, remoteAddress: RpcAddress): Unit = { + verifyOnNetworkErrorMessages(List(cause -> remoteAddress)) + } + + def verifyOnNetworkErrorMessages(expected: Seq[(Throwable, RpcAddress)]): Unit = { + assert(onNetworkErrorMessages === expected) + } +} diff --git a/common/src/test/scala/org/apache/celeborn/common/rpc/netty/InboxSuite.scala b/common/src/test/scala/org/apache/celeborn/common/rpc/netty/InboxSuite.scala new file mode 100644 index 00000000000..a8bc826dd4b --- /dev/null +++ b/common/src/test/scala/org/apache/celeborn/common/rpc/netty/InboxSuite.scala @@ -0,0 +1,158 @@ +/* + * 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.rpc.netty + +import java.util.concurrent.{CountDownLatch, TimeUnit} +import java.util.concurrent.atomic.AtomicInteger + +import org.mockito.Mockito._ + +import org.apache.celeborn.CelebornFunSuite +import org.apache.celeborn.common.rpc.{RpcAddress, TestRpcEndpoint} + +class InboxSuite extends CelebornFunSuite { + + test("post") { + val endpoint = new TestRpcEndpoint + val rpcEnvRef = mock(classOf[NettyRpcEndpointRef]) + val dispatcher = mock(classOf[Dispatcher]) + + val inbox = new Inbox(rpcEnvRef, endpoint) + val message = OneWayMessage(null, "hi") + inbox.post(message) + inbox.process(dispatcher) + assert(inbox.isEmpty) + + endpoint.verifySingleReceiveMessage("hi") + + inbox.stop() + inbox.process(dispatcher) + assert(inbox.isEmpty) + endpoint.verifyStarted() + endpoint.verifyStopped() + } + + test("post: with reply") { + val endpoint = new TestRpcEndpoint + val rpcEnvRef = mock(classOf[NettyRpcEndpointRef]) + val dispatcher = mock(classOf[Dispatcher]) + + val inbox = new Inbox(rpcEnvRef, endpoint) + val message = RpcMessage(null, "hi", null) + inbox.post(message) + inbox.process(dispatcher) + assert(inbox.isEmpty) + + endpoint.verifySingleReceiveAndReplyMessage("hi") + } + + test("post: multiple threads") { + val endpoint = new TestRpcEndpoint + val rpcEnvRef = mock(classOf[NettyRpcEndpointRef]) + val dispatcher = mock(classOf[Dispatcher]) + + val numDroppedMessages = new AtomicInteger(0) + val inbox = new Inbox(rpcEnvRef, endpoint) { + override def onDrop(message: InboxMessage): Unit = { + numDroppedMessages.incrementAndGet() + } + } + + val exitLatch = new CountDownLatch(10) + + for (_ <- 0 until 10) { + new Thread { + override def run(): Unit = { + for (_ <- 0 until 100) { + val message = OneWayMessage(null, "hi") + inbox.post(message) + } + exitLatch.countDown() + } + }.start() + } + // Try to process some messages + inbox.process(dispatcher) + inbox.stop() + // After `stop` is called, further messages will be dropped. However, while `stop` is called, + // some messages may be post to Inbox, so process them here. + inbox.process(dispatcher) + assert(inbox.isEmpty) + + exitLatch.await(30, TimeUnit.SECONDS) + + assert(1000 === endpoint.numReceiveMessages + numDroppedMessages.get) + endpoint.verifyStarted() + endpoint.verifyStopped() + } + + test("post: Associated") { + val endpoint = new TestRpcEndpoint + val rpcEnvRef = mock(classOf[NettyRpcEndpointRef]) + val dispatcher = mock(classOf[Dispatcher]) + val remoteAddress = RpcAddress("localhost", 11111) + + val inbox = new Inbox(rpcEnvRef, endpoint) + inbox.post(RemoteProcessConnected(remoteAddress)) + inbox.process(dispatcher) + + endpoint.verifySingleOnConnectedMessage(remoteAddress) + } + + test("post: Disassociated") { + val endpoint = new TestRpcEndpoint + val rpcEnvRef = mock(classOf[NettyRpcEndpointRef]) + val dispatcher = mock(classOf[Dispatcher]) + + val remoteAddress = RpcAddress("localhost", 11111) + + val inbox = new Inbox(rpcEnvRef, endpoint) + inbox.post(RemoteProcessDisconnected(remoteAddress)) + inbox.process(dispatcher) + + endpoint.verifySingleOnDisconnectedMessage(remoteAddress) + } + + test("post: AssociationError") { + val endpoint = new TestRpcEndpoint + val rpcEnvRef = mock(classOf[NettyRpcEndpointRef]) + val dispatcher = mock(classOf[Dispatcher]) + + val remoteAddress = RpcAddress("localhost", 11111) + val cause = new RuntimeException("Oops") + + val inbox = new Inbox(rpcEnvRef, endpoint) + inbox.post(RemoteProcessConnectionError(cause, remoteAddress)) + inbox.process(dispatcher) + + endpoint.verifySingleOnNetworkErrorMessage(cause, remoteAddress) + } + + test("should reduce the number of active threads when fatal error happens") { + val endpoint = mock(classOf[TestRpcEndpoint]) + when(endpoint.receive).thenThrow(new OutOfMemoryError()) + val rpcEnvRef = mock(classOf[NettyRpcEndpointRef]) + val dispatcher = mock(classOf[Dispatcher]) + val inbox = new Inbox(rpcEnvRef, endpoint) + inbox.post(OneWayMessage(null, "hi")) + intercept[OutOfMemoryError] { + inbox.process(dispatcher) + } + assert(inbox.getNumActiveThreads === 0) + } +} diff --git a/common/src/test/scala/org/apache/celeborn/common/rpc/netty/NettyRpcAddressSuite.scala b/common/src/test/scala/org/apache/celeborn/common/rpc/netty/NettyRpcAddressSuite.scala new file mode 100644 index 00000000000..1c3fe6ef4f2 --- /dev/null +++ b/common/src/test/scala/org/apache/celeborn/common/rpc/netty/NettyRpcAddressSuite.scala @@ -0,0 +1,34 @@ +/* + * 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.rpc.netty + +import org.apache.celeborn.CelebornFunSuite +import org.apache.celeborn.common.rpc.RpcEndpointAddress + +class NettyRpcAddressSuite extends CelebornFunSuite { + + test("toString") { + val addr = new RpcEndpointAddress("localhost", 12345, "test") + assert(addr.toString === "celeborn://test@localhost:12345") + } + + test("toString for client mode") { + val addr = RpcEndpointAddress(null, "test") + assert(addr.toString === "celeborn-client://test") + } +} diff --git a/common/src/test/scala/org/apache/celeborn/common/rpc/netty/NettyRpcEnvSuite.scala b/common/src/test/scala/org/apache/celeborn/common/rpc/netty/NettyRpcEnvSuite.scala new file mode 100644 index 00000000000..8afbf598050 --- /dev/null +++ b/common/src/test/scala/org/apache/celeborn/common/rpc/netty/NettyRpcEnvSuite.scala @@ -0,0 +1,138 @@ +/* + * 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.rpc.netty + +import java.util.concurrent.ExecutionException + +import scala.concurrent.duration._ + +import org.mockito.Mockito.mock +import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} + +import org.apache.celeborn.common.CelebornConf +import org.apache.celeborn.common.exception.CelebornException +import org.apache.celeborn.common.network.client.TransportClient +import org.apache.celeborn.common.rpc._ +import org.apache.celeborn.common.util.ThreadUtils + +class NettyRpcEnvSuite extends RpcEnvSuite with TimeLimits { + + implicit private val signaler: Signaler = ThreadSignaler + + override def createRpcEnv( + conf: CelebornConf, + name: String, + port: Int, + clientMode: Boolean = false): RpcEnv = { + val config = RpcEnvConfig(conf, "test", "localhost", "localhost", port, 0) + new NettyRpcEnvFactory().create(config) + } + + test("non-existent endpoint") { + val uri = RpcEndpointAddress(env.address, "nonexist-endpoint").toString + val e = intercept[CelebornException] { + env.setupEndpointRef(env.address, "nonexist-endpoint") + } + assert(e.getCause.isInstanceOf[RpcEndpointNotFoundException]) + assert(e.getCause.getMessage.contains(uri)) + } + + test("advertise address different from bind address") { + val celebornConf = createCelebornConf() + val config = RpcEnvConfig(celebornConf, "test", "localhost", "example.com", 0, 0) + val env = new NettyRpcEnvFactory().create(config) + try { + assert(env.address.hostPort.startsWith("example.com:")) + } finally { + env.shutdown() + } + } + + test("RequestMessage serialization") { + def assertRequestMessageEquals(expected: RequestMessage, actual: RequestMessage): Unit = { + assert(expected.senderAddress === actual.senderAddress) + assert(expected.receiver === actual.receiver) + assert(expected.content === actual.content) + } + + val nettyEnv = env.asInstanceOf[NettyRpcEnv] + val client = mock(classOf[TransportClient]) + val senderAddress = RpcAddress("localhost", 12345) + val receiverAddress = RpcEndpointAddress("localhost", 54321, "test") + val receiver = new NettyRpcEndpointRef(nettyEnv.celebornConf, receiverAddress, nettyEnv) + + val msg = new RequestMessage(senderAddress, receiver, "foo") + assertRequestMessageEquals( + msg, + RequestMessage(nettyEnv, client, msg.serialize(nettyEnv))) + + val msg2 = new RequestMessage(null, receiver, "foo") + assertRequestMessageEquals( + msg2, + RequestMessage(nettyEnv, client, msg2.serialize(nettyEnv))) + + val msg3 = new RequestMessage(senderAddress, receiver, null) + assertRequestMessageEquals( + msg3, + RequestMessage(nettyEnv, client, msg3.serialize(nettyEnv))) + } + + test("StackOverflowError should be sent back and Dispatcher should survive") { + val numUsableCores = 2 + val conf = createCelebornConf() + val config = RpcEnvConfig( + conf, + "test", + "localhost", + "localhost", + 0, + numUsableCores) + val anotherEnv = new NettyRpcEnvFactory().create(config) + anotherEnv.setupEndpoint( + "StackOverflowError", + new RpcEndpoint { + override val rpcEnv = anotherEnv + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + // scalastyle:off throwerror + case msg: String => throw new StackOverflowError + // scalastyle:on throwerror + case num: Int => context.reply(num) + } + }) + + val rpcEndpointRef = env.setupEndpointRef(anotherEnv.address, "StackOverflowError") + try { + // Send `numUsableCores` messages to trigger `numUsableCores` `StackOverflowError`s + for (_ <- 0 until numUsableCores) { + val e = intercept[CelebornException] { + rpcEndpointRef.askSync[String]("hello") + } + // The root cause `e.getCause.getCause` because it is boxed by Scala Promise. + assert(e.getCause.isInstanceOf[ExecutionException]) + assert(e.getCause.getCause.isInstanceOf[StackOverflowError]) + } + failAfter(10.seconds) { + assert(rpcEndpointRef.askSync[Int](100) === 100) + } + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } +} diff --git a/common/src/test/scala/org/apache/celeborn/common/rpc/netty/NettyRpcHandlerSuite.scala b/common/src/test/scala/org/apache/celeborn/common/rpc/netty/NettyRpcHandlerSuite.scala new file mode 100644 index 00000000000..3f6ebb0e6c5 --- /dev/null +++ b/common/src/test/scala/org/apache/celeborn/common/rpc/netty/NettyRpcHandlerSuite.scala @@ -0,0 +1,66 @@ +/* + * 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.rpc.netty + +import java.net.InetSocketAddress +import java.nio.ByteBuffer + +import io.netty.channel.Channel +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito._ + +import org.apache.celeborn.CelebornFunSuite +import org.apache.celeborn.common.network.client.{TransportClient, TransportResponseHandler} +import org.apache.celeborn.common.rpc.RpcAddress + +class NettyRpcHandlerSuite extends CelebornFunSuite { + + val env = mock(classOf[NettyRpcEnv]) + when(env.deserialize(any(classOf[TransportClient]), any(classOf[ByteBuffer]))(any())) + .thenReturn(new RequestMessage(RpcAddress("localhost", 12345), null, null)) + + test("receive") { + val dispatcher = mock(classOf[Dispatcher]) + val nettyRpcHandler = new NettyRpcHandler(dispatcher, env) + + val channel = mock(classOf[Channel]) + val client = new TransportClient(channel, mock(classOf[TransportResponseHandler])) + when(channel.remoteAddress()).thenReturn(new InetSocketAddress("localhost", 40000)) + nettyRpcHandler.channelActive(client) + + verify(dispatcher, times(1)).postToAll(RemoteProcessConnected(RpcAddress("localhost", 40000))) + } + + test("connectionTerminated") { + val dispatcher = mock(classOf[Dispatcher]) + val nettyRpcHandler = new NettyRpcHandler(dispatcher, env) + + val channel = mock(classOf[Channel]) + val client = new TransportClient(channel, mock(classOf[TransportResponseHandler])) + when(channel.remoteAddress()).thenReturn(new InetSocketAddress("localhost", 40000)) + nettyRpcHandler.channelActive(client) + + when(channel.remoteAddress()).thenReturn(new InetSocketAddress("localhost", 40000)) + nettyRpcHandler.channelInactive(client) + + verify(dispatcher, times(1)).postToAll(RemoteProcessConnected(RpcAddress("localhost", 40000))) + verify(dispatcher, times(1)).postToAll( + RemoteProcessDisconnected(RpcAddress("localhost", 40000))) + } + +} From 820c17ad7da5e2ed861b93b2552373faedebcf92 Mon Sep 17 00:00:00 2001 From: jiaoqingbo <1178404354@qq.com> Date: Fri, 24 Nov 2023 17:55:32 +0800 Subject: [PATCH 20/31] [CELEBORN-1140] Use try-with-resources to avoid FSDataInputStream not being closed ### What changes were proposed in this pull request? As Title ### Why are the changes needed? As Title ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? PASS GA Closes #2113 from jiaoqingbo/1140. Authored-by: jiaoqingbo <1178404354@qq.com> Signed-off-by: mingji --- .../client/read/DfsPartitionReader.java | 45 ++++++++++--------- 1 file changed, 24 insertions(+), 21 deletions(-) diff --git a/client/src/main/java/org/apache/celeborn/client/read/DfsPartitionReader.java b/client/src/main/java/org/apache/celeborn/client/read/DfsPartitionReader.java index 7acc4fc8c26..d379a1139a4 100644 --- a/client/src/main/java/org/apache/celeborn/client/read/DfsPartitionReader.java +++ b/client/src/main/java/org/apache/celeborn/client/read/DfsPartitionReader.java @@ -178,15 +178,16 @@ public DfsPartitionReader( private List getChunkOffsetsFromUnsortedIndex(CelebornConf conf, PartitionLocation location) throws IOException { - FSDataInputStream indexInputStream = + List offsets; + try (FSDataInputStream indexInputStream = ShuffleClient.getHdfsFs(conf) - .open(new Path(Utils.getIndexFilePath(location.getStorageInfo().getFilePath()))); - List offsets = new ArrayList<>(); - int offsetCount = indexInputStream.readInt(); - for (int i = 0; i < offsetCount; i++) { - offsets.add(indexInputStream.readLong()); + .open(new Path(Utils.getIndexFilePath(location.getStorageInfo().getFilePath())))) { + offsets = new ArrayList<>(); + int offsetCount = indexInputStream.readInt(); + for (int i = 0; i < offsetCount; i++) { + offsets.add(indexInputStream.readLong()); + } } - indexInputStream.close(); return offsets; } @@ -194,20 +195,22 @@ private List getChunkOffsetsFromSortedIndex( CelebornConf conf, PartitionLocation location, int startMapIndex, int endMapIndex) throws IOException { String indexPath = Utils.getIndexFilePath(location.getStorageInfo().getFilePath()); - FSDataInputStream indexInputStream = ShuffleClient.getHdfsFs(conf).open(new Path(indexPath)); - logger.debug("read sorted index {}", indexPath); - long indexSize = ShuffleClient.getHdfsFs(conf).getFileStatus(new Path(indexPath)).getLen(); - // Index size won't be large, so it's safe to do the conversion. - byte[] indexBuffer = new byte[(int) indexSize]; - indexInputStream.readFully(0L, indexBuffer); - List offsets = - new ArrayList<>( - ShuffleBlockInfoUtils.getChunkOffsetsFromShuffleBlockInfos( - startMapIndex, - endMapIndex, - shuffleChunkSize, - ShuffleBlockInfoUtils.parseShuffleBlockInfosFromByteBuffer(indexBuffer))); - indexInputStream.close(); + List offsets; + try (FSDataInputStream indexInputStream = + ShuffleClient.getHdfsFs(conf).open(new Path(indexPath))) { + logger.debug("read sorted index {}", indexPath); + long indexSize = ShuffleClient.getHdfsFs(conf).getFileStatus(new Path(indexPath)).getLen(); + // Index size won't be large, so it's safe to do the conversion. + byte[] indexBuffer = new byte[(int) indexSize]; + indexInputStream.readFully(0L, indexBuffer); + offsets = + new ArrayList<>( + ShuffleBlockInfoUtils.getChunkOffsetsFromShuffleBlockInfos( + startMapIndex, + endMapIndex, + shuffleChunkSize, + ShuffleBlockInfoUtils.parseShuffleBlockInfosFromByteBuffer(indexBuffer))); + } return offsets; } From aee41555c658a35b2e3f837d741d2bfd94573f2e Mon Sep 17 00:00:00 2001 From: "Erik.fang" Date: Sun, 26 Nov 2023 16:47:58 +0800 Subject: [PATCH 21/31] [CELEBORN-955] Re-run Spark Stage for Celeborn Shuffle Fetch Failure ### What changes were proposed in this pull request? Currently, Celeborn uses replication to handle shuffle data lost for celeborn shuffle reader, this PR implements an alternative solution by Spark stage resubmission. Design doc: https://docs.google.com/document/d/1dkG6fww3g99VAb1wkphNlUES_MpngVPNg8601chmVp8/edit ### Why are the changes needed? Spark stage resubmission uses less resources compared with replication, and some Celeborn users are also asking for it ### Does this PR introduce _any_ user-facing change? a new config celeborn.client.fetch.throwsFetchFailure is introduced to enable this feature ### How was this patch tested? two UTs are attached, and we also tested it in Ant Group's Dev spark cluster Closes #1924 from ErikFang/Re-run-Spark-Stage-for-Celeborn-Shuffle-Fetch-Failure. Lead-authored-by: Erik.fang Co-authored-by: Cheng Pan Signed-off-by: zky.zhoukeyong --- .../celeborn/ExecutorShuffleIdTracker.java | 47 ++++ .../celeborn/HashBasedShuffleWriter.java | 3 +- .../celeborn/SortBasedShuffleWriter.java | 3 +- .../shuffle/celeborn/SparkShuffleManager.java | 38 ++- .../spark/shuffle/celeborn/SparkUtils.java | 21 ++ .../celeborn/CelebornShuffleHandle.scala | 1 + .../celeborn/CelebornShuffleReader.scala | 56 ++++- .../CelebornShuffleWriterSuiteBase.java | 2 +- .../HashBasedShuffleWriterSuiteJ.java | 8 +- .../shuffle/celeborn/ShuffleManagerHook.java | 35 +++ .../SortBasedShuffleWriterSuiteJ.java | 1 + .../celeborn/TestCelebornShuffleManager.java | 45 ++++ .../ColumnarHashBasedShuffleWriter.java | 3 +- .../CelebornColumnarShuffleReader.scala | 6 +- .../ColumnarHashBasedShuffleWriterSuiteJ.java | 20 +- .../CelebornColumnarShuffleReaderSuite.scala | 8 +- .../celeborn/HashBasedShuffleWriter.java | 3 +- .../celeborn/SortBasedShuffleWriter.java | 4 +- .../shuffle/celeborn/SparkShuffleManager.java | 50 +++- .../spark/shuffle/celeborn/SparkUtils.java | 61 ++++- .../celeborn/CelebornShuffleHandle.scala | 1 + .../celeborn/CelebornShuffleReader.scala | 58 ++++- .../CelebornShuffleWriterSuiteBase.java | 2 +- .../HashBasedShuffleWriterSuiteJ.java | 8 +- .../shuffle/celeborn/ShuffleManagerHook.java | 35 +++ .../celeborn/TestCelebornShuffleManager.java | 67 ++++++ .../apache/celeborn/client/ShuffleClient.java | 9 + .../celeborn/client/ShuffleClientImpl.java | 38 +++ .../celeborn/client/LifecycleManager.scala | 154 ++++++++++++ .../celeborn/client/DummyShuffleClient.java | 10 + .../network/protocol/TransportMessage.java | 12 + common/src/main/proto/TransportMessages.proto | 23 ++ .../apache/celeborn/common/CelebornConf.scala | 9 + .../protocol/message/ControlMessages.scala | 24 ++ docs/configuration/client.md | 1 + tests/spark-it/pom.xml | 49 ++++ .../spark/CelebornFetchFailureSuite.scala | 223 ++++++++++++++++++ .../service/deploy/MiniClusterFeature.scala | 12 +- 38 files changed, 1092 insertions(+), 58 deletions(-) create mode 100644 client-spark/common/src/main/java/org/apache/spark/shuffle/celeborn/ExecutorShuffleIdTracker.java create mode 100644 client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/ShuffleManagerHook.java create mode 100644 client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/TestCelebornShuffleManager.java create mode 100644 client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/ShuffleManagerHook.java create mode 100644 client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/TestCelebornShuffleManager.java create mode 100644 tests/spark-it/src/test/scala/org/apache/celeborn/tests/spark/CelebornFetchFailureSuite.scala diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/celeborn/ExecutorShuffleIdTracker.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/celeborn/ExecutorShuffleIdTracker.java new file mode 100644 index 00000000000..254eb516d73 --- /dev/null +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/celeborn/ExecutorShuffleIdTracker.java @@ -0,0 +1,47 @@ +/* + * 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.HashSet; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.celeborn.client.ShuffleClient; +import org.apache.celeborn.common.util.JavaUtils; + +public class ExecutorShuffleIdTracker { + // track appShuffleId -> shuffleId Set in executor for cleanup + private ConcurrentHashMap> shuffleIdMap = + JavaUtils.newConcurrentHashMap(); + + public void track(int appShuffleId, int shuffleId) { + HashSet shuffleIds = shuffleIdMap.computeIfAbsent(appShuffleId, id -> new HashSet<>()); + + synchronized (shuffleIds) { + shuffleIds.add(shuffleId); + } + } + + public void unregisterAppShuffleId(ShuffleClient shuffleClient, int appShuffleId) { + HashSet shuffleIds = shuffleIdMap.remove(appShuffleId); + if (shuffleIds != null) { + synchronized (shuffleIds) { + shuffleIds.forEach(shuffleClient::cleanupShuffle); + } + } + } +} diff --git a/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/HashBasedShuffleWriter.java b/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/HashBasedShuffleWriter.java index 89826be0537..407b3284921 100644 --- a/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/HashBasedShuffleWriter.java +++ b/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/HashBasedShuffleWriter.java @@ -101,6 +101,7 @@ public class HashBasedShuffleWriter extends ShuffleWriter { // In order to facilitate the writing of unit test code, ShuffleClient needs to be passed in as // parameters. By the way, simplify the passed parameters. public HashBasedShuffleWriter( + int shuffleId, CelebornShuffleHandle handle, int mapId, TaskContext taskContext, @@ -110,7 +111,7 @@ public HashBasedShuffleWriter( throws IOException { this.mapId = mapId; this.dep = handle.dependency(); - this.shuffleId = dep.shuffleId(); + this.shuffleId = shuffleId; SerializerInstance serializer = dep.serializer().newInstance(); this.partitioner = dep.partitioner(); this.writeMetrics = taskContext.taskMetrics().shuffleWriteMetrics(); diff --git a/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriter.java b/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriter.java index 62c05858d36..51087c6dd89 100644 --- a/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriter.java +++ b/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriter.java @@ -94,6 +94,7 @@ public class SortBasedShuffleWriter extends ShuffleWriter { // In order to facilitate the writing of unit test code, ShuffleClient needs to be passed in as // parameters. By the way, simplify the passed parameters. public SortBasedShuffleWriter( + int shuffleId, ShuffleDependency dep, int numMappers, TaskContext taskContext, @@ -104,7 +105,7 @@ public SortBasedShuffleWriter( throws IOException { this.mapId = taskContext.partitionId(); this.dep = dep; - this.shuffleId = dep.shuffleId(); + this.shuffleId = shuffleId; SerializerInstance serializer = dep.serializer().newInstance(); this.partitioner = dep.partitioner(); this.writeMetrics = taskContext.taskMetrics().shuffleWriteMetrics(); diff --git a/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/SparkShuffleManager.java b/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/SparkShuffleManager.java index 071810ad29d..470d2e989ee 100644 --- a/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/SparkShuffleManager.java +++ b/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/SparkShuffleManager.java @@ -26,6 +26,7 @@ import org.apache.spark.*; import org.apache.spark.launcher.SparkLauncher; +import org.apache.spark.rdd.DeterministicLevel; import org.apache.spark.shuffle.*; import org.apache.spark.shuffle.sort.SortShuffleManager; import org.apache.spark.util.Utils; @@ -66,6 +67,8 @@ public class SparkShuffleManager implements ShuffleManager { private long sendBufferPoolCheckInterval; private long sendBufferPoolExpireTimeout; + private ExecutorShuffleIdTracker shuffleIdTracker = new ExecutorShuffleIdTracker(); + public SparkShuffleManager(SparkConf conf, boolean isDriver) { this.conf = conf; this.isDriver = isDriver; @@ -105,6 +108,12 @@ private void initializeLifecycleManager(String appId) { synchronized (this) { if (lifecycleManager == null) { lifecycleManager = new LifecycleManager(appId, celebornConf); + if (celebornConf.clientFetchThrowsFetchFailure()) { + MapOutputTrackerMaster mapOutputTracker = + (MapOutputTrackerMaster) SparkEnv.get().mapOutputTracker(); + lifecycleManager.registerShuffleTrackerCallback( + shuffleId -> mapOutputTracker.unregisterAllMapOutput(shuffleId)); + } } } } @@ -119,6 +128,10 @@ public ShuffleHandle registerShuffle( appUniqueId = SparkUtils.appUniqueId(dependency.rdd().context()); initializeLifecycleManager(appUniqueId); + lifecycleManager.registerAppShuffleDeterminate( + shuffleId, + dependency.rdd().getOutputDeterministicLevel() != DeterministicLevel.INDETERMINATE()); + if (fallbackPolicyRunner.applyAllFallbackPolicy( lifecycleManager, dependency.partitioner().numPartitions())) { logger.warn("Fallback to SortShuffleManager!"); @@ -131,23 +144,24 @@ public ShuffleHandle registerShuffle( lifecycleManager.getPort(), lifecycleManager.getUserIdentifier(), shuffleId, + celebornConf.clientFetchThrowsFetchFailure(), numMaps, dependency); } } @Override - public boolean unregisterShuffle(int shuffleId) { - if (sortShuffleIds.contains(shuffleId)) { - return sortShuffleManager().unregisterShuffle(shuffleId); + public boolean unregisterShuffle(int appShuffleId) { + if (sortShuffleIds.contains(appShuffleId)) { + return sortShuffleManager().unregisterShuffle(appShuffleId); } // For Spark driver side trigger unregister shuffle. if (lifecycleManager != null) { - lifecycleManager.unregisterShuffle(shuffleId); + lifecycleManager.unregisterAppShuffle(appShuffleId); } // For Spark executor side cleanup shuffle related info. if (shuffleClient != null) { - shuffleClient.cleanupShuffle(shuffleId); + shuffleIdTracker.unregisterAppShuffleId(shuffleClient, appShuffleId); } return true; } @@ -187,10 +201,14 @@ public ShuffleWriter getWriter( h.lifecycleManagerPort(), celebornConf, h.userIdentifier()); + int shuffleId = SparkUtils.celebornShuffleId(client, h, context, true); + shuffleIdTracker.track(h.shuffleId(), shuffleId); + if (ShuffleMode.SORT.equals(celebornConf.shuffleWriterMode())) { ExecutorService pushThread = celebornConf.clientPushSortPipelineEnabled() ? getPusherThread() : null; return new SortBasedShuffleWriter<>( + shuffleId, h.dependency(), h.numMaps(), context, @@ -200,6 +218,7 @@ public ShuffleWriter getWriter( SendBufferPool.get(cores, sendBufferPoolCheckInterval, sendBufferPoolExpireTimeout)); } else if (ShuffleMode.HASH.equals(celebornConf.shuffleWriterMode())) { return new HashBasedShuffleWriter<>( + shuffleId, h, mapId, context, @@ -225,7 +244,14 @@ public ShuffleReader getReader( @SuppressWarnings("unchecked") CelebornShuffleHandle h = (CelebornShuffleHandle) handle; return new CelebornShuffleReader<>( - h, startPartition, endPartition, 0, Int.MaxValue(), context, celebornConf); + h, + startPartition, + endPartition, + 0, + Int.MaxValue(), + context, + celebornConf, + shuffleIdTracker); } return _sortShuffleManager.getReader(handle, startPartition, endPartition, context); } diff --git a/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/SparkUtils.java b/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/SparkUtils.java index b4446f0da06..0f03b5688f3 100644 --- a/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/SparkUtils.java +++ b/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/SparkUtils.java @@ -26,6 +26,7 @@ import org.apache.spark.SparkConf; import org.apache.spark.SparkContext; +import org.apache.spark.TaskContext; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; import org.apache.spark.sql.execution.UnsafeRowSerializer; @@ -34,12 +35,15 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.celeborn.client.ShuffleClient; import org.apache.celeborn.common.CelebornConf; import org.apache.celeborn.common.util.Utils; public class SparkUtils { private static final Logger logger = LoggerFactory.getLogger(SparkUtils.class); + public static final String FETCH_FAILURE_ERROR_MSG = "Celeborn FetchFailure with shuffle id "; + public static MapStatus createMapStatus( BlockManagerId loc, long[] uncompressedSizes, long[] uncompressedRecords) throws IOException { @@ -115,6 +119,23 @@ public static String appUniqueId(SparkContext context) { } } + public static String getAppShuffleIdentifier(int appShuffleId, TaskContext context) { + return appShuffleId + "-" + context.stageId() + "-" + context.stageAttemptNumber(); + } + + public static int celebornShuffleId( + ShuffleClient client, + CelebornShuffleHandle handle, + TaskContext context, + Boolean isWriter) { + if (handle.throwsFetchFailure()) { + String appShuffleIdentifier = getAppShuffleIdentifier(handle.shuffleId(), context); + return client.getShuffleId(handle.shuffleId(), appShuffleIdentifier, isWriter); + } else { + return handle.shuffleId(); + } + } + // Create an instance of the class with the given name, possibly initializing it with our conf // Copied from SparkEnv public static T instantiateClass(String className, SparkConf conf, Boolean isDriver) { diff --git a/client-spark/spark-2/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleHandle.scala b/client-spark/spark-2/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleHandle.scala index ef1f2c881ca..4f67edaf325 100644 --- a/client-spark/spark-2/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleHandle.scala +++ b/client-spark/spark-2/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleHandle.scala @@ -28,6 +28,7 @@ class CelebornShuffleHandle[K, V, C]( val lifecycleManagerPort: Int, val userIdentifier: UserIdentifier, shuffleId: Int, + val throwsFetchFailure: Boolean, numMappers: Int, dependency: ShuffleDependency[K, V, C]) extends BaseShuffleHandle(shuffleId, numMappers, dependency) diff --git a/client-spark/spark-2/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleReader.scala b/client-spark/spark-2/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleReader.scala index 1059f3604a3..dec30522562 100644 --- a/client-spark/spark-2/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleReader.scala +++ b/client-spark/spark-2/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleReader.scala @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicReference import org.apache.spark.{InterruptibleIterator, TaskContext} import org.apache.spark.internal.Logging -import org.apache.spark.shuffle.ShuffleReader +import org.apache.spark.shuffle.{FetchFailedException, ShuffleReader} import org.apache.spark.shuffle.celeborn.CelebornShuffleReader.streamCreatorPool import org.apache.spark.util.CompletionIterator import org.apache.spark.util.collection.ExternalSorter @@ -32,7 +32,7 @@ import org.apache.celeborn.client.ShuffleClient import org.apache.celeborn.client.read.CelebornInputStream import org.apache.celeborn.client.read.MetricsCallback import org.apache.celeborn.common.CelebornConf -import org.apache.celeborn.common.exception.CelebornIOException +import org.apache.celeborn.common.exception.{CelebornIOException, PartitionUnRetryAbleException} import org.apache.celeborn.common.util.ThreadUtils class CelebornShuffleReader[K, C]( @@ -42,7 +42,8 @@ class CelebornShuffleReader[K, C]( startMapIndex: Int = 0, endMapIndex: Int = Int.MaxValue, context: TaskContext, - conf: CelebornConf) + conf: CelebornConf, + shuffleIdTracker: ExecutorShuffleIdTracker) extends ShuffleReader[K, C] with Logging { private val dep = handle.dependency @@ -59,6 +60,11 @@ class CelebornShuffleReader[K, C]( val serializerInstance = dep.serializer.newInstance() + val shuffleId = SparkUtils.celebornShuffleId(shuffleClient, handle, context, false) + shuffleIdTracker.track(handle.shuffleId, shuffleId) + logDebug( + s"get shuffleId $shuffleId for appShuffleId ${handle.shuffleId} attemptNum ${context.stageAttemptNumber()}") + // Update the context task metrics for each record read. val readMetrics = context.taskMetrics.createTempShuffleReadMetrics() val metricsCallback = new MetricsCallback { @@ -87,7 +93,7 @@ class CelebornShuffleReader[K, C]( if (exceptionRef.get() == null) { try { val inputStream = shuffleClient.readPartition( - handle.shuffleId, + shuffleId, partitionId, context.attemptNumber(), startMapIndex, @@ -113,7 +119,21 @@ class CelebornShuffleReader[K, C]( var inputStream: CelebornInputStream = streams.get(partitionId) while (inputStream == null) { if (exceptionRef.get() != null) { - throw exceptionRef.get() + exceptionRef.get() match { + case ce @ (_: CelebornIOException | _: PartitionUnRetryAbleException) => + if (handle.throwsFetchFailure && + shuffleClient.reportShuffleFetchFailure(handle.shuffleId, shuffleId)) { + throw new FetchFailedException( + null, + handle.shuffleId, + -1, + partitionId, + SparkUtils.FETCH_FAILURE_ERROR_MSG + shuffleId, + ce) + } else + throw ce + case e => throw e + } } Thread.sleep(50) inputStream = streams.get(partitionId) @@ -122,12 +142,30 @@ class CelebornShuffleReader[K, C]( TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startFetchWait)) // ensure inputStream is closed when task completes context.addTaskCompletionListener(_ => inputStream.close()) - inputStream + (partitionId, inputStream) } else { - CelebornInputStream.empty() + (partitionId, CelebornInputStream.empty()) } - }).flatMap( - serializerInstance.deserializeStream(_).asKeyValueIterator) + }).map { case (partitionId, inputStream) => + (partitionId, serializerInstance.deserializeStream(inputStream).asKeyValueIterator) + }.flatMap { case (partitionId, iter) => + try { + iter + } catch { + case e @ (_: CelebornIOException | _: PartitionUnRetryAbleException) => + if (handle.throwsFetchFailure && + shuffleClient.reportShuffleFetchFailure(handle.shuffleId, shuffleId)) { + throw new FetchFailedException( + null, + handle.shuffleId, + -1, + partitionId, + SparkUtils.FETCH_FAILURE_ERROR_MSG + shuffleId, + e) + } else + throw e + } + } val metricIter = CompletionIterator[(Any, Any), Iterator[(Any, Any)]]( recordIter.map { record => diff --git a/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/CelebornShuffleWriterSuiteBase.java b/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/CelebornShuffleWriterSuiteBase.java index bd9355190bb..f6f38a59289 100644 --- a/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/CelebornShuffleWriterSuiteBase.java +++ b/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/CelebornShuffleWriterSuiteBase.java @@ -227,7 +227,7 @@ private void check( final File tempFile = new File(tempDir, UUID.randomUUID().toString()); final CelebornShuffleHandle handle = new CelebornShuffleHandle<>( - appId, host, port, userIdentifier, shuffleId, numMaps, dependency); + appId, host, port, userIdentifier, shuffleId, false, numMaps, dependency); final ShuffleClient client = new DummyShuffleClient(conf, tempFile); ((DummyShuffleClient) client).initReducePartitionMap(shuffleId, numPartitions, 1); diff --git a/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/HashBasedShuffleWriterSuiteJ.java b/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/HashBasedShuffleWriterSuiteJ.java index 475efee24ca..0d38e2841f8 100644 --- a/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/HashBasedShuffleWriterSuiteJ.java +++ b/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/HashBasedShuffleWriterSuiteJ.java @@ -35,6 +35,12 @@ protected ShuffleWriter createShuffleWriter( throws IOException { // this test case is independent of the `mapId` value return new HashBasedShuffleWriter( - handle, /*mapId=*/ 0, context, conf, client, SendBufferPool.get(1, 30, 60)); + SparkUtils.celebornShuffleId(client, handle, context, true), + handle, + /*mapId=*/ 0, + context, + conf, + client, + SendBufferPool.get(1, 30, 60)); } } diff --git a/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/ShuffleManagerHook.java b/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/ShuffleManagerHook.java new file mode 100644 index 00000000000..3da77f01472 --- /dev/null +++ b/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/ShuffleManagerHook.java @@ -0,0 +1,35 @@ +/* + * 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 org.apache.spark.TaskContext; +import org.apache.spark.shuffle.ShuffleHandle; + +public interface ShuffleManagerHook { + + default void exec( + ShuffleHandle handle, int startPartition, int endPartition, TaskContext context) {} + + default void exec( + ShuffleHandle handle, + int startMapIndex, + int endMapIndex, + int startPartition, + int endPartition, + TaskContext context) {}; +} diff --git a/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriterSuiteJ.java b/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriterSuiteJ.java index efff3b97a9e..74ebab2b986 100644 --- a/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriterSuiteJ.java +++ b/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriterSuiteJ.java @@ -34,6 +34,7 @@ protected ShuffleWriter createShuffleWriter( CelebornShuffleHandle handle, TaskContext context, CelebornConf conf, ShuffleClient client) throws IOException { return new SortBasedShuffleWriter( + SparkUtils.celebornShuffleId(client, handle, context, true), handle.dependency(), numPartitions, context, diff --git a/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/TestCelebornShuffleManager.java b/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/TestCelebornShuffleManager.java new file mode 100644 index 00000000000..5c995e75306 --- /dev/null +++ b/client-spark/spark-2/src/test/java/org/apache/spark/shuffle/celeborn/TestCelebornShuffleManager.java @@ -0,0 +1,45 @@ +/* + * 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 org.apache.spark.SparkConf; +import org.apache.spark.TaskContext; +import org.apache.spark.shuffle.ShuffleHandle; +import org.apache.spark.shuffle.ShuffleReader; + +public class TestCelebornShuffleManager extends SparkShuffleManager { + + private static ShuffleManagerHook shuffleReaderGetHook = null; + + public TestCelebornShuffleManager(SparkConf conf) { + super(conf, true); + } + + public static void registerReaderGetHook(ShuffleManagerHook hook) { + shuffleReaderGetHook = hook; + } + + @Override + public ShuffleReader getReader( + ShuffleHandle handle, int startPartition, int endPartition, TaskContext context) { + if (shuffleReaderGetHook != null) { + shuffleReaderGetHook.exec(handle, startPartition, endPartition, context); + } + return super.getReader(handle, startPartition, endPartition, context); + } +} diff --git a/client-spark/spark-3-columnar-shuffle/src/main/java/org/apache/spark/shuffle/celeborn/ColumnarHashBasedShuffleWriter.java b/client-spark/spark-3-columnar-shuffle/src/main/java/org/apache/spark/shuffle/celeborn/ColumnarHashBasedShuffleWriter.java index be5d78c5023..b468c5b96ff 100644 --- a/client-spark/spark-3-columnar-shuffle/src/main/java/org/apache/spark/shuffle/celeborn/ColumnarHashBasedShuffleWriter.java +++ b/client-spark/spark-3-columnar-shuffle/src/main/java/org/apache/spark/shuffle/celeborn/ColumnarHashBasedShuffleWriter.java @@ -58,6 +58,7 @@ public class ColumnarHashBasedShuffleWriter extends HashBasedShuffleWri private final double columnarShuffleDictionaryMaxFactor; public ColumnarHashBasedShuffleWriter( + int shuffleId, CelebornShuffleHandle handle, TaskContext taskContext, CelebornConf conf, @@ -65,7 +66,7 @@ public ColumnarHashBasedShuffleWriter( ShuffleWriteMetricsReporter metrics, SendBufferPool sendBufferPool) throws IOException { - super(handle, taskContext, conf, client, metrics, sendBufferPool); + super(shuffleId, handle, taskContext, conf, client, metrics, sendBufferPool); columnarShuffleBatchSize = conf.columnarShuffleBatchSize(); columnarShuffleCodeGenEnabled = conf.columnarShuffleCodeGenEnabled(); columnarShuffleDictionaryEnabled = conf.columnarShuffleDictionaryEnabled(); diff --git a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornColumnarShuffleReader.scala b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornColumnarShuffleReader.scala index f47f9880ca8..fd888fb9dc1 100644 --- a/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornColumnarShuffleReader.scala +++ b/client-spark/spark-3-columnar-shuffle/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornColumnarShuffleReader.scala @@ -33,7 +33,8 @@ class CelebornColumnarShuffleReader[K, C]( endMapIndex: Int = Int.MaxValue, context: TaskContext, conf: CelebornConf, - metrics: ShuffleReadMetricsReporter) + metrics: ShuffleReadMetricsReporter, + shuffleIdTracker: ExecutorShuffleIdTracker) extends CelebornShuffleReader[K, C]( handle, startPartition, @@ -42,7 +43,8 @@ class CelebornColumnarShuffleReader[K, C]( endMapIndex, context, conf, - metrics) { + metrics, + shuffleIdTracker) { override def newSerializerInstance(dep: ShuffleDependency[K, _, C]): SerializerInstance = { val schema = CustomShuffleDependencyUtils.getSchema(dep) diff --git a/client-spark/spark-3-columnar-shuffle/src/test/java/org/apache/spark/shuffle/celeborn/ColumnarHashBasedShuffleWriterSuiteJ.java b/client-spark/spark-3-columnar-shuffle/src/test/java/org/apache/spark/shuffle/celeborn/ColumnarHashBasedShuffleWriterSuiteJ.java index e481b6181cc..92aee552fba 100644 --- a/client-spark/spark-3-columnar-shuffle/src/test/java/org/apache/spark/shuffle/celeborn/ColumnarHashBasedShuffleWriterSuiteJ.java +++ b/client-spark/spark-3-columnar-shuffle/src/test/java/org/apache/spark/shuffle/celeborn/ColumnarHashBasedShuffleWriterSuiteJ.java @@ -62,7 +62,7 @@ public void createColumnarShuffleWriter() throws Exception { ShuffleWriter writer = createShuffleWriterWithoutSchema( new CelebornShuffleHandle<>( - "appId", "host", 0, this.userIdentifier, 0, 10, this.dependency), + "appId", "host", 0, this.userIdentifier, 0, false, 10, this.dependency), taskContext, conf, client, @@ -75,7 +75,7 @@ public void createColumnarShuffleWriter() throws Exception { writer = createShuffleWriter( new CelebornShuffleHandle<>( - "appId", "host", 0, this.userIdentifier, 0, 10, this.dependency), + "appId", "host", 0, this.userIdentifier, 0, false, 10, this.dependency), taskContext, conf, client, @@ -108,7 +108,13 @@ protected ShuffleWriter createShuffleWriter( .when(() -> CustomShuffleDependencyUtils.getSchema(handle.dependency())) .thenReturn(schema); return SparkUtils.createColumnarHashBasedShuffleWriter( - handle, context, conf, client, metrics, SendBufferPool.get(1, 30, 60)); + SparkUtils.celebornShuffleId(client, handle, context, true), + handle, + context, + conf, + client, + metrics, + SendBufferPool.get(1, 30, 60)); } } @@ -119,6 +125,12 @@ private ShuffleWriter createShuffleWriterWithoutSchema( ShuffleClient client, ShuffleWriteMetricsReporter metrics) { return SparkUtils.createColumnarHashBasedShuffleWriter( - handle, context, conf, client, metrics, SendBufferPool.get(1, 30, 60)); + SparkUtils.celebornShuffleId(client, handle, context, true), + handle, + context, + conf, + client, + metrics, + SendBufferPool.get(1, 30, 60)); } } diff --git a/client-spark/spark-3-columnar-shuffle/src/test/scala/org/apache/spark/shuffle/celeborn/CelebornColumnarShuffleReaderSuite.scala b/client-spark/spark-3-columnar-shuffle/src/test/scala/org/apache/spark/shuffle/celeborn/CelebornColumnarShuffleReaderSuite.scala index 5a14d021906..5df434f5432 100644 --- a/client-spark/spark-3-columnar-shuffle/src/test/scala/org/apache/spark/shuffle/celeborn/CelebornColumnarShuffleReaderSuite.scala +++ b/client-spark/spark-3-columnar-shuffle/src/test/scala/org/apache/spark/shuffle/celeborn/CelebornColumnarShuffleReaderSuite.scala @@ -39,6 +39,7 @@ class CelebornColumnarShuffleReaderSuite { 0, new UserIdentifier("mock", "mock"), 0, + false, 10, null) @@ -53,7 +54,8 @@ class CelebornColumnarShuffleReaderSuite { 10, null, new CelebornConf(), - null) + null, + new ExecutorShuffleIdTracker()) assert(shuffleReader.getClass == classOf[CelebornColumnarShuffleReader[Int, String]]) } finally { if (shuffleClient != null) { @@ -74,6 +76,7 @@ class CelebornColumnarShuffleReaderSuite { 0, new UserIdentifier("mock", "mock"), 0, + false, 10, null), 0, @@ -82,7 +85,8 @@ class CelebornColumnarShuffleReaderSuite { 10, null, new CelebornConf(), - null) + null, + new ExecutorShuffleIdTracker()) val shuffleDependency = Mockito.mock(classOf[ShuffleDependency[Int, String, String]]) Mockito.when(shuffleDependency.shuffleId).thenReturn(0) Mockito.when(shuffleDependency.serializer).thenReturn(new KryoSerializer( diff --git a/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/HashBasedShuffleWriter.java b/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/HashBasedShuffleWriter.java index 793d06bff9e..7b30101f705 100644 --- a/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/HashBasedShuffleWriter.java +++ b/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/HashBasedShuffleWriter.java @@ -101,6 +101,7 @@ public class HashBasedShuffleWriter extends ShuffleWriter { // In order to facilitate the writing of unit test code, ShuffleClient needs to be passed in as // parameters. By the way, simplify the passed parameters. public HashBasedShuffleWriter( + int shuffleId, CelebornShuffleHandle handle, TaskContext taskContext, CelebornConf conf, @@ -110,7 +111,7 @@ public HashBasedShuffleWriter( throws IOException { this.mapId = taskContext.partitionId(); this.dep = handle.dependency(); - this.shuffleId = dep.shuffleId(); + this.shuffleId = shuffleId; SerializerInstance serializer = dep.serializer().newInstance(); this.partitioner = dep.partitioner(); this.writeMetrics = metrics; diff --git a/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriter.java b/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriter.java index 98c69430663..0b515c456b5 100644 --- a/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriter.java +++ b/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriter.java @@ -94,6 +94,7 @@ public class SortBasedShuffleWriter extends ShuffleWriter { // In order to facilitate the writing of unit test code, ShuffleClient needs to be passed in as // parameters. By the way, simplify the passed parameters. public SortBasedShuffleWriter( + int shuffleId, ShuffleDependency dep, int numMappers, TaskContext taskContext, @@ -105,7 +106,7 @@ public SortBasedShuffleWriter( throws IOException { this.mapId = taskContext.partitionId(); this.dep = dep; - this.shuffleId = dep.shuffleId(); + this.shuffleId = shuffleId; SerializerInstance serializer = dep.serializer().newInstance(); this.partitioner = dep.partitioner(); this.writeMetrics = metrics; @@ -179,6 +180,7 @@ public SortBasedShuffleWriter( SendBufferPool sendBufferPool) throws IOException { this( + SparkUtils.celebornShuffleId(client, handle, taskContext, true), handle.dependency(), handle.numMappers(), taskContext, diff --git a/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/SparkShuffleManager.java b/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/SparkShuffleManager.java index aa1c1a2c253..a1cb458cf1c 100644 --- a/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/SparkShuffleManager.java +++ b/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/SparkShuffleManager.java @@ -24,6 +24,7 @@ import org.apache.spark.*; import org.apache.spark.launcher.SparkLauncher; +import org.apache.spark.rdd.DeterministicLevel; import org.apache.spark.shuffle.*; import org.apache.spark.shuffle.sort.SortShuffleManager; import org.apache.spark.sql.internal.SQLConf; @@ -38,6 +39,17 @@ import org.apache.celeborn.common.util.ThreadUtils; import org.apache.celeborn.reflect.DynMethods; +/** + * In order to support Spark Stage resubmit with ShuffleReader FetchFails, Celeborn shuffleId has to + * be distinguished from Spark shuffleId. Spark shuffleId is assigned at ShuffleDependency + * construction time, and all Attempts of a Spark Stage have the same ShuffleId. When Celeborn + * ShuffleReader fails to fetch shuffle data from worker and throws {@link FetchFailedException}, + * Spark DAGScheduler resubmits the failed ResultStage and corresponding ShuffleMapStage , but + * Celeborn can't differentiate shuffle data from previous failed/resubmitted ShuffleMapStage with + * the same shuffleId. Current solution takes Stage retry in account, and has LifecycleManager to + * generate and track usage of spark shuffle id (appShuffleID) and Celeborn shuffle id (shuffle id). + * Spark shuffle Reader/Write gets shuffleId from LifecycleManager with GetShuffleId RPC + */ public class SparkShuffleManager implements ShuffleManager { private static final Logger logger = LoggerFactory.getLogger(SparkShuffleManager.class); @@ -79,6 +91,8 @@ public class SparkShuffleManager implements ShuffleManager { private long sendBufferPoolCheckInterval; private long sendBufferPoolExpireTimeout; + private ExecutorShuffleIdTracker shuffleIdTracker = new ExecutorShuffleIdTracker(); + public SparkShuffleManager(SparkConf conf, boolean isDriver) { if (conf.getBoolean(SQLConf.LOCAL_SHUFFLE_READER_ENABLED().key(), true)) { logger.warn( @@ -125,6 +139,13 @@ private void initializeLifecycleManager() { synchronized (this) { if (lifecycleManager == null) { lifecycleManager = new LifecycleManager(appUniqueId, celebornConf); + if (celebornConf.clientFetchThrowsFetchFailure()) { + MapOutputTrackerMaster mapOutputTracker = + (MapOutputTrackerMaster) SparkEnv.get().mapOutputTracker(); + + lifecycleManager.registerShuffleTrackerCallback( + shuffleId -> SparkUtils.unregisterAllMapOutput(mapOutputTracker, shuffleId)); + } } } } @@ -139,6 +160,10 @@ public ShuffleHandle registerShuffle( appUniqueId = SparkUtils.appUniqueId(dependency.rdd().context()); initializeLifecycleManager(); + lifecycleManager.registerAppShuffleDeterminate( + shuffleId, + dependency.rdd().getOutputDeterministicLevel() != DeterministicLevel.INDETERMINATE()); + if (fallbackPolicyRunner.applyAllFallbackPolicy( lifecycleManager, dependency.partitioner().numPartitions())) { if (conf.getBoolean("spark.dynamicAllocation.enabled", false) @@ -160,23 +185,24 @@ public ShuffleHandle registerShuffle( lifecycleManager.getPort(), lifecycleManager.getUserIdentifier(), shuffleId, + celebornConf.clientFetchThrowsFetchFailure(), dependency.rdd().getNumPartitions(), dependency); } } @Override - public boolean unregisterShuffle(int shuffleId) { - if (sortShuffleIds.contains(shuffleId)) { - return sortShuffleManager().unregisterShuffle(shuffleId); + public boolean unregisterShuffle(int appShuffleId) { + if (sortShuffleIds.contains(appShuffleId)) { + return sortShuffleManager().unregisterShuffle(appShuffleId); } // For Spark driver side trigger unregister shuffle. if (lifecycleManager != null) { - lifecycleManager.unregisterShuffle(shuffleId); + lifecycleManager.unregisterAppShuffle(appShuffleId); } // For Spark executor side cleanup shuffle related info. if (shuffleClient != null) { - shuffleClient.cleanupShuffle(shuffleId); + shuffleIdTracker.unregisterAppShuffleId(shuffleClient, appShuffleId); } return true; } @@ -217,10 +243,14 @@ public ShuffleWriter getWriter( h.lifecycleManagerPort(), celebornConf, h.userIdentifier()); + int shuffleId = SparkUtils.celebornShuffleId(shuffleClient, h, context, true); + shuffleIdTracker.track(h.shuffleId(), shuffleId); + if (ShuffleMode.SORT.equals(celebornConf.shuffleWriterMode())) { ExecutorService pushThread = celebornConf.clientPushSortPipelineEnabled() ? getPusherThread() : null; return new SortBasedShuffleWriter<>( + shuffleId, h.dependency(), h.numMappers(), context, @@ -234,10 +264,10 @@ public ShuffleWriter getWriter( SendBufferPool.get(cores, sendBufferPoolCheckInterval, sendBufferPoolExpireTimeout); if (COLUMNAR_SHUFFLE_CLASSES_PRESENT && celebornConf.columnarShuffleEnabled()) { return SparkUtils.createColumnarHashBasedShuffleWriter( - h, context, celebornConf, shuffleClient, metrics, pool); + shuffleId, h, context, celebornConf, shuffleClient, metrics, pool); } else { return new HashBasedShuffleWriter<>( - h, context, celebornConf, shuffleClient, metrics, pool); + shuffleId, h, context, celebornConf, shuffleClient, metrics, pool); } } else { throw new UnsupportedOperationException( @@ -340,7 +370,8 @@ public ShuffleReader getCelebornShuffleReader( endMapIndex, context, celebornConf, - metrics); + metrics, + shuffleIdTracker); } else { return new CelebornShuffleReader<>( h, @@ -350,7 +381,8 @@ public ShuffleReader getCelebornShuffleReader( endMapIndex, context, celebornConf, - metrics); + metrics, + shuffleIdTracker); } } diff --git a/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/SparkUtils.java b/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/SparkUtils.java index 46a54f2ffc7..e7a6a5b8b6c 100644 --- a/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/SparkUtils.java +++ b/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/SparkUtils.java @@ -21,6 +21,7 @@ import scala.Tuple2; +import org.apache.spark.MapOutputTrackerMaster; import org.apache.spark.SparkConf; import org.apache.spark.SparkContext; import org.apache.spark.TaskContext; @@ -46,6 +47,8 @@ public class SparkUtils { private static final Logger LOG = LoggerFactory.getLogger(SparkUtils.class); + public static final String FETCH_FAILURE_ERROR_MSG = "Celeborn FetchFailure with shuffle id "; + public static MapStatus createMapStatus( BlockManagerId loc, long[] uncompressedSizes, long mapTaskId) { return MapStatus$.MODULE$.apply(loc, uncompressedSizes, mapTaskId); @@ -92,6 +95,23 @@ public static String appUniqueId(SparkContext context) { .getOrElse(context::applicationId); } + public static String getAppShuffleIdentifier(int appShuffleId, TaskContext context) { + return appShuffleId + "-" + context.stageId() + "-" + context.stageAttemptNumber(); + } + + public static int celebornShuffleId( + ShuffleClient client, + CelebornShuffleHandle handle, + TaskContext context, + Boolean isWriter) { + if (handle.throwsFetchFailure()) { + String appShuffleIdentifier = getAppShuffleIdentifier(handle.shuffleId(), context); + return client.getShuffleId(handle.shuffleId(), appShuffleIdentifier, isWriter); + } else { + return handle.shuffleId(); + } + } + // Create an instance of the class with the given name, possibly initializing it with our conf // Copied from SparkEnv public static T instantiateClass(String className, SparkConf conf, Boolean isDriver) { @@ -164,6 +184,7 @@ public static ShuffleReader getReader( DynConstructors.builder() .impl( COLUMNAR_HASH_BASED_SHUFFLE_WRITER_CLASS, + int.class, CelebornShuffleHandle.class, TaskContext.class, CelebornConf.class, @@ -172,6 +193,7 @@ public static ShuffleReader getReader( SendBufferPool.class); public static HashBasedShuffleWriter createColumnarHashBasedShuffleWriter( + int shuffleId, CelebornShuffleHandle handle, TaskContext taskContext, CelebornConf conf, @@ -180,7 +202,7 @@ public static HashBasedShuffleWriter createColumnarHashBasedS SendBufferPool sendBufferPool) { return COLUMNAR_HASH_BASED_SHUFFLE_WRITER_CONSTRUCTOR_BUILDER .build() - .invoke(null, handle, taskContext, conf, client, metrics, sendBufferPool); + .invoke(null, shuffleId, handle, taskContext, conf, client, metrics, sendBufferPool); } public static final String COLUMNAR_SHUFFLE_READER_CLASS = @@ -196,7 +218,8 @@ public static HashBasedShuffleWriter createColumnarHashBasedS int.class, TaskContext.class, CelebornConf.class, - ShuffleReadMetricsReporter.class); + ShuffleReadMetricsReporter.class, + ExecutorShuffleIdTracker.class); public static CelebornShuffleReader createColumnarShuffleReader( CelebornShuffleHandle handle, @@ -206,7 +229,8 @@ public static CelebornShuffleReader createColumnarShuffleReader( int endMapIndex, TaskContext context, CelebornConf conf, - ShuffleReadMetricsReporter metrics) { + ShuffleReadMetricsReporter metrics, + ExecutorShuffleIdTracker shuffleIdTracker) { return COLUMNAR_SHUFFLE_READER_CONSTRUCTOR_BUILDER .build() .invoke( @@ -218,6 +242,35 @@ public static CelebornShuffleReader createColumnarShuffleReader( endMapIndex, context, conf, - metrics); + metrics, + shuffleIdTracker); + } + + // Added in SPARK-32920, for Spark 3.2 and above + private static final DynMethods.UnboundMethod UnregisterAllMapAndMergeOutput_METHOD = + DynMethods.builder("unregisterAllMapAndMergeOutput") + .impl(MapOutputTrackerMaster.class, Integer.TYPE) + .orNoop() + .build(); + + // for spark 3.1, see detail in SPARK-32920 + private static final DynMethods.UnboundMethod UnregisterAllMapOutput_METHOD = + DynMethods.builder("unregisterAllMapOutput") + .impl(MapOutputTrackerMaster.class, Integer.TYPE) + .orNoop() + .build(); + + public static void unregisterAllMapOutput( + MapOutputTrackerMaster mapOutputTracker, int shuffleId) { + if (!UnregisterAllMapAndMergeOutput_METHOD.isNoop()) { + UnregisterAllMapAndMergeOutput_METHOD.bind(mapOutputTracker).invoke(shuffleId); + return; + } + if (!UnregisterAllMapOutput_METHOD.isNoop()) { + UnregisterAllMapOutput_METHOD.bind(mapOutputTracker).invoke(shuffleId); + return; + } + throw new UnsupportedOperationException( + "unexpected! neither methods unregisterAllMapAndMergeOutput/unregisterAllMapOutput are found in MapOutputTrackerMaster"); } } diff --git a/client-spark/spark-3/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleHandle.scala b/client-spark/spark-3/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleHandle.scala index 9d411ceab16..18a3053e006 100644 --- a/client-spark/spark-3/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleHandle.scala +++ b/client-spark/spark-3/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleHandle.scala @@ -28,6 +28,7 @@ class CelebornShuffleHandle[K, V, C]( val lifecycleManagerPort: Int, val userIdentifier: UserIdentifier, shuffleId: Int, + val throwsFetchFailure: Boolean, val numMappers: Int, dependency: ShuffleDependency[K, V, C]) extends BaseShuffleHandle(shuffleId, dependency) diff --git a/client-spark/spark-3/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleReader.scala b/client-spark/spark-3/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleReader.scala index d83df1a5b76..fe7af83091a 100644 --- a/client-spark/spark-3/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleReader.scala +++ b/client-spark/spark-3/src/main/scala/org/apache/spark/shuffle/celeborn/CelebornShuffleReader.scala @@ -24,7 +24,7 @@ import java.util.concurrent.atomic.AtomicReference import org.apache.spark.{InterruptibleIterator, ShuffleDependency, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.serializer.SerializerInstance -import org.apache.spark.shuffle.{ShuffleReader, ShuffleReadMetricsReporter} +import org.apache.spark.shuffle.{FetchFailedException, ShuffleReader, ShuffleReadMetricsReporter} import org.apache.spark.shuffle.celeborn.CelebornShuffleReader.streamCreatorPool import org.apache.spark.util.CompletionIterator import org.apache.spark.util.collection.ExternalSorter @@ -32,7 +32,7 @@ import org.apache.spark.util.collection.ExternalSorter import org.apache.celeborn.client.ShuffleClient import org.apache.celeborn.client.read.{CelebornInputStream, MetricsCallback} import org.apache.celeborn.common.CelebornConf -import org.apache.celeborn.common.exception.CelebornIOException +import org.apache.celeborn.common.exception.{CelebornIOException, PartitionUnRetryAbleException} import org.apache.celeborn.common.util.ThreadUtils class CelebornShuffleReader[K, C]( @@ -43,7 +43,8 @@ class CelebornShuffleReader[K, C]( endMapIndex: Int = Int.MaxValue, context: TaskContext, conf: CelebornConf, - metrics: ShuffleReadMetricsReporter) + metrics: ShuffleReadMetricsReporter, + shuffleIdTracker: ExecutorShuffleIdTracker) extends ShuffleReader[K, C] with Logging { private val dep = handle.dependency @@ -60,6 +61,11 @@ class CelebornShuffleReader[K, C]( val serializerInstance = newSerializerInstance(dep) + val shuffleId = SparkUtils.celebornShuffleId(shuffleClient, handle, context, false) + shuffleIdTracker.track(handle.shuffleId, shuffleId) + logDebug( + s"get shuffleId $shuffleId for appShuffleId ${handle.shuffleId} attemptNum ${context.stageAttemptNumber()}") + // Update the context task metrics for each record read. val metricsCallback = new MetricsCallback { override def incBytesRead(bytesWritten: Long): Unit = { @@ -89,7 +95,7 @@ class CelebornShuffleReader[K, C]( if (exceptionRef.get() == null) { try { val inputStream = shuffleClient.readPartition( - handle.shuffleId, + shuffleId, partitionId, context.attemptNumber(), startMapIndex, @@ -115,7 +121,22 @@ class CelebornShuffleReader[K, C]( var inputStream: CelebornInputStream = streams.get(partitionId) while (inputStream == null) { if (exceptionRef.get() != null) { - throw exceptionRef.get() + exceptionRef.get() match { + case ce @ (_: CelebornIOException | _: PartitionUnRetryAbleException) => + if (handle.throwsFetchFailure && + shuffleClient.reportShuffleFetchFailure(handle.shuffleId, shuffleId)) { + throw new FetchFailedException( + null, + handle.shuffleId, + -1, + -1, + partitionId, + SparkUtils.FETCH_FAILURE_ERROR_MSG + shuffleId, + ce) + } else + throw ce + case e => throw e + } } Thread.sleep(50) inputStream = streams.get(partitionId) @@ -124,12 +145,31 @@ class CelebornShuffleReader[K, C]( TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startFetchWait)) // ensure inputStream is closed when task completes context.addTaskCompletionListener[Unit](_ => inputStream.close()) - inputStream + (partitionId, inputStream) } else { - CelebornInputStream.empty() + (partitionId, CelebornInputStream.empty()) } - }).flatMap( - serializerInstance.deserializeStream(_).asKeyValueIterator) + }).map { case (partitionId, inputStream) => + (partitionId, serializerInstance.deserializeStream(inputStream).asKeyValueIterator) + }.flatMap { case (partitionId, iter) => + try { + iter + } catch { + case e @ (_: CelebornIOException | _: PartitionUnRetryAbleException) => + if (handle.throwsFetchFailure && + shuffleClient.reportShuffleFetchFailure(handle.shuffleId, shuffleId)) { + throw new FetchFailedException( + null, + handle.shuffleId, + -1, + -1, + partitionId, + SparkUtils.FETCH_FAILURE_ERROR_MSG + shuffleId, + e) + } else + throw e + } + } val iterWithUpdatedRecordsRead = if (GlutenShuffleDependencyHelper.isGlutenDep(dep.getClass.getName)) { diff --git a/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/CelebornShuffleWriterSuiteBase.java b/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/CelebornShuffleWriterSuiteBase.java index f4109753ea4..65930d9ba5f 100644 --- a/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/CelebornShuffleWriterSuiteBase.java +++ b/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/CelebornShuffleWriterSuiteBase.java @@ -236,7 +236,7 @@ private void check( final File tempFile = new File(tempDir, UUID.randomUUID().toString()); final CelebornShuffleHandle handle = new CelebornShuffleHandle<>( - appId, host, port, userIdentifier, shuffleId, numMaps, dependency); + appId, host, port, userIdentifier, shuffleId, false, numMaps, dependency); final ShuffleClient client = new DummyShuffleClient(conf, tempFile); ((DummyShuffleClient) client).initReducePartitionMap(shuffleId, numPartitions, 1); diff --git a/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/HashBasedShuffleWriterSuiteJ.java b/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/HashBasedShuffleWriterSuiteJ.java index d8c514dc72f..e2876ff4a1e 100644 --- a/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/HashBasedShuffleWriterSuiteJ.java +++ b/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/HashBasedShuffleWriterSuiteJ.java @@ -37,6 +37,12 @@ protected ShuffleWriter createShuffleWriter( ShuffleWriteMetricsReporter metrics) throws IOException { return new HashBasedShuffleWriter( - handle, context, conf, client, metrics, SendBufferPool.get(1, 30, 60)); + SparkUtils.celebornShuffleId(client, handle, context, true), + handle, + context, + conf, + client, + metrics, + SendBufferPool.get(1, 30, 60)); } } diff --git a/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/ShuffleManagerHook.java b/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/ShuffleManagerHook.java new file mode 100644 index 00000000000..3da77f01472 --- /dev/null +++ b/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/ShuffleManagerHook.java @@ -0,0 +1,35 @@ +/* + * 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 org.apache.spark.TaskContext; +import org.apache.spark.shuffle.ShuffleHandle; + +public interface ShuffleManagerHook { + + default void exec( + ShuffleHandle handle, int startPartition, int endPartition, TaskContext context) {} + + default void exec( + ShuffleHandle handle, + int startMapIndex, + int endMapIndex, + int startPartition, + int endPartition, + TaskContext context) {}; +} diff --git a/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/TestCelebornShuffleManager.java b/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/TestCelebornShuffleManager.java new file mode 100644 index 00000000000..637f337713d --- /dev/null +++ b/client-spark/spark-3/src/test/java/org/apache/spark/shuffle/celeborn/TestCelebornShuffleManager.java @@ -0,0 +1,67 @@ +/* + * 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 org.apache.spark.SparkConf; +import org.apache.spark.TaskContext; +import org.apache.spark.shuffle.ShuffleHandle; +import org.apache.spark.shuffle.ShuffleReadMetricsReporter; +import org.apache.spark.shuffle.ShuffleReader; + +public class TestCelebornShuffleManager extends SparkShuffleManager { + + private static ShuffleManagerHook shuffleReaderGetHook = null; + + public TestCelebornShuffleManager(SparkConf conf) { + super(conf, true); + } + + public static void registerReaderGetHook(ShuffleManagerHook hook) { + shuffleReaderGetHook = hook; + } + + @Override + public ShuffleReader getReader( + ShuffleHandle handle, + int startMapIndex, + int endMapIndex, + int startPartition, + int endPartition, + TaskContext context, + ShuffleReadMetricsReporter metrics) { + if (shuffleReaderGetHook != null) { + shuffleReaderGetHook.exec( + handle, startMapIndex, endMapIndex, startPartition, endPartition, context); + } + return super.getReader( + handle, startMapIndex, endMapIndex, startPartition, endPartition, context, metrics); + } + + @Override + public ShuffleReader getReader( + ShuffleHandle handle, + int startPartition, + int endPartition, + TaskContext context, + ShuffleReadMetricsReporter metrics) { + if (shuffleReaderGetHook != null) { + shuffleReaderGetHook.exec(handle, startPartition, endPartition, context); + } + return super.getReader(handle, startPartition, endPartition, context, metrics); + } +} diff --git a/client/src/main/java/org/apache/celeborn/client/ShuffleClient.java b/client/src/main/java/org/apache/celeborn/client/ShuffleClient.java index b4b566d1a75..72230a536de 100644 --- a/client/src/main/java/org/apache/celeborn/client/ShuffleClient.java +++ b/client/src/main/java/org/apache/celeborn/client/ShuffleClient.java @@ -212,4 +212,13 @@ public abstract ConcurrentHashMap getPartitionLocati int shuffleId, int numMappers, int numPartitions); public abstract PushState getPushState(String mapKey); + + public abstract int getShuffleId(int appShuffleId, String appShuffleIdentifier, boolean isWriter); + + /** + * report shuffle data fetch failure to LifecycleManager for special handling, eg, shuffle status + * cleanup for spark app. It must be a sync call and make sure the cleanup is done, otherwise, + * incorrect shuffle data can be fetched in re-run tasks + */ + public abstract boolean reportShuffleFetchFailure(int appShuffleId, int shuffleId); } diff --git a/client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java b/client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java index a300ddf5f86..b1a37751720 100644 --- a/client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java +++ b/client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java @@ -86,6 +86,9 @@ public class ShuffleClientImpl extends ShuffleClient { protected final int BATCH_HEADER_SIZE = 4 * 4; + // key: appShuffleIdentifier, value: shuffleId + protected Map shuffleIdCache = JavaUtils.newConcurrentHashMap(); + // key: shuffleId, value: (partitionId, PartitionLocation) final Map> reducePartitionMap = JavaUtils.newConcurrentHashMap(); @@ -517,6 +520,41 @@ public PushState getPushState(String mapKey) { return pushStates.computeIfAbsent(mapKey, (s) -> new PushState(conf)); } + @Override + public int getShuffleId(int appShuffleId, String appShuffleIdentifier, boolean isWriter) { + return shuffleIdCache.computeIfAbsent( + appShuffleIdentifier, + (id) -> { + PbGetShuffleId pbGetShuffleId = + PbGetShuffleId.newBuilder() + .setAppShuffleId(appShuffleId) + .setAppShuffleIdentifier(appShuffleIdentifier) + .setIsShuffleWriter(isWriter) + .build(); + PbGetShuffleIdResponse pbGetShuffleIdResponse = + lifecycleManagerRef.askSync( + pbGetShuffleId, + conf.clientRpcRegisterShuffleRpcAskTimeout(), + ClassTag$.MODULE$.apply(PbGetShuffleIdResponse.class)); + return pbGetShuffleIdResponse.getShuffleId(); + }); + } + + @Override + public boolean reportShuffleFetchFailure(int appShuffleId, int shuffleId) { + PbReportShuffleFetchFailure pbReportShuffleFetchFailure = + PbReportShuffleFetchFailure.newBuilder() + .setAppShuffleId(appShuffleId) + .setShuffleId(shuffleId) + .build(); + PbReportShuffleFetchFailureResponse pbReportShuffleFetchFailureResponse = + lifecycleManagerRef.askSync( + pbReportShuffleFetchFailure, + conf.clientRpcRegisterShuffleRpcAskTimeout(), + ClassTag$.MODULE$.apply(PbReportShuffleFetchFailureResponse.class)); + return pbReportShuffleFetchFailureResponse.getSuccess(); + } + private ConcurrentHashMap registerShuffleInternal( int shuffleId, int numMappers, diff --git a/client/src/main/scala/org/apache/celeborn/client/LifecycleManager.scala b/client/src/main/scala/org/apache/celeborn/client/LifecycleManager.scala index cf54dfcd804..ed308840ae7 100644 --- a/client/src/main/scala/org/apache/celeborn/client/LifecycleManager.scala +++ b/client/src/main/scala/org/apache/celeborn/client/LifecycleManager.scala @@ -82,6 +82,13 @@ class LifecycleManager(val appUniqueId: String, val conf: CelebornConf) extends JavaUtils.newConcurrentHashMap[Int, ConcurrentHashMap[Int, PartitionLocation]]() private val userIdentifier: UserIdentifier = IdentityProvider.instantiate(conf).provide() private val availableStorageTypes = conf.availableStorageTypes + // app shuffle id -> LinkedHashMap of (app shuffle identifier, (shuffle id, fetch status)) + private val shuffleIdMapping = JavaUtils.newConcurrentHashMap[ + Int, + scala.collection.mutable.LinkedHashMap[String, (Int, Boolean)]]() + private val shuffleIdGenerator = new AtomicInteger(0) + // app shuffle id -> whether shuffle is determinate, rerun of a indeterminate shuffle gets different result + private val appShuffleDeterminateMap = JavaUtils.newConcurrentHashMap[Int, Boolean](); private val rpcCacheSize = conf.clientRpcCacheSize private val rpcCacheConcurrencyLevel = conf.clientRpcCacheConcurrencyLevel @@ -312,6 +319,19 @@ class LifecycleManager(val appUniqueId: String, val conf: CelebornConf) extends case GetReducerFileGroup(shuffleId: Int) => logDebug(s"Received GetShuffleFileGroup request for shuffleId $shuffleId.") handleGetReducerFileGroup(context, shuffleId) + + case pb: PbGetShuffleId => + val appShuffleId = pb.getAppShuffleId + val appShuffleIdentifier = pb.getAppShuffleIdentifier + val isWriter = pb.getIsShuffleWriter + logDebug(s"Received GetShuffleId request, appShuffleId $appShuffleId appShuffleIdentifier $appShuffleIdentifier isWriter $isWriter.") + handleGetShuffleIdForApp(context, appShuffleId, appShuffleIdentifier, isWriter) + + case pb: PbReportShuffleFetchFailure => + val appShuffleId = pb.getAppShuffleId + val shuffleId = pb.getShuffleId + logDebug(s"Received ReportShuffleFetchFailure request, appShuffleId $appShuffleId shuffleId $shuffleId") + handleReportShuffleFetchFailure(context, appShuffleId, shuffleId) } private def offerAndReserveSlots( @@ -628,6 +648,118 @@ class LifecycleManager(val appUniqueId: String, val conf: CelebornConf) extends commitManager.handleGetReducerFileGroup(context, shuffleId) } + private def handleGetShuffleIdForApp( + context: RpcCallContext, + appShuffleId: Int, + appShuffleIdentifier: String, + isWriter: Boolean): Unit = { + val shuffleIds = shuffleIdMapping.computeIfAbsent( + appShuffleId, + new function.Function[Int, scala.collection.mutable.LinkedHashMap[String, (Int, Boolean)]]() { + override def apply(id: Int) + : scala.collection.mutable.LinkedHashMap[String, (Int, Boolean)] = { + val newShuffleId = shuffleIdGenerator.getAndIncrement() + logInfo(s"generate new shuffleId $newShuffleId for appShuffleId $appShuffleId appShuffleIdentifier $appShuffleIdentifier") + scala.collection.mutable.LinkedHashMap(appShuffleIdentifier -> (newShuffleId, true)) + } + }) + + def isAllMaptaskEnd(shuffleId: Int): Boolean = { + !commitManager.getMapperAttempts(shuffleId).exists(_ < 0) + } + + shuffleIds.synchronized { + if (isWriter) { + shuffleIds.get(appShuffleIdentifier) match { + case Some((shuffleId, _)) => + val pbGetShuffleIdResponse = + PbGetShuffleIdResponse.newBuilder().setShuffleId(shuffleId).build() + context.reply(pbGetShuffleIdResponse) + case None => + Option(appShuffleDeterminateMap.get(appShuffleId)).map { determinate => + val candidateShuffle = + if (determinate) + shuffleIds.values.toSeq.reverse.find(e => e._2 == true) + else + None + + val shuffleId: Integer = + if (determinate && candidateShuffle.isDefined) { + val id = candidateShuffle.get._1 + logInfo(s"reuse existing shuffleId $id for appShuffleId $appShuffleId appShuffleIdentifier $appShuffleIdentifier") + id + } else { + val newShuffleId = shuffleIdGenerator.getAndIncrement() + logInfo(s"generate new shuffleId $newShuffleId for appShuffleId $appShuffleId appShuffleIdentifier $appShuffleIdentifier") + shuffleIds.put(appShuffleIdentifier, (newShuffleId, true)) + newShuffleId + } + val pbGetShuffleIdResponse = + PbGetShuffleIdResponse.newBuilder().setShuffleId(shuffleId).build() + context.reply(pbGetShuffleIdResponse) + }.orElse( + throw new UnsupportedOperationException( + s"unexpected! unknown appShuffleId $appShuffleId when checking shuffle deterministic level")) + } + } else { + shuffleIds.values.map(v => v._1).toSeq.reverse.find(isAllMaptaskEnd) match { + case Some(shuffleId) => + val pbGetShuffleIdResponse = { + logDebug( + s"get shuffleId $shuffleId for appShuffleId $appShuffleId appShuffleIdentifier $appShuffleIdentifier isWriter $isWriter") + PbGetShuffleIdResponse.newBuilder().setShuffleId(shuffleId).build() + } + context.reply(pbGetShuffleIdResponse) + case None => + throw new UnsupportedOperationException( + s"unexpected! there is no finished map stage associated with appShuffleId $appShuffleId") + } + } + } + } + + private def handleReportShuffleFetchFailure( + context: RpcCallContext, + appShuffleId: Int, + shuffleId: Int): Unit = { + + val shuffleIds = shuffleIdMapping.get(appShuffleId) + if (shuffleIds == null) { + throw new UnsupportedOperationException(s"unexpected! unknown appShuffleId $appShuffleId") + } + var ret = true + shuffleIds.synchronized { + shuffleIds.find(e => e._2._1 == shuffleId) match { + case Some((appShuffleIdentifier, (shuffleId, true))) => + logInfo(s"handle fetch failure for appShuffleId $appShuffleId shuffleId $shuffleId") + appShuffleTrackerCallback match { + case Some(callback) => + try { + callback.accept(appShuffleId) + } catch { + case t: Throwable => + logError(t.toString) + ret = false + } + shuffleIds.put(appShuffleIdentifier, (shuffleId, false)) + case None => + throw new UnsupportedOperationException( + "unexpected! appShuffleTrackerCallback is not registered") + } + case Some((appShuffleIdentifier, (shuffleId, false))) => + logInfo( + s"Ignoring fetch failure from appShuffleIdentifier $appShuffleIdentifier shuffleId $shuffleId, " + + "fetch failure is already reported and handled by other reader") + case None => throw new UnsupportedOperationException( + s"unexpected! unknown shuffleId $shuffleId for appShuffleId $appShuffleId") + } + } + + val pbReportShuffleFetchFailureResponse = + PbReportShuffleFetchFailureResponse.newBuilder().setSuccess(ret).build() + context.reply(pbReportShuffleFetchFailureResponse) + } + private def handleStageEnd(shuffleId: Int): Unit = { // check whether shuffle has registered if (!registeredShuffle.contains(shuffleId)) { @@ -701,6 +833,19 @@ class LifecycleManager(val appUniqueId: String, val conf: CelebornConf) extends logInfo(s"Unregister for $shuffleId success.") } + def unregisterAppShuffle(appShuffleId: Int): Unit = { + logInfo(s"Unregister appShuffleId $appShuffleId starts...") + appShuffleDeterminateMap.remove(appShuffleId) + val shuffleIds = shuffleIdMapping.remove(appShuffleId) + if (shuffleIds != null) { + shuffleIds.synchronized( + shuffleIds.values.map { + case (shuffleId, _) => + unregisterShuffle(shuffleId) + }) + } + } + /* ========================================================== * | END OF EVENT HANDLER | * ========================================================== */ @@ -1208,6 +1353,15 @@ class LifecycleManager(val appUniqueId: String, val conf: CelebornConf) extends workerStatusTracker.registerWorkerStatusListener(workerStatusListener) } + @volatile private var appShuffleTrackerCallback: Option[Consumer[Integer]] = None + def registerShuffleTrackerCallback(callback: Consumer[Integer]): Unit = { + appShuffleTrackerCallback = Some(callback) + } + + def registerAppShuffleDeterminate(appShuffleId: Int, determinate: Boolean): Unit = { + appShuffleDeterminateMap.put(appShuffleId, determinate) + } + // Initialize at the end of LifecycleManager construction. initialize() diff --git a/client/src/test/java/org/apache/celeborn/client/DummyShuffleClient.java b/client/src/test/java/org/apache/celeborn/client/DummyShuffleClient.java index 339b8b8599c..bb4f4fe4107 100644 --- a/client/src/test/java/org/apache/celeborn/client/DummyShuffleClient.java +++ b/client/src/test/java/org/apache/celeborn/client/DummyShuffleClient.java @@ -153,6 +153,16 @@ public PushState getPushState(String mapKey) { return new PushState(conf); } + @Override + public int getShuffleId(int appShuffleId, String appShuffleIdentifier, boolean isWriter) { + return appShuffleId; + } + + @Override + public boolean reportShuffleFetchFailure(int appShuffleId, int shuffleId) { + return true; + } + public void initReducePartitionMap(int shuffleId, int numPartitions, int workerNum) { ConcurrentHashMap map = JavaUtils.newConcurrentHashMap(); String host = "host"; diff --git a/common/src/main/java/org/apache/celeborn/common/network/protocol/TransportMessage.java b/common/src/main/java/org/apache/celeborn/common/network/protocol/TransportMessage.java index 8fa07a145d1..a716d7dedb4 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/protocol/TransportMessage.java +++ b/common/src/main/java/org/apache/celeborn/common/network/protocol/TransportMessage.java @@ -32,11 +32,15 @@ import org.apache.celeborn.common.protocol.PbBacklogAnnouncement; import org.apache.celeborn.common.protocol.PbBufferStreamEnd; import org.apache.celeborn.common.protocol.PbChunkFetchRequest; +import org.apache.celeborn.common.protocol.PbGetShuffleId; +import org.apache.celeborn.common.protocol.PbGetShuffleIdResponse; import org.apache.celeborn.common.protocol.PbOpenStream; import org.apache.celeborn.common.protocol.PbPushDataHandShake; import org.apache.celeborn.common.protocol.PbReadAddCredit; import org.apache.celeborn.common.protocol.PbRegionFinish; import org.apache.celeborn.common.protocol.PbRegionStart; +import org.apache.celeborn.common.protocol.PbReportShuffleFetchFailure; +import org.apache.celeborn.common.protocol.PbReportShuffleFetchFailureResponse; import org.apache.celeborn.common.protocol.PbStreamChunkSlice; import org.apache.celeborn.common.protocol.PbStreamHandler; import org.apache.celeborn.common.protocol.PbTransportableError; @@ -90,6 +94,14 @@ public T getParsedPayload() throws InvalidProtoco return (T) PbChunkFetchRequest.parseFrom(payload); case TRANSPORTABLE_ERROR_VALUE: return (T) PbTransportableError.parseFrom(payload); + case GET_SHUFFLE_ID_VALUE: + return (T) PbGetShuffleId.parseFrom(payload); + case GET_SHUFFLE_ID_RESPONSE_VALUE: + return (T) PbGetShuffleIdResponse.parseFrom(payload); + case REPORT_SHUFFLE_FETCH_FAILURE_VALUE: + return (T) PbReportShuffleFetchFailure.parseFrom(payload); + case REPORT_SHUFFLE_FETCH_FAILURE_RESPONSE_VALUE: + return (T) PbReportShuffleFetchFailureResponse.parseFrom(payload); default: logger.error("Unexpected type {}", type); } diff --git a/common/src/main/proto/TransportMessages.proto b/common/src/main/proto/TransportMessages.proto index 56f0e7561c9..a595a785df8 100644 --- a/common/src/main/proto/TransportMessages.proto +++ b/common/src/main/proto/TransportMessages.proto @@ -87,6 +87,10 @@ enum MessageType { TRANSPORTABLE_ERROR = 64; WORKER_EXCLUDE = 65; WORKER_EXCLUDE_RESPONSE = 66; + REPORT_SHUFFLE_FETCH_FAILURE = 67; + REPORT_SHUFFLE_FETCH_FAILURE_RESPONSE = 68; + GET_SHUFFLE_ID = 69; + GET_SHUFFLE_ID_RESPONSE = 70; } enum StreamType { @@ -296,6 +300,25 @@ message PbGetReducerFileGroupResponse { repeated int32 partitionIds = 4; } +message PbGetShuffleId { + int32 appShuffleId = 1; + string appShuffleIdentifier = 2; + bool isShuffleWriter = 3; +} + +message PbGetShuffleIdResponse { + int32 shuffleId = 1; +} + +message PbReportShuffleFetchFailure { + int32 appShuffleId = 1; + int32 shuffleId = 2; +} + +message PbReportShuffleFetchFailureResponse { + bool success = 1; +} + message PbUnregisterShuffle { string appId = 1; int32 shuffleId = 2; diff --git a/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala b/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala index 34fcfaca71e..5b170eb3033 100644 --- a/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala +++ b/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala @@ -777,6 +777,7 @@ class CelebornConf(loadDefaults: Boolean) extends Cloneable with Logging with Se def clientFetchTimeoutMs: Long = get(CLIENT_FETCH_TIMEOUT) def clientFetchMaxReqsInFlight: Int = get(CLIENT_FETCH_MAX_REQS_IN_FLIGHT) def clientFetchMaxRetriesForEachReplica: Int = get(CLIENT_FETCH_MAX_RETRIES_FOR_EACH_REPLICA) + def clientFetchThrowsFetchFailure: Boolean = get(CLIENT_FETCH_THROWS_FETCH_FAILURE) def clientFetchExcludeWorkerOnFailureEnabled: Boolean = get(CLIENT_FETCH_EXCLUDE_WORKER_ON_FAILURE_ENABLED) def clientFetchExcludedWorkerExpireTimeout: Long = @@ -3228,6 +3229,14 @@ object CelebornConf extends Logging { .intConf .createWithDefault(3) + val CLIENT_FETCH_THROWS_FETCH_FAILURE: ConfigEntry[Boolean] = + buildConf("celeborn.client.spark.fetch.throwsFetchFailure") + .categories("client") + .version("0.4.0") + .doc("client throws FetchFailedException instead of CelebornIOException") + .booleanConf + .createWithDefault(false) + val CLIENT_FETCH_EXCLUDE_WORKER_ON_FAILURE_ENABLED: ConfigEntry[Boolean] = buildConf("celeborn.client.fetch.excludeWorkerOnFailure.enabled") .categories("client") diff --git a/common/src/main/scala/org/apache/celeborn/common/protocol/message/ControlMessages.scala b/common/src/main/scala/org/apache/celeborn/common/protocol/message/ControlMessages.scala index 63b4f7b1fd4..63fae0775ed 100644 --- a/common/src/main/scala/org/apache/celeborn/common/protocol/message/ControlMessages.scala +++ b/common/src/main/scala/org/apache/celeborn/common/protocol/message/ControlMessages.scala @@ -487,6 +487,18 @@ object ControlMessages extends Logging { case pb: PbRegisterWorker => new TransportMessage(MessageType.REGISTER_WORKER, pb.toByteArray) + case pb: PbGetShuffleId => + new TransportMessage(MessageType.GET_SHUFFLE_ID, pb.toByteArray) + + case pb: PbGetShuffleIdResponse => + new TransportMessage(MessageType.GET_SHUFFLE_ID_RESPONSE, pb.toByteArray) + + case pb: PbReportShuffleFetchFailure => + new TransportMessage(MessageType.REPORT_SHUFFLE_FETCH_FAILURE, pb.toByteArray) + + case pb: PbReportShuffleFetchFailureResponse => + new TransportMessage(MessageType.REPORT_SHUFFLE_FETCH_FAILURE_RESPONSE, pb.toByteArray) + case HeartbeatFromWorker( host, rpcPort, @@ -980,6 +992,18 @@ object ControlMessages extends Logging { attempts, partitionIds) + case GET_SHUFFLE_ID_VALUE => + message.getParsedPayload() + + case GET_SHUFFLE_ID_RESPONSE_VALUE => + message.getParsedPayload() + + case REPORT_SHUFFLE_FETCH_FAILURE_VALUE => + message.getParsedPayload() + + case REPORT_SHUFFLE_FETCH_FAILURE_RESPONSE_VALUE => + message.getParsedPayload() + case UNREGISTER_SHUFFLE_VALUE => PbUnregisterShuffle.parseFrom(message.getPayload) diff --git a/docs/configuration/client.md b/docs/configuration/client.md index 1c1d4e7a620..4871ac5509a 100644 --- a/docs/configuration/client.md +++ b/docs/configuration/client.md @@ -98,6 +98,7 @@ license: | | celeborn.client.shuffle.partitionSplit.threshold | 1G | Shuffle file size threshold, if file size exceeds this, trigger split. | 0.3.0 | | celeborn.client.shuffle.rangeReadFilter.enabled | false | If a spark application have skewed partition, this value can set to true to improve performance. | 0.2.0 | | celeborn.client.slot.assign.maxWorkers | 10000 | Max workers that slots of one shuffle can be allocated on. Will choose the smaller positive one from Master side and Client side, see `celeborn.master.slot.assign.maxWorkers`. | 0.3.1 | +| celeborn.client.spark.fetch.throwsFetchFailure | false | client throws FetchFailedException instead of CelebornIOException | 0.4.0 | | celeborn.client.spark.push.sort.memory.threshold | 64m | When SortBasedPusher use memory over the threshold, will trigger push data. If the pipeline push feature is enabled (`celeborn.client.spark.push.sort.pipeline.enabled=true`), the SortBasedPusher will trigger a data push when the memory usage exceeds half of the threshold(by default, 32m). | 0.3.0 | | celeborn.client.spark.push.sort.pipeline.enabled | false | Whether to enable pipelining for sort based shuffle writer. If true, double buffering will be used to pipeline push | 0.3.0 | | celeborn.client.spark.push.unsafeRow.fastWrite.enabled | true | This is Celeborn's optimization on UnsafeRow for Spark and it's true by default. If you have changed UnsafeRow's memory layout set this to false. | 0.2.2 | diff --git a/tests/spark-it/pom.xml b/tests/spark-it/pom.xml index 783cbe1002a..75e50221b2c 100644 --- a/tests/spark-it/pom.xml +++ b/tests/spark-it/pom.xml @@ -93,6 +93,13 @@ ${project.version} test + + org.apache.celeborn + celeborn-client-spark-2_${scala.binary.version} + ${project.version} + test-jar + test + @@ -104,6 +111,13 @@ ${project.version} test + + org.apache.celeborn + celeborn-client-spark-3_${scala.binary.version} + ${project.version} + test-jar + test + @@ -115,6 +129,13 @@ ${project.version} test + + org.apache.celeborn + celeborn-client-spark-3_${scala.binary.version} + ${project.version} + test-jar + test + @@ -126,6 +147,13 @@ ${project.version} test + + org.apache.celeborn + celeborn-client-spark-3_${scala.binary.version} + ${project.version} + test-jar + test + @@ -137,6 +165,13 @@ ${project.version} test + + org.apache.celeborn + celeborn-client-spark-3_${scala.binary.version} + ${project.version} + test-jar + test + @@ -148,6 +183,13 @@ ${project.version} test + + org.apache.celeborn + celeborn-client-spark-3_${scala.binary.version} + ${project.version} + test-jar + test + @@ -159,6 +201,13 @@ ${project.version} test + + org.apache.celeborn + celeborn-client-spark-3_${scala.binary.version} + ${project.version} + test-jar + test + diff --git a/tests/spark-it/src/test/scala/org/apache/celeborn/tests/spark/CelebornFetchFailureSuite.scala b/tests/spark-it/src/test/scala/org/apache/celeborn/tests/spark/CelebornFetchFailureSuite.scala new file mode 100644 index 00000000000..07c6b35eb5a --- /dev/null +++ b/tests/spark-it/src/test/scala/org/apache/celeborn/tests/spark/CelebornFetchFailureSuite.scala @@ -0,0 +1,223 @@ +/* + * 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.tests.spark + +import java.io.File +import java.util.concurrent.atomic.AtomicBoolean + +import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.shuffle.ShuffleHandle +import org.apache.spark.shuffle.celeborn.{CelebornShuffleHandle, ShuffleManagerHook, SparkUtils, TestCelebornShuffleManager} +import org.apache.spark.sql.SparkSession +import org.scalatest.BeforeAndAfterEach +import org.scalatest.funsuite.AnyFunSuite + +import org.apache.celeborn.client.ShuffleClient +import org.apache.celeborn.common.CelebornConf +import org.apache.celeborn.common.protocol.ShuffleMode +import org.apache.celeborn.service.deploy.worker.Worker + +class CelebornFetchFailureSuite extends AnyFunSuite + with SparkTestBase + with BeforeAndAfterEach { + + override def beforeEach(): Unit = { + ShuffleClient.reset() + } + + override def afterEach(): Unit = { + System.gc() + } + + var workerDirs: Seq[String] = Seq.empty + + override def createWorker(map: Map[String, String]): Worker = { + val storageDir = createTmpDir() + workerDirs = workerDirs :+ storageDir + super.createWorker(map, storageDir) + } + + class ShuffleReaderGetHook(conf: CelebornConf) extends ShuffleManagerHook { + var executed: AtomicBoolean = new AtomicBoolean(false) + val lock = new Object + + override def exec( + handle: ShuffleHandle, + startPartition: Int, + endPartition: Int, + context: TaskContext): Unit = { + if (executed.get() == true) return + + lock.synchronized { + handle match { + case h: CelebornShuffleHandle[_, _, _] => { + val appUniqueId = h.appUniqueId + val shuffleClient = ShuffleClient.get( + h.appUniqueId, + h.lifecycleManagerHost, + h.lifecycleManagerPort, + conf, + h.userIdentifier) + val celebornShuffleId = SparkUtils.celebornShuffleId(shuffleClient, h, context, false) + val datafile = + workerDirs.map(dir => { + new File(s"$dir/celeborn-worker/shuffle_data/$appUniqueId/$celebornShuffleId") + }).filter(_.exists()) + .flatMap(_.listFiles().iterator).headOption + datafile match { + case Some(file) => file.delete() + case None => throw new RuntimeException("unexpected, there must be some data file") + } + } + case _ => throw new RuntimeException("unexpected, only support RssShuffleHandle here") + } + executed.set(true) + } + } + } + + test("celeborn spark integration test - Fetch Failure") { + val sparkConf = new SparkConf().setAppName("rss-demo").setMaster("local[2,3]") + val sparkSession = SparkSession.builder() + .config(updateSparkConf(sparkConf, ShuffleMode.HASH)) + .config("spark.sql.shuffle.partitions", 2) + .config("spark.celeborn.shuffle.forceFallback.partition.enabled", false) + .config("spark.celeborn.shuffle.enabled", "true") + .config("spark.celeborn.client.spark.fetch.throwsFetchFailure", "true") + .config( + "spark.shuffle.manager", + "org.apache.spark.shuffle.celeborn.TestCelebornShuffleManager") + .getOrCreate() + + val celebornConf = SparkUtils.fromSparkConf(sparkSession.sparkContext.getConf) + val hook = new ShuffleReaderGetHook(celebornConf) + TestCelebornShuffleManager.registerReaderGetHook(hook) + + val value = Range(1, 10000).mkString(",") + val tuples = sparkSession.sparkContext.parallelize(1 to 10000, 2) + .map { i => (i, value) }.groupByKey(16).collect() + + // verify result + assert(hook.executed.get() == true) + assert(tuples.length == 10000) + for (elem <- tuples) { + assert(elem._2.mkString(",").equals(value)) + } + + sparkSession.stop() + } + + test("celeborn spark integration test - Fetch Failure with multiple shuffle data") { + val sparkConf = new SparkConf().setAppName("rss-demo").setMaster("local[2,3]") + val sparkSession = SparkSession.builder() + .config(updateSparkConf(sparkConf, ShuffleMode.HASH)) + .config("spark.sql.shuffle.partitions", 2) + .config("spark.celeborn.shuffle.forceFallback.partition.enabled", false) + .config("spark.celeborn.shuffle.enabled", "true") + .config("spark.celeborn.client.spark.fetch.throwsFetchFailure", "true") + .config( + "spark.shuffle.manager", + "org.apache.spark.shuffle.celeborn.TestCelebornShuffleManager") + .getOrCreate() + + val celebornConf = SparkUtils.fromSparkConf(sparkSession.sparkContext.getConf) + val hook = new ShuffleReaderGetHook(celebornConf) + TestCelebornShuffleManager.registerReaderGetHook(hook) + + import sparkSession.implicits._ + + val df1 = Seq((1, "a"), (2, "b")).toDF("id", "data").groupBy("id").count() + val df2 = Seq((2, "c"), (2, "d")).toDF("id", "data").groupBy("id").count() + val tuples = df1.hint("merge").join(df2, "id").select("*").collect() + + // verify result + assert(hook.executed.get() == true) + val expect = "[2,1,2]" + assert(tuples.head.toString().equals(expect)) + sparkSession.stop() + } + + test("celeborn spark integration test - Fetch Failure with RDD reuse") { + val sparkConf = new SparkConf().setAppName("rss-demo").setMaster("local[2,3]") + val sparkSession = SparkSession.builder() + .config(updateSparkConf(sparkConf, ShuffleMode.HASH)) + .config("spark.sql.shuffle.partitions", 2) + .config("spark.celeborn.shuffle.forceFallback.partition.enabled", false) + .config("spark.celeborn.shuffle.enabled", "true") + .config("spark.celeborn.client.spark.fetch.throwsFetchFailure", "true") + .config( + "spark.shuffle.manager", + "org.apache.spark.shuffle.celeborn.TestCelebornShuffleManager") + .getOrCreate() + + val celebornConf = SparkUtils.fromSparkConf(sparkSession.sparkContext.getConf) + val hook = new ShuffleReaderGetHook(celebornConf) + TestCelebornShuffleManager.registerReaderGetHook(hook) + + val sc = sparkSession.sparkContext + val rdd1 = sc.parallelize(0 until 10000, 3).map(v => (v, v)).groupByKey() + val rdd2 = sc.parallelize(0 until 10000, 2).map(v => (v, v)).groupByKey() + val rdd3 = rdd1.map(v => (v._2, v._1)) + + hook.executed.set(true) + + rdd1.count() + rdd2.count() + + hook.executed.set(false) + rdd3.count() + hook.executed.set(false) + rdd3.count() + hook.executed.set(false) + rdd3.count() + hook.executed.set(false) + rdd3.count() + + sparkSession.stop() + } + + test("celeborn spark integration test - Fetch Failure with read write shuffles in one stage") { + val sparkConf = new SparkConf().setAppName("rss-demo").setMaster("local[2,3]") + val sparkSession = SparkSession.builder() + .config(updateSparkConf(sparkConf, ShuffleMode.HASH)) + .config("spark.sql.shuffle.partitions", 2) + .config("spark.celeborn.shuffle.forceFallback.partition.enabled", false) + .config("spark.celeborn.shuffle.enabled", "true") + .config("spark.celeborn.client.spark.fetch.throwsFetchFailure", "true") + .config( + "spark.shuffle.manager", + "org.apache.spark.shuffle.celeborn.TestCelebornShuffleManager") + .getOrCreate() + + val celebornConf = SparkUtils.fromSparkConf(sparkSession.sparkContext.getConf) + val hook = new ShuffleReaderGetHook(celebornConf) + TestCelebornShuffleManager.registerReaderGetHook(hook) + + val sc = sparkSession.sparkContext + val rdd1 = sc.parallelize(0 until 10000, 3).map(v => (v, v)).groupByKey() + val rdd2 = rdd1.map(v => (v._2, v._1)).groupByKey() + + hook.executed.set(true) + rdd1.count() + + hook.executed.set(false) + rdd2.count() + + sparkSession.stop() + } +} diff --git a/worker/src/test/scala/org/apache/celeborn/service/deploy/MiniClusterFeature.scala b/worker/src/test/scala/org/apache/celeborn/service/deploy/MiniClusterFeature.scala index f466eb9f197..8c8411910d3 100644 --- a/worker/src/test/scala/org/apache/celeborn/service/deploy/MiniClusterFeature.scala +++ b/worker/src/test/scala/org/apache/celeborn/service/deploy/MiniClusterFeature.scala @@ -35,13 +35,13 @@ trait MiniClusterFeature extends Logging { var masterInfo: (Master, Thread) = _ val workerInfos = new mutable.HashMap[Worker, Thread]() - private def runnerWrap[T](code: => T): Thread = new Thread(new Runnable { + def runnerWrap[T](code: => T): Thread = new Thread(new Runnable { override def run(): Unit = { Utils.tryLogNonFatalError(code) } }) - private def createTmpDir(): String = { + def createTmpDir(): String = { val tmpDir = Files.createTempDirectory("celeborn-") logInfo(s"created temp dir: $tmpDir") tmpDir.toFile.deleteOnExit() @@ -66,10 +66,14 @@ trait MiniClusterFeature extends Logging { master } - private def createWorker(map: Map[String, String] = null): Worker = { + def createWorker(map: Map[String, String] = null): Worker = { + createWorker(map, createTmpDir()) + } + + def createWorker(map: Map[String, String], storageDir: String): Worker = { logInfo("start create worker for mini cluster") val conf = new CelebornConf() - conf.set(CelebornConf.WORKER_STORAGE_DIRS.key, createTmpDir()) + conf.set(CelebornConf.WORKER_STORAGE_DIRS.key, storageDir) conf.set(CelebornConf.WORKER_DISK_MONITOR_ENABLED.key, "false") conf.set(CelebornConf.CLIENT_PUSH_BUFFER_MAX_SIZE.key, "256K") conf.set(CelebornConf.WORKER_HTTP_PORT.key, s"${workerHttpPort.incrementAndGet()}") From ad57c8b91ed8cdbaf9d7371f0d93ad75e861f14f Mon Sep 17 00:00:00 2001 From: Shuang Date: Mon, 27 Nov 2023 12:17:05 +0800 Subject: [PATCH 22/31] [CELEBORN-1052] Introduce dynamic ConfigService at SystemLevel and TenantLevel ### What changes were proposed in this pull request? This PR introduce dynamic ConfigService at SystemLevel and TenantLevel, Dynamic configuration is a type of configuration that can be changed at runtime as needed. It can be used at system level/tenant level. When applying dynamic configuration, the priority order is as follows: tenant level overrides system level, which in turn overrides static configuration(CelebornConf). This means that if a configuration is defined at the tenant level, it will be used instead of the system level or static configuration(CelebornConf). If the tenant-level configuration is missing, the system-level configuration will be used. If the system-level configuration is also missing, CelebornConf will be used as the default value. There are several other tasks related to this feature that will be implemented in the future. - [ ] [Add isDynamic property for CelebornConf](https://issues.apache.org/jira/browse/CELEBORN-1051) - [ ] [Support DB based Configserver](https://issues.apache.org/jira/browse/CELEBORN-1054) - [ ] [Add restAPI for configuration management](https://issues.apache.org/jira/browse/CELEBORN-1056) ### Why are the changes needed? The current configuration of the server (CelebornConf) is static. When the configuration is changed, the service needs to be restarted. This PR introduces a dynamic configuration solution. The server side can use dynamic configuration as needed. At the same time, it is considered that the tenant level will be supported in the future (such as supporting tenant level dynamic quota control) configuration, so this time we will also consider supporting dynamic tenant-level configuration, and this PR will provide a default implementation based on the file system. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? UT Closes #2100 from RexXiong/CELEBORN-1052. Authored-by: Shuang Signed-off-by: mingji --- .../apache/celeborn/common/CelebornConf.scala | 20 +++ conf/dynamicConfig.yaml.template | 27 ++++ docs/configuration/master.md | 2 + docs/configuration/worker.md | 2 + .../common/service/config/ConfigLevel.java | 23 ++++ .../common/service/config/ConfigService.java | 39 ++++++ .../common/service/config/DynamicConfig.java | 117 ++++++++++++++++ .../config/DynamicConfigServiceFactory.java | 32 +++++ .../service/config/FsConfigServiceImpl.java | 127 ++++++++++++++++++ .../common/service/config/SystemConfig.java | 52 +++++++ .../common/service/config/TenantConfig.java | 40 ++++++ service/src/test/resources/dynamicConfig.yaml | 36 +++++ .../src/test/resources/dynamicConfig_2.yaml | 20 +++ .../service/config/ConfigServiceSuiteJ.java | 123 +++++++++++++++++ 14 files changed, 660 insertions(+) create mode 100644 conf/dynamicConfig.yaml.template create mode 100644 service/src/main/scala/org/apache/celeborn/server/common/service/config/ConfigLevel.java create mode 100644 service/src/main/scala/org/apache/celeborn/server/common/service/config/ConfigService.java create mode 100644 service/src/main/scala/org/apache/celeborn/server/common/service/config/DynamicConfig.java create mode 100644 service/src/main/scala/org/apache/celeborn/server/common/service/config/DynamicConfigServiceFactory.java create mode 100644 service/src/main/scala/org/apache/celeborn/server/common/service/config/FsConfigServiceImpl.java create mode 100644 service/src/main/scala/org/apache/celeborn/server/common/service/config/SystemConfig.java create mode 100644 service/src/main/scala/org/apache/celeborn/server/common/service/config/TenantConfig.java create mode 100644 service/src/test/resources/dynamicConfig.yaml create mode 100644 service/src/test/resources/dynamicConfig_2.yaml create mode 100644 service/src/test/scala/org/apache/celeborn/server/common/service/config/ConfigServiceSuiteJ.java diff --git a/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala b/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala index 5b170eb3033..247727df210 100644 --- a/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala +++ b/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala @@ -366,6 +366,9 @@ class CelebornConf(loadDefaults: Boolean) extends Cloneable with Logging with Se } } + def dynamicConfigStoreBackend: String = get(DYNAMIC_CONFIG_STORE_BACKEND) + def dynamicConfigRefreshTime: Long = get(DYNAMIC_CONFIG_REFRESH_TIME) + // ////////////////////////////////////////////////////// // Network // // ////////////////////////////////////////////////////// @@ -4062,4 +4065,21 @@ object CelebornConf extends Logging { .doc("Kerberos keytab file path for HDFS storage connection.") .stringConf .createOptional + + val DYNAMIC_CONFIG_STORE_BACKEND: ConfigEntry[String] = + buildConf("celeborn.dynamicConfig.store.backend") + .categories("master", "worker") + .doc("Store backend for dynamic config, NONE means disabling dynamic config store") + .version("0.4.0") + .stringConf + .checkValues(Set("FS", "NONE")) + .createWithDefault("NONE") + + val DYNAMIC_CONFIG_REFRESH_TIME: ConfigEntry[Long] = + buildConf("celeborn.dynamicConfig.refresh.time") + .categories("master", "worker") + .version("0.4.0") + .doc("The time interval for refreshing the corresponding dynamic config periodically") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("120s") } diff --git a/conf/dynamicConfig.yaml.template b/conf/dynamicConfig.yaml.template new file mode 100644 index 00000000000..87789754bb3 --- /dev/null +++ b/conf/dynamicConfig.yaml.template @@ -0,0 +1,27 @@ +# +# 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. +# +- level: SYSTEM + config: + celeborn.worker.directMemoryRatioToPauseReceive: 0.75 + + +- tenantId: tenant_id + level: TENANT + config: + + + diff --git a/docs/configuration/master.md b/docs/configuration/master.md index c21a76edde5..025e9c15b3c 100644 --- a/docs/configuration/master.md +++ b/docs/configuration/master.md @@ -19,6 +19,8 @@ license: | | Key | Default | Description | Since | | --- | ------- | ----------- | ----- | +| celeborn.dynamicConfig.refresh.time | 120s | The time interval for refreshing the corresponding dynamic config periodically | 0.4.0 | +| celeborn.dynamicConfig.store.backend | NONE | Store backend for dynamic config, NONE means disabling dynamic config store | 0.4.0 | | celeborn.master.estimatedPartitionSize.initialSize | 64mb | Initial partition size for estimation, it will change according to runtime stats. | 0.3.0 | | celeborn.master.estimatedPartitionSize.update.initialDelay | 5min | Initial delay time before start updating partition size for estimation. | 0.3.0 | | celeborn.master.estimatedPartitionSize.update.interval | 10min | Interval of updating partition size for estimation. | 0.3.0 | diff --git a/docs/configuration/worker.md b/docs/configuration/worker.md index 1570d124991..a40cc9c5405 100644 --- a/docs/configuration/worker.md +++ b/docs/configuration/worker.md @@ -19,6 +19,8 @@ license: | | Key | Default | Description | Since | | --- | ------- | ----------- | ----- | +| celeborn.dynamicConfig.refresh.time | 120s | The time interval for refreshing the corresponding dynamic config periodically | 0.4.0 | +| celeborn.dynamicConfig.store.backend | NONE | Store backend for dynamic config, NONE means disabling dynamic config store | 0.4.0 | | celeborn.master.endpoints | <localhost>:9097 | Endpoints of master nodes for celeborn client to connect, allowed pattern is: `:[,:]*`, e.g. `clb1:9097,clb2:9098,clb3:9099`. If the port is omitted, 9097 will be used. | 0.2.0 | | celeborn.master.estimatedPartitionSize.minSize | 8mb | Ignore partition size smaller than this configuration of partition size for estimation. | 0.3.0 | | celeborn.shuffle.chunk.size | 8m | Max chunk size of reducer's merged shuffle data. For example, if a reducer's shuffle data is 128M and the data will need 16 fetch chunk requests to fetch. | 0.2.0 | diff --git a/service/src/main/scala/org/apache/celeborn/server/common/service/config/ConfigLevel.java b/service/src/main/scala/org/apache/celeborn/server/common/service/config/ConfigLevel.java new file mode 100644 index 00000000000..12182146187 --- /dev/null +++ b/service/src/main/scala/org/apache/celeborn/server/common/service/config/ConfigLevel.java @@ -0,0 +1,23 @@ +/* + * 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.server.common.service.config; + +public enum ConfigLevel { + SYSTEM, + TENANT, +} diff --git a/service/src/main/scala/org/apache/celeborn/server/common/service/config/ConfigService.java b/service/src/main/scala/org/apache/celeborn/server/common/service/config/ConfigService.java new file mode 100644 index 00000000000..362d2b71e40 --- /dev/null +++ b/service/src/main/scala/org/apache/celeborn/server/common/service/config/ConfigService.java @@ -0,0 +1,39 @@ +/* + * 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.server.common.service.config; + +public interface ConfigService { + + SystemConfig getSystemConfig(); + + TenantConfig getRawTenantConfig(String tenantId); + + default DynamicConfig getTenantConfig(String tenantId) { + TenantConfig tenantConfig = getRawTenantConfig(tenantId); + if (tenantConfig == null || tenantConfig.getConfigs().isEmpty()) { + return getSystemConfig(); + } else { + return tenantConfig; + } + } + + void refreshAllCache(); + + void shutdown(); + +} \ No newline at end of file diff --git a/service/src/main/scala/org/apache/celeborn/server/common/service/config/DynamicConfig.java b/service/src/main/scala/org/apache/celeborn/server/common/service/config/DynamicConfig.java new file mode 100644 index 00000000000..9a051eea7c8 --- /dev/null +++ b/service/src/main/scala/org/apache/celeborn/server/common/service/config/DynamicConfig.java @@ -0,0 +1,117 @@ +/* + * 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.server.common.service.config; + +import org.apache.celeborn.common.internal.config.ConfigEntry; +import org.apache.celeborn.common.util.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +/** + * Dynamic configuration is a type of configuration that can be changed at runtime as needed. It can be used at system level/tenant level. + * When applying dynamic configuration, the priority order is as follows: tenant level overrides system level, + * which in turn overrides static configuration(CelebornConf). This means that if a configuration is defined at the tenant level, + * it will be used instead of the system level or static configuration(CelebornConf). If the tenant-level configuration is missing, + * the system-level configuration will be used. If the system-level configuration is also missing, CelebornConf + * will be used as the default value. + */ +public abstract class DynamicConfig { + private static final Logger LOG = LoggerFactory.getLogger(DynamicConfig.class); + protected Map configs = new HashMap<>(); + + public abstract DynamicConfig getParentLevelConfig(); + + public T getWithDefaultValue( + String configKey, T defaultValue, Class finalType, ConfigType configType) { + String configValue = configs.get(configKey); + T formatValue = configValue != null ? formatValue(configKey, configValue, finalType, configType) : null; + if (formatValue == null) { + return defaultValue; + } else { + return formatValue; + } + } + + public T getValue(String configKey, ConfigEntry configEntry, Class finalType, ConfigType configType) { + String configValue = configs.get(configKey); + T formatValue = configValue != null ? formatValue(configKey, configValue, finalType, configType) : null; + if (formatValue == null) { + DynamicConfig parentLevelConfig = getParentLevelConfig(); + return parentLevelConfig != null? parentLevelConfig.getValue(configKey, configEntry, finalType, configType): null; + } else { + return formatValue; + } + } + + public T formatValue(String configKey, String configValue, Class finalType, ConfigType configType) { + try { + if (configValue != null) { + if (ConfigType.BYTES == configType) { + return convert(finalType, String.valueOf(Utils.byteStringAsBytes(configValue))); + } else if (ConfigType.TIME_MS == configType) { + return convert(finalType, String.valueOf(Utils.timeStringAsMs(configValue))); + } else { + return convert(finalType, configValue); + } + } + } catch (Exception e) { + LOG.warn("Config {} value format is not valid, refer to parent if exist", configKey, e); + } + return null; + } + + public Map getConfigs() { + return configs; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("DynamicConfig{"); + sb.append("configs=").append(configs); + sb.append('}'); + return sb.toString(); + } + + public enum ConfigType { + BYTES, + STRING, + TIME_MS, + } + + public static T convert(Class clazz, String value) { + if (Boolean.TYPE == clazz) { + return (T) Boolean.valueOf(value); + } else if (Byte.TYPE == clazz) { + return (T) Byte.valueOf(value); + } else if (Short.TYPE == clazz) { + return (T) Short.valueOf(value); + } else if (Integer.TYPE == clazz) { + return (T) Integer.valueOf(value); + } else if (Long.TYPE == clazz) { + return (T) Long.valueOf(value); + } else if (Float.TYPE == clazz) { + return (T) Float.valueOf(value); + } else if (Double.TYPE == clazz) { + return (T) Double.valueOf(value); + } + return (T) value; + } +} diff --git a/service/src/main/scala/org/apache/celeborn/server/common/service/config/DynamicConfigServiceFactory.java b/service/src/main/scala/org/apache/celeborn/server/common/service/config/DynamicConfigServiceFactory.java new file mode 100644 index 00000000000..7346ddc20f9 --- /dev/null +++ b/service/src/main/scala/org/apache/celeborn/server/common/service/config/DynamicConfigServiceFactory.java @@ -0,0 +1,32 @@ +/* + * 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.server.common.service.config; + +import org.apache.celeborn.common.CelebornConf; + +public class DynamicConfigServiceFactory { + + public static ConfigService getConfigService(CelebornConf celebornConf) { + String configStoreBackend = celebornConf.dynamicConfigStoreBackend(); + if ("FS".equals(configStoreBackend)) { + return new FsConfigServiceImpl(celebornConf); + } + + return null; + } +} \ No newline at end of file diff --git a/service/src/main/scala/org/apache/celeborn/server/common/service/config/FsConfigServiceImpl.java b/service/src/main/scala/org/apache/celeborn/server/common/service/config/FsConfigServiceImpl.java new file mode 100644 index 00000000000..7a9b60a88b7 --- /dev/null +++ b/service/src/main/scala/org/apache/celeborn/server/common/service/config/FsConfigServiceImpl.java @@ -0,0 +1,127 @@ +/* + * 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.server.common.service.config; + +import org.apache.celeborn.common.util.ThreadUtils; + +import java.io.File; +import java.io.FileInputStream; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.yaml.snakeyaml.Yaml; + +import org.apache.celeborn.common.CelebornConf; + +import scala.concurrent.duration.Duration; + +public class FsConfigServiceImpl implements ConfigService { + private static final Logger LOG = LoggerFactory.getLogger(FsConfigServiceImpl.class); + private CelebornConf celebornConf; + private final AtomicReference systemConfigAtomicReference = new AtomicReference<>(); + private final AtomicReference> tenantConfigAtomicReference = new AtomicReference<>(new HashMap<>()); + private static final String CONF_TENANT_ID = "tenantId"; + private static final String CONF_LEVEL = "level"; + private static final String CONF_CONFIG = "config"; + + private final ScheduledExecutorService configRefreshService = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("config-refresh-service"); + + public FsConfigServiceImpl(CelebornConf celebornConf) { + this.celebornConf = celebornConf; + this.refresh(); + long dynamicConfigRefreshTime = celebornConf.dynamicConfigRefreshTime(); + this.configRefreshService.scheduleWithFixedDelay( + () -> refresh(), + dynamicConfigRefreshTime, + dynamicConfigRefreshTime, + TimeUnit.MILLISECONDS); + } + + private synchronized void refresh() { + File configurationFile = getConfigurationFile(System.getenv()); + if (!configurationFile.exists()) { + return; + } + + SystemConfig systemConfig = null; + Map tenantConfs = new HashMap<>(); + try (FileInputStream fileInputStream = new FileInputStream(configurationFile)) { + Yaml yaml = new Yaml(); + List> dynamicConfigs = yaml.load(fileInputStream); + for (Map settings : dynamicConfigs) { + String tenantId = (String) settings.get(CONF_TENANT_ID); + String level = (String) settings.get(CONF_LEVEL); + Map config = + ((Map) settings.get(CONF_CONFIG)) + .entrySet().stream() + .collect(Collectors.toMap(a -> a.getKey(), a -> a.getValue().toString())); + if (ConfigLevel.TENANT.name().equals(level)) { + TenantConfig tenantConfig = new TenantConfig(this, tenantId, config); + tenantConfs.put(tenantId, tenantConfig); + } else { + systemConfig = new SystemConfig(celebornConf, config); + } + } + } catch (Exception e) { + LOG.warn("Refresh dynamic config error: {}", e.getMessage(), e); + } + + tenantConfigAtomicReference.set(tenantConfs); + systemConfigAtomicReference.set(systemConfig == null ? new SystemConfig(celebornConf) : systemConfig); + } + + @Override + public SystemConfig getSystemConfig() { + return systemConfigAtomicReference.get(); + } + + @Override + public TenantConfig getRawTenantConfig(String tenantId) { + return tenantConfigAtomicReference.get().get(tenantId); + } + + @Override + public void refreshAllCache() { + this.refresh(); + } + + @Override + public void shutdown() { + ThreadUtils.shutdown(configRefreshService, Duration.apply("800ms")); + } + + private File getConfigurationFile(Map env) { + if (!this.celebornConf.quotaConfigurationPath().isEmpty()) { + return new File(this.celebornConf.quotaConfigurationPath().get()); + } else { + String dynamicConfPath = + Optional.ofNullable(env.get("CELEBORN_CONF_DIR")) + .orElse(env.getOrDefault("CELEBORN_HOME", ".") + File.separator + "conf"); + return new File(dynamicConfPath + File.separator + "dynamicConfig.yaml"); + } + } +} diff --git a/service/src/main/scala/org/apache/celeborn/server/common/service/config/SystemConfig.java b/service/src/main/scala/org/apache/celeborn/server/common/service/config/SystemConfig.java new file mode 100644 index 00000000000..ff73bb7f2c2 --- /dev/null +++ b/service/src/main/scala/org/apache/celeborn/server/common/service/config/SystemConfig.java @@ -0,0 +1,52 @@ +/* + * 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.server.common.service.config; + +import org.apache.celeborn.common.CelebornConf; +import org.apache.celeborn.common.internal.config.ConfigEntry; + +import java.util.HashMap; +import java.util.Map; + +public class SystemConfig extends DynamicConfig { + private CelebornConf celebornConf; + public SystemConfig(CelebornConf celebornConf, Map configs) { + this.celebornConf = celebornConf; + this.configs.putAll(configs); + } + + public SystemConfig(CelebornConf celebornConf) { + this.celebornConf = celebornConf; + this.configs = new HashMap<>(); + } + + @Override + public DynamicConfig getParentLevelConfig() { + return null; + } + + public T getValue(String configKey, ConfigEntry configEntry, Class finalType, ConfigType configType) { + String configValue = configs.get(configKey); + T formatValue = configValue != null ? formatValue(configKey, configValue, finalType, configType) : null; + if (formatValue == null && configEntry != null) { + return convert(finalType, celebornConf.get(configEntry).toString()); + } else { + return formatValue; + } + } +} diff --git a/service/src/main/scala/org/apache/celeborn/server/common/service/config/TenantConfig.java b/service/src/main/scala/org/apache/celeborn/server/common/service/config/TenantConfig.java new file mode 100644 index 00000000000..26198d6e0b0 --- /dev/null +++ b/service/src/main/scala/org/apache/celeborn/server/common/service/config/TenantConfig.java @@ -0,0 +1,40 @@ +/* + * 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.server.common.service.config; + +import java.util.Map; + +public class TenantConfig extends DynamicConfig { + private String tenantId; + private ConfigService configService; + + public TenantConfig(ConfigService configService, String tenantId, Map configs) { + this.configService = configService; + this.configs.putAll(configs); + this.tenantId = tenantId; + } + + public Map getConfigs() { + return configs; + } + + @Override + public DynamicConfig getParentLevelConfig() { + return configService.getSystemConfig(); + } +} diff --git a/service/src/test/resources/dynamicConfig.yaml b/service/src/test/resources/dynamicConfig.yaml new file mode 100644 index 00000000000..22d17ee6245 --- /dev/null +++ b/service/src/test/resources/dynamicConfig.yaml @@ -0,0 +1,36 @@ +# +# 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. +# +- level: SYSTEM + config: + celeborn.client.push.buffer.initial.size: 100k + celeborn.client.push.buffer.max.size: 1000k + celeborn.worker.fetch.heartbeat.enabled: true + celeborn.client.push.buffer.initial.size.only: 10k + celeborn.test.timeoutMs.only: 100s + celeborn.test.enabled.only: false + celeborn.test.int.only: 10 + +- tenantId: tenant_id + level: TENANT + config: + celeborn.client.push.buffer.initial.size: 10k + celeborn.client.push.buffer.initial.size.only: 100k + celeborn.worker.fetch.heartbeat.enabled: false + celeborn.test.tenant.timeoutMs.only: 100s + celeborn.test.tenant.enabled.only: false + celeborn.test.tenant.int.only: 10 + diff --git a/service/src/test/resources/dynamicConfig_2.yaml b/service/src/test/resources/dynamicConfig_2.yaml new file mode 100644 index 00000000000..57645e0b839 --- /dev/null +++ b/service/src/test/resources/dynamicConfig_2.yaml @@ -0,0 +1,20 @@ +# +# 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. +# +- level: SYSTEM + config: + celeborn.test.int.only: 100 + diff --git a/service/src/test/scala/org/apache/celeborn/server/common/service/config/ConfigServiceSuiteJ.java b/service/src/test/scala/org/apache/celeborn/server/common/service/config/ConfigServiceSuiteJ.java new file mode 100644 index 00000000000..47c41b01171 --- /dev/null +++ b/service/src/test/scala/org/apache/celeborn/server/common/service/config/ConfigServiceSuiteJ.java @@ -0,0 +1,123 @@ +/* + * 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.server.common.service.config; + +import org.apache.celeborn.common.CelebornConf; +import org.apache.celeborn.server.common.service.config.ConfigService; +import org.apache.celeborn.server.common.service.config.DynamicConfig; +import org.apache.celeborn.server.common.service.config.DynamicConfig.ConfigType; +import org.apache.celeborn.server.common.service.config.FsConfigServiceImpl; +import org.apache.celeborn.server.common.service.config.SystemConfig; +import org.junit.Assert; +import org.junit.Test; + +public class ConfigServiceSuiteJ { + + @Test + public void testFsConfig() { + CelebornConf celebornConf = new CelebornConf(); + String file = getClass().getResource("/dynamicConfig.yaml").getFile(); + celebornConf.set(CelebornConf.QUOTA_CONFIGURATION_PATH(), file); + celebornConf.set(CelebornConf.DYNAMIC_CONFIG_REFRESH_TIME(), 5l); + FsConfigServiceImpl fsConfigService = new FsConfigServiceImpl(celebornConf); + + verifyConfig(fsConfigService); + + // change -> refresh config + file = getClass().getResource("/dynamicConfig_2.yaml").getFile(); + celebornConf.set(CelebornConf.QUOTA_CONFIGURATION_PATH(), file); + + fsConfigService.refreshAllCache(); + SystemConfig systemConfig = fsConfigService.getSystemConfig(); + + // verify systemConfig's intConf + Integer intConfValue = systemConfig.getValue("celeborn.test.int.only", null, Integer.TYPE, ConfigType.STRING); + Assert.assertEquals(intConfValue.intValue(), 100); + + // verify systemConfig's bytesConf -- defer to celebornConf + Long value = systemConfig.getValue(CelebornConf.SHUFFLE_PARTITION_SPLIT_THRESHOLD().key(), CelebornConf.SHUFFLE_PARTITION_SPLIT_THRESHOLD(), Long.TYPE, ConfigType.BYTES); + Assert.assertEquals(value.longValue(), 1073741824); + } + + public void verifyConfig(ConfigService configService) { + // ------------- Verify SystemConfig ----------------- // + SystemConfig systemConfig = configService.getSystemConfig(); + // verify systemConfig's bytesConf -- use systemConfig + Long value = systemConfig.getValue(CelebornConf.CLIENT_PUSH_BUFFER_INITIAL_SIZE().key(), CelebornConf.CLIENT_PUSH_BUFFER_INITIAL_SIZE(), Long.TYPE, ConfigType.BYTES); + Assert.assertEquals(value.longValue(), 102400); + + // verify systemConfig's bytesConf -- defer to celebornConf + value = systemConfig.getValue(CelebornConf.SHUFFLE_PARTITION_SPLIT_THRESHOLD().key(), CelebornConf.SHUFFLE_PARTITION_SPLIT_THRESHOLD(), Long.TYPE, ConfigType.BYTES); + Assert.assertEquals(value.longValue(), 1073741824); + + // verify systemConfig's bytesConf only -- use systemConfig + value = systemConfig.getValue("celeborn.client.push.buffer.initial.size.only", null, Long.TYPE, ConfigType.BYTES); + Assert.assertEquals(value.longValue(), 10240); + + // verify systemConfig's bytesConf with none + value = systemConfig.getValue("celeborn.client.push.buffer.initial.size.only.none", null, Long.TYPE, ConfigType.BYTES); + Assert.assertNull(value); + + // verify systemConfig's timesConf + value = systemConfig.getValue("celeborn.test.timeoutMs.only", null, Long.TYPE, ConfigType.TIME_MS); + Assert.assertEquals(value.longValue(), 100000); + + // verify systemConfig's BooleanConf + Boolean booleanConfValue = systemConfig.getValue("celeborn.test.timeoutMs.only", null, Boolean.TYPE, ConfigType.STRING); + Assert.assertFalse(booleanConfValue); + + // verify systemConfig's intConf + Integer intConfValue = systemConfig.getValue("celeborn.test.int.only", null, Integer.TYPE, ConfigType.STRING); + Assert.assertEquals(intConfValue.intValue(), 10); + + // ------------- Verify TenantConfig ----------------- // + DynamicConfig tenantConfig = configService.getTenantConfig("tenant_id"); + // verify tenantConfig's bytesConf -- use tenantConf + value = tenantConfig.getValue(CelebornConf.CLIENT_PUSH_BUFFER_INITIAL_SIZE().key(), CelebornConf.CLIENT_PUSH_BUFFER_INITIAL_SIZE(), Long.TYPE, ConfigType.BYTES); + Assert.assertEquals(value.longValue(), 10240); + + // verify tenantConfig's bytesConf -- defer to systemConf + value = tenantConfig.getValue(CelebornConf.CLIENT_PUSH_BUFFER_MAX_SIZE().key(), CelebornConf.CLIENT_PUSH_BUFFER_MAX_SIZE(), Long.TYPE, ConfigType.BYTES); + Assert.assertEquals(value.longValue(), 1024000); + + // verify tenantConfig's bytesConf -- defer to celebornConf + value = tenantConfig.getValue(CelebornConf.SHUFFLE_PARTITION_SPLIT_THRESHOLD().key(), CelebornConf.SHUFFLE_PARTITION_SPLIT_THRESHOLD(), Long.TYPE, ConfigType.BYTES); + Assert.assertEquals(value.longValue(), 1073741824); + + // verify tenantConfig's bytesConf only -- use tenantConf + value = tenantConfig.getValue("celeborn.client.push.buffer.initial.size.only", null, Long.TYPE, ConfigType.BYTES); + Assert.assertEquals(value.longValue(), 102400); + + // verify tenantConfig's bytesConf with none + value = tenantConfig.getValue("celeborn.client.push.buffer.initial.size.only.none", null, Long.TYPE, ConfigType.BYTES); + Assert.assertNull(value); + + DynamicConfig tenantConfigNone = configService.getTenantConfig("tenant_id_none"); + // verify tenantConfig's bytesConf -- defer to systemConf + value = tenantConfigNone.getValue(CelebornConf.CLIENT_PUSH_BUFFER_MAX_SIZE().key(), CelebornConf.CLIENT_PUSH_BUFFER_MAX_SIZE(), Long.TYPE, ConfigType.BYTES); + Assert.assertEquals(value.longValue(), 1024000); + + + // ------------- Verify with defaultValue ----------------- // + value = tenantConfig.getWithDefaultValue("celeborn.client.push.buffer.initial.size.only", 100l, Long.TYPE, ConfigType.BYTES); + Assert.assertEquals(value.longValue(), 102400); + + Long withDefaultValue = tenantConfigNone.getWithDefaultValue("none", 10l, Long.TYPE, ConfigType.STRING); + Assert.assertEquals(withDefaultValue.longValue(), 10); + } +} \ No newline at end of file From 927d62425bfae99e62278371f526cad07f153d47 Mon Sep 17 00:00:00 2001 From: onebox-li Date: Mon, 27 Nov 2023 18:01:38 +0800 Subject: [PATCH 23/31] [CELEBORN-1125][FOLLOWUP] Add failureaccess shade ### What changes were proposed in this pull request? Add failureaccess shade. ### Why are the changes needed? When test main branch, client got error like below: ``` Caused by: java.lang.NoClassDefFoundError: org/apache/celeborn/shaded/com/google/common/util/concurrent/internal/InternalFutureFailureAccess at org.apache.celeborn.shaded.com.google.common.cache.LocalCache$LoadingValueReference.(LocalCache.java:3517) at org.apache.celeborn.shaded.com.google.common.cache.LocalCache$LoadingValueReference.(LocalCache.java:3521) at org.apache.celeborn.shaded.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2170) at org.apache.celeborn.shaded.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2081) at org.apache.celeborn.shaded.com.google.common.cache.LocalCache.get(LocalCache.java:4019) at org.apache.celeborn.shaded.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4933) at org.apache.celeborn.client.commit.ReducePartitionCommitHandler.replyGetReducerFileGroup(ReducePartitionCommitHandler.scala:283) at org.apache.celeborn.client.commit.ReducePartitionCommitHandler.handleGetReducerFileGroup(ReducePartitionCommitHandler.scala:300) at org.apache.celeborn.client.CommitManager.handleGetReducerFileGroup(CommitManager.scala:266) at org.apache.celeborn.client.LifecycleManager.org$apache$celeborn$client$LifecycleManager$$handleGetReducerFileGroup(LifecycleManager.scala:628) at org.apache.celeborn.client.LifecycleManager$$anonfun$receiveAndReply$1.applyOrElse(LifecycleManager.scala:314) at org.apache.celeborn.common.rpc.netty.Inbox.$anonfun$process$1(Inbox.scala:115) at org.apache.celeborn.common.rpc.netty.Inbox.safelyCall(Inbox.scala:222) at org.apache.celeborn.common.rpc.netty.Inbox.process(Inbox.scala:110) at org.apache.celeborn.common.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:227) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manual test. Closes #2116 from onebox-li/shade-add-failureaccess. Authored-by: onebox-li Signed-off-by: mingji --- client-flink/flink-1.14-shaded/pom.xml | 1 + client-flink/flink-1.15-shaded/pom.xml | 1 + client-flink/flink-1.17-shaded/pom.xml | 1 + client-flink/flink-1.18-shaded/pom.xml | 1 + client-mr/mr-shaded/pom.xml | 1 + client-spark/spark-2-shaded/pom.xml | 1 + client-spark/spark-3-shaded/pom.xml | 1 + project/CelebornBuild.scala | 3 +++ 8 files changed, 10 insertions(+) diff --git a/client-flink/flink-1.14-shaded/pom.xml b/client-flink/flink-1.14-shaded/pom.xml index 9f3b52308e7..8f1d0892874 100644 --- a/client-flink/flink-1.14-shaded/pom.xml +++ b/client-flink/flink-1.14-shaded/pom.xml @@ -69,6 +69,7 @@ org.apache.celeborn:* com.google.protobuf:protobuf-java com.google.guava:guava + com.google.guava:failureaccess io.netty:* org.apache.commons:commons-lang3 org.roaringbitmap:RoaringBitmap diff --git a/client-flink/flink-1.15-shaded/pom.xml b/client-flink/flink-1.15-shaded/pom.xml index a4597d64b7c..84bc2959afb 100644 --- a/client-flink/flink-1.15-shaded/pom.xml +++ b/client-flink/flink-1.15-shaded/pom.xml @@ -69,6 +69,7 @@ org.apache.celeborn:* com.google.protobuf:protobuf-java com.google.guava:guava + com.google.guava:failureaccess io.netty:* org.apache.commons:commons-lang3 org.roaringbitmap:RoaringBitmap diff --git a/client-flink/flink-1.17-shaded/pom.xml b/client-flink/flink-1.17-shaded/pom.xml index 32754de0025..7ee2546466b 100644 --- a/client-flink/flink-1.17-shaded/pom.xml +++ b/client-flink/flink-1.17-shaded/pom.xml @@ -69,6 +69,7 @@ org.apache.celeborn:* com.google.protobuf:protobuf-java com.google.guava:guava + com.google.guava:failureaccess io.netty:* org.apache.commons:commons-lang3 org.roaringbitmap:RoaringBitmap diff --git a/client-flink/flink-1.18-shaded/pom.xml b/client-flink/flink-1.18-shaded/pom.xml index b03e83029ff..acd17f04d55 100644 --- a/client-flink/flink-1.18-shaded/pom.xml +++ b/client-flink/flink-1.18-shaded/pom.xml @@ -69,6 +69,7 @@ org.apache.celeborn:* com.google.protobuf:protobuf-java com.google.guava:guava + com.google.guava:failureaccess io.netty:* org.apache.commons:commons-lang3 org.roaringbitmap:RoaringBitmap diff --git a/client-mr/mr-shaded/pom.xml b/client-mr/mr-shaded/pom.xml index 9ecb39b9f23..795e28d018f 100644 --- a/client-mr/mr-shaded/pom.xml +++ b/client-mr/mr-shaded/pom.xml @@ -77,6 +77,7 @@ org.apache.celeborn:* com.google.protobuf:protobuf-java com.google.guava:guava + com.google.guava:failureaccess io.netty:* org.apache.commons:commons-lang3 org.scala-lang:scala-library diff --git a/client-spark/spark-2-shaded/pom.xml b/client-spark/spark-2-shaded/pom.xml index 962a4871e29..655e4b433c6 100644 --- a/client-spark/spark-2-shaded/pom.xml +++ b/client-spark/spark-2-shaded/pom.xml @@ -69,6 +69,7 @@ org.apache.celeborn:* com.google.protobuf:protobuf-java com.google.guava:guava + com.google.guava:failureaccess io.netty:* org.apache.commons:commons-lang3 org.roaringbitmap:RoaringBitmap diff --git a/client-spark/spark-3-shaded/pom.xml b/client-spark/spark-3-shaded/pom.xml index 2b55c8fee55..c8701776dc3 100644 --- a/client-spark/spark-3-shaded/pom.xml +++ b/client-spark/spark-3-shaded/pom.xml @@ -69,6 +69,7 @@ org.apache.celeborn:* com.google.protobuf:protobuf-java com.google.guava:guava + com.google.guava:failureaccess io.netty:* org.apache.commons:commons-lang3 org.roaringbitmap:RoaringBitmap diff --git a/project/CelebornBuild.scala b/project/CelebornBuild.scala index cf6b43e19e3..e940702cd3d 100644 --- a/project/CelebornBuild.scala +++ b/project/CelebornBuild.scala @@ -680,6 +680,7 @@ trait SparkClientProjects { !(name.startsWith("celeborn-") || name.startsWith("protobuf-java-") || name.startsWith("guava-") || + name.startsWith("failureaccess-") || name.startsWith("netty-") || name.startsWith("commons-lang3-") || name.startsWith("RoaringBitmap-")) @@ -883,6 +884,7 @@ trait FlinkClientProjects { !(name.startsWith("celeborn-") || name.startsWith("protobuf-java-") || name.startsWith("guava-") || + name.startsWith("failureaccess-") || name.startsWith("netty-") || name.startsWith("commons-lang3-") || name.startsWith("RoaringBitmap-")) @@ -977,6 +979,7 @@ object MRClientProjects { !(name.startsWith("celeborn-") || name.startsWith("protobuf-java-") || name.startsWith("guava-") || + name.startsWith("failureaccess-") || name.startsWith("netty-") || name.startsWith("commons-lang3-") || name.startsWith("RoaringBitmap-") || From 113311df3e99ce58715aaf6feedb62dd96f1e8b3 Mon Sep 17 00:00:00 2001 From: mingji Date: Tue, 28 Nov 2023 11:26:00 +0800 Subject: [PATCH 24/31] [CELEBORN-1081][FOLLOWUP] Remove UNKNOWN_DISK and allocate all slots to disk ### What changes were proposed in this pull request? 1. Remove UNKNOWN_DISK from StorageInfo. 2. Enable load-aware slots allocation when there is HDFS. ### Why are the changes needed? To support the application's config about available storage types. ### Does this PR introduce _any_ user-facing change? no. ### How was this patch tested? GA and Cluster. Closes #2098 from FMX/B1081-1. Authored-by: mingji Signed-off-by: Shuang --- .../celeborn/common/protocol/StorageInfo.java | 50 +++--- common/src/main/proto/TransportMessages.proto | 1 + .../celeborn/common/meta/DeviceInfo.scala | 21 ++- .../celeborn/common/meta/WorkerInfo.scala | 10 +- .../celeborn/common/util/PbSerDeUtils.scala | 8 +- .../protocol/PartitionLocationSuiteJ.java | 8 +- .../common/meta/WorkerInfoSuite.scala | 6 +- docs/developers/slotsallocation.md | 75 +++++++++ .../service/deploy/master/SlotsAllocator.java | 148 +++++++++++++----- .../deploy/master/clustermeta/MetaUtil.java | 5 +- master/src/main/proto/Resource.proto | 1 + .../service/deploy/master/Master.scala | 2 +- .../deploy/master/SlotsAllocatorSuiteJ.java | 121 ++++++++++++-- .../deploy/worker/storage/FileWriter.java | 3 +- .../service/deploy/worker/Worker.scala | 2 +- .../worker/storage/StorageManager.scala | 8 +- 16 files changed, 375 insertions(+), 94 deletions(-) diff --git a/common/src/main/java/org/apache/celeborn/common/protocol/StorageInfo.java b/common/src/main/java/org/apache/celeborn/common/protocol/StorageInfo.java index d3e1bf95b5f..5b59a10ed4b 100644 --- a/common/src/main/java/org/apache/celeborn/common/protocol/StorageInfo.java +++ b/common/src/main/java/org/apache/celeborn/common/protocol/StorageInfo.java @@ -38,7 +38,6 @@ public int getValue() { } } - @Deprecated public static String UNKNOWN_DISK = "UNKNOWN_DISK"; public static Map typesMap = new HashMap<>(); public static Set typeNames = new HashSet<>(); @@ -57,7 +56,7 @@ public int getValue() { // Default storage Type is MEMORY. private Type type = Type.MEMORY; - private String mountPoint = UNKNOWN_DISK; + private String mountPoint = ""; // if a file is committed, field "finalResult" will be true private boolean finalResult = false; private String filePath; @@ -72,27 +71,10 @@ public StorageInfo(Type type, boolean isFinal, String filePath) { this.filePath = filePath; } - public StorageInfo(String mountPoint, int availableStorageTypes) { + public StorageInfo(String mountPoint, StorageInfo.Type type, int availableStorageTypes) { this.mountPoint = mountPoint; - this.availableStorageTypes = availableStorageTypes; - } - - public StorageInfo(Type type, String mountPoint) { - this.type = type; - this.mountPoint = mountPoint; - } - - public StorageInfo(Type type, String mountPoint, boolean finalResult) { - this.type = type; - this.mountPoint = mountPoint; - this.finalResult = finalResult; - } - - public StorageInfo(Type type, String mountPoint, boolean finalResult, String filePath) { this.type = type; - this.mountPoint = mountPoint; - this.finalResult = finalResult; - this.filePath = filePath; + this.availableStorageTypes = availableStorageTypes; } public StorageInfo( @@ -147,21 +129,41 @@ public String toString() { + '}'; } - public boolean localDiskAvailable() { + public static boolean localDiskAvailable(int availableStorageTypes) { return availableStorageTypes == ALL_TYPES_AVAILABLE_MASK || (availableStorageTypes & LOCAL_DISK_MASK) > 0; } - public boolean HDFSAvailable() { + public boolean localDiskAvailable() { + return StorageInfo.localDiskAvailable(availableStorageTypes); + } + + public static boolean HDFSAvailable(int availableStorageTypes) { return availableStorageTypes == ALL_TYPES_AVAILABLE_MASK || (availableStorageTypes & HDFS_MASK) > 0; } - public boolean OSSAvailable() { + public boolean HDFSAvailable() { + return StorageInfo.HDFSAvailable(availableStorageTypes); + } + + public static boolean HDFSOnly(int availableStorageTypes) { + return availableStorageTypes == HDFS_MASK; + } + + public boolean HDFSOnly() { + return StorageInfo.HDFSOnly(availableStorageTypes); + } + + public static boolean OSSAvailable(int availableStorageTypes) { return availableStorageTypes == ALL_TYPES_AVAILABLE_MASK || (availableStorageTypes & OSS_MASK) > 0; } + public boolean OSSAvailable() { + return StorageInfo.OSSAvailable(availableStorageTypes); + } + @Override public boolean equals(Object o) { if (this == o) return true; diff --git a/common/src/main/proto/TransportMessages.proto b/common/src/main/proto/TransportMessages.proto index a595a785df8..b33271e0e83 100644 --- a/common/src/main/proto/TransportMessages.proto +++ b/common/src/main/proto/TransportMessages.proto @@ -137,6 +137,7 @@ message PbDiskInfo { int64 usedSlots = 4; int32 status = 5; int64 avgFetchTime = 6; + int32 storageType = 7; } message PbWorkerInfo { diff --git a/common/src/main/scala/org/apache/celeborn/common/meta/DeviceInfo.scala b/common/src/main/scala/org/apache/celeborn/common/meta/DeviceInfo.scala index 56933a2cf04..990a2b12734 100644 --- a/common/src/main/scala/org/apache/celeborn/common/meta/DeviceInfo.scala +++ b/common/src/main/scala/org/apache/celeborn/common/meta/DeviceInfo.scala @@ -49,6 +49,17 @@ class DiskInfo( this(mountPoint, usableSpace, avgFlushTime, avgFetchTime, activeSlots, List.empty, null) } + def this( + mountPoint: String, + usableSpace: Long, + avgFlushTime: Long, + avgFetchTime: Long, + activeSlots: Long, + storageType: StorageInfo.Type) = { + this(mountPoint, usableSpace, avgFlushTime, avgFetchTime, activeSlots, List.empty, null) + this.storageType = storageType + } + def this( mountPoint: String, dirs: List[File], @@ -70,10 +81,14 @@ class DiskInfo( var status: DiskStatus = DiskStatus.HEALTHY var threadCount = 1 var configuredUsableSpace = 0L - var storageType: StorageInfo.Type = _ + var storageType: StorageInfo.Type = StorageInfo.Type.SSD var maxSlots: Long = 0 lazy val shuffleAllocations = new util.HashMap[String, Integer]() + def setStorageType(storageType: StorageInfo.Type) = { + this.storageType = storageType + } + def setStatus(status: DiskStatus): this.type = this.synchronized { this.status = status this @@ -145,9 +160,11 @@ class DiskInfo( s" usableSpace: ${Utils.bytesToString(actualUsableSpace)}," + s" avgFlushTime: ${Utils.nanoDurationToString(avgFlushTime)}," + s" avgFetchTime: ${Utils.nanoDurationToString(avgFetchTime)}," + - s" activeSlots: $activeSlots)" + + s" activeSlots: $activeSlots," + + s" storageType: ${storageType})" + s" status: $status" + s" dirs ${dirs.mkString("\t")}" + } } diff --git a/common/src/main/scala/org/apache/celeborn/common/meta/WorkerInfo.scala b/common/src/main/scala/org/apache/celeborn/common/meta/WorkerInfo.scala index 65dc22918a2..05ea35d63ec 100644 --- a/common/src/main/scala/org/apache/celeborn/common/meta/WorkerInfo.scala +++ b/common/src/main/scala/org/apache/celeborn/common/meta/WorkerInfo.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.celeborn.common.identity.UserIdentifier import org.apache.celeborn.common.internal.Logging +import org.apache.celeborn.common.protocol.StorageInfo import org.apache.celeborn.common.quota.ResourceConsumption import org.apache.celeborn.common.rpc.RpcEndpointRef import org.apache.celeborn.common.rpc.netty.NettyRpcEndpointRef @@ -156,12 +157,12 @@ class WorkerInfo( curDisk.activeSlots = newDisk.activeSlots curDisk.avgFlushTime = newDisk.avgFlushTime curDisk.avgFetchTime = newDisk.avgFetchTime - if (estimatedPartitionSize.nonEmpty) { + if (estimatedPartitionSize.nonEmpty && curDisk.storageType != StorageInfo.Type.HDFS) { curDisk.maxSlots = curDisk.actualUsableSpace / estimatedPartitionSize.get } curDisk.setStatus(newDisk.status) } else { - if (estimatedPartitionSize.nonEmpty) { + if (estimatedPartitionSize.nonEmpty && newDisk.storageType != StorageInfo.Type.HDFS) { newDisk.maxSlots = newDisk.actualUsableSpace / estimatedPartitionSize.get } diskInfos.put(mountPoint, newDisk) @@ -239,6 +240,11 @@ class WorkerInfo( result = 31 * result + replicatePort.hashCode() result } + + def haveDisk(): Boolean = { + diskInfos.values().asScala.exists(p => + p.storageType == StorageInfo.Type.SSD || p.storageType == StorageInfo.Type.HDD) + } } object WorkerInfo { diff --git a/common/src/main/scala/org/apache/celeborn/common/util/PbSerDeUtils.scala b/common/src/main/scala/org/apache/celeborn/common/util/PbSerDeUtils.scala index 2748d6cf38d..d681152f7aa 100644 --- a/common/src/main/scala/org/apache/celeborn/common/util/PbSerDeUtils.scala +++ b/common/src/main/scala/org/apache/celeborn/common/util/PbSerDeUtils.scala @@ -63,14 +63,17 @@ object PbSerDeUtils { .setMinor(minor) .build.toByteArray - def fromPbDiskInfo(pbDiskInfo: PbDiskInfo): DiskInfo = - new DiskInfo( + def fromPbDiskInfo(pbDiskInfo: PbDiskInfo): DiskInfo = { + val diskInfo = new DiskInfo( pbDiskInfo.getMountPoint, pbDiskInfo.getUsableSpace, pbDiskInfo.getAvgFlushTime, pbDiskInfo.getAvgFetchTime, pbDiskInfo.getUsedSlots) .setStatus(Utils.toDiskStatus(pbDiskInfo.getStatus)) + diskInfo.setStorageType(StorageInfo.typesMap.get(pbDiskInfo.getStorageType)) + diskInfo + } def toPbDiskInfo(diskInfo: DiskInfo): PbDiskInfo = PbDiskInfo.newBuilder @@ -80,6 +83,7 @@ object PbSerDeUtils { .setAvgFetchTime(diskInfo.avgFetchTime) .setUsedSlots(diskInfo.activeSlots) .setStatus(diskInfo.status.getValue) + .setStorageType(diskInfo.storageType.getValue) .build def fromPbFileInfo(pbFileInfo: PbFileInfo): FileInfo = diff --git a/common/src/test/java/org/apache/celeborn/common/protocol/PartitionLocationSuiteJ.java b/common/src/test/java/org/apache/celeborn/common/protocol/PartitionLocationSuiteJ.java index 927b3e9c137..e4200c9134a 100644 --- a/common/src/test/java/org/apache/celeborn/common/protocol/PartitionLocationSuiteJ.java +++ b/common/src/test/java/org/apache/celeborn/common/protocol/PartitionLocationSuiteJ.java @@ -178,7 +178,9 @@ public void testToStringOutput() { PartitionLocation location2 = new PartitionLocation( partitionId, epoch, host, rpcPort, pushPort, fetchPort, replicatePort, mode, peer); - StorageInfo storageInfo = new StorageInfo(StorageInfo.Type.MEMORY, "/mnt/disk/0"); + StorageInfo storageInfo = + new StorageInfo( + "/mnt/disk/0", StorageInfo.Type.MEMORY, StorageInfo.ALL_TYPES_AVAILABLE_MASK); RoaringBitmap bitmap = new RoaringBitmap(); bitmap.add(1); bitmap.add(2); @@ -205,7 +207,7 @@ public void testToStringOutput() { + " host-rpcPort-pushPort-fetchPort-replicatePort:localhost-3-1-2-4\n" + " mode:PRIMARY\n" + " peer:(empty)\n" - + " storage hint:StorageInfo{type=MEMORY, mountPoint='UNKNOWN_DISK', finalResult=false, filePath=null}\n" + + " storage hint:StorageInfo{type=MEMORY, mountPoint='', finalResult=false, filePath=null}\n" + " mapIdBitMap:{}]"; String exp2 = "PartitionLocation[\n" @@ -213,7 +215,7 @@ public void testToStringOutput() { + " host-rpcPort-pushPort-fetchPort-replicatePort:localhost-3-1-2-4\n" + " mode:PRIMARY\n" + " peer:(host-rpcPort-pushPort-fetchPort-replicatePort:localhost-3-1-2-4)\n" - + " storage hint:StorageInfo{type=MEMORY, mountPoint='UNKNOWN_DISK', finalResult=false, filePath=null}\n" + + " storage hint:StorageInfo{type=MEMORY, mountPoint='', finalResult=false, filePath=null}\n" + " mapIdBitMap:{}]"; String exp3 = "PartitionLocation[\n" diff --git a/common/src/test/scala/org/apache/celeborn/common/meta/WorkerInfoSuite.scala b/common/src/test/scala/org/apache/celeborn/common/meta/WorkerInfoSuite.scala index b4286292488..ebd83682ee2 100644 --- a/common/src/test/scala/org/apache/celeborn/common/meta/WorkerInfoSuite.scala +++ b/common/src/test/scala/org/apache/celeborn/common/meta/WorkerInfoSuite.scala @@ -284,9 +284,9 @@ class WorkerInfoSuite extends CelebornFunSuite { |SlotsUsed: 60 |LastHeartbeat: 0 |Disks: $placeholder - | DiskInfo0: DiskInfo(maxSlots: 0, committed shuffles 0 shuffleAllocations: Map(), mountPoint: disk3, usableSpace: 2048.0 MiB, avgFlushTime: 3 ns, avgFetchTime: 3 ns, activeSlots: 30) status: HEALTHY dirs $placeholder - | DiskInfo1: DiskInfo(maxSlots: 0, committed shuffles 0 shuffleAllocations: Map(), mountPoint: disk1, usableSpace: 2048.0 MiB, avgFlushTime: 1 ns, avgFetchTime: 1 ns, activeSlots: 10) status: HEALTHY dirs $placeholder - | DiskInfo2: DiskInfo(maxSlots: 0, committed shuffles 0 shuffleAllocations: Map(), mountPoint: disk2, usableSpace: 2048.0 MiB, avgFlushTime: 2 ns, avgFetchTime: 2 ns, activeSlots: 20) status: HEALTHY dirs $placeholder + | DiskInfo0: DiskInfo(maxSlots: 0, committed shuffles 0 shuffleAllocations: Map(), mountPoint: disk3, usableSpace: 2048.0 MiB, avgFlushTime: 3 ns, avgFetchTime: 3 ns, activeSlots: 30, storageType: SSD) status: HEALTHY dirs $placeholder + | DiskInfo1: DiskInfo(maxSlots: 0, committed shuffles 0 shuffleAllocations: Map(), mountPoint: disk1, usableSpace: 2048.0 MiB, avgFlushTime: 1 ns, avgFetchTime: 1 ns, activeSlots: 10, storageType: SSD) status: HEALTHY dirs $placeholder + | DiskInfo2: DiskInfo(maxSlots: 0, committed shuffles 0 shuffleAllocations: Map(), mountPoint: disk2, usableSpace: 2048.0 MiB, avgFlushTime: 2 ns, avgFetchTime: 2 ns, activeSlots: 20, storageType: SSD) status: HEALTHY dirs $placeholder |UserResourceConsumption: $placeholder | UserIdentifier: `tenant1`.`name1`, ResourceConsumption: ResourceConsumption(diskBytesWritten: 20.0 MiB, diskFileCount: 1, hdfsBytesWritten: 50.0 MiB, hdfsFileCount: 1) |WorkerRef: null diff --git a/docs/developers/slotsallocation.md b/docs/developers/slotsallocation.md index e69de29bb2d..78644b16cc6 100644 --- a/docs/developers/slotsallocation.md +++ b/docs/developers/slotsallocation.md @@ -0,0 +1,75 @@ +--- +license: | + 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 + + https://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. +--- + +# Slots allocation + +This article describes the detailed design of Celeborn workers' slots allocation. +Slots allocation is the core components about how Celeborn distribute workload amount workers. +We have achieved two approaches of slots allocation. + +## Principle +Allocate slots to local disks unless explicit assigned to HDFS. + +## LoadAware +### Related configs +```properties +celeborn.master.slot.assign.policy LOADAWARE +celeborn.master.slot.assign.loadAware.numDiskGroups 5 +celeborn.master.slot.assign.loadAware.diskGroupGradient 0.1 +celeborn.master.slot.assign.loadAware.flushTimeWeight 0 +celeborn.master.slot.assign.loadAware.fetchTimeWeight 0 +[spark.client.]celeborn.storage.availableTypes HDD,SSD +``` +### Detail +Load-aware slots allocation will take following elements into consideration. +- disk's fetch time +- disk's flush time +- disk's usable space +- disk's used slot + +Slots allocator will find out all worker involved in this allocation and sort their disks by +`disk's average flushtime * flush time weight + disk's average fetch time * fetch time weight`. +After getting the sorted disks list, Celeborn will split the disks into +`celeborn.master.slot.assign.loadAware.numDiskGroups` groups. The slots number to be placed into a disk group +is controlled by the `celeborn.master.slot.assign.loadAware.diskGroupGradient` which means that a group's +allocated slots number will be (1+`celeborn.master.slot.assign.loadAware.diskGroupGradient`) +times to the group's slower than it. +For example, there is 5 groups, G1 , G2, G3, G4 and G5. If the G5 is allocated 100 slots. +Other groups will be G4:110, G3:121, G2:133, G1:146. + +After Celeborn has decided the slots number of a disk group, slots will be distributed in disks of a disk group. +Each disk has a usableSlots which is calculated by `(disk's usable space)/(average partition size)-usedSlots`. +The slots number to allocate in a disk is calculated by ` slots of this disk group * ( current disk's usableSlots / the sum of all disks' usableSlots in this group)`. +For example, G5 need to allocate 100 slots and have 3 disks D1 with usable slots 100, D2 with usable slots 50, D3 with usable slots 20. +The distribution will be D1:59, D2: 29, D3: 12. + +If all slots can be place in disk groups, the slots allocation process is done. + +requested slots are more than all usable slots, slots can not be placed into disks. +Worker will need to allocate these slots to workers with local disks one by one. + +## RoundRobin +### Detail +Roundrobin slots allocation will distribute all slots into all registered workers with disks. Celeborn will treat +all workers as an array and place 1 slots in a worker until all slots are allocated. +If a worker has multiple disks, the chosen disk index is `(monotone increasing disk index +1) % disk count`. + +## Celeborn Worker's Behavior +1. When reserve slots Celeborn worker will decide a slot be placed in local disks or HDFS when reserve slots. +2. If a partition is evicted from memory, the partition might be placed in HDFS. +3. If a slot is explicitly assigned to HDFS, worker will put the slot in HDFS. \ No newline at end of file diff --git a/master/src/main/java/org/apache/celeborn/service/deploy/master/SlotsAllocator.java b/master/src/main/java/org/apache/celeborn/service/deploy/master/SlotsAllocator.java index e964e57e556..e54ba8763e6 100644 --- a/master/src/main/java/org/apache/celeborn/service/deploy/master/SlotsAllocator.java +++ b/master/src/main/java/org/apache/celeborn/service/deploy/master/SlotsAllocator.java @@ -18,6 +18,7 @@ package org.apache.celeborn.service.deploy.master; import java.util.*; +import java.util.stream.Collectors; import scala.Double; import scala.Option; @@ -64,22 +65,30 @@ static class UsableDiskInfo { if (workers.size() < 2 && shouldReplicate) { return new HashMap<>(); } - Map> restrictions = new HashMap<>(); + Map> slotsRestrictions = new HashMap<>(); for (WorkerInfo worker : workers) { List usableDisks = - restrictions.computeIfAbsent(worker, v -> new ArrayList<>()); + slotsRestrictions.computeIfAbsent(worker, v -> new ArrayList<>()); for (Map.Entry diskInfoEntry : worker.diskInfos().entrySet()) { if (diskInfoEntry.getValue().status().equals(DiskStatus.HEALTHY)) { - usableDisks.add( - new UsableDiskInfo( - diskInfoEntry.getValue(), diskInfoEntry.getValue().availableSlots())); + if (StorageInfo.localDiskAvailable(availableStorageTypes) + && diskInfoEntry.getValue().storageType() != StorageInfo.Type.HDFS) { + usableDisks.add( + new UsableDiskInfo( + diskInfoEntry.getValue(), diskInfoEntry.getValue().availableSlots())); + } else if (StorageInfo.HDFSAvailable(availableStorageTypes) + && diskInfoEntry.getValue().storageType() == StorageInfo.Type.HDFS) { + usableDisks.add( + new UsableDiskInfo( + diskInfoEntry.getValue(), diskInfoEntry.getValue().availableSlots())); + } } } } return locateSlots( partitionIds, workers, - restrictions, + slotsRestrictions, shouldReplicate, shouldRackAware, availableStorageTypes); @@ -109,6 +118,10 @@ static class UsableDiskInfo { if (workers.size() < 2 && shouldReplicate) { return new HashMap<>(); } + if (StorageInfo.HDFSOnly(availableStorageTypes)) { + return offerSlotsRoundRobin( + workers, partitionIds, shouldReplicate, shouldRackAware, availableStorageTypes); + } List usableDisks = new ArrayList<>(); Map diskToWorkerMap = new HashMap<>(); @@ -126,7 +139,8 @@ static class UsableDiskInfo { diskReserveRatio.isEmpty() ? Option.empty() : Option.apply(diskReserveRatio.get())) - && diskInfo.status().equals(DiskStatus.HEALTHY)) { + && diskInfo.status().equals(DiskStatus.HEALTHY) + && diskInfo.storageType() != StorageInfo.Type.HDFS) { usableDisks.add(diskInfo); } })); @@ -151,15 +165,15 @@ static class UsableDiskInfo { initLoadAwareAlgorithm(diskGroupCount, diskGroupGradient); } - Map> restrictions = - getRestriction( + Map> slotsRestrictions = + getSlotsRestrictionsByLoadAwareAlgorithm( placeDisksToGroups(usableDisks, diskGroupCount, flushTimeWeight, fetchTimeWeight), diskToWorkerMap, shouldReplicate ? partitionIds.size() * 2 : partitionIds.size()); return locateSlots( partitionIds, workers, - restrictions, + slotsRestrictions, shouldReplicate, shouldRackAware, availableStorageTypes); @@ -172,16 +186,43 @@ private static StorageInfo getStorageInfo( Map workerDiskIndex, int availableStorageTypes) { WorkerInfo selectedWorker = workers.get(workerIndex); - List usableDiskInfos = restrictions.get(selectedWorker); + StorageInfo storageInfo; int diskIndex = workerDiskIndex.computeIfAbsent(selectedWorker, v -> 0); - while (usableDiskInfos.get(diskIndex).usableSlots <= 0) { - diskIndex = (diskIndex + 1) % usableDiskInfos.size(); + if (restrictions != null) { + List usableDiskInfos = restrictions.get(selectedWorker); + while (usableDiskInfos.get(diskIndex).usableSlots <= 0) { + diskIndex = (diskIndex + 1) % usableDiskInfos.size(); + } + usableDiskInfos.get(diskIndex).usableSlots--; + DiskInfo selectedDiskInfo = usableDiskInfos.get(diskIndex).diskInfo; + if (selectedDiskInfo.storageType() == StorageInfo.Type.HDFS) { + storageInfo = new StorageInfo("", StorageInfo.Type.HDFS, availableStorageTypes); + } else { + storageInfo = + new StorageInfo( + selectedDiskInfo.mountPoint(), + selectedDiskInfo.storageType(), + availableStorageTypes); + workerDiskIndex.put(selectedWorker, (diskIndex + 1) % usableDiskInfos.size()); + } + } else { + if (StorageInfo.localDiskAvailable(availableStorageTypes)) { + DiskInfo[] diskInfos = + selectedWorker.diskInfos().values().stream() + .filter(p -> p.storageType() != StorageInfo.Type.HDFS) + .collect(Collectors.toList()) + .toArray(new DiskInfo[0]); + storageInfo = + new StorageInfo( + diskInfos[diskIndex].mountPoint(), + diskInfos[diskIndex].storageType(), + availableStorageTypes); + diskIndex = (diskIndex + 1) % diskInfos.length; + workerDiskIndex.put(selectedWorker, (diskIndex + 1) % diskInfos.length); + } else { + storageInfo = new StorageInfo("", StorageInfo.Type.HDFS, availableStorageTypes); + } } - usableDiskInfos.get(diskIndex).usableSlots--; - StorageInfo storageInfo = - new StorageInfo( - usableDiskInfos.get(diskIndex).diskInfo.mountPoint(), availableStorageTypes); - workerDiskIndex.put(selectedWorker, (diskIndex + 1) % usableDiskInfos.size()); return storageInfo; } @@ -195,10 +236,10 @@ private static StorageInfo getStorageInfo( locateSlots( List partitionIds, List workers, - Map> restrictions, + Map> slotRestrictions, boolean shouldReplicate, boolean shouldRackAware, - int activeStorageTypes) { + int availableStorageTypes) { Map, List>> slots = new HashMap<>(); @@ -206,18 +247,24 @@ private static StorageInfo getStorageInfo( roundRobin( slots, partitionIds, - new LinkedList<>(restrictions.keySet()), - restrictions, + new LinkedList<>(slotRestrictions.keySet()), + slotRestrictions, shouldReplicate, shouldRackAware, - activeStorageTypes); + availableStorageTypes); if (!remain.isEmpty()) { remain = roundRobin( - slots, remain, workers, null, shouldReplicate, shouldRackAware, activeStorageTypes); + slots, + remain, + workers, + null, + shouldReplicate, + shouldRackAware, + availableStorageTypes); } if (!remain.isEmpty()) { - roundRobin(slots, remain, workers, null, shouldReplicate, false, activeStorageTypes); + roundRobin(slots, remain, workers, null, shouldReplicate, false, availableStorageTypes); } return slots; } @@ -226,7 +273,7 @@ private static List roundRobin( Map, List>> slots, List partitionIds, List workers, - Map> restrictions, + Map> slotsRestrictions, boolean shouldReplicate, boolean shouldRackAware, int availableStorageTypes) { @@ -241,9 +288,10 @@ private static List roundRobin( int nextPrimaryInd = primaryIndex; int partitionId = iter.next(); - StorageInfo storageInfo = new StorageInfo(); - if (restrictions != null) { - while (!haveUsableSlots(restrictions, workers, nextPrimaryInd)) { + StorageInfo storageInfo; + if (slotsRestrictions != null && !slotsRestrictions.isEmpty()) { + // this means that we'll select a mount point + while (!haveUsableSlots(slotsRestrictions, workers, nextPrimaryInd)) { nextPrimaryInd = (nextPrimaryInd + 1) % workers.size(); if (nextPrimaryInd == primaryIndex) { break outer; @@ -253,17 +301,29 @@ private static List roundRobin( getStorageInfo( workers, nextPrimaryInd, - restrictions, + slotsRestrictions, workerDiskIndexForPrimary, availableStorageTypes); + } else { + if (StorageInfo.localDiskAvailable(availableStorageTypes)) { + while (!workers.get(nextPrimaryInd).haveDisk()) { + nextPrimaryInd = (nextPrimaryInd + 1) % workers.size(); + if (nextPrimaryInd == primaryIndex) { + break outer; + } + } + } + storageInfo = + getStorageInfo( + workers, nextPrimaryInd, null, workerDiskIndexForPrimary, availableStorageTypes); } PartitionLocation primaryPartition = createLocation(partitionId, workers.get(nextPrimaryInd), null, storageInfo, true); if (shouldReplicate) { int nextReplicaInd = (nextPrimaryInd + 1) % workers.size(); - if (restrictions != null) { - while (!haveUsableSlots(restrictions, workers, nextReplicaInd) + if (slotsRestrictions != null) { + while (!haveUsableSlots(slotsRestrictions, workers, nextReplicaInd) || !satisfyRackAware(shouldRackAware, workers, nextPrimaryInd, nextReplicaInd)) { nextReplicaInd = (nextReplicaInd + 1) % workers.size(); if (nextReplicaInd == nextPrimaryInd) { @@ -274,7 +334,7 @@ private static List roundRobin( getStorageInfo( workers, nextReplicaInd, - restrictions, + slotsRestrictions, workerDiskIndexForReplica, availableStorageTypes); } else if (shouldRackAware) { @@ -284,6 +344,18 @@ private static List roundRobin( break outer; } } + } else { + if (StorageInfo.localDiskAvailable(availableStorageTypes)) { + while (!workers.get(nextPrimaryInd).haveDisk()) { + nextPrimaryInd = (nextPrimaryInd + 1) % workers.size(); + if (nextPrimaryInd == primaryIndex) { + break outer; + } + } + } + storageInfo = + getStorageInfo( + workers, nextReplicaInd, null, workerDiskIndexForReplica, availableStorageTypes); } PartitionLocation replicaPartition = createLocation( @@ -369,7 +441,11 @@ private static List> placeDisksToGroups( return diskGroups; } - private static Map> getRestriction( + /** + * This method implement the load aware slots allocation algorithm. See details at + * /docs/developers/slotsallocation.md + */ + private static Map> getSlotsRestrictionsByLoadAwareAlgorithm( List> groups, Map diskWorkerMap, int partitionCnt) { int groupSize = groups.size(); long[] groupAllocations = new long[groupSize]; @@ -505,8 +581,8 @@ public static Map> slotsToDiskAllocations( jointLocations.addAll(slots.get(worker)._2); for (PartitionLocation location : jointLocations) { String mountPoint = location.getStorageInfo().getMountPoint(); - // ignore slots for UNKNOWN_DISK - if (!mountPoint.equals(StorageInfo.UNKNOWN_DISK)) { + // skip non local disks slots + if (!mountPoint.isEmpty()) { if (slotsPerDisk.containsKey(mountPoint)) { slotsPerDisk.put(mountPoint, slotsPerDisk.get(mountPoint) + 1); } else { diff --git a/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/MetaUtil.java b/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/MetaUtil.java index ec3094fc276..03e1d21e361 100644 --- a/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/MetaUtil.java +++ b/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/MetaUtil.java @@ -24,6 +24,7 @@ import org.apache.celeborn.common.identity.UserIdentifier$; import org.apache.celeborn.common.meta.DiskInfo; import org.apache.celeborn.common.meta.WorkerInfo; +import org.apache.celeborn.common.protocol.StorageInfo; import org.apache.celeborn.common.quota.ResourceConsumption; import org.apache.celeborn.common.util.Utils; @@ -61,7 +62,8 @@ public static Map fromPbDiskInfos( v.getUsableSpace(), v.getAvgFlushTime(), v.getAvgFetchTime(), - v.getUsedSlots()); + v.getUsedSlots(), + StorageInfo.typesMap.get(v.getStorageType())); diskInfo.setStatus(Utils.toDiskStatus(v.getStatus())); map.put(k, diskInfo); }); @@ -81,6 +83,7 @@ public static Map toPbDiskInfos( .setAvgFlushTime(v.avgFlushTime()) .setAvgFetchTime(v.avgFetchTime()) .setUsedSlots(v.activeSlots()) + .setStorageType(v.storageType().getValue()) .setStatus(v.status().getValue()) .build())); return map; diff --git a/master/src/main/proto/Resource.proto b/master/src/main/proto/Resource.proto index c7cde94dce4..78dc477bd22 100644 --- a/master/src/main/proto/Resource.proto +++ b/master/src/main/proto/Resource.proto @@ -66,6 +66,7 @@ message DiskInfo { required int64 usedSlots = 4; required int32 status = 5; required int64 avgFetchTime = 6; + required int32 storageType =7; } message RequestSlotsRequest { diff --git a/master/src/main/scala/org/apache/celeborn/service/deploy/master/Master.scala b/master/src/main/scala/org/apache/celeborn/service/deploy/master/Master.scala index 1c03903a748..e17bb46bfc9 100644 --- a/master/src/main/scala/org/apache/celeborn/service/deploy/master/Master.scala +++ b/master/src/main/scala/org/apache/celeborn/service/deploy/master/Master.scala @@ -687,7 +687,7 @@ private[celeborn] class Master( val slots = masterSource.sample(MasterSource.OFFER_SLOTS_TIME, s"offerSlots-${Random.nextInt()}") { statusSystem.workers.synchronized { - if (slotsAssignPolicy == SlotsAssignPolicy.LOADAWARE && !hasHDFSStorage) { + if (slotsAssignPolicy == SlotsAssignPolicy.LOADAWARE) { SlotsAllocator.offerSlotsLoadAware( selectedWorkers, requestSlots.partitionIdList, diff --git a/master/src/test/java/org/apache/celeborn/service/deploy/master/SlotsAllocatorSuiteJ.java b/master/src/test/java/org/apache/celeborn/service/deploy/master/SlotsAllocatorSuiteJ.java index 92f2ecfbd14..fd724cd7032 100644 --- a/master/src/test/java/org/apache/celeborn/service/deploy/master/SlotsAllocatorSuiteJ.java +++ b/master/src/test/java/org/apache/celeborn/service/deploy/master/SlotsAllocatorSuiteJ.java @@ -17,15 +17,7 @@ package org.apache.celeborn.service.deploy.master; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.Set; +import java.util.*; import scala.Option; import scala.Tuple2; @@ -294,16 +286,34 @@ private void checkSlotsOnHDFS( List workers, List partitionIds, boolean shouldReplicate, - boolean expectSuccess) { + boolean expectSuccess, + boolean roundrobin) { String shuffleKey = "appId-1"; CelebornConf conf = new CelebornConf(); conf.set("celeborn.active.storage.levels", "HDFS"); - Map, List>> slots = - SlotsAllocator.offerSlotsRoundRobin( - workers, partitionIds, shouldReplicate, false, StorageInfo.ALL_TYPES_AVAILABLE_MASK); - + Map, List>> slots; + if (roundrobin) { + slots = + SlotsAllocator.offerSlotsRoundRobin( + workers, partitionIds, shouldReplicate, false, StorageInfo.HDFS_MASK); + } else { + slots = + SlotsAllocator.offerSlotsLoadAware( + workers, + partitionIds, + shouldReplicate, + false, + 1000_000_000, + Option.empty(), + 3, + 0.1, + 0, + 1, + StorageInfo.LOCAL_DISK_MASK | StorageInfo.HDFS_MASK); + } int allocatedPartitionCount = 0; - + Map> slotsDistribution = + SlotsAllocator.slotsToDiskAllocations(slots); for (Map.Entry, List>> workerToPartitions : slots.entrySet()) { WorkerInfo workerInfo = workerToPartitions.getKey(); @@ -332,7 +342,86 @@ public void testHDFSOnly() { partitionIds.add(i); } final boolean shouldReplicate = true; - checkSlotsOnHDFS(workers, partitionIds, shouldReplicate, true); + checkSlotsOnHDFS(workers, partitionIds, shouldReplicate, true, true); + } + + @Test + public void testLocalDisksAndHDFSOnRoundRobin() { + final List workers = prepareWorkers(true); + DiskInfo hdfs1 = + new DiskInfo( + "HDFS", Long.MAX_VALUE, 999999, 999999, Integer.MAX_VALUE, StorageInfo.Type.HDFS); + DiskInfo hdfs2 = + new DiskInfo( + "HDFS", Long.MAX_VALUE, 999999, 999999, Integer.MAX_VALUE, StorageInfo.Type.HDFS); + DiskInfo hdfs3 = + new DiskInfo( + "HDFS", Long.MAX_VALUE, 999999, 999999, Integer.MAX_VALUE, StorageInfo.Type.HDFS); + hdfs1.maxSlots_$eq(Long.MAX_VALUE); + hdfs2.maxSlots_$eq(Long.MAX_VALUE); + hdfs3.maxSlots_$eq(Long.MAX_VALUE); + workers.get(0).diskInfos().put("HDFS", hdfs1); + workers.get(1).diskInfos().put("HDFS", hdfs2); + workers.get(2).diskInfos().put("HDFS", hdfs3); + final List partitionIds = new ArrayList<>(); + for (int i = 0; i < 3000; i++) { + partitionIds.add(i); + } + final boolean shouldReplicate = true; + checkSlotsOnHDFS(workers, partitionIds, shouldReplicate, true, true); + } + + @Test + public void testLocalDisksAndHDFSOnLoadAware() { + final List workers = prepareWorkers(true); + DiskInfo hdfs1 = + new DiskInfo( + "HDFS", Long.MAX_VALUE, 999999, 999999, Integer.MAX_VALUE, StorageInfo.Type.HDFS); + DiskInfo hdfs2 = + new DiskInfo( + "HDFS", Long.MAX_VALUE, 999999, 999999, Integer.MAX_VALUE, StorageInfo.Type.HDFS); + // DiskInfo hdfs3 = new DiskInfo("HDFS", Long.MAX_VALUE, 999999, 999999, Integer.MAX_VALUE, + // StorageInfo.Type.HDFS); + hdfs1.maxSlots_$eq(Long.MAX_VALUE); + hdfs2.maxSlots_$eq(Long.MAX_VALUE); + // hdfs3.maxSlots_$eq(Long.MAX_VALUE); + workers.get(0).diskInfos().put("HDFS", hdfs1); + workers.get(1).diskInfos().put("HDFS", hdfs2); + // workers.get(2).diskInfos().put("HDFS", hdfs3); + final List partitionIds = new ArrayList<>(); + for (int i = 0; i < 3000; i++) { + partitionIds.add(i); + } + final boolean shouldReplicate = true; + checkSlotsOnHDFS(workers, partitionIds, shouldReplicate, true, false); + } + + @Test + public void testLocalDisksAndHDFSOnLoadAwareWithInsufficientSlots() { + final List workers = prepareWorkers(true); + DiskInfo hdfs1 = + new DiskInfo( + "HDFS", Long.MAX_VALUE, 999999, 999999, Integer.MAX_VALUE, StorageInfo.Type.HDFS); + DiskInfo hdfs2 = + new DiskInfo( + "HDFS", Long.MAX_VALUE, 999999, 999999, Integer.MAX_VALUE, StorageInfo.Type.HDFS); + // DiskInfo hdfs3 = new DiskInfo("HDFS", Long.MAX_VALUE, 999999, 999999, Integer.MAX_VALUE, + // StorageInfo.Type.HDFS); + hdfs1.maxSlots_$eq(Long.MAX_VALUE); + hdfs2.maxSlots_$eq(Long.MAX_VALUE); + // hdfs3.maxSlots_$eq(Long.MAX_VALUE); + workers.get(0).diskInfos().put("HDFS", hdfs1); + workers.get(1).diskInfos().put("HDFS", hdfs2); + for (Map.Entry diskEntry : workers.get(2).diskInfos().entrySet()) { + diskEntry.getValue().maxSlots_$eq(100); + } + // workers.get(2).diskInfos().put("HDFS", hdfs3); + final List partitionIds = new ArrayList<>(); + for (int i = 0; i < 3000; i++) { + partitionIds.add(i); + } + final boolean shouldReplicate = true; + checkSlotsOnHDFS(workers, partitionIds, shouldReplicate, true, false); } @Test diff --git a/worker/src/main/java/org/apache/celeborn/service/deploy/worker/storage/FileWriter.java b/worker/src/main/java/org/apache/celeborn/service/deploy/worker/storage/FileWriter.java index 8ec6497b4dc..ca38a49c46c 100644 --- a/worker/src/main/java/org/apache/celeborn/service/deploy/worker/storage/FileWriter.java +++ b/worker/src/main/java/org/apache/celeborn/service/deploy/worker/storage/FileWriter.java @@ -234,7 +234,8 @@ public RoaringBitmap getMapIdBitMap() { public StorageInfo getStorageInfo() { if (flusher instanceof LocalFlusher) { LocalFlusher localFlusher = (LocalFlusher) flusher; - return new StorageInfo(localFlusher.diskType(), localFlusher.mountPoint(), true); + // do not write file path to reduce rpc size + return new StorageInfo(localFlusher.diskType(), true, ""); } else { if (deleted) { return null; diff --git a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/Worker.scala b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/Worker.scala index 80e2073098c..d8c7049e586 100644 --- a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/Worker.scala +++ b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/Worker.scala @@ -343,7 +343,7 @@ private[celeborn] class Worker( val diskInfos = workerInfo.updateThenGetDiskInfos(storageManager.disksSnapshot().map { disk => disk.mountPoint -> disk - }.toMap.asJava).values().asScala.toSeq + }.toMap.asJava).values().asScala.toSeq ++ storageManager.hdfsDiskInfo val response = masterClient.askSync[HeartbeatFromWorkerResponse]( HeartbeatFromWorker( host, diff --git a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/StorageManager.scala b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/StorageManager.scala index b1239b49798..f3c4e99e00c 100644 --- a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/StorageManager.scala +++ b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/StorageManager.scala @@ -40,7 +40,7 @@ import org.apache.celeborn.common.internal.Logging import org.apache.celeborn.common.meta.{DeviceInfo, DiskInfo, DiskStatus, FileInfo, TimeWindow} import org.apache.celeborn.common.metrics.source.AbstractSource import org.apache.celeborn.common.network.util.{NettyUtils, TransportConf} -import org.apache.celeborn.common.protocol.{PartitionLocation, PartitionSplitMode, PartitionType} +import org.apache.celeborn.common.protocol.{PartitionLocation, PartitionSplitMode, PartitionType, StorageInfo} import org.apache.celeborn.common.quota.ResourceConsumption import org.apache.celeborn.common.util.{CelebornExitKind, CelebornHadoopUtils, JavaUtils, PbSerDeUtils, ThreadUtils, Utils} import org.apache.celeborn.service.deploy.worker._ @@ -72,6 +72,10 @@ final private[worker] class StorageManager(conf: CelebornConf, workerSource: Abs DeviceInfo.getDeviceAndDiskInfos(workingDirInfos, conf) } val mountPoints = new util.HashSet[String](diskInfos.keySet()) + val hdfsDiskInfo = + if (conf.hasHDFSStorage) + Option(new DiskInfo("HDFS", Long.MaxValue, 999999, 999999, 0, StorageInfo.Type.HDFS)) + else None def disksSnapshot(): List[DiskInfo] = { diskInfos.synchronized { @@ -362,7 +366,7 @@ final private[worker] class StorageManager(conf: CelebornConf, workerSource: Abs throw new IOException(s"No available disks! suggested mountPoint $suggestedMountPoint") } val shuffleKey = Utils.makeShuffleKey(appId, shuffleId) - if (dirs.isEmpty && location.getStorageInfo.HDFSAvailable()) { + if ((dirs.isEmpty && location.getStorageInfo.HDFSAvailable()) || location.getStorageInfo.HDFSOnly()) { val shuffleDir = new Path(new Path(hdfsDir, conf.workerWorkingDir), s"$appId/$shuffleId") val fileInfo = From 662330923716d2b5fe70c960f04853063a9d737c Mon Sep 17 00:00:00 2001 From: Angerszhuuuu Date: Tue, 28 Nov 2023 16:26:00 +0800 Subject: [PATCH 25/31] [CELEBORN-1143][BUG] SortBasedPusher pushData should inc memory spill metrics ### What changes were proposed in this pull request? SortBasedPusher `pushData` should inc memory spill metrics ### Why are the changes needed? Make metrics more acurate ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Closes #2117 from AngersZhuuuu/CELEBORN-1143. Authored-by: Angerszhuuuu Signed-off-by: Cheng Pan --- .../apache/spark/shuffle/celeborn/SortBasedPusher.java | 8 +++++++- .../spark/shuffle/celeborn/SortBasedPusherSuiteJ.java | 7 +++++++ .../spark/shuffle/celeborn/SortBasedShuffleWriter.java | 2 ++ .../spark/shuffle/celeborn/SortBasedShuffleWriter.java | 2 ++ 4 files changed, 18 insertions(+), 1 deletion(-) diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedPusher.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedPusher.java index 305f83bc2a9..aff8e7ce38a 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedPusher.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedPusher.java @@ -24,6 +24,7 @@ import java.util.concurrent.atomic.LongAdder; import java.util.function.Consumer; +import org.apache.spark.TaskContext; import org.apache.spark.memory.MemoryConsumer; import org.apache.spark.memory.SparkOutOfMemoryError; import org.apache.spark.memory.TaskMemoryManager; @@ -57,6 +58,7 @@ public class SortBasedPusher extends MemoryConsumer { private long pageCursor = -1; private final ShuffleClient shuffleClient; + private final TaskContext taskContext; private DataPusher dataPusher; private final int pushBufferMaxSize; private final long pushSortMemoryThreshold; @@ -78,6 +80,7 @@ public class SortBasedPusher extends MemoryConsumer { public SortBasedPusher( TaskMemoryManager memoryManager, ShuffleClient shuffleClient, + TaskContext taskContext, int shuffleId, int mapId, int attemptNumber, @@ -97,6 +100,7 @@ public SortBasedPusher( memoryManager.getTungstenMemoryMode()); this.shuffleClient = shuffleClient; + this.taskContext = taskContext; this.shuffleId = shuffleId; this.mapId = mapId; @@ -213,6 +217,7 @@ public long pushData() throws IOException { long freedBytes = freeMemory(); inMemSorter.freeMemory(); + taskContext.taskMetrics().incMemoryBytesSpilled(freedBytes); return freedBytes; } @@ -434,11 +439,12 @@ private long freeMemory() { } public void cleanupResources() { - freeMemory(); + long freedBytes = freeMemory(); if (inMemSorter != null) { inMemSorter.freeMemory(); inMemSorter = null; } + taskContext.taskMetrics().incMemoryBytesSpilled(freedBytes); } public void close() throws IOException { diff --git a/client-spark/common/src/test/java/org/apache/spark/shuffle/celeborn/SortBasedPusherSuiteJ.java b/client-spark/common/src/test/java/org/apache/spark/shuffle/celeborn/SortBasedPusherSuiteJ.java index 5bdd704d05e..bc86f9db793 100644 --- a/client-spark/common/src/test/java/org/apache/spark/shuffle/celeborn/SortBasedPusherSuiteJ.java +++ b/client-spark/common/src/test/java/org/apache/spark/shuffle/celeborn/SortBasedPusherSuiteJ.java @@ -27,6 +27,8 @@ import scala.collection.mutable.ListBuffer; import org.apache.spark.SparkConf; +import org.apache.spark.TaskContext$; +import org.apache.spark.TaskContextImpl; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.memory.UnifiedMemoryManager; import org.apache.spark.sql.catalyst.InternalRow; @@ -58,6 +60,8 @@ public class SortBasedPusherSuiteJ { private final TaskMemoryManager taskMemoryManager = new TaskMemoryManager(unifiedMemoryManager, 0); + private final TaskContextImpl taskContext = TaskContext$.MODULE$.empty(); + private final File tempFile = new File(tempDir, UUID.randomUUID().toString()); private static File tempDir = null; @@ -82,6 +86,7 @@ public void testMemoryUsage() throws Exception { new SortBasedPusher( taskMemoryManager, /*shuffleClient=*/ client, + /*taskContext=*/ taskContext, /*shuffleId=*/ 0, /*mapId=*/ 0, /*attemptNumber=*/ 0, @@ -125,6 +130,8 @@ public void testMemoryUsage() throws Exception { row5k.getBaseObject(), row5k.getBaseOffset(), row5k.getSizeInBytes(), 0, true)); pusher.close(); + + assertEquals(taskContext.taskMetrics().memoryBytesSpilled(), 2097152); } private static UnsafeRow genUnsafeRow(int size) { diff --git a/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriter.java b/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriter.java index 51087c6dd89..daf25104793 100644 --- a/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriter.java +++ b/client-spark/spark-2/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriter.java @@ -134,6 +134,7 @@ public SortBasedShuffleWriter( new SortBasedPusher( taskContext.taskMemoryManager(), shuffleClient, + taskContext, shuffleId, mapId, taskContext.attemptNumber(), @@ -154,6 +155,7 @@ public SortBasedShuffleWriter( new SortBasedPusher( taskContext.taskMemoryManager(), shuffleClient, + taskContext, shuffleId, mapId, taskContext.attemptNumber(), diff --git a/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriter.java b/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriter.java index 0b515c456b5..a2f3cdfb75f 100644 --- a/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriter.java +++ b/client-spark/spark-3/src/main/java/org/apache/spark/shuffle/celeborn/SortBasedShuffleWriter.java @@ -134,6 +134,7 @@ public SortBasedShuffleWriter( new SortBasedPusher( taskContext.taskMemoryManager(), shuffleClient, + taskContext, shuffleId, mapId, taskContext.attemptNumber(), @@ -154,6 +155,7 @@ public SortBasedShuffleWriter( new SortBasedPusher( taskContext.taskMemoryManager(), shuffleClient, + taskContext, shuffleId, mapId, taskContext.attemptNumber(), From 4dfcd9b56b9343594de682a219fd5ae2de7e3fd5 Mon Sep 17 00:00:00 2001 From: SteNicholas Date: Tue, 28 Nov 2023 20:45:08 +0800 Subject: [PATCH 26/31] [CELEBORN-1092] Introduce JVM monitoring in Celeborn Worker using JVMQuake ### What changes were proposed in this pull request? Introduce JVM monitoring in Celeborn Worker using JVMQuake to enable early detection of memory management issues and facilitate fast failure. ### Why are the changes needed? When facing out-of-control memory management in Celeborn Worker we typically use JVMkill as a remedy by killing the process and generating a heap dump for post-analysis. However, even with jvmkill protection, we may still encounter issues caused by JVM running out of memory, such as repeated execution of Full GC without performing any useful work during the pause time. Since the JVM does not exhaust 100% of resources, JVMkill will not be triggered. Therefore JVMQuake is introduced to provide more granular monitoring of GC behavior, enabling early detection of memory management issues and facilitating fast failure. Refers to the principle of [jvmquake](https://github.com/Netflix-Skunkworks/jvmquake) which is a JVMTI agent that attaches to your JVM and automatically signals and kills it when the program has become unstable. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? `JVMQuakeSuite` Closes #2061 from SteNicholas/CELEBORN-1092. Authored-by: SteNicholas Signed-off-by: Fu Chen --- LICENSE-binary | 2 + .../apache/celeborn/common/CelebornConf.scala | 86 ++++++++ .../apache/celeborn/common/util/Utils.scala | 2 + dev/dependencies.sh | 8 +- dev/deps/dependencies-client-flink-1.14 | 1 + dev/deps/dependencies-client-flink-1.15 | 1 + dev/deps/dependencies-client-flink-1.17 | 1 + dev/deps/dependencies-client-flink-1.18 | 1 + dev/deps/dependencies-client-mr | 1 + dev/deps/dependencies-client-spark-2.4 | 1 + dev/deps/dependencies-client-spark-3.0 | 1 + dev/deps/dependencies-client-spark-3.1 | 1 + dev/deps/dependencies-client-spark-3.2 | 1 + dev/deps/dependencies-client-spark-3.3 | 1 + dev/deps/dependencies-client-spark-3.4 | 1 + dev/deps/dependencies-client-spark-3.5 | 1 + dev/deps/dependencies-server | 1 + docs/configuration/worker.md | 8 + licenses-binary/LISENCE-jdktools.txt | 21 ++ pom.xml | 8 + project/CelebornBuild.scala | 4 + project/JDKTools.scala | 77 +++++++ .../service/deploy/worker/Worker.scala | 10 + .../deploy/worker/monitor/JVMQuake.scala | 197 ++++++++++++++++++ .../deploy/worker/monitor/JVMQuakeSuite.scala | 69 ++++++ 25 files changed, 501 insertions(+), 4 deletions(-) create mode 100644 licenses-binary/LISENCE-jdktools.txt create mode 100644 project/JDKTools.scala create mode 100644 worker/src/main/scala/org/apache/celeborn/service/deploy/worker/monitor/JVMQuake.scala create mode 100644 worker/src/test/scala/org/apache/celeborn/service/deploy/worker/monitor/JVMQuakeSuite.scala diff --git a/LICENSE-binary b/LICENSE-binary index 585c150c312..61838e78a2f 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -304,6 +304,8 @@ javax.servlet:javax.servlet-api MIT License ------------ +See license/LISENCE-jdktools.txt for details. +com.github.olivergondza:maven-jdk-tools-wrapper See license/LICENSE-slf4j.txt for details. org.slf4j:jul-to-slf4j org.slf4j:slf4j-api diff --git a/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala b/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala index 247727df210..422d0f55521 100644 --- a/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala +++ b/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala @@ -689,6 +689,21 @@ class CelebornConf(loadDefaults: Boolean) extends Cloneable with Logging with Se def workerFetchHeartbeatEnabled: Boolean = get(WORKER_FETCH_HEARTBEAT_ENABLED) def workerPartitionSplitEnabled: Boolean = get(WORKER_PARTITION_SPLIT_ENABLED) def workerActiveConnectionMax: Option[Long] = get(WORKER_ACTIVE_CONNECTION_MAX) + def workerJvmQuakeEnabled: Boolean = get(WORKER_JVM_QUAKE_ENABLED) + def workerJvmQuakeCheckInterval: Long = get(WORKER_JVM_QUAKE_CHECK_INTERVAL) + def workerJvmQuakeRuntimeWeight: Double = get(WORKER_JVM_QUAKE_RUNTIME_WEIGHT) + def workerJvmQuakeDumpEnabled: Boolean = get(WORKER_JVM_QUAKE_DUMP_ENABLED) + def workerJvmQuakeDumpPath: String = get(WORKER_JVM_QUAKE_DUMP_PATH) + + def workerJvmQuakeDumpThreshold: Duration = + getTimeAsMs( + WORKER_JVM_QUAKE_DUMP_THRESHOLD.key, + WORKER_JVM_QUAKE_DUMP_THRESHOLD.defaultValueString).microsecond + def workerJvmQuakeKillThreshold: Duration = + getTimeAsMs( + WORKER_JVM_QUAKE_KILL_THRESHOLD.key, + WORKER_JVM_QUAKE_KILL_THRESHOLD.defaultValueString).microsecond + def workerJvmQuakeExitCode: Int = get(WORKER_JVM_QUAKE_EXIT_CODE) // ////////////////////////////////////////////////////// // Metrics System // @@ -2896,6 +2911,77 @@ object CelebornConf extends Logging { .longConf .createOptional + val WORKER_JVM_QUAKE_ENABLED: ConfigEntry[Boolean] = + buildConf("celeborn.worker.jvmQuake.enabled") + .categories("worker") + .version("0.4.0") + .doc("When true, Celeborn worker will start the jvm quake to monitor of gc behavior, " + + "which enables early detection of memory management issues and facilitates fast failure.") + .booleanConf + .createWithDefault(false) + + val WORKER_JVM_QUAKE_CHECK_INTERVAL: ConfigEntry[Long] = + buildConf("celeborn.worker.jvmQuake.check.interval") + .categories("worker") + .version("0.4.0") + .doc("Interval of gc behavior checking for worker jvm quake.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("1s") + + val WORKER_JVM_QUAKE_RUNTIME_WEIGHT: ConfigEntry[Double] = + buildConf("celeborn.worker.jvmQuake.runtimeWeight") + .categories("worker") + .version("0.4.0") + .doc( + "The factor by which to multiply running JVM time, when weighing it against GCing time. " + + "'Deficit' is accumulated as `gc_time - runtime * runtime_weight`, and is compared against threshold " + + "to determine whether to take action.") + .doubleConf + .createWithDefault(5) + + val WORKER_JVM_QUAKE_DUMP_THRESHOLD: ConfigEntry[Long] = + buildConf("celeborn.worker.jvmQuake.dump.threshold") + .categories("worker") + .version("0.4.0") + .doc("The threshold of heap dump for the maximum GC 'deficit' which can be accumulated before jvmquake takes action. " + + "Meanwhile, there is no heap dump generated when dump threshold is greater than kill threshold.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("30s") + + val WORKER_JVM_QUAKE_KILL_THRESHOLD: ConfigEntry[Long] = + buildConf("celeborn.worker.jvmQuake.kill.threshold") + .categories("worker") + .version("0.4.0") + .doc("The threshold of system kill for the maximum GC 'deficit' which can be accumulated before jvmquake takes action.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("60s") + + val WORKER_JVM_QUAKE_DUMP_ENABLED: ConfigEntry[Boolean] = + buildConf("celeborn.worker.jvmQuake.dump.enabled") + .categories("worker") + .version("0.4.0") + .doc("Whether to heap dump for the maximum GC 'deficit' during worker jvm quake.") + .booleanConf + .createWithDefault(true) + + val WORKER_JVM_QUAKE_DUMP_PATH: ConfigEntry[String] = + buildConf("celeborn.worker.jvmQuake.dump.path") + .categories("worker") + .version("0.4.0") + .doc("The path of heap dump for the maximum GC 'deficit' during worker jvm quake.") + .stringConf + .transform(_.replace("", System.getProperty("java.io.tmpdir")) + .replace("", Utils.getProcessId)) + .createWithDefault(s"/jvm-quake/dump/") + + val WORKER_JVM_QUAKE_EXIT_CODE: ConfigEntry[Int] = + buildConf("celeborn.worker.jvmQuake.exitCode") + .categories("worker") + .version("0.4.0") + .doc("The exit code of system kill for the maximum GC 'deficit' during worker jvm quake.") + .intConf + .createWithDefault(502) + val APPLICATION_HEARTBEAT_INTERVAL: ConfigEntry[Long] = buildConf("celeborn.client.application.heartbeatInterval") .withAlternative("celeborn.application.heartbeatInterval") diff --git a/common/src/main/scala/org/apache/celeborn/common/util/Utils.scala b/common/src/main/scala/org/apache/celeborn/common/util/Utils.scala index 5704f3f88cc..47c26ae09dc 100644 --- a/common/src/main/scala/org/apache/celeborn/common/util/Utils.scala +++ b/common/src/main/scala/org/apache/celeborn/common/util/Utils.scala @@ -1091,4 +1091,6 @@ object Utils extends Logging { } labelPart(0).trim -> labelPart(1).trim } + + def getProcessId: String = ManagementFactory.getRuntimeMXBean.getName.split("@")(0) } diff --git a/dev/dependencies.sh b/dev/dependencies.sh index 73c6c41c462..ae44b2e4516 100755 --- a/dev/dependencies.sh +++ b/dev/dependencies.sh @@ -56,9 +56,9 @@ function mvn_build_classpath() { } function sbt_build_client_classpath() { - PATTERN="$SBT_PROJECT / Runtime / externalDependencyClasspath" + PATTERN="$SBT_PROJECT / Runtime / managedClasspath" deps=$( - $SBT -P$MODULE "clean; export Runtime/externalDependencyClasspath" | \ + $SBT -P$MODULE "clean; export Runtime/managedClasspath" | \ awk -v pat="$PATTERN" '$0 ~ pat { found=1 } found { print }' | \ awk 'NR==2' | \ tr ":" "\n" @@ -96,8 +96,8 @@ function sbt_build_client_classpath() { } function sbt_build_server_classpath() { - $SBT "error; clean; export externalDependencyClasspath" | \ - awk '/externalDependencyClasspath/ { found=1 } found { print }' | \ + $SBT "error; clean; export managedClasspath" | \ + awk '/managedClasspath/ { found=1 } found { print }' | \ awk 'NR % 2 == 0' | \ # This will skip the last line sed '$d' | \ diff --git a/dev/deps/dependencies-client-flink-1.14 b/dev/deps/dependencies-client-flink-1.14 index 456250bb961..be6b7d17fb4 100644 --- a/dev/deps/dependencies-client-flink-1.14 +++ b/dev/deps/dependencies-client-flink-1.14 @@ -30,6 +30,7 @@ jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar lz4-java/1.8.0//lz4-java-1.8.0.jar +maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/3.2.6//metrics-core-3.2.6.jar metrics-graphite/3.2.6//metrics-graphite-3.2.6.jar metrics-jvm/3.2.6//metrics-jvm-3.2.6.jar diff --git a/dev/deps/dependencies-client-flink-1.15 b/dev/deps/dependencies-client-flink-1.15 index 456250bb961..be6b7d17fb4 100644 --- a/dev/deps/dependencies-client-flink-1.15 +++ b/dev/deps/dependencies-client-flink-1.15 @@ -30,6 +30,7 @@ jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar lz4-java/1.8.0//lz4-java-1.8.0.jar +maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/3.2.6//metrics-core-3.2.6.jar metrics-graphite/3.2.6//metrics-graphite-3.2.6.jar metrics-jvm/3.2.6//metrics-jvm-3.2.6.jar diff --git a/dev/deps/dependencies-client-flink-1.17 b/dev/deps/dependencies-client-flink-1.17 index 456250bb961..be6b7d17fb4 100644 --- a/dev/deps/dependencies-client-flink-1.17 +++ b/dev/deps/dependencies-client-flink-1.17 @@ -30,6 +30,7 @@ jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar lz4-java/1.8.0//lz4-java-1.8.0.jar +maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/3.2.6//metrics-core-3.2.6.jar metrics-graphite/3.2.6//metrics-graphite-3.2.6.jar metrics-jvm/3.2.6//metrics-jvm-3.2.6.jar diff --git a/dev/deps/dependencies-client-flink-1.18 b/dev/deps/dependencies-client-flink-1.18 index 456250bb961..be6b7d17fb4 100644 --- a/dev/deps/dependencies-client-flink-1.18 +++ b/dev/deps/dependencies-client-flink-1.18 @@ -30,6 +30,7 @@ jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar lz4-java/1.8.0//lz4-java-1.8.0.jar +maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/3.2.6//metrics-core-3.2.6.jar metrics-graphite/3.2.6//metrics-graphite-3.2.6.jar metrics-jvm/3.2.6//metrics-jvm-3.2.6.jar diff --git a/dev/deps/dependencies-client-mr b/dev/deps/dependencies-client-mr index cc8ef1ade7c..b3764b2eaf7 100644 --- a/dev/deps/dependencies-client-mr +++ b/dev/deps/dependencies-client-mr @@ -122,6 +122,7 @@ kerby-util/1.0.1//kerby-util-1.0.1.jar kerby-xdr/1.0.1//kerby-xdr-1.0.1.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar lz4-java/1.8.0//lz4-java-1.8.0.jar +maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/3.2.6//metrics-core-3.2.6.jar metrics-graphite/3.2.6//metrics-graphite-3.2.6.jar metrics-jvm/3.2.6//metrics-jvm-3.2.6.jar diff --git a/dev/deps/dependencies-client-spark-2.4 b/dev/deps/dependencies-client-spark-2.4 index c439d7e08fe..9beaa1f8ba7 100644 --- a/dev/deps/dependencies-client-spark-2.4 +++ b/dev/deps/dependencies-client-spark-2.4 @@ -30,6 +30,7 @@ jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar lz4-java/1.4.0//lz4-java-1.4.0.jar +maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/3.2.6//metrics-core-3.2.6.jar metrics-graphite/3.2.6//metrics-graphite-3.2.6.jar metrics-jvm/3.2.6//metrics-jvm-3.2.6.jar diff --git a/dev/deps/dependencies-client-spark-3.0 b/dev/deps/dependencies-client-spark-3.0 index 8ad78eccb8a..03618f785a4 100644 --- a/dev/deps/dependencies-client-spark-3.0 +++ b/dev/deps/dependencies-client-spark-3.0 @@ -30,6 +30,7 @@ jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar lz4-java/1.7.1//lz4-java-1.7.1.jar +maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/3.2.6//metrics-core-3.2.6.jar metrics-graphite/3.2.6//metrics-graphite-3.2.6.jar metrics-jvm/3.2.6//metrics-jvm-3.2.6.jar diff --git a/dev/deps/dependencies-client-spark-3.1 b/dev/deps/dependencies-client-spark-3.1 index 1cbd01b35fc..da8331b10b2 100644 --- a/dev/deps/dependencies-client-spark-3.1 +++ b/dev/deps/dependencies-client-spark-3.1 @@ -30,6 +30,7 @@ jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar lz4-java/1.7.1//lz4-java-1.7.1.jar +maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/3.2.6//metrics-core-3.2.6.jar metrics-graphite/3.2.6//metrics-graphite-3.2.6.jar metrics-jvm/3.2.6//metrics-jvm-3.2.6.jar diff --git a/dev/deps/dependencies-client-spark-3.2 b/dev/deps/dependencies-client-spark-3.2 index e83df4a7192..1feb81a8ed3 100644 --- a/dev/deps/dependencies-client-spark-3.2 +++ b/dev/deps/dependencies-client-spark-3.2 @@ -30,6 +30,7 @@ jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar lz4-java/1.7.1//lz4-java-1.7.1.jar +maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/3.2.6//metrics-core-3.2.6.jar metrics-graphite/3.2.6//metrics-graphite-3.2.6.jar metrics-jvm/3.2.6//metrics-jvm-3.2.6.jar diff --git a/dev/deps/dependencies-client-spark-3.3 b/dev/deps/dependencies-client-spark-3.3 index 456250bb961..be6b7d17fb4 100644 --- a/dev/deps/dependencies-client-spark-3.3 +++ b/dev/deps/dependencies-client-spark-3.3 @@ -30,6 +30,7 @@ jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar lz4-java/1.8.0//lz4-java-1.8.0.jar +maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/3.2.6//metrics-core-3.2.6.jar metrics-graphite/3.2.6//metrics-graphite-3.2.6.jar metrics-jvm/3.2.6//metrics-jvm-3.2.6.jar diff --git a/dev/deps/dependencies-client-spark-3.4 b/dev/deps/dependencies-client-spark-3.4 index 35698f48756..eabdee5a80e 100644 --- a/dev/deps/dependencies-client-spark-3.4 +++ b/dev/deps/dependencies-client-spark-3.4 @@ -30,6 +30,7 @@ jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar lz4-java/1.8.0//lz4-java-1.8.0.jar +maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/3.2.6//metrics-core-3.2.6.jar metrics-graphite/3.2.6//metrics-graphite-3.2.6.jar metrics-jvm/3.2.6//metrics-jvm-3.2.6.jar diff --git a/dev/deps/dependencies-client-spark-3.5 b/dev/deps/dependencies-client-spark-3.5 index 481caffcc68..4aa62aef53b 100644 --- a/dev/deps/dependencies-client-spark-3.5 +++ b/dev/deps/dependencies-client-spark-3.5 @@ -30,6 +30,7 @@ jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar lz4-java/1.8.0//lz4-java-1.8.0.jar +maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/3.2.6//metrics-core-3.2.6.jar metrics-graphite/3.2.6//metrics-graphite-3.2.6.jar metrics-jvm/3.2.6//metrics-jvm-3.2.6.jar diff --git a/dev/deps/dependencies-server b/dev/deps/dependencies-server index 89ac60462c3..1be1e05dd05 100644 --- a/dev/deps/dependencies-server +++ b/dev/deps/dependencies-server @@ -37,6 +37,7 @@ log4j-api/2.17.2//log4j-api-2.17.2.jar log4j-core/2.17.2//log4j-core-2.17.2.jar log4j-slf4j-impl/2.17.2//log4j-slf4j-impl-2.17.2.jar lz4-java/1.8.0//lz4-java-1.8.0.jar +maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar metrics-core/3.2.6//metrics-core-3.2.6.jar metrics-graphite/3.2.6//metrics-graphite-3.2.6.jar metrics-jvm/3.2.6//metrics-jvm-3.2.6.jar diff --git a/docs/configuration/worker.md b/docs/configuration/worker.md index a40cc9c5405..aa0e555de2f 100644 --- a/docs/configuration/worker.md +++ b/docs/configuration/worker.md @@ -70,6 +70,14 @@ license: | | celeborn.worker.graceful.shutdown.timeout | 600s | The worker's graceful shutdown timeout time. | 0.2.0 | | celeborn.worker.http.host | <localhost> | Worker's http host. | 0.4.0 | | celeborn.worker.http.port | 9096 | Worker's http port. | 0.4.0 | +| celeborn.worker.jvmQuake.check.interval | 1s | Interval of gc behavior checking for worker jvm quake. | 0.4.0 | +| celeborn.worker.jvmQuake.dump.enabled | true | Whether to heap dump for the maximum GC 'deficit' during worker jvm quake. | 0.4.0 | +| celeborn.worker.jvmQuake.dump.path | <tmp>/jvm-quake/dump/<pid> | The path of heap dump for the maximum GC 'deficit' during worker jvm quake. | 0.4.0 | +| celeborn.worker.jvmQuake.dump.threshold | 30s | The threshold of heap dump for the maximum GC 'deficit' which can be accumulated before jvmquake takes action. Meanwhile, there is no heap dump generated when dump threshold is greater than kill threshold. | 0.4.0 | +| celeborn.worker.jvmQuake.enabled | false | When true, Celeborn worker will start the jvm quake to monitor of gc behavior, which enables early detection of memory management issues and facilitates fast failure. | 0.4.0 | +| celeborn.worker.jvmQuake.exitCode | 502 | The exit code of system kill for the maximum GC 'deficit' during worker jvm quake. | 0.4.0 | +| celeborn.worker.jvmQuake.kill.threshold | 60s | The threshold of system kill for the maximum GC 'deficit' which can be accumulated before jvmquake takes action. | 0.4.0 | +| celeborn.worker.jvmQuake.runtimeWeight | 5.0 | The factor by which to multiply running JVM time, when weighing it against GCing time. 'Deficit' is accumulated as `gc_time - runtime * runtime_weight`, and is compared against threshold to determine whether to take action. | 0.4.0 | | celeborn.worker.monitor.disk.check.interval | 30s | Intervals between device monitor to check disk. | 0.3.0 | | celeborn.worker.monitor.disk.check.timeout | 30s | Timeout time for worker check device status. | 0.3.0 | | celeborn.worker.monitor.disk.checklist | readwrite,diskusage | Monitor type for disk, available items are: iohang, readwrite and diskusage. | 0.2.0 | diff --git a/licenses-binary/LISENCE-jdktools.txt b/licenses-binary/LISENCE-jdktools.txt new file mode 100644 index 00000000000..08cc1ab0a30 --- /dev/null +++ b/licenses-binary/LISENCE-jdktools.txt @@ -0,0 +1,21 @@ +The MIT License + +Copyright (c) 2016 Red Hat, Inc. + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. diff --git a/pom.xml b/pom.xml index 34dfaa680a2..e1c96c70437 100644 --- a/pom.xml +++ b/pom.xml @@ -122,6 +122,7 @@ -XX:+IgnoreUnrecognizedVMOptions + --add-exports=jdk.internal.jvmstat/sun.jvmstat.monitor=ALL-UNNAMED --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED @@ -1156,6 +1157,13 @@ 8 + + + com.github.olivergondza + maven-jdk-tools-wrapper + 0.1 + + diff --git a/project/CelebornBuild.scala b/project/CelebornBuild.scala index e940702cd3d..b4f56782a96 100644 --- a/project/CelebornBuild.scala +++ b/project/CelebornBuild.scala @@ -50,6 +50,7 @@ object Dependencies { val junitVersion = "4.13.2" val leveldbJniVersion = "1.8" val log4j2Version = "2.17.2" + val jdkToolsVersion = "0.1" val metricsVersion = "3.2.6" val mockitoVersion = "4.11.0" val nettyVersion = "4.1.93.Final" @@ -71,6 +72,7 @@ object Dependencies { val commonsIo = "commons-io" % "commons-io" % commonsIoVersion val commonsLang3 = "org.apache.commons" % "commons-lang3" % commonsLang3Version val commonsLogging = "commons-logging" % "commons-logging" % commonsLoggingVersion + val jdkTools = "com.github.olivergondza" % "maven-jdk-tools-wrapper" % jdkToolsVersion val findbugsJsr305 = "com.google.code.findbugs" % "jsr305" % findbugsVersion val guava = "com.google.guava" % "guava" % guavaVersion excludeAll( ExclusionRule("org.checkerframework", "checker-qual"), @@ -167,6 +169,7 @@ object CelebornCommonSettings { Test / javaOptions ++= Seq( "-Xmx4g", "-XX:+IgnoreUnrecognizedVMOptions", + "--add-exports=jdk.internal.jvmstat/sun.jvmstat.monitor=ALL-UNNAMED", "--add-opens=java.base/java.lang=ALL-UNNAMED", "--add-opens=java.base/java.lang.invoke=ALL-UNNAMED", "--add-opens=java.base/java.lang.reflect=ALL-UNNAMED", @@ -322,6 +325,7 @@ object CelebornCommon { Dependencies.commonsLang3, Dependencies.hadoopClientApi, Dependencies.hadoopClientRuntime, + Dependencies.jdkTools, Dependencies.ratisClient, Dependencies.ratisCommon, Dependencies.leveldbJniAll, diff --git a/project/JDKTools.scala b/project/JDKTools.scala new file mode 100644 index 00000000000..4dd2a5d616d --- /dev/null +++ b/project/JDKTools.scala @@ -0,0 +1,77 @@ +/* + * 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.build + +import java.io.FileNotFoundException + +import sbt._ +import sbt.Keys._ + +import scala.sys.process._ + +import scala.util.Try + +/** + * Represents the main plugin to add tooling settings to projects using libraries from the Scala Debugger project. + */ +object JDKTools extends AutoPlugin { + override def requires = plugins.JvmPlugin + + override def trigger = allRequirements + + override def projectSettings: Seq[Def.Setting[_]] = inConfig(Compile)(settings) + + lazy val settings = baseScalaDebuggerToolsSettings + + lazy val baseScalaDebuggerToolsSettings: Seq[Def.Setting[_]] = + if (System.getProperty("java.specification.version").startsWith("1.")) + Seq( + // JDK Dependency (just for sbt, must exist on classpath for execution, cannot be redistributed) + unmanagedJars += { + Attributed.blank(JavaTools) + } + ) + else + // on Java 9+, we don't need to do anything at all + Seq() + + // + // NOTE: Taken from Ensime Server project (when under BSD 3-clause) + // https://github.com/ensime/ensime-server/blob/master/project/EnsimeBuild.scala + // + // WORKAROUND: https://github.com/typelevel/scala/issues/75 + lazy val JavaTools: File = List( + // manual + sys.env.get("JAVA_HOME"), + sys.env.get("JDK_HOME"), + // osx + Try("/usr/libexec/java_home".!!).toOption, + // fallback + // sys.props.get("java.home") returns jre home for JDK8 + sys.props.get("java.home").map(new File(_).getParent), + sys.props.get("java.home") + ).flatten.map { n => + new File(n.trim + "/lib/tools.jar") + }.find(_.exists()).getOrElse( + throw new FileNotFoundException( + """ + |Could not automatically find the JDK/lib/tools.jar. + |You must explicitly set JDK_HOME or JAVA_HOME. + """.stripMargin) + ) +} diff --git a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/Worker.scala b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/Worker.scala index d8c7049e586..ecf867b3e0f 100644 --- a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/Worker.scala +++ b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/Worker.scala @@ -51,6 +51,7 @@ import org.apache.celeborn.service.deploy.worker.WorkerSource.ACTIVE_CONNECTION_ import org.apache.celeborn.service.deploy.worker.congestcontrol.CongestionController import org.apache.celeborn.service.deploy.worker.memory.{ChannelsLimiter, MemoryManager} import org.apache.celeborn.service.deploy.worker.memory.MemoryManager.ServingState +import org.apache.celeborn.service.deploy.worker.monitor.JVMQuake import org.apache.celeborn.service.deploy.worker.storage.{PartitionFilesSorter, StorageManager} private[celeborn] class Worker( @@ -275,6 +276,12 @@ private[celeborn] class Worker( private val userResourceConsumptions = JavaUtils.newConcurrentHashMap[UserIdentifier, (ResourceConsumption, Long)]() + private var jvmQuake: JVMQuake = _ + if (conf.workerJvmQuakeEnabled) { + jvmQuake = JVMQuake.create(conf, workerInfo.toUniqueId().replace(":", "-")) + jvmQuake.start() + } + workerSource.addGauge(WorkerSource.REGISTERED_SHUFFLE_COUNT) { () => workerInfo.getShuffleKeySet.size } @@ -431,6 +438,9 @@ private[celeborn] class Worker( if (!stopped) { logInfo("Stopping Worker.") + if (jvmQuake != null) { + jvmQuake.stop() + } if (sendHeartbeatTask != null) { if (exitKind == CelebornExitKind.WORKER_GRACEFUL_SHUTDOWN) { sendHeartbeatTask.cancel(false) diff --git a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/monitor/JVMQuake.scala b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/monitor/JVMQuake.scala new file mode 100644 index 00000000000..a88a39e22fc --- /dev/null +++ b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/monitor/JVMQuake.scala @@ -0,0 +1,197 @@ +/* + * 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.service.deploy.worker.monitor + +import java.io.File +import java.lang.management.ManagementFactory +import java.nio.file.Files +import java.util.UUID +import java.util.concurrent.{ScheduledExecutorService, TimeUnit} + +import com.google.common.annotations.VisibleForTesting +import com.sun.management.HotSpotDiagnosticMXBean +import sun.jvmstat.monitor._ + +import org.apache.celeborn.common.CelebornConf +import org.apache.celeborn.common.internal.Logging +import org.apache.celeborn.common.util.{ThreadUtils, Utils} + +/** + * The JVM quake provides granular monitoring of GC behavior, which enables early detection of memory management + * issues and facilitates fast failure. + * + * Note: The principle is in alignment with GC instability detection algorithm for jvmquake project of Netflix: + * https://github.com/Netflix-Skunkworks/jvmquake. + * + * @param conf Celeborn configuration with jvm quake config. + */ +class JVMQuake(conf: CelebornConf, uniqueId: String = UUID.randomUUID().toString) extends Logging { + + import JVMQuake._ + + val dumpFile = s"worker-quake-heapdump-$uniqueId.hprof" + var heapDumped: Boolean = false + + private[this] val enabled = conf.workerJvmQuakeEnabled + private[this] val checkInterval = conf.workerJvmQuakeCheckInterval + private[this] val runtimeWeight = conf.workerJvmQuakeRuntimeWeight + private[this] val dumpThreshold = conf.workerJvmQuakeDumpThreshold.toNanos + private[this] val killThreshold = conf.workerJvmQuakeKillThreshold.toNanos + private[this] val dumpEnabled = conf.workerJvmQuakeDumpEnabled + private[this] val dumpPath = conf.workerJvmQuakeDumpPath + private[this] val exitCode = conf.workerJvmQuakeExitCode + + private[this] var lastExitTime: Long = 0L + private[this] var lastGCTime: Long = 0L + private[this] var bucket: Long = 0L + private[this] var scheduler: ScheduledExecutorService = _ + + def start(): Unit = { + if (enabled) { + lastExitTime = getLastExitTime + lastGCTime = getLastGCTime + scheduler = ThreadUtils.newDaemonSingleThreadScheduledExecutor("jvm-quake") + scheduler.scheduleWithFixedDelay( + new Runnable() { + override def run(): Unit = { + JVMQuake.this.run() + } + }, + 0, + checkInterval, + TimeUnit.MILLISECONDS) + } + } + + def stop(): Unit = { + if (enabled) { + scheduler.shutdown() + } + } + + private def run(): Unit = { + val currentExitTime = getLastExitTime + val currentGCTime = getLastGCTime + val gcTime = currentGCTime - lastGCTime + val runTime = currentExitTime - lastExitTime - gcTime + + bucket = Math.max(0, bucket + gcTime - BigDecimal(runTime * runtimeWeight).toLong) + logDebug(s"Time: (gc time: ${Utils.msDurationToString(gcTime)}, execution time: ${Utils.msDurationToString(runTime)})") + logDebug( + s"Capacity: (bucket: $bucket, dump threshold: $dumpThreshold, kill threshold: $killThreshold)") + + if (bucket > dumpThreshold) { + logError(s"JVM GC has reached the threshold: bucket: $bucket, dumpThreshold: $dumpThreshold.") + if (shouldHeapDump) { + val savePath = getHeapDumpSavePath + val linkPath = getHeapDumpLinkPath + heapDump(savePath, linkPath) + } else if (bucket > killThreshold) { + logError(s"Exit JVM with $exitCode. JVM GC has reached the threshold: bucket: $bucket, killThreshold: $killThreshold.") + System.exit(exitCode) + } + } + lastExitTime = currentExitTime + lastGCTime = currentGCTime + } + + def shouldHeapDump: Boolean = { + dumpEnabled && !heapDumped + } + + @VisibleForTesting + def getHeapDumpSavePath: String = + dumpPath + + @VisibleForTesting + def getHeapDumpLinkPath: String = + s"${new File(dumpPath).getParent}/link/${Utils.getProcessId}" + + private def heapDump(savePath: String, linkPath: String, live: Boolean = false): Unit = { + val saveDir = new File(savePath) + if (!saveDir.exists()) { + saveDir.mkdirs() + } + val heapDump = new File(saveDir, dumpFile) + if (heapDump.exists()) { + // Each worker process only generates one heap dump. Skip when heap dump of worker already exists. + logWarning(s"Skip because heap dump of worker already exists: $heapDump.") + heapDumped = true + return + } + logInfo(s"Starting heap dump: $heapDump.") + ManagementFactory.newPlatformMXBeanProxy( + ManagementFactory.getPlatformMBeanServer, + "com.sun.management:type=HotSpotDiagnostic", + classOf[HotSpotDiagnosticMXBean]).dumpHeap(heapDump.getAbsolutePath, live) + val linkDir = new File(linkPath) + if (linkDir.exists()) { + // Each worker process only generates one heap dump. Skip when symbolic link of heap dump exists. + logWarning(s"Skip because symbolic link of heap dump exists: $linkPath.") + } else if (!linkDir.getParentFile.exists()) { + linkDir.getParentFile.mkdirs() + } + try { + Files.createSymbolicLink(linkDir.toPath, saveDir.toPath) + logInfo(s"Created symbolic link: $linkPath.") + } catch { + case e: Exception => + logError("Create symbolic link failed.", e) + } finally { + heapDumped = true + logInfo(s"Finished heap dump: $dumpFile.") + } + } +} + +object JVMQuake { + + private[this] var quake: JVMQuake = _ + + def create(conf: CelebornConf, uniqueId: String): JVMQuake = { + set(new JVMQuake(conf, uniqueId)) + quake + } + + def get: JVMQuake = { + quake + } + + def set(quake: JVMQuake): Unit = { + this.quake = quake + } + + private[this] lazy val monitoredVm: MonitoredVm = { + val host = MonitoredHost.getMonitoredHost(new HostIdentifier("localhost")) + host.getMonitoredVm(new VmIdentifier("local://%s@localhost".format(Utils.getProcessId))) + } + + private[this] lazy val ygcExitTimeMonitor: Monitor = + monitoredVm.findByName("sun.gc.collector.0.lastExitTime") + private[this] lazy val fgcExitTimeMonitor: Monitor = + monitoredVm.findByName("sun.gc.collector.1.lastExitTime") + private[this] lazy val ygcTimeMonitor: Monitor = monitoredVm.findByName("sun.gc.collector.0.time") + private[this] lazy val fgcTimeMonitor: Monitor = monitoredVm.findByName("sun.gc.collector.1.time") + + private def getLastExitTime: Long = Math.max( + ygcExitTimeMonitor.getValue.asInstanceOf[Long], + fgcExitTimeMonitor.getValue.asInstanceOf[Long]) + + private def getLastGCTime: Long = + ygcTimeMonitor.getValue.asInstanceOf[Long] + fgcTimeMonitor.getValue.asInstanceOf[Long] +} diff --git a/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/monitor/JVMQuakeSuite.scala b/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/monitor/JVMQuakeSuite.scala new file mode 100644 index 00000000000..7ef49f22588 --- /dev/null +++ b/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/monitor/JVMQuakeSuite.scala @@ -0,0 +1,69 @@ +/* + * 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.service.deploy.worker.monitor + +import java.io.File + +import scala.collection.mutable.ArrayBuffer + +import org.junit.Assert.assertTrue + +import org.apache.celeborn.CelebornFunSuite +import org.apache.celeborn.common.CelebornConf +import org.apache.celeborn.common.CelebornConf._ +import org.apache.celeborn.common.util.JavaUtils + +class JVMQuakeSuite extends CelebornFunSuite { + + private val allocation = new ArrayBuffer[Array[Byte]]() + + override def afterEach(): Unit = { + allocation.clear() + System.gc() + } + + test("[CELEBORN-1092] Introduce JVM monitoring in Celeborn Worker using JVMQuake") { + val quake = new JVMQuake(new CelebornConf().set(WORKER_JVM_QUAKE_ENABLED.key, "true") + .set(WORKER_JVM_QUAKE_RUNTIME_WEIGHT.key, "1") + .set(WORKER_JVM_QUAKE_DUMP_THRESHOLD.key, "1s") + .set(WORKER_JVM_QUAKE_KILL_THRESHOLD.key, "2s")) + quake.start() + allocateMemory(quake) + quake.stop() + + assertTrue(quake.heapDumped) + val heapDump = new File(s"${quake.getHeapDumpSavePath}/${quake.dumpFile}") + assert(heapDump.exists()) + JavaUtils.deleteRecursively(heapDump) + JavaUtils.deleteRecursively(new File(quake.getHeapDumpLinkPath)) + } + + def allocateMemory(quake: JVMQuake): Unit = { + val capacity = 1024 * 100 + while (allocation.size * capacity < Runtime.getRuntime.maxMemory / 4) { + val bytes = new Array[Byte](capacity) + allocation.append(bytes) + } + while (quake.shouldHeapDump) { + for (index <- allocation.indices) { + val bytes = new Array[Byte](capacity) + allocation(index) = bytes + } + } + } +} From 98cb941c844023bf946bc215c91b4ddcf587aa28 Mon Sep 17 00:00:00 2001 From: jiaoqingbo <1178404354@qq.com> Date: Tue, 28 Nov 2023 20:54:24 +0800 Subject: [PATCH 27/31] [CELEBORN-1142] clear shuffleIdCache in shutdown method of ShuffleClientImpl ### What changes were proposed in this pull request? As Title ### Why are the changes needed? As Title ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? PASS GA Closes #2114 from jiaoqingbo/1142. Authored-by: jiaoqingbo <1178404354@qq.com> Signed-off-by: Fu Chen --- .../main/java/org/apache/celeborn/client/ShuffleClientImpl.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java b/client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java index b1a37751720..fe7a403ef9f 100644 --- a/client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java +++ b/client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java @@ -1676,6 +1676,8 @@ public void shutdown() { if (null != lifecycleManagerRef) { lifecycleManagerRef = null; } + + shuffleIdCache.clear(); pushExcludedWorkers.clear(); fetchExcludedWorkers.clear(); logger.warn("Shuffle client has been shutdown!"); From d37bb4c6824a170a62cbac20321910e8b57d186c Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 29 Nov 2023 21:06:23 +0800 Subject: [PATCH 28/31] [CELEBORN-1131] Add Client/Server bootstrap framework to transport layer ### What changes were proposed in this pull request? This adds the client/server bootstrap framework to transport layer in Celeborn. This is copied from Spark. It is part of the epic: https://issues.apache.org/jira/browse/CELEBORN-1011. ### Why are the changes needed? The changes are needed for adding authentication to Celeborn. See [CELEBORN-1011](https://issues.apache.org/jira/browse/CELEBORN-1011). ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Part of a larger change which has tests Closes #2120 from otterc/CELEBORN-1131-PR1. Lead-authored-by: Chandni Singh Co-authored-by: otterc Signed-off-by: Shuang --- .../network/FlinkTransportClientFactory.java | 3 +- .../common/network/TransportContext.java | 37 +++++++++++++---- .../client/TransportClientBootstrap.java | 40 +++++++++++++++++++ .../client/TransportClientFactory.java | 33 ++++++++++++++- .../network/server/TransportServer.java | 24 +++++++++-- .../server/TransportServerBootstrap.java | 36 +++++++++++++++++ 6 files changed, 159 insertions(+), 14 deletions(-) create mode 100644 common/src/main/java/org/apache/celeborn/common/network/client/TransportClientBootstrap.java create mode 100644 common/src/main/java/org/apache/celeborn/common/network/server/TransportServerBootstrap.java diff --git a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/network/FlinkTransportClientFactory.java b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/network/FlinkTransportClientFactory.java index e9e716ef337..68e418a54e9 100644 --- a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/network/FlinkTransportClientFactory.java +++ b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/network/FlinkTransportClientFactory.java @@ -18,6 +18,7 @@ package org.apache.celeborn.plugin.flink.network; import java.io.IOException; +import java.util.Collections; import java.util.concurrent.ConcurrentHashMap; import java.util.function.Supplier; @@ -39,7 +40,7 @@ public class FlinkTransportClientFactory extends TransportClientFactory { private final int fetchMaxRetries; public FlinkTransportClientFactory(TransportContext context, int fetchMaxRetries) { - super(context); + super(context, Collections.emptyList()); bufferSuppliers = JavaUtils.newConcurrentHashMap(); this.fetchMaxRetries = fetchMaxRetries; this.pooledAllocator = new UnpooledByteBufAllocator(true); diff --git a/common/src/main/java/org/apache/celeborn/common/network/TransportContext.java b/common/src/main/java/org/apache/celeborn/common/network/TransportContext.java index 50bb5cac145..c8796ea32b3 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/TransportContext.java +++ b/common/src/main/java/org/apache/celeborn/common/network/TransportContext.java @@ -17,6 +17,9 @@ package org.apache.celeborn.common.network; +import java.util.Collections; +import java.util.List; + import io.netty.channel.Channel; import io.netty.channel.ChannelDuplexHandler; import io.netty.channel.ChannelInboundHandlerAdapter; @@ -27,6 +30,7 @@ import org.apache.celeborn.common.metrics.source.AbstractSource; import org.apache.celeborn.common.network.client.TransportClient; +import org.apache.celeborn.common.network.client.TransportClientBootstrap; import org.apache.celeborn.common.network.client.TransportClientFactory; import org.apache.celeborn.common.network.client.TransportResponseHandler; import org.apache.celeborn.common.network.protocol.MessageEncoder; @@ -93,35 +97,52 @@ public TransportContext(TransportConf conf, BaseMessageHandler msgHandler) { this(conf, msgHandler, false, false, null); } + public TransportClientFactory createClientFactory(List bootstraps) { + return new TransportClientFactory(this, bootstraps); + } + public TransportClientFactory createClientFactory() { - return new TransportClientFactory(this); + return createClientFactory(Collections.emptyList()); } /** Create a server which will attempt to bind to a specific host and port. */ public TransportServer createServer(String host, int port) { - return new TransportServer(this, host, port, source); + return new TransportServer(this, host, port, source, msgHandler, Collections.emptyList()); + } + + public TransportServer createServer( + String host, int port, List bootstraps) { + return new TransportServer(this, host, port, source, msgHandler, bootstraps); } public TransportServer createServer(int port) { - return createServer(null, port); + return createServer(null, port, Collections.emptyList()); } /** For Suite only */ public TransportServer createServer() { - return createServer(null, 0); + return createServer(null, 0, Collections.emptyList()); } - public TransportChannelHandler initializePipeline(SocketChannel channel) { - return initializePipeline(channel, new TransportFrameDecoder()); + public TransportChannelHandler initializePipeline( + SocketChannel channel, ChannelInboundHandlerAdapter decoder) { + return initializePipeline(channel, decoder, msgHandler); } public TransportChannelHandler initializePipeline( - SocketChannel channel, ChannelInboundHandlerAdapter decoder) { + SocketChannel channel, BaseMessageHandler resolvedMsgHandler) { + return initializePipeline(channel, new TransportFrameDecoder(), resolvedMsgHandler); + } + + public TransportChannelHandler initializePipeline( + SocketChannel channel, + ChannelInboundHandlerAdapter decoder, + BaseMessageHandler resolvedMsgHandler) { try { if (channelsLimiter != null) { channel.pipeline().addLast("limiter", channelsLimiter); } - TransportChannelHandler channelHandler = createChannelHandler(channel, msgHandler); + TransportChannelHandler channelHandler = createChannelHandler(channel, resolvedMsgHandler); channel .pipeline() .addLast("encoder", ENCODER) diff --git a/common/src/main/java/org/apache/celeborn/common/network/client/TransportClientBootstrap.java b/common/src/main/java/org/apache/celeborn/common/network/client/TransportClientBootstrap.java new file mode 100644 index 00000000000..bdad119fcfa --- /dev/null +++ b/common/src/main/java/org/apache/celeborn/common/network/client/TransportClientBootstrap.java @@ -0,0 +1,40 @@ +/* + * 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.network.client; + +import io.netty.channel.Channel; + +/** + * A bootstrap which is executed on a TransportClient before it is returned to the user. This + * enables an initial exchange of information (e.g., SASL authentication tokens) on a once-per- + * connection basis. + * + *

Since connections (and TransportClients) are reused as much as possible, it is generally + * reasonable to perform an expensive bootstrapping operation, as they often share a lifespan with + * the JVM itself. + */ +public interface TransportClientBootstrap { + /** + * Performs the bootstrapping operation, throwing an exception on failure. + * + * @param client the transport client to bootstrap + * @param channel the associated channel with the transport client + * @throws RuntimeException + */ + void doBootstrap(TransportClient client, Channel channel) throws RuntimeException; +} diff --git a/common/src/main/java/org/apache/celeborn/common/network/client/TransportClientFactory.java b/common/src/main/java/org/apache/celeborn/common/network/client/TransportClientFactory.java index ba1a0dde972..a4d4d751518 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/client/TransportClientFactory.java +++ b/common/src/main/java/org/apache/celeborn/common/network/client/TransportClientFactory.java @@ -21,12 +21,15 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.net.SocketAddress; +import java.util.List; import java.util.Random; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; import io.netty.bootstrap.Bootstrap; import io.netty.buffer.ByteBufAllocator; import io.netty.channel.*; @@ -39,6 +42,7 @@ import org.apache.celeborn.common.network.server.TransportChannelHandler; import org.apache.celeborn.common.network.util.*; import org.apache.celeborn.common.util.JavaUtils; +import org.apache.celeborn.common.util.Utils; /** * Factory for creating {@link TransportClient}s by using createClient. @@ -68,6 +72,7 @@ private static class ClientPool { private static final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class); private final TransportContext context; + private final List clientBootstraps; private final ConcurrentHashMap connectionPool; /** Random number generator for picking connections between peers. */ @@ -84,9 +89,11 @@ private static class ClientPool { private EventLoopGroup workerGroup; protected ByteBufAllocator pooledAllocator; - public TransportClientFactory(TransportContext context) { + public TransportClientFactory( + TransportContext context, List clientBootstraps) { this.context = Preconditions.checkNotNull(context); TransportConf conf = context.getConf(); + this.clientBootstraps = Lists.newArrayList(Preconditions.checkNotNull(clientBootstraps)); this.connectionPool = JavaUtils.newConcurrentHashMap(); this.numConnectionsPerPeer = conf.numConnectionsPerPeer(); this.connectTimeoutMs = conf.connectTimeoutMs(); @@ -241,6 +248,7 @@ public void initChannel(SocketChannel ch) { }); // Connect to the remote server + long preConnect = System.nanoTime(); ChannelFuture cf = bootstrap.connect(address); if (!cf.await(connectTimeoutMs)) { throw new CelebornIOException( @@ -250,10 +258,31 @@ public void initChannel(SocketChannel ch) { } TransportClient client = clientRef.get(); + Channel channel = channelRef.get(); assert client != null : "Channel future completed successfully with null client"; + // Execute any client bootstraps synchronously before marking the Client as successful. + long preBootstrap = System.nanoTime(); + logger.debug("Running bootstraps for {} ...", address); + try { + for (TransportClientBootstrap clientBootstrap : clientBootstraps) { + clientBootstrap.doBootstrap(client, channel); + } + } catch (Exception e) { // catch non-RuntimeExceptions too as bootstrap may be written in Scala + long bootstrapTime = System.nanoTime() - preBootstrap; + logger.error( + "Exception while bootstrapping client after {}", + Utils.nanoDurationToString(bootstrapTime), + e); + client.close(); + throw Throwables.propagate(e); + } + long postBootstrap = System.nanoTime(); logger.debug( - "Connection from {} to {} successful", client.getChannel().localAddress(), address); + "Successfully created connection to {} after {} ({} spent in bootstraps)", + address, + Utils.nanoDurationToString(postBootstrap - preConnect), + Utils.nanoDurationToString(postBootstrap - preBootstrap)); return client; } diff --git a/common/src/main/java/org/apache/celeborn/common/network/server/TransportServer.java b/common/src/main/java/org/apache/celeborn/common/network/server/TransportServer.java index 73bb38bac83..53f7b716999 100644 --- a/common/src/main/java/org/apache/celeborn/common/network/server/TransportServer.java +++ b/common/src/main/java/org/apache/celeborn/common/network/server/TransportServer.java @@ -19,8 +19,11 @@ import java.io.Closeable; import java.net.InetSocketAddress; +import java.util.List; import java.util.concurrent.TimeUnit; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; import io.netty.bootstrap.ServerBootstrap; import io.netty.buffer.PooledByteBufAllocator; import io.netty.channel.ChannelFuture; @@ -44,17 +47,25 @@ public class TransportServer implements Closeable { private final TransportContext context; private final TransportConf conf; - + private final BaseMessageHandler appMessageHandler; + private final List bootstraps; private ServerBootstrap bootstrap; private ChannelFuture channelFuture; private AbstractSource source; private int port = -1; public TransportServer( - TransportContext context, String hostToBind, int portToBind, AbstractSource source) { + TransportContext context, + String hostToBind, + int portToBind, + AbstractSource source, + BaseMessageHandler appMessageHandler, + List bootstraps) { this.context = context; this.conf = context.getConf(); this.source = source; + this.appMessageHandler = Preconditions.checkNotNull(appMessageHandler); + this.bootstraps = Lists.newArrayList(Preconditions.checkNotNull(bootstraps)); boolean shouldClose = true; try { @@ -124,7 +135,14 @@ protected void initializeChannel(ServerBootstrap bootstrap) { new ChannelInitializer() { @Override protected void initChannel(SocketChannel ch) { - context.initializePipeline(ch); + BaseMessageHandler baseHandler = appMessageHandler; + logger.debug("number of bootstraps {}", bootstraps.size()); + for (TransportServerBootstrap bootstrap : bootstraps) { + logger.debug( + "Adding bootstrap to TransportServer {}.", bootstrap.getClass().getName()); + baseHandler = bootstrap.doBootstrap(ch, baseHandler); + } + context.initializePipeline(ch, baseHandler); } }); } diff --git a/common/src/main/java/org/apache/celeborn/common/network/server/TransportServerBootstrap.java b/common/src/main/java/org/apache/celeborn/common/network/server/TransportServerBootstrap.java new file mode 100644 index 00000000000..cc367a05f5f --- /dev/null +++ b/common/src/main/java/org/apache/celeborn/common/network/server/TransportServerBootstrap.java @@ -0,0 +1,36 @@ +/* + * 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.network.server; + +import io.netty.channel.Channel; + +/** + * A bootstrap which is executed on a TransportServer's client channel once a client connects to the + * server. This allows customizing the client channel to allow for things such as SASL + * authentication. + */ +public interface TransportServerBootstrap { + /** + * Customizes the channel to include new features, if needed. + * + * @param channel The connected channel opened by the client. + * @param baseMessageHandler The RPC handler for the server. + * @return The base message handler to use for the channel. + */ + BaseMessageHandler doBootstrap(Channel channel, BaseMessageHandler baseMessageHandler); +} From 8a15396cb64140fff83f157f5858862ea44a7205 Mon Sep 17 00:00:00 2001 From: exmy Date: Thu, 30 Nov 2023 18:56:03 +0800 Subject: [PATCH 29/31] [CELEBORN-1145] Separate clientPushBufferMaxSize from CelebornInputStreamImpl ### What changes were proposed in this pull request? The `clientPushBufferMaxSize` config is also used by `CelebornInputStreamImpl`, it's a config about push side and should not be used by fetch side. This pr introduces a fetch config to replace it. ### Why are the changes needed? As above ### Does this PR introduce _any_ user-facing change? Yes, a new config `celeborn.client.fetch.buffer.size` is introduced. ### How was this patch tested? Pass CI Closes #2118 from exmy/celeborn-1145. Authored-by: exmy Signed-off-by: Cheng Pan --- .../celeborn/client/read/CelebornInputStream.java | 8 ++++---- .../org/apache/celeborn/common/CelebornConf.scala | 11 +++++++++++ docs/configuration/client.md | 1 + 3 files changed, 16 insertions(+), 4 deletions(-) diff --git a/client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java b/client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java index 8484406c6cb..bb1e95ce9a9 100644 --- a/client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java +++ b/client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java @@ -182,15 +182,15 @@ private static final class CelebornInputStreamImpl extends CelebornInputStream { this.fetchExcludedWorkerExpireTimeout = conf.clientFetchExcludedWorkerExpireTimeout(); this.fetchExcludedWorkers = fetchExcludedWorkers; - int blockSize = conf.clientPushBufferMaxSize(); + int bufferSize = conf.clientFetchBufferSize(); if (shuffleCompressionEnabled) { int headerLen = Decompressor.getCompressionHeaderLength(conf); - blockSize += headerLen; - compressedBuf = new byte[blockSize]; + bufferSize += headerLen; + compressedBuf = new byte[bufferSize]; decompressor = Decompressor.getDecompressor(conf); } - rawDataBuf = new byte[blockSize]; + rawDataBuf = new byte[bufferSize]; if (conf.clientPushReplicateEnabled()) { fetchChunkMaxRetry = conf.clientFetchMaxRetriesForEachReplica() * 2; diff --git a/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala b/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala index 422d0f55521..75742b48aae 100644 --- a/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala +++ b/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala @@ -793,6 +793,7 @@ class CelebornConf(loadDefaults: Boolean) extends Cloneable with Logging with Se // Shuffle Client Fetch // // ////////////////////////////////////////////////////// def clientFetchTimeoutMs: Long = get(CLIENT_FETCH_TIMEOUT) + def clientFetchBufferSize: Int = get(CLIENT_FETCH_BUFFER_SIZE).toInt def clientFetchMaxReqsInFlight: Int = get(CLIENT_FETCH_MAX_REQS_IN_FLIGHT) def clientFetchMaxRetriesForEachReplica: Int = get(CLIENT_FETCH_MAX_RETRIES_FOR_EACH_REPLICA) def clientFetchThrowsFetchFailure: Boolean = get(CLIENT_FETCH_THROWS_FETCH_FAILURE) @@ -3299,6 +3300,16 @@ object CelebornConf extends Logging { .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("600s") + val CLIENT_FETCH_BUFFER_SIZE: ConfigEntry[Long] = + buildConf("celeborn.client.fetch.buffer.size") + .categories("client") + .version("0.4.0") + .doc("Size of reducer partition buffer memory for shuffle reader. The fetched data " + + "will be buffered in memory before consuming. For performance consideration keep " + + s"this buffer size not less than `${CLIENT_PUSH_BUFFER_MAX_SIZE.key}`.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("64k") + val CLIENT_FETCH_MAX_REQS_IN_FLIGHT: ConfigEntry[Int] = buildConf("celeborn.client.fetch.maxReqsInFlight") .withAlternative("celeborn.fetch.maxReqsInFlight") diff --git a/docs/configuration/client.md b/docs/configuration/client.md index 4871ac5509a..c46103e98f4 100644 --- a/docs/configuration/client.md +++ b/docs/configuration/client.md @@ -26,6 +26,7 @@ license: | | celeborn.client.eagerlyCreateInputStream.threads | 32 | Threads count for streamCreatorPool in CelebornShuffleReader. | 0.3.1 | | celeborn.client.excludePeerWorkerOnFailure.enabled | true | When true, Celeborn will exclude partition's peer worker on failure when push data to replica failed. | 0.3.0 | | celeborn.client.excludedWorker.expireTimeout | 180s | Timeout time for LifecycleManager to clear reserved excluded worker. Default to be 1.5 * `celeborn.master.heartbeat.worker.timeout`to cover worker heartbeat timeout check period | 0.3.0 | +| celeborn.client.fetch.buffer.size | 64k | Size of reducer partition buffer memory for shuffle reader. The fetched data will be buffered in memory before consuming. For performance consideration keep this buffer size not less than `celeborn.client.push.buffer.max.size`. | 0.4.0 | | celeborn.client.fetch.dfsReadChunkSize | 8m | Max chunk size for DfsPartitionReader. | 0.3.1 | | celeborn.client.fetch.excludeWorkerOnFailure.enabled | false | Whether to enable shuffle client-side fetch exclude workers on failure. | 0.3.0 | | celeborn.client.fetch.excludedWorker.expireTimeout | <value of celeborn.client.excludedWorker.expireTimeout> | ShuffleClient is a static object, it will be used in the whole lifecycle of Executor,We give a expire time for excluded workers to avoid a transient worker issues. | 0.3.0 | From 89b6cac5abd43c1a2bd89d5fda137222006e2896 Mon Sep 17 00:00:00 2001 From: sychen Date: Fri, 1 Dec 2023 15:41:04 +0800 Subject: [PATCH 30/31] [CELEBORN-1113] Bump Hadoop client version from 3.2.4 to 3.3.6 ### What changes were proposed in this pull request? ### Why are the changes needed? [[HADOOP-17098](https://issues.apache.org/jira/browse/HADOOP-17098)] Reduce Guava dependency in Hadoop source code The higher version of hadoop client removes many guava-related methods, which avoids some conflicts on guava. `hadoop-client-api` 3.3.6 `hadoop-client-runtime` 3.3.6 ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? Closes #2077 from cxzl25/CELEBORN-1113. Authored-by: sychen Signed-off-by: mingji --- client-flink/common/pom.xml | 6 ++ client-flink/flink-1.14/pom.xml | 6 ++ client-flink/flink-1.15/pom.xml | 6 ++ client-flink/flink-1.17/pom.xml | 6 ++ client-flink/flink-1.18/pom.xml | 6 ++ client-mr/mr/pom.xml | 11 ++ client-spark/common/pom.xml | 12 +++ client-spark/spark-3/pom.xml | 12 +++ dev/deps/dependencies-client-flink-1.14 | 6 +- dev/deps/dependencies-client-flink-1.15 | 6 +- dev/deps/dependencies-client-flink-1.17 | 6 +- dev/deps/dependencies-client-flink-1.18 | 6 +- dev/deps/dependencies-client-mr | 133 +++++++++++++----------- dev/deps/dependencies-client-spark-2.4 | 6 +- dev/deps/dependencies-client-spark-3.0 | 6 +- dev/deps/dependencies-client-spark-3.1 | 6 +- dev/deps/dependencies-client-spark-3.2 | 6 +- dev/deps/dependencies-client-spark-3.3 | 6 +- dev/deps/dependencies-client-spark-3.4 | 6 +- dev/deps/dependencies-client-spark-3.5 | 6 +- dev/deps/dependencies-server | 6 +- pom.xml | 2 +- project/CelebornBuild.scala | 3 +- 23 files changed, 178 insertions(+), 97 deletions(-) diff --git a/client-flink/common/pom.xml b/client-flink/common/pom.xml index 9cf960a2aff..499e086e0e8 100644 --- a/client-flink/common/pom.xml +++ b/client-flink/common/pom.xml @@ -43,6 +43,12 @@ org.apache.flink flink-runtime provided + + + org.xerial.snappy + snappy-java + + org.mockito diff --git a/client-flink/flink-1.14/pom.xml b/client-flink/flink-1.14/pom.xml index 903774e07e9..183cf6a6589 100644 --- a/client-flink/flink-1.14/pom.xml +++ b/client-flink/flink-1.14/pom.xml @@ -48,6 +48,12 @@ org.apache.flink flink-runtime provided + + + org.xerial.snappy + snappy-java + + diff --git a/client-flink/flink-1.15/pom.xml b/client-flink/flink-1.15/pom.xml index ec9f8ac7097..f11e34be05d 100644 --- a/client-flink/flink-1.15/pom.xml +++ b/client-flink/flink-1.15/pom.xml @@ -48,6 +48,12 @@ org.apache.flink flink-runtime provided + + + org.xerial.snappy + snappy-java + + diff --git a/client-flink/flink-1.17/pom.xml b/client-flink/flink-1.17/pom.xml index 96d8ff86317..8c86029874f 100644 --- a/client-flink/flink-1.17/pom.xml +++ b/client-flink/flink-1.17/pom.xml @@ -48,6 +48,12 @@ org.apache.flink flink-runtime provided + + + org.xerial.snappy + snappy-java + + diff --git a/client-flink/flink-1.18/pom.xml b/client-flink/flink-1.18/pom.xml index 1d51f8dc8ed..339b0b83291 100644 --- a/client-flink/flink-1.18/pom.xml +++ b/client-flink/flink-1.18/pom.xml @@ -48,6 +48,12 @@ org.apache.flink flink-runtime provided + + + org.xerial.snappy + snappy-java + + diff --git a/client-mr/mr/pom.xml b/client-mr/mr/pom.xml index 8b16614cb70..de9fe1e85cf 100644 --- a/client-mr/mr/pom.xml +++ b/client-mr/mr/pom.xml @@ -44,6 +44,17 @@ hadoop-mapreduce-client-app ${hadoop.version} + + org.slf4j + slf4j-reload4j + 1.7.36 + + + ch.qos.reload4j + reload4j + + + diff --git a/client-spark/common/pom.xml b/client-spark/common/pom.xml index 07e7699f9ad..c1165d621fe 100644 --- a/client-spark/common/pom.xml +++ b/client-spark/common/pom.xml @@ -43,6 +43,12 @@ org.apache.spark spark-core_${scala.binary.version} provided + + + org.xerial.snappy + snappy-java + + @@ -57,6 +63,12 @@ spark-core_${scala.binary.version} test-jar test + + + org.xerial.snappy + snappy-java + + org.apache.spark diff --git a/client-spark/spark-3/pom.xml b/client-spark/spark-3/pom.xml index 14bd07e0192..084e6e73160 100644 --- a/client-spark/spark-3/pom.xml +++ b/client-spark/spark-3/pom.xml @@ -54,6 +54,12 @@ org.apache.spark spark-core_${scala.binary.version} provided + + + org.xerial.snappy + snappy-java + + org.apache.spark @@ -73,6 +79,12 @@ spark-core_${scala.binary.version} test-jar test + + + org.xerial.snappy + snappy-java + + org.mockito diff --git a/dev/deps/dependencies-client-flink-1.14 b/dev/deps/dependencies-client-flink-1.14 index be6b7d17fb4..111bf56cf9a 100644 --- a/dev/deps/dependencies-client-flink-1.14 +++ b/dev/deps/dependencies-client-flink-1.14 @@ -22,9 +22,8 @@ commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar failureaccess/1.0.1//failureaccess-1.0.1.jar guava/32.1.3-jre//guava-32.1.3-jre.jar -hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar -hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar -htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar +hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar +hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jcl-over-slf4j/1.7.36//jcl-over-slf4j-1.7.36.jar jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar @@ -77,4 +76,5 @@ scala-reflect/2.12.15//scala-reflect-2.12.15.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/1.33//snakeyaml-1.33.jar +snappy-java/1.1.8.2//snappy-java-1.1.8.2.jar zstd-jni/1.5.2-1//zstd-jni-1.5.2-1.jar diff --git a/dev/deps/dependencies-client-flink-1.15 b/dev/deps/dependencies-client-flink-1.15 index be6b7d17fb4..111bf56cf9a 100644 --- a/dev/deps/dependencies-client-flink-1.15 +++ b/dev/deps/dependencies-client-flink-1.15 @@ -22,9 +22,8 @@ commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar failureaccess/1.0.1//failureaccess-1.0.1.jar guava/32.1.3-jre//guava-32.1.3-jre.jar -hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar -hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar -htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar +hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar +hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jcl-over-slf4j/1.7.36//jcl-over-slf4j-1.7.36.jar jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar @@ -77,4 +76,5 @@ scala-reflect/2.12.15//scala-reflect-2.12.15.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/1.33//snakeyaml-1.33.jar +snappy-java/1.1.8.2//snappy-java-1.1.8.2.jar zstd-jni/1.5.2-1//zstd-jni-1.5.2-1.jar diff --git a/dev/deps/dependencies-client-flink-1.17 b/dev/deps/dependencies-client-flink-1.17 index be6b7d17fb4..111bf56cf9a 100644 --- a/dev/deps/dependencies-client-flink-1.17 +++ b/dev/deps/dependencies-client-flink-1.17 @@ -22,9 +22,8 @@ commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar failureaccess/1.0.1//failureaccess-1.0.1.jar guava/32.1.3-jre//guava-32.1.3-jre.jar -hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar -hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar -htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar +hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar +hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jcl-over-slf4j/1.7.36//jcl-over-slf4j-1.7.36.jar jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar @@ -77,4 +76,5 @@ scala-reflect/2.12.15//scala-reflect-2.12.15.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/1.33//snakeyaml-1.33.jar +snappy-java/1.1.8.2//snappy-java-1.1.8.2.jar zstd-jni/1.5.2-1//zstd-jni-1.5.2-1.jar diff --git a/dev/deps/dependencies-client-flink-1.18 b/dev/deps/dependencies-client-flink-1.18 index be6b7d17fb4..111bf56cf9a 100644 --- a/dev/deps/dependencies-client-flink-1.18 +++ b/dev/deps/dependencies-client-flink-1.18 @@ -22,9 +22,8 @@ commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar failureaccess/1.0.1//failureaccess-1.0.1.jar guava/32.1.3-jre//guava-32.1.3-jre.jar -hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar -hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar -htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar +hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar +hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jcl-over-slf4j/1.7.36//jcl-over-slf4j-1.7.36.jar jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar @@ -77,4 +76,5 @@ scala-reflect/2.12.15//scala-reflect-2.12.15.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/1.33//snakeyaml-1.33.jar +snappy-java/1.1.8.2//snappy-java-1.1.8.2.jar zstd-jni/1.5.2-1//zstd-jni-1.5.2-1.jar diff --git a/dev/deps/dependencies-client-mr b/dev/deps/dependencies-client-mr index b3764b2eaf7..d1e8720c434 100644 --- a/dev/deps/dependencies-client-mr +++ b/dev/deps/dependencies-client-mr @@ -17,28 +17,28 @@ HikariCP-java7/2.4.12//HikariCP-java7-2.4.12.jar RoaringBitmap/0.9.32//RoaringBitmap-0.9.32.jar -accessors-smart/2.4.7//accessors-smart-2.4.7.jar aopalliance/1.0//aopalliance-1.0.jar -asm/9.1//asm-9.1.jar +asm-commons/9.4//asm-commons-9.4.jar +asm-tree/9.4//asm-tree-9.4.jar audience-annotations/0.5.0//audience-annotations-0.5.0.jar avro/1.7.7//avro-1.7.7.jar commons-beanutils/1.9.4//commons-beanutils-1.9.4.jar commons-cli/1.2//commons-cli-1.2.jar -commons-codec/1.11//commons-codec-1.11.jar +commons-codec/1.15//commons-codec-1.15.jar commons-collections/3.2.2//commons-collections-3.2.2.jar commons-compress/1.4.1//commons-compress-1.4.1.jar -commons-configuration2/2.1.1//commons-configuration2-2.1.1.jar +commons-configuration2/2.8.0//commons-configuration2-2.8.0.jar commons-crypto/1.0.0//commons-crypto-1.0.0.jar commons-daemon/1.0.13//commons-daemon-1.0.13.jar commons-io/2.13.0//commons-io-2.13.0.jar commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar commons-math3/3.1.1//commons-math3-3.1.1.jar -commons-net/3.6//commons-net-3.6.jar -commons-text/1.4//commons-text-1.4.jar -curator-client/2.13.0//curator-client-2.13.0.jar -curator-framework/2.13.0//curator-framework-2.13.0.jar -curator-recipes/2.13.0//curator-recipes-2.13.0.jar +commons-net/3.9.0//commons-net-3.9.0.jar +commons-text/1.10.0//commons-text-1.10.0.jar +curator-client/5.2.0//curator-client-5.2.0.jar +curator-framework/5.2.0//curator-framework-5.2.0.jar +curator-recipes/5.2.0//curator-recipes-5.2.0.jar dnsjava/2.1.7//dnsjava-2.1.7.jar ehcache/3.3.1//ehcache-3.3.1.jar failureaccess/1.0.1//failureaccess-1.0.1.jar @@ -47,61 +47,69 @@ gson/2.9.0//gson-2.9.0.jar guava/32.1.3-jre//guava-32.1.3-jre.jar guice-servlet/4.0//guice-servlet-4.0.jar guice/4.0//guice-4.0.jar -hadoop-annotations/3.2.4//hadoop-annotations-3.2.4.jar -hadoop-auth/3.2.4//hadoop-auth-3.2.4.jar -hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar -hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar -hadoop-common/3.2.4//hadoop-common-3.2.4.jar -hadoop-hdfs-client/3.2.4//hadoop-hdfs-client-3.2.4.jar -hadoop-mapreduce-client-app/3.2.4//hadoop-mapreduce-client-app-3.2.4.jar -hadoop-mapreduce-client-common/3.2.4//hadoop-mapreduce-client-common-3.2.4.jar -hadoop-mapreduce-client-core/3.2.4//hadoop-mapreduce-client-core-3.2.4.jar -hadoop-mapreduce-client-shuffle/3.2.4//hadoop-mapreduce-client-shuffle-3.2.4.jar -hadoop-yarn-api/3.2.4//hadoop-yarn-api-3.2.4.jar -hadoop-yarn-client/3.2.4//hadoop-yarn-client-3.2.4.jar -hadoop-yarn-common/3.2.4//hadoop-yarn-common-3.2.4.jar -hadoop-yarn-registry/3.2.4//hadoop-yarn-registry-3.2.4.jar -hadoop-yarn-server-common/3.2.4//hadoop-yarn-server-common-3.2.4.jar -hadoop-yarn-server-nodemanager/3.2.4//hadoop-yarn-server-nodemanager-3.2.4.jar -hadoop-yarn-server-web-proxy/3.2.4//hadoop-yarn-server-web-proxy-3.2.4.jar -htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar +hadoop-annotations/3.3.6//hadoop-annotations-3.3.6.jar +hadoop-auth/3.3.6//hadoop-auth-3.3.6.jar +hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar +hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar +hadoop-common/3.3.6//hadoop-common-3.3.6.jar +hadoop-hdfs-client/3.3.6//hadoop-hdfs-client-3.3.6.jar +hadoop-mapreduce-client-app/3.3.6//hadoop-mapreduce-client-app-3.3.6.jar +hadoop-mapreduce-client-common/3.3.6//hadoop-mapreduce-client-common-3.3.6.jar +hadoop-mapreduce-client-core/3.3.6//hadoop-mapreduce-client-core-3.3.6.jar +hadoop-mapreduce-client-shuffle/3.3.6//hadoop-mapreduce-client-shuffle-3.3.6.jar +hadoop-registry/3.3.6//hadoop-registry-3.3.6.jar +hadoop-shaded-guava/1.1.1//hadoop-shaded-guava-1.1.1.jar +hadoop-shaded-protobuf_3_7/1.1.1//hadoop-shaded-protobuf_3_7-1.1.1.jar +hadoop-yarn-api/3.3.6//hadoop-yarn-api-3.3.6.jar +hadoop-yarn-client/3.3.6//hadoop-yarn-client-3.3.6.jar +hadoop-yarn-common/3.3.6//hadoop-yarn-common-3.3.6.jar +hadoop-yarn-server-common/3.3.6//hadoop-yarn-server-common-3.3.6.jar +hadoop-yarn-server-nodemanager/3.3.6//hadoop-yarn-server-nodemanager-3.3.6.jar +hadoop-yarn-server-web-proxy/3.3.6//hadoop-yarn-server-web-proxy-3.3.6.jar httpclient/4.5.13//httpclient-4.5.13.jar httpcore/4.4.13//httpcore-4.4.13.jar jackson-core-asl/1.9.13//jackson-core-asl-1.9.13.jar -jackson-core/2.10.5//jackson-core-2.10.5.jar -jackson-jaxrs-base/2.10.5//jackson-jaxrs-base-2.10.5.jar -jackson-jaxrs-json-provider/2.10.5//jackson-jaxrs-json-provider-2.10.5.jar -jackson-jaxrs/1.9.2//jackson-jaxrs-1.9.2.jar +jackson-core/2.12.7//jackson-core-2.12.7.jar +jackson-jaxrs-base/2.12.7//jackson-jaxrs-base-2.12.7.jar +jackson-jaxrs-json-provider/2.12.7//jackson-jaxrs-json-provider-2.12.7.jar jackson-mapper-asl/1.9.13//jackson-mapper-asl-1.9.13.jar -jackson-module-jaxb-annotations/2.10.5//jackson-module-jaxb-annotations-2.10.5.jar -jackson-xc/1.9.2//jackson-xc-1.9.2.jar +jackson-module-jaxb-annotations/2.12.7//jackson-module-jaxb-annotations-2.12.7.jar jakarta.activation-api/1.2.1//jakarta.activation-api-1.2.1.jar jakarta.xml.bind-api/2.3.2//jakarta.xml.bind-api-2.3.2.jar -javax.activation-api/1.2.0//javax.activation-api-1.2.0.jar +javax-websocket-client-impl/9.4.51.v20230217//javax-websocket-client-impl-9.4.51.v20230217.jar +javax-websocket-server-impl/9.4.51.v20230217//javax-websocket-server-impl-9.4.51.v20230217.jar +javax.annotation-api/1.3.2//javax.annotation-api-1.3.2.jar javax.inject/1//javax.inject-1.jar javax.servlet-api/3.1.0//javax.servlet-api-3.1.0.jar +javax.websocket-api/1.0//javax.websocket-api-1.0.jar +javax.websocket-client-api/1.0//javax.websocket-client-api-1.0.jar jaxb-api/2.2.11//jaxb-api-2.2.11.jar jaxb-impl/2.2.3-1//jaxb-impl-2.2.3-1.jar jcip-annotations/1.0-1//jcip-annotations-1.0-1.jar jcl-over-slf4j/1.7.36//jcl-over-slf4j-1.7.36.jar -jersey-client/1.19//jersey-client-1.19.jar -jersey-core/1.19//jersey-core-1.19.jar -jersey-guice/1.19//jersey-guice-1.19.jar -jersey-json/1.19//jersey-json-1.19.jar -jersey-server/1.19//jersey-server-1.19.jar -jersey-servlet/1.19//jersey-servlet-1.19.jar -jettison/1.1//jettison-1.1.jar -jetty-http/9.4.43.v20210629//jetty-http-9.4.43.v20210629.jar -jetty-io/9.4.43.v20210629//jetty-io-9.4.43.v20210629.jar -jetty-security/9.4.43.v20210629//jetty-security-9.4.43.v20210629.jar -jetty-server/9.4.43.v20210629//jetty-server-9.4.43.v20210629.jar -jetty-servlet/9.4.43.v20210629//jetty-servlet-9.4.43.v20210629.jar -jetty-util-ajax/9.4.43.v20210629//jetty-util-ajax-9.4.43.v20210629.jar -jetty-util/9.4.43.v20210629//jetty-util-9.4.43.v20210629.jar -jetty-webapp/9.4.43.v20210629//jetty-webapp-9.4.43.v20210629.jar -jetty-xml/9.4.43.v20210629//jetty-xml-9.4.43.v20210629.jar +jersey-client/1.19.4//jersey-client-1.19.4.jar +jersey-core/1.19.4//jersey-core-1.19.4.jar +jersey-guice/1.19.4//jersey-guice-1.19.4.jar +jersey-json/1.20//jersey-json-1.20.jar +jersey-server/1.19.4//jersey-server-1.19.4.jar +jersey-servlet/1.19.4//jersey-servlet-1.19.4.jar +jettison/1.5.4//jettison-1.5.4.jar +jetty-annotations/9.4.51.v20230217//jetty-annotations-9.4.51.v20230217.jar +jetty-client/9.4.51.v20230217//jetty-client-9.4.51.v20230217.jar +jetty-http/9.4.51.v20230217//jetty-http-9.4.51.v20230217.jar +jetty-io/9.4.51.v20230217//jetty-io-9.4.51.v20230217.jar +jetty-jndi/9.4.51.v20230217//jetty-jndi-9.4.51.v20230217.jar +jetty-plus/9.4.51.v20230217//jetty-plus-9.4.51.v20230217.jar +jetty-security/9.4.51.v20230217//jetty-security-9.4.51.v20230217.jar +jetty-server/9.4.51.v20230217//jetty-server-9.4.51.v20230217.jar +jetty-servlet/9.4.51.v20230217//jetty-servlet-9.4.51.v20230217.jar +jetty-util-ajax/9.4.51.v20230217//jetty-util-ajax-9.4.51.v20230217.jar +jetty-util/9.4.51.v20230217//jetty-util-9.4.51.v20230217.jar +jetty-webapp/9.4.51.v20230217//jetty-webapp-9.4.51.v20230217.jar +jetty-xml/9.4.51.v20230217//jetty-xml-9.4.51.v20230217.jar +jline/3.9.0//jline-3.9.0.jar +jna/5.2.0//jna-5.2.0.jar jsch/0.1.55//jsch-0.1.55.jar -json-smart/2.4.7//json-smart-2.4.7.jar jsp-api/2.1//jsp-api-2.1.jar jsr305/1.3.9//jsr305-1.3.9.jar jsr311-api/1.1.1//jsr311-api-1.1.1.jar @@ -120,6 +128,8 @@ kerby-config/1.0.1//kerby-config-1.0.1.jar kerby-pkix/1.0.1//kerby-pkix-1.0.1.jar kerby-util/1.0.1//kerby-util-1.0.1.jar kerby-xdr/1.0.1//kerby-xdr-1.0.1.jar +kotlin-stdlib-common/1.4.10//kotlin-stdlib-common-1.4.10.jar +kotlin-stdlib/1.4.10//kotlin-stdlib-1.4.10.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar lz4-java/1.8.0//lz4-java-1.8.0.jar maven-jdk-tools-wrapper/0.1//maven-jdk-tools-wrapper-0.1.jar @@ -162,8 +172,8 @@ netty-transport-udt/4.1.93.Final//netty-transport-udt-4.1.93.Final.jar netty-transport/4.1.93.Final//netty-transport-4.1.93.Final.jar netty/3.10.6.Final//netty-3.10.6.Final.jar nimbus-jose-jwt/9.8.1//nimbus-jose-jwt-9.8.1.jar -okhttp/2.7.5//okhttp-2.7.5.jar -okio/1.6.0//okio-1.6.0.jar +okhttp/4.9.3//okhttp-4.9.3.jar +okio/2.8.0//okio-2.8.0.jar paranamer/2.3//paranamer-2.3.jar protobuf-java/3.19.2//protobuf-java-3.19.2.jar ratis-client/2.5.1//ratis-client-2.5.1.jar @@ -171,18 +181,23 @@ ratis-common/2.5.1//ratis-common-2.5.1.jar ratis-proto/2.5.1//ratis-proto-2.5.1.jar ratis-thirdparty-misc/1.0.4//ratis-thirdparty-misc-1.0.4.jar re2j/1.1//re2j-1.1.jar -reload4j/1.2.18.3//reload4j-1.2.18.3.jar +reload4j/1.2.22//reload4j-1.2.22.jar scala-library/2.12.15//scala-library-2.12.15.jar scala-reflect/2.12.15//scala-reflect-2.12.15.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar -slf4j-reload4j/1.7.35//slf4j-reload4j-1.7.35.jar +slf4j-reload4j/1.7.36//slf4j-reload4j-1.7.36.jar snakeyaml/1.33//snakeyaml-1.33.jar -snappy-java/1.0.5//snappy-java-1.0.5.jar -spotbugs-annotations/3.1.9//spotbugs-annotations-3.1.9.jar +snappy-java/1.1.8.2//snappy-java-1.1.8.2.jar stax2-api/4.2.1//stax2-api-4.2.1.jar token-provider/1.0.1//token-provider-1.0.1.jar -woodstox-core/5.3.0//woodstox-core-5.3.0.jar +websocket-api/9.4.51.v20230217//websocket-api-9.4.51.v20230217.jar +websocket-client/9.4.51.v20230217//websocket-client-9.4.51.v20230217.jar +websocket-common/9.4.51.v20230217//websocket-common-9.4.51.v20230217.jar +websocket-server/9.4.51.v20230217//websocket-server-9.4.51.v20230217.jar +websocket-servlet/9.4.51.v20230217//websocket-servlet-9.4.51.v20230217.jar +woodstox-core/5.4.0//woodstox-core-5.4.0.jar xz/1.0//xz-1.0.jar -zookeeper/3.4.14//zookeeper-3.4.14.jar +zookeeper-jute/3.6.3//zookeeper-jute-3.6.3.jar +zookeeper/3.6.3//zookeeper-3.6.3.jar zstd-jni/1.5.2-1//zstd-jni-1.5.2-1.jar diff --git a/dev/deps/dependencies-client-spark-2.4 b/dev/deps/dependencies-client-spark-2.4 index 9beaa1f8ba7..a002ea22b6e 100644 --- a/dev/deps/dependencies-client-spark-2.4 +++ b/dev/deps/dependencies-client-spark-2.4 @@ -22,9 +22,8 @@ commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar failureaccess/1.0.1//failureaccess-1.0.1.jar guava/32.1.3-jre//guava-32.1.3-jre.jar -hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar -hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar -htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar +hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar +hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jcl-over-slf4j/1.7.36//jcl-over-slf4j-1.7.36.jar jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar @@ -77,4 +76,5 @@ scala-reflect/2.11.12//scala-reflect-2.11.12.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/1.33//snakeyaml-1.33.jar +snappy-java/1.1.8.2//snappy-java-1.1.8.2.jar zstd-jni/1.4.4-3//zstd-jni-1.4.4-3.jar diff --git a/dev/deps/dependencies-client-spark-3.0 b/dev/deps/dependencies-client-spark-3.0 index 03618f785a4..00adfa19070 100644 --- a/dev/deps/dependencies-client-spark-3.0 +++ b/dev/deps/dependencies-client-spark-3.0 @@ -22,9 +22,8 @@ commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar failureaccess/1.0.1//failureaccess-1.0.1.jar guava/32.1.3-jre//guava-32.1.3-jre.jar -hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar -hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar -htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar +hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar +hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jcl-over-slf4j/1.7.36//jcl-over-slf4j-1.7.36.jar jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar @@ -77,4 +76,5 @@ scala-reflect/2.12.10//scala-reflect-2.12.10.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/1.33//snakeyaml-1.33.jar +snappy-java/1.1.8.2//snappy-java-1.1.8.2.jar zstd-jni/1.4.4-3//zstd-jni-1.4.4-3.jar diff --git a/dev/deps/dependencies-client-spark-3.1 b/dev/deps/dependencies-client-spark-3.1 index da8331b10b2..b9c2c81fda8 100644 --- a/dev/deps/dependencies-client-spark-3.1 +++ b/dev/deps/dependencies-client-spark-3.1 @@ -22,9 +22,8 @@ commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar failureaccess/1.0.1//failureaccess-1.0.1.jar guava/32.1.3-jre//guava-32.1.3-jre.jar -hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar -hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar -htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar +hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar +hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jcl-over-slf4j/1.7.36//jcl-over-slf4j-1.7.36.jar jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar @@ -77,4 +76,5 @@ scala-reflect/2.12.10//scala-reflect-2.12.10.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/1.33//snakeyaml-1.33.jar +snappy-java/1.1.8.2//snappy-java-1.1.8.2.jar zstd-jni/1.4.8-1//zstd-jni-1.4.8-1.jar diff --git a/dev/deps/dependencies-client-spark-3.2 b/dev/deps/dependencies-client-spark-3.2 index 1feb81a8ed3..2c3cf16720c 100644 --- a/dev/deps/dependencies-client-spark-3.2 +++ b/dev/deps/dependencies-client-spark-3.2 @@ -22,9 +22,8 @@ commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar failureaccess/1.0.1//failureaccess-1.0.1.jar guava/32.1.3-jre//guava-32.1.3-jre.jar -hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar -hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar -htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar +hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar +hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jcl-over-slf4j/1.7.36//jcl-over-slf4j-1.7.36.jar jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar @@ -77,4 +76,5 @@ scala-reflect/2.12.15//scala-reflect-2.12.15.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/1.33//snakeyaml-1.33.jar +snappy-java/1.1.8.2//snappy-java-1.1.8.2.jar zstd-jni/1.5.0-4//zstd-jni-1.5.0-4.jar diff --git a/dev/deps/dependencies-client-spark-3.3 b/dev/deps/dependencies-client-spark-3.3 index be6b7d17fb4..111bf56cf9a 100644 --- a/dev/deps/dependencies-client-spark-3.3 +++ b/dev/deps/dependencies-client-spark-3.3 @@ -22,9 +22,8 @@ commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar failureaccess/1.0.1//failureaccess-1.0.1.jar guava/32.1.3-jre//guava-32.1.3-jre.jar -hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar -hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar -htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar +hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar +hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jcl-over-slf4j/1.7.36//jcl-over-slf4j-1.7.36.jar jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar @@ -77,4 +76,5 @@ scala-reflect/2.12.15//scala-reflect-2.12.15.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/1.33//snakeyaml-1.33.jar +snappy-java/1.1.8.2//snappy-java-1.1.8.2.jar zstd-jni/1.5.2-1//zstd-jni-1.5.2-1.jar diff --git a/dev/deps/dependencies-client-spark-3.4 b/dev/deps/dependencies-client-spark-3.4 index eabdee5a80e..dd2ad1a3eb6 100644 --- a/dev/deps/dependencies-client-spark-3.4 +++ b/dev/deps/dependencies-client-spark-3.4 @@ -22,9 +22,8 @@ commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar failureaccess/1.0.1//failureaccess-1.0.1.jar guava/32.1.3-jre//guava-32.1.3-jre.jar -hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar -hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar -htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar +hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar +hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jcl-over-slf4j/1.7.36//jcl-over-slf4j-1.7.36.jar jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar @@ -77,4 +76,5 @@ scala-reflect/2.12.17//scala-reflect-2.12.17.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/1.33//snakeyaml-1.33.jar +snappy-java/1.1.8.2//snappy-java-1.1.8.2.jar zstd-jni/1.5.2-5//zstd-jni-1.5.2-5.jar diff --git a/dev/deps/dependencies-client-spark-3.5 b/dev/deps/dependencies-client-spark-3.5 index 4aa62aef53b..d3a66fca3a3 100644 --- a/dev/deps/dependencies-client-spark-3.5 +++ b/dev/deps/dependencies-client-spark-3.5 @@ -22,9 +22,8 @@ commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar failureaccess/1.0.1//failureaccess-1.0.1.jar guava/32.1.3-jre//guava-32.1.3-jre.jar -hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar -hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar -htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar +hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar +hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar jcl-over-slf4j/1.7.36//jcl-over-slf4j-1.7.36.jar jsr305/1.3.9//jsr305-1.3.9.jar jul-to-slf4j/1.7.36//jul-to-slf4j-1.7.36.jar @@ -77,4 +76,5 @@ scala-reflect/2.12.18//scala-reflect-2.12.18.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/1.33//snakeyaml-1.33.jar +snappy-java/1.1.8.2//snappy-java-1.1.8.2.jar zstd-jni/1.5.5-4//zstd-jni-1.5.5-4.jar diff --git a/dev/deps/dependencies-server b/dev/deps/dependencies-server index 1be1e05dd05..efc6c68c482 100644 --- a/dev/deps/dependencies-server +++ b/dev/deps/dependencies-server @@ -23,9 +23,8 @@ commons-lang3/3.12.0//commons-lang3-3.12.0.jar commons-logging/1.1.3//commons-logging-1.1.3.jar failureaccess/1.0.1//failureaccess-1.0.1.jar guava/32.1.3-jre//guava-32.1.3-jre.jar -hadoop-client-api/3.2.4//hadoop-client-api-3.2.4.jar -hadoop-client-runtime/3.2.4//hadoop-client-runtime-3.2.4.jar -htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar +hadoop-client-api/3.3.6//hadoop-client-api-3.3.6.jar +hadoop-client-runtime/3.3.6//hadoop-client-runtime-3.3.6.jar javassist/3.28.0-GA//javassist-3.28.0-GA.jar javax.servlet-api/3.1.0//javax.servlet-api-3.1.0.jar jcl-over-slf4j/1.7.36//jcl-over-slf4j-1.7.36.jar @@ -92,4 +91,5 @@ scala-reflect/2.12.15//scala-reflect-2.12.15.jar shims/0.9.32//shims-0.9.32.jar slf4j-api/1.7.36//slf4j-api-1.7.36.jar snakeyaml/1.33//snakeyaml-1.33.jar +snappy-java/1.1.8.2//snappy-java-1.1.8.2.jar zstd-jni/1.5.2-1//zstd-jni-1.5.2-1.jar diff --git a/pom.xml b/pom.xml index e1c96c70437..cbf1f3c6bf9 100644 --- a/pom.xml +++ b/pom.xml @@ -67,7 +67,7 @@ 3.3.2 - 3.2.4 + 3.3.6 3.2.6 3.12.0 diff --git a/project/CelebornBuild.scala b/project/CelebornBuild.scala index b4f56782a96..ebe8d1c5282 100644 --- a/project/CelebornBuild.scala +++ b/project/CelebornBuild.scala @@ -43,7 +43,7 @@ object Dependencies { val commonsLang3Version = "3.12.0" val findbugsVersion = "1.3.9" val guavaVersion = "32.1.3-jre" - val hadoopVersion = "3.2.4" + val hadoopVersion = "3.3.6" val javaxServletVersion = "3.1.0" val junitInterfaceVersion = "0.13.3" // don't forget update `junitInterfaceVersion` when we upgrade junit @@ -83,6 +83,7 @@ object Dependencies { val hadoopClientApi = "org.apache.hadoop" % "hadoop-client-api" % hadoopVersion val hadoopClientRuntime = "org.apache.hadoop" % "hadoop-client-runtime" % hadoopVersion val hadoopMapreduceClientApp = "org.apache.hadoop" % "hadoop-mapreduce-client-app" % hadoopVersion excludeAll( + ExclusionRule("io.netty", "netty-transport-native-epoll"), ExclusionRule("com.google.guava", "guava"), ExclusionRule("com.fasterxml.jackson.core", "jackson-annotations"), ExclusionRule("com.fasterxml.jackson.core", "jackson-databind"), From 1c7cd1bd1392c8d3fd4396c539696dc79835f8dc Mon Sep 17 00:00:00 2001 From: sychen Date: Fri, 1 Dec 2023 17:24:37 +0800 Subject: [PATCH 31/31] [CELEBORN-1113][FOLLOWUP] Bump Hadoop client version from 3.2.4 to 3.3.6 ### What changes were proposed in this pull request? ### Why are the changes needed? https://github.com/apache/incubator-celeborn/pull/2077#issuecomment-1835701576 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? GA Closes #2127 from cxzl25/CELEBORN-1113-FOLLOWUP. Authored-by: sychen Signed-off-by: mingji --- project/CelebornBuild.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/CelebornBuild.scala b/project/CelebornBuild.scala index ebe8d1c5282..96f590768e1 100644 --- a/project/CelebornBuild.scala +++ b/project/CelebornBuild.scala @@ -953,7 +953,8 @@ object MRClientProjects { copyDepsSettings, libraryDependencies ++= Seq( "org.apache.hadoop" % "hadoop-client-minicluster" % Dependencies.hadoopVersion % "test", - "org.apache.hadoop" % "hadoop-mapreduce-examples" % Dependencies.hadoopVersion % "test" + "org.apache.hadoop" % "hadoop-mapreduce-examples" % Dependencies.hadoopVersion % "test", + "org.bouncycastle" % "bcpkix-jdk15on" % "1.68" % "test" ) ++ commonUnitTestDependencies ) }