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 be 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        if cfg!(feature = "force_validate") {
331            return Self::new(keys, values);
332        }
333
334        let data_type = DataType::Dictionary(
335            Box::new(keys.data_type().clone()),
336            Box::new(values.data_type().clone()),
337        );
338
339        Self {
340            data_type,
341            keys,
342            values,
343            is_ordered: false,
344        }
345    }
346
347    /// Deconstruct this array into its constituent parts
348    pub fn into_parts(self) -> (PrimitiveArray<K>, ArrayRef) {
349        (self.keys, self.values)
350    }
351
352    /// Return an array view of the keys of this dictionary as a PrimitiveArray.
353    pub fn keys(&self) -> &PrimitiveArray<K> {
354        &self.keys
355    }
356
357    /// If `value` is present in `values` (aka the dictionary),
358    /// returns the corresponding key (index into the `values`
359    /// array). Otherwise returns `None`.
360    ///
361    /// Panics if `values` is not a [`StringArray`].
362    pub fn lookup_key(&self, value: &str) -> Option<K::Native> {
363        let rd_buf: &StringArray = self.values.as_any().downcast_ref::<StringArray>().unwrap();
364
365        (0..rd_buf.len())
366            .position(|i| rd_buf.value(i) == value)
367            .and_then(K::Native::from_usize)
368    }
369
370    /// Returns a reference to the dictionary values array
371    pub fn values(&self) -> &ArrayRef {
372        &self.values
373    }
374
375    /// Returns a clone of the value type of this list.
376    pub fn value_type(&self) -> DataType {
377        self.values.data_type().clone()
378    }
379
380    /// The length of the dictionary is the length of the keys array.
381    pub fn len(&self) -> usize {
382        self.keys.len()
383    }
384
385    /// Whether this dictionary is empty
386    pub fn is_empty(&self) -> bool {
387        self.keys.is_empty()
388    }
389
390    /// Currently exists for compatibility purposes with Arrow IPC.
391    pub fn is_ordered(&self) -> bool {
392        self.is_ordered
393    }
394
395    /// Return an iterator over the keys (indexes into the dictionary)
396    pub fn keys_iter(&self) -> impl Iterator<Item = Option<usize>> + '_ {
397        self.keys.iter().map(|key| key.map(|k| k.as_usize()))
398    }
399
400    /// Return the value of `keys` (the dictionary key) at index `i`,
401    /// cast to `usize`, `None` if the value at `i` is `NULL`.
402    pub fn key(&self, i: usize) -> Option<usize> {
403        self.keys.is_valid(i).then(|| self.keys.value(i).as_usize())
404    }
405
406    /// Returns a zero-copy slice of this array with the indicated offset and length.
407    pub fn slice(&self, offset: usize, length: usize) -> Self {
408        Self {
409            data_type: self.data_type.clone(),
410            keys: self.keys.slice(offset, length),
411            values: self.values.clone(),
412            is_ordered: self.is_ordered,
413        }
414    }
415
416    /// Downcast this dictionary to a [`TypedDictionaryArray`]
417    ///
418    /// ```
419    /// use arrow_array::{Array, ArrayAccessor, DictionaryArray, StringArray, types::Int32Type};
420    ///
421    /// let orig = [Some("a"), Some("b"), None];
422    /// let dictionary = DictionaryArray::<Int32Type>::from_iter(orig);
423    /// let typed = dictionary.downcast_dict::<StringArray>().unwrap();
424    /// assert_eq!(typed.value(0), "a");
425    /// assert_eq!(typed.value(1), "b");
426    /// assert!(typed.is_null(2));
427    /// ```
428    ///
429    pub fn downcast_dict<V: 'static>(&self) -> Option<TypedDictionaryArray<'_, K, V>> {
430        let values = self.values.as_any().downcast_ref()?;
431        Some(TypedDictionaryArray {
432            dictionary: self,
433            values,
434        })
435    }
436
437    /// Returns a new dictionary with the same keys as the current instance
438    /// but with a different set of dictionary values
439    ///
440    /// This can be used to perform an operation on the values of a dictionary
441    ///
442    /// # Panics
443    ///
444    /// Panics if `values` has a length less than the current values
445    ///
446    /// ```
447    /// # use std::sync::Arc;
448    /// # use arrow_array::builder::PrimitiveDictionaryBuilder;
449    /// # use arrow_array::{Int8Array, Int64Array, ArrayAccessor};
450    /// # use arrow_array::types::{Int32Type, Int8Type};
451    ///
452    /// // Construct a Dict(Int32, Int8)
453    /// let mut builder = PrimitiveDictionaryBuilder::<Int32Type, Int8Type>::with_capacity(2, 200);
454    /// for i in 0..100 {
455    ///     builder.append(i % 2).unwrap();
456    /// }
457    ///
458    /// let dictionary = builder.finish();
459    ///
460    /// // Perform a widening cast of dictionary values
461    /// let typed_dictionary = dictionary.downcast_dict::<Int8Array>().unwrap();
462    /// let values: Int64Array = typed_dictionary.values().unary(|x| x as i64);
463    ///
464    /// // Create a Dict(Int32,
465    /// let new = dictionary.with_values(Arc::new(values));
466    ///
467    /// // Verify values are as expected
468    /// let new_typed = new.downcast_dict::<Int64Array>().unwrap();
469    /// for i in 0..100 {
470    ///     assert_eq!(new_typed.value(i), (i % 2) as i64)
471    /// }
472    /// ```
473    ///
474    pub fn with_values(&self, values: ArrayRef) -> Self {
475        assert!(values.len() >= self.values.len());
476        let data_type =
477            DataType::Dictionary(Box::new(K::DATA_TYPE), Box::new(values.data_type().clone()));
478        Self {
479            data_type,
480            keys: self.keys.clone(),
481            values,
482            is_ordered: false,
483        }
484    }
485
486    /// Returns `PrimitiveDictionaryBuilder` of this dictionary array for mutating
487    /// its keys and values if the underlying data buffer is not shared by others.
488    pub fn into_primitive_dict_builder<V>(self) -> Result<PrimitiveDictionaryBuilder<K, V>, Self>
489    where
490        V: ArrowPrimitiveType,
491    {
492        if !self.value_type().is_primitive() {
493            return Err(self);
494        }
495
496        let key_array = self.keys().clone();
497        let value_array = self.values().as_primitive::<V>().clone();
498
499        drop(self.keys);
500        drop(self.values);
501
502        let key_builder = key_array.into_builder();
503        let value_builder = value_array.into_builder();
504
505        match (key_builder, value_builder) {
506            (Ok(key_builder), Ok(value_builder)) => Ok(unsafe {
507                PrimitiveDictionaryBuilder::new_from_builders(key_builder, value_builder)
508            }),
509            (Err(key_array), Ok(mut value_builder)) => {
510                Err(Self::try_new(key_array, Arc::new(value_builder.finish())).unwrap())
511            }
512            (Ok(mut key_builder), Err(value_array)) => {
513                Err(Self::try_new(key_builder.finish(), Arc::new(value_array)).unwrap())
514            }
515            (Err(key_array), Err(value_array)) => {
516                Err(Self::try_new(key_array, Arc::new(value_array)).unwrap())
517            }
518        }
519    }
520
521    /// Applies an unary and infallible function to a mutable dictionary array.
522    /// Mutable dictionary array means that the buffers are not shared with other arrays.
523    /// As a result, this mutates the buffers directly without allocating new buffers.
524    ///
525    /// # Implementation
526    ///
527    /// This will apply the function for all dictionary values, including those on null slots.
528    /// This implies that the operation must be infallible for any value of the corresponding type
529    /// or this function may panic.
530    /// # Example
531    /// ```
532    /// # use std::sync::Arc;
533    /// # use arrow_array::{Array, ArrayAccessor, DictionaryArray, StringArray, types::{Int8Type, Int32Type}};
534    /// # use arrow_array::{Int8Array, Int32Array};
535    /// let values = Int32Array::from(vec![Some(10), Some(20), None]);
536    /// let keys = Int8Array::from_iter_values([0, 0, 1, 2]);
537    /// let dictionary = DictionaryArray::<Int8Type>::try_new(keys, Arc::new(values)).unwrap();
538    /// let c = dictionary.unary_mut::<_, Int32Type>(|x| x + 1).unwrap();
539    /// let typed = c.downcast_dict::<Int32Array>().unwrap();
540    /// assert_eq!(typed.value(0), 11);
541    /// assert_eq!(typed.value(1), 11);
542    /// assert_eq!(typed.value(2), 21);
543    /// ```
544    pub fn unary_mut<F, V>(self, op: F) -> Result<DictionaryArray<K>, DictionaryArray<K>>
545    where
546        V: ArrowPrimitiveType,
547        F: Fn(V::Native) -> V::Native,
548    {
549        let mut builder: PrimitiveDictionaryBuilder<K, V> = self.into_primitive_dict_builder()?;
550        builder
551            .values_slice_mut()
552            .iter_mut()
553            .for_each(|v| *v = op(*v));
554        Ok(builder.finish())
555    }
556
557    /// Computes an occupancy mask for this dictionary's values
558    ///
559    /// For each value in [`Self::values`] the corresponding bit will be set in the
560    /// returned mask if it is referenced by a key in this [`DictionaryArray`]
561    pub fn occupancy(&self) -> BooleanBuffer {
562        let len = self.values.len();
563        let mut builder = BooleanBufferBuilder::new(len);
564        builder.resize(len);
565        let slice = builder.as_slice_mut();
566        match self.keys.nulls().filter(|n| n.null_count() > 0) {
567            Some(n) => {
568                let v = self.keys.values();
569                n.valid_indices()
570                    .for_each(|idx| set_bit(slice, v[idx].as_usize()))
571            }
572            None => {
573                let v = self.keys.values();
574                v.iter().for_each(|v| set_bit(slice, v.as_usize()))
575            }
576        }
577        builder.finish()
578    }
579}
580
581/// Constructs a `DictionaryArray` from an array data reference.
582impl<T: ArrowDictionaryKeyType> From<ArrayData> for DictionaryArray<T> {
583    fn from(data: ArrayData) -> Self {
584        assert_eq!(
585            data.buffers().len(),
586            1,
587            "DictionaryArray data should contain a single buffer only (keys)."
588        );
589        assert_eq!(
590            data.child_data().len(),
591            1,
592            "DictionaryArray should contain a single child array (values)."
593        );
594
595        if let DataType::Dictionary(key_data_type, _) = data.data_type() {
596            assert_eq!(
597                &T::DATA_TYPE,
598                key_data_type.as_ref(),
599                "DictionaryArray's data type must match, expected {} got {}",
600                T::DATA_TYPE,
601                key_data_type
602            );
603
604            let values = make_array(data.child_data()[0].clone());
605            let data_type = data.data_type().clone();
606
607            // create a zero-copy of the keys' data
608            // SAFETY:
609            // ArrayData is valid and verified type above
610
611            let keys = PrimitiveArray::<T>::from(unsafe {
612                data.into_builder()
613                    .data_type(T::DATA_TYPE)
614                    .child_data(vec![])
615                    .build_unchecked()
616            });
617
618            Self {
619                data_type,
620                keys,
621                values,
622                is_ordered: false,
623            }
624        } else {
625            panic!("DictionaryArray must have Dictionary data type.")
626        }
627    }
628}
629
630impl<T: ArrowDictionaryKeyType> From<DictionaryArray<T>> for ArrayData {
631    fn from(array: DictionaryArray<T>) -> Self {
632        let builder = array
633            .keys
634            .into_data()
635            .into_builder()
636            .data_type(array.data_type)
637            .child_data(vec![array.values.to_data()]);
638
639        unsafe { builder.build_unchecked() }
640    }
641}
642
643/// Constructs a `DictionaryArray` from an iterator of optional strings.
644///
645/// # Example:
646/// ```
647/// use arrow_array::{DictionaryArray, PrimitiveArray, StringArray, types::Int8Type};
648///
649/// let test = vec!["a", "a", "b", "c"];
650/// let array: DictionaryArray<Int8Type> = test
651///     .iter()
652///     .map(|&x| if x == "b" { None } else { Some(x) })
653///     .collect();
654/// assert_eq!(
655///     "DictionaryArray {keys: PrimitiveArray<Int8>\n[\n  0,\n  0,\n  null,\n  1,\n] values: StringArray\n[\n  \"a\",\n  \"c\",\n]}\n",
656///     format!("{:?}", array)
657/// );
658/// ```
659impl<'a, T: ArrowDictionaryKeyType> FromIterator<Option<&'a str>> for DictionaryArray<T> {
660    fn from_iter<I: IntoIterator<Item = Option<&'a str>>>(iter: I) -> Self {
661        let it = iter.into_iter();
662        let (lower, _) = it.size_hint();
663        let mut builder = StringDictionaryBuilder::with_capacity(lower, 256, 1024);
664        builder.extend(it);
665        builder.finish()
666    }
667}
668
669/// Constructs a `DictionaryArray` from an iterator of strings.
670///
671/// # Example:
672///
673/// ```
674/// use arrow_array::{DictionaryArray, PrimitiveArray, StringArray, types::Int8Type};
675///
676/// let test = vec!["a", "a", "b", "c"];
677/// let array: DictionaryArray<Int8Type> = test.into_iter().collect();
678/// assert_eq!(
679///     "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",
680///     format!("{:?}", array)
681/// );
682/// ```
683impl<'a, T: ArrowDictionaryKeyType> FromIterator<&'a str> for DictionaryArray<T> {
684    fn from_iter<I: IntoIterator<Item = &'a str>>(iter: I) -> Self {
685        let it = iter.into_iter();
686        let (lower, _) = it.size_hint();
687        let mut builder = StringDictionaryBuilder::with_capacity(lower, 256, 1024);
688        it.for_each(|i| {
689            builder
690                .append(i)
691                .expect("Unable to append a value to a dictionary array.");
692        });
693
694        builder.finish()
695    }
696}
697
698impl<T: ArrowDictionaryKeyType> Array for DictionaryArray<T> {
699    fn as_any(&self) -> &dyn Any {
700        self
701    }
702
703    fn to_data(&self) -> ArrayData {
704        self.clone().into()
705    }
706
707    fn into_data(self) -> ArrayData {
708        self.into()
709    }
710
711    fn data_type(&self) -> &DataType {
712        &self.data_type
713    }
714
715    fn slice(&self, offset: usize, length: usize) -> ArrayRef {
716        Arc::new(self.slice(offset, length))
717    }
718
719    fn len(&self) -> usize {
720        self.keys.len()
721    }
722
723    fn is_empty(&self) -> bool {
724        self.keys.is_empty()
725    }
726
727    fn shrink_to_fit(&mut self) {
728        self.keys.shrink_to_fit();
729        self.values.shrink_to_fit();
730    }
731
732    fn offset(&self) -> usize {
733        self.keys.offset()
734    }
735
736    fn nulls(&self) -> Option<&NullBuffer> {
737        self.keys.nulls()
738    }
739
740    fn logical_nulls(&self) -> Option<NullBuffer> {
741        match self.values.logical_nulls() {
742            None => self.nulls().cloned(),
743            Some(value_nulls) => {
744                let mut builder = BooleanBufferBuilder::new(self.len());
745                match self.keys.nulls() {
746                    Some(n) => builder.append_buffer(n.inner()),
747                    None => builder.append_n(self.len(), true),
748                }
749                for (idx, k) in self.keys.values().iter().enumerate() {
750                    let k = k.as_usize();
751                    // Check range to allow for nulls
752                    if k < value_nulls.len() && value_nulls.is_null(k) {
753                        builder.set_bit(idx, false);
754                    }
755                }
756                Some(builder.finish().into())
757            }
758        }
759    }
760
761    fn logical_null_count(&self) -> usize {
762        match (self.keys.nulls(), self.values.logical_nulls()) {
763            (None, None) => 0,
764            (Some(key_nulls), None) => key_nulls.null_count(),
765            (None, Some(value_nulls)) => self
766                .keys
767                .values()
768                .iter()
769                .filter(|k| value_nulls.is_null(k.as_usize()))
770                .count(),
771            (Some(key_nulls), Some(value_nulls)) => self
772                .keys
773                .values()
774                .iter()
775                .enumerate()
776                .filter(|(idx, k)| key_nulls.is_null(*idx) || value_nulls.is_null(k.as_usize()))
777                .count(),
778        }
779    }
780
781    fn is_nullable(&self) -> bool {
782        !self.is_empty() && (self.nulls().is_some() || self.values.is_nullable())
783    }
784
785    fn get_buffer_memory_size(&self) -> usize {
786        self.keys.get_buffer_memory_size() + self.values.get_buffer_memory_size()
787    }
788
789    fn get_array_memory_size(&self) -> usize {
790        std::mem::size_of::<Self>()
791            + self.keys.get_buffer_memory_size()
792            + self.values.get_array_memory_size()
793    }
794}
795
796impl<T: ArrowDictionaryKeyType> std::fmt::Debug for DictionaryArray<T> {
797    fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
798        writeln!(
799            f,
800            "DictionaryArray {{keys: {:?} values: {:?}}}",
801            self.keys, self.values
802        )
803    }
804}
805
806/// A [`DictionaryArray`] typed on its child values array
807///
808/// Implements [`ArrayAccessor`] allowing fast access to its elements
809///
810/// ```
811/// use arrow_array::{DictionaryArray, StringArray, types::Int32Type};
812///
813/// let orig = ["a", "b", "a", "b"];
814/// let dictionary = DictionaryArray::<Int32Type>::from_iter(orig);
815///
816/// // `TypedDictionaryArray` allows you to access the values directly
817/// let typed = dictionary.downcast_dict::<StringArray>().unwrap();
818///
819/// for (maybe_val, orig) in typed.into_iter().zip(orig) {
820///     assert_eq!(maybe_val.unwrap(), orig)
821/// }
822/// ```
823pub struct TypedDictionaryArray<'a, K: ArrowDictionaryKeyType, V> {
824    /// The dictionary array
825    dictionary: &'a DictionaryArray<K>,
826    /// The values of the dictionary
827    values: &'a V,
828}
829
830// Manually implement `Clone` to avoid `V: Clone` type constraint
831impl<K: ArrowDictionaryKeyType, V> Clone for TypedDictionaryArray<'_, K, V> {
832    fn clone(&self) -> Self {
833        *self
834    }
835}
836
837impl<K: ArrowDictionaryKeyType, V> Copy for TypedDictionaryArray<'_, K, V> {}
838
839impl<K: ArrowDictionaryKeyType, V> std::fmt::Debug for TypedDictionaryArray<'_, K, V> {
840    fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
841        writeln!(f, "TypedDictionaryArray({:?})", self.dictionary)
842    }
843}
844
845impl<'a, K: ArrowDictionaryKeyType, V> TypedDictionaryArray<'a, K, V> {
846    /// Returns the keys of this [`TypedDictionaryArray`]
847    pub fn keys(&self) -> &'a PrimitiveArray<K> {
848        self.dictionary.keys()
849    }
850
851    /// Returns the values of this [`TypedDictionaryArray`]
852    pub fn values(&self) -> &'a V {
853        self.values
854    }
855}
856
857impl<K: ArrowDictionaryKeyType, V: Sync> Array for TypedDictionaryArray<'_, K, V> {
858    fn as_any(&self) -> &dyn Any {
859        self.dictionary
860    }
861
862    fn to_data(&self) -> ArrayData {
863        self.dictionary.to_data()
864    }
865
866    fn into_data(self) -> ArrayData {
867        self.dictionary.into_data()
868    }
869
870    fn data_type(&self) -> &DataType {
871        self.dictionary.data_type()
872    }
873
874    fn slice(&self, offset: usize, length: usize) -> ArrayRef {
875        Arc::new(self.dictionary.slice(offset, length))
876    }
877
878    fn len(&self) -> usize {
879        self.dictionary.len()
880    }
881
882    fn is_empty(&self) -> bool {
883        self.dictionary.is_empty()
884    }
885
886    fn offset(&self) -> usize {
887        self.dictionary.offset()
888    }
889
890    fn nulls(&self) -> Option<&NullBuffer> {
891        self.dictionary.nulls()
892    }
893
894    fn logical_nulls(&self) -> Option<NullBuffer> {
895        self.dictionary.logical_nulls()
896    }
897
898    fn logical_null_count(&self) -> usize {
899        self.dictionary.logical_null_count()
900    }
901
902    fn is_nullable(&self) -> bool {
903        self.dictionary.is_nullable()
904    }
905
906    fn get_buffer_memory_size(&self) -> usize {
907        self.dictionary.get_buffer_memory_size()
908    }
909
910    fn get_array_memory_size(&self) -> usize {
911        self.dictionary.get_array_memory_size()
912    }
913}
914
915impl<K, V> IntoIterator for TypedDictionaryArray<'_, K, V>
916where
917    K: ArrowDictionaryKeyType,
918    Self: ArrayAccessor,
919{
920    type Item = Option<<Self as ArrayAccessor>::Item>;
921    type IntoIter = ArrayIter<Self>;
922
923    fn into_iter(self) -> Self::IntoIter {
924        ArrayIter::new(self)
925    }
926}
927
928impl<'a, K, V> ArrayAccessor for TypedDictionaryArray<'a, K, V>
929where
930    K: ArrowDictionaryKeyType,
931    V: Sync + Send,
932    &'a V: ArrayAccessor,
933    <&'a V as ArrayAccessor>::Item: Default,
934{
935    type Item = <&'a V as ArrayAccessor>::Item;
936
937    fn value(&self, index: usize) -> Self::Item {
938        assert!(
939            index < self.len(),
940            "Trying to access an element at index {} from a TypedDictionaryArray of length {}",
941            index,
942            self.len()
943        );
944        unsafe { self.value_unchecked(index) }
945    }
946
947    unsafe fn value_unchecked(&self, index: usize) -> Self::Item {
948        let val = self.dictionary.keys.value_unchecked(index);
949        let value_idx = val.as_usize();
950
951        // As dictionary keys are only verified for non-null indexes
952        // we must check the value is within bounds
953        match value_idx < self.values.len() {
954            true => self.values.value_unchecked(value_idx),
955            false => Default::default(),
956        }
957    }
958}
959
960/// A [`DictionaryArray`] with the key type erased
961///
962/// This can be used to efficiently implement kernels for all possible dictionary
963/// keys without needing to create specialized implementations for each key type
964///
965/// For example
966///
967/// ```
968/// # use arrow_array::*;
969/// # use arrow_array::cast::AsArray;
970/// # use arrow_array::builder::PrimitiveDictionaryBuilder;
971/// # use arrow_array::types::*;
972/// # use arrow_schema::ArrowError;
973/// # use std::sync::Arc;
974///
975/// fn to_string(a: &dyn Array) -> Result<ArrayRef, ArrowError> {
976///     if let Some(d) = a.as_any_dictionary_opt() {
977///         // Recursively handle dictionary input
978///         let r = to_string(d.values().as_ref())?;
979///         return Ok(d.with_values(r));
980///     }
981///     downcast_primitive_array! {
982///         a => Ok(Arc::new(a.iter().map(|x| x.map(|x| format!("{x:?}"))).collect::<StringArray>())),
983///         d => Err(ArrowError::InvalidArgumentError(format!("{d:?} not supported")))
984///     }
985/// }
986///
987/// let result = to_string(&Int32Array::from(vec![1, 2, 3])).unwrap();
988/// let actual = result.as_string::<i32>().iter().map(Option::unwrap).collect::<Vec<_>>();
989/// assert_eq!(actual, &["1", "2", "3"]);
990///
991/// let mut dict = PrimitiveDictionaryBuilder::<Int32Type, UInt16Type>::new();
992/// dict.extend([Some(1), Some(1), Some(2), Some(3), Some(2)]);
993/// let dict = dict.finish();
994///
995/// let r = to_string(&dict).unwrap();
996/// let r = r.as_dictionary::<Int32Type>().downcast_dict::<StringArray>().unwrap();
997/// assert_eq!(r.keys(), dict.keys()); // Keys are the same
998///
999/// let actual = r.into_iter().map(Option::unwrap).collect::<Vec<_>>();
1000/// assert_eq!(actual, &["1", "1", "2", "3", "2"]);
1001/// ```
1002///
1003/// See [`AsArray::as_any_dictionary_opt`] and [`AsArray::as_any_dictionary`]
1004pub trait AnyDictionaryArray: Array {
1005    /// Returns the primitive keys of this dictionary as an [`Array`]
1006    fn keys(&self) -> &dyn Array;
1007
1008    /// Returns the values of this dictionary
1009    fn values(&self) -> &ArrayRef;
1010
1011    /// Returns the keys of this dictionary as usize
1012    ///
1013    /// The values for nulls will be arbitrary, but are guaranteed
1014    /// to be in the range `0..self.values.len()`
1015    ///
1016    /// # Panic
1017    ///
1018    /// Panics if `values.len() == 0`
1019    fn normalized_keys(&self) -> Vec<usize>;
1020
1021    /// Create a new [`DictionaryArray`] replacing `values` with the new values
1022    ///
1023    /// See [`DictionaryArray::with_values`]
1024    fn with_values(&self, values: ArrayRef) -> ArrayRef;
1025}
1026
1027impl<K: ArrowDictionaryKeyType> AnyDictionaryArray for DictionaryArray<K> {
1028    fn keys(&self) -> &dyn Array {
1029        &self.keys
1030    }
1031
1032    fn values(&self) -> &ArrayRef {
1033        self.values()
1034    }
1035
1036    fn normalized_keys(&self) -> Vec<usize> {
1037        let v_len = self.values().len();
1038        assert_ne!(v_len, 0);
1039        let iter = self.keys().values().iter();
1040        iter.map(|x| x.as_usize().min(v_len - 1)).collect()
1041    }
1042
1043    fn with_values(&self, values: ArrayRef) -> ArrayRef {
1044        Arc::new(self.with_values(values))
1045    }
1046}
1047
1048#[cfg(test)]
1049mod tests {
1050    use super::*;
1051    use crate::cast::as_dictionary_array;
1052    use crate::{Int16Array, Int32Array, Int8Array, RunArray};
1053    use arrow_buffer::{Buffer, ToByteSlice};
1054
1055    #[test]
1056    fn test_dictionary_array() {
1057        // Construct a value array
1058        let value_data = ArrayData::builder(DataType::Int8)
1059            .len(8)
1060            .add_buffer(Buffer::from(
1061                [10_i8, 11, 12, 13, 14, 15, 16, 17].to_byte_slice(),
1062            ))
1063            .build()
1064            .unwrap();
1065
1066        // Construct a buffer for value offsets, for the nested array:
1067        let keys = Buffer::from([2_i16, 3, 4].to_byte_slice());
1068
1069        // Construct a dictionary array from the above two
1070        let key_type = DataType::Int16;
1071        let value_type = DataType::Int8;
1072        let dict_data_type = DataType::Dictionary(Box::new(key_type), Box::new(value_type));
1073        let dict_data = ArrayData::builder(dict_data_type.clone())
1074            .len(3)
1075            .add_buffer(keys.clone())
1076            .add_child_data(value_data.clone())
1077            .build()
1078            .unwrap();
1079        let dict_array = Int16DictionaryArray::from(dict_data);
1080
1081        let values = dict_array.values();
1082        assert_eq!(value_data, values.to_data());
1083        assert_eq!(DataType::Int8, dict_array.value_type());
1084        assert_eq!(3, dict_array.len());
1085
1086        // Null count only makes sense in terms of the component arrays.
1087        assert_eq!(0, dict_array.null_count());
1088        assert_eq!(0, dict_array.values().null_count());
1089        assert_eq!(dict_array.keys(), &Int16Array::from(vec![2_i16, 3, 4]));
1090
1091        // Now test with a non-zero offset
1092        let dict_data = ArrayData::builder(dict_data_type)
1093            .len(2)
1094            .offset(1)
1095            .add_buffer(keys)
1096            .add_child_data(value_data.clone())
1097            .build()
1098            .unwrap();
1099        let dict_array = Int16DictionaryArray::from(dict_data);
1100
1101        let values = dict_array.values();
1102        assert_eq!(value_data, values.to_data());
1103        assert_eq!(DataType::Int8, dict_array.value_type());
1104        assert_eq!(2, dict_array.len());
1105        assert_eq!(dict_array.keys(), &Int16Array::from(vec![3_i16, 4]));
1106    }
1107
1108    #[test]
1109    fn test_dictionary_builder_append_many() {
1110        let mut builder = PrimitiveDictionaryBuilder::<UInt8Type, UInt32Type>::new();
1111
1112        builder.append(1).unwrap();
1113        builder.append_n(2, 2).unwrap();
1114        builder.append_options(None, 2);
1115        builder.append_options(Some(3), 3);
1116
1117        let array = builder.finish();
1118
1119        let values = array
1120            .values()
1121            .as_primitive::<UInt32Type>()
1122            .iter()
1123            .map(Option::unwrap)
1124            .collect::<Vec<_>>();
1125        assert_eq!(values, &[1, 2, 3]);
1126        let keys = array.keys().iter().collect::<Vec<_>>();
1127        assert_eq!(
1128            keys,
1129            &[
1130                Some(0),
1131                Some(1),
1132                Some(1),
1133                None,
1134                None,
1135                Some(2),
1136                Some(2),
1137                Some(2)
1138            ]
1139        );
1140    }
1141
1142    #[test]
1143    fn test_string_dictionary_builder_append_many() {
1144        let mut builder = StringDictionaryBuilder::<Int8Type>::new();
1145
1146        builder.append("a").unwrap();
1147        builder.append_n("b", 2).unwrap();
1148        builder.append_options(None::<&str>, 2);
1149        builder.append_options(Some("c"), 3);
1150
1151        let array = builder.finish();
1152
1153        let values = array
1154            .values()
1155            .as_string::<i32>()
1156            .iter()
1157            .map(Option::unwrap)
1158            .collect::<Vec<_>>();
1159        assert_eq!(values, &["a", "b", "c"]);
1160        let keys = array.keys().iter().collect::<Vec<_>>();
1161        assert_eq!(
1162            keys,
1163            &[
1164                Some(0),
1165                Some(1),
1166                Some(1),
1167                None,
1168                None,
1169                Some(2),
1170                Some(2),
1171                Some(2)
1172            ]
1173        );
1174    }
1175
1176    #[test]
1177    fn test_dictionary_array_fmt_debug() {
1178        let mut builder = PrimitiveDictionaryBuilder::<UInt8Type, UInt32Type>::with_capacity(3, 2);
1179        builder.append(12345678).unwrap();
1180        builder.append_null();
1181        builder.append(22345678).unwrap();
1182        let array = builder.finish();
1183        assert_eq!(
1184            "DictionaryArray {keys: PrimitiveArray<UInt8>\n[\n  0,\n  null,\n  1,\n] values: PrimitiveArray<UInt32>\n[\n  12345678,\n  22345678,\n]}\n",
1185            format!("{array:?}")
1186        );
1187
1188        let mut builder = PrimitiveDictionaryBuilder::<UInt8Type, UInt32Type>::with_capacity(20, 2);
1189        for _ in 0..20 {
1190            builder.append(1).unwrap();
1191        }
1192        let array = builder.finish();
1193        assert_eq!(
1194            "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",
1195            format!("{array:?}")
1196        );
1197    }
1198
1199    #[test]
1200    fn test_dictionary_array_from_iter() {
1201        let test = vec!["a", "a", "b", "c"];
1202        let array: DictionaryArray<Int8Type> = test
1203            .iter()
1204            .map(|&x| if x == "b" { None } else { Some(x) })
1205            .collect();
1206        assert_eq!(
1207            "DictionaryArray {keys: PrimitiveArray<Int8>\n[\n  0,\n  0,\n  null,\n  1,\n] values: StringArray\n[\n  \"a\",\n  \"c\",\n]}\n",
1208            format!("{array:?}")
1209        );
1210
1211        let array: DictionaryArray<Int8Type> = test.into_iter().collect();
1212        assert_eq!(
1213            "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",
1214            format!("{array:?}")
1215        );
1216    }
1217
1218    #[test]
1219    fn test_dictionary_array_reverse_lookup_key() {
1220        let test = vec!["a", "a", "b", "c"];
1221        let array: DictionaryArray<Int8Type> = test.into_iter().collect();
1222
1223        assert_eq!(array.lookup_key("c"), Some(2));
1224
1225        // Direction of building a dictionary is the iterator direction
1226        let test = vec!["t3", "t3", "t2", "t2", "t1", "t3", "t4", "t1", "t0"];
1227        let array: DictionaryArray<Int8Type> = test.into_iter().collect();
1228
1229        assert_eq!(array.lookup_key("t1"), Some(2));
1230        assert_eq!(array.lookup_key("non-existent"), None);
1231    }
1232
1233    #[test]
1234    fn test_dictionary_keys_as_primitive_array() {
1235        let test = vec!["a", "b", "c", "a"];
1236        let array: DictionaryArray<Int8Type> = test.into_iter().collect();
1237
1238        let keys = array.keys();
1239        assert_eq!(&DataType::Int8, keys.data_type());
1240        assert_eq!(0, keys.null_count());
1241        assert_eq!(&[0, 1, 2, 0], keys.values());
1242    }
1243
1244    #[test]
1245    fn test_dictionary_keys_as_primitive_array_with_null() {
1246        let test = vec![Some("a"), None, Some("b"), None, None, Some("a")];
1247        let array: DictionaryArray<Int32Type> = test.into_iter().collect();
1248
1249        let keys = array.keys();
1250        assert_eq!(&DataType::Int32, keys.data_type());
1251        assert_eq!(3, keys.null_count());
1252
1253        assert!(keys.is_valid(0));
1254        assert!(!keys.is_valid(1));
1255        assert!(keys.is_valid(2));
1256        assert!(!keys.is_valid(3));
1257        assert!(!keys.is_valid(4));
1258        assert!(keys.is_valid(5));
1259
1260        assert_eq!(0, keys.value(0));
1261        assert_eq!(1, keys.value(2));
1262        assert_eq!(0, keys.value(5));
1263    }
1264
1265    #[test]
1266    fn test_dictionary_all_nulls() {
1267        let test = vec![None, None, None];
1268        let array: DictionaryArray<Int32Type> = test.into_iter().collect();
1269        array
1270            .into_data()
1271            .validate_full()
1272            .expect("All null array has valid array data");
1273    }
1274
1275    #[test]
1276    fn test_dictionary_iter() {
1277        // Construct a value array
1278        let values = Int8Array::from_iter_values([10_i8, 11, 12, 13, 14, 15, 16, 17]);
1279        let keys = Int16Array::from_iter_values([2_i16, 3, 4]);
1280
1281        // Construct a dictionary array from the above two
1282        let dict_array = DictionaryArray::new(keys, Arc::new(values));
1283
1284        let mut key_iter = dict_array.keys_iter();
1285        assert_eq!(2, key_iter.next().unwrap().unwrap());
1286        assert_eq!(3, key_iter.next().unwrap().unwrap());
1287        assert_eq!(4, key_iter.next().unwrap().unwrap());
1288        assert!(key_iter.next().is_none());
1289
1290        let mut iter = dict_array
1291            .values()
1292            .as_any()
1293            .downcast_ref::<Int8Array>()
1294            .unwrap()
1295            .take_iter(dict_array.keys_iter());
1296
1297        assert_eq!(12, iter.next().unwrap().unwrap());
1298        assert_eq!(13, iter.next().unwrap().unwrap());
1299        assert_eq!(14, iter.next().unwrap().unwrap());
1300        assert!(iter.next().is_none());
1301    }
1302
1303    #[test]
1304    fn test_dictionary_iter_with_null() {
1305        let test = vec![Some("a"), None, Some("b"), None, None, Some("a")];
1306        let array: DictionaryArray<Int32Type> = test.into_iter().collect();
1307
1308        let mut iter = array
1309            .values()
1310            .as_any()
1311            .downcast_ref::<StringArray>()
1312            .unwrap()
1313            .take_iter(array.keys_iter());
1314
1315        assert_eq!("a", iter.next().unwrap().unwrap());
1316        assert!(iter.next().unwrap().is_none());
1317        assert_eq!("b", iter.next().unwrap().unwrap());
1318        assert!(iter.next().unwrap().is_none());
1319        assert!(iter.next().unwrap().is_none());
1320        assert_eq!("a", iter.next().unwrap().unwrap());
1321        assert!(iter.next().is_none());
1322    }
1323
1324    #[test]
1325    fn test_dictionary_key() {
1326        let keys = Int8Array::from(vec![Some(2), None, Some(1)]);
1327        let values = StringArray::from(vec!["foo", "bar", "baz", "blarg"]);
1328
1329        let array = DictionaryArray::new(keys, Arc::new(values));
1330        assert_eq!(array.key(0), Some(2));
1331        assert_eq!(array.key(1), None);
1332        assert_eq!(array.key(2), Some(1));
1333    }
1334
1335    #[test]
1336    fn test_try_new() {
1337        let values: StringArray = [Some("foo"), Some("bar"), Some("baz")]
1338            .into_iter()
1339            .collect();
1340        let keys: Int32Array = [Some(0), Some(2), None, Some(1)].into_iter().collect();
1341
1342        let array = DictionaryArray::new(keys, Arc::new(values));
1343        assert_eq!(array.keys().data_type(), &DataType::Int32);
1344        assert_eq!(array.values().data_type(), &DataType::Utf8);
1345
1346        assert_eq!(array.null_count(), 1);
1347        assert_eq!(array.logical_null_count(), 1);
1348
1349        assert!(array.keys().is_valid(0));
1350        assert!(array.keys().is_valid(1));
1351        assert!(array.keys().is_null(2));
1352        assert!(array.keys().is_valid(3));
1353
1354        assert_eq!(array.keys().value(0), 0);
1355        assert_eq!(array.keys().value(1), 2);
1356        assert_eq!(array.keys().value(3), 1);
1357
1358        assert_eq!(
1359            "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",
1360            format!("{array:?}")
1361        );
1362    }
1363
1364    #[test]
1365    #[should_panic(expected = "Invalid dictionary key 3 at index 1, expected 0 <= key < 2")]
1366    fn test_try_new_index_too_large() {
1367        let values: StringArray = [Some("foo"), Some("bar")].into_iter().collect();
1368        // dictionary only has 2 values, so offset 3 is out of bounds
1369        let keys: Int32Array = [Some(0), Some(3)].into_iter().collect();
1370        DictionaryArray::new(keys, Arc::new(values));
1371    }
1372
1373    #[test]
1374    #[should_panic(expected = "Invalid dictionary key -100 at index 0, expected 0 <= key < 2")]
1375    fn test_try_new_index_too_small() {
1376        let values: StringArray = [Some("foo"), Some("bar")].into_iter().collect();
1377        let keys: Int32Array = [Some(-100)].into_iter().collect();
1378        DictionaryArray::new(keys, Arc::new(values));
1379    }
1380
1381    #[test]
1382    #[should_panic(expected = "DictionaryArray's data type must match, expected Int64 got Int32")]
1383    fn test_from_array_data_validation() {
1384        let a = DictionaryArray::<Int32Type>::from_iter(["32"]);
1385        let _ = DictionaryArray::<Int64Type>::from(a.into_data());
1386    }
1387
1388    #[test]
1389    fn test_into_primitive_dict_builder() {
1390        let values = Int32Array::from_iter_values([10_i32, 12, 15]);
1391        let keys = Int8Array::from_iter_values([1_i8, 0, 2, 0]);
1392
1393        let dict_array = DictionaryArray::new(keys, Arc::new(values));
1394
1395        let boxed: ArrayRef = Arc::new(dict_array);
1396        let col: DictionaryArray<Int8Type> = as_dictionary_array(&boxed).clone();
1397
1398        drop(boxed);
1399
1400        let mut builder = col.into_primitive_dict_builder::<Int32Type>().unwrap();
1401
1402        let slice = builder.values_slice_mut();
1403        assert_eq!(slice, &[10, 12, 15]);
1404
1405        slice[0] = 4;
1406        slice[1] = 2;
1407        slice[2] = 1;
1408
1409        let values = Int32Array::from_iter_values([4_i32, 2, 1]);
1410        let keys = Int8Array::from_iter_values([1_i8, 0, 2, 0]);
1411
1412        let expected = DictionaryArray::new(keys, Arc::new(values));
1413
1414        let new_array = builder.finish();
1415        assert_eq!(expected, new_array);
1416    }
1417
1418    #[test]
1419    fn test_into_primitive_dict_builder_cloned_array() {
1420        let values = Int32Array::from_iter_values([10_i32, 12, 15]);
1421        let keys = Int8Array::from_iter_values([1_i8, 0, 2, 0]);
1422
1423        let dict_array = DictionaryArray::new(keys, Arc::new(values));
1424
1425        let boxed: ArrayRef = Arc::new(dict_array);
1426
1427        let col: DictionaryArray<Int8Type> = DictionaryArray::<Int8Type>::from(boxed.to_data());
1428        let err = col.into_primitive_dict_builder::<Int32Type>();
1429
1430        let returned = err.unwrap_err();
1431
1432        let values = Int32Array::from_iter_values([10_i32, 12, 15]);
1433        let keys = Int8Array::from_iter_values([1_i8, 0, 2, 0]);
1434
1435        let expected = DictionaryArray::new(keys, Arc::new(values));
1436        assert_eq!(expected, returned);
1437    }
1438
1439    #[test]
1440    fn test_occupancy() {
1441        let keys = Int32Array::new((100..200).collect(), None);
1442        let values = Int32Array::from(vec![0; 1024]);
1443        let dict = DictionaryArray::new(keys, Arc::new(values));
1444        for (idx, v) in dict.occupancy().iter().enumerate() {
1445            let expected = (100..200).contains(&idx);
1446            assert_eq!(v, expected, "{idx}");
1447        }
1448
1449        let keys = Int32Array::new(
1450            (0..100).collect(),
1451            Some((0..100).map(|x| x % 4 == 0).collect()),
1452        );
1453        let values = Int32Array::from(vec![0; 1024]);
1454        let dict = DictionaryArray::new(keys, Arc::new(values));
1455        for (idx, v) in dict.occupancy().iter().enumerate() {
1456            let expected = idx % 4 == 0 && idx < 100;
1457            assert_eq!(v, expected, "{idx}");
1458        }
1459    }
1460
1461    #[test]
1462    fn test_iterator_nulls() {
1463        let keys = Int32Array::new(
1464            vec![0, 700, 1, 2].into(),
1465            Some(NullBuffer::from(vec![true, false, true, true])),
1466        );
1467        let values = Int32Array::from(vec![Some(50), None, Some(2)]);
1468        let dict = DictionaryArray::new(keys, Arc::new(values));
1469        let values: Vec<_> = dict
1470            .downcast_dict::<Int32Array>()
1471            .unwrap()
1472            .into_iter()
1473            .collect();
1474        assert_eq!(values, &[Some(50), None, None, Some(2)])
1475    }
1476
1477    #[test]
1478    fn test_logical_nulls() -> Result<(), ArrowError> {
1479        let values = Arc::new(RunArray::try_new(
1480            &Int32Array::from(vec![1, 3, 7]),
1481            &Int32Array::from(vec![Some(1), None, Some(3)]),
1482        )?) as ArrayRef;
1483
1484        // For this test to be meaningful, the values array need to have different nulls and logical nulls
1485        assert_eq!(values.null_count(), 0);
1486        assert_eq!(values.logical_null_count(), 2);
1487
1488        // Construct a trivial dictionary with 1-1 mapping to underlying array
1489        let dictionary = DictionaryArray::<Int8Type>::try_new(
1490            Int8Array::from((0..values.len()).map(|i| i as i8).collect::<Vec<_>>()),
1491            Arc::clone(&values),
1492        )?;
1493
1494        // No keys are null
1495        assert_eq!(dictionary.null_count(), 0);
1496        // Dictionary array values are logically nullable
1497        assert_eq!(dictionary.logical_null_count(), values.logical_null_count());
1498        assert_eq!(dictionary.logical_nulls(), values.logical_nulls());
1499        assert!(dictionary.is_nullable());
1500
1501        // Construct a trivial dictionary with 1-1 mapping to underlying array except that key 0 is nulled out
1502        let dictionary = DictionaryArray::<Int8Type>::try_new(
1503            Int8Array::from(
1504                (0..values.len())
1505                    .map(|i| i as i8)
1506                    .map(|i| if i == 0 { None } else { Some(i) })
1507                    .collect::<Vec<_>>(),
1508            ),
1509            Arc::clone(&values),
1510        )?;
1511
1512        // One key is null
1513        assert_eq!(dictionary.null_count(), 1);
1514
1515        // Dictionary array values are logically nullable
1516        assert_eq!(
1517            dictionary.logical_null_count(),
1518            values.logical_null_count() + 1
1519        );
1520        assert!(dictionary.is_nullable());
1521
1522        Ok(())
1523    }
1524
1525    #[test]
1526    fn test_normalized_keys() {
1527        let values = vec![132, 0, 1].into();
1528        let nulls = NullBuffer::from(vec![false, true, true]);
1529        let keys = Int32Array::new(values, Some(nulls));
1530        let dictionary = DictionaryArray::new(keys, Arc::new(Int32Array::new_null(2)));
1531        assert_eq!(&dictionary.normalized_keys(), &[1, 0, 1])
1532    }
1533}