-
Notifications
You must be signed in to change notification settings - Fork 151
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
chore: Make shuffle compression level configurable #632
base: main
Are you sure you want to change the base?
Changes from 6 commits
625f877
d746ad4
b135c0c
0213eca
afdd25d
886f754
1e92732
8410431
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -77,6 +77,7 @@ fn benchmark(c: &mut Criterion) { | |
false, | ||
0, | ||
None, | ||
1, | ||
) | ||
.unwrap(); | ||
}); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -128,27 +128,32 @@ pub struct PhysicalPlanner { | |
exec_context_id: i64, | ||
execution_props: ExecutionProps, | ||
session_ctx: Arc<SessionContext>, | ||
compression_level: i32, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It seems like that we are passing around the Could we leverage the |
||
} | ||
|
||
#[cfg(test)] | ||
impl Default for PhysicalPlanner { | ||
/// Create default planner (for use in tests only) | ||
fn default() -> Self { | ||
let session_ctx = Arc::new(SessionContext::new()); | ||
let execution_props = ExecutionProps::new(); | ||
Self { | ||
exec_context_id: TEST_EXEC_CONTEXT_ID, | ||
execution_props, | ||
session_ctx, | ||
compression_level: 1, | ||
} | ||
} | ||
} | ||
|
||
impl PhysicalPlanner { | ||
pub fn new(session_ctx: Arc<SessionContext>) -> Self { | ||
pub fn new(session_ctx: Arc<SessionContext>, compression_level: i32) -> Self { | ||
let execution_props = ExecutionProps::new(); | ||
Self { | ||
exec_context_id: TEST_EXEC_CONTEXT_ID, | ||
execution_props, | ||
session_ctx, | ||
compression_level, | ||
} | ||
} | ||
|
||
|
@@ -157,6 +162,7 @@ impl PhysicalPlanner { | |
exec_context_id, | ||
execution_props: self.execution_props, | ||
session_ctx: self.session_ctx.clone(), | ||
compression_level: self.compression_level, | ||
} | ||
} | ||
|
||
|
@@ -863,6 +869,7 @@ impl PhysicalPlanner { | |
partitioning, | ||
writer.output_data_file.clone(), | ||
writer.output_index_file.clone(), | ||
self.compression_level, | ||
)?), | ||
)) | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -28,6 +28,11 @@ use std::{ | |
task::{Context, Poll}, | ||
}; | ||
|
||
use crate::{ | ||
common::bit::ceil, | ||
errors::{CometError, CometResult}, | ||
execution::datafusion::spark_hash::{create_murmur3_hashes, pmod}, | ||
}; | ||
use arrow::{datatypes::*, ipc::writer::StreamWriter}; | ||
use async_trait::async_trait; | ||
use bytes::Buf; | ||
|
@@ -59,12 +64,6 @@ use itertools::Itertools; | |
use simd_adler32::Adler32; | ||
use tokio::task; | ||
|
||
use crate::{ | ||
common::bit::ceil, | ||
errors::{CometError, CometResult}, | ||
execution::datafusion::spark_hash::{create_murmur3_hashes, pmod}, | ||
}; | ||
|
||
/// The shuffle writer operator maps each input partition to M output partitions based on a | ||
/// partitioning scheme. No guarantees are made about the order of the resulting partitions. | ||
#[derive(Debug)] | ||
|
@@ -80,6 +79,8 @@ pub struct ShuffleWriterExec { | |
/// Metrics | ||
metrics: ExecutionPlanMetricsSet, | ||
cache: PlanProperties, | ||
/// zstd compression level | ||
compression_level: i32, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If the compression_level is stored in SessionConf, we may retrieve that when executing instead of defining a new field? |
||
} | ||
|
||
impl DisplayAs for ShuffleWriterExec { | ||
|
@@ -118,6 +119,7 @@ impl ExecutionPlan for ShuffleWriterExec { | |
self.partitioning.clone(), | ||
self.output_data_file.clone(), | ||
self.output_index_file.clone(), | ||
self.compression_level, | ||
)?)), | ||
_ => panic!("ShuffleWriterExec wrong number of children"), | ||
} | ||
|
@@ -142,6 +144,7 @@ impl ExecutionPlan for ShuffleWriterExec { | |
self.partitioning.clone(), | ||
metrics, | ||
context, | ||
self.compression_level, | ||
) | ||
.map_err(|e| ArrowError::ExternalError(Box::new(e))), | ||
) | ||
|
@@ -169,6 +172,7 @@ impl ShuffleWriterExec { | |
partitioning: Partitioning, | ||
output_data_file: String, | ||
output_index_file: String, | ||
compression_level: i32, | ||
) -> Result<Self> { | ||
let cache = PlanProperties::new( | ||
EquivalenceProperties::new(input.schema().clone()), | ||
|
@@ -183,6 +187,7 @@ impl ShuffleWriterExec { | |
output_data_file, | ||
output_index_file, | ||
cache, | ||
compression_level, | ||
}) | ||
} | ||
} | ||
|
@@ -201,17 +206,20 @@ struct PartitionBuffer { | |
/// The maximum number of rows in a batch. Once `num_active_rows` reaches `batch_size`, | ||
/// the active array builders will be frozen and appended to frozen buffer `frozen`. | ||
batch_size: usize, | ||
/// zstd compression level | ||
compression_level: i32, | ||
} | ||
|
||
impl PartitionBuffer { | ||
fn new(schema: SchemaRef, batch_size: usize) -> Self { | ||
fn new(schema: SchemaRef, batch_size: usize, compression_level: i32) -> Self { | ||
Self { | ||
schema, | ||
frozen: vec![], | ||
active: vec![], | ||
active_slots_mem_size: 0, | ||
num_active_rows: 0, | ||
batch_size, | ||
compression_level, | ||
} | ||
} | ||
|
||
|
@@ -285,7 +293,7 @@ impl PartitionBuffer { | |
let frozen_capacity_old = self.frozen.capacity(); | ||
let mut cursor = Cursor::new(&mut self.frozen); | ||
cursor.seek(SeekFrom::End(0))?; | ||
write_ipc_compressed(&frozen_batch, &mut cursor)?; | ||
write_ipc_compressed(&frozen_batch, &mut cursor, self.compression_level)?; | ||
|
||
mem_diff += (self.frozen.capacity() - frozen_capacity_old) as isize; | ||
Ok(mem_diff) | ||
|
@@ -577,6 +585,8 @@ struct ShuffleRepartitioner { | |
partition_ids: Vec<u64>, | ||
/// The configured batch size | ||
batch_size: usize, | ||
/// zstd compression level | ||
compression_level: i32, | ||
} | ||
|
||
struct ShuffleRepartitionerMetrics { | ||
|
@@ -611,6 +621,7 @@ impl ShuffleRepartitioner { | |
metrics: ShuffleRepartitionerMetrics, | ||
runtime: Arc<RuntimeEnv>, | ||
batch_size: usize, | ||
compression_level: i32, | ||
) -> Self { | ||
let num_output_partitions = partitioning.partition_count(); | ||
let reservation = MemoryConsumer::new(format!("ShuffleRepartitioner[{}]", partition_id)) | ||
|
@@ -633,7 +644,7 @@ impl ShuffleRepartitioner { | |
schema: schema.clone(), | ||
buffered_partitions: Mutex::new( | ||
(0..num_output_partitions) | ||
.map(|_| PartitionBuffer::new(schema.clone(), batch_size)) | ||
.map(|_| PartitionBuffer::new(schema.clone(), batch_size, compression_level)) | ||
.collect::<Vec<_>>(), | ||
), | ||
spills: Mutex::new(vec![]), | ||
|
@@ -645,6 +656,7 @@ impl ShuffleRepartitioner { | |
hashes_buf, | ||
partition_ids, | ||
batch_size, | ||
compression_level, | ||
} | ||
} | ||
|
||
|
@@ -955,6 +967,7 @@ impl Debug for ShuffleRepartitioner { | |
} | ||
} | ||
|
||
#[allow(clippy::too_many_arguments)] | ||
async fn external_shuffle( | ||
mut input: SendableRecordBatchStream, | ||
partition_id: usize, | ||
|
@@ -963,6 +976,7 @@ async fn external_shuffle( | |
partitioning: Partitioning, | ||
metrics: ShuffleRepartitionerMetrics, | ||
context: Arc<TaskContext>, | ||
compression_level: i32, | ||
) -> Result<SendableRecordBatchStream> { | ||
let schema = input.schema(); | ||
let mut repartitioner = ShuffleRepartitioner::new( | ||
|
@@ -974,6 +988,7 @@ async fn external_shuffle( | |
metrics, | ||
context.runtime_env(), | ||
context.session_config().batch_size(), | ||
compression_level, | ||
); | ||
|
||
while let Some(batch) = input.next().await { | ||
|
@@ -1353,6 +1368,7 @@ impl Checksum { | |
pub(crate) fn write_ipc_compressed<W: Write + Seek>( | ||
batch: &RecordBatch, | ||
output: &mut W, | ||
compression_level: i32, | ||
) -> Result<usize> { | ||
if batch.num_rows() == 0 { | ||
return Ok(0); | ||
|
@@ -1363,8 +1379,10 @@ pub(crate) fn write_ipc_compressed<W: Write + Seek>( | |
output.write_all(&[0u8; 8])?; | ||
|
||
// write ipc data | ||
// TODO: make compression level configurable | ||
let mut arrow_writer = StreamWriter::try_new(zstd::Encoder::new(output, 1)?, &batch.schema())?; | ||
let mut arrow_writer = StreamWriter::try_new( | ||
zstd::Encoder::new(output, compression_level)?, | ||
&batch.schema(), | ||
)?; | ||
arrow_writer.write(batch)?; | ||
arrow_writer.finish()?; | ||
|
||
|
@@ -1465,6 +1483,7 @@ mod test { | |
Partitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16), | ||
"/tmp/data.out".to_string(), | ||
"/tmp/index.out".to_string(), | ||
1, | ||
) | ||
.unwrap(); | ||
let ctx = SessionContext::new(); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -89,6 +89,8 @@ struct ExecutionContext { | |
pub session_ctx: Arc<SessionContext>, | ||
/// Whether to enable additional debugging checks & messages | ||
pub debug_native: bool, | ||
/// zstd compression level | ||
pub compression_level: i32, | ||
} | ||
|
||
/// Accept serialized query plan and return the address of the native query plan. | ||
|
@@ -132,6 +134,11 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( | |
.and_then(|x| x.parse::<bool>().ok()) | ||
.unwrap_or(false); | ||
|
||
let compression_level = configs | ||
.get("compression_level") | ||
.and_then(|x| x.parse::<i32>().ok()) | ||
.unwrap_or(1); | ||
|
||
// Use multi-threaded tokio runtime to prevent blocking spawned tasks if any | ||
let runtime = tokio::runtime::Builder::new_multi_thread() | ||
.enable_all() | ||
|
@@ -169,6 +176,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( | |
metrics, | ||
session_ctx: Arc::new(session), | ||
debug_native, | ||
compression_level, | ||
}); | ||
|
||
Ok(Box::into_raw(exec_context) as i64) | ||
|
@@ -317,8 +325,11 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan( | |
// Because we don't know if input arrays are dictionary-encoded when we create | ||
// query plan, we need to defer stream initialization to first time execution. | ||
if exec_context.root_op.is_none() { | ||
let planner = PhysicalPlanner::new(exec_context.session_ctx.clone()) | ||
.with_exec_id(exec_context_id); | ||
let planner = PhysicalPlanner::new( | ||
exec_context.session_ctx.clone(), | ||
exec_context.compression_level, | ||
) | ||
.with_exec_id(exec_context_id); | ||
let (scans, root_op) = planner.create_plan( | ||
&exec_context.spark_plan, | ||
&mut exec_context.input_sources.clone(), | ||
|
@@ -455,6 +466,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_writeSortedFileNative | |
checksum_enabled: jboolean, | ||
checksum_algo: jint, | ||
current_checksum: jlong, | ||
compression_level: jlong, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I believe you need to update jni code in the JVM side too: org.apache.comet.Native#writeSortedFileNative |
||
) -> jlongArray { | ||
try_unwrap_or_throw(&e, |mut env| unsafe { | ||
let data_types = convert_datatype_arrays(&mut env, serialized_datatypes)?; | ||
|
@@ -493,6 +505,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_writeSortedFileNative | |
checksum_enabled, | ||
checksum_algo, | ||
current_checksum, | ||
compression_level as i32, | ||
)?; | ||
|
||
let checksum = if let Some(checksum) = checksum { | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Perhaps we could have a check here for valid values?