@@ -92,8 +92,6 @@ impl TransformHashJoinBuild {
92
92
if wait {
93
93
self . step = HashJoinBuildStep :: WaitSpill ;
94
94
} else {
95
- // Make `need_spill` to false for `SpillCoordinator`
96
- spill_state. spill_coordinator . no_need_spill ( ) ;
97
95
// Before notify all processors to spill, we need to collect all buffered data in `RowSpace` and `Chunks`
98
96
// Partition all rows and stat how many partitions and rows in each partition.
99
97
// Then choose the largest partitions(which contain rows that can avoid oom exactly) to spill.
@@ -116,12 +114,17 @@ impl TransformHashJoinBuild {
116
114
117
115
// Called after processor read spilled data
118
116
// It means next round build will start, need to reset some variables.
119
- fn reset ( & mut self ) -> Result < ( ) > {
117
+ async fn reset ( & mut self ) -> Result < ( ) > {
120
118
self . finalize_finished = false ;
121
119
self . from_spill = true ;
122
120
// Only need to reset the following variables once
123
- if self . build_state . row_space_builders . load ( Ordering :: Relaxed ) == 0 {
124
- self . build_state . send_val . store ( 2 , Ordering :: Relaxed ) ;
121
+ if self
122
+ . build_state
123
+ . row_space_builders
124
+ . fetch_add ( 1 , Ordering :: Acquire )
125
+ == 0
126
+ {
127
+ self . build_state . send_val . store ( 2 , Ordering :: Release ) ;
125
128
// Before build processors into `WaitProbe` state, set the channel message to false.
126
129
// Then after all probe processors are ready, the last one will send true to channel and wake up all build processors.
127
130
self . build_state
@@ -130,16 +133,14 @@ impl TransformHashJoinBuild {
130
133
. send ( false )
131
134
. map_err ( |_| ErrorCode :: TokioError ( "continue_build_watcher channel is closed" ) ) ?;
132
135
let worker_num = self . build_state . build_worker_num . load ( Ordering :: Relaxed ) as usize ;
133
- self . build_state
134
- . row_space_builders
135
- . store ( worker_num, Ordering :: Relaxed ) ;
136
136
self . build_state
137
137
. hash_join_state
138
138
. hash_table_builders
139
139
. store ( worker_num, Ordering :: Relaxed ) ;
140
140
self . build_state . hash_join_state . reset ( ) ;
141
141
}
142
142
self . step = HashJoinBuildStep :: Running ;
143
+ self . build_state . restore_barrier . wait ( ) . await ;
143
144
Ok ( ( ) )
144
145
}
145
146
}
@@ -166,11 +167,13 @@ impl Processor for TransformHashJoinBuild {
166
167
// The processor won't be triggered spill, because there won't be data from input port
167
168
// Add the processor to `non_spill_processors`
168
169
let spill_coordinator = & spill_state. spill_coordinator ;
169
- let waiting_spill_count = spill_coordinator. waiting_spill_count . load ( Ordering :: Relaxed ) ;
170
- let non_spill_processors = spill_coordinator. increase_non_spill_processors ( ) ;
171
- info ! ( "waiting_spill_count: {:?}, non_spill_processors: {:?}, total_builder_count: {:?}" , waiting_spill_count, non_spill_processors, spill_state. spill_coordinator. total_builder_count) ;
172
- if waiting_spill_count != 0 && non_spill_processors + waiting_spill_count == spill_state. spill_coordinator . total_builder_count {
170
+ let mut non_spill_processors = spill_coordinator. non_spill_processors . write ( ) ;
171
+ * non_spill_processors += 1 ;
172
+ let waiting_spill_count = spill_coordinator. waiting_spill_count . load ( Ordering :: Acquire ) ;
173
+ info ! ( "waiting_spill_count: {:?}, non_spill_processors: {:?}, total_builder_count: {:?}" , waiting_spill_count, * non_spill_processors, spill_state. spill_coordinator. total_builder_count) ;
174
+ if ( waiting_spill_count != 0 && * non_spill_processors + waiting_spill_count == spill_state. spill_coordinator . total_builder_count ) && spill_coordinator. get_need_spill ( ) {
173
175
spill_coordinator. no_need_spill ( ) ;
176
+ drop ( non_spill_processors) ;
174
177
let mut spill_task = spill_coordinator. spill_tasks . lock ( ) ;
175
178
spill_state. split_spill_tasks ( spill_coordinator. active_processor_num ( ) , & mut spill_task) ?;
176
179
spill_coordinator. waiting_spill_count . store ( 0 , Ordering :: Relaxed ) ;
@@ -201,15 +204,7 @@ impl Processor for TransformHashJoinBuild {
201
204
true => {
202
205
// If join spill is enabled, we should wait probe to spill.
203
206
// Then restore data from disk and build hash table, util all spilled data are processed.
204
- if let Some ( spill_state) = & mut self . spill_state {
205
- // Send spilled partition to `HashJoinState`, used by probe spill.
206
- // The method should be called only once.
207
- if !self . send_partition_set {
208
- self . build_state
209
- . hash_join_state
210
- . set_spilled_partition ( & spill_state. spiller . spilled_partition_set ) ;
211
- self . send_partition_set = true ;
212
- }
207
+ if self . spill_state . is_some ( ) {
213
208
self . step = HashJoinBuildStep :: WaitProbe ;
214
209
Ok ( Event :: Async )
215
210
} else {
@@ -273,6 +268,16 @@ impl Processor for TransformHashJoinBuild {
273
268
self . build_state . finalize ( task)
274
269
} else {
275
270
self . finalize_finished = true ;
271
+ if let Some ( spill_state) = & mut self . spill_state {
272
+ // Send spilled partition to `HashJoinState`, used by probe spill.
273
+ // The method should be called only once.
274
+ if !self . send_partition_set {
275
+ self . build_state
276
+ . hash_join_state
277
+ . set_spilled_partition ( & spill_state. spiller . spilled_partition_set ) ;
278
+ self . send_partition_set = true ;
279
+ }
280
+ }
276
281
self . build_state . build_done ( )
277
282
}
278
283
}
@@ -364,7 +369,7 @@ impl Processor for TransformHashJoinBuild {
364
369
self . input_data = Some ( DataBlock :: concat ( & spilled_data) ?) ;
365
370
}
366
371
self . build_state . restore_barrier . wait ( ) . await ;
367
- self . reset ( ) ?;
372
+ self . reset ( ) . await ?;
368
373
}
369
374
_ => unreachable ! ( ) ,
370
375
}
0 commit comments