arrow_array/builder/
generic_bytes_dictionary_builder.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
18use crate::builder::{ArrayBuilder, GenericByteBuilder, PrimitiveBuilder};
19use crate::types::{ArrowDictionaryKeyType, ByteArrayType, GenericBinaryType, GenericStringType};
20use crate::{
21    Array, ArrayRef, DictionaryArray, GenericByteArray, PrimitiveArray, TypedDictionaryArray,
22};
23use arrow_buffer::ArrowNativeType;
24use arrow_schema::{ArrowError, DataType};
25use hashbrown::HashTable;
26use num::NumCast;
27use std::any::Any;
28use std::sync::Arc;
29
30/// Builder for [`DictionaryArray`] of [`GenericByteArray`]
31///
32/// For example to map a set of byte indices to String values. Note that
33/// the use of a `HashMap` here will not scale to very large arrays or
34/// result in an ordered dictionary.
35#[derive(Debug)]
36pub struct GenericByteDictionaryBuilder<K, T>
37where
38    K: ArrowDictionaryKeyType,
39    T: ByteArrayType,
40{
41    state: ahash::RandomState,
42    dedup: HashTable<usize>,
43
44    keys_builder: PrimitiveBuilder<K>,
45    values_builder: GenericByteBuilder<T>,
46}
47
48impl<K, T> Default for GenericByteDictionaryBuilder<K, T>
49where
50    K: ArrowDictionaryKeyType,
51    T: ByteArrayType,
52{
53    fn default() -> Self {
54        Self::new()
55    }
56}
57
58impl<K, T> GenericByteDictionaryBuilder<K, T>
59where
60    K: ArrowDictionaryKeyType,
61    T: ByteArrayType,
62{
63    /// Creates a new `GenericByteDictionaryBuilder`
64    pub fn new() -> Self {
65        let keys_builder = PrimitiveBuilder::new();
66        let values_builder = GenericByteBuilder::<T>::new();
67        Self {
68            state: Default::default(),
69            dedup: HashTable::with_capacity(keys_builder.capacity()),
70            keys_builder,
71            values_builder,
72        }
73    }
74
75    /// Creates a new `GenericByteDictionaryBuilder` with the provided capacities
76    ///
77    /// `keys_capacity`: the number of keys, i.e. length of array to build
78    /// `value_capacity`: the number of distinct dictionary values, i.e. size of dictionary
79    /// `data_capacity`: the total number of bytes of all distinct bytes in the dictionary
80    pub fn with_capacity(
81        keys_capacity: usize,
82        value_capacity: usize,
83        data_capacity: usize,
84    ) -> Self {
85        Self {
86            state: Default::default(),
87            dedup: Default::default(),
88            keys_builder: PrimitiveBuilder::with_capacity(keys_capacity),
89            values_builder: GenericByteBuilder::<T>::with_capacity(value_capacity, data_capacity),
90        }
91    }
92
93    /// Creates a new `GenericByteDictionaryBuilder` from a keys capacity and a dictionary
94    /// which is initialized with the given values.
95    /// The indices of those dictionary values are used as keys.
96    ///
97    /// # Example
98    ///
99    /// ```
100    /// # use arrow_array::builder::StringDictionaryBuilder;
101    /// # use arrow_array::{Int16Array, StringArray};
102    ///
103    /// let dictionary_values = StringArray::from(vec![None, Some("abc"), Some("def")]);
104    ///
105    /// let mut builder = StringDictionaryBuilder::new_with_dictionary(3, &dictionary_values).unwrap();
106    /// builder.append("def").unwrap();
107    /// builder.append_null();
108    /// builder.append("abc").unwrap();
109    ///
110    /// let dictionary_array = builder.finish();
111    ///
112    /// let keys = dictionary_array.keys();
113    ///
114    /// assert_eq!(keys, &Int16Array::from(vec![Some(2), None, Some(1)]));
115    /// ```
116    pub fn new_with_dictionary(
117        keys_capacity: usize,
118        dictionary_values: &GenericByteArray<T>,
119    ) -> Result<Self, ArrowError> {
120        let state = ahash::RandomState::default();
121        let dict_len = dictionary_values.len();
122
123        let mut dedup = HashTable::with_capacity(dict_len);
124
125        let values_len = dictionary_values.value_data().len();
126        let mut values_builder = GenericByteBuilder::<T>::with_capacity(dict_len, values_len);
127
128        K::Native::from_usize(dictionary_values.len())
129            .ok_or(ArrowError::DictionaryKeyOverflowError)?;
130
131        for (idx, maybe_value) in dictionary_values.iter().enumerate() {
132            match maybe_value {
133                Some(value) => {
134                    let value_bytes: &[u8] = value.as_ref();
135                    let hash = state.hash_one(value_bytes);
136
137                    dedup
138                        .entry(
139                            hash,
140                            |idx: &usize| value_bytes == get_bytes(&values_builder, *idx),
141                            |idx: &usize| state.hash_one(get_bytes(&values_builder, *idx)),
142                        )
143                        .or_insert(idx);
144
145                    values_builder.append_value(value);
146                }
147                None => values_builder.append_null(),
148            }
149        }
150
151        Ok(Self {
152            state,
153            dedup,
154            keys_builder: PrimitiveBuilder::with_capacity(keys_capacity),
155            values_builder,
156        })
157    }
158
159    /// Creates a new `GenericByteDictionaryBuilder` from the existing builder with the same
160    /// keys and values, but with a new data type for the keys.
161    ///
162    /// # Example
163    /// ```
164    /// #
165    /// # use arrow_array::builder::StringDictionaryBuilder;
166    /// # use arrow_array::types::{UInt8Type, UInt16Type};
167    /// # use arrow_array::UInt16Array;
168    /// # use arrow_schema::ArrowError;
169    ///
170    /// let mut u8_keyed_builder = StringDictionaryBuilder::<UInt8Type>::new();
171    ///
172    /// // appending too many values causes the dictionary to overflow
173    /// for i in 0..256 {
174    ///     u8_keyed_builder.append_value(format!("{}", i));
175    /// }
176    /// let result = u8_keyed_builder.append("256");
177    /// assert!(matches!(result, Err(ArrowError::DictionaryKeyOverflowError{})));
178    ///
179    /// // we need to upgrade to a larger key type
180    /// let mut u16_keyed_builder = StringDictionaryBuilder::<UInt16Type>::try_new_from_builder(u8_keyed_builder).unwrap();
181    /// let dictionary_array = u16_keyed_builder.finish();
182    /// let keys = dictionary_array.keys();
183    ///
184    /// assert_eq!(keys, &UInt16Array::from_iter(0..256));
185    /// ```
186    pub fn try_new_from_builder<K2>(
187        mut source: GenericByteDictionaryBuilder<K2, T>,
188    ) -> Result<Self, ArrowError>
189    where
190        K::Native: NumCast,
191        K2: ArrowDictionaryKeyType,
192        K2::Native: NumCast,
193    {
194        let state = source.state;
195        let dedup = source.dedup;
196        let values_builder = source.values_builder;
197
198        let source_keys = source.keys_builder.finish();
199        let new_keys: PrimitiveArray<K> = source_keys.try_unary(|value| {
200            num::cast::cast::<K2::Native, K::Native>(value).ok_or_else(|| {
201                ArrowError::CastError(format!(
202                    "Can't cast dictionary keys from source type {:?} to type {:?}",
203                    K2::DATA_TYPE,
204                    K::DATA_TYPE
205                ))
206            })
207        })?;
208
209        // drop source key here because currently source_keys and new_keys are holding reference to
210        // the same underlying null_buffer. Below we want to call new_keys.into_builder() it must
211        // be the only reference holder.
212        drop(source_keys);
213
214        Ok(Self {
215            state,
216            dedup,
217            keys_builder: new_keys
218                .into_builder()
219                .expect("underlying buffer has no references"),
220            values_builder,
221        })
222    }
223}
224
225impl<K, T> ArrayBuilder for GenericByteDictionaryBuilder<K, T>
226where
227    K: ArrowDictionaryKeyType,
228    T: ByteArrayType,
229{
230    /// Returns the builder as an non-mutable `Any` reference.
231    fn as_any(&self) -> &dyn Any {
232        self
233    }
234
235    /// Returns the builder as an mutable `Any` reference.
236    fn as_any_mut(&mut self) -> &mut dyn Any {
237        self
238    }
239
240    /// Returns the boxed builder as a box of `Any`.
241    fn into_box_any(self: Box<Self>) -> Box<dyn Any> {
242        self
243    }
244
245    /// Returns the number of array slots in the builder
246    fn len(&self) -> usize {
247        self.keys_builder.len()
248    }
249
250    /// Builds the array and reset this builder.
251    fn finish(&mut self) -> ArrayRef {
252        Arc::new(self.finish())
253    }
254
255    /// Builds the array without resetting the builder.
256    fn finish_cloned(&self) -> ArrayRef {
257        Arc::new(self.finish_cloned())
258    }
259}
260
261impl<K, T> GenericByteDictionaryBuilder<K, T>
262where
263    K: ArrowDictionaryKeyType,
264    T: ByteArrayType,
265{
266    fn get_or_insert_key(&mut self, value: impl AsRef<T::Native>) -> Result<K::Native, ArrowError> {
267        let value_native: &T::Native = value.as_ref();
268        let value_bytes: &[u8] = value_native.as_ref();
269
270        let state = &self.state;
271        let storage = &mut self.values_builder;
272        let hash = state.hash_one(value_bytes);
273
274        let idx = *self
275            .dedup
276            .entry(
277                hash,
278                |idx| value_bytes == get_bytes(storage, *idx),
279                |idx| state.hash_one(get_bytes(storage, *idx)),
280            )
281            .or_insert_with(|| {
282                let idx = storage.len();
283                storage.append_value(value);
284                idx
285            })
286            .get();
287
288        let key = K::Native::from_usize(idx).ok_or(ArrowError::DictionaryKeyOverflowError)?;
289
290        Ok(key)
291    }
292
293    /// Append a value to the array. Return an existing index
294    /// if already present in the values array or a new index if the
295    /// value is appended to the values array.
296    ///
297    /// Returns an error if the new index would overflow the key type.
298    pub fn append(&mut self, value: impl AsRef<T::Native>) -> Result<K::Native, ArrowError> {
299        let key = self.get_or_insert_key(value)?;
300        self.keys_builder.append_value(key);
301        Ok(key)
302    }
303
304    /// Append a value multiple times to the array.
305    /// This is the same as `append` but allows to append the same value multiple times without doing multiple lookups.
306    ///
307    /// Returns an error if the new index would overflow the key type.
308    pub fn append_n(
309        &mut self,
310        value: impl AsRef<T::Native>,
311        count: usize,
312    ) -> Result<K::Native, ArrowError> {
313        let key = self.get_or_insert_key(value)?;
314        self.keys_builder.append_value_n(key, count);
315        Ok(key)
316    }
317
318    /// Infallibly append a value to this builder
319    ///
320    /// # Panics
321    ///
322    /// Panics if the resulting length of the dictionary values array would exceed `T::Native::MAX`
323    pub fn append_value(&mut self, value: impl AsRef<T::Native>) {
324        self.append(value).expect("dictionary key overflow");
325    }
326
327    /// Infallibly append a value to this builder repeatedly `count` times.
328    /// This is the same as `append_value` but allows to append the same value multiple times without doing multiple lookups.
329    ///
330    /// # Panics
331    ///
332    /// Panics if the resulting length of the dictionary values array would exceed `T::Native::MAX`
333    pub fn append_values(&mut self, value: impl AsRef<T::Native>, count: usize) {
334        self.append_n(value, count)
335            .expect("dictionary key overflow");
336    }
337
338    /// Appends a null slot into the builder
339    #[inline]
340    pub fn append_null(&mut self) {
341        self.keys_builder.append_null()
342    }
343
344    /// Infallibly append `n` null slots into the builder
345    #[inline]
346    pub fn append_nulls(&mut self, n: usize) {
347        self.keys_builder.append_nulls(n)
348    }
349
350    /// Append an `Option` value into the builder
351    ///
352    /// # Panics
353    ///
354    /// Panics if the resulting length of the dictionary values array would exceed `T::Native::MAX`
355    #[inline]
356    pub fn append_option(&mut self, value: Option<impl AsRef<T::Native>>) {
357        match value {
358            None => self.append_null(),
359            Some(v) => self.append_value(v),
360        };
361    }
362
363    /// Append an `Option` value into the builder repeatedly `count` times.
364    /// This is the same as `append_option` but allows to append the same value multiple times without doing multiple lookups.
365    ///
366    /// # Panics
367    ///
368    /// Panics if the resulting length of the dictionary values array would exceed `T::Native::MAX`
369    pub fn append_options(&mut self, value: Option<impl AsRef<T::Native>>, count: usize) {
370        match value {
371            None => self.keys_builder.append_nulls(count),
372            Some(v) => self.append_values(v, count),
373        };
374    }
375
376    /// Extends builder with an existing dictionary array.
377    ///
378    /// This is the same as [`Self::extend`] but is faster as it translates
379    /// the dictionary values once rather than doing a lookup for each item in the iterator
380    ///
381    /// when dictionary values are null (the actual mapped values) the keys are null
382    ///
383    pub fn extend_dictionary(
384        &mut self,
385        dictionary: &TypedDictionaryArray<K, GenericByteArray<T>>,
386    ) -> Result<(), ArrowError> {
387        let values = dictionary.values();
388
389        let v_len = values.len();
390        let k_len = dictionary.keys().len();
391        if v_len == 0 && k_len == 0 {
392            return Ok(());
393        }
394
395        // All nulls
396        if v_len == 0 {
397            self.append_nulls(k_len);
398            return Ok(());
399        }
400
401        if k_len == 0 {
402            return Err(ArrowError::InvalidArgumentError(
403                "Dictionary keys should not be empty when values are not empty".to_string(),
404            ));
405        }
406
407        // Orphan values will be carried over to the new dictionary
408        let mapped_values = values
409            .iter()
410            // Dictionary values can technically be null, so we need to handle that
411            .map(|dict_value| {
412                dict_value
413                    .map(|dict_value| self.get_or_insert_key(dict_value))
414                    .transpose()
415            })
416            .collect::<Result<Vec<_>, _>>()?;
417
418        // Just insert the keys without additional lookups
419        dictionary.keys().iter().for_each(|key| match key {
420            None => self.append_null(),
421            Some(original_dict_index) => {
422                let index = original_dict_index.as_usize().min(v_len - 1);
423                match mapped_values[index] {
424                    None => self.append_null(),
425                    Some(mapped_value) => self.keys_builder.append_value(mapped_value),
426                }
427            }
428        });
429
430        Ok(())
431    }
432
433    /// Builds the `DictionaryArray` and reset this builder.
434    pub fn finish(&mut self) -> DictionaryArray<K> {
435        self.dedup.clear();
436        let values = self.values_builder.finish();
437        let keys = self.keys_builder.finish();
438
439        let data_type = DataType::Dictionary(Box::new(K::DATA_TYPE), Box::new(T::DATA_TYPE));
440
441        let builder = keys
442            .into_data()
443            .into_builder()
444            .data_type(data_type)
445            .child_data(vec![values.into_data()]);
446
447        DictionaryArray::from(unsafe { builder.build_unchecked() })
448    }
449
450    /// Builds the `DictionaryArray` without resetting the builder.
451    pub fn finish_cloned(&self) -> DictionaryArray<K> {
452        let values = self.values_builder.finish_cloned();
453        let keys = self.keys_builder.finish_cloned();
454
455        let data_type = DataType::Dictionary(Box::new(K::DATA_TYPE), Box::new(T::DATA_TYPE));
456
457        let builder = keys
458            .into_data()
459            .into_builder()
460            .data_type(data_type)
461            .child_data(vec![values.into_data()]);
462
463        DictionaryArray::from(unsafe { builder.build_unchecked() })
464    }
465
466    /// Builds the `DictionaryArray` without resetting the values builder or
467    /// the internal de-duplication map.
468    ///
469    /// The advantage of doing this is that the values will represent the entire
470    /// set of what has been built so-far by this builder and ensures
471    /// consistency in the assignment of keys to values across multiple calls
472    /// to `finish_preserve_values`. This enables ipc writers to efficiently
473    /// emit delta dictionaries.
474    ///
475    /// The downside to this is that building the record requires creating a
476    /// copy of the values, which can become slowly more expensive if the
477    /// dictionary grows.
478    ///
479    /// Additionally, if record batches from multiple different dictionary
480    /// builders for the same column are fed into a single ipc writer, beware
481    /// that entire dictionaries are likely to be re-sent frequently even when
482    /// the majority of the values are not used by the current record batch.
483    pub fn finish_preserve_values(&mut self) -> DictionaryArray<K> {
484        let values = self.values_builder.finish_cloned();
485        let keys = self.keys_builder.finish();
486
487        let data_type = DataType::Dictionary(Box::new(K::DATA_TYPE), Box::new(T::DATA_TYPE));
488
489        let builder = keys
490            .into_data()
491            .into_builder()
492            .data_type(data_type)
493            .child_data(vec![values.into_data()]);
494
495        DictionaryArray::from(unsafe { builder.build_unchecked() })
496    }
497
498    /// Returns the current null buffer as a slice
499    pub fn validity_slice(&self) -> Option<&[u8]> {
500        self.keys_builder.validity_slice()
501    }
502}
503
504impl<K: ArrowDictionaryKeyType, T: ByteArrayType, V: AsRef<T::Native>> Extend<Option<V>>
505    for GenericByteDictionaryBuilder<K, T>
506{
507    #[inline]
508    fn extend<I: IntoIterator<Item = Option<V>>>(&mut self, iter: I) {
509        for v in iter {
510            self.append_option(v)
511        }
512    }
513}
514
515fn get_bytes<T: ByteArrayType>(values: &GenericByteBuilder<T>, idx: usize) -> &[u8] {
516    let offsets = values.offsets_slice();
517    let values = values.values_slice();
518
519    let end_offset = offsets[idx + 1].as_usize();
520    let start_offset = offsets[idx].as_usize();
521
522    &values[start_offset..end_offset]
523}
524
525/// Builder for [`DictionaryArray`] of [`StringArray`](crate::array::StringArray)
526///
527/// ```
528/// // Create a dictionary array indexed by bytes whose values are Strings.
529/// // It can thus hold up to 256 distinct string values.
530///
531/// # use arrow_array::builder::StringDictionaryBuilder;
532/// # use arrow_array::{Int8Array, StringArray};
533/// # use arrow_array::types::Int8Type;
534///
535/// let mut builder = StringDictionaryBuilder::<Int8Type>::new();
536///
537/// // The builder builds the dictionary value by value
538/// builder.append("abc").unwrap();
539/// builder.append_null();
540/// builder.append_n("def", 2).unwrap();  // appends "def" twice with a single lookup
541/// builder.append("abc").unwrap();
542/// let array = builder.finish();
543///
544/// assert_eq!(
545///   array.keys(),
546///   &Int8Array::from(vec![Some(0), None, Some(1), Some(1), Some(0)])
547/// );
548///
549/// // Values are polymorphic and so require a downcast.
550/// let av = array.values();
551/// let ava: &StringArray = av.as_any().downcast_ref::<StringArray>().unwrap();
552///
553/// assert_eq!(ava.value(0), "abc");
554/// assert_eq!(ava.value(1), "def");
555///
556/// ```
557pub type StringDictionaryBuilder<K> = GenericByteDictionaryBuilder<K, GenericStringType<i32>>;
558
559/// Builder for [`DictionaryArray`] of [`LargeStringArray`](crate::array::LargeStringArray)
560pub type LargeStringDictionaryBuilder<K> = GenericByteDictionaryBuilder<K, GenericStringType<i64>>;
561
562/// Builder for [`DictionaryArray`] of [`BinaryArray`](crate::array::BinaryArray)
563///
564/// ```
565/// // Create a dictionary array indexed by bytes whose values are binary.
566/// // It can thus hold up to 256 distinct binary values.
567///
568/// # use arrow_array::builder::BinaryDictionaryBuilder;
569/// # use arrow_array::{BinaryArray, Int8Array};
570/// # use arrow_array::types::Int8Type;
571///
572/// let mut builder = BinaryDictionaryBuilder::<Int8Type>::new();
573///
574/// // The builder builds the dictionary value by value
575/// builder.append(b"abc").unwrap();
576/// builder.append_null();
577/// builder.append(b"def").unwrap();
578/// builder.append(b"def").unwrap();
579/// builder.append(b"abc").unwrap();
580/// let array = builder.finish();
581///
582/// assert_eq!(
583///   array.keys(),
584///   &Int8Array::from(vec![Some(0), None, Some(1), Some(1), Some(0)])
585/// );
586///
587/// // Values are polymorphic and so require a downcast.
588/// let av = array.values();
589/// let ava: &BinaryArray = av.as_any().downcast_ref::<BinaryArray>().unwrap();
590///
591/// assert_eq!(ava.value(0), b"abc");
592/// assert_eq!(ava.value(1), b"def");
593///
594/// ```
595pub type BinaryDictionaryBuilder<K> = GenericByteDictionaryBuilder<K, GenericBinaryType<i32>>;
596
597/// Builder for [`DictionaryArray`] of [`LargeBinaryArray`](crate::array::LargeBinaryArray)
598pub type LargeBinaryDictionaryBuilder<K> = GenericByteDictionaryBuilder<K, GenericBinaryType<i64>>;
599
600#[cfg(test)]
601mod tests {
602    use super::*;
603
604    use crate::array::Int8Array;
605    use crate::cast::AsArray;
606    use crate::types::{Int16Type, Int32Type, Int8Type, UInt16Type, UInt8Type, Utf8Type};
607    use crate::{ArrowPrimitiveType, BinaryArray, StringArray};
608
609    fn test_bytes_dictionary_builder<T>(values: Vec<&T::Native>)
610    where
611        T: ByteArrayType,
612        <T as ByteArrayType>::Native: PartialEq,
613        <T as ByteArrayType>::Native: AsRef<<T as ByteArrayType>::Native>,
614    {
615        let mut builder = GenericByteDictionaryBuilder::<Int8Type, T>::new();
616        builder.append(values[0]).unwrap();
617        builder.append_null();
618        builder.append(values[1]).unwrap();
619        builder.append(values[1]).unwrap();
620        builder.append(values[0]).unwrap();
621        let array = builder.finish();
622
623        assert_eq!(
624            array.keys(),
625            &Int8Array::from(vec![Some(0), None, Some(1), Some(1), Some(0)])
626        );
627
628        // Values are polymorphic and so require a downcast.
629        let av = array.values();
630        let ava: &GenericByteArray<T> = av.as_any().downcast_ref::<GenericByteArray<T>>().unwrap();
631
632        assert_eq!(*ava.value(0), *values[0]);
633        assert_eq!(*ava.value(1), *values[1]);
634    }
635
636    #[test]
637    fn test_string_dictionary_builder() {
638        test_bytes_dictionary_builder::<GenericStringType<i32>>(vec!["abc", "def"]);
639    }
640
641    #[test]
642    fn test_binary_dictionary_builder() {
643        test_bytes_dictionary_builder::<GenericBinaryType<i32>>(vec![b"abc", b"def"]);
644    }
645
646    fn test_bytes_dictionary_builder_finish_cloned<T>(values: Vec<&T::Native>)
647    where
648        T: ByteArrayType,
649        <T as ByteArrayType>::Native: PartialEq,
650        <T as ByteArrayType>::Native: AsRef<<T as ByteArrayType>::Native>,
651    {
652        let mut builder = GenericByteDictionaryBuilder::<Int8Type, T>::new();
653
654        builder.append(values[0]).unwrap();
655        builder.append_null();
656        builder.append(values[1]).unwrap();
657        builder.append(values[1]).unwrap();
658        builder.append(values[0]).unwrap();
659        let mut array = builder.finish_cloned();
660
661        assert_eq!(
662            array.keys(),
663            &Int8Array::from(vec![Some(0), None, Some(1), Some(1), Some(0)])
664        );
665
666        // Values are polymorphic and so require a downcast.
667        let av = array.values();
668        let ava: &GenericByteArray<T> = av.as_any().downcast_ref::<GenericByteArray<T>>().unwrap();
669
670        assert_eq!(ava.value(0), values[0]);
671        assert_eq!(ava.value(1), values[1]);
672
673        builder.append(values[0]).unwrap();
674        builder.append(values[2]).unwrap();
675        builder.append(values[1]).unwrap();
676
677        array = builder.finish();
678
679        assert_eq!(
680            array.keys(),
681            &Int8Array::from(vec![
682                Some(0),
683                None,
684                Some(1),
685                Some(1),
686                Some(0),
687                Some(0),
688                Some(2),
689                Some(1)
690            ])
691        );
692
693        // Values are polymorphic and so require a downcast.
694        let av2 = array.values();
695        let ava2: &GenericByteArray<T> =
696            av2.as_any().downcast_ref::<GenericByteArray<T>>().unwrap();
697
698        assert_eq!(ava2.value(0), values[0]);
699        assert_eq!(ava2.value(1), values[1]);
700        assert_eq!(ava2.value(2), values[2]);
701    }
702
703    #[test]
704    fn test_string_dictionary_builder_finish_cloned() {
705        test_bytes_dictionary_builder_finish_cloned::<GenericStringType<i32>>(vec![
706            "abc", "def", "ghi",
707        ]);
708    }
709
710    #[test]
711    fn test_binary_dictionary_builder_finish_cloned() {
712        test_bytes_dictionary_builder_finish_cloned::<GenericBinaryType<i32>>(vec![
713            b"abc", b"def", b"ghi",
714        ]);
715    }
716
717    fn _test_try_new_from_builder_generic_for_key_types<K1, K2, T>(values: Vec<&T::Native>)
718    where
719        K1: ArrowDictionaryKeyType,
720        K1::Native: NumCast,
721        K2: ArrowDictionaryKeyType,
722        K2::Native: NumCast + From<u8>,
723        T: ByteArrayType,
724        <T as ByteArrayType>::Native: PartialEq + AsRef<<T as ByteArrayType>::Native>,
725    {
726        let mut source = GenericByteDictionaryBuilder::<K1, T>::new();
727        source.append(values[0]).unwrap();
728        source.append(values[1]).unwrap();
729        source.append_null();
730        source.append(values[2]).unwrap();
731
732        let mut result =
733            GenericByteDictionaryBuilder::<K2, T>::try_new_from_builder(source).unwrap();
734        let array = result.finish();
735
736        let mut expected_keys_builder = PrimitiveBuilder::<K2>::new();
737        expected_keys_builder
738            .append_value(<<K2 as ArrowPrimitiveType>::Native as From<u8>>::from(0u8));
739        expected_keys_builder
740            .append_value(<<K2 as ArrowPrimitiveType>::Native as From<u8>>::from(1u8));
741        expected_keys_builder.append_null();
742        expected_keys_builder
743            .append_value(<<K2 as ArrowPrimitiveType>::Native as From<u8>>::from(2u8));
744        let expected_keys = expected_keys_builder.finish();
745        assert_eq!(array.keys(), &expected_keys);
746
747        let av = array.values();
748        let ava: &GenericByteArray<T> = av.as_any().downcast_ref::<GenericByteArray<T>>().unwrap();
749        assert_eq!(ava.value(0), values[0]);
750        assert_eq!(ava.value(1), values[1]);
751        assert_eq!(ava.value(2), values[2]);
752    }
753
754    fn test_try_new_from_builder<T>(values: Vec<&T::Native>)
755    where
756        T: ByteArrayType,
757        <T as ByteArrayType>::Native: PartialEq + AsRef<<T as ByteArrayType>::Native>,
758    {
759        // test cast to bigger size unsigned
760        _test_try_new_from_builder_generic_for_key_types::<UInt8Type, UInt16Type, T>(
761            values.clone(),
762        );
763        // test cast going to smaller size unsigned
764        _test_try_new_from_builder_generic_for_key_types::<UInt16Type, UInt8Type, T>(
765            values.clone(),
766        );
767        // test cast going to bigger size signed
768        _test_try_new_from_builder_generic_for_key_types::<Int8Type, Int16Type, T>(values.clone());
769        // test cast going to smaller size signed
770        _test_try_new_from_builder_generic_for_key_types::<Int32Type, Int16Type, T>(values.clone());
771        // test going from signed to signed for different size changes
772        _test_try_new_from_builder_generic_for_key_types::<UInt8Type, Int16Type, T>(values.clone());
773        _test_try_new_from_builder_generic_for_key_types::<Int8Type, UInt8Type, T>(values.clone());
774        _test_try_new_from_builder_generic_for_key_types::<Int8Type, UInt16Type, T>(values.clone());
775        _test_try_new_from_builder_generic_for_key_types::<Int32Type, Int16Type, T>(values.clone());
776    }
777
778    #[test]
779    fn test_string_dictionary_builder_try_new_from_builder() {
780        test_try_new_from_builder::<GenericStringType<i32>>(vec!["abc", "def", "ghi"]);
781    }
782
783    #[test]
784    fn test_binary_dictionary_builder_try_new_from_builder() {
785        test_try_new_from_builder::<GenericBinaryType<i32>>(vec![b"abc", b"def", b"ghi"]);
786    }
787
788    #[test]
789    fn test_try_new_from_builder_cast_fails() {
790        let mut source_builder = StringDictionaryBuilder::<UInt16Type>::new();
791        for i in 0..257 {
792            source_builder.append_value(format!("val{i}"));
793        }
794
795        // there should be too many values that we can't downcast to the underlying type
796        // we have keys that wouldn't fit into UInt8Type
797        let result = StringDictionaryBuilder::<UInt8Type>::try_new_from_builder(source_builder);
798        assert!(result.is_err());
799        if let Err(e) = result {
800            assert!(matches!(e, ArrowError::CastError(_)));
801            assert_eq!(
802                e.to_string(),
803                "Cast error: Can't cast dictionary keys from source type UInt16 to type UInt8"
804            );
805        }
806    }
807
808    fn test_bytes_dictionary_builder_with_existing_dictionary<T>(
809        dictionary: GenericByteArray<T>,
810        values: Vec<&T::Native>,
811    ) where
812        T: ByteArrayType,
813        <T as ByteArrayType>::Native: PartialEq,
814        <T as ByteArrayType>::Native: AsRef<<T as ByteArrayType>::Native>,
815    {
816        let mut builder =
817            GenericByteDictionaryBuilder::<Int8Type, T>::new_with_dictionary(6, &dictionary)
818                .unwrap();
819        builder.append(values[0]).unwrap();
820        builder.append_null();
821        builder.append(values[1]).unwrap();
822        builder.append(values[1]).unwrap();
823        builder.append(values[0]).unwrap();
824        builder.append(values[2]).unwrap();
825        let array = builder.finish();
826
827        assert_eq!(
828            array.keys(),
829            &Int8Array::from(vec![Some(2), None, Some(1), Some(1), Some(2), Some(3)])
830        );
831
832        // Values are polymorphic and so require a downcast.
833        let av = array.values();
834        let ava: &GenericByteArray<T> = av.as_any().downcast_ref::<GenericByteArray<T>>().unwrap();
835
836        assert!(!ava.is_valid(0));
837        assert_eq!(ava.value(1), values[1]);
838        assert_eq!(ava.value(2), values[0]);
839        assert_eq!(ava.value(3), values[2]);
840    }
841
842    #[test]
843    fn test_string_dictionary_builder_with_existing_dictionary() {
844        test_bytes_dictionary_builder_with_existing_dictionary::<GenericStringType<i32>>(
845            StringArray::from(vec![None, Some("def"), Some("abc")]),
846            vec!["abc", "def", "ghi"],
847        );
848    }
849
850    #[test]
851    fn test_binary_dictionary_builder_with_existing_dictionary() {
852        let values: Vec<Option<&[u8]>> = vec![None, Some(b"def"), Some(b"abc")];
853        test_bytes_dictionary_builder_with_existing_dictionary::<GenericBinaryType<i32>>(
854            BinaryArray::from(values),
855            vec![b"abc", b"def", b"ghi"],
856        );
857    }
858
859    fn test_bytes_dictionary_builder_with_reserved_null_value<T>(
860        dictionary: GenericByteArray<T>,
861        values: Vec<&T::Native>,
862    ) where
863        T: ByteArrayType,
864        <T as ByteArrayType>::Native: PartialEq,
865        <T as ByteArrayType>::Native: AsRef<<T as ByteArrayType>::Native>,
866    {
867        let mut builder =
868            GenericByteDictionaryBuilder::<Int16Type, T>::new_with_dictionary(4, &dictionary)
869                .unwrap();
870        builder.append(values[0]).unwrap();
871        builder.append_null();
872        builder.append(values[1]).unwrap();
873        builder.append(values[0]).unwrap();
874        let array = builder.finish();
875
876        assert!(array.is_null(1));
877        assert!(!array.is_valid(1));
878
879        let keys = array.keys();
880
881        assert_eq!(keys.value(0), 1);
882        assert!(keys.is_null(1));
883        // zero initialization is currently guaranteed by Buffer allocation and resizing
884        assert_eq!(keys.value(1), 0);
885        assert_eq!(keys.value(2), 2);
886        assert_eq!(keys.value(3), 1);
887    }
888
889    #[test]
890    fn test_string_dictionary_builder_with_reserved_null_value() {
891        let v: Vec<Option<&str>> = vec![None];
892        test_bytes_dictionary_builder_with_reserved_null_value::<GenericStringType<i32>>(
893            StringArray::from(v),
894            vec!["abc", "def"],
895        );
896    }
897
898    #[test]
899    fn test_binary_dictionary_builder_with_reserved_null_value() {
900        let values: Vec<Option<&[u8]>> = vec![None];
901        test_bytes_dictionary_builder_with_reserved_null_value::<GenericBinaryType<i32>>(
902            BinaryArray::from(values),
903            vec![b"abc", b"def"],
904        );
905    }
906
907    #[test]
908    fn test_extend() {
909        let mut builder = GenericByteDictionaryBuilder::<Int32Type, Utf8Type>::new();
910        builder.extend(["a", "b", "c", "a", "b", "c"].into_iter().map(Some));
911        builder.extend(["c", "d", "a"].into_iter().map(Some));
912        let dict = builder.finish();
913        assert_eq!(dict.keys().values(), &[0, 1, 2, 0, 1, 2, 2, 3, 0]);
914        assert_eq!(dict.values().len(), 4);
915    }
916
917    #[test]
918    fn test_extend_dictionary() {
919        let some_dict = {
920            let mut builder = GenericByteDictionaryBuilder::<Int32Type, Utf8Type>::new();
921            builder.extend(["a", "b", "c", "a", "b", "c"].into_iter().map(Some));
922            builder.extend([None::<&str>]);
923            builder.extend(["c", "d", "a"].into_iter().map(Some));
924            builder.append_null();
925            builder.finish()
926        };
927
928        let mut builder = GenericByteDictionaryBuilder::<Int32Type, Utf8Type>::new();
929        builder.extend(["e", "e", "f", "e", "d"].into_iter().map(Some));
930        builder
931            .extend_dictionary(&some_dict.downcast_dict().unwrap())
932            .unwrap();
933        let dict = builder.finish();
934
935        assert_eq!(dict.values().len(), 6);
936
937        let values = dict
938            .downcast_dict::<GenericByteArray<Utf8Type>>()
939            .unwrap()
940            .into_iter()
941            .collect::<Vec<_>>();
942
943        assert_eq!(
944            values,
945            [
946                Some("e"),
947                Some("e"),
948                Some("f"),
949                Some("e"),
950                Some("d"),
951                Some("a"),
952                Some("b"),
953                Some("c"),
954                Some("a"),
955                Some("b"),
956                Some("c"),
957                None,
958                Some("c"),
959                Some("d"),
960                Some("a"),
961                None
962            ]
963        );
964    }
965    #[test]
966    fn test_extend_dictionary_with_null_in_mapped_value() {
967        let some_dict = {
968            let mut values_builder = GenericByteBuilder::<Utf8Type>::new();
969            let mut keys_builder = PrimitiveBuilder::<Int32Type>::new();
970
971            // Manually build a dictionary values that the mapped values have null
972            values_builder.append_null();
973            keys_builder.append_value(0);
974            values_builder.append_value("I like worm hugs");
975            keys_builder.append_value(1);
976
977            let values = values_builder.finish();
978            let keys = keys_builder.finish();
979
980            let data_type = DataType::Dictionary(
981                Box::new(Int32Type::DATA_TYPE),
982                Box::new(Utf8Type::DATA_TYPE),
983            );
984
985            let builder = keys
986                .into_data()
987                .into_builder()
988                .data_type(data_type)
989                .child_data(vec![values.into_data()]);
990
991            DictionaryArray::from(unsafe { builder.build_unchecked() })
992        };
993
994        let some_dict_values = some_dict.values().as_string::<i32>();
995        assert_eq!(
996            some_dict_values.into_iter().collect::<Vec<_>>(),
997            &[None, Some("I like worm hugs")]
998        );
999
1000        let mut builder = GenericByteDictionaryBuilder::<Int32Type, Utf8Type>::new();
1001        builder
1002            .extend_dictionary(&some_dict.downcast_dict().unwrap())
1003            .unwrap();
1004        let dict = builder.finish();
1005
1006        assert_eq!(dict.values().len(), 1);
1007
1008        let values = dict
1009            .downcast_dict::<GenericByteArray<Utf8Type>>()
1010            .unwrap()
1011            .into_iter()
1012            .collect::<Vec<_>>();
1013
1014        assert_eq!(values, [None, Some("I like worm hugs")]);
1015    }
1016
1017    #[test]
1018    fn test_extend_all_null_dictionary() {
1019        let some_dict = {
1020            let mut builder = GenericByteDictionaryBuilder::<Int32Type, Utf8Type>::new();
1021            builder.append_nulls(2);
1022            builder.finish()
1023        };
1024
1025        let mut builder = GenericByteDictionaryBuilder::<Int32Type, Utf8Type>::new();
1026        builder
1027            .extend_dictionary(&some_dict.downcast_dict().unwrap())
1028            .unwrap();
1029        let dict = builder.finish();
1030
1031        assert_eq!(dict.values().len(), 0);
1032
1033        let values = dict
1034            .downcast_dict::<GenericByteArray<Utf8Type>>()
1035            .unwrap()
1036            .into_iter()
1037            .collect::<Vec<_>>();
1038
1039        assert_eq!(values, [None, None]);
1040    }
1041
1042    #[test]
1043    fn test_finish_preserve_values() {
1044        // Create the first dictionary
1045        let mut builder = GenericByteDictionaryBuilder::<Int32Type, Utf8Type>::new();
1046        builder.append("a").unwrap();
1047        builder.append("b").unwrap();
1048        builder.append("c").unwrap();
1049        let dict = builder.finish_preserve_values();
1050        assert_eq!(dict.keys().values(), &[0, 1, 2]);
1051        assert_eq!(dict.values().len(), 3);
1052        let values = dict
1053            .downcast_dict::<GenericByteArray<Utf8Type>>()
1054            .unwrap()
1055            .into_iter()
1056            .collect::<Vec<_>>();
1057        assert_eq!(values, [Some("a"), Some("b"), Some("c")]);
1058
1059        // Create a new dictionary
1060        builder.append("d").unwrap();
1061        builder.append("e").unwrap();
1062        let dict2 = builder.finish_preserve_values();
1063
1064        // Make sure the keys are assigned after the old ones and we have the
1065        // right values
1066        assert_eq!(dict2.keys().values(), &[3, 4]);
1067        let values = dict2
1068            .downcast_dict::<GenericByteArray<Utf8Type>>()
1069            .unwrap()
1070            .into_iter()
1071            .collect::<Vec<_>>();
1072        assert_eq!(values, [Some("d"), Some("e")]);
1073
1074        // Check that we have all of the expected values
1075        assert_eq!(dict2.values().len(), 5);
1076        let all_values = dict2
1077            .values()
1078            .as_any()
1079            .downcast_ref::<StringArray>()
1080            .unwrap()
1081            .into_iter()
1082            .collect::<Vec<_>>();
1083        assert_eq!(
1084            all_values,
1085            [Some("a"), Some("b"), Some("c"), Some("d"), Some("e"),]
1086        );
1087    }
1088}