@@ -136,6 +136,8 @@ struct ExecutionContext {
136136 pub metrics_update_interval : Option < Duration > ,
137137 // The last update time of metrics
138138 pub metrics_last_update_time : Instant ,
139+ /// Counter to avoid checking time on every poll iteration (reduces syscalls)
140+ pub poll_count_since_metrics_check : u32 ,
139141 /// The time it took to create the native plan and configure the context
140142 pub plan_creation_time : Duration ,
141143 /// DataFusion SessionContext
@@ -272,6 +274,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(
272274 metrics,
273275 metrics_update_interval,
274276 metrics_last_update_time : Instant :: now ( ) ,
277+ poll_count_since_metrics_check : 0 ,
275278 plan_creation_time,
276279 session_ctx : Arc :: new ( session) ,
277280 debug_native,
@@ -503,60 +506,74 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan(
503506 pull_input_batches ( exec_context) ?;
504507 }
505508
506- loop {
507- // Polling the stream.
508- let next_item = exec_context. stream . as_mut ( ) . unwrap ( ) . next ( ) ;
509- let poll_output = get_runtime ( ) . block_on ( async { poll ! ( next_item) } ) ;
510-
511- // update metrics at interval
512- if let Some ( interval) = exec_context. metrics_update_interval {
513- let now = Instant :: now ( ) ;
514- if now - exec_context. metrics_last_update_time >= interval {
515- update_metrics ( & mut env, exec_context) ?;
516- exec_context. metrics_last_update_time = now;
509+ // Enter the runtime once for the entire polling loop to avoid repeated
510+ // Runtime::enter() overhead
511+ get_runtime ( ) . block_on ( async {
512+ loop {
513+ // Polling the stream.
514+ let next_item = exec_context. stream . as_mut ( ) . unwrap ( ) . next ( ) ;
515+ let poll_output = poll ! ( next_item) ;
516+
517+ // update metrics at interval
518+ // Only check time every 100 polls to reduce syscall overhead
519+ if let Some ( interval) = exec_context. metrics_update_interval {
520+ exec_context. poll_count_since_metrics_check += 1 ;
521+ if exec_context. poll_count_since_metrics_check >= 100 {
522+ let now = Instant :: now ( ) ;
523+ if now - exec_context. metrics_last_update_time >= interval {
524+ update_metrics ( & mut env, exec_context) ?;
525+ exec_context. metrics_last_update_time = now;
526+ }
527+ exec_context. poll_count_since_metrics_check = 0 ;
528+ }
517529 }
518- }
519530
520- match poll_output {
521- Poll :: Ready ( Some ( output) ) => {
522- // prepare output for FFI transfer
523- return prepare_output (
524- & mut env,
525- array_addrs,
526- schema_addrs,
527- output?,
528- exec_context. debug_native ,
529- ) ;
530- }
531- Poll :: Ready ( None ) => {
532- // Reaches EOF of output.
533- if exec_context. explain_native {
534- if let Some ( plan) = & exec_context. root_op {
535- let formatted_plan_str = DisplayableExecutionPlan :: with_metrics (
536- plan. native_plan . as_ref ( ) ,
537- )
538- . indent ( true ) ;
539- info ! (
540- "Comet native query plan with metrics (Plan #{} Stage {} Partition {}):\
541- \n plan creation took {:?}:\
542- \n {formatted_plan_str:}",
543- plan. plan_id, stage_id, partition, exec_context. plan_creation_time
544- ) ;
531+ match poll_output {
532+ Poll :: Ready ( Some ( output) ) => {
533+ // prepare output for FFI transfer
534+ return prepare_output (
535+ & mut env,
536+ array_addrs,
537+ schema_addrs,
538+ output?,
539+ exec_context. debug_native ,
540+ ) ;
541+ }
542+ Poll :: Ready ( None ) => {
543+ // Reaches EOF of output.
544+ if exec_context. explain_native {
545+ if let Some ( plan) = & exec_context. root_op {
546+ let formatted_plan_str = DisplayableExecutionPlan :: with_metrics (
547+ plan. native_plan . as_ref ( ) ,
548+ )
549+ . indent ( true ) ;
550+ info ! (
551+ "Comet native query plan with metrics (Plan #{} Stage {} Partition {}):\
552+ \n plan creation took {:?}:\
553+ \n {formatted_plan_str:}",
554+ plan. plan_id, stage_id, partition, exec_context. plan_creation_time
555+ ) ;
556+ }
545557 }
558+ return Ok ( -1 ) ;
559+ }
560+ // A poll pending means there are more than one blocking operators,
561+ // we don't need go back-forth between JVM/Native. Just keeping polling.
562+ Poll :: Pending => {
563+ // TODO: Investigate if JNI calls are safe without block_in_place.
564+ // block_in_place prevents Tokio from migrating this task to another thread,
565+ // which is necessary because JNI env is thread-local. If we can guarantee
566+ // thread safety another way, we could remove this wrapper for better perf.
567+ tokio:: task:: block_in_place ( || {
568+ pull_input_batches ( exec_context)
569+ } ) ?;
570+
571+ // Output not ready yet
572+ continue ;
546573 }
547- return Ok ( -1 ) ;
548- }
549- // A poll pending means there are more than one blocking operators,
550- // we don't need go back-forth between JVM/Native. Just keeping polling.
551- Poll :: Pending => {
552- // Pull input batches
553- pull_input_batches ( exec_context) ?;
554-
555- // Output not ready yet
556- continue ;
557574 }
558575 }
559- }
576+ } )
560577 } )
561578 } )
562579}
0 commit comments