@@ -79,6 +79,7 @@ use datafusion_common::{
7979 DataFusionError , JoinSide , JoinType , NullEquality , Result , assert_or_internal_err,
8080 internal_err, plan_err, project_schema,
8181} ;
82+ use datafusion_common_runtime:: SpawnedTask ;
8283use datafusion_execution:: TaskContext ;
8384use datafusion_execution:: memory_pool:: { MemoryConsumer , MemoryReservation } ;
8485use datafusion_expr:: Accumulator ;
@@ -95,6 +96,7 @@ use datafusion_physical_expr_common::physical_expr::fmt_sql;
9596use datafusion_physical_expr_common:: utils:: evaluate_expressions_to_arrays;
9697use futures:: TryStreamExt ;
9798use parking_lot:: Mutex ;
99+ use tokio:: sync:: oneshot;
98100
99101use super :: partitioned_hash_eval:: SeededRandomState ;
100102
@@ -1374,25 +1376,42 @@ impl ExecutionPlan for HashJoinExec {
13741376 let build_accumulator = build_accumulator
13751377 . as_ref ( )
13761378 . map ( |acc| ( Arc :: clone ( acc) , partition) ) ;
1379+ let random_state = self . random_state . random_state ( ) . clone ( ) ;
1380+ let with_visited_indices_bitmap =
1381+ need_produce_result_in_final ( self . join_type ) ;
1382+ let config = Arc :: clone ( context. session_config ( ) . options ( ) ) ;
1383+ let null_equality = self . null_equality ;
1384+ let background_join_metrics = join_metrics. clone ( ) ;
13771385
13781386 if build_accumulator. is_some ( ) {
1379- let task = tokio:: task:: spawn ( collect_left_input_and_maybe_report (
1380- self . random_state . random_state ( ) . clone ( ) ,
1381- left_stream,
1382- on_left. clone ( ) ,
1383- join_metrics. clone ( ) ,
1384- reservation,
1385- need_produce_result_in_final ( self . join_type ) ,
1386- 1 ,
1387- enable_dynamic_filter_pushdown,
1388- Arc :: clone ( context. session_config ( ) . options ( ) ) ,
1389- self . null_equality ,
1390- array_map_created_count,
1391- build_accumulator,
1392- ) ) ;
1387+ let ( tx, rx) = oneshot:: channel ( ) ;
1388+ let ( build_accumulator, partition) =
1389+ build_accumulator. expect ( "checked is_some above" ) ;
1390+ let background_build_accumulator = Arc :: clone ( & build_accumulator) ;
1391+ let task = SpawnedTask :: spawn ( async move {
1392+ let result = collect_left_input_and_maybe_report (
1393+ random_state,
1394+ left_stream,
1395+ on_left. clone ( ) ,
1396+ background_join_metrics,
1397+ reservation,
1398+ with_visited_indices_bitmap,
1399+ 1 ,
1400+ enable_dynamic_filter_pushdown,
1401+ config,
1402+ null_equality,
1403+ array_map_created_count,
1404+ Some ( ( background_build_accumulator, partition) ) ,
1405+ )
1406+ . await ;
1407+ let _ = tx. send ( result) ;
1408+ } ) ;
1409+ build_accumulator. register_background_task ( task) ;
13931410 OnceFut :: new ( async move {
1394- task. await . map_err ( |err| {
1395- DataFusionError :: ExecutionJoin ( Box :: new ( err) )
1411+ rx. await . map_err ( |err| {
1412+ DataFusionError :: Execution ( format ! (
1413+ "hash join build task ended before sending its result: {err}"
1414+ ) )
13961415 } ) ?
13971416 } )
13981417 } else {
0 commit comments