@@ -37,6 +37,7 @@ use jni::{
3737 sys:: { jbyteArray, jint, jlong, jlongArray} ,
3838 JNIEnv ,
3939} ;
40+ use std:: time:: { Duration , Instant } ;
4041use std:: { collections:: HashMap , sync:: Arc , task:: Poll } ;
4142
4243use super :: { serde, utils:: SparkArrowConvert , CometMemoryPool } ;
@@ -81,6 +82,8 @@ struct ExecutionContext {
8182 pub runtime : Runtime ,
8283 /// Native metrics
8384 pub metrics : Arc < GlobalRef > ,
85+ /// The time it took to create the native plan and configure the context
86+ pub plan_creation_time : Duration ,
8487 /// DataFusion SessionContext
8588 pub session_ctx : Arc < SessionContext > ,
8689 /// Whether to enable additional debugging checks & messages
@@ -109,6 +112,8 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(
109112 // Init JVM classes
110113 JVMClasses :: init ( & mut env) ;
111114
115+ let start = Instant :: now ( ) ;
116+
112117 let array = unsafe { JPrimitiveArray :: from_raw ( serialized_query) } ;
113118 let bytes = env. convert_byte_array ( array) ?;
114119
@@ -167,6 +172,8 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(
167172 // dictionaries will be dropped as well.
168173 let session = prepare_datafusion_session_context ( & configs, task_memory_manager) ?;
169174
175+ let plan_creation_time = start. elapsed ( ) ;
176+
170177 let exec_context = Box :: new ( ExecutionContext {
171178 id,
172179 spark_plan,
@@ -177,6 +184,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(
177184 conf : configs,
178185 runtime,
179186 metrics,
187+ plan_creation_time,
180188 session_ctx : Arc :: new ( session) ,
181189 debug_native,
182190 explain_native,
@@ -321,6 +329,8 @@ fn pull_input_batches(exec_context: &mut ExecutionContext) -> Result<(), CometEr
321329pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan (
322330 e : JNIEnv ,
323331 _class : JClass ,
332+ stage_id : jint ,
333+ partition : jint ,
324334 exec_context : jlong ,
325335 array_addrs : jlongArray ,
326336 schema_addrs : jlongArray ,
@@ -335,20 +345,23 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan(
335345 // Because we don't know if input arrays are dictionary-encoded when we create
336346 // query plan, we need to defer stream initialization to first time execution.
337347 if exec_context. root_op . is_none ( ) {
348+ let start = Instant :: now ( ) ;
338349 let planner = PhysicalPlanner :: new ( Arc :: clone ( & exec_context. session_ctx ) )
339350 . with_exec_id ( exec_context_id) ;
340351 let ( scans, root_op) = planner. create_plan (
341352 & exec_context. spark_plan ,
342353 & mut exec_context. input_sources . clone ( ) ,
343354 ) ?;
355+ let physical_plan_time = start. elapsed ( ) ;
344356
357+ exec_context. plan_creation_time += physical_plan_time;
345358 exec_context. root_op = Some ( Arc :: clone ( & root_op) ) ;
346359 exec_context. scans = scans;
347360
348361 if exec_context. explain_native {
349362 let formatted_plan_str =
350363 DisplayableExecutionPlan :: new ( root_op. as_ref ( ) ) . indent ( true ) ;
351- info ! ( "Comet native query plan:\n {formatted_plan_str:}" ) ;
364+ info ! ( "Comet native query plan:\n {formatted_plan_str:}" ) ;
352365 }
353366
354367 let task_ctx = exec_context. session_ctx . task_ctx ( ) ;
@@ -388,7 +401,12 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan(
388401 if let Some ( plan) = & exec_context. root_op {
389402 let formatted_plan_str =
390403 DisplayableExecutionPlan :: with_metrics ( plan. as_ref ( ) ) . indent ( true ) ;
391- info ! ( "Comet native query plan with metrics:\n {formatted_plan_str:}" ) ;
404+ info ! (
405+ "Comet native query plan with metrics:\
406+ \n [Stage {} Partition {}] plan creation (including CometScans fetching first batches) took {:?}:\
407+ \n {formatted_plan_str:}",
408+ stage_id, partition, exec_context. plan_creation_time
409+ ) ;
392410 }
393411 }
394412
0 commit comments