-
Notifications
You must be signed in to change notification settings - Fork 1.2k
/
hash_join.rs
3000 lines (2652 loc) · 105 KB
/
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.
//! [`HashJoinExec`] Partitioned Hash Join Operator
use std::fmt;
use std::mem::size_of;
use std::sync::Arc;
use std::task::Poll;
use std::{any::Any, usize, vec};
use crate::joins::utils::{
adjust_indices_by_join_type, apply_join_filter_to_indices, build_batch_from_indices,
calculate_join_output_ordering, get_final_indices_from_bit_map,
need_produce_result_in_final,
};
use crate::DisplayAs;
use crate::{
coalesce_batches::concat_batches,
coalesce_partitions::CoalescePartitionsExec,
expressions::Column,
expressions::PhysicalSortExpr,
hash_utils::create_hashes,
joins::hash_join_utils::{JoinHashMap, JoinHashMapType},
joins::utils::{
adjust_right_output_partitioning, build_join_schema, check_join_is_valid,
estimate_join_statistics, partitioned_join_output_partitioning,
BuildProbeJoinMetrics, ColumnIndex, JoinFilter, JoinOn,
},
metrics::{ExecutionPlanMetricsSet, MetricsSet},
DisplayFormatType, Distribution, ExecutionPlan, Partitioning, PhysicalExpr,
RecordBatchStream, SendableRecordBatchStream, Statistics,
};
use super::{
utils::{OnceAsync, OnceFut},
PartitionMode,
};
use arrow::array::{
Array, ArrayRef, BooleanArray, BooleanBufferBuilder, PrimitiveArray, UInt32Array,
UInt32BufferBuilder, UInt64Array, UInt64BufferBuilder,
};
use arrow::compute::kernels::cmp::{eq, not_distinct};
use arrow::compute::{and, take, FilterBuilder};
use arrow::datatypes::{Schema, SchemaRef};
use arrow::record_batch::RecordBatch;
use arrow::util::bit_util;
use arrow_array::cast::downcast_array;
use arrow_schema::ArrowError;
use datafusion_common::{
exec_err, internal_err, plan_err, DataFusionError, JoinSide, JoinType, Result,
};
use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation};
use datafusion_execution::TaskContext;
use datafusion_physical_expr::equivalence::join_equivalence_properties;
use datafusion_physical_expr::EquivalenceProperties;
use ahash::RandomState;
use futures::{ready, Stream, StreamExt, TryStreamExt};
type JoinLeftData = (JoinHashMap, RecordBatch, MemoryReservation);
/// Join execution plan: Evaluates eqijoin predicates in parallel on multiple
/// partitions using a hash table and an optional filter list to apply post
/// join.
///
/// # Join Expressions
///
/// This implementation is optimized for evaluating eqijoin predicates (
/// `<col1> = <col2>`) expressions, which are represented as a list of `Columns`
/// in [`Self::on`].
///
/// Non-equality predicates, which can not pushed down to a join inputs (e.g.
/// `<col1> != <col2>`) are known as "filter expressions" and are evaluated
/// after the equijoin predicates.
///
/// # "Build Side" vs "Probe Side"
///
/// HashJoin takes two inputs, which are referred to as the "build" and the
/// "probe". The build side is the first child, and the probe side is the second
/// child.
///
/// The two inputs are treated differently and it is VERY important that the
/// *smaller* input is placed on the build side to minimize the work of creating
/// the hash table.
///
/// ```text
/// ┌───────────┐
/// │ HashJoin │
/// │ │
/// └───────────┘
/// │ │
/// ┌─────┘ └─────┐
/// ▼ ▼
/// ┌────────────┐ ┌─────────────┐
/// │ Input │ │ Input │
/// │ [0] │ │ [1] │
/// └────────────┘ └─────────────┘
///
/// "build side" "probe side"
/// ```
///
/// Execution proceeds in 2 stages:
///
/// 1. the **build phase** where a hash table is created from the tuples of the
/// build side.
///
/// 2. the **probe phase** where the tuples of the probe side are streamed
/// through, checking for matches of the join keys in the hash table.
///
/// ```text
/// ┌────────────────┐ ┌────────────────┐
/// │ ┌─────────┐ │ │ ┌─────────┐ │
/// │ │ Hash │ │ │ │ Hash │ │
/// │ │ Table │ │ │ │ Table │ │
/// │ │(keys are│ │ │ │(keys are│ │
/// │ │equi join│ │ │ │equi join│ │ Stage 2: batches from
/// Stage 1: the │ │columns) │ │ │ │columns) │ │ the probe side are
/// *entire* build │ │ │ │ │ │ │ │ streamed through, and
/// side is read │ └─────────┘ │ │ └─────────┘ │ checked against the
/// into the hash │ ▲ │ │ ▲ │ contents of the hash
/// table │ HashJoin │ │ HashJoin │ table
/// └──────┼─────────┘ └──────────┼─────┘
/// ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─
/// │ │
///
/// │ │
/// ┌────────────┐ ┌────────────┐
/// │RecordBatch │ │RecordBatch │
/// └────────────┘ └────────────┘
/// ┌────────────┐ ┌────────────┐
/// │RecordBatch │ │RecordBatch │
/// └────────────┘ └────────────┘
/// ... ...
/// ┌────────────┐ ┌────────────┐
/// │RecordBatch │ │RecordBatch │
/// └────────────┘ └────────────┘
///
/// build side probe side
///
/// ```
///
/// # Example "Optimal" Plans
///
/// The differences in the inputs means that for classic "Star Schema Query",
/// the optimal plan will be a **"Right Deep Tree"** . A Star Schema Query is
/// one where there is one large table and several smaller "dimension" tables,
/// joined on `Foreign Key = Primary Key` predicates.
///
/// A "Right Deep Tree" looks like this large table as the probe side on the
/// lowest join:
///
/// ```text
/// ┌───────────┐
/// │ HashJoin │
/// │ │
/// └───────────┘
/// │ │
/// ┌───────┘ └──────────┐
/// ▼ ▼
/// ┌───────────────┐ ┌───────────┐
/// │ small table 1 │ │ HashJoin │
/// │ "dimension" │ │ │
/// └───────────────┘ └───┬───┬───┘
/// ┌──────────┘ └───────┐
/// │ │
/// ▼ ▼
/// ┌───────────────┐ ┌───────────┐
/// │ small table 2 │ │ HashJoin │
/// │ "dimension" │ │ │
/// └───────────────┘ └───┬───┬───┘
/// ┌────────┘ └────────┐
/// │ │
/// ▼ ▼
/// ┌───────────────┐ ┌───────────────┐
/// │ small table 3 │ │ large table │
/// │ "dimension" │ │ "fact" │
/// └───────────────┘ └───────────────┘
/// ```
#[derive(Debug)]
pub struct HashJoinExec {
/// left (build) side which gets hashed
pub left: Arc<dyn ExecutionPlan>,
/// right (probe) side which are filtered by the hash table
pub right: Arc<dyn ExecutionPlan>,
/// Set of equijoin columns from the relations: `(left_col, right_col)`
pub on: Vec<(Column, Column)>,
/// Filters which are applied while finding matching rows
pub filter: Option<JoinFilter>,
/// How the join is performed (`OUTER`, `INNER`, etc)
pub join_type: JoinType,
/// The output schema for the join
schema: SchemaRef,
/// Build-side data
left_fut: OnceAsync<JoinLeftData>,
/// Shared the `RandomState` for the hashing algorithm
random_state: RandomState,
/// Output order
output_order: Option<Vec<PhysicalSortExpr>>,
/// Partitioning mode to use
pub mode: PartitionMode,
/// Execution metrics
metrics: ExecutionPlanMetricsSet,
/// Information of index and left / right placement of columns
column_indices: Vec<ColumnIndex>,
/// Null matching behavior: If `null_equals_null` is true, rows that have
/// `null`s in both left and right equijoin columns will be matched.
/// Otherwise, rows that have `null`s in the join columns will not be
/// matched and thus will not appear in the output.
pub null_equals_null: bool,
}
impl HashJoinExec {
/// Tries to create a new [HashJoinExec].
///
/// # Error
/// This function errors when it is not possible to join the left and right sides on keys `on`.
pub fn try_new(
left: Arc<dyn ExecutionPlan>,
right: Arc<dyn ExecutionPlan>,
on: JoinOn,
filter: Option<JoinFilter>,
join_type: &JoinType,
partition_mode: PartitionMode,
null_equals_null: bool,
) -> Result<Self> {
let left_schema = left.schema();
let right_schema = right.schema();
if on.is_empty() {
return plan_err!("On constraints in HashJoinExec should be non-empty");
}
check_join_is_valid(&left_schema, &right_schema, &on)?;
let (schema, column_indices) =
build_join_schema(&left_schema, &right_schema, join_type);
let random_state = RandomState::with_seeds(0, 0, 0, 0);
let output_order = calculate_join_output_ordering(
left.output_ordering().unwrap_or(&[]),
right.output_ordering().unwrap_or(&[]),
*join_type,
&on,
left_schema.fields.len(),
&Self::maintains_input_order(*join_type),
Some(Self::probe_side()),
);
Ok(HashJoinExec {
left,
right,
on,
filter,
join_type: *join_type,
schema: Arc::new(schema),
left_fut: Default::default(),
random_state,
mode: partition_mode,
metrics: ExecutionPlanMetricsSet::new(),
column_indices,
null_equals_null,
output_order,
})
}
/// left (build) side which gets hashed
pub fn left(&self) -> &Arc<dyn ExecutionPlan> {
&self.left
}
/// right (probe) side which are filtered by the hash table
pub fn right(&self) -> &Arc<dyn ExecutionPlan> {
&self.right
}
/// Set of common columns used to join on
pub fn on(&self) -> &[(Column, Column)] {
&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
}
/// The partitioning mode of this hash join
pub fn partition_mode(&self) -> &PartitionMode {
&self.mode
}
/// Get null_equals_null
pub fn null_equals_null(&self) -> bool {
self.null_equals_null
}
/// Calculate order preservation flags for this hash join.
fn maintains_input_order(join_type: JoinType) -> Vec<bool> {
vec![
false,
matches!(
join_type,
JoinType::Inner | JoinType::RightAnti | JoinType::RightSemi
),
]
}
/// Get probe side information for the hash join.
pub fn probe_side() -> JoinSide {
// In current implementation right side is always probe side.
JoinSide::Right
}
}
impl DisplayAs for HashJoinExec {
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,
"HashJoinExec: mode={:?}, join_type={:?}, on=[{}]{}",
self.mode, self.join_type, on, display_filter
)
}
}
}
}
impl ExecutionPlan for HashJoinExec {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn required_input_distribution(&self) -> Vec<Distribution> {
match self.mode {
PartitionMode::CollectLeft => vec![
Distribution::SinglePartition,
Distribution::UnspecifiedDistribution,
],
PartitionMode::Partitioned => {
let (left_expr, right_expr) = self
.on
.iter()
.map(|(l, r)| {
(
Arc::new(l.clone()) as Arc<dyn PhysicalExpr>,
Arc::new(r.clone()) as Arc<dyn PhysicalExpr>,
)
})
.unzip();
vec![
Distribution::HashPartitioned(left_expr),
Distribution::HashPartitioned(right_expr),
]
}
PartitionMode::Auto => vec![
Distribution::UnspecifiedDistribution,
Distribution::UnspecifiedDistribution,
],
}
}
/// Specifies whether this plan generates an infinite stream of records.
/// If the plan does not support pipelining, but its input(s) are
/// infinite, returns an error to indicate this.
fn unbounded_output(&self, children: &[bool]) -> Result<bool> {
let (left, right) = (children[0], children[1]);
// If left is unbounded, or right is unbounded with JoinType::Right,
// JoinType::Full, JoinType::RightAnti types.
let breaking = left
|| (right
&& matches!(
self.join_type,
JoinType::Left
| JoinType::Full
| JoinType::LeftAnti
| JoinType::LeftSemi
));
if breaking {
plan_err!(
"Join Error: The join with cannot be executed with unbounded inputs. {}",
if left && right {
"Currently, we do not support unbounded inputs on both sides."
} else {
"Please consider a different type of join or sources."
}
)
} else {
Ok(left || right)
}
}
fn output_partitioning(&self) -> Partitioning {
let left_columns_len = self.left.schema().fields.len();
match self.mode {
PartitionMode::CollectLeft => match self.join_type {
JoinType::Inner | JoinType::Right => adjust_right_output_partitioning(
self.right.output_partitioning(),
left_columns_len,
),
JoinType::RightSemi | JoinType::RightAnti => {
self.right.output_partitioning()
}
JoinType::Left
| JoinType::LeftSemi
| JoinType::LeftAnti
| JoinType::Full => Partitioning::UnknownPartitioning(
self.right.output_partitioning().partition_count(),
),
},
PartitionMode::Partitioned => partitioned_join_output_partitioning(
self.join_type,
self.left.output_partitioning(),
self.right.output_partitioning(),
left_columns_len,
),
PartitionMode::Auto => Partitioning::UnknownPartitioning(
self.right.output_partitioning().partition_count(),
),
}
}
fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
self.output_order.as_deref()
}
// For [JoinType::Inner] and [JoinType::RightSemi] in hash joins, the probe phase initiates by
// applying the hash function to convert the join key(s) in each row into a hash value from the
// probe side table in the order they're arranged. The hash value is used to look up corresponding
// entries in the hash table that was constructed from the build side table during the build phase.
//
// Because of the immediate generation of result rows once a match is found,
// the output of the join tends to follow the order in which the rows were read from
// the probe side table. This is simply due to the sequence in which the rows were processed.
// Hence, it appears that the hash join is preserving the order of the probe side.
//
// Meanwhile, in the case of a [JoinType::RightAnti] hash join,
// the unmatched rows from the probe side are also kept in order.
// This is because the **`RightAnti`** join is designed to return rows from the right
// (probe side) table that have no match in the left (build side) table. Because the rows
// are processed sequentially in the probe phase, and unmatched rows are directly output
// as results, these results tend to retain the order of the probe side table.
fn maintains_input_order(&self) -> Vec<bool> {
Self::maintains_input_order(self.join_type)
}
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(),
Some(Self::probe_side()),
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(HashJoinExec::try_new(
children[0].clone(),
children[1].clone(),
self.on.clone(),
self.filter.clone(),
&self.join_type,
self.mode,
self.null_equals_null,
)?))
}
fn execute(
&self,
partition: usize,
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
let on_left = self.on.iter().map(|on| on.0.clone()).collect::<Vec<_>>();
let on_right = self.on.iter().map(|on| on.1.clone()).collect::<Vec<_>>();
let left_partitions = self.left.output_partitioning().partition_count();
let right_partitions = self.right.output_partitioning().partition_count();
if self.mode == PartitionMode::Partitioned && left_partitions != right_partitions
{
return internal_err!(
"Invalid HashJoinExec, partition count mismatch {left_partitions}!={right_partitions},\
consider using RepartitionExec"
);
}
let join_metrics = BuildProbeJoinMetrics::new(partition, &self.metrics);
let left_fut = match self.mode {
PartitionMode::CollectLeft => self.left_fut.once(|| {
let reservation =
MemoryConsumer::new("HashJoinInput").register(context.memory_pool());
collect_left_input(
None,
self.random_state.clone(),
self.left.clone(),
on_left.clone(),
context.clone(),
join_metrics.clone(),
reservation,
)
}),
PartitionMode::Partitioned => {
let reservation =
MemoryConsumer::new(format!("HashJoinInput[{partition}]"))
.register(context.memory_pool());
OnceFut::new(collect_left_input(
Some(partition),
self.random_state.clone(),
self.left.clone(),
on_left.clone(),
context.clone(),
join_metrics.clone(),
reservation,
))
}
PartitionMode::Auto => {
return plan_err!(
"Invalid HashJoinExec, unsupported PartitionMode {:?} in execute()",
PartitionMode::Auto
);
}
};
let reservation = MemoryConsumer::new(format!("HashJoinStream[{partition}]"))
.register(context.memory_pool());
// we have the batches and the hash map with their keys. We can how create a stream
// over the right that uses this information to issue new batches.
let right_stream = self.right.execute(partition, context)?;
Ok(Box::pin(HashJoinStream {
schema: self.schema(),
on_left,
on_right,
filter: self.filter.clone(),
join_type: self.join_type,
left_fut,
visited_left_side: None,
right: right_stream,
column_indices: self.column_indices.clone(),
random_state: self.random_state.clone(),
join_metrics,
null_equals_null: self.null_equals_null,
is_exhausted: false,
reservation,
}))
}
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
// There are some special cases though, for example:
// - `A LEFT JOIN B ON A.col=B.col` with `COUNT_DISTINCT(B.col)=COUNT(B.col)`
estimate_join_statistics(
self.left.clone(),
self.right.clone(),
self.on.clone(),
&self.join_type,
&self.schema,
)
}
}
async fn collect_left_input(
partition: Option<usize>,
random_state: RandomState,
left: Arc<dyn ExecutionPlan>,
on_left: Vec<Column>,
context: Arc<TaskContext>,
metrics: BuildProbeJoinMetrics,
reservation: MemoryReservation,
) -> Result<JoinLeftData> {
let schema = left.schema();
let (left_input, left_input_partition) = if let Some(partition) = partition {
(left, partition)
} else if left.output_partitioning().partition_count() != 1 {
(Arc::new(CoalescePartitionsExec::new(left)) as _, 0)
} else {
(left, 0)
};
// Depending on partition argument load single partition or whole left side in memory
let stream = left_input.execute(left_input_partition, context.clone())?;
// This operation performs 2 steps at once:
// 1. creates a [JoinHashMap] of all batches from the stream
// 2. stores the batches in a vector.
let initial = (Vec::new(), 0, metrics, reservation);
let (batches, num_rows, metrics, mut reservation) = stream
.try_fold(initial, |mut acc, batch| async {
let batch_size = batch.get_array_memory_size();
// Reserve memory for incoming batch
acc.3.try_grow(batch_size)?;
// Update metrics
acc.2.build_mem_used.add(batch_size);
acc.2.build_input_batches.add(1);
acc.2.build_input_rows.add(batch.num_rows());
// Update rowcount
acc.1 += batch.num_rows();
// Push batch to output
acc.0.push(batch);
Ok(acc)
})
.await?;
// Estimation of memory size, required for hashtable, prior to allocation.
// Final result can be verified using `RawTable.allocation_info()`
//
// For majority of cases hashbrown overestimates buckets qty to keep ~1/8 of them empty.
// This formula leads to overallocation for small tables (< 8 elements) but fine overall.
let estimated_buckets = (num_rows.checked_mul(8).ok_or_else(|| {
DataFusionError::Execution(
"usize overflow while estimating number of hasmap buckets".to_string(),
)
})? / 7)
.next_power_of_two();
// 16 bytes per `(u64, u64)`
// + 1 byte for each bucket
// + fixed size of JoinHashMap (RawTable + Vec)
let estimated_hastable_size =
16 * estimated_buckets + estimated_buckets + size_of::<JoinHashMap>();
reservation.try_grow(estimated_hastable_size)?;
metrics.build_mem_used.add(estimated_hastable_size);
let mut hashmap = JoinHashMap::with_capacity(num_rows);
let mut hashes_buffer = Vec::new();
let mut offset = 0;
for batch in batches.iter() {
hashes_buffer.clear();
hashes_buffer.resize(batch.num_rows(), 0);
update_hash(
&on_left,
batch,
&mut hashmap,
offset,
&random_state,
&mut hashes_buffer,
0,
)?;
offset += batch.num_rows();
}
// Merge all batches into a single batch, so we
// can directly index into the arrays
let single_batch = concat_batches(&schema, &batches, num_rows)?;
Ok((hashmap, single_batch, reservation))
}
/// Updates `hash` with new entries from [RecordBatch] evaluated against the expressions `on`,
/// assuming that the [RecordBatch] corresponds to the `index`th
pub fn update_hash<T>(
on: &[Column],
batch: &RecordBatch,
hash_map: &mut T,
offset: usize,
random_state: &RandomState,
hashes_buffer: &mut Vec<u64>,
deleted_offset: usize,
) -> Result<()>
where
T: JoinHashMapType,
{
// evaluate the keys
let keys_values = on
.iter()
.map(|c| Ok(c.evaluate(batch)?.into_array(batch.num_rows())))
.collect::<Result<Vec<_>>>()?;
// calculate the hash values
let hash_values = create_hashes(&keys_values, random_state, hashes_buffer)?;
// For usual JoinHashmap, the implementation is void.
hash_map.extend_zero(batch.num_rows());
// insert hashes to key of the hashmap
let (mut_map, mut_list) = hash_map.get_mut();
for (row, hash_value) in hash_values.iter().enumerate() {
let item = mut_map.get_mut(*hash_value, |(hash, _)| *hash_value == *hash);
if let Some((_, index)) = item {
// Already exists: add index to next array
let prev_index = *index;
// Store new value inside hashmap
*index = (row + offset + 1) as u64;
// Update chained Vec at row + offset with previous value
mut_list[row + offset - deleted_offset] = prev_index;
} else {
mut_map.insert(
*hash_value,
// store the value + 1 as 0 value reserved for end of list
(*hash_value, (row + offset + 1) as u64),
|(hash, _)| *hash,
);
// chained list at (row + offset) is already initialized with 0
// meaning end of list
}
}
Ok(())
}
/// A stream that issues [RecordBatch]es as they arrive from the right of the join.
struct HashJoinStream {
/// Input schema
schema: Arc<Schema>,
/// columns from the left
on_left: Vec<Column>,
/// columns from the right used to compute the hash
on_right: Vec<Column>,
/// join filter
filter: Option<JoinFilter>,
/// type of the join
join_type: JoinType,
/// future for data from left side
left_fut: OnceFut<JoinLeftData>,
/// Keeps track of the left side rows whether they are visited
visited_left_side: Option<BooleanBufferBuilder>,
/// right
right: SendableRecordBatchStream,
/// Random state used for hashing initialization
random_state: RandomState,
/// There is nothing to process anymore and left side is processed in case of left join
is_exhausted: bool,
/// Metrics
join_metrics: BuildProbeJoinMetrics,
/// Information of index and left / right placement of columns
column_indices: Vec<ColumnIndex>,
/// If null_equals_null is true, null == null else null != null
null_equals_null: bool,
/// Memory reservation
reservation: MemoryReservation,
}
impl RecordBatchStream for HashJoinStream {
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}
// Returns build/probe indices satisfying the equality condition.
// On LEFT.b1 = RIGHT.b2
// LEFT Table:
// a1 b1 c1
// 1 1 10
// 3 3 30
// 5 5 50
// 7 7 70
// 9 8 90
// 11 8 110
// 13 10 130
// RIGHT Table:
// a2 b2 c2
// 2 2 20
// 4 4 40
// 6 6 60
// 8 8 80
// 10 10 100
// 12 10 120
// The result is
// "+----+----+-----+----+----+-----+",
// "| a1 | b1 | c1 | a2 | b2 | c2 |",
// "+----+----+-----+----+----+-----+",
// "| 9 | 8 | 90 | 8 | 8 | 80 |",
// "| 11 | 8 | 110 | 8 | 8 | 80 |",
// "| 13 | 10 | 130 | 10 | 10 | 100 |",
// "| 13 | 10 | 130 | 12 | 10 | 120 |",
// "+----+----+-----+----+----+-----+"
// And the result of build and probe indices are:
// Build indices: 4, 5, 6, 6
// Probe indices: 3, 3, 4, 5
#[allow(clippy::too_many_arguments)]
pub fn build_equal_condition_join_indices<T: JoinHashMapType>(
build_hashmap: &T,
build_input_buffer: &RecordBatch,
probe_batch: &RecordBatch,
build_on: &[Column],
probe_on: &[Column],
random_state: &RandomState,
null_equals_null: bool,
hashes_buffer: &mut Vec<u64>,
filter: Option<&JoinFilter>,
build_side: JoinSide,
deleted_offset: Option<usize>,
) -> Result<(UInt64Array, UInt32Array)> {
let keys_values = probe_on
.iter()
.map(|c| Ok(c.evaluate(probe_batch)?.into_array(probe_batch.num_rows())))
.collect::<Result<Vec<_>>>()?;
let build_join_values = build_on
.iter()
.map(|c| {
Ok(c.evaluate(build_input_buffer)?
.into_array(build_input_buffer.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)?;
// Using a buffer builder to avoid slower normal builder
let mut build_indices = UInt64BufferBuilder::new(0);
let mut probe_indices = UInt32BufferBuilder::new(0);
// The chained list algorithm generates build indices for each probe row in a reversed sequence 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 hash_map = build_hashmap.get_map();
let next_chain = build_hashmap.get_list();
for (row, hash_value) in hash_values.iter().enumerate().rev() {
// Get the hash and find it in the build index
// For every item on the build and probe we check if it matches
// This possibly contains rows with hash collisions,
// So we have to check here whether rows are equal or not
if let Some((_, index)) =
hash_map.get(*hash_value, |(hash, _)| *hash_value == *hash)
{
let mut i = *index - 1;
loop {
let build_row_value = if let Some(offset) = deleted_offset {
// This arguments means that we prune the next index way before here.
if i < offset as u64 {
// End of the list due to pruning
break;
}
i - offset as u64
} else {
i
};
build_indices.append(build_row_value);
probe_indices.append(row as u32);
// Follow the chain to get the next index value
let next = next_chain[build_row_value as usize];
if next == 0 {
// end of list
break;
}
i = next - 1;
}
}
}
// Reversing both sets of indices
build_indices.as_slice_mut().reverse();
probe_indices.as_slice_mut().reverse();
let left: UInt64Array = PrimitiveArray::new(build_indices.finish().into(), None);
let right: UInt32Array = PrimitiveArray::new(probe_indices.finish().into(), None);
let (left, right) = if let Some(filter) = filter {
// Filter the indices which satisfy the non-equal join condition, like `left.b1 = 10`
apply_join_filter_to_indices(
build_input_buffer,
probe_batch,
left,
right,
filter,
build_side,
)?
} else {
(left, right)
};
equal_rows_arr(
&left,
&right,
&build_join_values,
&keys_values,
null_equals_null,
)
}
// version of eq_dyn supporting equality on null arrays
fn eq_dyn_null(
left: &dyn Array,
right: &dyn Array,
null_equals_null: bool,
) -> Result<BooleanArray, ArrowError> {
match (left.data_type(), right.data_type()) {
_ if null_equals_null => not_distinct(&left, &right),
_ => eq(&left, &right),
}
}
pub fn equal_rows_arr(
indices_left: &UInt64Array,
indices_right: &UInt32Array,
left_arrays: &[ArrayRef],
right_arrays: &[ArrayRef],
null_equals_null: bool,
) -> Result<(UInt64Array, UInt32Array)> {
let mut iter = left_arrays.iter().zip(right_arrays.iter());
let (first_left, first_right) = iter.next().ok_or_else(|| {
DataFusionError::Internal(
"At least one array should be provided for both left and right".to_string(),
)
})?;
let arr_left = take(first_left.as_ref(), indices_left, None)?;
let arr_right = take(first_right.as_ref(), indices_right, None)?;
let mut equal: BooleanArray = eq_dyn_null(&arr_left, &arr_right, null_equals_null)?;
// Use map and try_fold to iterate over the remaining pairs of arrays.
// In each iteration, take is used on the pair of arrays and their equality is determined.
// The results are then folded (combined) using the and function to get a final equality result.
equal = iter
.map(|(left, right)| {
let arr_left = take(left.as_ref(), indices_left, None)?;
let arr_right = take(right.as_ref(), indices_right, None)?;
eq_dyn_null(arr_left.as_ref(), arr_right.as_ref(), null_equals_null)
})
.try_fold(equal, |acc, equal2| and(&acc, &equal2?))?;
let filter_builder = FilterBuilder::new(&equal).optimize().build();
let left_filtered = filter_builder.filter(indices_left)?;
let right_filtered = filter_builder.filter(indices_right)?;
Ok((
downcast_array(left_filtered.as_ref()),
downcast_array(right_filtered.as_ref()),
))
}
impl HashJoinStream {
/// Separate implementation function that unpins the [`HashJoinStream`] so
/// that partial borrows work correctly