Skip to content

Commit ea6d205

Browse files
authored
feat: Make native shuffle compression configurable and respect spark.shuffle.compress (#1185)
* Make shuffle compression codec and level configurable * remove lz4 references * docs * update comment * clippy * fix benches * clippy * clippy * disable test for miri * remove lz4 reference from proto
1 parent 8f4a8a5 commit ea6d205

File tree

13 files changed

+221
-63
lines changed

13 files changed

+221
-63
lines changed

common/src/main/scala/org/apache/comet/CometConf.scala

Lines changed: 11 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -272,13 +272,21 @@ object CometConf extends ShimCometConf {
272272
.booleanConf
273273
.createWithDefault(false)
274274

275-
val COMET_EXEC_SHUFFLE_CODEC: ConfigEntry[String] = conf(
276-
s"$COMET_EXEC_CONFIG_PREFIX.shuffle.codec")
275+
val COMET_EXEC_SHUFFLE_COMPRESSION_CODEC: ConfigEntry[String] = conf(
276+
s"$COMET_EXEC_CONFIG_PREFIX.shuffle.compression.codec")
277277
.doc(
278-
"The codec of Comet native shuffle used to compress shuffle data. Only zstd is supported.")
278+
"The codec of Comet native shuffle used to compress shuffle data. Only zstd is supported. " +
279+
"Compression can be disabled by setting spark.shuffle.compress=false.")
279280
.stringConf
281+
.checkValues(Set("zstd"))
280282
.createWithDefault("zstd")
281283

284+
val COMET_EXEC_SHUFFLE_COMPRESSION_LEVEL: ConfigEntry[Int] =
285+
conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.compression.level")
286+
.doc("The compression level to use when compression shuffle files.")
287+
.intConf
288+
.createWithDefault(1)
289+
282290
val COMET_COLUMNAR_SHUFFLE_ASYNC_ENABLED: ConfigEntry[Boolean] =
283291
conf("spark.comet.columnar.shuffle.async.enabled")
284292
.doc("Whether to enable asynchronous shuffle for Arrow-based shuffle.")

common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -110,8 +110,10 @@ case class IpcInputStreamIterator(
110110
currentLimitedInputStream = is
111111

112112
if (decompressingNeeded) {
113-
val zs = ShuffleUtils.compressionCodecForShuffling.compressedInputStream(is)
114-
Channels.newChannel(zs)
113+
ShuffleUtils.compressionCodecForShuffling match {
114+
case Some(codec) => Channels.newChannel(codec.compressedInputStream(is))
115+
case _ => Channels.newChannel(is)
116+
}
115117
} else {
116118
Channels.newChannel(is)
117119
}

common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala

Lines changed: 21 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -21,22 +21,33 @@ package org.apache.spark.sql.comet.execution.shuffle
2121

2222
import org.apache.spark.SparkEnv
2323
import org.apache.spark.internal.Logging
24-
import org.apache.spark.internal.config.IO_COMPRESSION_CODEC
24+
import org.apache.spark.internal.config.{IO_COMPRESSION_CODEC, SHUFFLE_COMPRESS}
2525
import org.apache.spark.io.CompressionCodec
26-
import org.apache.spark.sql.internal.SQLConf
2726

2827
import org.apache.comet.CometConf
2928

3029
private[spark] object ShuffleUtils extends Logging {
31-
lazy val compressionCodecForShuffling: CompressionCodec = {
30+
// optional compression codec to use when compressing shuffle files
31+
lazy val compressionCodecForShuffling: Option[CompressionCodec] = {
3232
val sparkConf = SparkEnv.get.conf
33-
val codecName = CometConf.COMET_EXEC_SHUFFLE_CODEC.get(SQLConf.get)
34-
35-
// only zstd compression is supported at the moment
36-
if (codecName != "zstd") {
37-
logWarning(
38-
s"Overriding config ${IO_COMPRESSION_CODEC}=${codecName} in shuffling, force using zstd")
33+
val shuffleCompressionEnabled = sparkConf.getBoolean(SHUFFLE_COMPRESS.key, true)
34+
val sparkShuffleCodec = sparkConf.get(IO_COMPRESSION_CODEC.key, "lz4")
35+
val cometShuffleCodec = CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_CODEC.get()
36+
if (shuffleCompressionEnabled) {
37+
if (sparkShuffleCodec != cometShuffleCodec) {
38+
logWarning(
39+
s"Overriding config $IO_COMPRESSION_CODEC=$sparkShuffleCodec in shuffling, " +
40+
s"force using $cometShuffleCodec")
41+
}
42+
cometShuffleCodec match {
43+
case "zstd" =>
44+
Some(CompressionCodec.createCodec(sparkConf, "zstd"))
45+
case other =>
46+
throw new UnsupportedOperationException(
47+
s"Unsupported shuffle compression codec: $other")
48+
}
49+
} else {
50+
None
3951
}
40-
CompressionCodec.createCodec(sparkConf, "zstd")
4152
}
4253
}

docs/source/user-guide/configs.md

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -50,7 +50,8 @@ Comet provides the following configuration settings.
5050
| spark.comet.exec.memoryFraction | The fraction of memory from Comet memory overhead that the native memory manager can use for execution. The purpose of this config is to set aside memory for untracked data structures, as well as imprecise size estimation during memory acquisition. | 0.7 |
5151
| spark.comet.exec.project.enabled | Whether to enable project by default. | true |
5252
| spark.comet.exec.replaceSortMergeJoin | Experimental feature to force Spark to replace SortMergeJoin with ShuffledHashJoin for improved performance. This feature is not stable yet. For more information, refer to the Comet Tuning Guide (https://datafusion.apache.org/comet/user-guide/tuning.html). | false |
53-
| spark.comet.exec.shuffle.codec | The codec of Comet native shuffle used to compress shuffle data. Only zstd is supported. | zstd |
53+
| spark.comet.exec.shuffle.compression.codec | The codec of Comet native shuffle used to compress shuffle data. Only zstd is supported. Compression can be disabled by setting spark.shuffle.compress=false. | zstd |
54+
| spark.comet.exec.shuffle.compression.level | The compression level to use when compression shuffle files. | 1 |
5455
| spark.comet.exec.shuffle.enabled | Whether to enable Comet native shuffle. Note that this requires setting 'spark.shuffle.manager' to 'org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager'. 'spark.shuffle.manager' must be set before starting the Spark application and cannot be changed during the application. | true |
5556
| spark.comet.exec.sort.enabled | Whether to enable sort by default. | true |
5657
| spark.comet.exec.sortMergeJoin.enabled | Whether to enable sortMergeJoin by default. | true |

docs/source/user-guide/tuning.md

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -103,6 +103,12 @@ native shuffle currently only supports `HashPartitioning` and `SinglePartitionin
103103
To enable native shuffle, set `spark.comet.exec.shuffle.mode` to `native`. If this mode is explicitly set,
104104
then any shuffle operations that cannot be supported in this mode will fall back to Spark.
105105

106+
### Shuffle Compression
107+
108+
By default, Spark compresses shuffle files using LZ4 compression. Comet overrides this behavior with ZSTD compression.
109+
Compression can be disabled by setting `spark.shuffle.compress=false`, which may result in faster shuffle times in
110+
certain environments, such as single-node setups with fast NVMe drives, at the expense of increased disk space usage.
111+
106112
## Explain Plan
107113
### Extended Explain
108114
With Spark 4.0.0 and newer, Comet can provide extended explain plan information in the Spark UI. Currently this lists

native/core/benches/shuffle_writer.rs

Lines changed: 63 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -15,36 +15,47 @@
1515
// specific language governing permissions and limitations
1616
// under the License.
1717

18+
use arrow_array::builder::Int32Builder;
1819
use arrow_array::{builder::StringBuilder, RecordBatch};
1920
use arrow_schema::{DataType, Field, Schema};
20-
use comet::execution::shuffle::ShuffleWriterExec;
21+
use comet::execution::shuffle::{write_ipc_compressed, CompressionCodec, ShuffleWriterExec};
2122
use criterion::{criterion_group, criterion_main, Criterion};
23+
use datafusion::physical_plan::metrics::Time;
2224
use datafusion::{
2325
physical_plan::{common::collect, memory::MemoryExec, ExecutionPlan},
2426
prelude::SessionContext,
2527
};
2628
use datafusion_physical_expr::{expressions::Column, Partitioning};
29+
use std::io::Cursor;
2730
use std::sync::Arc;
2831
use tokio::runtime::Runtime;
2932

3033
fn criterion_benchmark(c: &mut Criterion) {
31-
let batch = create_batch();
32-
let mut batches = Vec::new();
33-
for _ in 0..10 {
34-
batches.push(batch.clone());
35-
}
36-
let partitions = &[batches];
37-
let exec = ShuffleWriterExec::try_new(
38-
Arc::new(MemoryExec::try_new(partitions, batch.schema(), None).unwrap()),
39-
Partitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16),
40-
"/tmp/data.out".to_string(),
41-
"/tmp/index.out".to_string(),
42-
)
43-
.unwrap();
44-
4534
let mut group = c.benchmark_group("shuffle_writer");
46-
group.bench_function("shuffle_writer", |b| {
35+
group.bench_function("shuffle_writer: encode (no compression))", |b| {
36+
let batch = create_batch(8192, true);
37+
let mut buffer = vec![];
38+
let mut cursor = Cursor::new(&mut buffer);
39+
let ipc_time = Time::default();
40+
b.iter(|| write_ipc_compressed(&batch, &mut cursor, &CompressionCodec::None, &ipc_time));
41+
});
42+
group.bench_function("shuffle_writer: encode and compress (zstd level 1)", |b| {
43+
let batch = create_batch(8192, true);
44+
let mut buffer = vec![];
45+
let mut cursor = Cursor::new(&mut buffer);
46+
let ipc_time = Time::default();
47+
b.iter(|| write_ipc_compressed(&batch, &mut cursor, &CompressionCodec::Zstd(1), &ipc_time));
48+
});
49+
group.bench_function("shuffle_writer: encode and compress (zstd level 6)", |b| {
50+
let batch = create_batch(8192, true);
51+
let mut buffer = vec![];
52+
let mut cursor = Cursor::new(&mut buffer);
53+
let ipc_time = Time::default();
54+
b.iter(|| write_ipc_compressed(&batch, &mut cursor, &CompressionCodec::Zstd(6), &ipc_time));
55+
});
56+
group.bench_function("shuffle_writer: end to end", |b| {
4757
let ctx = SessionContext::new();
58+
let exec = create_shuffle_writer_exec(CompressionCodec::Zstd(1));
4859
b.iter(|| {
4960
let task_ctx = ctx.task_ctx();
5061
let stream = exec.execute(0, task_ctx).unwrap();
@@ -54,19 +65,47 @@ fn criterion_benchmark(c: &mut Criterion) {
5465
});
5566
}
5667

57-
fn create_batch() -> RecordBatch {
58-
let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, true)]));
68+
fn create_shuffle_writer_exec(compression_codec: CompressionCodec) -> ShuffleWriterExec {
69+
let batches = create_batches(8192, 10);
70+
let schema = batches[0].schema();
71+
let partitions = &[batches];
72+
ShuffleWriterExec::try_new(
73+
Arc::new(MemoryExec::try_new(partitions, schema, None).unwrap()),
74+
Partitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16),
75+
compression_codec,
76+
"/tmp/data.out".to_string(),
77+
"/tmp/index.out".to_string(),
78+
)
79+
.unwrap()
80+
}
81+
82+
fn create_batches(size: usize, count: usize) -> Vec<RecordBatch> {
83+
let batch = create_batch(size, true);
84+
let mut batches = Vec::new();
85+
for _ in 0..count {
86+
batches.push(batch.clone());
87+
}
88+
batches
89+
}
90+
91+
fn create_batch(num_rows: usize, allow_nulls: bool) -> RecordBatch {
92+
let schema = Arc::new(Schema::new(vec![
93+
Field::new("c0", DataType::Int32, true),
94+
Field::new("c1", DataType::Utf8, true),
95+
]));
96+
let mut a = Int32Builder::new();
5997
let mut b = StringBuilder::new();
60-
for i in 0..8192 {
61-
if i % 10 == 0 {
98+
for i in 0..num_rows {
99+
a.append_value(i as i32);
100+
if allow_nulls && i % 10 == 0 {
62101
b.append_null();
63102
} else {
64-
b.append_value(format!("{i}"));
103+
b.append_value(format!("this is string number {i}"));
65104
}
66105
}
67-
let array = b.finish();
68-
69-
RecordBatch::try_new(schema.clone(), vec![Arc::new(array)]).unwrap()
106+
let a = a.finish();
107+
let b = b.finish();
108+
RecordBatch::try_new(schema.clone(), vec![Arc::new(a), Arc::new(b)]).unwrap()
70109
}
71110

72111
fn config() -> Criterion {

native/core/src/execution/planner.rs

Lines changed: 15 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -68,6 +68,7 @@ use datafusion_comet_spark_expr::{create_comet_physical_fun, create_negate_expr}
6868
use datafusion_functions_nested::concat::ArrayAppend;
6969
use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr};
7070

71+
use crate::execution::shuffle::CompressionCodec;
7172
use crate::execution::spark_plan::SparkPlan;
7273
use datafusion_comet_proto::{
7374
spark_expression::{
@@ -76,8 +77,8 @@ use datafusion_comet_proto::{
7677
},
7778
spark_operator::{
7879
self, lower_window_frame_bound::LowerFrameBoundStruct, operator::OpStruct,
79-
upper_window_frame_bound::UpperFrameBoundStruct, BuildSide, JoinType, Operator,
80-
WindowFrameType,
80+
upper_window_frame_bound::UpperFrameBoundStruct, BuildSide,
81+
CompressionCodec as SparkCompressionCodec, JoinType, Operator, WindowFrameType,
8182
},
8283
spark_partitioning::{partitioning::PartitioningStruct, Partitioning as SparkPartitioning},
8384
};
@@ -1049,9 +1050,21 @@ impl PhysicalPlanner {
10491050
let partitioning = self
10501051
.create_partitioning(writer.partitioning.as_ref().unwrap(), child.schema())?;
10511052

1053+
let codec = match writer.codec.try_into() {
1054+
Ok(SparkCompressionCodec::None) => Ok(CompressionCodec::None),
1055+
Ok(SparkCompressionCodec::Zstd) => {
1056+
Ok(CompressionCodec::Zstd(writer.compression_level))
1057+
}
1058+
_ => Err(ExecutionError::GeneralError(format!(
1059+
"Unsupported shuffle compression codec: {:?}",
1060+
writer.codec
1061+
))),
1062+
}?;
1063+
10521064
let shuffle_writer = Arc::new(ShuffleWriterExec::try_new(
10531065
Arc::clone(&child.native_plan),
10541066
partitioning,
1067+
codec,
10551068
writer.output_data_file.clone(),
10561069
writer.output_index_file.clone(),
10571070
)?);

native/core/src/execution/shuffle/mod.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,4 +19,4 @@ mod list;
1919
mod map;
2020
pub mod row;
2121
mod shuffle_writer;
22-
pub use shuffle_writer::ShuffleWriterExec;
22+
pub use shuffle_writer::{write_ipc_compressed, CompressionCodec, ShuffleWriterExec};

native/core/src/execution/shuffle/row.rs

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -292,6 +292,7 @@ macro_rules! downcast_builder_ref {
292292
}
293293

294294
// Expose the macro for other modules.
295+
use crate::execution::shuffle::shuffle_writer::CompressionCodec;
295296
pub(crate) use downcast_builder_ref;
296297

297298
/// Appends field of row to the given struct builder. `dt` is the data type of the field.
@@ -3358,7 +3359,9 @@ pub fn process_sorted_row_partition(
33583359

33593360
// we do not collect metrics in Native_writeSortedFileNative
33603361
let ipc_time = Time::default();
3361-
written += write_ipc_compressed(&batch, &mut cursor, &ipc_time)?;
3362+
// compression codec is not configurable for CometBypassMergeSortShuffleWriter
3363+
let codec = CompressionCodec::Zstd(1);
3364+
written += write_ipc_compressed(&batch, &mut cursor, &codec, &ipc_time)?;
33623365

33633366
if let Some(checksum) = &mut current_checksum {
33643367
checksum.update(&mut cursor)?;

0 commit comments

Comments
 (0)