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