-
Notifications
You must be signed in to change notification settings - Fork 307
Expand file tree
/
Copy pathjni_api.rs
More file actions
1042 lines (934 loc) · 40.8 KB
/
jni_api.rs
File metadata and controls
1042 lines (934 loc) · 40.8 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// 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.
//! Define JNI APIs which can be called from Java/Scala.
use super::{serde, utils::SparkArrowConvert};
use crate::{
errors::{try_unwrap_or_throw, CometError, CometResult},
execution::{
metrics::utils::{build_metric_layout, update_comet_metric, MetricLayout},
planner::PhysicalPlanner,
serde::to_arrow_datatype,
shuffle::spark_unsafe::row::process_sorted_row_partition,
sort::RdxSort,
},
jvm_bridge::{jni_new_global_ref, JVMClasses},
};
use arrow::array::{Array, RecordBatch, UInt32Array};
use arrow::compute::{take, TakeOptions};
use arrow::datatypes::DataType as ArrowDataType;
use datafusion::common::{Result as DataFusionResult, ScalarValue};
use datafusion::execution::disk_manager::DiskManagerMode;
use datafusion::execution::memory_pool::MemoryPool;
use datafusion::execution::runtime_env::RuntimeEnvBuilder;
use datafusion::logical_expr::ScalarUDF;
use datafusion::{
execution::disk_manager::DiskManagerBuilder,
physical_plan::{display::DisplayableExecutionPlan, SendableRecordBatchStream},
prelude::{SessionConfig, SessionContext},
};
use datafusion_comet_proto::spark_operator::Operator;
use datafusion_spark::function::bitwise::bit_count::SparkBitCount;
use datafusion_spark::function::bitwise::bit_get::SparkBitGet;
use datafusion_spark::function::bitwise::bitwise_not::SparkBitwiseNot;
use datafusion_spark::function::datetime::date_add::SparkDateAdd;
use datafusion_spark::function::datetime::date_sub::SparkDateSub;
use datafusion_spark::function::datetime::last_day::SparkLastDay;
use datafusion_spark::function::datetime::next_day::SparkNextDay;
use datafusion_spark::function::hash::crc32::SparkCrc32;
use datafusion_spark::function::hash::sha1::SparkSha1;
use datafusion_spark::function::hash::sha2::SparkSha2;
use datafusion_spark::function::map::map_from_entries::MapFromEntries;
use datafusion_spark::function::math::expm1::SparkExpm1;
use datafusion_spark::function::math::hex::SparkHex;
use datafusion_spark::function::math::width_bucket::SparkWidthBucket;
use datafusion_spark::function::string::char::CharFunc;
use datafusion_spark::function::string::concat::SparkConcat;
use datafusion_spark::function::string::space::SparkSpace;
use futures::poll;
use futures::stream::StreamExt;
use jni::objects::JByteBuffer;
use jni::sys::{jlongArray, JNI_FALSE};
use jni::{
errors::Result as JNIResult,
objects::{
GlobalRef, JByteArray, JClass, JIntArray, JLongArray, JObject, JObjectArray, JString,
ReleaseMode,
},
sys::{jboolean, jdouble, jint, jlong},
JNIEnv,
};
use std::collections::HashMap;
use std::path::PathBuf;
use std::time::{Duration, Instant};
use std::{sync::Arc, task::Poll};
use tokio::runtime::Runtime;
use tokio::sync::mpsc;
use crate::execution::memory_pools::{
create_memory_pool, handle_task_shared_pool_release, parse_memory_pool_config, MemoryPoolConfig,
};
use crate::execution::operators::ScanExec;
use crate::execution::shuffle::{read_ipc_compressed, CompressionCodec};
use crate::execution::spark_plan::SparkPlan;
use crate::execution::tracing::{log_memory_usage, trace_begin, trace_end, with_trace};
use crate::execution::memory_pools::logging_pool::LoggingMemoryPool;
use crate::execution::spark_config::{
SparkConfig, COMET_DEBUG_ENABLED, COMET_DEBUG_MEMORY, COMET_EXPLAIN_NATIVE_ENABLED,
COMET_MAX_TEMP_DIRECTORY_SIZE, COMET_TRACING_ENABLED, SPARK_EXECUTOR_CORES,
};
use crate::parquet::encryption_support::{CometEncryptionFactory, ENCRYPTION_FACTORY_ID};
use datafusion_comet_proto::spark_operator::operator::OpStruct;
use log::info;
use std::sync::OnceLock;
#[cfg(feature = "jemalloc")]
use tikv_jemalloc_ctl::{epoch, stats};
static TOKIO_RUNTIME: OnceLock<Runtime> = OnceLock::new();
fn parse_usize_env_var(name: &str) -> Option<usize> {
std::env::var_os(name).and_then(|n| n.to_str().and_then(|s| s.parse::<usize>().ok()))
}
fn build_runtime(default_worker_threads: Option<usize>) -> Runtime {
let mut builder = tokio::runtime::Builder::new_multi_thread();
if let Some(n) = parse_usize_env_var("COMET_WORKER_THREADS") {
info!("Comet tokio runtime: using COMET_WORKER_THREADS={n}");
builder.worker_threads(n);
} else if let Some(n) = default_worker_threads {
info!("Comet tokio runtime: using spark.executor.cores={n} worker threads");
builder.worker_threads(n);
} else {
info!("Comet tokio runtime: using default thread count");
}
if let Some(n) = parse_usize_env_var("COMET_MAX_BLOCKING_THREADS") {
builder.max_blocking_threads(n);
}
builder
.enable_all()
.build()
.expect("Failed to create Tokio runtime")
}
/// Initialize the global Tokio runtime with the given default worker thread count.
/// If the runtime is already initialized, this is a no-op.
pub fn init_runtime(default_worker_threads: usize) {
TOKIO_RUNTIME.get_or_init(|| build_runtime(Some(default_worker_threads)));
}
/// Function to get a handle to the global Tokio runtime
pub fn get_runtime() -> &'static Runtime {
TOKIO_RUNTIME.get_or_init(|| build_runtime(None))
}
/// Comet native execution context. Kept alive across JNI calls.
struct ExecutionContext {
/// The id of the execution context.
pub id: i64,
/// Task attempt id
pub task_attempt_id: i64,
/// The deserialized Spark plan
pub spark_plan: Operator,
/// The number of partitions
pub partition_count: usize,
/// The DataFusion root operator converted from the `spark_plan`
pub root_op: Option<Arc<SparkPlan>>,
/// The input sources for the DataFusion plan
pub scans: Vec<ScanExec>,
/// The global reference of input sources for the DataFusion plan
pub input_sources: Vec<Arc<GlobalRef>>,
/// The record batch stream to pull results from
pub stream: Option<SendableRecordBatchStream>,
/// Receives batches from a spawned tokio task (async I/O path)
pub batch_receiver: Option<mpsc::Receiver<DataFusionResult<RecordBatch>>>,
/// Native metrics
pub metrics: Arc<GlobalRef>,
// The interval in milliseconds to update metrics
pub metrics_update_interval: Option<Duration>,
// The last update time of metrics
pub metrics_last_update_time: Instant,
/// Counter to avoid checking time on every poll iteration (reduces syscalls)
pub poll_count_since_metrics_check: u32,
/// The time it took to create the native plan and configure the context
pub plan_creation_time: Duration,
/// DataFusion SessionContext
pub session_ctx: Arc<SessionContext>,
/// Whether to enable additional debugging checks & messages
pub debug_native: bool,
/// Whether to write native plans with metrics to stdout
pub explain_native: bool,
/// Memory pool config
pub memory_pool_config: MemoryPoolConfig,
/// Whether to log memory usage on each call to execute_plan
pub tracing_enabled: bool,
/// Pre-computed metric layout for flat array metric updates
pub metric_layout: Option<MetricLayout>,
}
/// Accept serialized query plan and return the address of the native query plan.
/// # Safety
/// This function is inherently unsafe since it deals with raw pointers passed from JNI.
#[no_mangle]
pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(
e: JNIEnv,
_class: JClass,
id: jlong,
iterators: JObjectArray,
serialized_query: JByteArray,
serialized_spark_configs: JByteArray,
partition_count: jint,
metrics_node: JObject,
metrics_update_interval: jlong,
comet_task_memory_manager_obj: JObject,
local_dirs: JObjectArray,
batch_size: jint,
off_heap_mode: jboolean,
memory_pool_type: JString,
memory_limit: jlong,
memory_limit_per_task: jlong,
task_attempt_id: jlong,
task_cpus: jlong,
key_unwrapper_obj: JObject,
) -> jlong {
try_unwrap_or_throw(&e, |mut env| {
// Deserialize Spark configs
let bytes = env.convert_byte_array(serialized_spark_configs)?;
let spark_configs = serde::deserialize_config(bytes.as_slice())?;
let spark_config: HashMap<String, String> = spark_configs.entries.into_iter().collect();
// Initialize the tokio runtime with spark.executor.cores as the default
// worker thread count, falling back to 1 if not set.
let executor_cores = spark_config.get_usize(SPARK_EXECUTOR_CORES, 1);
init_runtime(executor_cores);
// Access Comet configs
let debug_native = spark_config.get_bool(COMET_DEBUG_ENABLED);
let explain_native = spark_config.get_bool(COMET_EXPLAIN_NATIVE_ENABLED);
let tracing_enabled = spark_config.get_bool(COMET_TRACING_ENABLED);
let max_temp_directory_size =
spark_config.get_u64(COMET_MAX_TEMP_DIRECTORY_SIZE, 100 * 1024 * 1024 * 1024);
let logging_memory_pool = spark_config.get_bool(COMET_DEBUG_MEMORY);
with_trace("createPlan", tracing_enabled, || {
// Init JVM classes
JVMClasses::init(&mut env);
let start = Instant::now();
// Deserialize query plan
let bytes = env.convert_byte_array(serialized_query)?;
let spark_plan = serde::deserialize_op(bytes.as_slice())?;
let metrics = Arc::new(jni_new_global_ref!(env, metrics_node)?);
// Get the global references of input sources
let mut input_sources = vec![];
let num_inputs = env.get_array_length(&iterators)?;
for i in 0..num_inputs {
let input_source = env.get_object_array_element(&iterators, i)?;
let input_source = Arc::new(jni_new_global_ref!(env, input_source)?);
input_sources.push(input_source);
}
// Create DataFusion memory pool
let task_memory_manager =
Arc::new(jni_new_global_ref!(env, comet_task_memory_manager_obj)?);
let memory_pool_type = env.get_string(&memory_pool_type)?.into();
let memory_pool_config = parse_memory_pool_config(
off_heap_mode != JNI_FALSE,
memory_pool_type,
memory_limit,
memory_limit_per_task,
)?;
let memory_pool =
create_memory_pool(&memory_pool_config, task_memory_manager, task_attempt_id);
let memory_pool = if logging_memory_pool {
Arc::new(LoggingMemoryPool::new(task_attempt_id as u64, memory_pool))
} else {
memory_pool
};
// Get local directories for storing spill files
let num_local_dirs = env.get_array_length(&local_dirs)?;
let mut local_dirs_vec = vec![];
for i in 0..num_local_dirs {
let local_dir: JString = env.get_object_array_element(&local_dirs, i)?.into();
let local_dir = env.get_string(&local_dir)?;
local_dirs_vec.push(local_dir.into());
}
// We need to keep the session context alive. Some session state like temporary
// dictionaries are stored in session context. If it is dropped, the temporary
// dictionaries will be dropped as well.
let session = prepare_datafusion_session_context(
batch_size as usize,
memory_pool,
local_dirs_vec,
max_temp_directory_size,
task_cpus as usize,
&spark_config,
)?;
let plan_creation_time = start.elapsed();
let metrics_update_interval = if metrics_update_interval > 0 {
Some(Duration::from_millis(metrics_update_interval as u64))
} else {
None
};
// Handle key unwrapper for encrypted files
if !key_unwrapper_obj.is_null() {
let encryption_factory = CometEncryptionFactory {
key_unwrapper: jni_new_global_ref!(env, key_unwrapper_obj)?,
};
session.runtime_env().register_parquet_encryption_factory(
ENCRYPTION_FACTORY_ID,
Arc::new(encryption_factory),
);
}
let exec_context = Box::new(ExecutionContext {
id,
task_attempt_id,
spark_plan,
partition_count: partition_count as usize,
root_op: None,
scans: vec![],
input_sources,
stream: None,
batch_receiver: None,
metrics,
metrics_update_interval,
metrics_last_update_time: Instant::now(),
poll_count_since_metrics_check: 0,
plan_creation_time,
session_ctx: Arc::new(session),
debug_native,
explain_native,
memory_pool_config,
tracing_enabled,
metric_layout: None,
});
Ok(Box::into_raw(exec_context) as i64)
})
})
}
/// Configure DataFusion session context.
fn prepare_datafusion_session_context(
batch_size: usize,
memory_pool: Arc<dyn MemoryPool>,
local_dirs: Vec<String>,
max_temp_directory_size: u64,
task_cpus: usize,
spark_config: &HashMap<String, String>,
) -> CometResult<SessionContext> {
let paths = local_dirs.into_iter().map(PathBuf::from).collect();
let disk_manager = DiskManagerBuilder::default()
.with_mode(DiskManagerMode::Directories(paths))
.with_max_temp_directory_size(max_temp_directory_size);
let mut rt_config = RuntimeEnvBuilder::new().with_disk_manager_builder(disk_manager);
rt_config = rt_config.with_memory_pool(memory_pool);
let mut session_config = SessionConfig::new()
.with_target_partitions(task_cpus)
// This DataFusion context is within the scope of an executing Spark Task. We want to set
// its internal parallelism to the number of CPUs allocated to Spark Tasks. This can be
// modified by changing spark.task.cpus in the Spark config.
.with_batch_size(batch_size)
// DataFusion partial aggregates can emit duplicate rows so we disable the
// skip partial aggregation feature because this is not compatible with Spark's
// use of partial aggregates.
.set(
"datafusion.execution.skip_partial_aggregation_probe_ratio_threshold",
// this is the threshold of number of groups / number of rows and the
// maximum value is 1.0, so we set the threshold a little higher just
// to be safe
&ScalarValue::Float64(Some(1.1)),
);
// Pass through DataFusion configs from Spark.
// e.g: spark-shell --conf spark.comet.datafusion.sql_parser.parse_float_as_decimal=true
// becomes datafusion.sql_parser.parse_float_as_decimal=true
const SPARK_COMET_DF_PREFIX: &str = "spark.comet.datafusion.";
for (key, value) in spark_config {
if let Some(df_key) = key.strip_prefix(SPARK_COMET_DF_PREFIX) {
let df_key = format!("datafusion.{df_key}");
session_config = session_config.set_str(&df_key, value);
}
}
let runtime = rt_config.build()?;
let mut session_ctx = SessionContext::new_with_config_rt(session_config, Arc::new(runtime));
datafusion::functions_nested::register_all(&mut session_ctx)?;
register_datafusion_spark_function(&session_ctx);
// Must be the last one to override existing functions with the same name
datafusion_comet_spark_expr::register_all_comet_functions(&mut session_ctx)?;
Ok(session_ctx)
}
// register UDFs from datafusion-spark crate
fn register_datafusion_spark_function(session_ctx: &SessionContext) {
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkExpm1::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkSha2::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(CharFunc::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkBitGet::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkDateAdd::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkDateSub::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkLastDay::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkNextDay::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkSha1::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkConcat::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkBitwiseNot::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkHex::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkWidthBucket::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(MapFromEntries::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkCrc32::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkSpace::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkBitCount::default()));
}
/// Prepares arrow arrays for output.
fn prepare_output(
env: &mut JNIEnv,
array_addrs: JLongArray,
schema_addrs: JLongArray,
output_batch: RecordBatch,
validate: bool,
) -> CometResult<jlong> {
let num_cols = env.get_array_length(&array_addrs)? as usize;
let array_addrs = unsafe { env.get_array_elements(&array_addrs, ReleaseMode::NoCopyBack)? };
let array_addrs = &*array_addrs;
let schema_addrs = unsafe { env.get_array_elements(&schema_addrs, ReleaseMode::NoCopyBack)? };
let schema_addrs = &*schema_addrs;
let results = output_batch.columns();
let num_rows = output_batch.num_rows();
// there are edge cases where num_cols can be zero due to Spark optimizations
// when the results of a query are not used
if num_cols > 0 {
if results.len() != num_cols {
return Err(CometError::Internal(format!(
"Output column count mismatch: expected {num_cols}, got {}",
results.len()
)));
}
if validate {
// Validate the output arrays.
for array in results.iter() {
let array_data = array.to_data();
array_data
.validate_full()
.expect("Invalid output array data");
}
}
let mut i = 0;
while i < results.len() {
let array_ref = results.get(i).ok_or(CometError::IndexOutOfBounds(i))?;
if array_ref.offset() != 0 {
// https://github.com/apache/datafusion-comet/issues/2051
// Bug with non-zero offset FFI, so take to a new array which will have an offset of 0.
// We expect this to be a cold code path, hence the check_bounds: true and assert_eq.
let indices = UInt32Array::from((0..num_rows as u32).collect::<Vec<u32>>());
let new_array = take(
array_ref,
&indices,
Some(TakeOptions { check_bounds: true }),
)?;
assert_eq!(new_array.offset(), 0);
new_array
.to_data()
.move_to_spark(array_addrs[i], schema_addrs[i])?;
} else {
array_ref
.to_data()
.move_to_spark(array_addrs[i], schema_addrs[i])?;
}
i += 1;
}
}
Ok(num_rows as jlong)
}
/// Pull the next input from JVM. Note that we cannot pull input batches in
/// `ScanStream.poll_next` when the execution stream is polled for output.
/// Because the input source could be another native execution stream, which
/// will be executed in another tokio blocking thread. It causes JNI throw
/// Java exception. So we pull input batches here and insert them into scan
/// operators before polling the stream,
#[inline]
fn pull_input_batches(exec_context: &mut ExecutionContext) -> Result<(), CometError> {
exec_context.scans.iter_mut().try_for_each(|scan| {
scan.get_next_batch()?;
Ok::<(), CometError>(())
})
}
/// Accept serialized query plan and the addresses of Arrow Arrays from Spark,
/// then execute the query. Return addresses of arrow vector.
/// # Safety
/// This function is inherently unsafe since it deals with raw pointers passed from JNI.
#[no_mangle]
pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan(
e: JNIEnv,
_class: JClass,
stage_id: jint,
partition: jint,
exec_context: jlong,
array_addrs: JLongArray,
schema_addrs: JLongArray,
) -> jlong {
try_unwrap_or_throw(&e, |mut env| {
// Retrieve the query
let exec_context = get_execution_context(exec_context);
let tracing_event_name = match &exec_context.spark_plan.op_struct {
Some(OpStruct::ShuffleWriter(_)) => "executePlan(ShuffleWriter)",
_ => "executePlan",
};
if exec_context.tracing_enabled {
#[cfg(feature = "jemalloc")]
{
let e = epoch::mib().unwrap();
let allocated = stats::allocated::mib().unwrap();
e.advance().unwrap();
use crate::execution::tracing::log_memory_usage;
log_memory_usage("jemalloc_allocated", allocated.read().unwrap() as u64);
}
}
with_trace(tracing_event_name, exec_context.tracing_enabled, || {
let exec_context_id = exec_context.id;
// Initialize the execution stream.
// 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 start = Instant::now();
let planner =
PhysicalPlanner::new(Arc::clone(&exec_context.session_ctx), partition)
.with_exec_id(exec_context_id);
let (scans, root_op) = planner.create_plan(
&exec_context.spark_plan,
&mut exec_context.input_sources.clone(),
exec_context.partition_count,
)?;
let physical_plan_time = start.elapsed();
exec_context.plan_creation_time += physical_plan_time;
exec_context.root_op = Some(Arc::clone(&root_op));
exec_context.scans = scans;
// Build the flat metric layout for efficient metric updates
let metrics = exec_context.metrics.as_obj();
exec_context.metric_layout = Some(build_metric_layout(&mut env, metrics)?);
if exec_context.explain_native {
let formatted_plan_str =
DisplayableExecutionPlan::new(root_op.native_plan.as_ref()).indent(true);
info!("Comet native query plan:\n{formatted_plan_str:}");
}
let task_ctx = exec_context.session_ctx.task_ctx();
// Each Comet native execution corresponds to a single Spark partition,
// so we should always execute partition 0.
let stream = root_op.native_plan.execute(0, task_ctx)?;
if exec_context.scans.is_empty() {
// No JVM data sources — spawn onto tokio so the executor
// thread parks in blocking_recv instead of busy-polling.
//
// Channel capacity of 2 allows the producer to work one batch
// ahead while the consumer processes the current one via JNI,
// without buffering excessive memory. Increasing this would
// trade memory for latency hiding if JNI/FFI overhead dominates;
// decreasing to 1 would serialize production and consumption.
let (tx, rx) = mpsc::channel(2);
let mut stream = stream;
get_runtime().spawn(async move {
while let Some(batch) = stream.next().await {
if tx.send(batch).await.is_err() {
break;
}
}
});
exec_context.batch_receiver = Some(rx);
} else {
exec_context.stream = Some(stream);
}
} else {
// Pull input batches
pull_input_batches(exec_context)?;
}
if let Some(rx) = &mut exec_context.batch_receiver {
match rx.blocking_recv() {
Some(Ok(batch)) => {
update_metrics(&mut env, exec_context)?;
return prepare_output(
&mut env,
array_addrs,
schema_addrs,
batch,
exec_context.debug_native,
);
}
Some(Err(e)) => {
return Err(e.into());
}
None => {
log_plan_metrics(exec_context, stage_id, partition);
return Ok(-1);
}
}
}
// ScanExec path: busy-poll to interleave JVM batch pulls with stream polling
get_runtime().block_on(async {
loop {
let next_item = exec_context.stream.as_mut().unwrap().next();
let poll_output = poll!(next_item);
// Only check time every 100 polls to reduce syscall overhead
if let Some(interval) = exec_context.metrics_update_interval {
exec_context.poll_count_since_metrics_check += 1;
if exec_context.poll_count_since_metrics_check >= 100 {
let now = Instant::now();
if now - exec_context.metrics_last_update_time >= interval {
update_metrics(&mut env, exec_context)?;
exec_context.metrics_last_update_time = now;
}
exec_context.poll_count_since_metrics_check = 0;
}
}
match poll_output {
Poll::Ready(Some(output)) => {
return prepare_output(
&mut env,
array_addrs,
schema_addrs,
output?,
exec_context.debug_native,
);
}
Poll::Ready(None) => {
log_plan_metrics(exec_context, stage_id, partition);
return Ok(-1);
}
Poll::Pending => {
// JNI call to pull batches from JVM into ScanExec operators.
// block_in_place lets tokio move other tasks off this worker
// while we wait for JVM data.
tokio::task::block_in_place(|| pull_input_batches(exec_context))?;
}
}
}
})
})
})
}
#[no_mangle]
/// Drop the native query plan object and context object.
pub extern "system" fn Java_org_apache_comet_Native_releasePlan(
e: JNIEnv,
_class: JClass,
exec_context: jlong,
) {
try_unwrap_or_throw(&e, |mut env| unsafe {
let execution_context = get_execution_context(exec_context);
// Update metrics
update_metrics(&mut env, execution_context)?;
handle_task_shared_pool_release(
execution_context.memory_pool_config.pool_type,
execution_context.task_attempt_id,
);
let _: Box<ExecutionContext> = Box::from_raw(execution_context);
Ok(())
})
}
/// Updates the metrics of the query plan.
fn update_metrics(env: &mut JNIEnv, exec_context: &mut ExecutionContext) -> CometResult<()> {
if let Some(ref native_query) = exec_context.root_op {
let native_query = Arc::clone(native_query);
let metrics = exec_context.metrics.as_obj();
if let Some(ref mut layout) = exec_context.metric_layout {
update_comet_metric(env, metrics, &native_query, layout)
} else {
Ok(())
}
} else {
Ok(())
}
}
fn log_plan_metrics(exec_context: &ExecutionContext, stage_id: jint, partition: jint) {
if exec_context.explain_native {
if let Some(plan) = &exec_context.root_op {
let formatted_plan_str =
DisplayableExecutionPlan::with_metrics(plan.native_plan.as_ref()).indent(true);
info!(
"Comet native query plan with metrics (Plan #{} Stage {} Partition {}):\
\n plan creation took {:?}:\
\n{formatted_plan_str:}",
plan.plan_id, stage_id, partition, exec_context.plan_creation_time
);
}
}
}
fn convert_datatype_arrays(
env: &'_ mut JNIEnv<'_>,
serialized_datatypes: JObjectArray,
) -> JNIResult<Vec<ArrowDataType>> {
let array_len = env.get_array_length(&serialized_datatypes)?;
let mut res: Vec<ArrowDataType> = Vec::new();
for i in 0..array_len {
let inner_array = env.get_object_array_element(&serialized_datatypes, i)?;
let inner_array: JByteArray = inner_array.into();
let bytes = env.convert_byte_array(inner_array)?;
let data_type = serde::deserialize_data_type(bytes.as_slice()).unwrap();
let arrow_dt = to_arrow_datatype(&data_type);
res.push(arrow_dt);
}
Ok(res)
}
fn get_execution_context<'a>(id: i64) -> &'a mut ExecutionContext {
unsafe {
(id as *mut ExecutionContext)
.as_mut()
.expect("Comet execution context shouldn't be null!")
}
}
/// Used by Comet shuffle external sorter to write sorted records to disk.
/// # Safety
/// This function is inherently unsafe since it deals with raw pointers passed from JNI.
#[no_mangle]
pub unsafe extern "system" fn Java_org_apache_comet_Native_writeSortedFileNative(
e: JNIEnv,
_class: JClass,
row_addresses: JLongArray,
row_sizes: JIntArray,
serialized_datatypes: JObjectArray,
file_path: JString,
prefer_dictionary_ratio: jdouble,
batch_size: jlong,
checksum_enabled: jboolean,
checksum_algo: jint,
current_checksum: jlong,
compression_codec: JString,
compression_level: jint,
tracing_enabled: jboolean,
) -> jlongArray {
try_unwrap_or_throw(&e, |mut env| unsafe {
with_trace(
"writeSortedFileNative",
tracing_enabled != JNI_FALSE,
|| {
let data_types = convert_datatype_arrays(&mut env, serialized_datatypes)?;
let row_num = env.get_array_length(&row_addresses)? as usize;
let row_addresses =
env.get_array_elements(&row_addresses, ReleaseMode::NoCopyBack)?;
let row_sizes = env.get_array_elements(&row_sizes, ReleaseMode::NoCopyBack)?;
let row_addresses_ptr = row_addresses.as_ptr();
let row_sizes_ptr = row_sizes.as_ptr();
let output_path: String = env.get_string(&file_path).unwrap().into();
let checksum_enabled = checksum_enabled == 1;
let current_checksum = if current_checksum == i64::MIN {
// Initial checksum is not available.
None
} else {
Some(current_checksum as u32)
};
let compression_codec: String = env.get_string(&compression_codec).unwrap().into();
let compression_codec = match compression_codec.as_str() {
"zstd" => CompressionCodec::Zstd(compression_level),
"lz4" => CompressionCodec::Lz4Frame,
"snappy" => CompressionCodec::Snappy,
_ => CompressionCodec::Lz4Frame,
};
let (written_bytes, checksum) = process_sorted_row_partition(
row_num,
batch_size as usize,
row_addresses_ptr,
row_sizes_ptr,
&data_types,
output_path,
prefer_dictionary_ratio,
checksum_enabled,
checksum_algo,
current_checksum,
&compression_codec,
)?;
let checksum = if let Some(checksum) = checksum {
checksum as i64
} else {
// Spark checksums (CRC32 or Adler32) are both u32, so we use i64::MIN to indicate
// checksum is not available.
i64::MIN
};
let long_array = env.new_long_array(2)?;
env.set_long_array_region(&long_array, 0, &[written_bytes, checksum])?;
Ok(long_array.into_raw())
},
)
})
}
#[no_mangle]
/// Used by Comet shuffle external sorter to sort in-memory row partition ids.
pub extern "system" fn Java_org_apache_comet_Native_sortRowPartitionsNative(
e: JNIEnv,
_class: JClass,
address: jlong,
size: jlong,
tracing_enabled: jboolean,
) {
try_unwrap_or_throw(&e, |_| {
with_trace(
"sortRowPartitionsNative",
tracing_enabled != JNI_FALSE,
|| {
// SAFETY: JVM unsafe memory allocation is aligned with long.
let array =
unsafe { std::slice::from_raw_parts_mut(address as *mut i64, size as usize) };
array.rdxsort();
Ok(())
},
)
})
}
#[no_mangle]
/// Used by Comet native shuffle reader
/// # Safety
/// This function is inherently unsafe since it deals with raw pointers passed from JNI.
pub unsafe extern "system" fn Java_org_apache_comet_Native_decodeShuffleBlock(
e: JNIEnv,
_class: JClass,
byte_buffer: JByteBuffer,
length: jint,
array_addrs: JLongArray,
schema_addrs: JLongArray,
tracing_enabled: jboolean,
) -> jlong {
try_unwrap_or_throw(&e, |mut env| {
with_trace("decodeShuffleBlock", tracing_enabled != JNI_FALSE, || {
let raw_pointer = env.get_direct_buffer_address(&byte_buffer)?;
let length = length as usize;
let slice: &[u8] = unsafe { std::slice::from_raw_parts(raw_pointer, length) };
let batch = read_ipc_compressed(slice)?;
prepare_output(&mut env, array_addrs, schema_addrs, batch, false)
})
})
}
#[no_mangle]
/// # Safety
/// This function is inherently unsafe since it deals with raw pointers passed from JNI.
pub unsafe extern "system" fn Java_org_apache_comet_Native_traceBegin(
e: JNIEnv,
_class: JClass,
event: JString,
) {
try_unwrap_or_throw(&e, |mut env| {
let name: String = env.get_string(&event).unwrap().into();
trace_begin(&name);
Ok(())
})
}
#[no_mangle]
/// # Safety
/// This function is inherently unsafe since it deals with raw pointers passed from JNI.
pub unsafe extern "system" fn Java_org_apache_comet_Native_traceEnd(
e: JNIEnv,
_class: JClass,
event: JString,
) {
try_unwrap_or_throw(&e, |mut env| {
let name: String = env.get_string(&event).unwrap().into();
trace_end(&name);
Ok(())
})
}
#[no_mangle]
/// # Safety
/// This function is inherently unsafe since it deals with raw pointers passed from JNI.
pub unsafe extern "system" fn Java_org_apache_comet_Native_logMemoryUsage(
e: JNIEnv,
_class: JClass,
name: JString,
value: jlong,
) {
try_unwrap_or_throw(&e, |mut env| {
let name: String = env.get_string(&name).unwrap().into();
log_memory_usage(&name, value as u64);
Ok(())
})
}
// ============================================================================
// Native Columnar to Row Conversion
// ============================================================================
use crate::execution::columnar_to_row::ColumnarToRowContext;
use arrow::ffi::{from_ffi, FFI_ArrowArray, FFI_ArrowSchema};
/// Initialize a native columnar to row converter.
///
/// # Safety
/// This function is inherently unsafe since it deals with raw pointers passed from JNI.
#[no_mangle]
pub unsafe extern "system" fn Java_org_apache_comet_Native_columnarToRowInit(
e: JNIEnv,
_class: JClass,
serialized_schema: JObjectArray,
batch_size: jint,
) -> jlong {
try_unwrap_or_throw(&e, |mut env| {
// Deserialize the schema
let schema = convert_datatype_arrays(&mut env, serialized_schema)?;
// Create the context
let ctx = Box::new(ColumnarToRowContext::new(schema, batch_size as usize));
Ok(Box::into_raw(ctx) as jlong)
})
}
/// Convert Arrow columnar data to Spark UnsafeRow format.
///
/// # Safety
/// This function is inherently unsafe since it deals with raw pointers passed from JNI.
#[no_mangle]
pub unsafe extern "system" fn Java_org_apache_comet_Native_columnarToRowConvert(
e: JNIEnv,
_class: JClass,
c2r_handle: jlong,
array_addrs: JLongArray,
schema_addrs: JLongArray,
num_rows: jint,
) -> jni::sys::jobject {
try_unwrap_or_throw(&e, |mut env| {
// Get the context
let ctx = (c2r_handle as *mut ColumnarToRowContext)
.as_mut()
.ok_or_else(|| CometError::Internal("Null columnar to row context".to_string()))?;
let num_cols = env.get_array_length(&array_addrs)? as usize;
// Get array and schema addresses
let array_addrs_elements = env.get_array_elements(&array_addrs, ReleaseMode::NoCopyBack)?;
let schema_addrs_elements =
env.get_array_elements(&schema_addrs, ReleaseMode::NoCopyBack)?;
// Import Arrow arrays from FFI
let mut arrays = Vec::with_capacity(num_cols);
for i in 0..num_cols {
let array_ptr = array_addrs_elements[i] as *mut FFI_ArrowArray;
let schema_ptr = schema_addrs_elements[i] as *mut FFI_ArrowSchema;
// Take ownership of the FFI structures
let ffi_array = std::ptr::read(array_ptr);
let ffi_schema = std::ptr::read(schema_ptr);
// Convert to Arrow ArrayData
let array_data = from_ffi(ffi_array, &ffi_schema)
.map_err(|e| CometError::Internal(format!("Failed to import array: {}", e)))?;
arrays.push(arrow::array::make_array(array_data));
}
// Convert columnar to row
let (buffer_ptr, offsets, lengths) = ctx.convert(&arrays, num_rows as usize)?;