@@ -64,7 +64,7 @@ use datafusion_physical_expr::PhysicalExpr;
64
64
use datafusion_physical_expr_common:: sort_expr:: { LexOrdering , LexRequirement } ;
65
65
use futures:: stream:: Stream ;
66
66
use futures:: { ready, StreamExt } ;
67
- use hashbrown:: raw :: RawTable ;
67
+ use hashbrown:: hash_table :: HashTable ;
68
68
use log:: debug;
69
69
70
70
/// Window execution plan
@@ -440,16 +440,16 @@ pub struct LinearSearch {
440
440
/// is ordered by a, b and the window expression contains a PARTITION BY b, a
441
441
/// clause, this attribute stores [1, 0].
442
442
ordered_partition_by_indices : Vec < usize > ,
443
- /// We use this [`RawTable `] to calculate unique partitions for each new
443
+ /// We use this [`HashTable `] to calculate unique partitions for each new
444
444
/// RecordBatch. First entry in the tuple is the hash value, the second
445
445
/// entry is the unique ID for each partition (increments from 0 to n).
446
- row_map_batch : RawTable < ( u64 , usize ) > ,
447
- /// We use this [`RawTable `] to calculate the output columns that we can
446
+ row_map_batch : HashTable < ( u64 , usize ) > ,
447
+ /// We use this [`HashTable `] to calculate the output columns that we can
448
448
/// produce at each cycle. First entry in the tuple is the hash value, the
449
449
/// second entry is the unique ID for each partition (increments from 0 to n).
450
450
/// The third entry stores how many new outputs are calculated for the
451
451
/// corresponding partition.
452
- row_map_out : RawTable < ( u64 , usize , usize ) > ,
452
+ row_map_out : HashTable < ( u64 , usize , usize ) > ,
453
453
input_schema : SchemaRef ,
454
454
}
455
455
@@ -608,8 +608,8 @@ impl LinearSearch {
608
608
input_buffer_hashes : VecDeque :: new ( ) ,
609
609
random_state : Default :: default ( ) ,
610
610
ordered_partition_by_indices,
611
- row_map_batch : RawTable :: with_capacity ( 256 ) ,
612
- row_map_out : RawTable :: with_capacity ( 256 ) ,
611
+ row_map_batch : HashTable :: with_capacity ( 256 ) ,
612
+ row_map_out : HashTable :: with_capacity ( 256 ) ,
613
613
input_schema,
614
614
}
615
615
}
@@ -629,7 +629,7 @@ impl LinearSearch {
629
629
// res stores PartitionKey and row indices (indices where these partition occurs in the `batch`) for each partition.
630
630
let mut result: Vec < ( PartitionKey , Vec < u32 > ) > = vec ! [ ] ;
631
631
for ( hash, row_idx) in batch_hashes. into_iter ( ) . zip ( 0u32 ..) {
632
- let entry = self . row_map_batch . get_mut ( hash, |( _, group_idx) | {
632
+ let entry = self . row_map_batch . find_mut ( hash, |( _, group_idx) | {
633
633
// We can safely get the first index of the partition indices
634
634
// since partition indices has one element during initialization.
635
635
let row = get_row_at_idx ( columns, row_idx as usize ) . unwrap ( ) ;
@@ -639,8 +639,11 @@ impl LinearSearch {
639
639
if let Some ( ( _, group_idx) ) = entry {
640
640
result[ * group_idx] . 1 . push ( row_idx)
641
641
} else {
642
- self . row_map_batch
643
- . insert ( hash, ( hash, result. len ( ) ) , |( hash, _) | * hash) ;
642
+ self . row_map_batch . insert_unique (
643
+ hash,
644
+ ( hash, result. len ( ) ) ,
645
+ |( hash, _) | * hash,
646
+ ) ;
644
647
let row = get_row_at_idx ( columns, row_idx as usize ) ?;
645
648
// This is a new partition its only index is row_idx for now.
646
649
result. push ( ( row, vec ! [ row_idx] ) ) ;
@@ -665,7 +668,7 @@ impl LinearSearch {
665
668
self . row_map_out . clear ( ) ;
666
669
let mut partition_indices: Vec < ( PartitionKey , Vec < u32 > ) > = vec ! [ ] ;
667
670
for ( hash, row_idx) in self . input_buffer_hashes . iter ( ) . zip ( 0u32 ..) {
668
- let entry = self . row_map_out . get_mut ( * hash, |( _, group_idx, _) | {
671
+ let entry = self . row_map_out . find_mut ( * hash, |( _, group_idx, _) | {
669
672
let row =
670
673
get_row_at_idx ( & partition_by_columns, row_idx as usize ) . unwrap ( ) ;
671
674
row == partition_indices[ * group_idx] . 0
@@ -691,7 +694,7 @@ impl LinearSearch {
691
694
if min_out == 0 {
692
695
break ;
693
696
}
694
- self . row_map_out . insert (
697
+ self . row_map_out . insert_unique (
695
698
* hash,
696
699
( * hash, partition_indices. len ( ) , min_out) ,
697
700
|( hash, _, _) | * hash,
0 commit comments