1212// See the License for the specific language governing permissions and
1313// limitations under the License.
1414
15- use std:: sync:: atomic:: Ordering ;
1615use std:: time:: Instant ;
1716
1817use databend_common_exception:: Result ;
19- use databend_common_expression:: DataBlock ;
2018use databend_common_storages_fuse:: TableContext ;
2119
2220use super :: convert:: build_runtime_filter_infos;
2321use super :: global:: get_global_runtime_filter_packet;
24- use crate :: pipelines:: processors:: transforms:: build_runtime_filter_packet ;
22+ use crate :: pipelines:: processors:: transforms:: JoinRuntimeFilterPacket ;
2523use crate :: pipelines:: processors:: HashJoinBuildState ;
2624
2725pub async fn build_and_push_down_runtime_filter (
28- build_chunks : & [ DataBlock ] ,
29- build_num_rows : usize ,
26+ mut packet : JoinRuntimeFilterPacket ,
3027 join : & HashJoinBuildState ,
3128) -> Result < ( ) > {
3229 let overall_start = Instant :: now ( ) ;
3330
34- let is_spill_happened = join. hash_join_state . need_next_round . load ( Ordering :: Acquire )
35- || join
36- . hash_join_state
37- . is_spill_happened
38- . load ( Ordering :: Acquire ) ;
39-
40- let inlist_threshold = join
41- . ctx
42- . get_settings ( )
43- . get_inlist_runtime_filter_threshold ( ) ? as usize ;
44- let bloom_threshold = join
45- . ctx
46- . get_settings ( )
47- . get_bloom_runtime_filter_threshold ( ) ? as usize ;
48- let min_max_threshold = join
49- . ctx
50- . get_settings ( )
51- . get_min_max_runtime_filter_threshold ( ) ? as usize ;
52- let selectivity_threshold = join
53- . ctx
54- . get_settings ( )
55- . get_join_runtime_filter_selectivity_threshold ( ) ?;
56-
57- let build_start = Instant :: now ( ) ;
58- let mut packet = build_runtime_filter_packet (
59- build_chunks,
60- build_num_rows,
61- join. runtime_filter_desc ( ) ,
62- & join. func_ctx ,
63- inlist_threshold,
64- bloom_threshold,
65- min_max_threshold,
66- selectivity_threshold,
67- is_spill_happened,
68- ) ?;
69- let build_time = build_start. elapsed ( ) ;
70-
71- log:: info!( "RUNTIME-FILTER: build runtime filter packet: {:?}, build_num_rows: {}, runtime_filter_desc: {:?}" , packet, build_num_rows, join. runtime_filter_desc( ) ) ;
72-
7331 if let Some ( broadcast_id) = join. broadcast_id {
7432 let merge_start = Instant :: now ( ) ;
7533 packet = get_global_runtime_filter_packet ( broadcast_id, packet, & join. ctx ) . await ?;
@@ -85,7 +43,12 @@ pub async fn build_and_push_down_runtime_filter(
8543 . iter ( )
8644 . map ( |r| ( r. id , r) )
8745 . collect ( ) ;
46+ let selectivity_threshold = join
47+ . ctx
48+ . get_settings ( )
49+ . get_join_runtime_filter_selectivity_threshold ( ) ?;
8850 let max_threads = join. ctx . get_settings ( ) . get_max_threads ( ) ? as usize ;
51+ let build_rows = packet. build_rows ;
8952 let runtime_filter_infos = build_runtime_filter_infos (
9053 packet,
9154 runtime_filter_descs,
@@ -98,11 +61,10 @@ pub async fn build_and_push_down_runtime_filter(
9861 let filter_count = runtime_filter_infos. len ( ) ;
9962
10063 log:: info!(
101- "RUNTIME-FILTER: Built and deployed {} filters in {:?} (build: {:?}) for {} rows" ,
64+ "RUNTIME-FILTER: Built and deployed {} filters in {:?} for {} rows" ,
10265 filter_count,
10366 total_time,
104- build_time,
105- build_num_rows
67+ build_rows
10668 ) ;
10769 log:: info!(
10870 "RUNTIME-FILTER: runtime_filter_infos: {:?}" ,
0 commit comments