-
Notifications
You must be signed in to change notification settings - Fork 1.2k
/
symmetric_hash_join.rs
2258 lines (2098 loc) · 80 KB
/
symmetric_hash_join.rs
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
//! This file implements the symmetric hash join algorithm with range-based
//! data pruning to join two (potentially infinite) streams.
//!
//! A [`SymmetricHashJoinExec`] plan takes two children plan (with appropriate
//! output ordering) and produces the join output according to the given join
//! type and other options.
//!
//! This plan uses the [`OneSideHashJoiner`] object to facilitate join calculations
//! for both its children.
use std::any::Any;
use std::fmt::{self, Debug};
use std::sync::Arc;
use std::task::Poll;
use std::{usize, vec};
use crate::common::SharedMemoryReservation;
use crate::joins::hash_join::{equal_rows_arr, update_hash};
use crate::joins::stream_join_utils::{
calculate_filter_expr_intervals, combine_two_batches,
convert_sort_expr_with_filter_schema, get_pruning_anti_indices,
get_pruning_semi_indices, prepare_sorted_exprs, record_visited_indices,
EagerJoinStream, EagerJoinStreamState, PruningJoinHashMap, SortedFilterExpr,
StreamJoinMetrics,
};
use crate::joins::utils::{
apply_join_filter_to_indices, build_batch_from_indices, build_join_schema,
check_join_is_valid, partitioned_join_output_partitioning, ColumnIndex, JoinFilter,
JoinHashMapType, JoinOn, StatefulStreamResult,
};
use crate::{
expressions::PhysicalSortExpr,
joins::StreamJoinPartitionMode,
metrics::{ExecutionPlanMetricsSet, MetricsSet},
DisplayAs, DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan,
Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics,
};
use arrow::array::{
ArrowPrimitiveType, NativeAdapter, PrimitiveArray, PrimitiveBuilder, UInt32Array,
UInt64Array,
};
use arrow::compute::concat_batches;
use arrow::datatypes::{Schema, SchemaRef};
use arrow::record_batch::RecordBatch;
use datafusion_common::hash_utils::create_hashes;
use datafusion_common::utils::bisect;
use datafusion_common::{
internal_err, plan_err, DataFusionError, JoinSide, JoinType, Result,
};
use datafusion_execution::memory_pool::MemoryConsumer;
use datafusion_execution::TaskContext;
use datafusion_expr::interval_arithmetic::Interval;
use datafusion_physical_expr::equivalence::join_equivalence_properties;
use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph;
use ahash::RandomState;
use datafusion_physical_expr::{PhysicalExprRef, PhysicalSortRequirement};
use futures::Stream;
use hashbrown::HashSet;
use parking_lot::Mutex;
const HASHMAP_SHRINK_SCALE_FACTOR: usize = 4;
/// A symmetric hash join with range conditions is when both streams are hashed on the
/// join key and the resulting hash tables are used to join the streams.
/// The join is considered symmetric because the hash table is built on the join keys from both
/// streams, and the matching of rows is based on the values of the join keys in both streams.
/// This type of join is efficient in streaming context as it allows for fast lookups in the hash
/// table, rather than having to scan through one or both of the streams to find matching rows, also it
/// only considers the elements from the stream that fall within a certain sliding window (w/ range conditions),
/// making it more efficient and less likely to store stale data. This enables operating on unbounded streaming
/// data without any memory issues.
///
/// For each input stream, create a hash table.
/// - For each new [RecordBatch] in build side, hash and insert into inputs hash table. Update offsets.
/// - Test if input is equal to a predefined set of other inputs.
/// - If so record the visited rows. If the matched row results must be produced (INNER, LEFT), output the [RecordBatch].
/// - Try to prune other side (probe) with new [RecordBatch].
/// - If the join type indicates that the unmatched rows results must be produced (LEFT, FULL etc.),
/// output the [RecordBatch] when a pruning happens or at the end of the data.
///
///
/// ``` text
/// +-------------------------+
/// | |
/// left stream ---------| Left OneSideHashJoiner |---+
/// | | |
/// +-------------------------+ |
/// |
/// |--------- Joined output
/// |
/// +-------------------------+ |
/// | | |
/// right stream ---------| Right OneSideHashJoiner |---+
/// | |
/// +-------------------------+
///
/// Prune build side when the new RecordBatch comes to the probe side. We utilize interval arithmetic
/// on JoinFilter's sorted PhysicalExprs to calculate the joinable range.
///
///
/// PROBE SIDE BUILD SIDE
/// BUFFER BUFFER
/// +-------------+ +------------+
/// | | | | Unjoinable
/// | | | | Range
/// | | | |
/// | | |---------------------------------
/// | | | | |
/// | | | | |
/// | | / | |
/// | | | | |
/// | | | | |
/// | | | | |
/// | | | | |
/// | | | | | Joinable
/// | |/ | | Range
/// | || | |
/// |+-----------+|| | |
/// || Record || | |
/// || Batch || | |
/// |+-----------+|| | |
/// +-------------+\ +------------+
/// |
/// \
/// |---------------------------------
///
/// This happens when range conditions are provided on sorted columns. E.g.
///
/// SELECT * FROM left_table, right_table
/// ON
/// left_key = right_key AND
/// left_time > right_time - INTERVAL 12 MINUTES AND left_time < right_time + INTERVAL 2 HOUR
///
/// or
/// SELECT * FROM left_table, right_table
/// ON
/// left_key = right_key AND
/// left_sorted > right_sorted - 3 AND left_sorted < right_sorted + 10
///
/// For general purpose, in the second scenario, when the new data comes to probe side, the conditions can be used to
/// determine a specific threshold for discarding rows from the inner buffer. For example, if the sort order the
/// two columns ("left_sorted" and "right_sorted") are ascending (it can be different in another scenarios)
/// and the join condition is "left_sorted > right_sorted - 3" and the latest value on the right input is 1234, meaning
/// that the left side buffer must only keep rows where "leftTime > rightTime - 3 > 1234 - 3 > 1231" ,
/// making the smallest value in 'left_sorted' 1231 and any rows below (since ascending)
/// than that can be dropped from the inner buffer.
/// ```
#[derive(Debug)]
pub struct SymmetricHashJoinExec {
/// Left side stream
pub(crate) left: Arc<dyn ExecutionPlan>,
/// Right side stream
pub(crate) right: Arc<dyn ExecutionPlan>,
/// Set of common columns used to join on
pub(crate) on: Vec<(PhysicalExprRef, PhysicalExprRef)>,
/// Filters applied when finding matching rows
pub(crate) filter: Option<JoinFilter>,
/// How the join is performed
pub(crate) join_type: JoinType,
/// The schema once the join is applied
schema: SchemaRef,
/// Shares the `RandomState` for the hashing algorithm
random_state: RandomState,
/// Execution metrics
metrics: ExecutionPlanMetricsSet,
/// Information of index and left / right placement of columns
column_indices: Vec<ColumnIndex>,
/// If null_equals_null is true, null == null else null != null
pub(crate) null_equals_null: bool,
/// Left side sort expression(s)
pub(crate) left_sort_exprs: Option<Vec<PhysicalSortExpr>>,
/// Right side sort expression(s)
pub(crate) right_sort_exprs: Option<Vec<PhysicalSortExpr>>,
/// Partition Mode
mode: StreamJoinPartitionMode,
}
impl SymmetricHashJoinExec {
/// Tries to create a new [SymmetricHashJoinExec].
/// # Error
/// This function errors when:
/// - It is not possible to join the left and right sides on keys `on`, or
/// - It fails to construct `SortedFilterExpr`s, or
/// - It fails to create the [ExprIntervalGraph].
#[allow(clippy::too_many_arguments)]
pub fn try_new(
left: Arc<dyn ExecutionPlan>,
right: Arc<dyn ExecutionPlan>,
on: JoinOn,
filter: Option<JoinFilter>,
join_type: &JoinType,
null_equals_null: bool,
left_sort_exprs: Option<Vec<PhysicalSortExpr>>,
right_sort_exprs: Option<Vec<PhysicalSortExpr>>,
mode: StreamJoinPartitionMode,
) -> Result<Self> {
let left_schema = left.schema();
let right_schema = right.schema();
// Error out if no "on" contraints are given:
if on.is_empty() {
return plan_err!(
"On constraints in SymmetricHashJoinExec should be non-empty"
);
}
// Check if the join is valid with the given on constraints:
check_join_is_valid(&left_schema, &right_schema, &on)?;
// Build the join schema from the left and right schemas:
let (schema, column_indices) =
build_join_schema(&left_schema, &right_schema, join_type);
// Initialize the random state for the join operation:
let random_state = RandomState::with_seeds(0, 0, 0, 0);
Ok(SymmetricHashJoinExec {
left,
right,
on,
filter,
join_type: *join_type,
schema: Arc::new(schema),
random_state,
metrics: ExecutionPlanMetricsSet::new(),
column_indices,
null_equals_null,
left_sort_exprs,
right_sort_exprs,
mode,
})
}
/// left stream
pub fn left(&self) -> &Arc<dyn ExecutionPlan> {
&self.left
}
/// right stream
pub fn right(&self) -> &Arc<dyn ExecutionPlan> {
&self.right
}
/// Set of common columns used to join on
pub fn on(&self) -> &[(PhysicalExprRef, PhysicalExprRef)] {
&self.on
}
/// Filters applied before join output
pub fn filter(&self) -> Option<&JoinFilter> {
self.filter.as_ref()
}
/// How the join is performed
pub fn join_type(&self) -> &JoinType {
&self.join_type
}
/// Get null_equals_null
pub fn null_equals_null(&self) -> bool {
self.null_equals_null
}
/// Get partition mode
pub fn partition_mode(&self) -> StreamJoinPartitionMode {
self.mode
}
/// Get left_sort_exprs
pub fn left_sort_exprs(&self) -> Option<&[PhysicalSortExpr]> {
self.left_sort_exprs.as_deref()
}
/// Get right_sort_exprs
pub fn right_sort_exprs(&self) -> Option<&[PhysicalSortExpr]> {
self.right_sort_exprs.as_deref()
}
/// Check if order information covers every column in the filter expression.
pub fn check_if_order_information_available(&self) -> Result<bool> {
if let Some(filter) = self.filter() {
let left = self.left();
if let Some(left_ordering) = left.output_ordering() {
let right = self.right();
if let Some(right_ordering) = right.output_ordering() {
let left_convertible = convert_sort_expr_with_filter_schema(
&JoinSide::Left,
filter,
&left.schema(),
&left_ordering[0],
)?
.is_some();
let right_convertible = convert_sort_expr_with_filter_schema(
&JoinSide::Right,
filter,
&right.schema(),
&right_ordering[0],
)?
.is_some();
return Ok(left_convertible && right_convertible);
}
}
}
Ok(false)
}
}
impl DisplayAs for SymmetricHashJoinExec {
fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
let display_filter = self.filter.as_ref().map_or_else(
|| "".to_string(),
|f| format!(", filter={}", f.expression()),
);
let on = self
.on
.iter()
.map(|(c1, c2)| format!("({}, {})", c1, c2))
.collect::<Vec<String>>()
.join(", ");
write!(
f,
"SymmetricHashJoinExec: mode={:?}, join_type={:?}, on=[{}]{}",
self.mode, self.join_type, on, display_filter
)
}
}
}
}
impl ExecutionPlan for SymmetricHashJoinExec {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn unbounded_output(&self, children: &[bool]) -> Result<bool> {
Ok(children.iter().any(|u| *u))
}
fn required_input_distribution(&self) -> Vec<Distribution> {
match self.mode {
StreamJoinPartitionMode::Partitioned => {
let (left_expr, right_expr) = self
.on
.iter()
.map(|(l, r)| (l.clone() as _, r.clone() as _))
.unzip();
vec![
Distribution::HashPartitioned(left_expr),
Distribution::HashPartitioned(right_expr),
]
}
StreamJoinPartitionMode::SinglePartition => {
vec![Distribution::SinglePartition, Distribution::SinglePartition]
}
}
}
fn required_input_ordering(&self) -> Vec<Option<Vec<PhysicalSortRequirement>>> {
vec![
self.left_sort_exprs
.as_ref()
.map(PhysicalSortRequirement::from_sort_exprs),
self.right_sort_exprs
.as_ref()
.map(PhysicalSortRequirement::from_sort_exprs),
]
}
fn output_partitioning(&self) -> Partitioning {
let left_columns_len = self.left.schema().fields.len();
partitioned_join_output_partitioning(
self.join_type,
self.left.output_partitioning(),
self.right.output_partitioning(),
left_columns_len,
)
}
// TODO: Output ordering might be kept for some cases.
fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
None
}
fn equivalence_properties(&self) -> EquivalenceProperties {
join_equivalence_properties(
self.left.equivalence_properties(),
self.right.equivalence_properties(),
&self.join_type,
self.schema(),
&self.maintains_input_order(),
// Has alternating probe side
None,
self.on(),
)
}
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
vec![self.left.clone(), self.right.clone()]
}
fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
Ok(Arc::new(SymmetricHashJoinExec::try_new(
children[0].clone(),
children[1].clone(),
self.on.clone(),
self.filter.clone(),
&self.join_type,
self.null_equals_null,
self.left_sort_exprs.clone(),
self.right_sort_exprs.clone(),
self.mode,
)?))
}
fn metrics(&self) -> Option<MetricsSet> {
Some(self.metrics.clone_inner())
}
fn statistics(&self) -> Result<Statistics> {
// TODO stats: it is not possible in general to know the output size of joins
Ok(Statistics::new_unknown(&self.schema()))
}
fn execute(
&self,
partition: usize,
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
let left_partitions = self.left.output_partitioning().partition_count();
let right_partitions = self.right.output_partitioning().partition_count();
if left_partitions != right_partitions {
return internal_err!(
"Invalid SymmetricHashJoinExec, partition count mismatch {left_partitions}!={right_partitions},\
consider using RepartitionExec"
);
}
// If `filter_state` and `filter` are both present, then calculate sorted filter expressions
// for both sides, and build an expression graph.
let (left_sorted_filter_expr, right_sorted_filter_expr, graph) =
match (&self.left_sort_exprs, &self.right_sort_exprs, &self.filter) {
(Some(left_sort_exprs), Some(right_sort_exprs), Some(filter)) => {
let (left, right, graph) = prepare_sorted_exprs(
filter,
&self.left,
&self.right,
left_sort_exprs,
right_sort_exprs,
)?;
(Some(left), Some(right), Some(graph))
}
// If `filter_state` or `filter` is not present, then return None for all three values:
_ => (None, None, None),
};
let (on_left, on_right) = self.on.iter().cloned().unzip();
let left_side_joiner =
OneSideHashJoiner::new(JoinSide::Left, on_left, self.left.schema());
let right_side_joiner =
OneSideHashJoiner::new(JoinSide::Right, on_right, self.right.schema());
let left_stream = self.left.execute(partition, context.clone())?;
let right_stream = self.right.execute(partition, context.clone())?;
let reservation = Arc::new(Mutex::new(
MemoryConsumer::new(format!("SymmetricHashJoinStream[{partition}]"))
.register(context.memory_pool()),
));
if let Some(g) = graph.as_ref() {
reservation.lock().try_grow(g.size())?;
}
Ok(Box::pin(SymmetricHashJoinStream {
left_stream,
right_stream,
schema: self.schema(),
filter: self.filter.clone(),
join_type: self.join_type,
random_state: self.random_state.clone(),
left: left_side_joiner,
right: right_side_joiner,
column_indices: self.column_indices.clone(),
metrics: StreamJoinMetrics::new(partition, &self.metrics),
graph,
left_sorted_filter_expr,
right_sorted_filter_expr,
null_equals_null: self.null_equals_null,
state: EagerJoinStreamState::PullRight,
reservation,
}))
}
}
/// A stream that issues [RecordBatch]es as they arrive from the right of the join.
struct SymmetricHashJoinStream {
/// Input streams
left_stream: SendableRecordBatchStream,
right_stream: SendableRecordBatchStream,
/// Input schema
schema: Arc<Schema>,
/// join filter
filter: Option<JoinFilter>,
/// type of the join
join_type: JoinType,
// left hash joiner
left: OneSideHashJoiner,
/// right hash joiner
right: OneSideHashJoiner,
/// Information of index and left / right placement of columns
column_indices: Vec<ColumnIndex>,
// Expression graph for range pruning.
graph: Option<ExprIntervalGraph>,
// Left globally sorted filter expr
left_sorted_filter_expr: Option<SortedFilterExpr>,
// Right globally sorted filter expr
right_sorted_filter_expr: Option<SortedFilterExpr>,
/// Random state used for hashing initialization
random_state: RandomState,
/// If null_equals_null is true, null == null else null != null
null_equals_null: bool,
/// Metrics
metrics: StreamJoinMetrics,
/// Memory reservation
reservation: SharedMemoryReservation,
/// State machine for input execution
state: EagerJoinStreamState,
}
impl RecordBatchStream for SymmetricHashJoinStream {
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}
impl Stream for SymmetricHashJoinStream {
type Item = Result<RecordBatch>;
fn poll_next(
mut self: std::pin::Pin<&mut Self>,
cx: &mut std::task::Context<'_>,
) -> Poll<Option<Self::Item>> {
self.poll_next_impl(cx)
}
}
/// Determine the pruning length for `buffer`.
///
/// This function evaluates the build side filter expression, converts the
/// result into an array and determines the pruning length by performing a
/// binary search on the array.
///
/// # Arguments
///
/// * `buffer`: The record batch to be pruned.
/// * `build_side_filter_expr`: The filter expression on the build side used
/// to determine the pruning length.
///
/// # Returns
///
/// A [Result] object that contains the pruning length. The function will return
/// an error if
/// - there is an issue evaluating the build side filter expression;
/// - there is an issue converting the build side filter expression into an array
fn determine_prune_length(
buffer: &RecordBatch,
build_side_filter_expr: &SortedFilterExpr,
) -> Result<usize> {
let origin_sorted_expr = build_side_filter_expr.origin_sorted_expr();
let interval = build_side_filter_expr.interval();
// Evaluate the build side filter expression and convert it into an array
let batch_arr = origin_sorted_expr
.expr
.evaluate(buffer)?
.into_array(buffer.num_rows())?;
// Get the lower or upper interval based on the sort direction
let target = if origin_sorted_expr.options.descending {
interval.upper().clone()
} else {
interval.lower().clone()
};
// Perform binary search on the array to determine the length of the record batch to be pruned
bisect::<true>(&[batch_arr], &[target], &[origin_sorted_expr.options])
}
/// This method determines if the result of the join should be produced in the final step or not.
///
/// # Arguments
///
/// * `build_side` - Enum indicating the side of the join used as the build side.
/// * `join_type` - Enum indicating the type of join to be performed.
///
/// # Returns
///
/// A boolean indicating whether the result of the join should be produced in the final step or not.
/// The result will be true if the build side is JoinSide::Left and the join type is one of
/// JoinType::Left, JoinType::LeftAnti, JoinType::Full or JoinType::LeftSemi.
/// If the build side is JoinSide::Right, the result will be true if the join type
/// is one of JoinType::Right, JoinType::RightAnti, JoinType::Full, or JoinType::RightSemi.
fn need_to_produce_result_in_final(build_side: JoinSide, join_type: JoinType) -> bool {
if build_side == JoinSide::Left {
matches!(
join_type,
JoinType::Left | JoinType::LeftAnti | JoinType::Full | JoinType::LeftSemi
)
} else {
matches!(
join_type,
JoinType::Right | JoinType::RightAnti | JoinType::Full | JoinType::RightSemi
)
}
}
/// Calculate indices by join type.
///
/// This method returns a tuple of two arrays: build and probe indices.
/// The length of both arrays will be the same.
///
/// # Arguments
///
/// * `build_side`: Join side which defines the build side.
/// * `prune_length`: Length of the prune data.
/// * `visited_rows`: Hash set of visited rows of the build side.
/// * `deleted_offset`: Deleted offset of the build side.
/// * `join_type`: The type of join to be performed.
///
/// # Returns
///
/// A tuple of two arrays of primitive types representing the build and probe indices.
///
fn calculate_indices_by_join_type<L: ArrowPrimitiveType, R: ArrowPrimitiveType>(
build_side: JoinSide,
prune_length: usize,
visited_rows: &HashSet<usize>,
deleted_offset: usize,
join_type: JoinType,
) -> Result<(PrimitiveArray<L>, PrimitiveArray<R>)>
where
NativeAdapter<L>: From<<L as ArrowPrimitiveType>::Native>,
{
// Store the result in a tuple
let result = match (build_side, join_type) {
// In the case of `Left` or `Right` join, or `Full` join, get the anti indices
(JoinSide::Left, JoinType::Left | JoinType::LeftAnti)
| (JoinSide::Right, JoinType::Right | JoinType::RightAnti)
| (_, JoinType::Full) => {
let build_unmatched_indices =
get_pruning_anti_indices(prune_length, deleted_offset, visited_rows);
let mut builder =
PrimitiveBuilder::<R>::with_capacity(build_unmatched_indices.len());
builder.append_nulls(build_unmatched_indices.len());
let probe_indices = builder.finish();
(build_unmatched_indices, probe_indices)
}
// In the case of `LeftSemi` or `RightSemi` join, get the semi indices
(JoinSide::Left, JoinType::LeftSemi) | (JoinSide::Right, JoinType::RightSemi) => {
let build_unmatched_indices =
get_pruning_semi_indices(prune_length, deleted_offset, visited_rows);
let mut builder =
PrimitiveBuilder::<R>::with_capacity(build_unmatched_indices.len());
builder.append_nulls(build_unmatched_indices.len());
let probe_indices = builder.finish();
(build_unmatched_indices, probe_indices)
}
// The case of other join types is not considered
_ => unreachable!(),
};
Ok(result)
}
/// This function produces unmatched record results based on the build side,
/// join type and other parameters.
///
/// The method uses first `prune_length` rows from the build side input buffer
/// to produce results.
///
/// # Arguments
///
/// * `output_schema` - The schema of the final output record batch.
/// * `prune_length` - The length of the determined prune length.
/// * `probe_schema` - The schema of the probe [RecordBatch].
/// * `join_type` - The type of join to be performed.
/// * `column_indices` - Indices of columns that are being joined.
///
/// # Returns
///
/// * `Option<RecordBatch>` - The final output record batch if required, otherwise [None].
pub(crate) fn build_side_determined_results(
build_hash_joiner: &OneSideHashJoiner,
output_schema: &SchemaRef,
prune_length: usize,
probe_schema: SchemaRef,
join_type: JoinType,
column_indices: &[ColumnIndex],
) -> Result<Option<RecordBatch>> {
// Check if we need to produce a result in the final output:
if prune_length > 0
&& need_to_produce_result_in_final(build_hash_joiner.build_side, join_type)
{
// Calculate the indices for build and probe sides based on join type and build side:
let (build_indices, probe_indices) = calculate_indices_by_join_type(
build_hash_joiner.build_side,
prune_length,
&build_hash_joiner.visited_rows,
build_hash_joiner.deleted_offset,
join_type,
)?;
// Create an empty probe record batch:
let empty_probe_batch = RecordBatch::new_empty(probe_schema);
// Build the final result from the indices of build and probe sides:
build_batch_from_indices(
output_schema.as_ref(),
&build_hash_joiner.input_buffer,
&empty_probe_batch,
&build_indices,
&probe_indices,
column_indices,
build_hash_joiner.build_side,
)
.map(|batch| (batch.num_rows() > 0).then_some(batch))
} else {
// If we don't need to produce a result, return None
Ok(None)
}
}
/// This method performs a join between the build side input buffer and the probe side batch.
///
/// # Arguments
///
/// * `build_hash_joiner` - Build side hash joiner
/// * `probe_hash_joiner` - Probe side hash joiner
/// * `schema` - A reference to the schema of the output record batch.
/// * `join_type` - The type of join to be performed.
/// * `on_probe` - An array of columns on which the join will be performed. The columns are from the probe side of the join.
/// * `filter` - An optional filter on the join condition.
/// * `probe_batch` - The second record batch to be joined.
/// * `column_indices` - An array of columns to be selected for the result of the join.
/// * `random_state` - The random state for the join.
/// * `null_equals_null` - A boolean indicating whether NULL values should be treated as equal when joining.
///
/// # Returns
///
/// A [Result] containing an optional record batch if the join type is not one of `LeftAnti`, `RightAnti`, `LeftSemi` or `RightSemi`.
/// If the join type is one of the above four, the function will return [None].
#[allow(clippy::too_many_arguments)]
pub(crate) fn join_with_probe_batch(
build_hash_joiner: &mut OneSideHashJoiner,
probe_hash_joiner: &mut OneSideHashJoiner,
schema: &SchemaRef,
join_type: JoinType,
filter: Option<&JoinFilter>,
probe_batch: &RecordBatch,
column_indices: &[ColumnIndex],
random_state: &RandomState,
null_equals_null: bool,
) -> Result<Option<RecordBatch>> {
if build_hash_joiner.input_buffer.num_rows() == 0 || probe_batch.num_rows() == 0 {
return Ok(None);
}
let (build_indices, probe_indices) = lookup_join_hashmap(
&build_hash_joiner.hashmap,
&build_hash_joiner.input_buffer,
probe_batch,
&build_hash_joiner.on,
&probe_hash_joiner.on,
random_state,
null_equals_null,
&mut build_hash_joiner.hashes_buffer,
Some(build_hash_joiner.deleted_offset),
)?;
let (build_indices, probe_indices) = if let Some(filter) = filter {
apply_join_filter_to_indices(
&build_hash_joiner.input_buffer,
probe_batch,
build_indices,
probe_indices,
filter,
build_hash_joiner.build_side,
)?
} else {
(build_indices, probe_indices)
};
if need_to_produce_result_in_final(build_hash_joiner.build_side, join_type) {
record_visited_indices(
&mut build_hash_joiner.visited_rows,
build_hash_joiner.deleted_offset,
&build_indices,
);
}
if need_to_produce_result_in_final(build_hash_joiner.build_side.negate(), join_type) {
record_visited_indices(
&mut probe_hash_joiner.visited_rows,
probe_hash_joiner.offset,
&probe_indices,
);
}
if matches!(
join_type,
JoinType::LeftAnti
| JoinType::RightAnti
| JoinType::LeftSemi
| JoinType::RightSemi
) {
Ok(None)
} else {
build_batch_from_indices(
schema,
&build_hash_joiner.input_buffer,
probe_batch,
&build_indices,
&probe_indices,
column_indices,
build_hash_joiner.build_side,
)
.map(|batch| (batch.num_rows() > 0).then_some(batch))
}
}
/// This method performs lookups against JoinHashMap by hash values of join-key columns, and handles potential
/// hash collisions.
///
/// # Arguments
///
/// * `build_hashmap` - hashmap collected from build side data.
/// * `build_batch` - Build side record batch.
/// * `probe_batch` - Probe side record batch.
/// * `build_on` - An array of columns on which the join will be performed. The columns are from the build side of the join.
/// * `probe_on` - An array of columns on which the join will be performed. The columns are from the probe side of the join.
/// * `random_state` - The random state for the join.
/// * `null_equals_null` - A boolean indicating whether NULL values should be treated as equal when joining.
/// * `hashes_buffer` - Buffer used for probe side keys hash calculation.
/// * `deleted_offset` - deleted offset for build side data.
///
/// # Returns
///
/// A [Result] containing a tuple with two equal length arrays, representing indices of rows from build and probe side,
/// matched by join key columns.
#[allow(clippy::too_many_arguments)]
fn lookup_join_hashmap(
build_hashmap: &PruningJoinHashMap,
build_batch: &RecordBatch,
probe_batch: &RecordBatch,
build_on: &[PhysicalExprRef],
probe_on: &[PhysicalExprRef],
random_state: &RandomState,
null_equals_null: bool,
hashes_buffer: &mut Vec<u64>,
deleted_offset: Option<usize>,
) -> Result<(UInt64Array, UInt32Array)> {
let keys_values = probe_on
.iter()
.map(|c| c.evaluate(probe_batch)?.into_array(probe_batch.num_rows()))
.collect::<Result<Vec<_>>>()?;
let build_join_values = build_on
.iter()
.map(|c| c.evaluate(build_batch)?.into_array(build_batch.num_rows()))
.collect::<Result<Vec<_>>>()?;
hashes_buffer.clear();
hashes_buffer.resize(probe_batch.num_rows(), 0);
let hash_values = create_hashes(&keys_values, random_state, hashes_buffer)?;
// As SymmetricHashJoin uses LIFO JoinHashMap, the chained list algorithm
// will return build indices for each probe row in a reverse order as such:
// Build Indices: [5, 4, 3]
// Probe Indices: [1, 1, 1]
//
// This affects the output sequence. Hypothetically, it's possible to preserve the lexicographic order on the build side.
// Let's consider probe rows [0,1] as an example:
//
// When the probe iteration sequence is reversed, the following pairings can be derived:
//
// For probe row 1:
// (5, 1)
// (4, 1)
// (3, 1)
//
// For probe row 0:
// (5, 0)
// (4, 0)
// (3, 0)
//
// After reversing both sets of indices, we obtain reversed indices:
//
// (3,0)
// (4,0)
// (5,0)
// (3,1)
// (4,1)
// (5,1)
//
// With this approach, the lexicographic order on both the probe side and the build side is preserved.
let (mut matched_probe, mut matched_build) = build_hashmap
.get_matched_indices(hash_values.iter().enumerate().rev(), deleted_offset);
matched_probe.as_slice_mut().reverse();
matched_build.as_slice_mut().reverse();
let build_indices: UInt64Array =
PrimitiveArray::new(matched_build.finish().into(), None);
let probe_indices: UInt32Array =
PrimitiveArray::new(matched_probe.finish().into(), None);
let (build_indices, probe_indices) = equal_rows_arr(
&build_indices,
&probe_indices,
&build_join_values,
&keys_values,
null_equals_null,
)?;
Ok((build_indices, probe_indices))
}
pub struct OneSideHashJoiner {
/// Build side
build_side: JoinSide,
/// Input record batch buffer
pub input_buffer: RecordBatch,
/// Columns from the side
pub(crate) on: Vec<PhysicalExprRef>,
/// Hashmap
pub(crate) hashmap: PruningJoinHashMap,
/// Reuse the hashes buffer
pub(crate) hashes_buffer: Vec<u64>,
/// Matched rows
pub(crate) visited_rows: HashSet<usize>,
/// Offset
pub(crate) offset: usize,
/// Deleted offset
pub(crate) deleted_offset: usize,
}
impl OneSideHashJoiner {
pub fn size(&self) -> usize {
let mut size = 0;
size += std::mem::size_of_val(self);
size += std::mem::size_of_val(&self.build_side);
size += self.input_buffer.get_array_memory_size();
size += std::mem::size_of_val(&self.on);
size += self.hashmap.size();
size += self.hashes_buffer.capacity() * std::mem::size_of::<u64>();
size += self.visited_rows.capacity() * std::mem::size_of::<usize>();
size += std::mem::size_of_val(&self.offset);
size += std::mem::size_of_val(&self.deleted_offset);
size
}
pub fn new(
build_side: JoinSide,
on: Vec<PhysicalExprRef>,
schema: SchemaRef,
) -> Self {
Self {
build_side,
input_buffer: RecordBatch::new_empty(schema),
on,
hashmap: PruningJoinHashMap::with_capacity(0),
hashes_buffer: vec![],
visited_rows: HashSet::new(),
offset: 0,
deleted_offset: 0,
}
}
/// Updates the internal state of the [OneSideHashJoiner] with the incoming batch.
///