@@ -71,6 +71,29 @@ use crate::execution::spark_plan::SparkPlan;
7171use log:: info;
7272use once_cell:: sync:: { Lazy , OnceCell } ;
7373
74+ static TOKIO_RUNTIME : Lazy < Runtime > = Lazy :: new ( || {
75+ let mut builder = tokio:: runtime:: Builder :: new_multi_thread ( ) ;
76+ if let Some ( n) = parse_usize_env_var ( "COMET_WORKER_THREADS" ) {
77+ builder. worker_threads ( n) ;
78+ }
79+ if let Some ( n) = parse_usize_env_var ( "COMET_MAX_BLOCKING_THREADS" ) {
80+ builder. max_blocking_threads ( n) ;
81+ }
82+ builder
83+ . enable_all ( )
84+ . build ( )
85+ . expect ( "Failed to create Tokio runtime" )
86+ } ) ;
87+
88+ fn parse_usize_env_var ( name : & str ) -> Option < usize > {
89+ std:: env:: var_os ( name) . and_then ( |n| n. to_str ( ) . and_then ( |s| s. parse :: < usize > ( ) . ok ( ) ) )
90+ }
91+
92+ /// Function to get a handle to the global Tokio runtime
93+ pub fn get_runtime ( ) -> & ' static Runtime {
94+ & TOKIO_RUNTIME
95+ }
96+
7497/// Comet native execution context. Kept alive across JNI calls.
7598struct ExecutionContext {
7699 /// The id of the execution context.
@@ -89,8 +112,6 @@ struct ExecutionContext {
89112 pub input_sources : Vec < Arc < GlobalRef > > ,
90113 /// The record batch stream to pull results from
91114 pub stream : Option < SendableRecordBatchStream > ,
92- /// The Tokio runtime used for async.
93- pub runtime : Runtime ,
94115 /// Native metrics
95116 pub metrics : Arc < GlobalRef > ,
96117 // The interval in milliseconds to update metrics
@@ -177,8 +198,6 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(
177198 task_attempt_id : jlong ,
178199 debug_native : jboolean ,
179200 explain_native : jboolean ,
180- worker_threads : jint ,
181- blocking_threads : jint ,
182201) -> jlong {
183202 try_unwrap_or_throw ( & e, |mut env| {
184203 // Init JVM classes
@@ -192,13 +211,6 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(
192211 // Deserialize query plan
193212 let spark_plan = serde:: deserialize_op ( bytes. as_slice ( ) ) ?;
194213
195- // Use multi-threaded tokio runtime to prevent blocking spawned tasks if any
196- let runtime = tokio:: runtime:: Builder :: new_multi_thread ( )
197- . worker_threads ( worker_threads as usize )
198- . max_blocking_threads ( blocking_threads as usize )
199- . enable_all ( )
200- . build ( ) ?;
201-
202214 let metrics = Arc :: new ( jni_new_global_ref ! ( env, metrics_node) ?) ;
203215
204216 // Get the global references of input sources
@@ -258,7 +270,6 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(
258270 scans : vec ! [ ] ,
259271 input_sources,
260272 stream : None ,
261- runtime,
262273 metrics,
263274 metrics_update_interval,
264275 metrics_last_update_time : Instant :: now ( ) ,
@@ -559,7 +570,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan(
559570 loop {
560571 // Polling the stream.
561572 let next_item = exec_context. stream . as_mut ( ) . unwrap ( ) . next ( ) ;
562- let poll_output = exec_context . runtime . block_on ( async { poll ! ( next_item) } ) ;
573+ let poll_output = get_runtime ( ) . block_on ( async { poll ! ( next_item) } ) ;
563574
564575 // update metrics at interval
565576 if let Some ( interval) = exec_context. metrics_update_interval {
0 commit comments