risingwave_common/hash/
dispatcher.rs1use super::HashKey;
16use crate::dispatch_data_types;
17use crate::hash::{
18 self, HashKeySer, HeapNullBitmap, MAX_GROUP_KEYS_ON_STACK, NullBitmap, StackNullBitmap,
19};
20use crate::types::DataType;
21
22#[derive(Copy, Clone, Debug, Eq, PartialEq)]
24pub enum HashKeyKind {
25 Key8,
26 Key16,
27 Key32,
28 Key64,
29 Key128,
30 Key256,
31 KeySerialized,
32}
33
34impl HashKeyKind {
35 fn order_by_key_size() -> impl IntoIterator<Item = (HashKeyKind, usize)> {
36 use HashKeyKind::*;
37 [
38 (Key8, 1),
39 (Key16, 2),
40 (Key32, 4),
41 (Key64, 8),
42 (Key128, 16),
43 (Key256, 32),
44 ]
45 }
46}
47
48enum HashKeySize {
50 Fixed(usize),
52 Variable,
54}
55
56pub trait HashKeyDispatcher: Sized {
62 type Output;
63
64 fn dispatch_impl<K: HashKey>(self) -> Self::Output;
65
66 fn data_types(&self) -> &[DataType];
68
69 fn dispatch(self) -> Self::Output {
74 if self.data_types().len() <= MAX_GROUP_KEYS_ON_STACK {
75 self.dispatch_by_key_size::<StackNullBitmap>()
76 } else {
77 self.dispatch_by_key_size::<HeapNullBitmap>()
78 }
79 }
80
81 fn dispatch_by_key_size<T: NullBitmap>(self) -> Self::Output {
85 match calc_hash_key_kind(self.data_types()) {
86 HashKeyKind::Key8 => self.dispatch_impl::<hash::Key8<T>>(),
87 HashKeyKind::Key16 => self.dispatch_impl::<hash::Key16<T>>(),
88 HashKeyKind::Key32 => self.dispatch_impl::<hash::Key32<T>>(),
89 HashKeyKind::Key64 => self.dispatch_impl::<hash::Key64<T>>(),
90 HashKeyKind::Key128 => self.dispatch_impl::<hash::Key128<T>>(),
91 HashKeyKind::Key256 => self.dispatch_impl::<hash::Key256<T>>(),
92 HashKeyKind::KeySerialized => self.dispatch_impl::<hash::KeySerialized<T>>(),
93 }
94 }
95}
96
97fn hash_key_size(data_type: &DataType) -> HashKeySize {
98 let exact_size = dispatch_data_types!(data_type, [S = ScalarRef], { S::exact_size() });
99
100 match exact_size {
101 Some(size) => HashKeySize::Fixed(size),
102 None => HashKeySize::Variable,
103 }
104}
105
106const MAX_FIXED_SIZE_KEY_ELEMENTS: usize = 8;
107
108pub fn calc_hash_key_kind(data_types: &[DataType]) -> HashKeyKind {
118 if data_types.len() > MAX_FIXED_SIZE_KEY_ELEMENTS {
119 return HashKeyKind::KeySerialized;
120 }
121
122 let mut total_data_size: usize = 0;
123 for data_type in data_types {
124 match hash_key_size(data_type) {
125 HashKeySize::Fixed(size) => {
126 total_data_size += size;
127 }
128 HashKeySize::Variable => {
129 return HashKeyKind::KeySerialized;
130 }
131 }
132 }
133
134 for (kind, max_size) in HashKeyKind::order_by_key_size() {
135 if total_data_size <= max_size {
136 return kind;
137 }
138 }
139
140 HashKeyKind::KeySerialized
141}
142
143#[cfg(test)]
144mod tests {
145
146 use super::{HashKeyKind, calc_hash_key_kind};
147 use crate::types::DataType;
148
149 fn all_data_types() -> Vec<DataType> {
150 vec![
151 DataType::Boolean, DataType::Int16, DataType::Int32, DataType::Int64, DataType::Float32, DataType::Float64, DataType::Decimal, DataType::Varchar, DataType::Timestamp, ]
161 }
162
163 fn compare_key_kinds(input_indices: &[usize], expected: HashKeyKind) {
164 let all_types = all_data_types();
165
166 let input_types = input_indices
167 .iter()
168 .map(|idx| all_types[*idx].clone())
169 .collect::<Vec<DataType>>();
170
171 let calculated_kind = calc_hash_key_kind(&input_types);
172 assert_eq!(expected, calculated_kind);
173 }
174
175 #[test]
176 fn test_calc_hash_key_kind() {
177 compare_key_kinds(&[0], HashKeyKind::Key8);
178 compare_key_kinds(&[1], HashKeyKind::Key16);
179 compare_key_kinds(&[2], HashKeyKind::Key32);
180 compare_key_kinds(&[3], HashKeyKind::Key64);
181 compare_key_kinds(&[8], HashKeyKind::Key128);
182 compare_key_kinds(&[3, 4], HashKeyKind::Key128);
183 compare_key_kinds(&[3, 4, 6], HashKeyKind::Key256);
184 compare_key_kinds(&[7], HashKeyKind::KeySerialized);
185 compare_key_kinds(&[1, 7], HashKeyKind::KeySerialized);
186 }
187}