arrow_array/builder/
mod.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18//! Defines push-based APIs for constructing arrays
19//!
20//! # Basic Usage
21//!
22//! Builders can be used to build simple, non-nested arrays
23//!
24//! ```
25//! # use arrow_array::builder::Int32Builder;
26//! # use arrow_array::PrimitiveArray;
27//! let mut a = Int32Builder::new();
28//! a.append_value(1);
29//! a.append_null();
30//! a.append_value(2);
31//! let a = a.finish();
32//!
33//! assert_eq!(a, PrimitiveArray::from(vec![Some(1), None, Some(2)]));
34//! ```
35//!
36//! ```
37//! # use arrow_array::builder::StringBuilder;
38//! # use arrow_array::{Array, StringArray};
39//! let mut a = StringBuilder::new();
40//! a.append_value("foo");
41//! a.append_value("bar");
42//! a.append_null();
43//! let a = a.finish();
44//!
45//! assert_eq!(a, StringArray::from_iter([Some("foo"), Some("bar"), None]));
46//! ```
47//!
48//! # Nested Usage
49//!
50//! Builders can also be used to build more complex nested arrays, such as lists
51//!
52//! ```
53//! # use arrow_array::builder::{Int32Builder, ListBuilder};
54//! # use arrow_array::ListArray;
55//! # use arrow_array::types::Int32Type;
56//! let mut a = ListBuilder::new(Int32Builder::new());
57//! // [1, 2]
58//! a.values().append_value(1);
59//! a.values().append_value(2);
60//! a.append(true);
61//! // null
62//! a.append(false);
63//! // []
64//! a.append(true);
65//! // [3, null]
66//! a.values().append_value(3);
67//! a.values().append_null();
68//! a.append(true);
69//!
70//! // [[1, 2], null, [], [3, null]]
71//! let a = a.finish();
72//!
73//! assert_eq!(a, ListArray::from_iter_primitive::<Int32Type, _, _>([
74//!     Some(vec![Some(1), Some(2)]),
75//!     None,
76//!     Some(vec![]),
77//!     Some(vec![Some(3), None])]
78//! ))
79//! ```
80//!
81//! # Using the [`Extend`] trait to append values from an iterable:
82//!
83//! ```
84//! # use arrow_array::{Array};
85//! # use arrow_array::builder::{ArrayBuilder, StringBuilder};
86//!
87//! let mut builder = StringBuilder::new();
88//! builder.extend(vec![Some("🍐"), Some("🍎"), None]);
89//! assert_eq!(builder.finish().len(), 3);
90//! ```
91//!
92//! # Using the [`Extend`] trait to write generic functions:
93//!
94//! ```
95//! # use arrow_array::{Array, ArrayRef, StringArray};
96//! # use arrow_array::builder::{ArrayBuilder, Int32Builder, ListBuilder, StringBuilder};
97//!
98//! // For generic methods that fill a list of values for an [`ArrayBuilder`], use the [`Extend`] trait.
99//! fn filter_and_fill<V, I: IntoIterator<Item = V>>(builder: &mut impl Extend<V>, values: I, filter: V)
100//! where V: PartialEq
101//! {
102//!     builder.extend(values.into_iter().filter(|v| *v == filter));
103//! }
104//! let mut string_builder = StringBuilder::new();
105//! filter_and_fill(
106//!     &mut string_builder,
107//!     vec![Some("🍐"), Some("🍎"), None],
108//!     Some("🍎"),
109//! );
110//! assert_eq!(string_builder.finish().len(), 1);
111//!
112//! let mut int_builder = Int32Builder::new();
113//! filter_and_fill(
114//!     &mut int_builder,
115//!     vec![Some(11), Some(42), None],
116//!     Some(42),
117//! );
118//! assert_eq!(int_builder.finish().len(), 1);
119//!
120//! // For generic methods that fill lists-of-lists for an [`ArrayBuilder`], use the [`Extend`] trait.
121//! fn filter_and_fill_if_contains<T, V, I: IntoIterator<Item = Option<V>>>(
122//!     list_builder: &mut impl Extend<Option<V>>,
123//!     values: I,
124//!     filter: Option<T>,
125//! ) where
126//!     T: PartialEq,
127//!     for<'a> &'a V: IntoIterator<Item = &'a Option<T>>,
128//! {
129//!     list_builder.extend(values.into_iter().filter(|string: &Option<V>| {
130//!         string
131//!             .as_ref()
132//!             .map(|str: &V| str.into_iter().any(|ch: &Option<T>| ch == &filter))
133//!             .unwrap_or(false)
134//!     }));
135//!  }
136//! let builder = StringBuilder::new();
137//! let mut list_builder = ListBuilder::new(builder);
138//! let pear_pear = vec![Some("🍐"),Some("🍐")];
139//! let pear_app = vec![Some("🍐"),Some("🍎")];
140//! filter_and_fill_if_contains(
141//!     &mut list_builder,
142//!     vec![Some(pear_pear), Some(pear_app), None],
143//!     Some("🍎"),
144//! );
145//! assert_eq!(list_builder.finish().len(), 1);
146//! ```
147//!
148//! # Custom Builders
149//!
150//! It is common to have a collection of statically defined Rust types that
151//! you want to convert to Arrow arrays.
152//!
153//! An example of doing so is below
154//!
155//! ```
156//! # use std::any::Any;
157//! # use arrow_array::builder::{ArrayBuilder, Int32Builder, ListBuilder, StringBuilder};
158//! # use arrow_array::{ArrayRef, RecordBatch, StructArray};
159//! # use arrow_schema::{DataType, Field};
160//! # use std::sync::Arc;
161//! /// A custom row representation
162//! struct MyRow {
163//!     i32: i32,
164//!     optional_i32: Option<i32>,
165//!     string: Option<String>,
166//!     i32_list: Option<Vec<Option<i32>>>,
167//! }
168//!
169//! /// Converts `Vec<Row>` into `StructArray`
170//! #[derive(Debug, Default)]
171//! struct MyRowBuilder {
172//!     i32: Int32Builder,
173//!     string: StringBuilder,
174//!     i32_list: ListBuilder<Int32Builder>,
175//! }
176//!
177//! impl MyRowBuilder {
178//!     fn append(&mut self, row: &MyRow) {
179//!         self.i32.append_value(row.i32);
180//!         self.string.append_option(row.string.as_ref());
181//!         self.i32_list.append_option(row.i32_list.as_ref().map(|x| x.iter().copied()));
182//!     }
183//!
184//!     /// Note: returns StructArray to allow nesting within another array if desired
185//!     fn finish(&mut self) -> StructArray {
186//!         let i32 = Arc::new(self.i32.finish()) as ArrayRef;
187//!         let i32_field = Arc::new(Field::new("i32", DataType::Int32, false));
188//!
189//!         let string = Arc::new(self.string.finish()) as ArrayRef;
190//!         let string_field = Arc::new(Field::new("i32", DataType::Utf8, false));
191//!
192//!         let i32_list = Arc::new(self.i32_list.finish()) as ArrayRef;
193//!         let value_field = Arc::new(Field::new_list_field(DataType::Int32, true));
194//!         let i32_list_field = Arc::new(Field::new("i32_list", DataType::List(value_field), true));
195//!
196//!         StructArray::from(vec![
197//!             (i32_field, i32),
198//!             (string_field, string),
199//!             (i32_list_field, i32_list),
200//!         ])
201//!     }
202//! }
203//!
204//! /// For building arrays in generic code, use Extend instead of the append_* methods
205//! /// e.g. append_value, append_option, append_null
206//! impl<'a> Extend<&'a MyRow> for MyRowBuilder {
207//!     fn extend<T: IntoIterator<Item = &'a MyRow>>(&mut self, iter: T) {
208//!         iter.into_iter().for_each(|row| self.append(row));
209//!     }
210//! }
211//!
212//! /// Converts a slice of [`MyRow`] to a [`RecordBatch`]
213//! fn rows_to_batch(rows: &[MyRow]) -> RecordBatch {
214//!     let mut builder = MyRowBuilder::default();
215//!     builder.extend(rows);
216//!     RecordBatch::from(&builder.finish())
217//! }
218//! ```
219//!
220//! # Null / Validity Masks
221//!
222//! The [`NullBufferBuilder`] is optimized for creating the null mask for an array.
223//!
224//! ```
225//! # use arrow_array::builder::NullBufferBuilder;
226//! let mut builder = NullBufferBuilder::new(8);
227//! let mut builder = NullBufferBuilder::new(8);
228//! builder.append_n_non_nulls(7);
229//! builder.append_null();
230//! let buffer = builder.finish().unwrap();
231//! assert_eq!(buffer.len(), 8);
232//! assert_eq!(buffer.iter().collect::<Vec<_>>(), vec![true, true, true, true, true, true, true, false]);
233//! ```
234
235pub use arrow_buffer::BooleanBufferBuilder;
236pub use arrow_buffer::NullBufferBuilder;
237
238mod boolean_builder;
239pub use boolean_builder::*;
240mod buffer_builder;
241pub use buffer_builder::*;
242mod fixed_size_binary_builder;
243pub use fixed_size_binary_builder::*;
244mod fixed_size_list_builder;
245pub use fixed_size_list_builder::*;
246mod fixed_size_binary_dictionary_builder;
247pub use fixed_size_binary_dictionary_builder::*;
248mod generic_bytes_builder;
249pub use generic_bytes_builder::*;
250mod generic_list_builder;
251pub use generic_list_builder::*;
252mod map_builder;
253pub use map_builder::*;
254mod null_builder;
255pub use null_builder::*;
256mod primitive_builder;
257pub use primitive_builder::*;
258mod primitive_dictionary_builder;
259pub use primitive_dictionary_builder::*;
260mod primitive_run_builder;
261pub use primitive_run_builder::*;
262mod struct_builder;
263pub use struct_builder::*;
264mod generic_bytes_dictionary_builder;
265pub use generic_bytes_dictionary_builder::*;
266mod generic_byte_run_builder;
267pub use generic_byte_run_builder::*;
268mod generic_bytes_view_builder;
269pub use generic_bytes_view_builder::*;
270mod generic_list_view_builder;
271pub use generic_list_view_builder::*;
272mod union_builder;
273
274pub use union_builder::*;
275
276use crate::types::{Int16Type, Int32Type, Int64Type, Int8Type};
277use crate::ArrayRef;
278use arrow_schema::{DataType, IntervalUnit, TimeUnit};
279use std::any::Any;
280
281/// Trait for dealing with different array builders at runtime
282///
283/// # Example
284///
285/// ```
286/// // Create
287/// # use arrow_array::{ArrayRef, StringArray};
288/// # use arrow_array::builder::{ArrayBuilder, Float64Builder, Int64Builder, StringBuilder};
289///
290/// let mut data_builders: Vec<Box<dyn ArrayBuilder>> = vec![
291///     Box::new(Float64Builder::new()),
292///     Box::new(Int64Builder::new()),
293///     Box::new(StringBuilder::new()),
294/// ];
295///
296/// // Fill
297/// data_builders[0]
298///     .as_any_mut()
299///     .downcast_mut::<Float64Builder>()
300///     .unwrap()
301///     .append_value(3.14);
302/// data_builders[1]
303///     .as_any_mut()
304///     .downcast_mut::<Int64Builder>()
305///     .unwrap()
306///     .append_value(-1);
307/// data_builders[2]
308///     .as_any_mut()
309///     .downcast_mut::<StringBuilder>()
310///     .unwrap()
311///     .append_value("🍎");
312///
313/// // Finish
314/// let array_refs: Vec<ArrayRef> = data_builders
315///     .iter_mut()
316///     .map(|builder| builder.finish())
317///     .collect();
318/// assert_eq!(array_refs[0].len(), 1);
319/// assert_eq!(array_refs[1].is_null(0), false);
320/// assert_eq!(
321///     array_refs[2]
322///         .as_any()
323///         .downcast_ref::<StringArray>()
324///         .unwrap()
325///         .value(0),
326///     "🍎"
327/// );
328/// ```
329pub trait ArrayBuilder: Any + Send + Sync {
330    /// Returns the number of array slots in the builder
331    fn len(&self) -> usize;
332
333    /// Returns whether number of array slots is zero
334    fn is_empty(&self) -> bool {
335        self.len() == 0
336    }
337
338    /// Builds the array
339    fn finish(&mut self) -> ArrayRef;
340
341    /// Builds the array without resetting the underlying builder.
342    fn finish_cloned(&self) -> ArrayRef;
343
344    /// Returns the builder as a non-mutable `Any` reference.
345    ///
346    /// This is most useful when one wants to call non-mutable APIs on a specific builder
347    /// type. In this case, one can first cast this into a `Any`, and then use
348    /// `downcast_ref` to get a reference on the specific builder.
349    fn as_any(&self) -> &dyn Any;
350
351    /// Returns the builder as a mutable `Any` reference.
352    ///
353    /// This is most useful when one wants to call mutable APIs on a specific builder
354    /// type. In this case, one can first cast this into a `Any`, and then use
355    /// `downcast_mut` to get a reference on the specific builder.
356    fn as_any_mut(&mut self) -> &mut dyn Any;
357
358    /// Returns the boxed builder as a box of `Any`.
359    fn into_box_any(self: Box<Self>) -> Box<dyn Any>;
360}
361
362impl ArrayBuilder for Box<dyn ArrayBuilder> {
363    fn len(&self) -> usize {
364        (**self).len()
365    }
366
367    fn is_empty(&self) -> bool {
368        (**self).is_empty()
369    }
370
371    fn finish(&mut self) -> ArrayRef {
372        (**self).finish()
373    }
374
375    fn finish_cloned(&self) -> ArrayRef {
376        (**self).finish_cloned()
377    }
378
379    fn as_any(&self) -> &dyn Any {
380        (**self).as_any()
381    }
382
383    fn as_any_mut(&mut self) -> &mut dyn Any {
384        (**self).as_any_mut()
385    }
386
387    fn into_box_any(self: Box<Self>) -> Box<dyn Any> {
388        self
389    }
390}
391
392/// Builder for [`ListArray`](crate::array::ListArray)
393pub type ListBuilder<T> = GenericListBuilder<i32, T>;
394
395/// Builder for [`LargeListArray`](crate::array::LargeListArray)
396pub type LargeListBuilder<T> = GenericListBuilder<i64, T>;
397
398/// Builder for [`ListViewArray`](crate::array::ListViewArray)
399pub type ListViewBuilder<T> = GenericListViewBuilder<i32, T>;
400
401/// Builder for [`LargeListViewArray`](crate::array::LargeListViewArray)
402pub type LargeListViewBuilder<T> = GenericListViewBuilder<i64, T>;
403
404/// Builder for [`BinaryArray`](crate::array::BinaryArray)
405///
406/// See examples on [`GenericBinaryBuilder`]
407pub type BinaryBuilder = GenericBinaryBuilder<i32>;
408
409/// Builder for [`LargeBinaryArray`](crate::array::LargeBinaryArray)
410///
411/// See examples on [`GenericBinaryBuilder`]
412pub type LargeBinaryBuilder = GenericBinaryBuilder<i64>;
413
414/// Builder for [`StringArray`](crate::array::StringArray)
415///
416/// See examples on [`GenericStringBuilder`]
417pub type StringBuilder = GenericStringBuilder<i32>;
418
419/// Builder for [`LargeStringArray`](crate::array::LargeStringArray)
420///
421/// See examples on [`GenericStringBuilder`]
422pub type LargeStringBuilder = GenericStringBuilder<i64>;
423
424/// Returns a builder with capacity for `capacity` elements of datatype
425/// `DataType`.
426///
427/// This function is useful to construct arrays from an arbitrary vectors with
428/// known/expected schema.
429///
430/// See comments on [StructBuilder] for retrieving collection builders built by
431/// make_builder.
432pub fn make_builder(datatype: &DataType, capacity: usize) -> Box<dyn ArrayBuilder> {
433    use crate::builder::*;
434    match datatype {
435        DataType::Null => Box::new(NullBuilder::new()),
436        DataType::Boolean => Box::new(BooleanBuilder::with_capacity(capacity)),
437        DataType::Int8 => Box::new(Int8Builder::with_capacity(capacity)),
438        DataType::Int16 => Box::new(Int16Builder::with_capacity(capacity)),
439        DataType::Int32 => Box::new(Int32Builder::with_capacity(capacity)),
440        DataType::Int64 => Box::new(Int64Builder::with_capacity(capacity)),
441        DataType::UInt8 => Box::new(UInt8Builder::with_capacity(capacity)),
442        DataType::UInt16 => Box::new(UInt16Builder::with_capacity(capacity)),
443        DataType::UInt32 => Box::new(UInt32Builder::with_capacity(capacity)),
444        DataType::UInt64 => Box::new(UInt64Builder::with_capacity(capacity)),
445        DataType::Float16 => Box::new(Float16Builder::with_capacity(capacity)),
446        DataType::Float32 => Box::new(Float32Builder::with_capacity(capacity)),
447        DataType::Float64 => Box::new(Float64Builder::with_capacity(capacity)),
448        DataType::Binary => Box::new(BinaryBuilder::with_capacity(capacity, 1024)),
449        DataType::LargeBinary => Box::new(LargeBinaryBuilder::with_capacity(capacity, 1024)),
450        DataType::FixedSizeBinary(len) => {
451            Box::new(FixedSizeBinaryBuilder::with_capacity(capacity, *len))
452        }
453        DataType::Decimal128(p, s) => Box::new(
454            Decimal128Builder::with_capacity(capacity).with_data_type(DataType::Decimal128(*p, *s)),
455        ),
456        DataType::Decimal256(p, s) => Box::new(
457            Decimal256Builder::with_capacity(capacity).with_data_type(DataType::Decimal256(*p, *s)),
458        ),
459        DataType::Utf8 => Box::new(StringBuilder::with_capacity(capacity, 1024)),
460        DataType::LargeUtf8 => Box::new(LargeStringBuilder::with_capacity(capacity, 1024)),
461        DataType::Date32 => Box::new(Date32Builder::with_capacity(capacity)),
462        DataType::Date64 => Box::new(Date64Builder::with_capacity(capacity)),
463        DataType::Time32(TimeUnit::Second) => {
464            Box::new(Time32SecondBuilder::with_capacity(capacity))
465        }
466        DataType::Time32(TimeUnit::Millisecond) => {
467            Box::new(Time32MillisecondBuilder::with_capacity(capacity))
468        }
469        DataType::Time64(TimeUnit::Microsecond) => {
470            Box::new(Time64MicrosecondBuilder::with_capacity(capacity))
471        }
472        DataType::Time64(TimeUnit::Nanosecond) => {
473            Box::new(Time64NanosecondBuilder::with_capacity(capacity))
474        }
475        DataType::Timestamp(TimeUnit::Second, tz) => Box::new(
476            TimestampSecondBuilder::with_capacity(capacity)
477                .with_data_type(DataType::Timestamp(TimeUnit::Second, tz.clone())),
478        ),
479        DataType::Timestamp(TimeUnit::Millisecond, tz) => Box::new(
480            TimestampMillisecondBuilder::with_capacity(capacity)
481                .with_data_type(DataType::Timestamp(TimeUnit::Millisecond, tz.clone())),
482        ),
483        DataType::Timestamp(TimeUnit::Microsecond, tz) => Box::new(
484            TimestampMicrosecondBuilder::with_capacity(capacity)
485                .with_data_type(DataType::Timestamp(TimeUnit::Microsecond, tz.clone())),
486        ),
487        DataType::Timestamp(TimeUnit::Nanosecond, tz) => Box::new(
488            TimestampNanosecondBuilder::with_capacity(capacity)
489                .with_data_type(DataType::Timestamp(TimeUnit::Nanosecond, tz.clone())),
490        ),
491        DataType::Interval(IntervalUnit::YearMonth) => {
492            Box::new(IntervalYearMonthBuilder::with_capacity(capacity))
493        }
494        DataType::Interval(IntervalUnit::DayTime) => {
495            Box::new(IntervalDayTimeBuilder::with_capacity(capacity))
496        }
497        DataType::Interval(IntervalUnit::MonthDayNano) => {
498            Box::new(IntervalMonthDayNanoBuilder::with_capacity(capacity))
499        }
500        DataType::Duration(TimeUnit::Second) => {
501            Box::new(DurationSecondBuilder::with_capacity(capacity))
502        }
503        DataType::Duration(TimeUnit::Millisecond) => {
504            Box::new(DurationMillisecondBuilder::with_capacity(capacity))
505        }
506        DataType::Duration(TimeUnit::Microsecond) => {
507            Box::new(DurationMicrosecondBuilder::with_capacity(capacity))
508        }
509        DataType::Duration(TimeUnit::Nanosecond) => {
510            Box::new(DurationNanosecondBuilder::with_capacity(capacity))
511        }
512        DataType::List(field) => {
513            let builder = make_builder(field.data_type(), capacity);
514            Box::new(ListBuilder::with_capacity(builder, capacity).with_field(field.clone()))
515        }
516        DataType::LargeList(field) => {
517            let builder = make_builder(field.data_type(), capacity);
518            Box::new(LargeListBuilder::with_capacity(builder, capacity).with_field(field.clone()))
519        }
520        DataType::FixedSizeList(field, size) => {
521            let size = *size;
522            let values_builder_capacity = {
523                let size: usize = size.try_into().unwrap();
524                capacity * size
525            };
526            let builder = make_builder(field.data_type(), values_builder_capacity);
527            Box::new(
528                FixedSizeListBuilder::with_capacity(builder, size, capacity)
529                    .with_field(field.clone()),
530            )
531        }
532        DataType::ListView(field) => {
533            let builder = make_builder(field.data_type(), capacity);
534            Box::new(ListViewBuilder::with_capacity(builder, capacity).with_field(field.clone()))
535        }
536        DataType::LargeListView(field) => {
537            let builder = make_builder(field.data_type(), capacity);
538            Box::new(
539                LargeListViewBuilder::with_capacity(builder, capacity).with_field(field.clone()),
540            )
541        }
542        DataType::Map(field, _) => match field.data_type() {
543            DataType::Struct(fields) => {
544                let map_field_names = MapFieldNames {
545                    key: fields[0].name().clone(),
546                    value: fields[1].name().clone(),
547                    entry: field.name().clone(),
548                };
549                let key_builder = make_builder(fields[0].data_type(), capacity);
550                let value_builder = make_builder(fields[1].data_type(), capacity);
551                Box::new(
552                    MapBuilder::with_capacity(
553                        Some(map_field_names),
554                        key_builder,
555                        value_builder,
556                        capacity,
557                    )
558                    .with_keys_field(fields[0].clone())
559                    .with_values_field(fields[1].clone()),
560                )
561            }
562            t => panic!("The field of Map data type {t:?} should have a child Struct field"),
563        },
564        DataType::Struct(fields) => Box::new(StructBuilder::from_fields(fields.clone(), capacity)),
565        t @ DataType::Dictionary(key_type, value_type) => {
566            macro_rules! dict_builder {
567                ($key_type:ty) => {
568                    match &**value_type {
569                        DataType::Utf8 => {
570                            let dict_builder: StringDictionaryBuilder<$key_type> =
571                                StringDictionaryBuilder::with_capacity(capacity, 256, 1024);
572                            Box::new(dict_builder)
573                        }
574                        DataType::LargeUtf8 => {
575                            let dict_builder: LargeStringDictionaryBuilder<$key_type> =
576                                LargeStringDictionaryBuilder::with_capacity(capacity, 256, 1024);
577                            Box::new(dict_builder)
578                        }
579                        DataType::Binary => {
580                            let dict_builder: BinaryDictionaryBuilder<$key_type> =
581                                BinaryDictionaryBuilder::with_capacity(capacity, 256, 1024);
582                            Box::new(dict_builder)
583                        }
584                        DataType::LargeBinary => {
585                            let dict_builder: LargeBinaryDictionaryBuilder<$key_type> =
586                                LargeBinaryDictionaryBuilder::with_capacity(capacity, 256, 1024);
587                            Box::new(dict_builder)
588                        }
589                        t => panic!("Dictionary value type {t:?} is not currently supported"),
590                    }
591                };
592            }
593            match &**key_type {
594                DataType::Int8 => dict_builder!(Int8Type),
595                DataType::Int16 => dict_builder!(Int16Type),
596                DataType::Int32 => dict_builder!(Int32Type),
597                DataType::Int64 => dict_builder!(Int64Type),
598                _ => {
599                    panic!("Data type {t:?} with key type {key_type:?} is not currently supported")
600                }
601            }
602        }
603        t => panic!("Data type {t:?} is not currently supported"),
604    }
605}