@@ -77,8 +77,26 @@ use arrow_buffer::BooleanBuffer;
77
77
use datafusion_expr:: Operator ;
78
78
use datafusion_physical_expr_common:: datum:: compare_op_for_nested;
79
79
use futures:: { ready, Stream , StreamExt , TryStreamExt } ;
80
+ use log:: debug;
80
81
use parking_lot:: Mutex ;
81
82
83
+ pub const RANDOM_STATE : RandomState = RandomState :: with_seeds ( 0 , 0 , 0 , 0 ) ;
84
+
85
+ #[ derive( Default ) ]
86
+ pub struct JoinContext {
87
+ build_state : Mutex < Option < Arc < JoinLeftData > > > ,
88
+ }
89
+
90
+ impl JoinContext {
91
+ pub fn set_build_state ( & self , state : Arc < JoinLeftData > ) {
92
+ self . build_state . lock ( ) . replace ( state) ;
93
+ }
94
+
95
+ pub fn get_build_state ( & self ) -> Option < Arc < JoinLeftData > > {
96
+ self . build_state . lock ( ) . clone ( )
97
+ }
98
+ }
99
+
82
100
pub struct SharedJoinState {
83
101
state_impl : Arc < dyn SharedJoinStateImpl > ,
84
102
}
@@ -128,7 +146,7 @@ pub trait SharedJoinStateImpl: Send + Sync + 'static {
128
146
type SharedBitmapBuilder = Mutex < BooleanBufferBuilder > ;
129
147
130
148
/// HashTable and input data for the left (build side) of a join
131
- struct JoinLeftData {
149
+ pub struct JoinLeftData {
132
150
/// The hash table with indices into `batch`
133
151
hash_map : JoinHashMap ,
134
152
/// The input rows for the build side
@@ -165,6 +183,10 @@ impl JoinLeftData {
165
183
}
166
184
}
167
185
186
+ pub fn contains_hash ( & self , hash : u64 ) -> bool {
187
+ self . hash_map . contains_hash ( hash)
188
+ }
189
+
168
190
/// return a reference to the hash map
169
191
fn hash_map ( & self ) -> & JoinHashMap {
170
192
& self . hash_map
@@ -768,6 +790,7 @@ impl ExecutionPlan for HashJoinExec {
768
790
769
791
let distributed_state =
770
792
context. session_config ( ) . get_extension :: < SharedJoinState > ( ) ;
793
+ let join_context = context. session_config ( ) . get_extension :: < JoinContext > ( ) ;
771
794
772
795
let join_metrics = BuildProbeJoinMetrics :: new ( partition, & self . metrics ) ;
773
796
let left_fut = match self . mode {
@@ -855,6 +878,7 @@ impl ExecutionPlan for HashJoinExec {
855
878
batch_size,
856
879
hashes_buffer : vec ! [ ] ,
857
880
right_side_ordered : self . right . output_ordering ( ) . is_some ( ) ,
881
+ join_context,
858
882
} ) )
859
883
}
860
884
@@ -1187,6 +1211,7 @@ struct HashJoinStream {
1187
1211
hashes_buffer : Vec < u64 > ,
1188
1212
/// Specifies whether the right side has an ordering to potentially preserve
1189
1213
right_side_ordered : bool ,
1214
+ join_context : Option < Arc < JoinContext > > ,
1190
1215
}
1191
1216
1192
1217
impl RecordBatchStream for HashJoinStream {
@@ -1399,6 +1424,11 @@ impl HashJoinStream {
1399
1424
. get_shared( cx) ) ?;
1400
1425
build_timer. done ( ) ;
1401
1426
1427
+ if let Some ( ctx) = self . join_context . as_ref ( ) {
1428
+ debug ! ( "setting join left data in join context" ) ;
1429
+ ctx. set_build_state ( Arc :: clone ( & left_data) ) ;
1430
+ }
1431
+
1402
1432
self . state = HashJoinStreamState :: FetchProbeBatch ;
1403
1433
self . build_side = BuildSide :: Ready ( BuildSideReadyState { left_data } ) ;
1404
1434
0 commit comments