forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathhash_join.rs
2162 lines (1944 loc) · 72.7 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.
//! Defines the join plan for executing partitions in parallel and then joining the results
//! into a set of partitions.
use ahash::CallHasher;
use ahash::RandomState;
use arrow::{
array::{
ArrayData, ArrayRef, BooleanArray, Date32Array, Date64Array, Float32Array,
Float64Array, LargeStringArray, PrimitiveArray, TimestampMicrosecondArray,
TimestampMillisecondArray, TimestampNanosecondArray, UInt32BufferBuilder,
UInt32Builder, UInt64BufferBuilder, UInt64Builder,
},
compute,
datatypes::{TimeUnit, UInt32Type, UInt64Type},
};
use smallvec::{smallvec, SmallVec};
use std::{any::Any, usize};
use std::{hash::Hasher, sync::Arc};
use std::{time::Instant, vec};
use async_trait::async_trait;
use futures::{Stream, StreamExt, TryStreamExt};
use hashbrown::HashMap;
use tokio::sync::Mutex;
use arrow::array::Array;
use arrow::datatypes::DataType;
use arrow::datatypes::{Schema, SchemaRef};
use arrow::error::Result as ArrowResult;
use arrow::record_batch::RecordBatch;
use arrow::array::{
Int16Array, Int32Array, Int64Array, Int8Array, StringArray, UInt16Array, UInt32Array,
UInt64Array, UInt8Array,
};
use super::expressions::Column;
use super::{
hash_utils::{build_join_schema, check_join_is_valid, JoinOn, JoinType},
coalesce_partitions::CoalescePartitionsExec,
};
use crate::error::{DataFusionError, Result};
use super::{
DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream,
SendableRecordBatchStream,
};
use crate::physical_plan::coalesce_batches::concat_batches;
use crate::physical_plan::PhysicalExpr;
use log::debug;
// Maps a `u64` hash value based on the left ["on" values] to a list of indices with this key's value.
//
// Note that the `u64` keys are not stored in the hashmap (hence the `()` as key), but are only used
// to put the indices in a certain bucket.
// By allocating a `HashMap` with capacity for *at least* the number of rows for entries at the left side,
// we make sure that we don't have to re-hash the hashmap, which needs access to the key (the hash in this case) value.
// E.g. 1 -> [3, 6, 8] indicates that the column values map to rows 3, 6 and 8 for hash value 1
// As the key is a hash value, we need to check possible hash collisions in the probe stage
// During this stage it might be the case that a row is contained the same hashmap value,
// but the values don't match. Those are checked in the [equal_rows] macro
// TODO: speed up collission check and move away from using a hashbrown HashMap
// https://github.com/apache/arrow-datafusion/issues/50
type JoinHashMap = HashMap<(), SmallVec<[u64; 1]>, IdHashBuilder>;
type JoinLeftData = Arc<(JoinHashMap, RecordBatch)>;
/// join execution plan executes partitions in parallel and combines them into a set of
/// partitions.
#[derive(Debug)]
pub struct HashJoinExec {
/// left (build) side which gets hashed
left: Arc<dyn ExecutionPlan>,
/// right (probe) side which are filtered by the hash table
right: Arc<dyn ExecutionPlan>,
/// Set of common columns used to join on
on: Vec<(Column, Column)>,
/// How the join is performed
join_type: JoinType,
/// The schema once the join is applied
schema: SchemaRef,
/// Build-side
build_side: Arc<Mutex<Option<JoinLeftData>>>,
/// Shares the `RandomState` for the hashing algorithm
random_state: RandomState,
/// Partitioning mode to use
mode: PartitionMode,
}
#[derive(Clone, Copy, Debug, PartialEq)]
/// Partitioning mode to use for hash join
pub enum PartitionMode {
/// Left/right children are partitioned using the left and right keys
Partitioned,
/// Left side will collected into one partition
CollectLeft,
}
/// Information about the index and placement (left or right) of the columns
struct ColumnIndex {
/// Index of the column
index: usize,
/// Whether the column is at the left or right side
is_left: 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,
join_type: &JoinType,
partition_mode: PartitionMode,
) -> Result<Self> {
let left_schema = left.schema();
let right_schema = right.schema();
check_join_is_valid(&left_schema, &right_schema, &on)?;
let schema = Arc::new(build_join_schema(
&left_schema,
&right_schema,
&on,
join_type,
));
let random_state = RandomState::with_seeds(0, 0, 0, 0);
Ok(HashJoinExec {
left,
right,
on,
join_type: *join_type,
schema,
build_side: Arc::new(Mutex::new(None)),
random_state,
mode: partition_mode,
})
}
/// 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
}
/// How the join is performed
pub fn join_type(&self) -> &JoinType {
&self.join_type
}
/// Calculates column indices and left/right placement on input / output schemas and jointype
fn column_indices_from_schema(&self) -> ArrowResult<Vec<ColumnIndex>> {
let (primary_is_left, primary_schema, secondary_schema) = match self.join_type {
JoinType::Inner
| JoinType::Left
| JoinType::Full
| JoinType::Semi
| JoinType::Anti => (true, self.left.schema(), self.right.schema()),
JoinType::Right => (false, self.right.schema(), self.left.schema()),
};
let mut column_indices = Vec::with_capacity(self.schema.fields().len());
for field in self.schema.fields() {
let (is_primary, index) = match primary_schema.index_of(field.name()) {
Ok(i) => Ok((true, i)),
Err(_) => {
match secondary_schema.index_of(field.name()) {
Ok(i) => Ok((false, i)),
_ => Err(DataFusionError::Internal(
format!("During execution, the column {} was not found in neither the left or right side of the join", field.name()).to_string()
))
}
}
}.map_err(DataFusionError::into_arrow_external_error)?;
let is_left =
is_primary && primary_is_left || !is_primary && !primary_is_left;
column_indices.push(ColumnIndex { index, is_left });
}
Ok(column_indices)
}
}
#[async_trait]
impl ExecutionPlan for HashJoinExec {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
vec![self.left.clone(), self.right.clone()]
}
fn with_new_children(
&self,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
match children.len() {
2 => Ok(Arc::new(HashJoinExec::try_new(
children[0].clone(),
children[1].clone(),
self.on.clone(),
&self.join_type,
self.mode,
)?)),
_ => Err(DataFusionError::Internal(
"HashJoinExec wrong number of children".to_string(),
)),
}
}
fn output_partitioning(&self) -> Partitioning {
self.right.output_partitioning()
}
async fn execute(&self, partition: usize) -> Result<SendableRecordBatchStream> {
let on_left = self.on.iter().map(|on| on.0.clone()).collect::<Vec<_>>();
// we only want to compute the build side once for PartitionMode::CollectLeft
let left_data = {
match self.mode {
PartitionMode::CollectLeft => {
let mut build_side = self.build_side.lock().await;
match build_side.as_ref() {
Some(stream) => stream.clone(),
None => {
let start = Instant::now();
// merge all left parts into a single stream
let merge = CoalescePartitionsExec::new(self.left.clone());
let stream = merge.execute(0).await?;
// 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 = (0, Vec::new());
let (num_rows, batches) = stream
.try_fold(initial, |mut acc, batch| async {
acc.0 += batch.num_rows();
acc.1.push(batch);
Ok(acc)
})
.await?;
let mut hashmap = JoinHashMap::with_capacity_and_hasher(
num_rows,
IdHashBuilder {},
);
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,
&self.random_state,
&mut hashes_buffer,
)?;
offset += batch.num_rows();
}
// Merge all batches into a single batch, so we
// can directly index into the arrays
let single_batch =
concat_batches(&self.left.schema(), &batches, num_rows)?;
let left_side = Arc::new((hashmap, single_batch));
*build_side = Some(left_side.clone());
debug!(
"Built build-side of hash join containing {} rows in {} ms",
num_rows,
start.elapsed().as_millis()
);
left_side
}
}
}
PartitionMode::Partitioned => {
let start = Instant::now();
// Load 1 partition of left side in memory
let stream = self.left.execute(partition).await?;
// 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 = (0, Vec::new());
let (num_rows, batches) = stream
.try_fold(initial, |mut acc, batch| async {
acc.0 += batch.num_rows();
acc.1.push(batch);
Ok(acc)
})
.await?;
let mut hashmap =
JoinHashMap::with_capacity_and_hasher(num_rows, IdHashBuilder {});
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,
&self.random_state,
&mut hashes_buffer,
)?;
offset += batch.num_rows();
}
// Merge all batches into a single batch, so we
// can directly index into the arrays
let single_batch =
concat_batches(&self.left.schema(), &batches, num_rows)?;
let left_side = Arc::new((hashmap, single_batch));
debug!(
"Built build-side {} of hash join containing {} rows in {} ms",
partition,
num_rows,
start.elapsed().as_millis()
);
left_side
}
}
};
// 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).await?;
let on_right = self.on.iter().map(|on| on.1.clone()).collect::<Vec<_>>();
let column_indices = self.column_indices_from_schema()?;
let num_rows = left_data.1.num_rows();
let visited_left_side = match self.join_type {
JoinType::Left | JoinType::Full | JoinType::Semi | JoinType::Anti => {
vec![false; num_rows]
}
JoinType::Inner | JoinType::Right => vec![],
};
Ok(Box::pin(HashJoinStream::new(
self.schema.clone(),
on_left,
on_right,
self.join_type,
left_data,
right_stream,
column_indices,
self.random_state.clone(),
visited_left_side,
)))
}
fn fmt_as(
&self,
t: DisplayFormatType,
f: &mut std::fmt::Formatter,
) -> std::fmt::Result {
match t {
DisplayFormatType::Default => {
write!(
f,
"HashJoinExec: mode={:?}, join_type={:?}, on={:?}",
self.mode, self.join_type, self.on
)
}
}
}
}
/// Updates `hash` with new entries from [RecordBatch] evaluated against the expressions `on`,
/// assuming that the [RecordBatch] corresponds to the `index`th
fn update_hash(
on: &[Column],
batch: &RecordBatch,
hash: &mut JoinHashMap,
offset: usize,
random_state: &RandomState,
hashes_buffer: &mut Vec<u64>,
) -> Result<()> {
// 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)?;
// insert hashes to key of the hashmap
for (row, hash_value) in hash_values.iter().enumerate() {
match hash.raw_entry_mut().from_hash(*hash_value, |_| true) {
hashbrown::hash_map::RawEntryMut::Occupied(mut entry) => {
entry.get_mut().push((row + offset) as u64);
}
hashbrown::hash_map::RawEntryMut::Vacant(entry) => {
entry.insert_hashed_nocheck(
*hash_value,
(),
smallvec![(row + offset) as u64],
);
}
};
}
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>,
/// type of the join
join_type: JoinType,
/// information from the left
left_data: JoinLeftData,
/// right
right: SendableRecordBatchStream,
/// Information of index and left / right placement of columns
column_indices: Vec<ColumnIndex>,
/// number of input batches
num_input_batches: usize,
/// number of input rows
num_input_rows: usize,
/// number of batches produced
num_output_batches: usize,
/// number of rows produced
num_output_rows: usize,
/// total time for joining probe-side batches to the build-side batches
join_time: usize,
/// Random state used for hashing initialization
random_state: RandomState,
/// Keeps track of the left side rows whether they are visited
visited_left_side: Vec<bool>, // TODO: use a more memory efficient data structure, https://github.com/apache/arrow-datafusion/issues/240
/// There is nothing to process anymore and left side is processed in case of left join
is_exhausted: bool,
}
#[allow(clippy::too_many_arguments)]
impl HashJoinStream {
fn new(
schema: Arc<Schema>,
on_left: Vec<Column>,
on_right: Vec<Column>,
join_type: JoinType,
left_data: JoinLeftData,
right: SendableRecordBatchStream,
column_indices: Vec<ColumnIndex>,
random_state: RandomState,
visited_left_side: Vec<bool>,
) -> Self {
HashJoinStream {
schema,
on_left,
on_right,
join_type,
left_data,
right,
column_indices,
num_input_batches: 0,
num_input_rows: 0,
num_output_batches: 0,
num_output_rows: 0,
join_time: 0,
random_state,
visited_left_side,
is_exhausted: false,
}
}
}
impl RecordBatchStream for HashJoinStream {
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}
/// Returns a new [RecordBatch] by combining the `left` and `right` according to `indices`.
/// The resulting batch has [Schema] `schema`.
/// # Error
/// This function errors when:
/// *
fn build_batch_from_indices(
schema: &Schema,
left: &RecordBatch,
right: &RecordBatch,
left_indices: UInt64Array,
right_indices: UInt32Array,
column_indices: &[ColumnIndex],
) -> ArrowResult<(RecordBatch, UInt64Array)> {
// build the columns of the new [RecordBatch]:
// 1. pick whether the column is from the left or right
// 2. based on the pick, `take` items from the different RecordBatches
let mut columns: Vec<Arc<dyn Array>> = Vec::with_capacity(schema.fields().len());
for column_index in column_indices {
let array = if column_index.is_left {
let array = left.column(column_index.index);
compute::take(array.as_ref(), &left_indices, None)?
} else {
let array = right.column(column_index.index);
compute::take(array.as_ref(), &right_indices, None)?
};
columns.push(array);
}
RecordBatch::try_new(Arc::new(schema.clone()), columns).map(|x| (x, left_indices))
}
#[allow(clippy::too_many_arguments)]
fn build_batch(
batch: &RecordBatch,
left_data: &JoinLeftData,
on_left: &[Column],
on_right: &[Column],
join_type: JoinType,
schema: &Schema,
column_indices: &[ColumnIndex],
random_state: &RandomState,
) -> ArrowResult<(RecordBatch, UInt64Array)> {
let (left_indices, right_indices) =
build_join_indexes(left_data, batch, join_type, on_left, on_right, random_state)
.unwrap();
if matches!(join_type, JoinType::Semi | JoinType::Anti) {
return Ok((
RecordBatch::new_empty(Arc::new(schema.clone())),
left_indices,
));
}
build_batch_from_indices(
schema,
&left_data.1,
batch,
left_indices,
right_indices,
column_indices,
)
}
/// returns a vector with (index from left, index from right).
/// The size of this vector corresponds to the total size of a joined batch
// For a join on column A:
// left right
// batch 1
// A B A D
// ---------------
// 1 a 3 6
// 2 b 1 2
// 3 c 2 4
// batch 2
// A B A D
// ---------------
// 1 a 5 10
// 2 b 2 2
// 4 d 1 1
// indices (batch, batch_row)
// left right
// (0, 2) (0, 0)
// (0, 0) (0, 1)
// (0, 1) (0, 2)
// (1, 0) (0, 1)
// (1, 1) (0, 2)
// (0, 1) (1, 1)
// (0, 0) (1, 2)
// (1, 1) (1, 1)
// (1, 0) (1, 2)
fn build_join_indexes(
left_data: &JoinLeftData,
right: &RecordBatch,
join_type: JoinType,
left_on: &[Column],
right_on: &[Column],
random_state: &RandomState,
) -> Result<(UInt64Array, UInt32Array)> {
let keys_values = right_on
.iter()
.map(|c| Ok(c.evaluate(right)?.into_array(right.num_rows())))
.collect::<Result<Vec<_>>>()?;
let left_join_values = left_on
.iter()
.map(|c| Ok(c.evaluate(&left_data.1)?.into_array(left_data.1.num_rows())))
.collect::<Result<Vec<_>>>()?;
let hashes_buffer = &mut vec![0; keys_values[0].len()];
let hash_values = create_hashes(&keys_values, random_state, hashes_buffer)?;
let left = &left_data.0;
match join_type {
JoinType::Inner | JoinType::Semi | JoinType::Anti => {
// Using a buffer builder to avoid slower normal builder
let mut left_indices = UInt64BufferBuilder::new(0);
let mut right_indices = UInt32BufferBuilder::new(0);
// Visit all of the right rows
for (row, hash_value) in hash_values.iter().enumerate() {
// Get the hash and find it in the build index
// For every item on the left and right 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((_, indices)) =
left.raw_entry().from_hash(*hash_value, |_| true)
{
for &i in indices {
// Check hash collisions
if equal_rows(i as usize, row, &left_join_values, &keys_values)? {
left_indices.append(i);
right_indices.append(row as u32);
}
}
}
}
let left = ArrayData::builder(DataType::UInt64)
.len(left_indices.len())
.add_buffer(left_indices.finish())
.build();
let right = ArrayData::builder(DataType::UInt32)
.len(right_indices.len())
.add_buffer(right_indices.finish())
.build();
Ok((
PrimitiveArray::<UInt64Type>::from(left),
PrimitiveArray::<UInt32Type>::from(right),
))
}
JoinType::Left => {
let mut left_indices = UInt64Builder::new(0);
let mut right_indices = UInt32Builder::new(0);
// First visit all of the rows
for (row, hash_value) in hash_values.iter().enumerate() {
if let Some((_, indices)) =
left.raw_entry().from_hash(*hash_value, |_| true)
{
for &i in indices {
// Collision check
if equal_rows(i as usize, row, &left_join_values, &keys_values)? {
left_indices.append_value(i)?;
right_indices.append_value(row as u32)?;
}
}
};
}
Ok((left_indices.finish(), right_indices.finish()))
}
JoinType::Right | JoinType::Full => {
let mut left_indices = UInt64Builder::new(0);
let mut right_indices = UInt32Builder::new(0);
for (row, hash_value) in hash_values.iter().enumerate() {
match left.raw_entry().from_hash(*hash_value, |_| true) {
Some((_, indices)) => {
for &i in indices {
if equal_rows(
i as usize,
row,
&left_join_values,
&keys_values,
)? {
left_indices.append_value(i)?;
} else {
left_indices.append_null()?;
}
right_indices.append_value(row as u32)?;
}
}
None => {
// when no match, add the row with None for the left side
left_indices.append_null()?;
right_indices.append_value(row as u32)?;
}
}
}
Ok((left_indices.finish(), right_indices.finish()))
}
}
}
use core::hash::BuildHasher;
/// `Hasher` that returns the same `u64` value as a hash, to avoid re-hashing
/// it when inserting/indexing or regrowing the `HashMap`
struct IdHasher {
hash: u64,
}
impl Hasher for IdHasher {
fn finish(&self) -> u64 {
self.hash
}
fn write_u64(&mut self, i: u64) {
self.hash = i;
}
fn write(&mut self, _bytes: &[u8]) {
unreachable!("IdHasher should only be used for u64 keys")
}
}
#[derive(Debug)]
struct IdHashBuilder {}
impl BuildHasher for IdHashBuilder {
type Hasher = IdHasher;
fn build_hasher(&self) -> Self::Hasher {
IdHasher { hash: 0 }
}
}
// Combines two hashes into one hash
#[inline]
fn combine_hashes(l: u64, r: u64) -> u64 {
let hash = (17 * 37u64).wrapping_add(l);
hash.wrapping_mul(37).wrapping_add(r)
}
macro_rules! equal_rows_elem {
($array_type:ident, $l: ident, $r: ident, $left: ident, $right: ident) => {{
let left_array = $l.as_any().downcast_ref::<$array_type>().unwrap();
let right_array = $r.as_any().downcast_ref::<$array_type>().unwrap();
match (left_array.is_null($left), left_array.is_null($right)) {
(false, false) => left_array.value($left) == right_array.value($right),
_ => false,
}
}};
}
/// Left and right row have equal values
fn equal_rows(
left: usize,
right: usize,
left_arrays: &[ArrayRef],
right_arrays: &[ArrayRef],
) -> Result<bool> {
let mut err = None;
let res = left_arrays
.iter()
.zip(right_arrays)
.all(|(l, r)| match l.data_type() {
DataType::Null => true,
DataType::Boolean => equal_rows_elem!(BooleanArray, l, r, left, right),
DataType::Int8 => equal_rows_elem!(Int8Array, l, r, left, right),
DataType::Int16 => equal_rows_elem!(Int16Array, l, r, left, right),
DataType::Int32 => equal_rows_elem!(Int32Array, l, r, left, right),
DataType::Int64 => equal_rows_elem!(Int64Array, l, r, left, right),
DataType::UInt8 => equal_rows_elem!(UInt8Array, l, r, left, right),
DataType::UInt16 => equal_rows_elem!(UInt16Array, l, r, left, right),
DataType::UInt32 => equal_rows_elem!(UInt32Array, l, r, left, right),
DataType::UInt64 => equal_rows_elem!(UInt64Array, l, r, left, right),
DataType::Timestamp(_, None) => {
equal_rows_elem!(Int64Array, l, r, left, right)
}
DataType::Utf8 => equal_rows_elem!(StringArray, l, r, left, right),
DataType::LargeUtf8 => equal_rows_elem!(LargeStringArray, l, r, left, right),
_ => {
// This is internal because we should have caught this before.
err = Some(Err(DataFusionError::Internal(
"Unsupported data type in hasher".to_string(),
)));
false
}
});
err.unwrap_or(Ok(res))
}
macro_rules! hash_array {
($array_type:ident, $column: ident, $ty: ident, $hashes: ident, $random_state: ident, $multi_col: ident) => {
let array = $column.as_any().downcast_ref::<$array_type>().unwrap();
if array.null_count() == 0 {
if $multi_col {
for (i, hash) in $hashes.iter_mut().enumerate() {
*hash = combine_hashes(
$ty::get_hash(&array.value(i), $random_state),
*hash,
);
}
} else {
for (i, hash) in $hashes.iter_mut().enumerate() {
*hash = $ty::get_hash(&array.value(i), $random_state);
}
}
} else {
if $multi_col {
for (i, hash) in $hashes.iter_mut().enumerate() {
if !array.is_null(i) {
*hash = combine_hashes(
$ty::get_hash(&array.value(i), $random_state),
*hash,
);
}
}
} else {
for (i, hash) in $hashes.iter_mut().enumerate() {
if !array.is_null(i) {
*hash = $ty::get_hash(&array.value(i), $random_state);
}
}
}
}
};
}
macro_rules! hash_array_primitive {
($array_type:ident, $column: ident, $ty: ident, $hashes: ident, $random_state: ident, $multi_col: ident) => {
let array = $column.as_any().downcast_ref::<$array_type>().unwrap();
let values = array.values();
if array.null_count() == 0 {
if $multi_col {
for (hash, value) in $hashes.iter_mut().zip(values.iter()) {
*hash = combine_hashes($ty::get_hash(value, $random_state), *hash);
}
} else {
for (hash, value) in $hashes.iter_mut().zip(values.iter()) {
*hash = $ty::get_hash(value, $random_state)
}
}
} else {
if $multi_col {
for (i, (hash, value)) in
$hashes.iter_mut().zip(values.iter()).enumerate()
{
if !array.is_null(i) {
*hash =
combine_hashes($ty::get_hash(value, $random_state), *hash);
}
}
} else {
for (i, (hash, value)) in
$hashes.iter_mut().zip(values.iter()).enumerate()
{
if !array.is_null(i) {
*hash = $ty::get_hash(value, $random_state);
}
}
}
}
};
}
macro_rules! hash_array_float {
($array_type:ident, $column: ident, $ty: ident, $hashes: ident, $random_state: ident, $multi_col: ident) => {
let array = $column.as_any().downcast_ref::<$array_type>().unwrap();
let values = array.values();
if array.null_count() == 0 {
if $multi_col {
for (hash, value) in $hashes.iter_mut().zip(values.iter()) {
*hash = combine_hashes(
$ty::get_hash(
&$ty::from_le_bytes(value.to_le_bytes()),
$random_state,
),
*hash,
);
}
} else {
for (hash, value) in $hashes.iter_mut().zip(values.iter()) {
*hash = $ty::get_hash(
&$ty::from_le_bytes(value.to_le_bytes()),
$random_state,
)
}
}
} else {
if $multi_col {
for (i, (hash, value)) in
$hashes.iter_mut().zip(values.iter()).enumerate()
{
if !array.is_null(i) {
*hash = combine_hashes(
$ty::get_hash(
&$ty::from_le_bytes(value.to_le_bytes()),
$random_state,
),
*hash,
);
}
}
} else {
for (i, (hash, value)) in
$hashes.iter_mut().zip(values.iter()).enumerate()
{
if !array.is_null(i) {
*hash = $ty::get_hash(
&$ty::from_le_bytes(value.to_le_bytes()),
$random_state,
);
}
}
}
}
};
}
/// Creates hash values for every element in the row based on the values in the columns
pub fn create_hashes<'a>(
arrays: &[ArrayRef],
random_state: &RandomState,
hashes_buffer: &'a mut Vec<u64>,
) -> Result<&'a mut Vec<u64>> {
// combine hashes with `combine_hashes` if we have more than 1 column
let multi_col = arrays.len() > 1;
for col in arrays {
match col.data_type() {
DataType::UInt8 => {
hash_array_primitive!(
UInt8Array,
col,
u8,
hashes_buffer,
random_state,
multi_col
);
}
DataType::UInt16 => {
hash_array_primitive!(
UInt16Array,
col,
u16,
hashes_buffer,
random_state,
multi_col
);
}
DataType::UInt32 => {
hash_array_primitive!(
UInt32Array,
col,
u32,
hashes_buffer,
random_state,
multi_col
);
}
DataType::UInt64 => {
hash_array_primitive!(
UInt64Array,
col,
u64,
hashes_buffer,
random_state,
multi_col
);
}
DataType::Int8 => {
hash_array_primitive!(
Int8Array,
col,
i8,
hashes_buffer,