This is an automated email from the ASF dual-hosted git repository.

philo pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
     new e182d659b [CELEBORN] Check `spark.shuffle.compress` first to decide 
whether to compress shuffle data (#5228)
e182d659b is described below

commit e182d659b78ef7e384a2020c903dbf65c502012d
Author: Kerwin Zhang <[email protected]>
AuthorDate: Mon Apr 8 16:27:46 2024 +0800

    [CELEBORN] Check `spark.shuffle.compress` first to decide whether to 
compress shuffle data (#5228)
---
 .github/workflows/velox_docker.yml                             |  6 +++---
 .../spark/shuffle/CelebornHashBasedColumnarShuffleWriter.scala | 10 ++++++++--
 2 files changed, 11 insertions(+), 5 deletions(-)

diff --git a/.github/workflows/velox_docker.yml 
b/.github/workflows/velox_docker.yml
index afadfa5d2..0f23c863c 100644
--- a/.github/workflows/velox_docker.yml
+++ b/.github/workflows/velox_docker.yml
@@ -21,9 +21,9 @@ on:
       - '.github/workflows/velox_docker.yml'
       - 'pom.xml'
       - 'backends-velox/**'
-      - 'gluten-celeborn/common'
-      - 'gluten-celeborn/package'
-      - 'gluten-celeborn/velox'
+      - 'gluten-celeborn/common/**'
+      - 'gluten-celeborn/package/**'
+      - 'gluten-celeborn/velox/**'
       - 'gluten-ras/**'
       - 'gluten-core/**'
       - 'gluten-data/**'
diff --git 
a/gluten-celeborn/common/src/main/scala/org/apache/spark/shuffle/CelebornHashBasedColumnarShuffleWriter.scala
 
b/gluten-celeborn/common/src/main/scala/org/apache/spark/shuffle/CelebornHashBasedColumnarShuffleWriter.scala
index 399a65f71..292ff3cc1 100644
--- 
a/gluten-celeborn/common/src/main/scala/org/apache/spark/shuffle/CelebornHashBasedColumnarShuffleWriter.scala
+++ 
b/gluten-celeborn/common/src/main/scala/org/apache/spark/shuffle/CelebornHashBasedColumnarShuffleWriter.scala
@@ -20,6 +20,7 @@ import org.apache.gluten.GlutenConfig
 
 import org.apache.spark._
 import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.SHUFFLE_COMPRESS
 import org.apache.spark.scheduler.MapStatus
 import org.apache.spark.shuffle.celeborn.CelebornShuffleHandle
 import org.apache.spark.storage.BlockManager
@@ -63,9 +64,14 @@ abstract class CelebornHashBasedColumnarShuffleWriter[K, V](
 
   protected val blockManager: BlockManager = SparkEnv.get.blockManager
 
-  protected val customizedCompressionCodec: String = 
GlutenShuffleUtils.getCompressionCodec(conf)
+  protected val customizedCompressionCodec: String =
+    if (conf.getBoolean(SHUFFLE_COMPRESS.key, 
SHUFFLE_COMPRESS.defaultValue.get)) {
+      GlutenShuffleUtils.getCompressionCodec(conf)
+    } else {
+      null // uncompressed
+    }
 
-  protected val compressionLevel =
+  protected val compressionLevel: Int =
     GlutenShuffleUtils.getCompressionLevel(conf, customizedCompressionCodec, 
null)
 
   protected val bufferCompressThreshold: Int =


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to