arrow_array/array/
dictionary_array.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::{PrimitiveDictionaryBuilder, StringDictionaryBuilder};
19use crate::cast::AsArray;
20use crate::iterator::ArrayIter;
21use crate::types::*;
22use crate::{
23    make_array, Array, ArrayAccessor, ArrayRef, ArrowNativeTypeOp, PrimitiveArray, Scalar,
24    StringArray,
25};
26use arrow_buffer::bit_util::set_bit;
27use arrow_buffer::buffer::NullBuffer;
28use arrow_buffer::{ArrowNativeType, BooleanBuffer, BooleanBufferBuilder};
29use arrow_data::ArrayData;
30use arrow_schema::{ArrowError, DataType};
31use std::any::Any;
32use std::sync::Arc;
33
34/// A [`DictionaryArray`] indexed by `i8`
35///
36/// # Example: Using `collect`
37/// ```
38/// # use arrow_array::{Array, Int8DictionaryArray, Int8Array, StringArray};
39/// # use std::sync::Arc;
40///
41/// let array: Int8DictionaryArray = vec!["a", "a", "b", "c"].into_iter().collect();
42/// let values: Arc<dyn Array> = Arc::new(StringArray::from(vec!["a", "b", "c"]));
43/// assert_eq!(array.keys(), &Int8Array::from(vec![0, 0, 1, 2]));
44/// assert_eq!(array.values(), &values);
45/// ```
46///
47/// See [`DictionaryArray`] for more information and examples
48pub type Int8DictionaryArray = DictionaryArray<Int8Type>;
49
50/// A [`DictionaryArray`] indexed by `i16`
51///
52/// # Example: Using `collect`
53/// ```
54/// # use arrow_array::{Array, Int16DictionaryArray, Int16Array, StringArray};
55/// # use std::sync::Arc;
56///
57/// let array: Int16DictionaryArray = vec!["a", "a", "b", "c"].into_iter().collect();
58/// let values: Arc<dyn Array> = Arc::new(StringArray::from(vec!["a", "b", "c"]));
59/// assert_eq!(array.keys(), &Int16Array::from(vec![0, 0, 1, 2]));
60/// assert_eq!(array.values(), &values);
61/// ```
62///
63/// See [`DictionaryArray`] for more information and examples
64pub type Int16DictionaryArray = DictionaryArray<Int16Type>;
65
66/// A [`DictionaryArray`] indexed by `i32`
67///
68/// # Example: Using `collect`
69/// ```
70/// # use arrow_array::{Array, Int32DictionaryArray, Int32Array, StringArray};
71/// # use std::sync::Arc;
72///
73/// let array: Int32DictionaryArray = vec!["a", "a", "b", "c"].into_iter().collect();
74/// let values: Arc<dyn Array> = Arc::new(StringArray::from(vec!["a", "b", "c"]));
75/// assert_eq!(array.keys(), &Int32Array::from(vec![0, 0, 1, 2]));
76/// assert_eq!(array.values(), &values);
77/// ```
78///
79/// See [`DictionaryArray`] for more information and examples
80pub type Int32DictionaryArray = DictionaryArray<Int32Type>;
81
82/// A [`DictionaryArray`] indexed by `i64`
83///
84/// # Example: Using `collect`
85/// ```
86/// # use arrow_array::{Array, Int64DictionaryArray, Int64Array, StringArray};
87/// # use std::sync::Arc;
88///
89/// let array: Int64DictionaryArray = vec!["a", "a", "b", "c"].into_iter().collect();
90/// let values: Arc<dyn Array> = Arc::new(StringArray::from(vec!["a", "b", "c"]));
91/// assert_eq!(array.keys(), &Int64Array::from(vec![0, 0, 1, 2]));
92/// assert_eq!(array.values(), &values);
93/// ```
94///
95/// See [`DictionaryArray`] for more information and examples
96pub type Int64DictionaryArray = DictionaryArray<Int64Type>;
97
98/// A [`DictionaryArray`] indexed by `u8`
99///
100/// # Example: Using `collect`
101/// ```
102/// # use arrow_array::{Array, UInt8DictionaryArray, UInt8Array, StringArray};
103/// # use std::sync::Arc;
104///
105/// let array: UInt8DictionaryArray = vec!["a", "a", "b", "c"].into_iter().collect();
106/// let values: Arc<dyn Array> = Arc::new(StringArray::from(vec!["a", "b", "c"]));
107/// assert_eq!(array.keys(), &UInt8Array::from(vec![0, 0, 1, 2]));
108/// assert_eq!(array.values(), &values);
109/// ```
110///
111/// See [`DictionaryArray`] for more information and examples
112pub type UInt8DictionaryArray = DictionaryArray<UInt8Type>;
113
114/// A [`DictionaryArray`] indexed by `u16`
115///
116/// # Example: Using `collect`
117/// ```
118/// # use arrow_array::{Array, UInt16DictionaryArray, UInt16Array, StringArray};
119/// # use std::sync::Arc;
120///
121/// let array: UInt16DictionaryArray = vec!["a", "a", "b", "c"].into_iter().collect();
122/// let values: Arc<dyn Array> = Arc::new(StringArray::from(vec!["a", "b", "c"]));
123/// assert_eq!(array.keys(), &UInt16Array::from(vec![0, 0, 1, 2]));
124/// assert_eq!(array.values(), &values);
125/// ```
126///
127/// See [`DictionaryArray`] for more information and examples
128pub type UInt16DictionaryArray = DictionaryArray<UInt16Type>;
129
130/// A [`DictionaryArray`] indexed by `u32`
131///
132/// # Example: Using `collect`
133/// ```
134/// # use arrow_array::{Array, UInt32DictionaryArray, UInt32Array, StringArray};
135/// # use std::sync::Arc;
136///
137/// let array: UInt32DictionaryArray = vec!["a", "a", "b", "c"].into_iter().collect();
138/// let values: Arc<dyn Array> = Arc::new(StringArray::from(vec!["a", "b", "c"]));
139/// assert_eq!(array.keys(), &UInt32Array::from(vec![0, 0, 1, 2]));
140/// assert_eq!(array.values(), &values);
141/// ```
142///
143/// See [`DictionaryArray`] for more information and examples
144pub type UInt32DictionaryArray = DictionaryArray<UInt32Type>;
145
146/// A [`DictionaryArray`] indexed by `u64`
147///
148/// # Example: Using `collect`
149/// ```
150/// # use arrow_array::{Array, UInt64DictionaryArray, UInt64Array, StringArray};
151/// # use std::sync::Arc;
152///
153/// let array: UInt64DictionaryArray = vec!["a", "a", "b", "c"].into_iter().collect();
154/// let values: Arc<dyn Array> = Arc::new(StringArray::from(vec!["a", "b", "c"]));
155/// assert_eq!(array.keys(), &UInt64Array::from(vec![0, 0, 1, 2]));
156/// assert_eq!(array.values(), &values);
157/// ```
158///
159/// See [`DictionaryArray`] for more information and examples
160pub type UInt64DictionaryArray = DictionaryArray<UInt64Type>;
161
162/// An array of [dictionary encoded values](https://arrow.apache.org/docs/format/Columnar.html#dictionary-encoded-layout)
163///
164/// This is mostly used to represent strings or a limited set of primitive types as integers,
165/// for example when doing NLP analysis or representing chromosomes by name.
166///
167/// [`DictionaryArray`] are represented using a `keys` array and a
168/// `values` array, which may be different lengths. The `keys` array
169/// stores indexes in the `values` array which holds
170/// the corresponding logical value, as shown here:
171///
172/// ```text
173/// ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─
174///   ┌─────────────────┐  ┌─────────┐ │     ┌─────────────────┐
175/// │ │        A        │  │    0    │       │        A        │     values[keys[0]]
176///   ├─────────────────┤  ├─────────┤ │     ├─────────────────┤
177/// │ │        D        │  │    2    │       │        B        │     values[keys[1]]
178///   ├─────────────────┤  ├─────────┤ │     ├─────────────────┤
179/// │ │        B        │  │    2    │       │        B        │     values[keys[2]]
180///   └─────────────────┘  ├─────────┤ │     ├─────────────────┤
181/// │                      │    1    │       │        D        │     values[keys[3]]
182///                        ├─────────┤ │     ├─────────────────┤
183/// │                      │    1    │       │        D        │     values[keys[4]]
184///                        ├─────────┤ │     ├─────────────────┤
185/// │                      │    0    │       │        A        │     values[keys[5]]
186///                        └─────────┘ │     └─────────────────┘
187/// │       values            keys
188///  ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘
189///                                             Logical array
190///                                                Contents
191///           DictionaryArray
192///              length = 6
193/// ```
194///
195/// # Example: From Nullable Data
196///
197/// ```
198/// # use arrow_array::{DictionaryArray, Int8Array, types::Int8Type};
199/// let test = vec!["a", "a", "b", "c"];
200/// let array : DictionaryArray<Int8Type> = test.iter().map(|&x| if x == "b" {None} else {Some(x)}).collect();
201/// assert_eq!(array.keys(), &Int8Array::from(vec![Some(0), Some(0), None, Some(1)]));
202/// ```
203///
204/// # Example: From Non-Nullable Data
205///
206/// ```
207/// # use arrow_array::{DictionaryArray, Int8Array, types::Int8Type};
208/// let test = vec!["a", "a", "b", "c"];
209/// let array : DictionaryArray<Int8Type> = test.into_iter().collect();
210/// assert_eq!(array.keys(), &Int8Array::from(vec![0, 0, 1, 2]));
211/// ```
212///
213/// # Example: From Existing Arrays
214///
215/// ```
216/// # use std::sync::Arc;
217/// # use arrow_array::{DictionaryArray, Int8Array, StringArray, types::Int8Type};
218/// // You can form your own DictionaryArray by providing the
219/// // values (dictionary) and keys (indexes into the dictionary):
220/// let values = StringArray::from_iter_values(["a", "b", "c"]);
221/// let keys = Int8Array::from_iter_values([0, 0, 1, 2]);
222/// let array = DictionaryArray::<Int8Type>::try_new(keys, Arc::new(values)).unwrap();
223/// let expected: DictionaryArray::<Int8Type> = vec!["a", "a", "b", "c"].into_iter().collect();
224/// assert_eq!(&array, &expected);
225/// ```
226///
227/// # Example: Using Builder
228///
229/// ```
230/// # use arrow_array::{Array, StringArray};
231/// # use arrow_array::builder::StringDictionaryBuilder;
232/// # use arrow_array::types::Int32Type;
233/// let mut builder = StringDictionaryBuilder::<Int32Type>::new();
234/// builder.append_value("a");
235/// builder.append_null();
236/// builder.append_value("a");
237/// builder.append_value("b");
238/// let array = builder.finish();
239///
240/// let values: Vec<_> = array.downcast_dict::<StringArray>().unwrap().into_iter().collect();
241/// assert_eq!(&values, &[Some("a"), None, Some("a"), Some("b")]);
242/// ```
243pub struct DictionaryArray<K: ArrowDictionaryKeyType> {
244    data_type: DataType,
245
246    /// The keys of this dictionary. These are constructed from the
247    /// buffer and null bitmap of `data`.  Also, note that these do
248    /// not correspond to the true values of this array. Rather, they
249    /// map to the real values.
250    keys: PrimitiveArray<K>,
251
252    /// Array of dictionary values (can by any DataType).
253    values: ArrayRef,
254
255    /// Values are ordered.
256    is_ordered: bool,
257}
258
259impl<K: ArrowDictionaryKeyType> Clone for DictionaryArray<K> {
260    fn clone(&self) -> Self {
261        Self {
262            data_type: self.data_type.clone(),
263            keys: self.keys.clone(),
264            values: self.values.clone(),
265            is_ordered: self.is_ordered,
266        }
267    }
268}
269
270impl<K: ArrowDictionaryKeyType> DictionaryArray<K> {
271    /// Attempt to create a new DictionaryArray with a specified keys
272    /// (indexes into the dictionary) and values (dictionary)
273    /// array.
274    ///
275    /// # Panics
276    ///
277    /// Panics if [`Self::try_new`] returns an error
278    pub fn new(keys: PrimitiveArray<K>, values: ArrayRef) -> Self {
279        Self::try_new(keys, values).unwrap()
280    }
281
282    /// Attempt to create a new DictionaryArray with a specified keys
283    /// (indexes into the dictionary) and values (dictionary)
284    /// array.
285    ///
286    /// # Errors
287    ///
288    /// Returns an error if any `keys[i] >= values.len() || keys[i] < 0`
289    pub fn try_new(keys: PrimitiveArray<K>, values: ArrayRef) -> Result<Self, ArrowError> {
290        let data_type = DataType::Dictionary(
291            Box::new(keys.data_type().clone()),
292            Box::new(values.data_type().clone()),
293        );
294
295        let zero = K::Native::usize_as(0);
296        let values_len = values.len();
297
298        if let Some((idx, v)) =
299            keys.values().iter().enumerate().find(|(idx, v)| {
300                (v.is_lt(zero) || v.as_usize() >= values_len) && keys.is_valid(*idx)
301            })
302        {
303            return Err(ArrowError::InvalidArgumentError(format!(
304                "Invalid dictionary key {v:?} at index {idx}, expected 0 <= key < {values_len}",
305            )));
306        }
307
308        Ok(Self {
309            data_type,
310            keys,
311            values,
312            is_ordered: false,
313        })
314    }
315
316    /// Create a new [`Scalar`] from `value`
317    pub fn new_scalar<T: Array + 'static>(value: Scalar<T>) -> Scalar<Self> {
318        Scalar::new(Self::new(
319            PrimitiveArray::new(vec![K::Native::usize_as(0)].into(), None),
320            Arc::new(value.into_inner()),
321        ))
322    }
323
324    /// Create a new [`DictionaryArray`] without performing validation
325    ///
326    /// # Safety
327    ///
328    /// Safe provided [`Self::try_new`] would not return an error
329    pub unsafe fn new_unchecked(keys: PrimitiveArray<K>, values: ArrayRef) -> Self {
330        let data_type = DataType::Dictionary(
331            Box::new(keys.data_type().clone()),
332            Box::new(values.data_type().clone()),
333        );
334
335        Self {
336            data_type,
337            keys,
338            values,
339            is_ordered: false,
340        }
341    }
342
343    /// Deconstruct this array into its constituent parts
344    pub fn into_parts(self) -> (PrimitiveArray<K>, ArrayRef) {
345        (self.keys, self.values)
346    }
347
348    /// Return an array view of the keys of this dictionary as a PrimitiveArray.
349    pub fn keys(&self) -> &PrimitiveArray<K> {
350        &self.keys
351    }
352
353    /// If `value` is present in `values` (aka the dictionary),
354    /// returns the corresponding key (index into the `values`
355    /// array). Otherwise returns `None`.
356    ///
357    /// Panics if `values` is not a [`StringArray`].
358    pub fn lookup_key(&self, value: &str) -> Option<K::Native> {
359        let rd_buf: &StringArray = self.values.as_any().downcast_ref::<StringArray>().unwrap();
360
361        (0..rd_buf.len())
362            .position(|i| rd_buf.value(i) == value)
363            .and_then(K::Native::from_usize)
364    }
365
366    /// Returns a reference to the dictionary values array
367    pub fn values(&self) -> &ArrayRef {
368        &self.values
369    }
370
371    /// Returns a clone of the value type of this list.
372    pub fn value_type(&self) -> DataType {
373        self.values.data_type().clone()
374    }
375
376    /// The length of the dictionary is the length of the keys array.
377    pub fn len(&self) -> usize {
378        self.keys.len()
379    }
380
381    /// Whether this dictionary is empty
382    pub fn is_empty(&self) -> bool {
383        self.keys.is_empty()
384    }
385
386    /// Currently exists for compatibility purposes with Arrow IPC.
387    pub fn is_ordered(&self) -> bool {
388        self.is_ordered
389    }
390
391    /// Return an iterator over the keys (indexes into the dictionary)
392    pub fn keys_iter(&self) -> impl Iterator<Item = Option<usize>> + '_ {
393        self.keys.iter().map(|key| key.map(|k| k.as_usize()))
394    }
395
396    /// Return the value of `keys` (the dictionary key) at index `i`,
397    /// cast to `usize`, `None` if the value at `i` is `NULL`.
398    pub fn key(&self, i: usize) -> Option<usize> {
399        self.keys.is_valid(i).then(|| self.keys.value(i).as_usize())
400    }
401
402    /// Returns a zero-copy slice of this array with the indicated offset and length.
403    pub fn slice(&self, offset: usize, length: usize) -> Self {
404        Self {
405            data_type: self.data_type.clone(),
406            keys: self.keys.slice(offset, length),
407            values: self.values.clone(),
408            is_ordered: self.is_ordered,
409        }
410    }
411
412    /// Downcast this dictionary to a [`TypedDictionaryArray`]
413    ///
414    /// ```
415    /// use arrow_array::{Array, ArrayAccessor, DictionaryArray, StringArray, types::Int32Type};
416    ///
417    /// let orig = [Some("a"), Some("b"), None];
418    /// let dictionary = DictionaryArray::<Int32Type>::from_iter(orig);
419    /// let typed = dictionary.downcast_dict::<StringArray>().unwrap();
420    /// assert_eq!(typed.value(0), "a");
421    /// assert_eq!(typed.value(1), "b");
422    /// assert!(typed.is_null(2));
423    /// ```
424    ///
425    pub fn downcast_dict<V: 'static>(&self) -> Option<TypedDictionaryArray<'_, K, V>> {
426        let values = self.values.as_any().downcast_ref()?;
427        Some(TypedDictionaryArray {
428            dictionary: self,
429            values,
430        })
431    }
432
433    /// Returns a new dictionary with the same keys as the current instance
434    /// but with a different set of dictionary values
435    ///
436    /// This can be used to perform an operation on the values of a dictionary
437    ///
438    /// # Panics
439    ///
440    /// Panics if `values` has a length less than the current values
441    ///
442    /// ```
443    /// # use std::sync::Arc;
444    /// # use arrow_array::builder::PrimitiveDictionaryBuilder;
445    /// # use arrow_array::{Int8Array, Int64Array, ArrayAccessor};
446    /// # use arrow_array::types::{Int32Type, Int8Type};
447    ///
448    /// // Construct a Dict(Int32, Int8)
449    /// let mut builder = PrimitiveDictionaryBuilder::<Int32Type, Int8Type>::with_capacity(2, 200);
450    /// for i in 0..100 {
451    ///     builder.append(i % 2).unwrap();
452    /// }
453    ///
454    /// let dictionary = builder.finish();
455    ///
456    /// // Perform a widening cast of dictionary values
457    /// let typed_dictionary = dictionary.downcast_dict::<Int8Array>().unwrap();
458    /// let values: Int64Array = typed_dictionary.values().unary(|x| x as i64);
459    ///
460    /// // Create a Dict(Int32,
461    /// let new = dictionary.with_values(Arc::new(values));
462    ///
463    /// // Verify values are as expected
464    /// let new_typed = new.downcast_dict::<Int64Array>().unwrap();
465    /// for i in 0..100 {
466    ///     assert_eq!(new_typed.value(i), (i % 2) as i64)
467    /// }
468    /// ```
469    ///
470    pub fn with_values(&self, values: ArrayRef) -> Self {
471        assert!(values.len() >= self.values.len());
472        let data_type =
473            DataType::Dictionary(Box::new(K::DATA_TYPE), Box::new(values.data_type().clone()));
474        Self {
475            data_type,
476            keys: self.keys.clone(),
477            values,
478            is_ordered: false,
479        }
480    }
481
482    /// Returns `PrimitiveDictionaryBuilder` of this dictionary array for mutating
483    /// its keys and values if the underlying data buffer is not shared by others.
484    pub fn into_primitive_dict_builder<V>(self) -> Result<PrimitiveDictionaryBuilder<K, V>, Self>
485    where
486        V: ArrowPrimitiveType,
487    {
488        if !self.value_type().is_primitive() {
489            return Err(self);
490        }
491
492        let key_array = self.keys().clone();
493        let value_array = self.values().as_primitive::<V>().clone();
494
495        drop(self.keys);
496        drop(self.values);
497
498        let key_builder = key_array.into_builder();
499        let value_builder = value_array.into_builder();
500
501        match (key_builder, value_builder) {
502            (Ok(key_builder), Ok(value_builder)) => Ok(unsafe {
503                PrimitiveDictionaryBuilder::new_from_builders(key_builder, value_builder)
504            }),
505            (Err(key_array), Ok(mut value_builder)) => {
506                Err(Self::try_new(key_array, Arc::new(value_builder.finish())).unwrap())
507            }
508            (Ok(mut key_builder), Err(value_array)) => {
509                Err(Self::try_new(key_builder.finish(), Arc::new(value_array)).unwrap())
510            }
511            (Err(key_array), Err(value_array)) => {
512                Err(Self::try_new(key_array, Arc::new(value_array)).unwrap())
513            }
514        }
515    }
516
517    /// Applies an unary and infallible function to a mutable dictionary array.
518    /// Mutable dictionary array means that the buffers are not shared with other arrays.
519    /// As a result, this mutates the buffers directly without allocating new buffers.
520    ///
521    /// # Implementation
522    ///
523    /// This will apply the function for all dictionary values, including those on null slots.
524    /// This implies that the operation must be infallible for any value of the corresponding type
525    /// or this function may panic.
526    /// # Example
527    /// ```
528    /// # use std::sync::Arc;
529    /// # use arrow_array::{Array, ArrayAccessor, DictionaryArray, StringArray, types::{Int8Type, Int32Type}};
530    /// # use arrow_array::{Int8Array, Int32Array};
531    /// let values = Int32Array::from(vec![Some(10), Some(20), None]);
532    /// let keys = Int8Array::from_iter_values([0, 0, 1, 2]);
533    /// let dictionary = DictionaryArray::<Int8Type>::try_new(keys, Arc::new(values)).unwrap();
534    /// let c = dictionary.unary_mut::<_, Int32Type>(|x| x + 1).unwrap();
535    /// let typed = c.downcast_dict::<Int32Array>().unwrap();
536    /// assert_eq!(typed.value(0), 11);
537    /// assert_eq!(typed.value(1), 11);
538    /// assert_eq!(typed.value(2), 21);
539    /// ```
540    pub fn unary_mut<F, V>(self, op: F) -> Result<DictionaryArray<K>, DictionaryArray<K>>
541    where
542        V: ArrowPrimitiveType,
543        F: Fn(V::Native) -> V::Native,
544    {
545        let mut builder: PrimitiveDictionaryBuilder<K, V> = self.into_primitive_dict_builder()?;
546        builder
547            .values_slice_mut()
548            .iter_mut()
549            .for_each(|v| *v = op(*v));
550        Ok(builder.finish())
551    }
552
553    /// Computes an occupancy mask for this dictionary's values
554    ///
555    /// For each value in [`Self::values`] the corresponding bit will be set in the
556    /// returned mask if it is referenced by a key in this [`DictionaryArray`]
557    pub fn occupancy(&self) -> BooleanBuffer {
558        let len = self.values.len();
559        let mut builder = BooleanBufferBuilder::new(len);
560        builder.resize(len);
561        let slice = builder.as_slice_mut();
562        match self.keys.nulls().filter(|n| n.null_count() > 0) {
563            Some(n) => {
564                let v = self.keys.values();
565                n.valid_indices()
566                    .for_each(|idx| set_bit(slice, v[idx].as_usize()))
567            }
568            None => {
569                let v = self.keys.values();
570                v.iter().for_each(|v| set_bit(slice, v.as_usize()))
571            }
572        }
573        builder.finish()
574    }
575}
576
577/// Constructs a `DictionaryArray` from an array data reference.
578impl<T: ArrowDictionaryKeyType> From<ArrayData> for DictionaryArray<T> {
579    fn from(data: ArrayData) -> Self {
580        assert_eq!(
581            data.buffers().len(),
582            1,
583            "DictionaryArray data should contain a single buffer only (keys)."
584        );
585        assert_eq!(
586            data.child_data().len(),
587            1,
588            "DictionaryArray should contain a single child array (values)."
589        );
590
591        if let DataType::Dictionary(key_data_type, _) = data.data_type() {
592            assert_eq!(
593                &T::DATA_TYPE,
594                key_data_type.as_ref(),
595                "DictionaryArray's data type must match, expected {} got {}",
596                T::DATA_TYPE,
597                key_data_type
598            );
599
600            let values = make_array(data.child_data()[0].clone());
601            let data_type = data.data_type().clone();
602
603            // create a zero-copy of the keys' data
604            // SAFETY:
605            // ArrayData is valid and verified type above
606
607            let keys = PrimitiveArray::<T>::from(unsafe {
608                data.into_builder()
609                    .data_type(T::DATA_TYPE)
610                    .child_data(vec![])
611                    .build_unchecked()
612            });
613
614            Self {
615                data_type,
616                keys,
617                values,
618                is_ordered: false,
619            }
620        } else {
621            panic!("DictionaryArray must have Dictionary data type.")
622        }
623    }
624}
625
626impl<T: ArrowDictionaryKeyType> From<DictionaryArray<T>> for ArrayData {
627    fn from(array: DictionaryArray<T>) -> Self {
628        let builder = array
629            .keys
630            .into_data()
631            .into_builder()
632            .data_type(array.data_type)
633            .child_data(vec![array.values.to_data()]);
634
635        unsafe { builder.build_unchecked() }
636    }
637}
638
639/// Constructs a `DictionaryArray` from an iterator of optional strings.
640///
641/// # Example:
642/// ```
643/// use arrow_array::{DictionaryArray, PrimitiveArray, StringArray, types::Int8Type};
644///
645/// let test = vec!["a", "a", "b", "c"];
646/// let array: DictionaryArray<Int8Type> = test
647///     .iter()
648///     .map(|&x| if x == "b" { None } else { Some(x) })
649///     .collect();
650/// assert_eq!(
651///     "DictionaryArray {keys: PrimitiveArray<Int8>\n[\n  0,\n  0,\n  null,\n  1,\n] values: StringArray\n[\n  \"a\",\n  \"c\",\n]}\n",
652///     format!("{:?}", array)
653/// );
654/// ```
655impl<'a, T: ArrowDictionaryKeyType> FromIterator<Option<&'a str>> for DictionaryArray<T> {
656    fn from_iter<I: IntoIterator<Item = Option<&'a str>>>(iter: I) -> Self {
657        let it = iter.into_iter();
658        let (lower, _) = it.size_hint();
659        let mut builder = StringDictionaryBuilder::with_capacity(lower, 256, 1024);
660        builder.extend(it);
661        builder.finish()
662    }
663}
664
665/// Constructs a `DictionaryArray` from an iterator of strings.
666///
667/// # Example:
668///
669/// ```
670/// use arrow_array::{DictionaryArray, PrimitiveArray, StringArray, types::Int8Type};
671///
672/// let test = vec!["a", "a", "b", "c"];
673/// let array: DictionaryArray<Int8Type> = test.into_iter().collect();
674/// assert_eq!(
675///     "DictionaryArray {keys: PrimitiveArray<Int8>\n[\n  0,\n  0,\n  1,\n  2,\n] values: StringArray\n[\n  \"a\",\n  \"b\",\n  \"c\",\n]}\n",
676///     format!("{:?}", array)
677/// );
678/// ```
679impl<'a, T: ArrowDictionaryKeyType> FromIterator<&'a str> for DictionaryArray<T> {
680    fn from_iter<I: IntoIterator<Item = &'a str>>(iter: I) -> Self {
681        let it = iter.into_iter();
682        let (lower, _) = it.size_hint();
683        let mut builder = StringDictionaryBuilder::with_capacity(lower, 256, 1024);
684        it.for_each(|i| {
685            builder
686                .append(i)
687                .expect("Unable to append a value to a dictionary array.");
688        });
689
690        builder.finish()
691    }
692}
693
694impl<T: ArrowDictionaryKeyType> Array for DictionaryArray<T> {
695    fn as_any(&self) -> &dyn Any {
696        self
697    }
698
699    fn to_data(&self) -> ArrayData {
700        self.clone().into()
701    }
702
703    fn into_data(self) -> ArrayData {
704        self.into()
705    }
706
707    fn data_type(&self) -> &DataType {
708        &self.data_type
709    }
710
711    fn slice(&self, offset: usize, length: usize) -> ArrayRef {
712        Arc::new(self.slice(offset, length))
713    }
714
715    fn len(&self) -> usize {
716        self.keys.len()
717    }
718
719    fn is_empty(&self) -> bool {
720        self.keys.is_empty()
721    }
722
723    fn offset(&self) -> usize {
724        self.keys.offset()
725    }
726
727    fn nulls(&self) -> Option<&NullBuffer> {
728        self.keys.nulls()
729    }
730
731    fn logical_nulls(&self) -> Option<NullBuffer> {
732        match self.values.nulls() {
733            None => self.nulls().cloned(),
734            Some(value_nulls) => {
735                let mut builder = BooleanBufferBuilder::new(self.len());
736                match self.keys.nulls() {
737                    Some(n) => builder.append_buffer(n.inner()),
738                    None => builder.append_n(self.len(), true),
739                }
740                for (idx, k) in self.keys.values().iter().enumerate() {
741                    let k = k.as_usize();
742                    // Check range to allow for nulls
743                    if k < value_nulls.len() && value_nulls.is_null(k) {
744                        builder.set_bit(idx, false);
745                    }
746                }
747                Some(builder.finish().into())
748            }
749        }
750    }
751
752    fn is_nullable(&self) -> bool {
753        !self.is_empty() && (self.nulls().is_some() || self.values.is_nullable())
754    }
755
756    fn get_buffer_memory_size(&self) -> usize {
757        self.keys.get_buffer_memory_size() + self.values.get_buffer_memory_size()
758    }
759
760    fn get_array_memory_size(&self) -> usize {
761        std::mem::size_of::<Self>()
762            + self.keys.get_buffer_memory_size()
763            + self.values.get_array_memory_size()
764    }
765}
766
767impl<T: ArrowDictionaryKeyType> std::fmt::Debug for DictionaryArray<T> {
768    fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
769        writeln!(
770            f,
771            "DictionaryArray {{keys: {:?} values: {:?}}}",
772            self.keys, self.values
773        )
774    }
775}
776
777/// A [`DictionaryArray`] typed on its child values array
778///
779/// Implements [`ArrayAccessor`] allowing fast access to its elements
780///
781/// ```
782/// use arrow_array::{DictionaryArray, StringArray, types::Int32Type};
783///
784/// let orig = ["a", "b", "a", "b"];
785/// let dictionary = DictionaryArray::<Int32Type>::from_iter(orig);
786///
787/// // `TypedDictionaryArray` allows you to access the values directly
788/// let typed = dictionary.downcast_dict::<StringArray>().unwrap();
789///
790/// for (maybe_val, orig) in typed.into_iter().zip(orig) {
791///     assert_eq!(maybe_val.unwrap(), orig)
792/// }
793/// ```
794pub struct TypedDictionaryArray<'a, K: ArrowDictionaryKeyType, V> {
795    /// The dictionary array
796    dictionary: &'a DictionaryArray<K>,
797    /// The values of the dictionary
798    values: &'a V,
799}
800
801// Manually implement `Clone` to avoid `V: Clone` type constraint
802impl<K: ArrowDictionaryKeyType, V> Clone for TypedDictionaryArray<'_, K, V> {
803    fn clone(&self) -> Self {
804        *self
805    }
806}
807
808impl<K: ArrowDictionaryKeyType, V> Copy for TypedDictionaryArray<'_, K, V> {}
809
810impl<K: ArrowDictionaryKeyType, V> std::fmt::Debug for TypedDictionaryArray<'_, K, V> {
811    fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
812        writeln!(f, "TypedDictionaryArray({:?})", self.dictionary)
813    }
814}
815
816impl<'a, K: ArrowDictionaryKeyType, V> TypedDictionaryArray<'a, K, V> {
817    /// Returns the keys of this [`TypedDictionaryArray`]
818    pub fn keys(&self) -> &'a PrimitiveArray<K> {
819        self.dictionary.keys()
820    }
821
822    /// Returns the values of this [`TypedDictionaryArray`]
823    pub fn values(&self) -> &'a V {
824        self.values
825    }
826}
827
828impl<K: ArrowDictionaryKeyType, V: Sync> Array for TypedDictionaryArray<'_, K, V> {
829    fn as_any(&self) -> &dyn Any {
830        self.dictionary
831    }
832
833    fn to_data(&self) -> ArrayData {
834        self.dictionary.to_data()
835    }
836
837    fn into_data(self) -> ArrayData {
838        self.dictionary.into_data()
839    }
840
841    fn data_type(&self) -> &DataType {
842        self.dictionary.data_type()
843    }
844
845    fn slice(&self, offset: usize, length: usize) -> ArrayRef {
846        Arc::new(self.dictionary.slice(offset, length))
847    }
848
849    fn len(&self) -> usize {
850        self.dictionary.len()
851    }
852
853    fn is_empty(&self) -> bool {
854        self.dictionary.is_empty()
855    }
856
857    fn offset(&self) -> usize {
858        self.dictionary.offset()
859    }
860
861    fn nulls(&self) -> Option<&NullBuffer> {
862        self.dictionary.nulls()
863    }
864
865    fn logical_nulls(&self) -> Option<NullBuffer> {
866        self.dictionary.logical_nulls()
867    }
868
869    fn logical_null_count(&self) -> usize {
870        self.dictionary.logical_null_count()
871    }
872
873    fn is_nullable(&self) -> bool {
874        self.dictionary.is_nullable()
875    }
876
877    fn get_buffer_memory_size(&self) -> usize {
878        self.dictionary.get_buffer_memory_size()
879    }
880
881    fn get_array_memory_size(&self) -> usize {
882        self.dictionary.get_array_memory_size()
883    }
884}
885
886impl<K, V> IntoIterator for TypedDictionaryArray<'_, K, V>
887where
888    K: ArrowDictionaryKeyType,
889    Self: ArrayAccessor,
890{
891    type Item = Option<<Self as ArrayAccessor>::Item>;
892    type IntoIter = ArrayIter<Self>;
893
894    fn into_iter(self) -> Self::IntoIter {
895        ArrayIter::new(self)
896    }
897}
898
899impl<'a, K, V> ArrayAccessor for TypedDictionaryArray<'a, K, V>
900where
901    K: ArrowDictionaryKeyType,
902    V: Sync + Send,
903    &'a V: ArrayAccessor,
904    <&'a V as ArrayAccessor>::Item: Default,
905{
906    type Item = <&'a V as ArrayAccessor>::Item;
907
908    fn value(&self, index: usize) -> Self::Item {
909        assert!(
910            index < self.len(),
911            "Trying to access an element at index {} from a TypedDictionaryArray of length {}",
912            index,
913            self.len()
914        );
915        unsafe { self.value_unchecked(index) }
916    }
917
918    unsafe fn value_unchecked(&self, index: usize) -> Self::Item {
919        let val = self.dictionary.keys.value_unchecked(index);
920        let value_idx = val.as_usize();
921
922        // As dictionary keys are only verified for non-null indexes
923        // we must check the value is within bounds
924        match value_idx < self.values.len() {
925            true => self.values.value_unchecked(value_idx),
926            false => Default::default(),
927        }
928    }
929}
930
931/// A [`DictionaryArray`] with the key type erased
932///
933/// This can be used to efficiently implement kernels for all possible dictionary
934/// keys without needing to create specialized implementations for each key type
935///
936/// For example
937///
938/// ```
939/// # use arrow_array::*;
940/// # use arrow_array::cast::AsArray;
941/// # use arrow_array::builder::PrimitiveDictionaryBuilder;
942/// # use arrow_array::types::*;
943/// # use arrow_schema::ArrowError;
944/// # use std::sync::Arc;
945///
946/// fn to_string(a: &dyn Array) -> Result<ArrayRef, ArrowError> {
947///     if let Some(d) = a.as_any_dictionary_opt() {
948///         // Recursively handle dictionary input
949///         let r = to_string(d.values().as_ref())?;
950///         return Ok(d.with_values(r));
951///     }
952///     downcast_primitive_array! {
953///         a => Ok(Arc::new(a.iter().map(|x| x.map(|x| format!("{x:?}"))).collect::<StringArray>())),
954///         d => Err(ArrowError::InvalidArgumentError(format!("{d:?} not supported")))
955///     }
956/// }
957///
958/// let result = to_string(&Int32Array::from(vec![1, 2, 3])).unwrap();
959/// let actual = result.as_string::<i32>().iter().map(Option::unwrap).collect::<Vec<_>>();
960/// assert_eq!(actual, &["1", "2", "3"]);
961///
962/// let mut dict = PrimitiveDictionaryBuilder::<Int32Type, UInt16Type>::new();
963/// dict.extend([Some(1), Some(1), Some(2), Some(3), Some(2)]);
964/// let dict = dict.finish();
965///
966/// let r = to_string(&dict).unwrap();
967/// let r = r.as_dictionary::<Int32Type>().downcast_dict::<StringArray>().unwrap();
968/// assert_eq!(r.keys(), dict.keys()); // Keys are the same
969///
970/// let actual = r.into_iter().map(Option::unwrap).collect::<Vec<_>>();
971/// assert_eq!(actual, &["1", "1", "2", "3", "2"]);
972/// ```
973///
974/// See [`AsArray::as_any_dictionary_opt`] and [`AsArray::as_any_dictionary`]
975pub trait AnyDictionaryArray: Array {
976    /// Returns the primitive keys of this dictionary as an [`Array`]
977    fn keys(&self) -> &dyn Array;
978
979    /// Returns the values of this dictionary
980    fn values(&self) -> &ArrayRef;
981
982    /// Returns the keys of this dictionary as usize
983    ///
984    /// The values for nulls will be arbitrary, but are guaranteed
985    /// to be in the range `0..self.values.len()`
986    ///
987    /// # Panic
988    ///
989    /// Panics if `values.len() == 0`
990    fn normalized_keys(&self) -> Vec<usize>;
991
992    /// Create a new [`DictionaryArray`] replacing `values` with the new values
993    ///
994    /// See [`DictionaryArray::with_values`]
995    fn with_values(&self, values: ArrayRef) -> ArrayRef;
996}
997
998impl<K: ArrowDictionaryKeyType> AnyDictionaryArray for DictionaryArray<K> {
999    fn keys(&self) -> &dyn Array {
1000        &self.keys
1001    }
1002
1003    fn values(&self) -> &ArrayRef {
1004        self.values()
1005    }
1006
1007    fn normalized_keys(&self) -> Vec<usize> {
1008        let v_len = self.values().len();
1009        assert_ne!(v_len, 0);
1010        let iter = self.keys().values().iter();
1011        iter.map(|x| x.as_usize().min(v_len - 1)).collect()
1012    }
1013
1014    fn with_values(&self, values: ArrayRef) -> ArrayRef {
1015        Arc::new(self.with_values(values))
1016    }
1017}
1018
1019#[cfg(test)]
1020mod tests {
1021    use super::*;
1022    use crate::cast::as_dictionary_array;
1023    use crate::{Int16Array, Int32Array, Int8Array};
1024    use arrow_buffer::{Buffer, ToByteSlice};
1025
1026    #[test]
1027    fn test_dictionary_array() {
1028        // Construct a value array
1029        let value_data = ArrayData::builder(DataType::Int8)
1030            .len(8)
1031            .add_buffer(Buffer::from(
1032                [10_i8, 11, 12, 13, 14, 15, 16, 17].to_byte_slice(),
1033            ))
1034            .build()
1035            .unwrap();
1036
1037        // Construct a buffer for value offsets, for the nested array:
1038        let keys = Buffer::from([2_i16, 3, 4].to_byte_slice());
1039
1040        // Construct a dictionary array from the above two
1041        let key_type = DataType::Int16;
1042        let value_type = DataType::Int8;
1043        let dict_data_type = DataType::Dictionary(Box::new(key_type), Box::new(value_type));
1044        let dict_data = ArrayData::builder(dict_data_type.clone())
1045            .len(3)
1046            .add_buffer(keys.clone())
1047            .add_child_data(value_data.clone())
1048            .build()
1049            .unwrap();
1050        let dict_array = Int16DictionaryArray::from(dict_data);
1051
1052        let values = dict_array.values();
1053        assert_eq!(value_data, values.to_data());
1054        assert_eq!(DataType::Int8, dict_array.value_type());
1055        assert_eq!(3, dict_array.len());
1056
1057        // Null count only makes sense in terms of the component arrays.
1058        assert_eq!(0, dict_array.null_count());
1059        assert_eq!(0, dict_array.values().null_count());
1060        assert_eq!(dict_array.keys(), &Int16Array::from(vec![2_i16, 3, 4]));
1061
1062        // Now test with a non-zero offset
1063        let dict_data = ArrayData::builder(dict_data_type)
1064            .len(2)
1065            .offset(1)
1066            .add_buffer(keys)
1067            .add_child_data(value_data.clone())
1068            .build()
1069            .unwrap();
1070        let dict_array = Int16DictionaryArray::from(dict_data);
1071
1072        let values = dict_array.values();
1073        assert_eq!(value_data, values.to_data());
1074        assert_eq!(DataType::Int8, dict_array.value_type());
1075        assert_eq!(2, dict_array.len());
1076        assert_eq!(dict_array.keys(), &Int16Array::from(vec![3_i16, 4]));
1077    }
1078
1079    #[test]
1080    fn test_dictionary_builder_append_many() {
1081        let mut builder = PrimitiveDictionaryBuilder::<UInt8Type, UInt32Type>::new();
1082
1083        builder.append(1).unwrap();
1084        builder.append_n(2, 2).unwrap();
1085        builder.append_options(None, 2);
1086        builder.append_options(Some(3), 3);
1087
1088        let array = builder.finish();
1089
1090        let values = array
1091            .values()
1092            .as_primitive::<UInt32Type>()
1093            .iter()
1094            .map(Option::unwrap)
1095            .collect::<Vec<_>>();
1096        assert_eq!(values, &[1, 2, 3]);
1097        let keys = array.keys().iter().collect::<Vec<_>>();
1098        assert_eq!(
1099            keys,
1100            &[
1101                Some(0),
1102                Some(1),
1103                Some(1),
1104                None,
1105                None,
1106                Some(2),
1107                Some(2),
1108                Some(2)
1109            ]
1110        );
1111    }
1112
1113    #[test]
1114    fn test_string_dictionary_builder_append_many() {
1115        let mut builder = StringDictionaryBuilder::<Int8Type>::new();
1116
1117        builder.append("a").unwrap();
1118        builder.append_n("b", 2).unwrap();
1119        builder.append_options(None::<&str>, 2);
1120        builder.append_options(Some("c"), 3);
1121
1122        let array = builder.finish();
1123
1124        let values = array
1125            .values()
1126            .as_string::<i32>()
1127            .iter()
1128            .map(Option::unwrap)
1129            .collect::<Vec<_>>();
1130        assert_eq!(values, &["a", "b", "c"]);
1131        let keys = array.keys().iter().collect::<Vec<_>>();
1132        assert_eq!(
1133            keys,
1134            &[
1135                Some(0),
1136                Some(1),
1137                Some(1),
1138                None,
1139                None,
1140                Some(2),
1141                Some(2),
1142                Some(2)
1143            ]
1144        );
1145    }
1146
1147    #[test]
1148    fn test_dictionary_array_fmt_debug() {
1149        let mut builder = PrimitiveDictionaryBuilder::<UInt8Type, UInt32Type>::with_capacity(3, 2);
1150        builder.append(12345678).unwrap();
1151        builder.append_null();
1152        builder.append(22345678).unwrap();
1153        let array = builder.finish();
1154        assert_eq!(
1155            "DictionaryArray {keys: PrimitiveArray<UInt8>\n[\n  0,\n  null,\n  1,\n] values: PrimitiveArray<UInt32>\n[\n  12345678,\n  22345678,\n]}\n",
1156            format!("{array:?}")
1157        );
1158
1159        let mut builder = PrimitiveDictionaryBuilder::<UInt8Type, UInt32Type>::with_capacity(20, 2);
1160        for _ in 0..20 {
1161            builder.append(1).unwrap();
1162        }
1163        let array = builder.finish();
1164        assert_eq!(
1165            "DictionaryArray {keys: PrimitiveArray<UInt8>\n[\n  0,\n  0,\n  0,\n  0,\n  0,\n  0,\n  0,\n  0,\n  0,\n  0,\n  0,\n  0,\n  0,\n  0,\n  0,\n  0,\n  0,\n  0,\n  0,\n  0,\n] values: PrimitiveArray<UInt32>\n[\n  1,\n]}\n",
1166            format!("{array:?}")
1167        );
1168    }
1169
1170    #[test]
1171    fn test_dictionary_array_from_iter() {
1172        let test = vec!["a", "a", "b", "c"];
1173        let array: DictionaryArray<Int8Type> = test
1174            .iter()
1175            .map(|&x| if x == "b" { None } else { Some(x) })
1176            .collect();
1177        assert_eq!(
1178            "DictionaryArray {keys: PrimitiveArray<Int8>\n[\n  0,\n  0,\n  null,\n  1,\n] values: StringArray\n[\n  \"a\",\n  \"c\",\n]}\n",
1179            format!("{array:?}")
1180        );
1181
1182        let array: DictionaryArray<Int8Type> = test.into_iter().collect();
1183        assert_eq!(
1184            "DictionaryArray {keys: PrimitiveArray<Int8>\n[\n  0,\n  0,\n  1,\n  2,\n] values: StringArray\n[\n  \"a\",\n  \"b\",\n  \"c\",\n]}\n",
1185            format!("{array:?}")
1186        );
1187    }
1188
1189    #[test]
1190    fn test_dictionary_array_reverse_lookup_key() {
1191        let test = vec!["a", "a", "b", "c"];
1192        let array: DictionaryArray<Int8Type> = test.into_iter().collect();
1193
1194        assert_eq!(array.lookup_key("c"), Some(2));
1195
1196        // Direction of building a dictionary is the iterator direction
1197        let test = vec!["t3", "t3", "t2", "t2", "t1", "t3", "t4", "t1", "t0"];
1198        let array: DictionaryArray<Int8Type> = test.into_iter().collect();
1199
1200        assert_eq!(array.lookup_key("t1"), Some(2));
1201        assert_eq!(array.lookup_key("non-existent"), None);
1202    }
1203
1204    #[test]
1205    fn test_dictionary_keys_as_primitive_array() {
1206        let test = vec!["a", "b", "c", "a"];
1207        let array: DictionaryArray<Int8Type> = test.into_iter().collect();
1208
1209        let keys = array.keys();
1210        assert_eq!(&DataType::Int8, keys.data_type());
1211        assert_eq!(0, keys.null_count());
1212        assert_eq!(&[0, 1, 2, 0], keys.values());
1213    }
1214
1215    #[test]
1216    fn test_dictionary_keys_as_primitive_array_with_null() {
1217        let test = vec![Some("a"), None, Some("b"), None, None, Some("a")];
1218        let array: DictionaryArray<Int32Type> = test.into_iter().collect();
1219
1220        let keys = array.keys();
1221        assert_eq!(&DataType::Int32, keys.data_type());
1222        assert_eq!(3, keys.null_count());
1223
1224        assert!(keys.is_valid(0));
1225        assert!(!keys.is_valid(1));
1226        assert!(keys.is_valid(2));
1227        assert!(!keys.is_valid(3));
1228        assert!(!keys.is_valid(4));
1229        assert!(keys.is_valid(5));
1230
1231        assert_eq!(0, keys.value(0));
1232        assert_eq!(1, keys.value(2));
1233        assert_eq!(0, keys.value(5));
1234    }
1235
1236    #[test]
1237    fn test_dictionary_all_nulls() {
1238        let test = vec![None, None, None];
1239        let array: DictionaryArray<Int32Type> = test.into_iter().collect();
1240        array
1241            .into_data()
1242            .validate_full()
1243            .expect("All null array has valid array data");
1244    }
1245
1246    #[test]
1247    fn test_dictionary_iter() {
1248        // Construct a value array
1249        let values = Int8Array::from_iter_values([10_i8, 11, 12, 13, 14, 15, 16, 17]);
1250        let keys = Int16Array::from_iter_values([2_i16, 3, 4]);
1251
1252        // Construct a dictionary array from the above two
1253        let dict_array = DictionaryArray::new(keys, Arc::new(values));
1254
1255        let mut key_iter = dict_array.keys_iter();
1256        assert_eq!(2, key_iter.next().unwrap().unwrap());
1257        assert_eq!(3, key_iter.next().unwrap().unwrap());
1258        assert_eq!(4, key_iter.next().unwrap().unwrap());
1259        assert!(key_iter.next().is_none());
1260
1261        let mut iter = dict_array
1262            .values()
1263            .as_any()
1264            .downcast_ref::<Int8Array>()
1265            .unwrap()
1266            .take_iter(dict_array.keys_iter());
1267
1268        assert_eq!(12, iter.next().unwrap().unwrap());
1269        assert_eq!(13, iter.next().unwrap().unwrap());
1270        assert_eq!(14, iter.next().unwrap().unwrap());
1271        assert!(iter.next().is_none());
1272    }
1273
1274    #[test]
1275    fn test_dictionary_iter_with_null() {
1276        let test = vec![Some("a"), None, Some("b"), None, None, Some("a")];
1277        let array: DictionaryArray<Int32Type> = test.into_iter().collect();
1278
1279        let mut iter = array
1280            .values()
1281            .as_any()
1282            .downcast_ref::<StringArray>()
1283            .unwrap()
1284            .take_iter(array.keys_iter());
1285
1286        assert_eq!("a", iter.next().unwrap().unwrap());
1287        assert!(iter.next().unwrap().is_none());
1288        assert_eq!("b", iter.next().unwrap().unwrap());
1289        assert!(iter.next().unwrap().is_none());
1290        assert!(iter.next().unwrap().is_none());
1291        assert_eq!("a", iter.next().unwrap().unwrap());
1292        assert!(iter.next().is_none());
1293    }
1294
1295    #[test]
1296    fn test_dictionary_key() {
1297        let keys = Int8Array::from(vec![Some(2), None, Some(1)]);
1298        let values = StringArray::from(vec!["foo", "bar", "baz", "blarg"]);
1299
1300        let array = DictionaryArray::new(keys, Arc::new(values));
1301        assert_eq!(array.key(0), Some(2));
1302        assert_eq!(array.key(1), None);
1303        assert_eq!(array.key(2), Some(1));
1304    }
1305
1306    #[test]
1307    fn test_try_new() {
1308        let values: StringArray = [Some("foo"), Some("bar"), Some("baz")]
1309            .into_iter()
1310            .collect();
1311        let keys: Int32Array = [Some(0), Some(2), None, Some(1)].into_iter().collect();
1312
1313        let array = DictionaryArray::new(keys, Arc::new(values));
1314        assert_eq!(array.keys().data_type(), &DataType::Int32);
1315        assert_eq!(array.values().data_type(), &DataType::Utf8);
1316
1317        assert_eq!(array.null_count(), 1);
1318        assert_eq!(array.logical_null_count(), 1);
1319
1320        assert!(array.keys().is_valid(0));
1321        assert!(array.keys().is_valid(1));
1322        assert!(array.keys().is_null(2));
1323        assert!(array.keys().is_valid(3));
1324
1325        assert_eq!(array.keys().value(0), 0);
1326        assert_eq!(array.keys().value(1), 2);
1327        assert_eq!(array.keys().value(3), 1);
1328
1329        assert_eq!(
1330            "DictionaryArray {keys: PrimitiveArray<Int32>\n[\n  0,\n  2,\n  null,\n  1,\n] values: StringArray\n[\n  \"foo\",\n  \"bar\",\n  \"baz\",\n]}\n",
1331            format!("{array:?}")
1332        );
1333    }
1334
1335    #[test]
1336    #[should_panic(expected = "Invalid dictionary key 3 at index 1, expected 0 <= key < 2")]
1337    fn test_try_new_index_too_large() {
1338        let values: StringArray = [Some("foo"), Some("bar")].into_iter().collect();
1339        // dictionary only has 2 values, so offset 3 is out of bounds
1340        let keys: Int32Array = [Some(0), Some(3)].into_iter().collect();
1341        DictionaryArray::new(keys, Arc::new(values));
1342    }
1343
1344    #[test]
1345    #[should_panic(expected = "Invalid dictionary key -100 at index 0, expected 0 <= key < 2")]
1346    fn test_try_new_index_too_small() {
1347        let values: StringArray = [Some("foo"), Some("bar")].into_iter().collect();
1348        let keys: Int32Array = [Some(-100)].into_iter().collect();
1349        DictionaryArray::new(keys, Arc::new(values));
1350    }
1351
1352    #[test]
1353    #[should_panic(expected = "DictionaryArray's data type must match, expected Int64 got Int32")]
1354    fn test_from_array_data_validation() {
1355        let a = DictionaryArray::<Int32Type>::from_iter(["32"]);
1356        let _ = DictionaryArray::<Int64Type>::from(a.into_data());
1357    }
1358
1359    #[test]
1360    fn test_into_primitive_dict_builder() {
1361        let values = Int32Array::from_iter_values([10_i32, 12, 15]);
1362        let keys = Int8Array::from_iter_values([1_i8, 0, 2, 0]);
1363
1364        let dict_array = DictionaryArray::new(keys, Arc::new(values));
1365
1366        let boxed: ArrayRef = Arc::new(dict_array);
1367        let col: DictionaryArray<Int8Type> = as_dictionary_array(&boxed).clone();
1368
1369        drop(boxed);
1370
1371        let mut builder = col.into_primitive_dict_builder::<Int32Type>().unwrap();
1372
1373        let slice = builder.values_slice_mut();
1374        assert_eq!(slice, &[10, 12, 15]);
1375
1376        slice[0] = 4;
1377        slice[1] = 2;
1378        slice[2] = 1;
1379
1380        let values = Int32Array::from_iter_values([4_i32, 2, 1]);
1381        let keys = Int8Array::from_iter_values([1_i8, 0, 2, 0]);
1382
1383        let expected = DictionaryArray::new(keys, Arc::new(values));
1384
1385        let new_array = builder.finish();
1386        assert_eq!(expected, new_array);
1387    }
1388
1389    #[test]
1390    fn test_into_primitive_dict_builder_cloned_array() {
1391        let values = Int32Array::from_iter_values([10_i32, 12, 15]);
1392        let keys = Int8Array::from_iter_values([1_i8, 0, 2, 0]);
1393
1394        let dict_array = DictionaryArray::new(keys, Arc::new(values));
1395
1396        let boxed: ArrayRef = Arc::new(dict_array);
1397
1398        let col: DictionaryArray<Int8Type> = DictionaryArray::<Int8Type>::from(boxed.to_data());
1399        let err = col.into_primitive_dict_builder::<Int32Type>();
1400
1401        let returned = err.unwrap_err();
1402
1403        let values = Int32Array::from_iter_values([10_i32, 12, 15]);
1404        let keys = Int8Array::from_iter_values([1_i8, 0, 2, 0]);
1405
1406        let expected = DictionaryArray::new(keys, Arc::new(values));
1407        assert_eq!(expected, returned);
1408    }
1409
1410    #[test]
1411    fn test_occupancy() {
1412        let keys = Int32Array::new((100..200).collect(), None);
1413        let values = Int32Array::from(vec![0; 1024]);
1414        let dict = DictionaryArray::new(keys, Arc::new(values));
1415        for (idx, v) in dict.occupancy().iter().enumerate() {
1416            let expected = (100..200).contains(&idx);
1417            assert_eq!(v, expected, "{idx}");
1418        }
1419
1420        let keys = Int32Array::new(
1421            (0..100).collect(),
1422            Some((0..100).map(|x| x % 4 == 0).collect()),
1423        );
1424        let values = Int32Array::from(vec![0; 1024]);
1425        let dict = DictionaryArray::new(keys, Arc::new(values));
1426        for (idx, v) in dict.occupancy().iter().enumerate() {
1427            let expected = idx % 4 == 0 && idx < 100;
1428            assert_eq!(v, expected, "{idx}");
1429        }
1430    }
1431
1432    #[test]
1433    fn test_iterator_nulls() {
1434        let keys = Int32Array::new(
1435            vec![0, 700, 1, 2].into(),
1436            Some(NullBuffer::from(vec![true, false, true, true])),
1437        );
1438        let values = Int32Array::from(vec![Some(50), None, Some(2)]);
1439        let dict = DictionaryArray::new(keys, Arc::new(values));
1440        let values: Vec<_> = dict
1441            .downcast_dict::<Int32Array>()
1442            .unwrap()
1443            .into_iter()
1444            .collect();
1445        assert_eq!(values, &[Some(50), None, None, Some(2)])
1446    }
1447
1448    #[test]
1449    fn test_normalized_keys() {
1450        let values = vec![132, 0, 1].into();
1451        let nulls = NullBuffer::from(vec![false, true, true]);
1452        let keys = Int32Array::new(values, Some(nulls));
1453        let dictionary = DictionaryArray::new(keys, Arc::new(Int32Array::new_null(2)));
1454        assert_eq!(&dictionary.normalized_keys(), &[1, 0, 1])
1455    }
1456}